You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by jerrypeng <gi...@git.apache.org> on 2016/01/28 00:15:05 UTC

[GitHub] storm pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

GitHub user jerrypeng opened a pull request:

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

    [STORM-1336] - Evalute/Port JStorm cgroup support and implement cgroup support for RAS

    

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

    $ git pull https://github.com/jerrypeng/storm STORM-1336

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

    https://github.com/apache/storm/pull/1053.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 #1053
    
----
commit d39c22c44c5a9b41d47c84fb1190d6f078b4a63a
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Date:   2016-01-25T20:48:42Z

    [STORM-1336] - Evalute/Port JStorm cgroup 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52278901
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return true;
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.isMounted(hierarchy)) {
    +            LOG.error("{} is already mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subSystems) {
    +            if (this.getHierarchyWithSubSystem(type) != null) {
    +                LOG.error("subSystem: {} is busy", type.name());
    +                subSystems.remove(type);
    +            }
    +        }
    +        if (subSystems.size() == 0) {
    +            return;
    +        }
    +        if (!Utils.CheckDirExists(hierarchy.getDir())) {
    +            new File(hierarchy.getDir()).mkdirs();
    +        }
    +        String subSystemsName = CgroupUtils.subSystemsToString(subSystems);
    +        SystemOperation.mount(subSystemsName, hierarchy.getDir(), "cgroup", subSystemsName);
    +
    +    }
    +
    +    @Override
    +    public void umount(Hierarchy hierarchy) throws IOException {
    +        if (this.isMounted(hierarchy)) {
    +            hierarchy.getRootCgroups().delete();
    +            SystemOperation.umount(hierarchy.getDir());
    +            CgroupUtils.deleteDir(hierarchy.getDir());
    +        } else {
    +            LOG.error("{} is not mounted", hierarchy.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void createCgroup(CgroupCommon cgroup) throws SecurityException {
    +        if (cgroup.isRoot()) {
    +            LOG.error("You can't create rootCgroup in this function");
    +            return;
    +        }
    +        CgroupCommon parent = cgroup.getParent();
    +        while (parent != null) {
    +            if (!Utils.CheckDirExists(parent.getDir())) {
    +                LOG.error("Parent {} does not exist", parent.getDir());
    +                return;
    +            }
    +            parent = parent.getParent();
    +        }
    +        Hierarchy h = cgroup.getHierarchy();
    +        if (!isMounted(h)) {
    +            LOG.error("hierarchy {} is not mounted", h.getDir());
    +            return;
    +        }
    +        if (Utils.CheckDirExists(cgroup.getDir())) {
    +            LOG.error("cgroup {} already exists", cgroup.getDir());
    +            return;
    +        }
    +
    +        //Todo perhaps thrown exception or print out error message is dir is not created successfully
    --- End diff --
    
    this todo can be removed. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51286731
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,225 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52420985
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +/**
    + * A class that implements system operations for using cgroups
    + */
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    }
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ")
    +                .append(type)
    +                .append(" -o ")
    +                .append(data)
    --- End diff --
    
    will rename


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52352681
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/DevicesCore.java ---
    @@ -0,0 +1,185 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.Device;
    +
    +import java.io.IOException;
    +import java.util.List;
    +
    +public class DevicesCore implements CgroupCore {
    +
    +    private final String dir;
    +
    +    public static final String DEVICES_ALLOW = "/devices.allow";
    +    public static final String DEVICES_DENY = "/devices.deny";
    +    public static final String DEVICES_LIST = "/devices.list";
    +
    +    public static final char TYPE_ALL = 'a';
    +    public static final char TYPE_BLOCK = 'b';
    +    public static final char TYPE_CHAR = 'c';
    +
    +    public static final int ACCESS_READ = 1;
    +    public static final int ACCESS_WRITE = 2;
    +    public static final int ACCESS_CREATE = 4;
    +
    +    public static final char ACCESS_READ_CH = 'r';
    +    public static final char ACCESS_WRITE_CH = 'w';
    +    public static final char ACCESS_CREATE_CH = 'm';
    --- End diff --
    
    Can we make these final values private? Since they are only used in this 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52350775
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52124710
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/Utils.java ---
    @@ -1367,12 +1367,67 @@ public static int toPositive(int number) {
             return number & Integer.MAX_VALUE;
         }
     
    -    public static RuntimeException wrapInRuntime(Exception e){
    -        if (e instanceof RuntimeException){
    -            return (RuntimeException)e;
    -        }else {
    +    public static RuntimeException wrapInRuntime(Exception e) {
    +        if (e instanceof RuntimeException) {
    +            return (RuntimeException) e;
    +        } else {
                 return new RuntimeException(e);
             }
         }
    +
    +    public static void ensure_process_killed(Integer pid) {
    +        // in this function, just kill the process 5 times
    +        // make sure the process be killed definitely
    +        for (int i = 0; i < 5; i++) {
    +            try {
    +                exec_command("kill -9 " + pid);
    +                LOG.info("kill -9 process " + pid);
    +                sleepMs(100);
    +            } catch (ExecuteException e) {
    +                LOG.info("Error when trying to kill " + pid + ". Process has been killed");
    +                return;
    +            } catch (Exception e) {
    +                LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +            }
    +        }
    +    }
    +
    +    public static void process_killed(Integer pid) {
    +        try {
    +            exec_command("kill " + pid);
    +            LOG.info("kill process " + pid);
    +        } catch (ExecuteException e) {
    +            LOG.info("Error when trying to kill " + pid + ". Process has been killed. ");
    +        } catch (Exception e) {
    +            LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +        }
    +    }
    +
    +    public static void kill(Integer pid) {
    +        process_killed(pid);
    +
    +        sleepMs(1000);
    +
    +        ensure_process_killed(pid);
    +    }
    +
    +    public static void exec_command(String command) throws ExecuteException, IOException {
    +        String[] cmdlist = command.split(" ");
    +        CommandLine cmd = new CommandLine(cmdlist[0]);
    +        for (int i = 1; i < cmdlist.length; i++) {
    +            cmd.addArgument(cmdlist[i]);
    +        }
    +
    +        DefaultExecutor exec = new DefaultExecutor();
    +        exec.execute(cmd);
    +    }
    +
    +    public static void sleepMs(long ms) {
    --- End diff --
    
    get rid of


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52421707
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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.container.cgroup;
    +
    +import com.google.common.io.Files;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.nio.charset.Charset;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    /**
    +     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
    +     */
    +    public static Set<SubSystemType> getSubSystemsFromString(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            //return null to mount options in string that is not part of cgroups
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    /**
    +     * Get a string that is a comma delimited list of subsystems
    +     */
    +    public static String subSystemsToString(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return Utils.checkFileExists(CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String filePath) throws IOException {
    +        return Files.readLines(new File(filePath), Charset.defaultCharset());
    +    }
    +
    +    public static void writeFileByLine(String filePath, List<String> linesToWrite) throws IOException {
    +        LOG.debug("For CGroups - writing {} to {} ", linesToWrite, filePath);
    +        File file = new File(filePath);
    +        if (!file.exists()) {
    +            LOG.error("{} does not exist", filePath);
    +            return;
    +        }
    +        try (FileWriter writer = new FileWriter(file, true);
    --- End diff --
    
    doesn't make a different for the cgroup config files


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52074621
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java ---
    @@ -0,0 +1,58 @@
    +/**
    + * 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.container.cgroup;
    +
    +public enum SubSystemType {
    +
    +    // net_cls,ns is not supposted in ubuntu
    --- End diff --
    
    Do we need this 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#issuecomment-183452870
  
    One comment about the new tolerance config name, other than that it looks 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52354392
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/Device.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.container.cgroup;
    +
    +/**
    + * a class that represents a device in linux
    + */
    +public class Device {
    +
    +    public final int major;
    +    public final int minor;
    +
    +    public Device(int major, int minor) {
    +        this.major = major;
    +        this.minor = minor;
    +    }
    +
    +    public Device(String str) {
    +        String[] strArgs = str.split(":");
    +        this.major = Integer.valueOf(strArgs[0]);
    +        this.minor = Integer.valueOf(strArgs[1]);
    +    }
    +
    +    @Override
    +    public String toString() {
    --- End diff --
    
    we are not using lombok annotations in storm currently. Perhaps we can int the future, but perhaps @abhishekagarwal87 can file a jira to investigate whether we should use 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52350475
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!getCores().keySet().contains(SubSystemType.cpuset)) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getCgroupCloneChildren() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(eventFd);
    +        sb.append(' ');
    +        sb.append(controlFd);
    +        for (String arg : args) {
    +            sb.append(' ');
    +            sb.append(arg);
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_EVENT_CONTROL), sb.toString());
    +    }
    +
    +    public Hierarchy getHierarchy() {
    +        return hierarchy;
    +    }
    +
    +    public String getName() {
    +        return name;
    +    }
    +
    +    public String getDir() {
    +        return dir;
    +    }
    +
    +    public CgroupCommon getParent() {
    +        return parent;
    +    }
    +
    +    public Set<CgroupCommon> getChildren() {
    +
    +        File file = new File(this.dir);
    +        File[] files = file.listFiles();
    +        if (files == null) {
    +            return null;
    +        }
    +        Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +        for (File child : files) {
    +            if (child.isDirectory()) {
    +                children.add(new CgroupCommon(child.getName(), this.hierarchy, this));
    +            }
    +        }
    +        return children;
    +    }
    +
    +    public boolean isRoot() {
    +        return isRoot;
    +    }
    +
    +    public Map<SubSystemType, CgroupCore> getCores() {
    +        return CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
    +    }
    +
    +    public void delete() throws IOException {
    +        this.free();
    +        if (!this.isRoot) {
    +            this.parent.getChildren().remove(this);
    +        }
    +    }
    +
    +    private void free() throws IOException {
    +        for (CgroupCommon child : getChildren()) {
    +            child.free();
    +        }
    +        if (this.isRoot) {
    +            return;
    +        }
    +        Set<Integer> tasks = this.getTasks();
    +        if (tasks != null) {
    +            for (Integer task : tasks) {
    +                this.parent.addTask(task);
    +            }
    +        }
    +        CgroupUtils.deleteDir(this.dir);
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        boolean ret = false;
    +        if (o != null && (o instanceof CgroupCommon)) {
    +            ret = (((CgroupCommon)o).hierarchy.equals(this.hierarchy)
    +                    && ((CgroupCommon)o).name.equals(this.name))
    +                    && ((CgroupCommon)o).dir.equals(this.dir);
    +
    +        }
    +        return ret;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        final int prime = 31;
    +        int result = 1;
    +        result = prime * result + (this.name != null ? this.name.hashCode() : 0);
    --- End diff --
    
    will correct


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52342665
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java ---
    @@ -0,0 +1,80 @@
    +/**
    + * 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.container.cgroup;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * An interface to manage cgroups
    + */
    +public interface CgroupOperation {
    +
    +    /**
    +     * Get a list of hierarchies
    +     */
    +    public List<Hierarchy> getHierarchies();
    +
    +    /**
    +     * get a list of available subsystems
    +     */
    +    public Set<SubSystem> getSubSystems();
    +
    +    /**
    +     * Check if a subsystem is enabled
    +     */
    +    public boolean isSubSystemEnabled(SubSystemType subsystem);
    +
    +    /**
    +     * get the first hierarchy that has a certain subsystem isMounted
    +     */
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subsystem);
    +
    +    /**
    +     * get the first hierarchy that has a certain list of subsystems isMounted
    +     */
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems);
    +
    --- End diff --
    
    extra 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52082924
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    +        File file = new File(dir);
    +        return file.exists();
    +    }
    +
    +    public static boolean dirExists(String dir) {
    +        File file = new File(dir);
    +        return file.isDirectory();
    +    }
    +
    +    public static Set<SubSystemType> analyse(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static String reAnalyse(Set<SubSystemType> subSystems) {
    --- End diff --
    
    will rename and add 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52098065
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    };
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ").append(type).append(" -o ").append(data).append(" ").append(name).append(" ").append(target);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static void umount(String name) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("umount ").append(name);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static String exec(String cmd) throws IOException {
    +        LOG.debug("Shell cmd: {}", cmd);
    +        Process process = new ProcessBuilder(new String[] { "/bin/bash", "-c", cmd }).start();
    +        try {
    +            process.waitFor();
    +            String output = IOUtils.toString(process.getInputStream());
    +            String errorOutput = IOUtils.toString(process.getErrorStream());
    +            LOG.debug("Shell Output: {}", output);
    +            if (errorOutput.length() != 0) {
    +                LOG.error("Shell Error Output: {}", errorOutput);
    +                throw new IOException(errorOutput);
    +            }
    +            return output;
    +        } catch (InterruptedException ie) {
    +            throw new IOException(ie.toString());
    +        }
    +    }
    +}
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52416971
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/CpusetCore.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +
    +public class CpusetCore implements CgroupCore {
    +
    +    public static final String CPUSET_CPUS = "/cpuset.cpus";
    +    public static final String CPUSET_MEMS = "/cpuset.mems";
    +    public static final String CPUSET_MEMORY_MIGRATE = "/cpuset.memory_migrate";
    +    public static final String CPUSET_CPU_EXCLUSIVE = "/cpuset.cpu_exclusive";
    +    public static final String CPUSET_MEM_EXCLUSIVE = "/cpuset.mem_exclusive";
    +    public static final String CPUSET_MEM_HARDWALL = "/cpuset.mem_hardwall";
    +    public static final String CPUSET_MEMORY_PRESSURE = "/cpuset.memory_pressure";
    +    public static final String CPUSET_MEMORY_PRESSURE_ENABLED = "/cpuset.memory_pressure_enabled";
    +    public static final String CPUSET_MEMORY_SPREAD_PAGE = "/cpuset.memory_spread_page";
    +    public static final String CPUSET_MEMORY_SPREAD_SLAB = "/cpuset.memory_spread_slab";
    +    public static final String CPUSET_SCHED_LOAD_BALANCE = "/cpuset.sched_load_balance";
    +    public static final String CPUSET_SCHED_RELAX_DOMAIN_LEVEL = "/cpuset.sched_relax_domain_level";
    +
    +    private final String dir;
    +
    +    public CpusetCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.cpuset;
    +    }
    +
    +    public void setCpus(int[] nums) throws IOException {
    +        setConfigs(nums, CPUSET_CPUS);
    +    }
    +
    +    public int[] getCpus() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPUS)).get(0);
    +        return parseNums(output);
    +    }
    +
    +    public void setMems(int[] nums) throws IOException {
    +        setConfigs(nums, CPUSET_MEMS);
    +    }
    +
    +    private void setConfigs(int[] nums, String config) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        for (int num : nums) {
    +            sb.append(num);
    +            sb.append(',');
    +        }
    +        sb.deleteCharAt(sb.length() - 1);
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, config), sb.toString());
    +    }
    +
    +    public int[] getMems() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMS)).get(0);
    +        return parseNums(output);
    +    }
    +
    +    public void setMemMigrate(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_MIGRATE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemMigrate() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_MIGRATE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setCpuExclusive(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPU_EXCLUSIVE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isCpuExclusive() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPU_EXCLUSIVE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemExclusive(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_EXCLUSIVE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemExclusive() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_EXCLUSIVE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemHardwall(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_HARDWALL), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemHardwall() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_HARDWALL)).get(0));
    +        return output > 0;
    +    }
    +
    +    public int getMemPressure() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE)).get(0);
    +        return Integer.parseInt(output);
    +    }
    +
    +    public void setMemPressureEnabled(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemPressureEnabled() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemSpreadPage(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemSpreadPage() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemSpreadSlab(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemSpreadSlab() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setSchedLoadBlance(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isSchedLoadBlance() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setSchedRelaxDomainLevel(int value) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL), String.valueOf(value));
    +    }
    +
    +    public int getSchedRelaxDomainLevel() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL)).get(0);
    +        return Integer.parseInt(output);
    +    }
    +
    +    public static int[] parseNums(String outputStr) {
    +        char[] output = outputStr.toCharArray();
    +        LinkedList<Integer> numList = new LinkedList<Integer>();
    +        int value = 0;
    +        int start = 0;
    +        boolean isHyphen = false;
    +        for (char ch : output) {
    --- End diff --
    
    can this be simplified to 
    ```
    for (str in outputStr.split(",")) 
    if (!str.contains("-"))
    numList.add(Integer.valueOf(str))
    else {
    Integer start = Integer.valueOf(str.split("-")[0])
    Integer end = Integer.valueOf(str.split("-")[1])
    // Add from start to end to numList
    }
    ```
    It is based on my understanding of list format. Hope it covers the 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52415538
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +/**
    + * A class that implements system operations for using cgroups
    + */
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    }
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ")
    +                .append(type)
    +                .append(" -o ")
    +                .append(data)
    +                .append(" ")
    +                .append(name)
    +                .append(" ")
    +                .append(target);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static void umount(String name) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("umount ").append(name);
    --- End diff --
    
    variable should be named dir.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52072773
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52044357
  
    --- Diff: conf/cgconfig.conf.example ---
    @@ -0,0 +1,41 @@
    +# 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.
    +
    +mount {
    +	cpuset	= /cgroup/cpuset;
    +	cpu	= /cgroup/storm_resources;
    +	cpuacct	= /cgroup/cpuacct;
    +	memory	= /cgroup/storm_resources;
    +	devices	= /cgroup/devices;
    +	freezer	= /cgroup/freezer;
    +	net_cls	= /cgroup/net_cls;
    +	blkio	= /cgroup/blkio;
    +}
    +
    +group storm {
    +       perm {
    +               task {
    +                      uid = 500;
    +                      gid = 500;
    +               }
    +               admin {
    +                      uid = 500;
    +                      gid = 500;
    +               }
    +       }
    +       cpu {
    +       }
    +}
    --- End diff --
    
    Newline.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51286347
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    +        this.center.create(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    --- End diff --
    
    Would it be worth filing a follow on JIRA so that we can control if it is a hard CPU limit or a CPU guarantee?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52367487
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,218 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (CgroupUtils.enabled()) {
    +            instance = new CgroupCenter();
    +            return instance;
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                //Some mount options (i.e. rw and relatime) in type are not cgroups related
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                int hierarchyID = Integer.valueOf(split[1]);
    +                int cgroupNum = Integer.valueOf(split[2]);
    +                boolean enable =  Integer.valueOf(split[3]).intValue() == 1 ? true : false;
    +                subSystems.add(new SubSystem(type, hierarchyID, cgroupNum, enable));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    +            List<Hierarchy> hierarchies = this.getHierarchies();
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return true;
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +        if (this.isMounted(hierarchy)) {
    +            LOG.error("{} is already mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subSystems) {
    +            if (this.getHierarchyWithSubSystem(type) != null) {
    +                LOG.error("subSystem: {} is busy", type.name());
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52285713
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    --- End diff --
    
    should the name be same as dir? 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52046993
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    --- End diff --
    
    Why bother assigning 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52074497
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java ---
    @@ -0,0 +1,58 @@
    +/**
    + * 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.container.cgroup;
    +
    +public enum SubSystemType {
    +
    +    // net_cls,ns is not supposted in ubuntu
    +    blkio, cpu, cpuacct, cpuset, devices, freezer, memory, perf_event, net_cls, net_prio;
    +
    +    public static SubSystemType getSubSystem(String str) {
    +        if (str.equals("blkio")) {
    +            return blkio;
    +        }
    +        else if (str.equals("cpu")) {
    +            return cpu;
    +        }
    +        else if (str.equals("cpuacct")) {
    +            return cpuacct;
    +        }
    +        else if (str.equals("cpuset")) {
    +            return cpuset;
    +        }
    +        else if (str.equals("devices")) {
    +            return devices;
    +        }
    +        else if (str.equals("freezer")) {
    +            return freezer;
    +        }
    +        else if (str.equals("memory")) {
    +            return memory;
    +        }
    +        else if (str.equals("perf_event")) {
    +            return perf_event;
    +        }
    +        else if (str.equals("net_cls")) {
    +            return net_cls;
    +        }
    +        else if (str.equals("net_prio")) {
    +            return net_prio;
    +        }
    --- End diff --
    
    Can we rewrite this as a `switch` ?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52288285
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * initalize subsystems
    +     */
    +    private void prepareSubSystem(Map conf) throws IOException {
    +        List<SubSystemType> subSystemTypes = new LinkedList<>();
    +        for (String resource : Config.getCgroupStormResources(conf)) {
    +            subSystemTypes.add(SubSystemType.getSubSystem(resource));
    +        }
    +
    +        this.hierarchy = center.getHierarchyWithSubSystems(subSystemTypes);
    +
    +        if (this.hierarchy == null) {
    +            Set<SubSystemType> types = new HashSet<SubSystemType>();
    +            types.add(SubSystemType.cpu);
    +            this.hierarchy = new Hierarchy(Config.getCgroupStormHierarchyName(conf), types, Config.getCgroupStormHierarchyDir(conf));
    +        }
    +        this.rootCgroup = new CgroupCommon(this.rootDir, this.hierarchy, this.hierarchy.getRootCgroups());
    +
    +        // set upper limit to how much cpu can be used by all workers running on supervisor node.
    +        // This is done so that some cpu cycles will remain free to run the daemons and other miscellaneous OS operations.
    +        CpuCore supervisorRootCPU = (CpuCore)  this.rootCgroup.getCores().get(SubSystemType.cpu);
    +        setCpuUsageUpperLimit(supervisorRootCPU, ((Number) this.conf.get(Config.SUPERVISOR_CPU_CAPACITY)).intValue());
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g.
    +     * If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public void reserveResourcesForWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = null;
    +        // The manually set STORM_WORKER_CGROUP_CPU_LIMIT config on supervisor will overwrite resources assigned by RAS (Resource Aware Scheduler)
    +        if (this.conf.get(Config.STORM_WORKER_CGROUP_CPU_LIMIT) != null) {
    +            cpuNum = (Number) this.conf.get(Config.STORM_WORKER_CGROUP_CPU_LIMIT);
    +        } else if(resourcesMap.get("cpu") != null) {
    +            cpuNum = (Number) resourcesMap.get("cpu");
    +        }
    +
    +        Number totalMem = null;
    +        // The manually set STORM_WORKER_CGROUP_MEMORY_MB_LIMIT config on supervisor will overwrite resources assigned by RAS (Resource Aware Scheduler)
    +        if (this.conf.get(Config.STORM_WORKER_CGROUP_MEMORY_MB_LIMIT) != null) {
    +            totalMem = (Number) this.conf.get(Config.STORM_WORKER_CGROUP_MEMORY_MB_LIMIT);
    +        } else if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, this.hierarchy, this.rootCgroup);
    +        this.center.createCgroup(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    +            } catch (IOException e) {
    +                throw new RuntimeException("Cannot set cpu.shares! Exception: " + e);
    +            }
    +        }
    +
    +        if (totalMem != null) {
    +            MemoryCore memCore = (MemoryCore) workerGroup.getCores().get(SubSystemType.memory);
    +            try {
    +                memCore.setPhysicalUsageLimit(Long.valueOf(totalMem.longValue() * 1024 * 1024));
    +            } catch (IOException e) {
    +                throw new RuntimeException("Cannot set memory.limit_in_bytes! Exception: " + e);
    --- End diff --
    
    use, e instead of + e


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51379125
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    +        this.center.create(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    --- End diff --
    
    ok, I agree we should test this feature to make sure idle workers can re-acquire the resources they are guaranteed once they "wake 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51135122
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null)
    +            instance = new CgroupCenter();
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup"))
    +                    continue;
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null)
    +                    continue;
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]), Integer.valueOf(split[3]).intValue() == 1 ? true
    +                        : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error ", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem)
    --- End diff --
    
    Can you please update all of the blocks to include `{` and `}` even for single line blocks.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52286090
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    --- End diff --
    
    can you put a check that number of returned rows by readFileByLine is more than zero.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51133971
  
    --- Diff: storm-core/src/jvm/org/apache/storm/Config.java ---
    @@ -2359,4 +2417,24 @@ public void setTopologyStrategy(Class<? extends IStrategy> clazz) {
                 this.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, clazz.getName());
             }
         }
    +
    +    public static String getCgroupRootDir(Map conf) {
    --- End diff --
    
    Should we capitalize the G like in the other 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52327592
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    --- End diff --
    
    Not necessary please see my previous comment on this subject manner.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51917226
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    --- End diff --
    
    renamed


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52782841
  
    --- Diff: storm-core/src/jvm/org/apache/storm/Config.java ---
    @@ -2194,6 +2195,73 @@
         @isString
         public static final Object CLIENT_JAR_TRANSFORMER = "client.jartransformer.class";
     
    +
    +    /**
    +     * The plugin to be used for resource isolation
    +     */
    +    @isImplementationOfClass(implementsClass = ResourceIsolationInterface.class)
    +    public static final Object STORM_RESOURCE_ISOLATION_PLUGIN = "storm.resource.isolation.plugin";
    +
    +    /**
    +     * CGroup Setting below
    +     */
    +
    +    /**
    +     * root directory of the storm cgroup hierarchy
    +     */
    +    @isString
    +    public static final Object STORM_CGROUP_HIERARCHY_DIR = "storm.cgroup.hierarchy.dir";
    +
    +    /**
    +     * resources to to be controlled by cgroups
    +     */
    +    @isStringList
    +    public static final Object STORM_CGROUP_RESOURCES = "storm.cgroup.resources";
    +
    +    /**
    +     * name for the cgroup hierarchy
    +     */
    +    @isString
    +    public static final Object STORM_CGROUP_HIERARCHY_NAME = "storm.cgroup.hierarchy.name";
    +
    +    /**
    +     * flag to determine whether to use a resource isolation plugin
    +     * Also determines whether the unit tests for cgroup runs.
    +     * If storm.resource.isolation.plugin.enable is set to false the unit tests for cgroups will not run
    +     */
    +    @isBoolean
    +    public static final String STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE = "storm.resource.isolation.plugin.enable";
    +
    +    /**
    +     * root directory for cgoups
    +     */
    +    @isString
    +    public static String STORM_SUPERVISOR_CGROUP_ROOTDIR = "storm.supervisor.cgroup.rootdir";
    +
    +    /**
    +     * the manually set memory limit (in MB) for each CGroup on supervisor node
    +     */
    +    @isPositiveNumber
    +    public static String STORM_WORKER_CGROUP_MEMORY_MB_LIMIT = "storm.worker.cgroup.memory.mb.limit";
    +
    +    /**
    +     * the manually set cpu share for each CGroup on supervisor node
    +     */
    +    @isPositiveNumber
    +    public static String STORM_WORKER_CGROUP_CPU_LIMIT = "storm.worker.cgroup.cpu.limit";
    +
    +    /**
    +     * full path to cgexec command
    +     */
    +    @isString
    +    public static String STORM_CGROUP_CGEXEC_CMD = "storm.cgroup.cgexec.cmd";
    +
    +    /**
    +     * The amount of memory a worker can exceed its allocation before cgroup will kill it
    +     */
    +    @isPositiveNumber
    +    public static String STORM_CGROUP_MEMORY_MB_LIMIT_TOLERANCE_MARGIN = "storm.cgroup.memory.limit.tolerance.margin.mb";
    --- End diff --
    
    We should make the java variable match the config name string like the others.
    `STORM_CGROUP_MEMORY_LIMIT_TOLERANCE_MARGIN_MB`.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52072165
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1178,7 +1212,7 @@
               (FileUtils/copyDirectory (File. (.getFile url)) (File. target-dir)))))))
     
     (defmethod launch-worker
    -    :local [supervisor storm-id port worker-id mem-onheap]
    +    :local [supervisor storm-id port worker-id resources]
    --- End diff --
    
    revert?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52332678
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52097932
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52416222
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/CpuacctCore.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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class CpuacctCore implements CgroupCore {
    +
    +    public static final String CPUACCT_USAGE = "/cpuacct.usage";
    +    public static final String CPUACCT_STAT = "/cpuacct.stat";
    +    public static final String CPUACCT_USAGE_PERCPU = "/cpuacct.usage_percpu";
    +
    +    private final String dir;
    +
    +    public CpuacctCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.cpuacct;
    +    }
    +
    +    public Long getCpuUsage() throws IOException {
    +        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_USAGE)).get(0));
    +    }
    +
    +    public Map<StatType, Long> getCpuStat() throws IOException {
    +        List<String> strs = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_STAT));
    +        Map<StatType, Long> result = new HashMap<StatType, Long>();
    +        result.put(StatType.user, Long.parseLong(strs.get(0).split(" ")[1]));
    +        result.put(StatType.system, Long.parseLong(strs.get(1).split(" ")[1]));
    +        return result;
    +    }
    +
    +    public Long[] getPerCpuUsage() throws IOException {
    +        String str = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_USAGE_PERCPU)).get(0);
    +        String[] strArgs = str.split(" ");
    +        Long[] result = new Long[strArgs.length];
    +        for (int i = 0; i < result.length; i++) {
    +            result[i] = Long.parseLong(strArgs[i]);
    +        }
    +        return result;
    +    }
    +
    +    public enum StatType {
    --- End diff --
    
    It is better to declare inner class,enums static.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52079277
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/Utils.java ---
    @@ -1367,12 +1367,67 @@ public static int toPositive(int number) {
             return number & Integer.MAX_VALUE;
         }
     
    -    public static RuntimeException wrapInRuntime(Exception e){
    -        if (e instanceof RuntimeException){
    -            return (RuntimeException)e;
    -        }else {
    +    public static RuntimeException wrapInRuntime(Exception e) {
    +        if (e instanceof RuntimeException) {
    +            return (RuntimeException) e;
    +        } else {
                 return new RuntimeException(e);
             }
         }
    +
    +    public static void ensure_process_killed(Integer pid) {
    +        // in this function, just kill the process 5 times
    +        // make sure the process be killed definitely
    +        for (int i = 0; i < 5; i++) {
    +            try {
    +                exec_command("kill -9 " + pid);
    +                LOG.info("kill -9 process " + pid);
    +                sleepMs(100);
    +            } catch (ExecuteException e) {
    +                LOG.info("Error when trying to kill " + pid + ". Process has been killed");
    +                return;
    +            } catch (Exception e) {
    +                LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +            }
    +        }
    +    }
    +
    +    public static void process_killed(Integer pid) {
    +        try {
    +            exec_command("kill " + pid);
    +            LOG.info("kill process " + pid);
    +        } catch (ExecuteException e) {
    +            LOG.info("Error when trying to kill " + pid + ". Process has been killed. ");
    +        } catch (Exception e) {
    +            LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +        }
    +    }
    +
    +    public static void kill(Integer pid) {
    --- End diff --
    
    We already have process killing stuff in Util.java.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52070908
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1062,9 +1072,15 @@
                             (add-to-classpath [stormjar])
                             (add-to-classpath topo-classpath))
               top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
    -          mem-onheap (if (and mem-onheap (> mem-onheap 0)) ;; not nil and not zero
    -                       (int (Math/ceil mem-onheap)) ;; round up
    +          mem-onheap (if (and (.get_mem_on_heap resources) (> (.get_mem_on_heap resources) 0)) ;; not nil and not zero
    +                       (int (Math/ceil (.get_mem_on_heap resources))) ;; round up
                            (storm-conf WORKER-HEAP-MEMORY-MB)) ;; otherwise use default value
    +          mem-offheap (if (.get_mem_off_heap resources)
    --- End diff --
    
    Non-negative check for off-heap?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52098096
  
    --- Diff: storm-core/test/clj/org/apache/storm/supervisor_test.clj ---
    @@ -362,8 +361,8 @@
                                           mock-storm-id
                                           mock-port
                                           mock-worker-id
    -                                      mock-mem-onheap)
    -                (verify-first-call-args-for-indices launch-process
    +                                      (WorkerResources.))
    +            (verify-first-call-args-for-indices launch-process
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52111616
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/ResourceIsolationInterface.java ---
    @@ -0,0 +1,43 @@
    +/**
    + * 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.container;
    +
    +import java.util.Map;
    +
    +/**
    + * A plugin to support resource isolation and limitation within Storm
    + */
    +public interface ResourceIsolationInterface {
    +
    +    /**
    +     * @param workerId worker id of the worker to start
    +     * @param resources set of resources to limit
    +     * @return a String that includes to command on how to start the worker.  The string returned from this function
    +     * will be concatenated to the front of the command to launch logwriter/worker in supervisor.clj
    +     */
    +    public String startNewWorker(String workerId, Map resources);
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52680658
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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.container.cgroup;
    +
    +import com.google.common.io.Files;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.nio.charset.Charset;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52045815
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -344,6 +348,12 @@
        :sync-retry (atom 0)
        :download-lock (Object.)
        :stormid->profiler-actions (atom {})
    +   :cgroup-manager (if (conf STORM-CGROUP-ENABLE)
    --- End diff --
    
    Also, why do we only load the isolation plugin if cgroups is enabled? Maybe this config can go away and we can just check if `storm.resource.isolation.plugin` is set?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52649491
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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.container.cgroup;
    +
    +import com.google.common.io.Files;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.nio.charset.Charset;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    --- End diff --
    
    Can we un-nest this so it's easier to follow? Something like: 
    ```
    if (!d.exists()) {
        Log.warn("dir {} does not exist!", dir);
        return;
    }
    if (!d.isDirectory()) {
        throw new RuntimeException("dir " + dir + " is not a directory!");
    }
    if (!d.delete()) {
        throw new RuntimeException("Cannot delete dir " + dir);
    }
    ```



---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52286577
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    --- End diff --
    
    what is the expected behavior if this is called again? Currently it doesn't override the existing flag but appends 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52277252
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    --- End diff --
    
    should this be the first line 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52278297
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    --- End diff --
    
    This condition should be before this.getHierarchies. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52073695
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    --- End diff --
    
    ` ? true : false` can be deleted


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52335429
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52415458
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +/**
    + * A class that implements system operations for using cgroups
    + */
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    }
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ")
    +                .append(type)
    +                .append(" -o ")
    +                .append(data)
    +                .append(" ")
    +                .append(name)
    +                .append(" ")
    +                .append(target);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static void umount(String name) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("umount ").append(name);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static String exec(String cmd) throws IOException {
    +        LOG.debug("Shell cmd: {}", cmd);
    +        Process process = new ProcessBuilder(new String[] { "/bin/bash", "-c", cmd }).start();
    +        try {
    +            process.waitFor();
    +            String output = IOUtils.toString(process.getInputStream());
    +            String errorOutput = IOUtils.toString(process.getErrorStream());
    +            LOG.debug("Shell Output: {}", output);
    +            if (errorOutput.length() != 0) {
    +                LOG.error("Shell Error Output: {}", errorOutput);
    +                throw new IOException(errorOutput);
    +            }
    +            return output;
    +        } catch (InterruptedException ie) {
    +            throw new IOException(ie.toString());
    --- End diff --
    
    ie can be wrapped instead of ie.toString()


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52098040
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SubSystem.java ---
    @@ -0,0 +1,78 @@
    +/**
    + * 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.container.cgroup;
    +
    +public class SubSystem {
    +
    +    private SubSystemType type;
    +
    +    private int hierarchyID;
    +
    +    private int cgroupsNum;
    +
    +    private boolean enable;
    +
    +    public SubSystem(SubSystemType type, int hierarchyID, int cgroupNum, boolean enable) {
    +        this.type = type;
    +        this.hierarchyID = hierarchyID;
    +        this.cgroupsNum = cgroupNum;
    +        this.enable = enable;
    +    }
    +
    +    public SubSystemType getType() {
    +        return type;
    +    }
    +
    +    public void setType(SubSystemType type) {
    +        this.type = type;
    +    }
    +
    +    public int getHierarchyID() {
    +        return hierarchyID;
    +    }
    +
    +    public void setHierarchyID(int hierarchyID) {
    +        this.hierarchyID = hierarchyID;
    +    }
    +
    +    public int getCgroupsNum() {
    +        return cgroupsNum;
    +    }
    +
    +    public void setCgroupsNum(int cgroupsNum) {
    +        this.cgroupsNum = cgroupsNum;
    +    }
    +
    +    public boolean isEnable() {
    +        return enable;
    +    }
    +
    +    public void setEnable(boolean enable) {
    +        this.enable = enable;
    +    }
    +
    +    @Override
    +    public boolean equals(Object object) {
    +        boolean ret = false;
    +        if (object != null && object instanceof SubSystem) {
    +            ret = (this.type.equals(((SubSystem)object).getType()) && this.hierarchyID == ((SubSystem)object).getHierarchyID());
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52367053
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return true;
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.isMounted(hierarchy)) {
    +            LOG.error("{} is already mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subSystems) {
    +            if (this.getHierarchyWithSubSystem(type) != null) {
    +                LOG.error("subSystem: {} is busy", type.name());
    +                subSystems.remove(type);
    --- End diff --
    
    Understood 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52325588
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    --- End diff --
    
    I don't think its necessary.  The format for the cgroup files are set and the whole statement is in a try statement to catch any exception.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52417674
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,119 @@
    +/**
    + * 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.container.cgroup;
    +
    +import com.google.common.io.Files;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.nio.charset.Charset;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    /**
    +     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
    +     */
    +    public static Set<SubSystemType> getSubSystemsFromString(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            //return null to mount options in string that is not part of cgroups
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    /**
    +     * Get a string that is a comma delimited list of subsystems
    +     */
    +    public static String subSystemsToString(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return Utils.checkFileExists(CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String filePath) throws IOException {
    +        return Files.readLines(new File(filePath), Charset.defaultCharset());
    +    }
    +
    +    public static void writeFileByLine(String filePath, List<String> linesToWrite) throws IOException {
    +        LOG.debug("For CGroups - writing {} to {} ", linesToWrite, filePath);
    +        File file = new File(filePath);
    +        if (!file.exists()) {
    +            LOG.error("{} does not exist", filePath);
    +            return;
    +        }
    +        try (FileWriter writer = new FileWriter(file, true);
    --- End diff --
    
    append is set to true here. should it be set to false


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52084693
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java ---
    @@ -0,0 +1,46 @@
    +/**
    + * 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.container.cgroup;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Set;
    +
    +public interface CgroupOperation {
    --- End diff --
    
    will refactor and add 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52053109
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/Constants.java ---
    @@ -0,0 +1,30 @@
    +/**
    + * 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.container.cgroup;
    +
    +public class Constants {
    --- End diff --
    
    I don't think we probably want a class called `Constants`.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by jerrypeng <gi...@git.apache.org>.
Github user jerrypeng commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-178072782
  
    @revans2 @bastiliu @vesense Thanks for the reviews! I really appreciated it!  I think I have addressed all of your comments.  Does anyone have any other concerns?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52072956
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52277813
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    --- End diff --
    
    Instead of using split[1], split[2] directly, it will be good to have declared variables e.g. name = split[1]. Similar to what you have done in getHierarchies. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#issuecomment-184966911
  
    Travis CI [error](https://travis-ci.org/apache/storm/jobs/109239992#L1570) does not seem related. Created [STORM-1555](https://issues.apache.org/jira/browse/STORM-1555)


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by vesense <gi...@git.apache.org>.
Github user vesense commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-185514609
  
    LGTM


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52124528
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java ---
    @@ -0,0 +1,58 @@
    +/**
    + * 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.container.cgroup;
    +
    +public enum SubSystemType {
    +
    +    // net_cls,ns is not supposted in ubuntu
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52333105
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    --- End diff --
    
    for comparing enums .equals or "==" is 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r53109116
  
    --- Diff: storm-core/src/jvm/org/apache/storm/Config.java ---
    @@ -2194,6 +2195,73 @@
         @isString
         public static final Object CLIENT_JAR_TRANSFORMER = "client.jartransformer.class";
     
    +
    +    /**
    +     * The plugin to be used for resource isolation
    +     */
    +    @isImplementationOfClass(implementsClass = ResourceIsolationInterface.class)
    +    public static final Object STORM_RESOURCE_ISOLATION_PLUGIN = "storm.resource.isolation.plugin";
    +
    +    /**
    +     * CGroup Setting below
    +     */
    +
    +    /**
    +     * root directory of the storm cgroup hierarchy
    +     */
    +    @isString
    +    public static final Object STORM_CGROUP_HIERARCHY_DIR = "storm.cgroup.hierarchy.dir";
    +
    +    /**
    +     * resources to to be controlled by cgroups
    +     */
    +    @isStringList
    +    public static final Object STORM_CGROUP_RESOURCES = "storm.cgroup.resources";
    +
    +    /**
    +     * name for the cgroup hierarchy
    +     */
    +    @isString
    +    public static final Object STORM_CGROUP_HIERARCHY_NAME = "storm.cgroup.hierarchy.name";
    +
    +    /**
    +     * flag to determine whether to use a resource isolation plugin
    +     * Also determines whether the unit tests for cgroup runs.
    +     * If storm.resource.isolation.plugin.enable is set to false the unit tests for cgroups will not run
    +     */
    +    @isBoolean
    +    public static final String STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE = "storm.resource.isolation.plugin.enable";
    +
    +    /**
    +     * root directory for cgoups
    +     */
    +    @isString
    +    public static String STORM_SUPERVISOR_CGROUP_ROOTDIR = "storm.supervisor.cgroup.rootdir";
    +
    +    /**
    +     * the manually set memory limit (in MB) for each CGroup on supervisor node
    +     */
    +    @isPositiveNumber
    +    public static String STORM_WORKER_CGROUP_MEMORY_MB_LIMIT = "storm.worker.cgroup.memory.mb.limit";
    +
    +    /**
    +     * the manually set cpu share for each CGroup on supervisor node
    +     */
    +    @isPositiveNumber
    +    public static String STORM_WORKER_CGROUP_CPU_LIMIT = "storm.worker.cgroup.cpu.limit";
    +
    +    /**
    +     * full path to cgexec command
    +     */
    +    @isString
    +    public static String STORM_CGROUP_CGEXEC_CMD = "storm.cgroup.cgexec.cmd";
    +
    +    /**
    +     * The amount of memory a worker can exceed its allocation before cgroup will kill it
    +     */
    +    @isPositiveNumber
    +    public static String STORM_CGROUP_MEMORY_MB_LIMIT_TOLERANCE_MARGIN = "storm.cgroup.memory.mb.limit.tolerance.margin";
    --- End diff --
    
    This is not exactly what I had in mind from my review comment. Now the Java name is consistent with the config name, but if we look at the other config definitions, there is also a consistency in the units of the value being appended to the name, like `topology.worker.max.heap.size.mb`. This PR does something different with `storm.worker.cgroup.memory.mb.limit` and `storm.cgroup.memory.mb.limit.tolerance.margin` where the unit is somewhere in the middle of the config.  I think it is better to be consistent with the other existing configs.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52351122
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * initalize subsystems
    +     */
    +    private void prepareSubSystem(Map conf) throws IOException {
    +        List<SubSystemType> subSystemTypes = new LinkedList<>();
    +        for (String resource : Config.getCgroupStormResources(conf)) {
    +            subSystemTypes.add(SubSystemType.getSubSystem(resource));
    +        }
    +
    +        this.hierarchy = center.getHierarchyWithSubSystems(subSystemTypes);
    +
    +        if (this.hierarchy == null) {
    +            Set<SubSystemType> types = new HashSet<SubSystemType>();
    +            types.add(SubSystemType.cpu);
    +            this.hierarchy = new Hierarchy(Config.getCgroupStormHierarchyName(conf), types, Config.getCgroupStormHierarchyDir(conf));
    +        }
    +        this.rootCgroup = new CgroupCommon(this.rootDir, this.hierarchy, this.hierarchy.getRootCgroups());
    +
    +        // set upper limit to how much cpu can be used by all workers running on supervisor node.
    +        // This is done so that some cpu cycles will remain free to run the daemons and other miscellaneous OS operations.
    +        CpuCore supervisorRootCPU = (CpuCore)  this.rootCgroup.getCores().get(SubSystemType.cpu);
    +        setCpuUsageUpperLimit(supervisorRootCPU, ((Number) this.conf.get(Config.SUPERVISOR_CPU_CAPACITY)).intValue());
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g.
    +     * If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public void reserveResourcesForWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = null;
    +        // The manually set STORM_WORKER_CGROUP_CPU_LIMIT config on supervisor will overwrite resources assigned by RAS (Resource Aware Scheduler)
    +        if (this.conf.get(Config.STORM_WORKER_CGROUP_CPU_LIMIT) != null) {
    +            cpuNum = (Number) this.conf.get(Config.STORM_WORKER_CGROUP_CPU_LIMIT);
    +        } else if(resourcesMap.get("cpu") != null) {
    +            cpuNum = (Number) resourcesMap.get("cpu");
    +        }
    +
    +        Number totalMem = null;
    +        // The manually set STORM_WORKER_CGROUP_MEMORY_MB_LIMIT config on supervisor will overwrite resources assigned by RAS (Resource Aware Scheduler)
    +        if (this.conf.get(Config.STORM_WORKER_CGROUP_MEMORY_MB_LIMIT) != null) {
    +            totalMem = (Number) this.conf.get(Config.STORM_WORKER_CGROUP_MEMORY_MB_LIMIT);
    +        } else if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, this.hierarchy, this.rootCgroup);
    +        this.center.createCgroup(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    +            } catch (IOException e) {
    +                throw new RuntimeException("Cannot set cpu.shares! Exception: " + e);
    +            }
    +        }
    +
    +        if (totalMem != null) {
    +            MemoryCore memCore = (MemoryCore) workerGroup.getCores().get(SubSystemType.memory);
    +            try {
    +                memCore.setPhysicalUsageLimit(Long.valueOf(totalMem.longValue() * 1024 * 1024));
    +            } catch (IOException e) {
    +                throw new RuntimeException("Cannot set memory.limit_in_bytes! Exception: " + e);
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52287186
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!getCores().keySet().contains(SubSystemType.cpuset)) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getCgroupCloneChildren() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(eventFd);
    +        sb.append(' ');
    +        sb.append(controlFd);
    +        for (String arg : args) {
    +            sb.append(' ');
    +            sb.append(arg);
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_EVENT_CONTROL), sb.toString());
    +    }
    +
    +    public Hierarchy getHierarchy() {
    +        return hierarchy;
    +    }
    +
    +    public String getName() {
    +        return name;
    +    }
    +
    +    public String getDir() {
    +        return dir;
    +    }
    +
    +    public CgroupCommon getParent() {
    +        return parent;
    +    }
    +
    +    public Set<CgroupCommon> getChildren() {
    +
    +        File file = new File(this.dir);
    +        File[] files = file.listFiles();
    +        if (files == null) {
    --- End diff --
    
    this won't happen.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52065249
  
    --- Diff: storm-core/test/clj/org/apache/storm/supervisor_test.clj ---
    @@ -335,8 +334,8 @@
                                           mock-storm-id
                                           mock-port
                                           mock-worker-id
    -                                      mock-mem-onheap)
    -                (verify-first-call-args-for-indices launch-process
    +                                      (WorkerResources.))
    +            (verify-first-call-args-for-indices launch-process
    --- End diff --
    
    Wrong indentation.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52064795
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/CpusetCore.java ---
    @@ -0,0 +1,212 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.Constants;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +
    +public class CpusetCore implements CgroupCore {
    +
    +    public static final String CPUSET_CPUS = "/cpuset.cpus";
    +    public static final String CPUSET_MEMS = "/cpuset.mems";
    +    public static final String CPUSET_MEMORY_MIGRATE = "/cpuset.memory_migrate";
    +    public static final String CPUSET_CPU_EXCLUSIVE = "/cpuset.cpu_exclusive";
    +    public static final String CPUSET_MEM_EXCLUSIVE = "/cpuset.mem_exclusive";
    +    public static final String CPUSET_MEM_HARDWALL = "/cpuset.mem_hardwall";
    +    public static final String CPUSET_MEMORY_PRESSURE = "/cpuset.memory_pressure";
    +    public static final String CPUSET_MEMORY_PRESSURE_ENABLED = "/cpuset.memory_pressure_enabled";
    +    public static final String CPUSET_MEMORY_SPREAD_PAGE = "/cpuset.memory_spread_page";
    +    public static final String CPUSET_MEMORY_SPREAD_SLAB = "/cpuset.memory_spread_slab";
    +    public static final String CPUSET_SCHED_LOAD_BALANCE = "/cpuset.sched_load_balance";
    +    public static final String CPUSET_SCHED_RELAX_DOMAIN_LEVEL = "/cpuset.sched_relax_domain_level";
    +
    +    private final String dir;
    +
    +    public CpusetCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.cpuset;
    +    }
    +
    +    public void setCpus(int[] nums) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        for (int num : nums) {
    +            sb.append(num);
    +            sb.append(',');
    +        }
    +        sb.deleteCharAt(sb.length() - 1);
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CPUSET_CPUS), sb.toString());
    +    }
    +
    +    public int[] getCpus() throws IOException {
    +        String output = CgroupUtils.readFileByLine(Constants.getDir(this.dir, CPUSET_CPUS)).get(0);
    +        return parseNums(output);
    +    }
    +
    +    public void setMems(int[] nums) throws IOException {
    --- End diff --
    
    D.R.Y. `setCpus`


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52277531
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    --- End diff --
    
    is the map being used to generate unique entries for each type? If so, do you want to log a warn message for two entries with same type


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52333625
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    --- End diff --
    
    will move this.getHierarchies() under the if conditional



---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51288016
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            for (SubSystemType type : hierarchy.getSubSystems()) {
    +                if (type == subsystem) {
    +                    return hierarchy;
    +                }
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subsystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subsystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy mounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (CgroupUtils.dirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return h;
    +                }
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.mounted(hierarchy) != null) {
    +            LOG.error("{} is mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subsystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subsystems) {
    +            if (this.busy(type) != null) {
    +                LOG.error("subsystem: {} is busy", type.name());
    +                subsystems.remove(type);
    +            }
    +        }
    +        if (subsystems.size() == 0) {
    +            return;
    +        }
    +        if (!CgroupUtils.dirExists(hierarchy.getDir())) {
    +            new File(hierarchy.getDir()).mkdirs();
    +        }
    +        String subSystems = CgroupUtils.reAnalyse(subsystems);
    +        SystemOperation.mount(subSystems, hierarchy.getDir(), "cgroup", subSystems);
    +
    +    }
    +
    +    @Override
    +    public void umount(Hierarchy hierarchy) throws IOException {
    +        if (this.mounted(hierarchy) != null) {
    +            hierarchy.getRootCgroups().delete();
    +            SystemOperation.umount(hierarchy.getDir());
    +            CgroupUtils.deleteDir(hierarchy.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void create(CgroupCommon cgroup) throws SecurityException {
    +        if (cgroup.isRoot()) {
    +            LOG.error("You can't create rootCgroup in this function");
    +            return;
    +        }
    +        CgroupCommon parent = cgroup.getParent();
    +        while (parent != null) {
    +            if (!CgroupUtils.dirExists(parent.getDir())) {
    +                LOG.error(" {} is not existed", parent.getDir());
    +                return;
    +            }
    +            parent = parent.getParent();
    +        }
    +        Hierarchy h = cgroup.getHierarchy();
    +        if (mounted(h) == null) {
    +            LOG.error("{} is not mounted", h.getDir());
    +            return;
    +        }
    +        if (CgroupUtils.dirExists(cgroup.getDir())) {
    +            LOG.error("{} is existed", cgroup.getDir());
    +            return;
    +        }
    +
    +        //Todo perhaps thrown exception or print out error message is dir is not created successfully
    +        if (!(new File(cgroup.getDir())).mkdir()) {
    +            LOG.error("Could not create cgroup dir at {}", cgroup.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void delete(CgroupCommon cgroup) throws IOException {
    +
    +        cgroup.delete();
    +    }
    +
    +    public static void main(String args[]) {
    +        System.out.println(CgroupCenter.getInstance().getHierarchies().get(0).getRootCgroups().getChildren().size());
    --- End diff --
    
    will remove the main method, and please refer to my previous 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52286989
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!getCores().keySet().contains(SubSystemType.cpuset)) {
    --- End diff --
    
    It is not clear why this condition is present. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52421257
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/CpuacctCore.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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class CpuacctCore implements CgroupCore {
    +
    +    public static final String CPUACCT_USAGE = "/cpuacct.usage";
    +    public static final String CPUACCT_STAT = "/cpuacct.stat";
    +    public static final String CPUACCT_USAGE_PERCPU = "/cpuacct.usage_percpu";
    +
    +    private final String dir;
    +
    +    public CpuacctCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.cpuacct;
    +    }
    +
    +    public Long getCpuUsage() throws IOException {
    +        return Long.parseLong(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_USAGE)).get(0));
    +    }
    +
    +    public Map<StatType, Long> getCpuStat() throws IOException {
    +        List<String> strs = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_STAT));
    +        Map<StatType, Long> result = new HashMap<StatType, Long>();
    +        result.put(StatType.user, Long.parseLong(strs.get(0).split(" ")[1]));
    +        result.put(StatType.system, Long.parseLong(strs.get(1).split(" ")[1]));
    +        return result;
    +    }
    +
    +    public Long[] getPerCpuUsage() throws IOException {
    +        String str = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUACCT_USAGE_PERCPU)).get(0);
    +        String[] strArgs = str.split(" ");
    +        Long[] result = new Long[strArgs.length];
    +        for (int i = 0; i < result.length; i++) {
    +            result[i] = Long.parseLong(strArgs[i]);
    +        }
    +        return result;
    +    }
    +
    +    public enum StatType {
    --- End diff --
    
    Nested enum types are implicitly static.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52070982
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1062,9 +1072,15 @@
                             (add-to-classpath [stormjar])
                             (add-to-classpath topo-classpath))
               top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
    -          mem-onheap (if (and mem-onheap (> mem-onheap 0)) ;; not nil and not zero
    -                       (int (Math/ceil mem-onheap)) ;; round up
    +          mem-onheap (if (and (.get_mem_on_heap resources) (> (.get_mem_on_heap resources) 0)) ;; not nil and not zero
    +                       (int (Math/ceil (.get_mem_on_heap resources))) ;; round up
                            (storm-conf WORKER-HEAP-MEMORY-MB)) ;; otherwise use default value
    +          mem-offheap (if (.get_mem_off_heap resources)
    +                        (int (Math/ceil (.get_mem_off_heap resources))) ;; round up
    +                        0)
    +
    +          cpu (int (Math/ceil (.get_cpu resources)))
    --- End diff --
    
    Is the "cpu" resource already validated, or do we need to check its values 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52112633
  
    --- Diff: storm-core/src/jvm/org/apache/storm/Config.java ---
    @@ -2155,6 +2156,63 @@
         @isString
         public static final Object CLIENT_JAR_TRANSFORMER = "client.jartransformer.class";
     
    +
    +    @isImplementationOfClass(implementsClass = ResourceIsolationInterface.class)
    +    public static final Object STORM_RESOURCE_ISOLATION_PLUGIN = "storm.resource.isolation.plugin";
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51286930
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51287016
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class SystemOperation {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52079318
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/Utils.java ---
    @@ -1367,12 +1367,67 @@ public static int toPositive(int number) {
             return number & Integer.MAX_VALUE;
         }
     
    -    public static RuntimeException wrapInRuntime(Exception e){
    -        if (e instanceof RuntimeException){
    -            return (RuntimeException)e;
    -        }else {
    +    public static RuntimeException wrapInRuntime(Exception e) {
    +        if (e instanceof RuntimeException) {
    +            return (RuntimeException) e;
    +        } else {
                 return new RuntimeException(e);
             }
         }
    +
    +    public static void ensure_process_killed(Integer pid) {
    +        // in this function, just kill the process 5 times
    +        // make sure the process be killed definitely
    +        for (int i = 0; i < 5; i++) {
    +            try {
    +                exec_command("kill -9 " + pid);
    +                LOG.info("kill -9 process " + pid);
    +                sleepMs(100);
    +            } catch (ExecuteException e) {
    +                LOG.info("Error when trying to kill " + pid + ". Process has been killed");
    +                return;
    +            } catch (Exception e) {
    +                LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +            }
    +        }
    +    }
    +
    +    public static void process_killed(Integer pid) {
    +        try {
    +            exec_command("kill " + pid);
    +            LOG.info("kill process " + pid);
    +        } catch (ExecuteException e) {
    +            LOG.info("Error when trying to kill " + pid + ". Process has been killed. ");
    +        } catch (Exception e) {
    +            LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +        }
    +    }
    +
    +    public static void kill(Integer pid) {
    +        process_killed(pid);
    +
    +        sleepMs(1000);
    +
    +        ensure_process_killed(pid);
    +    }
    +
    +    public static void exec_command(String command) throws ExecuteException, IOException {
    +        String[] cmdlist = command.split(" ");
    +        CommandLine cmd = new CommandLine(cmdlist[0]);
    +        for (int i = 1; i < cmdlist.length; i++) {
    +            cmd.addArgument(cmdlist[i]);
    +        }
    +
    +        DefaultExecutor exec = new DefaultExecutor();
    +        exec.execute(cmd);
    +    }
    +
    +    public static void sleepMs(long ms) {
    --- End diff --
    
    And sleep stuff in `Util` or `Time` or something.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51178357
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,174 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        try {
    +            String cmd = "rmdir " + dir;
    +            SystemOperation.exec(cmd);
    +        } catch (IOException e) {
    +            // TODO Auto-generated catch block
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52352581
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,128 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    --- End diff --
    
    Yes we can use FileUtils.deleteDirectory() but we won't get the the helpful warning messages.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51290894
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1088,8 +1104,26 @@
                                                 (str "file:///" storm-log4j2-conf-dir))
                                               storm-log4j2-conf-dir)
                                          file-path-separator "worker.xml")
    +
    +          cgroup-command (if (conf STORM-CGROUP-ENABLE)
    --- End diff --
    
    cgroups is disabled by default (via default.yaml) and the code you referenced is the code that will be executed for "Cluster mode" and not "local mode".


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52277955
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    --- End diff --
    
    It should use equals instead of identity 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#issuecomment-176227654
  
    Rat is complaining about some of your files not having the correct license.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51788986
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/MemoryCore.java ---
    @@ -0,0 +1,189 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.Constants;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +
    +import java.io.IOException;
    +
    +public class MemoryCore implements CgroupCore {
    +
    +    public static final String MEMORY_STAT = "/memory.stat";
    +    public static final String MEMORY_USAGE_IN_BYTES = "/memory.usage_in_bytes";
    +    public static final String MEMORY_MEMSW_USAGE_IN_BYTES = "/memory.memsw.usage_in_bytes";
    +    public static final String MEMORY_MAX_USAGE_IN_BYTES = "/memory.max_usage_in_bytes";
    +    public static final String MEMORY_MEMSW_MAX_USAGE_IN_BYTES = "/memory.memsw.max_usage_in_bytes";
    +    public static final String MEMORY_LIMIT_IN_BYTES = "/memory.limit_in_bytes";
    +    public static final String MEMORY_MEMSW_LIMIT_IN_BYTES = "/memory.memsw.limit_in_bytes";
    +    public static final String MEMORY_FAILCNT = "/memory.failcnt";
    +    public static final String MEMORY_MEMSW_FAILCNT = "/memory.memsw.failcnt";
    +    public static final String MEMORY_FORCE_EMPTY = "/memory.force_empty";
    +    public static final String MEMORY_SWAPPINESS = "/memory.swappiness";
    +    public static final String MEMORY_USE_HIERARCHY = "/memory.use_hierarchy";
    +    public static final String MEMORY_OOM_CONTROL = "/memory.oom_control";
    +
    +    private final String dir;
    +
    +    public MemoryCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.memory;
    +    }
    +
    +    public static class Stat {
    +        public final long cacheSize;
    +        public final long rssSize;
    +        public final long mappedFileSize;
    +        public final long pgpginNum;
    +        public final long pgpgoutNum;
    +        public final long swapSize;
    +        public final long activeAnonSize;
    +        public final long inactiveAnonSize;
    +        public final long activeFileSize;
    +        public final long inactiveFileSize;
    +        public final long unevictableSize;
    +        public final long hierarchicalMemoryLimitSize;
    +        public final long hierarchicalMemswLimitSize;
    +        public final long totalCacheSize;
    +        public final long totalRssSize;
    +        public final long totalMappedFileSize;
    +        public final long totalPgpginNum;
    +        public final long totalPgpgoutNum;
    +        public final long totalSwapSize;
    +        public final long totalActiveAnonSize;
    +        public final long totalInactiveAnonSize;
    +        public final long totalActiveFileSize;
    +        public final long totalInactiveFileSize;
    +        public final long totalUnevictableSize;
    +        public final long totalHierarchicalMemoryLimitSize;
    +        public final long totalHierarchicalMemswLimitSize;
    +
    +        public Stat(String output) {
    +            String[] splits = output.split("\n");
    +            this.cacheSize = Long.parseLong(splits[0]);
    +            this.rssSize = Long.parseLong(splits[1]);
    +            this.mappedFileSize = Long.parseLong(splits[2]);
    +            this.pgpginNum = Long.parseLong(splits[3]);
    +            this.pgpgoutNum = Long.parseLong(splits[4]);
    +            this.swapSize = Long.parseLong(splits[5]);
    +            this.inactiveAnonSize = Long.parseLong(splits[6]);
    +            this.activeAnonSize = Long.parseLong(splits[7]);
    +            this.inactiveFileSize = Long.parseLong(splits[8]);
    +            this.activeFileSize = Long.parseLong(splits[9]);
    +            this.unevictableSize = Long.parseLong(splits[10]);
    +            this.hierarchicalMemoryLimitSize = Long.parseLong(splits[11]);
    +            this.hierarchicalMemswLimitSize = Long.parseLong(splits[12]);
    +            this.totalCacheSize = Long.parseLong(splits[13]);
    +            this.totalRssSize = Long.parseLong(splits[14]);
    +            this.totalMappedFileSize = Long.parseLong(splits[15]);
    +            this.totalPgpginNum = Long.parseLong(splits[16]);
    +            this.totalPgpgoutNum = Long.parseLong(splits[17]);
    +            this.totalSwapSize = Long.parseLong(splits[18]);
    +            this.totalInactiveAnonSize = Long.parseLong(splits[19]);
    +            this.totalActiveAnonSize = Long.parseLong(splits[20]);
    +            this.totalInactiveFileSize = Long.parseLong(splits[21]);
    +            this.totalActiveFileSize = Long.parseLong(splits[22]);
    +            this.totalUnevictableSize = Long.parseLong(splits[23]);
    +            this.totalHierarchicalMemoryLimitSize = Long.parseLong(splits[24]);
    +            this.totalHierarchicalMemswLimitSize = Long.parseLong(splits[25]);
    --- End diff --
    
    I presume replacing it with totalHierarchialMemSwapLimitSize would be more readable, if it is what it means


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52327513
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    --- End diff --
    
    "type" is just a comma delimited list of subsystems that are mounted on a hierarchy. Cgroup does not support mounting the same set of subsystems on multiple heirarchies (a.k.a mirrored hierarchies).  Thus, using the comma delimited list of subsystems as key is going to be distinct


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51134861
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null)
    +            instance = new CgroupCenter();
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup"))
    +                    continue;
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    --- End diff --
    
    This is also just about everywhere 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51156711
  
    --- Diff: storm-core/test/jvm/org/apache/storm/TestCGroup.java ---
    @@ -0,0 +1,94 @@
    +package org.apache.storm;
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52678716
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -344,6 +347,12 @@
        :sync-retry (atom 0)
        :download-lock (Object.)
        :stormid->profiler-actions (atom {})
    +   :resource-isolation-manager (if (conf STORM-RESOURCE-ISOLATION-PLUGIN-ENABLE)
    +                     (let [resource-isolation-manager (Utils/newInstance (conf STORM-RESOURCE-ISOLATION-PLUGIN))]
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52052704
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    +        File file = new File(dir);
    +        return file.exists();
    +    }
    +
    +    public static boolean dirExists(String dir) {
    +        File file = new File(dir);
    +        return file.isDirectory();
    +    }
    +
    +    public static Set<SubSystemType> analyse(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static String reAnalyse(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return CgroupUtils.fileExists(Constants.CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    +        List<String> result = new ArrayList<String>();
    +        File file = new File(fileDir);
    +        try (FileReader fileReader = new FileReader(file);
    +             BufferedReader reader = new BufferedReader(fileReader)) {
    +            String tempString = null;
    +            while ((tempString = reader.readLine()) != null) {
    +                result.add(tempString);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static void writeFileByLine(String fileDir, List<String> strings) throws IOException {
    +        File file = new File(fileDir);
    +        if (!file.exists()) {
    +            LOG.error("{} is no existed", fileDir);
    +            return;
    +        }
    +        try (FileWriter writer = new FileWriter(file, true);
    +             BufferedWriter bw = new BufferedWriter(writer)) {
    +            for (String string : strings) {
    +                bw.write(string);
    +                bw.newLine();
    +                bw.flush();
    +            }
    +        }
    +    }
    +
    +    public static void writeFileByLine(String fileDir, String string) throws IOException {
    --- End diff --
    
    This whole method's body could be
    ```
    {
        writeFileByLine(fileDir, Arrays.asList(string));
    }
    ```


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52335002
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return true;
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.isMounted(hierarchy)) {
    +            LOG.error("{} is already mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subSystems) {
    +            if (this.getHierarchyWithSubSystem(type) != null) {
    +                LOG.error("subSystem: {} is busy", type.name());
    +                subSystems.remove(type);
    +            }
    +        }
    +        if (subSystems.size() == 0) {
    +            return;
    +        }
    +        if (!Utils.CheckDirExists(hierarchy.getDir())) {
    +            new File(hierarchy.getDir()).mkdirs();
    +        }
    +        String subSystemsName = CgroupUtils.subSystemsToString(subSystems);
    +        SystemOperation.mount(subSystemsName, hierarchy.getDir(), "cgroup", subSystemsName);
    +
    +    }
    +
    +    @Override
    +    public void umount(Hierarchy hierarchy) throws IOException {
    +        if (this.isMounted(hierarchy)) {
    +            hierarchy.getRootCgroups().delete();
    +            SystemOperation.umount(hierarchy.getDir());
    +            CgroupUtils.deleteDir(hierarchy.getDir());
    +        } else {
    +            LOG.error("{} is not mounted", hierarchy.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void createCgroup(CgroupCommon cgroup) throws SecurityException {
    +        if (cgroup.isRoot()) {
    +            LOG.error("You can't create rootCgroup in this function");
    +            return;
    +        }
    +        CgroupCommon parent = cgroup.getParent();
    +        while (parent != null) {
    +            if (!Utils.CheckDirExists(parent.getDir())) {
    +                LOG.error("Parent {} does not exist", parent.getDir());
    +                return;
    +            }
    +            parent = parent.getParent();
    +        }
    +        Hierarchy h = cgroup.getHierarchy();
    +        if (!isMounted(h)) {
    +            LOG.error("hierarchy {} is not mounted", h.getDir());
    +            return;
    +        }
    +        if (Utils.CheckDirExists(cgroup.getDir())) {
    +            LOG.error("cgroup {} already exists", cgroup.getDir());
    +            return;
    +        }
    +
    +        //Todo perhaps thrown exception or print out error message is dir is not created successfully
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52342723
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * initalize subsystems
    +     */
    +    private void prepareSubSystem(Map conf) throws IOException {
    +        List<SubSystemType> subSystemTypes = new LinkedList<>();
    +        for (String resource : Config.getCgroupStormResources(conf)) {
    +            subSystemTypes.add(SubSystemType.getSubSystem(resource));
    +        }
    +
    +        this.hierarchy = center.getHierarchyWithSubSystems(subSystemTypes);
    +
    +        if (this.hierarchy == null) {
    +            Set<SubSystemType> types = new HashSet<SubSystemType>();
    +            types.add(SubSystemType.cpu);
    +            this.hierarchy = new Hierarchy(Config.getCgroupStormHierarchyName(conf), types, Config.getCgroupStormHierarchyDir(conf));
    +        }
    +        this.rootCgroup = new CgroupCommon(this.rootDir, this.hierarchy, this.hierarchy.getRootCgroups());
    +
    +        // set upper limit to how much cpu can be used by all workers running on supervisor node.
    +        // This is done so that some cpu cycles will remain free to run the daemons and other miscellaneous OS operations.
    +        CpuCore supervisorRootCPU = (CpuCore)  this.rootCgroup.getCores().get(SubSystemType.cpu);
    --- End diff --
    
    extra space


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52417705
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    --- End diff --
    
    I understand that. But cgroup config files should one value only in the file at a time? writeFileByLine should override the file with latest value. If you read getNotifyOnRelease, you will read the first line of file which is actually the oldest. please refer to my comment on writeFileByLine


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51189689
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null)
    +            instance = new CgroupCenter();
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup"))
    +                    continue;
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error", e);
    --- End diff --
    
    perhaps we can file a follow up jira to investigate which functions we want to throw exceptions vs. eating 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52338605
  
    --- Diff: conf/defaults.yaml ---
    @@ -263,7 +263,7 @@ topology.state.checkpoint.interval.ms: 1000
     # topology priority describing the importance of the topology in decreasing importance starting from 0 (i.e. 0 is the highest priority and the priority importance decreases as the priority number increases).
     # Recommended range of 0-29 but no hard limit set.
     topology.priority: 29
    -topology.component.resources.onheap.memory.mb: 128.0
    +topology.component.resources.onheap.memory.mb: 256.0
    --- End diff --
    
    Do we have a reason for doubling the default per-instance memory usage?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52098115
  
    --- Diff: conf/defaults.yaml ---
    @@ -281,3 +281,16 @@ pacemaker.thread.timeout: 10
     pacemaker.childopts: "-Xmx1024m"
     pacemaker.auth.method: "NONE"
     pacemaker.kerberos.users: []
    +
    +storm.resource.isolation.plugin: "org.apache.storm.container.cgroup.CgroupManager"
    +
    +# Configs for CGroup support
    +storm.cgroup.hierarchy.dir: "/cgroup/storm_resources"
    +storm.cgroup.resources:
    +    - cpu
    +    - memory
    +storm.cgroup.hierarchy.name: "storm"
    +# Also determines whether the unit tests for cgroup runs.  If cgroup.enable is set to false the unit tests for cgroups will not run
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52338397
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    --- End diff --
    
    It doesn't matter.  All the cgroup config "files" only takes the the latest value written to the 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51134290
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null)
    +            instance = new CgroupCenter();
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup"))
    +                    continue;
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    --- End diff --
    
    Can we update this to be java 7 like using the Closable and the try.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52339067
  
    --- Diff: conf/defaults.yaml ---
    @@ -263,7 +263,7 @@ topology.state.checkpoint.interval.ms: 1000
     # topology priority describing the importance of the topology in decreasing importance starting from 0 (i.e. 0 is the highest priority and the priority importance decreases as the priority number increases).
     # Recommended range of 0-29 but no hard limit set.
     topology.priority: 29
    -topology.component.resources.onheap.memory.mb: 128.0
    +topology.component.resources.onheap.memory.mb: 256.0
    --- End diff --
    
    So system components (i.e. acker and writer) use more memory than 128.0mb. When I turned on cgroups, workers will just system components on them will get consistently killed by cgroups for using more memory than allowed


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52075801
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    +        File file = new File(dir);
    +        return file.exists();
    +    }
    +
    +    public static boolean dirExists(String dir) {
    +        File file = new File(dir);
    +        return file.isDirectory();
    +    }
    +
    +    public static Set<SubSystemType> analyse(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static String reAnalyse(Set<SubSystemType> subSystems) {
    --- End diff --
    
    I agree this is confusing. `analyse` seems to parse the CSV string into a Set of SubSystemTypes, and `reAnalyse` seems to do the reverse. It is not necessarily true that `reAnalyse(analyse(o)).equals(o)`.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51138519
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java ---
    @@ -0,0 +1,47 @@
    +/**
    + * 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.container.cgroup;
    +
    +import java.io.IOException;
    +import java.util.Set;
    +
    +public interface CgroupCommonOperation {
    +
    +    public void addTask(int taskid) throws IOException;
    +
    +    public Set<Integer> getTasks() throws IOException;
    +
    +    public void addProcs(int pid) throws IOException;
    +
    +    public Set<Integer> getPids() throws IOException;
    +
    +    public void setNotifyOnRelease(boolean flag) throws IOException;
    +
    +    public boolean getNotifyOnRelease() throws IOException;
    +
    +    public void setReleaseAgent(String command) throws IOException;
    +
    +    public String getReleaseAgent() throws IOException;
    +
    +    public void setCgroupCloneChildren(boolean flag) throws IOException;
    +
    +    public boolean getCgroupCloneChildren() throws IOException;
    +
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException;
    +
    +}
    --- End diff --
    
    IS there any way to add some javadocs to these.  It would make it a lot simpler to understand what each of these are supposed to do.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51381992
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    +        this.center.create(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    --- End diff --
    
    Anyways adding cpu.cfs_period_us and cpu.cfs_quota_us to set a hard time should be a minor and easy change to the current code in the PR, if we find that cpu.share doesn't work 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51916863
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    --- End diff --
    
    will spit


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51175096
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null)
    +            instance = new CgroupCenter();
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup"))
    +                    continue;
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by jerrypeng <gi...@git.apache.org>.
Github user jerrypeng commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-183445741
  
    I upmerged


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52648907
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java ---
    @@ -0,0 +1,78 @@
    +/**
    + * 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.container.cgroup;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * An interface to manage cgroups
    --- End diff --
    
    This doc could use a little expansion. It's so vague as to be basically useless. The rest in this file look 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52421562
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/CpusetCore.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +
    +public class CpusetCore implements CgroupCore {
    +
    +    public static final String CPUSET_CPUS = "/cpuset.cpus";
    +    public static final String CPUSET_MEMS = "/cpuset.mems";
    +    public static final String CPUSET_MEMORY_MIGRATE = "/cpuset.memory_migrate";
    +    public static final String CPUSET_CPU_EXCLUSIVE = "/cpuset.cpu_exclusive";
    +    public static final String CPUSET_MEM_EXCLUSIVE = "/cpuset.mem_exclusive";
    +    public static final String CPUSET_MEM_HARDWALL = "/cpuset.mem_hardwall";
    +    public static final String CPUSET_MEMORY_PRESSURE = "/cpuset.memory_pressure";
    +    public static final String CPUSET_MEMORY_PRESSURE_ENABLED = "/cpuset.memory_pressure_enabled";
    +    public static final String CPUSET_MEMORY_SPREAD_PAGE = "/cpuset.memory_spread_page";
    +    public static final String CPUSET_MEMORY_SPREAD_SLAB = "/cpuset.memory_spread_slab";
    +    public static final String CPUSET_SCHED_LOAD_BALANCE = "/cpuset.sched_load_balance";
    +    public static final String CPUSET_SCHED_RELAX_DOMAIN_LEVEL = "/cpuset.sched_relax_domain_level";
    +
    +    private final String dir;
    +
    +    public CpusetCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.cpuset;
    +    }
    +
    +    public void setCpus(int[] nums) throws IOException {
    +        setConfigs(nums, CPUSET_CPUS);
    +    }
    +
    +    public int[] getCpus() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPUS)).get(0);
    +        return parseNums(output);
    +    }
    +
    +    public void setMems(int[] nums) throws IOException {
    +        setConfigs(nums, CPUSET_MEMS);
    +    }
    +
    +    private void setConfigs(int[] nums, String config) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        for (int num : nums) {
    +            sb.append(num);
    +            sb.append(',');
    +        }
    +        sb.deleteCharAt(sb.length() - 1);
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, config), sb.toString());
    +    }
    +
    +    public int[] getMems() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMS)).get(0);
    +        return parseNums(output);
    +    }
    +
    +    public void setMemMigrate(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_MIGRATE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemMigrate() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_MIGRATE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setCpuExclusive(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPU_EXCLUSIVE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isCpuExclusive() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPU_EXCLUSIVE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemExclusive(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_EXCLUSIVE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemExclusive() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_EXCLUSIVE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemHardwall(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_HARDWALL), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemHardwall() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_HARDWALL)).get(0));
    +        return output > 0;
    +    }
    +
    +    public int getMemPressure() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE)).get(0);
    +        return Integer.parseInt(output);
    +    }
    +
    +    public void setMemPressureEnabled(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemPressureEnabled() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemSpreadPage(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemSpreadPage() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemSpreadSlab(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemSpreadSlab() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setSchedLoadBlance(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isSchedLoadBlance() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setSchedRelaxDomainLevel(int value) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL), String.valueOf(value));
    +    }
    +
    +    public int getSchedRelaxDomainLevel() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL)).get(0);
    +        return Integer.parseInt(output);
    +    }
    +
    +    public static int[] parseNums(String outputStr) {
    +        char[] output = outputStr.toCharArray();
    +        LinkedList<Integer> numList = new LinkedList<Integer>();
    +        int value = 0;
    +        int start = 0;
    +        boolean isHyphen = false;
    +        for (char ch : output) {
    +            if (ch == ',') {
    +                if (isHyphen) {
    +                    for (; start <= value; start++) {
    +                        numList.add(start);
    +                    }
    +                    isHyphen = false;
    +                } else {
    +                    numList.add(value);
    +                }
    +                value = 0;
    +            } else if (ch == '-') {
    +                isHyphen = true;
    +                start = value;
    +                value = 0;
    +            } else {
    +                value = value * 10 + (ch - '0');
    +            }
    +        }
    +        if (output[output.length - 1] != ',') {
    +            if (isHyphen) {
    +                for (; start <= value; start++) {
    +                    numList.add(start);
    +                }
    +            } else {
    +                numList.add(value);
    +            }
    +        }
    +
    +        int[] nums = new int[numList.size()];
    --- End diff --
    
    not that simple. numList is a List<Integer> while the function returns int[].


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52061120
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SubSystem.java ---
    @@ -0,0 +1,78 @@
    +/**
    + * 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.container.cgroup;
    +
    +public class SubSystem {
    +
    +    private SubSystemType type;
    +
    +    private int hierarchyID;
    +
    +    private int cgroupsNum;
    +
    +    private boolean enable;
    +
    +    public SubSystem(SubSystemType type, int hierarchyID, int cgroupNum, boolean enable) {
    +        this.type = type;
    +        this.hierarchyID = hierarchyID;
    +        this.cgroupsNum = cgroupNum;
    +        this.enable = enable;
    +    }
    +
    +    public SubSystemType getType() {
    +        return type;
    +    }
    +
    +    public void setType(SubSystemType type) {
    +        this.type = type;
    +    }
    +
    +    public int getHierarchyID() {
    +        return hierarchyID;
    +    }
    +
    +    public void setHierarchyID(int hierarchyID) {
    +        this.hierarchyID = hierarchyID;
    +    }
    +
    +    public int getCgroupsNum() {
    +        return cgroupsNum;
    +    }
    +
    +    public void setCgroupsNum(int cgroupsNum) {
    +        this.cgroupsNum = cgroupsNum;
    +    }
    +
    +    public boolean isEnable() {
    +        return enable;
    +    }
    +
    +    public void setEnable(boolean enable) {
    +        this.enable = enable;
    +    }
    +
    +    @Override
    +    public boolean equals(Object object) {
    +        boolean ret = false;
    +        if (object != null && object instanceof SubSystem) {
    +            ret = (this.type.equals(((SubSystem)object).getType()) && this.hierarchyID == ((SubSystem)object).getHierarchyID());
    --- End diff --
    
    Cant enums be compared with `==` 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52678894
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,210 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The storm.supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{} is not existing.", file.getPath());
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * initalize subsystems
    +     */
    +    private void prepareSubSystem(Map conf) throws IOException {
    +        List<SubSystemType> subSystemTypes = new LinkedList<>();
    +        for (String resource : Config.getCgroupStormResources(conf)) {
    +            subSystemTypes.add(SubSystemType.getSubSystem(resource));
    +        }
    +
    +        this.hierarchy = center.getHierarchyWithSubSystems(subSystemTypes);
    +
    +        if (this.hierarchy == null) {
    +            Set<SubSystemType> types = new HashSet<SubSystemType>();
    +            types.add(SubSystemType.cpu);
    +            this.hierarchy = new Hierarchy(Config.getCgroupStormHierarchyName(conf), types, Config.getCgroupStormHierarchyDir(conf));
    +        }
    +        this.rootCgroup = new CgroupCommon(this.rootDir, this.hierarchy, this.hierarchy.getRootCgroups());
    +
    +        // set upper limit to how much cpu can be used by all workers running on supervisor node.
    +        // This is done so that some cpu cycles will remain free to run the daemons and other miscellaneous OS operations.
    +        CpuCore supervisorRootCPU = (CpuCore) this.rootCgroup.getCores().get(SubSystemType.cpu);
    +        setCpuUsageUpperLimit(supervisorRootCPU, ((Number) this.conf.get(Config.SUPERVISOR_CPU_CAPACITY)).intValue());
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g.
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52366910
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,218 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (CgroupUtils.enabled()) {
    +            instance = new CgroupCenter();
    +            return instance;
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                //Some mount options (i.e. rw and relatime) in type are not cgroups related
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                int hierarchyID = Integer.valueOf(split[1]);
    +                int cgroupNum = Integer.valueOf(split[2]);
    +                boolean enable =  Integer.valueOf(split[3]).intValue() == 1 ? true : false;
    +                subSystems.add(new SubSystem(type, hierarchyID, cgroupNum, enable));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    +            List<Hierarchy> hierarchies = this.getHierarchies();
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return true;
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +        if (this.isMounted(hierarchy)) {
    +            LOG.error("{} is already mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subSystems) {
    +            if (this.getHierarchyWithSubSystem(type) != null) {
    +                LOG.error("subSystem: {} is busy", type.name());
    --- End diff --
    
    can you also log the hierarchy directory to which it is already attached.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51189494
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommonOperation.java ---
    @@ -0,0 +1,47 @@
    +/**
    + * 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.container.cgroup;
    +
    +import java.io.IOException;
    +import java.util.Set;
    +
    +public interface CgroupCommonOperation {
    +
    +    public void addTask(int taskid) throws IOException;
    +
    +    public Set<Integer> getTasks() throws IOException;
    +
    +    public void addProcs(int pid) throws IOException;
    +
    +    public Set<Integer> getPids() throws IOException;
    +
    +    public void setNotifyOnRelease(boolean flag) throws IOException;
    +
    +    public boolean getNotifyOnRelease() throws IOException;
    +
    +    public void setReleaseAgent(String command) throws IOException;
    +
    +    public String getReleaseAgent() throws IOException;
    +
    +    public void setCgroupCloneChildren(boolean flag) throws IOException;
    +
    +    public boolean getCgroupCloneChildren() throws IOException;
    +
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException;
    +
    +}
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52287866
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    --- End diff --
    
    you can simplify by using file.getPath()


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51133599
  
    --- Diff: conf/defaults.yaml ---
    @@ -281,3 +281,16 @@ pacemaker.thread.timeout: 10
     pacemaker.childopts: "-Xmx1024m"
     pacemaker.auth.method: "NONE"
     pacemaker.kerberos.users: []
    +
    +storm.resource.isolation.plugin: "org.apache.storm.container.cgroup.CgroupManager"
    +
    +# Configs for CGroup support
    +cgroup.storm.hierarchy.dir: "/cgroup/storm_resources"
    +cgroup.storm.resources:
    +    - cpu
    +    - memory
    +cgroup.storm.hierarchy.name: "storm"
    +# Also determines whether the unit tests for cgroup runs.  If cgroup.enable is set to false the unit tests for cgroups will not run
    +cgroup.enable: false
    +cgroup.supervisor.rootdir: "storm"
    +cgroup.cgexec.cmd: "/bin/cgexec"
    --- End diff --
    
    Can we please prefix these configs with "storm."  and probably also "storm.supervisor."


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52351834
  
    --- Diff: conf/defaults.yaml ---
    @@ -263,7 +263,7 @@ topology.state.checkpoint.interval.ms: 1000
     # topology priority describing the importance of the topology in decreasing importance starting from 0 (i.e. 0 is the highest priority and the priority importance decreases as the priority number increases).
     # Recommended range of 0-29 but no hard limit set.
     topology.priority: 29
    -topology.component.resources.onheap.memory.mb: 128.0
    +topology.component.resources.onheap.memory.mb: 256.0
    --- End diff --
    
    Should we have a separate default value for system components? Otherwise, users' topologies might use more memory than their previous impression. E.g., a WordCount example might need 6GB to start. Since this is an important setting, @revans2 , what is your opinion? 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r53114746
  
    --- Diff: storm-core/src/jvm/org/apache/storm/Config.java ---
    @@ -2194,6 +2195,73 @@
         @isString
         public static final Object CLIENT_JAR_TRANSFORMER = "client.jartransformer.class";
     
    +
    +    /**
    +     * The plugin to be used for resource isolation
    +     */
    +    @isImplementationOfClass(implementsClass = ResourceIsolationInterface.class)
    +    public static final Object STORM_RESOURCE_ISOLATION_PLUGIN = "storm.resource.isolation.plugin";
    +
    +    /**
    +     * CGroup Setting below
    +     */
    +
    +    /**
    +     * root directory of the storm cgroup hierarchy
    +     */
    +    @isString
    +    public static final Object STORM_CGROUP_HIERARCHY_DIR = "storm.cgroup.hierarchy.dir";
    +
    +    /**
    +     * resources to to be controlled by cgroups
    +     */
    +    @isStringList
    +    public static final Object STORM_CGROUP_RESOURCES = "storm.cgroup.resources";
    +
    +    /**
    +     * name for the cgroup hierarchy
    +     */
    +    @isString
    +    public static final Object STORM_CGROUP_HIERARCHY_NAME = "storm.cgroup.hierarchy.name";
    +
    +    /**
    +     * flag to determine whether to use a resource isolation plugin
    +     * Also determines whether the unit tests for cgroup runs.
    +     * If storm.resource.isolation.plugin.enable is set to false the unit tests for cgroups will not run
    +     */
    +    @isBoolean
    +    public static final String STORM_RESOURCE_ISOLATION_PLUGIN_ENABLE = "storm.resource.isolation.plugin.enable";
    +
    +    /**
    +     * root directory for cgoups
    +     */
    +    @isString
    +    public static String STORM_SUPERVISOR_CGROUP_ROOTDIR = "storm.supervisor.cgroup.rootdir";
    +
    +    /**
    +     * the manually set memory limit (in MB) for each CGroup on supervisor node
    +     */
    +    @isPositiveNumber
    +    public static String STORM_WORKER_CGROUP_MEMORY_MB_LIMIT = "storm.worker.cgroup.memory.mb.limit";
    +
    +    /**
    +     * the manually set cpu share for each CGroup on supervisor node
    +     */
    +    @isPositiveNumber
    +    public static String STORM_WORKER_CGROUP_CPU_LIMIT = "storm.worker.cgroup.cpu.limit";
    +
    +    /**
    +     * full path to cgexec command
    +     */
    +    @isString
    +    public static String STORM_CGROUP_CGEXEC_CMD = "storm.cgroup.cgexec.cmd";
    +
    +    /**
    +     * The amount of memory a worker can exceed its allocation before cgroup will kill it
    +     */
    +    @isPositiveNumber
    +    public static String STORM_CGROUP_MEMORY_MB_LIMIT_TOLERANCE_MARGIN = "storm.cgroup.memory.mb.limit.tolerance.margin";
    --- End diff --
    
    Ok will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52680901
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,210 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The storm.supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{} is not existing.", file.getPath());
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * initalize subsystems
    +     */
    +    private void prepareSubSystem(Map conf) throws IOException {
    +        List<SubSystemType> subSystemTypes = new LinkedList<>();
    +        for (String resource : Config.getCgroupStormResources(conf)) {
    +            subSystemTypes.add(SubSystemType.getSubSystem(resource));
    +        }
    +
    +        this.hierarchy = center.getHierarchyWithSubSystems(subSystemTypes);
    +
    +        if (this.hierarchy == null) {
    +            Set<SubSystemType> types = new HashSet<SubSystemType>();
    +            types.add(SubSystemType.cpu);
    +            this.hierarchy = new Hierarchy(Config.getCgroupStormHierarchyName(conf), types, Config.getCgroupStormHierarchyDir(conf));
    +        }
    +        this.rootCgroup = new CgroupCommon(this.rootDir, this.hierarchy, this.hierarchy.getRootCgroups());
    +
    +        // set upper limit to how much cpu can be used by all workers running on supervisor node.
    +        // This is done so that some cpu cycles will remain free to run the daemons and other miscellaneous OS operations.
    +        CpuCore supervisorRootCPU = (CpuCore) this.rootCgroup.getCores().get(SubSystemType.cpu);
    +        setCpuUsageUpperLimit(supervisorRootCPU, ((Number) this.conf.get(Config.SUPERVISOR_CPU_CAPACITY)).intValue());
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g.
    +     * If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    --- End diff --
    
    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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51285551
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    +        this.center.create(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    --- End diff --
    
    The downside to this is that a topology might perform different when the cluster is relatively free vs if the cluster is heavily utilized.  This may give a user a harder time to judge how much resources to allocate to a topology to achieve a certain performance.  However, if we use cpu.share to limit cpu, clusters may achieve better cpu utilization.  And using cpu.share will also give storm clusters some inherent elasticity, since a topology's use of CPU resources can grow (as long as there are free resources) and shrink depending on need since there are no hard limits set.  By using cpu.shares we can have this elastic benefit and at the same time have a way to guarantee a certain amount of resources.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52053569
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    --- End diff --
    
    Don't we have Utils functions for this type 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52416018
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java ---
    @@ -0,0 +1,226 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.Device;
    +
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class BlkioCore implements CgroupCore {
    +
    +    public static final String BLKIO_WEIGHT = "/blkio.weight";
    +    public static final String BLKIO_WEIGHT_DEVICE = "/blkio.weight_device";
    +    public static final String BLKIO_RESET_STATS = "/blkio.reset_stats";
    +
    +    public static final String BLKIO_THROTTLE_READ_BPS_DEVICE = "/blkio.throttle.read_bps_device";
    +    public static final String BLKIO_THROTTLE_WRITE_BPS_DEVICE = "/blkio.throttle.write_bps_device";
    +    public static final String BLKIO_THROTTLE_READ_IOPS_DEVICE = "/blkio.throttle.read_iops_device";
    +    public static final String BLKIO_THROTTLE_WRITE_IOPS_DEVICE = "/blkio.throttle.write_iops_device";
    +
    +    public static final String BLKIO_THROTTLE_IO_SERVICED = "/blkio.throttle.io_serviced";
    +    public static final String BLKIO_THROTTLE_IO_SERVICE_BYTES = "/blkio.throttle.io_service_bytes";
    +
    +    public static final String BLKIO_TIME = "/blkio.time";
    +    public static final String BLKIO_SECTORS = "/blkio.sectors";
    +    public static final String BLKIO_IO_SERVICED = "/blkio.io_serviced";
    +    public static final String BLKIO_IO_SERVICE_BYTES = "/blkio.io_service_bytes";
    +    public static final String BLKIO_IO_SERVICE_TIME = "/blkio.io_service_time";
    +    public static final String BLKIO_IO_WAIT_TIME = "/blkio.io_wait_time";
    +    public static final String BLKIO_IO_MERGED = "/blkio.io_merged";
    +    public static final String BLKIO_IO_QUEUED = "/blkio.io_queued";
    +
    +    private final String dir;
    +
    +    public BlkioCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.blkio;
    +    }
    +
    +    /* weight: 100-1000 */
    +    public void setBlkioWeight(int weight) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT), String.valueOf(weight));
    +    }
    +
    +    public int getBlkioWeight() throws IOException {
    +        return Integer.valueOf(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT)).get(0)).intValue();
    +    }
    +
    +    public void setBlkioWeightDevice(Device device, int weight) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT_DEVICE), makeContext(device, weight));
    +    }
    +
    +    public Map<Device, Integer> getBlkioWeightDevice() throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT_DEVICE));
    +        Map<Device, Integer> result = new HashMap<Device, Integer>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Integer weight = Integer.valueOf(strArgs[1]);
    +            result.put(device, weight);
    +        }
    +        return result;
    +    }
    +
    +    public void setReadBps(Device device, long bps) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_READ_BPS_DEVICE), makeContext(device, bps));
    +    }
    +
    +    public Map<Device, Long> getReadBps() throws IOException {
    +        return parseConfig(BLKIO_THROTTLE_READ_BPS_DEVICE);
    +    }
    +
    +    public void setWriteBps(Device device, long bps) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_WRITE_BPS_DEVICE), makeContext(device, bps));
    +    }
    +
    +    public Map<Device, Long> getWriteBps() throws IOException {
    +        return parseConfig(BLKIO_THROTTLE_WRITE_BPS_DEVICE);
    +    }
    +
    +    public void setReadIOps(Device device, long iops) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE), makeContext(device, iops));
    +    }
    +
    +    public Map<Device, Long> getReadIOps() throws IOException {
    +        return parseConfig(BLKIO_THROTTLE_READ_IOPS_DEVICE);
    +    }
    +
    +    public void setWriteIOps(Device device, long iops) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE), makeContext(device, iops));
    +    }
    +
    +    public Map<Device, Long> getWriteIOps() throws IOException {
    +        return parseConfig(BLKIO_THROTTLE_WRITE_IOPS_DEVICE);
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getThrottleIOServiced() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICED)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getThrottleIOServiceByte() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICE_BYTES)));
    +    }
    +
    +    public Map<Device, Long> getBlkioTime() throws IOException {
    +        return parseConfig(BLKIO_TIME);
    +    }
    +
    +    public Map<Device, Long> getBlkioSectors() throws IOException {
    +        return parseConfig(BLKIO_SECTORS);
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOServiced() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICED)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOServiceBytes() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICE_BYTES)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOServiceTime() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICE_TIME)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOWaitTime() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_WAIT_TIME)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOMerged() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_MERGED)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOQueued() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_QUEUED)));
    +    }
    +
    +    public void resetStats() throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_RESET_STATS), "1");
    +    }
    +
    +    private String makeContext(Device device, Object data) {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(device.toString()).append(" ").append(data);
    +        return sb.toString();
    +    }
    +
    +    private Map<Device, Long> parseConfig(String config) throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, config));
    +        Map<Device, Long> result = new HashMap<Device, Long>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Long value = Long.valueOf(strArgs[1]);
    +            result.put(device, value);
    +        }
    +        return result;
    +    }
    +
    +    private Map<Device, Map<RecordType, Long>> analyseRecord(List<String> strs) {
    +        Map<Device, Map<RecordType, Long>> result = new HashMap<Device, Map<RecordType, Long>>();
    +        for (String str : strs) {
    +            String[] strArgs = str.split(" ");
    +            if (strArgs.length != 3) {
    +                continue;
    +            }
    +            Device device = new Device(strArgs[0]);
    +            RecordType key = RecordType.getType(strArgs[1]);
    +            Long value = Long.parseLong(strArgs[2]);
    +            Map<RecordType, Long> record = result.get(device);
    +            if (record == null) {
    +                record = new HashMap<RecordType, Long>();
    +                result.put(device, record);
    +            }
    +            record.put(key, value);
    +        }
    +        return result;
    +    }
    +
    +    public enum RecordType {
    +        read, write, sync, async, total;
    +
    +        public static RecordType getType(String type) {
    +            if (type.equals("Read")) {
    --- End diff --
    
    (type == null) ? null : RecordType.valueOf(type.toLowerCase())


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-180571721
  
    The proposed changes are launching commands using `exec.command` which may not work in multi-tenant setup ( for killing workers etc.). It will not work in multi-tenant mode.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52111796
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1062,9 +1072,15 @@
                             (add-to-classpath [stormjar])
                             (add-to-classpath topo-classpath))
               top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
    -          mem-onheap (if (and mem-onheap (> mem-onheap 0)) ;; not nil and not zero
    -                       (int (Math/ceil mem-onheap)) ;; round up
    +          mem-onheap (if (and (.get_mem_on_heap resources) (> (.get_mem_on_heap resources) 0)) ;; not nil and not zero
    +                       (int (Math/ceil (.get_mem_on_heap resources))) ;; round up
                            (storm-conf WORKER-HEAP-MEMORY-MB)) ;; otherwise use default value
    +          mem-offheap (if (.get_mem_off_heap resources)
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52289421
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.container.cgroup;
    +
    +/**
    + * A enum class to described the subsystems that can be used
    + */
    +public enum SubSystemType {
    +
    +    // net_cls,ns is not supported in ubuntu
    +    blkio, cpu, cpuacct, cpuset, devices, freezer, memory, perf_event, net_cls, net_prio;
    +
    +    public static SubSystemType getSubSystem(String str) {
    --- End diff --
    
    SubSystemType.valueOf(str) will do the exact same 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52679696
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupOperation.java ---
    @@ -0,0 +1,78 @@
    +/**
    + * 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.container.cgroup;
    +
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * An interface to manage cgroups
    --- End diff --
    
    will elaborate


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52286889
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    --- End diff --
    
    put a check for number of rows that are returned. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52073012
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52065327
  
    --- Diff: storm-core/test/clj/org/apache/storm/supervisor_test.clj ---
    @@ -362,8 +361,8 @@
                                           mock-storm-id
                                           mock-port
                                           mock-worker-id
    -                                      mock-mem-onheap)
    -                (verify-first-call-args-for-indices launch-process
    +                                      (WorkerResources.))
    +            (verify-first-call-args-for-indices launch-process
    --- End diff --
    
    Wrong indentation.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52064293
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java ---
    @@ -0,0 +1,259 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.Constants;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.Device;
    +
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class BlkioCore implements CgroupCore {
    +
    +    public static final String BLKIO_WEIGHT = "/blkio.weight";
    +    public static final String BLKIO_WEIGHT_DEVICE = "/blkio.weight_device";
    +    public static final String BLKIO_RESET_STATS = "/blkio.reset_stats";
    +
    +    public static final String BLKIO_THROTTLE_READ_BPS_DEVICE = "/blkio.throttle.read_bps_device";
    +    public static final String BLKIO_THROTTLE_WRITE_BPS_DEVICE = "/blkio.throttle.write_bps_device";
    +    public static final String BLKIO_THROTTLE_READ_IOPS_DEVICE = "/blkio.throttle.read_iops_device";
    +    public static final String BLKIO_THROTTLE_WRITE_IOPS_DEVICE = "/blkio.throttle.write_iops_device";
    +
    +    public static final String BLKIO_THROTTLE_IO_SERVICED = "/blkio.throttle.io_serviced";
    +    public static final String BLKIO_THROTTLE_IO_SERVICE_BYTES = "/blkio.throttle.io_service_bytes";
    +
    +    public static final String BLKIO_TIME = "/blkio.time";
    +    public static final String BLKIO_SECTORS = "/blkio.sectors";
    +    public static final String BLKIO_IO_SERVICED = "/blkio.io_serviced";
    +    public static final String BLKIO_IO_SERVICE_BYTES = "/blkio.io_service_bytes";
    +    public static final String BLKIO_IO_SERVICE_TIME = "/blkio.io_service_time";
    +    public static final String BLKIO_IO_WAIT_TIME = "/blkio.io_wait_time";
    +    public static final String BLKIO_IO_MERGED = "/blkio.io_merged";
    +    public static final String BLKIO_IO_QUEUED = "/blkio.io_queued";
    +
    +    private final String dir;
    +
    +    public BlkioCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.blkio;
    +    }
    +
    +    /* weight: 100-1000 */
    +    public void setBlkioWeight(int weight) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, BLKIO_WEIGHT), String.valueOf(weight));
    +    }
    +
    +    public int getBlkioWeight() throws IOException {
    +        return Integer.valueOf(CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_WEIGHT)).get(0)).intValue();
    +    }
    +
    +    public void setBlkioWeightDevice(Device device, int weight) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, BLKIO_WEIGHT_DEVICE), makeContext(device, weight));
    +    }
    +
    +    public Map<Device, Integer> getBlkioWeightDevice() throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_WEIGHT_DEVICE));
    +        Map<Device, Integer> result = new HashMap<Device, Integer>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Integer weight = Integer.valueOf(strArgs[1]);
    +            result.put(device, weight);
    +        }
    +        return result;
    +    }
    +
    +    public void setReadBps(Device device, long bps) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_READ_BPS_DEVICE), makeContext(device, bps));
    +    }
    +
    +    public Map<Device, Long> getReadBps() throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_READ_BPS_DEVICE));
    +        Map<Device, Long> result = new HashMap<Device, Long>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Long bps = Long.valueOf(strArgs[1]);
    +            result.put(device, bps);
    +        }
    +        return result;
    +    }
    +
    +    public void setWriteBps(Device device, long bps) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_WRITE_BPS_DEVICE), makeContext(device, bps));
    +    }
    +
    +    public Map<Device, Long> getWriteBps() throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_WRITE_BPS_DEVICE));
    +        Map<Device, Long> result = new HashMap<Device, Long>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Long bps = Long.valueOf(strArgs[1]);
    +            result.put(device, bps);
    +        }
    +        return result;
    +    }
    +
    +    public void setReadIOps(Device device, long iops) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE), makeContext(device, iops));
    +    }
    +
    +    public Map<Device, Long> getReadIOps() throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE));
    +        Map<Device, Long> result = new HashMap<Device, Long>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Long iops = Long.valueOf(strArgs[1]);
    +            result.put(device, iops);
    +        }
    +        return result;
    +    }
    +
    +    public void setWriteIOps(Device device, long iops) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE), makeContext(device, iops));
    +    }
    +
    +    public Map<Device, Long> getWriteIOps() throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE));
    +        Map<Device, Long> result = new HashMap<Device, Long>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Long iops = Long.valueOf(strArgs[1]);
    +            result.put(device, iops);
    +        }
    +        return result;
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getThrottleIOServiced() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICED)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getThrottleIOServiceByte() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICE_BYTES)));
    +    }
    +
    +    public Map<Device, Long> getBlkioTime() throws IOException {
    +        Map<Device, Long> result = new HashMap<Device, Long>();
    +        List<String> strs = CgroupUtils.readFileByLine(Constants.getDir(this.dir, BLKIO_TIME));
    +        for (String str : strs) {
    +            String[] strArgs = str.split(" ");
    +            result.put(new Device(strArgs[0]), Long.parseLong(strArgs[1]));
    +        }
    +        return result;
    +    }
    +
    +    public Map<Device, Long> getBlkioSectors() throws IOException {
    --- End diff --
    
    More D.R.Y.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52782270
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/CpusetCore.java ---
    @@ -0,0 +1,209 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +
    +public class CpusetCore implements CgroupCore {
    +
    +    public static final String CPUSET_CPUS = "/cpuset.cpus";
    +    public static final String CPUSET_MEMS = "/cpuset.mems";
    +    public static final String CPUSET_MEMORY_MIGRATE = "/cpuset.memory_migrate";
    +    public static final String CPUSET_CPU_EXCLUSIVE = "/cpuset.cpu_exclusive";
    +    public static final String CPUSET_MEM_EXCLUSIVE = "/cpuset.mem_exclusive";
    +    public static final String CPUSET_MEM_HARDWALL = "/cpuset.mem_hardwall";
    +    public static final String CPUSET_MEMORY_PRESSURE = "/cpuset.memory_pressure";
    +    public static final String CPUSET_MEMORY_PRESSURE_ENABLED = "/cpuset.memory_pressure_enabled";
    +    public static final String CPUSET_MEMORY_SPREAD_PAGE = "/cpuset.memory_spread_page";
    +    public static final String CPUSET_MEMORY_SPREAD_SLAB = "/cpuset.memory_spread_slab";
    +    public static final String CPUSET_SCHED_LOAD_BALANCE = "/cpuset.sched_load_balance";
    +    public static final String CPUSET_SCHED_RELAX_DOMAIN_LEVEL = "/cpuset.sched_relax_domain_level";
    +
    +    private final String dir;
    +
    +    public CpusetCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.cpuset;
    +    }
    +
    +    public void setCpus(int[] nums) throws IOException {
    +        setConfigs(nums, CPUSET_CPUS);
    +    }
    +
    +    public int[] getCpus() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPUS)).get(0);
    +        return parseNums(output);
    +    }
    +
    +    public void setMems(int[] nums) throws IOException {
    +        setConfigs(nums, CPUSET_MEMS);
    +    }
    +
    +    private void setConfigs(int[] nums, String config) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        for (int num : nums) {
    +            sb.append(num);
    +            sb.append(',');
    +        }
    +        sb.deleteCharAt(sb.length() - 1);
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, config), sb.toString());
    +    }
    +
    +    public int[] getMems() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMS)).get(0);
    +        return parseNums(output);
    +    }
    +
    +    public void setMemMigrate(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_MIGRATE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemMigrate() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_MIGRATE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setCpuExclusive(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPU_EXCLUSIVE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isCpuExclusive() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_CPU_EXCLUSIVE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemExclusive(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_EXCLUSIVE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemExclusive() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_EXCLUSIVE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemHardwall(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_HARDWALL), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemHardwall() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEM_HARDWALL)).get(0));
    +        return output > 0;
    +    }
    +
    +    public int getMemPressure() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE)).get(0);
    +        return Integer.parseInt(output);
    +    }
    +
    +    public void setMemPressureEnabled(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemPressureEnabled() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemSpreadPage(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemSpreadPage() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setMemSpreadSlab(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isMemSpreadSlab() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setSchedLoadBlance(boolean flag) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE), String.valueOf(flag ? 1 : 0));
    +    }
    +
    +    public boolean isSchedLoadBlance() throws IOException {
    +        int output = Integer.parseInt(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE)).get(0));
    +        return output > 0;
    +    }
    +
    +    public void setSchedRelaxDomainLevel(int value) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL), String.valueOf(value));
    +    }
    +
    +    public int getSchedRelaxDomainLevel() throws IOException {
    +        String output = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL)).get(0);
    +        return Integer.parseInt(output);
    +    }
    +
    +    public static int[] parseNums(String outputStr) {
    +        char[] output = outputStr.toCharArray();
    +        LinkedList<Integer> numList = new LinkedList<Integer>();
    +        int value = 0;
    +        int start = 0;
    +        boolean isHyphen = false;
    +        for (char ch : output) {
    --- End diff --
    
    I agree this is not the most straightforward way of doing this in Java and
    looks more C-like. 
    
    I think I would be OK with leaving the code as-is, and maybe we could add a
    comment explaining what this does for others since it is? We could clean it up
    in follow-on work if it still is a problem.
    
    ```
    /** 
     * Accepts a string of comma-separated values or ranges of values.  Specify a
     * value like '42' and a range like '12-17'.
     *
     * Example: '42,12-17,100' -> {42, 12, 13, 14, 15, 16, 17, 100}
     */
    ```


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52420950
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +/**
    + * A class that implements system operations for using cgroups
    + */
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    }
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ")
    +                .append(type)
    +                .append(" -o ")
    +                .append(data)
    +                .append(" ")
    +                .append(name)
    +                .append(" ")
    +                .append(target);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static void umount(String name) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("umount ").append(name);
    --- End diff --
    
    will rename


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51790816
  
    --- Diff: storm-core/test/jvm/org/apache/storm/TestCGroup.java ---
    @@ -0,0 +1,118 @@
    +/**
    + * 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;
    +
    +import org.junit.Assert;
    +import org.junit.Assume;
    +import org.apache.storm.container.cgroup.CgroupManager;
    +import org.apache.storm.utils.Utils;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.nio.file.Files;
    +import java.nio.file.Paths;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.UUID;
    +
    +/**
    + * Unit tests for CGroups
    + */
    +public class TestCGroup {
    +
    +    /**
    +     * Test whether cgroups are setup up correctly for use.  Also tests whether Cgroups produces the right command to
    +     * start a worker and cleans up correctly after the worker is shutdown
    +     */
    +    @Test
    +    public void testSetupAndTearDown() throws IOException {
    +        Config config = new Config();
    +        config.putAll(Utils.readDefaultConfig());
    +        //We don't want to run the test is CGroups are not setup
    +        Assume.assumeTrue("Check if CGroups are setup", ((boolean) config.get(Config.STORM_CGROUP_ENABLE)) == true);
    +
    +        Assert.assertTrue("Check if STORM_CGROUP_HIERARCHY_DIR exists", stormCgroupHierarchyExists(config));
    +        Assert.assertTrue("Check if STORM_SUPERVISOR_CGROUP_ROOTDIR exists", stormCgroupSupervisorRootDirExists(config));
    +
    +        CgroupManager manager = new CgroupManager();
    +        manager.prepare(config);
    +
    +        Map<String, Object> resourcesMap = new HashMap<String, Object>();
    +        resourcesMap.put("cpu", 200);
    +        resourcesMap.put("memory", 1024);
    +        String workerId = UUID.randomUUID().toString();
    +        String command = manager.startNewWorker(workerId, resourcesMap);
    +
    +        String correctCommand1 = config.get(Config.STORM_CGROUP_CGEXEC_CMD) + " -g memory,cpu:/"
    +                + config.get(Config.STORM_SUPERVISOR_CGROUP_ROOTDIR) + "/" + workerId;
    +        String correctCommand2 = config.get(Config.STORM_CGROUP_CGEXEC_CMD) + " -g cpu,memory:/"
    +                + config.get(Config.STORM_SUPERVISOR_CGROUP_ROOTDIR) + "/" + workerId;
    +        Assert.assertTrue("Check if cgroup launch command is correct", command.equals(correctCommand1) || command.equals(correctCommand2));
    +
    +        String pathToWorkerCgroupDir = ((String) config.get(Config.STORM_CGROUP_HIERARCHY_DIR))
    +                + "/" + ((String) config.get(Config.STORM_SUPERVISOR_CGROUP_ROOTDIR)) + "/" + workerId;
    +
    +        Assert.assertTrue("Check if cgroup directory exists for worker", dirExists(pathToWorkerCgroupDir));
    +
    +        /* validate cpu settings */
    +
    +        String pathToCpuShares = pathToWorkerCgroupDir + "/cpu.shares";
    +        Assert.assertTrue("Check if cpu.shares file exists", fileExists(pathToCpuShares));
    +        Assert.assertEquals("Check if the correct value is written into cpu.shares", "200", readFileAll(pathToCpuShares));
    +
    +        /* validate memory settings */
    +
    +        String pathTomemoryLimitInBytes = pathToWorkerCgroupDir + "/memory.limit_in_bytes";
    +
    +        Assert.assertTrue("Check if memory.limit_in_bytes file exists", fileExists(pathTomemoryLimitInBytes));
    +        Assert.assertEquals("Check if the correct value is written into memory.limit_in_bytes", String.valueOf(1024 * 1024 * 1024), readFileAll(pathTomemoryLimitInBytes));
    +
    +        manager.shutDownWorker(workerId, true);
    +
    +        Assert.assertFalse("Make sure cgroup was removed properly", dirExists(pathToWorkerCgroupDir));
    +    }
    +
    +    private boolean stormCgroupHierarchyExists(Map config) {
    +        String pathToStormCgroupHierarchy = (String) config.get(Config.STORM_CGROUP_HIERARCHY_DIR);
    +        return dirExists(pathToStormCgroupHierarchy);
    +    }
    +
    +    private boolean stormCgroupSupervisorRootDirExists(Map config) {
    +        String pathTostormCgroupSupervisorRootDir = ((String) config.get(Config.STORM_CGROUP_HIERARCHY_DIR))
    +                + "/" + ((String) config.get(Config.STORM_SUPERVISOR_CGROUP_ROOTDIR));
    +
    +        return dirExists(pathTostormCgroupSupervisorRootDir);
    +    }
    +
    +    private boolean dirExists(String rawPath) {
    +        File path = new File(rawPath);
    +        return path.exists() && path.isDirectory();
    +    }
    +
    +    private boolean fileExists(String rawPath) {
    +        File path = new File(rawPath);
    +        return path.exists() && !path.isDirectory();
    +    }
    +
    +    private String readFileAll(String filePath) throws IOException {
    +        byte[] data = Files.readAllBytes(Paths.get(filePath));
    +        return new String(data).trim();
    +    }
    +}
    --- End diff --
    
    new line at the end of 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51139248
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,174 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        try {
    +            String cmd = "rmdir " + dir;
    +            SystemOperation.exec(cmd);
    --- End diff --
    
    I know that this came from JStorm, but is there a reason we should not just do 
    ```
    File d = new File(dir);
    if (d.exists()) {
        if (d.isDirectory()) {
            d.delete(); //TODO need to check results
        } else {
            throw some error;
        }
    } 
    ```
    
    The current code is not safe, it opens up the potential for command injection.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52777206
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java ---
    @@ -0,0 +1,117 @@
    +/**
    + * 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.container.cgroup;
    +
    +import java.util.Set;
    +
    +public class Hierarchy {
    +
    +    private final String name;
    +
    +    private final Set<SubSystemType> subSystems;
    +
    +    private final String type;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon rootCgroups;
    +
    +    public Hierarchy(String name, Set<SubSystemType> subSystems, String dir) {
    +        this.name = name;
    +        this.subSystems = subSystems;
    +        this.dir = dir;
    +        this.rootCgroups = new CgroupCommon(this, dir);
    +        this.type = CgroupUtils.reAnalyse(subSystems);
    +    }
    +
    +    public Set<SubSystemType> getSubSystems() {
    +        return subSystems;
    +    }
    +
    +    public String getType() {
    +        return type;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        final int prime = 31;
    +        int result = 1;
    +        result = prime * result + ((dir == null) ? 0 : dir.hashCode());
    +        result = prime * result + ((name == null) ? 0 : name.hashCode());
    +        result = prime * result + ((type == null) ? 0 : type.hashCode());
    +        return result;
    +    }
    --- End diff --
    
    I see, so since one is generated from the other, we do not need to check both.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52422886
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    --- End diff --
    
    The cgroup config file is not actually mounted to a filesystem thus a write to it is not written to disk necessary.  The config files serve as a way to pass in parameters into cgroups like linux kernel modules sometimes do.  For example:
    
    $ cat cpu.shares
    75
    $ echo "80" >> cpu.shares
    $ cat cpu.shares
    80
    $ echo "75" > cpu.shares
    $ cat cpu.shares
    75


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52072010
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -344,6 +348,12 @@
        :sync-retry (atom 0)
        :download-lock (Object.)
        :stormid->profiler-actions (atom {})
    +   :cgroup-manager (if (conf STORM-CGROUP-ENABLE)
    +                     (let [cgroup-manager (.newInstance (Class/forName (conf STORM-RESOURCE-ISOLATION-PLUGIN)))]
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51289205
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    +        this.center.create(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    --- End diff --
    
    We can file a follow on JIRA to add a hard CPU limit on a per worker basis.  Currently, the code does support setting a upper hard limit to how many cores/cpu resources all of the workers on a node can use (e.g. if a node has 24 cores but you want workers to use only 12 of the cores, you can).


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52284079
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    --- End diff --
    
    would be better to use a consistent patter e.g. cgroup_procs


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52415563
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +/**
    + * A class that implements system operations for using cgroups
    + */
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    }
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ")
    +                .append(type)
    +                .append(" -o ")
    +                .append(data)
    --- End diff --
    
    variable data can be renamed to options. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by zhuoliu <gi...@git.apache.org>.
Github user zhuoliu commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-184812340
  
    Looks good! +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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52046682
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    --- End diff --
    
    This comment is useless. Please clarify or 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52341401
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!getCores().keySet().contains(SubSystemType.cpuset)) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getCgroupCloneChildren() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(eventFd);
    +        sb.append(' ');
    +        sb.append(controlFd);
    +        for (String arg : args) {
    +            sb.append(' ');
    +            sb.append(arg);
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_EVENT_CONTROL), sb.toString());
    +    }
    +
    +    public Hierarchy getHierarchy() {
    +        return hierarchy;
    +    }
    +
    +    public String getName() {
    +        return name;
    +    }
    +
    +    public String getDir() {
    +        return dir;
    +    }
    +
    +    public CgroupCommon getParent() {
    +        return parent;
    +    }
    +
    +    public Set<CgroupCommon> getChildren() {
    +
    +        File file = new File(this.dir);
    +        File[] files = file.listFiles();
    +        if (files == null) {
    --- End diff --
    
    please look the function definition for listFiles().  The function can return 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52053303
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    +        File file = new File(dir);
    +        return file.exists();
    +    }
    +
    +    public static boolean dirExists(String dir) {
    +        File file = new File(dir);
    +        return file.isDirectory();
    +    }
    +
    +    public static Set<SubSystemType> analyse(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static String reAnalyse(Set<SubSystemType> subSystems) {
    --- End diff --
    
    Why is this called `reAnalyse`? What does that mean?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52111843
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1062,9 +1072,15 @@
                             (add-to-classpath [stormjar])
                             (add-to-classpath topo-classpath))
               top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
    -          mem-onheap (if (and mem-onheap (> mem-onheap 0)) ;; not nil and not zero
    -                       (int (Math/ceil mem-onheap)) ;; round up
    +          mem-onheap (if (and (.get_mem_on_heap resources) (> (.get_mem_on_heap resources) 0)) ;; not nil and not zero
    +                       (int (Math/ceil (.get_mem_on_heap resources))) ;; round up
                            (storm-conf WORKER-HEAP-MEMORY-MB)) ;; otherwise use default value
    +          mem-offheap (if (.get_mem_off_heap resources)
    --- End diff --
    
    we actually don't need to check any of the resource configs since the configs are validated in StormSubmitter when the topology is submitted


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52098093
  
    --- Diff: storm-core/test/clj/org/apache/storm/supervisor_test.clj ---
    @@ -335,8 +334,8 @@
                                           mock-storm-id
                                           mock-port
                                           mock-worker-id
    -                                      mock-mem-onheap)
    -                (verify-first-call-args-for-indices launch-process
    +                                      (WorkerResources.))
    +            (verify-first-call-args-for-indices launch-process
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52124783
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            for (SubSystemType type : hierarchy.getSubSystems()) {
    +                if (type == subsystem) {
    +                    return hierarchy;
    +                }
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subsystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subsystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    --- End diff --
    
    will rename


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52341020
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!getCores().keySet().contains(SubSystemType.cpuset)) {
    --- End diff --
    
    because its a config for only the subsystem cpuset


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51784920
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,225 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.init();
    +        cores = CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.init();
    +        cores = CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(Constants.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(Constants.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!this.cores.keySet().contains(SubSystemType.cpuset)) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getCgroupCloneChildren() throws IOException {
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, CGROUP_CLONE_CHILDREN)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(eventFd);
    +        sb.append(' ');
    +        sb.append(controlFd);
    +        for (String arg : args) {
    +            sb.append(' ');
    +            sb.append(arg);
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_EVENT_CONTROL), sb.toString());
    +    }
    +
    +    public Hierarchy getHierarchy() {
    +        return hierarchy;
    +    }
    +
    +    public String getName() {
    +        return name;
    +    }
    +
    +    public String getDir() {
    +        return dir;
    +    }
    +
    +    public CgroupCommon getParent() {
    +        return parent;
    +    }
    +
    +    public Set<CgroupCommon> getChildren() {
    +        return children;
    +    }
    +
    +    public boolean isRoot() {
    +        return isRoot;
    +    }
    +
    +    public Map<SubSystemType, CgroupCore> getCores() {
    +        return cores;
    +    }
    +
    +    public void delete() throws IOException {
    +        this.free();
    +        if (!this.isRoot) {
    +            this.parent.getChildren().remove(this);
    +        }
    +    }
    +
    +    private void free() throws IOException {
    +        for (CgroupCommon child : this.children)
    --- End diff --
    
    no paranthesis


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51168995
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    --- End diff --
    
    will change to Cgroup


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51145886
  
    --- Diff: storm-core/test/jvm/org/apache/storm/TestCGroup.java ---
    @@ -0,0 +1,94 @@
    +package org.apache.storm;
    --- End diff --
    
    Needs Apache 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52340405
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    --- End diff --
    
    not necessary, see previous 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51168186
  
    --- Diff: conf/defaults.yaml ---
    @@ -281,3 +281,16 @@ pacemaker.thread.timeout: 10
     pacemaker.childopts: "-Xmx1024m"
     pacemaker.auth.method: "NONE"
     pacemaker.kerberos.users: []
    +
    +storm.resource.isolation.plugin: "org.apache.storm.container.cgroup.CgroupManager"
    +
    +# Configs for CGroup support
    +cgroup.storm.hierarchy.dir: "/cgroup/storm_resources"
    +cgroup.storm.resources:
    +    - cpu
    +    - memory
    +cgroup.storm.hierarchy.name: "storm"
    +# Also determines whether the unit tests for cgroup runs.  If cgroup.enable is set to false the unit tests for cgroups will not run
    +cgroup.enable: false
    +cgroup.supervisor.rootdir: "storm"
    +cgroup.cgexec.cmd: "/bin/cgexec"
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52287512
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!getCores().keySet().contains(SubSystemType.cpuset)) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getCgroupCloneChildren() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(eventFd);
    +        sb.append(' ');
    +        sb.append(controlFd);
    +        for (String arg : args) {
    +            sb.append(' ');
    +            sb.append(arg);
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_EVENT_CONTROL), sb.toString());
    +    }
    +
    +    public Hierarchy getHierarchy() {
    +        return hierarchy;
    +    }
    +
    +    public String getName() {
    +        return name;
    +    }
    +
    +    public String getDir() {
    +        return dir;
    +    }
    +
    +    public CgroupCommon getParent() {
    +        return parent;
    +    }
    +
    +    public Set<CgroupCommon> getChildren() {
    +
    +        File file = new File(this.dir);
    +        File[] files = file.listFiles();
    +        if (files == null) {
    +            return null;
    +        }
    +        Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +        for (File child : files) {
    +            if (child.isDirectory()) {
    +                children.add(new CgroupCommon(child.getName(), this.hierarchy, this));
    +            }
    +        }
    +        return children;
    +    }
    +
    +    public boolean isRoot() {
    +        return isRoot;
    +    }
    +
    +    public Map<SubSystemType, CgroupCore> getCores() {
    +        return CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
    +    }
    +
    +    public void delete() throws IOException {
    +        this.free();
    +        if (!this.isRoot) {
    +            this.parent.getChildren().remove(this);
    +        }
    +    }
    +
    +    private void free() throws IOException {
    +        for (CgroupCommon child : getChildren()) {
    +            child.free();
    +        }
    +        if (this.isRoot) {
    +            return;
    +        }
    +        Set<Integer> tasks = this.getTasks();
    +        if (tasks != null) {
    +            for (Integer task : tasks) {
    +                this.parent.addTask(task);
    +            }
    +        }
    +        CgroupUtils.deleteDir(this.dir);
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        boolean ret = false;
    +        if (o != null && (o instanceof CgroupCommon)) {
    +            ret = (((CgroupCommon)o).hierarchy.equals(this.hierarchy)
    +                    && ((CgroupCommon)o).name.equals(this.name))
    +                    && ((CgroupCommon)o).dir.equals(this.dir);
    +
    +        }
    +        return ret;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        final int prime = 31;
    +        int result = 1;
    +        result = prime * result + (this.name != null ? this.name.hashCode() : 0);
    --- End diff --
    
    you can use lombok plugin to generate these methods. The null checks you are doing in hashCode, should be done in equals method 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52082237
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/Constants.java ---
    @@ -0,0 +1,30 @@
    +/**
    + * 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.container.cgroup;
    +
    +public class Constants {
    --- End diff --
    
    will move 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52069182
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/ResourceIsolationInterface.java ---
    @@ -0,0 +1,43 @@
    +/**
    + * 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.container;
    +
    +import java.util.Map;
    +
    +/**
    + * A plugin to support resource isolation and limitation within Storm
    + */
    +public interface ResourceIsolationInterface {
    +
    +    /**
    +     * @param workerId worker id of the worker to start
    +     * @param resources set of resources to limit
    +     * @return a String that includes to command on how to start the worker.  The string returned from this function
    +     * will be concatenated to the front of the command to launch logwriter/worker in supervisor.clj
    +     */
    --- End diff --
    
    minor: `The string returned`... Maybe we should describe that the string is a series of command line options for launching a JVM. It seems strange to document what the caller does with the string.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52050562
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    --- End diff --
    
    `subSystem` vs `subsystem` is confusing. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51178274
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null)
    +            instance = new CgroupCenter();
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup"))
    +                    continue;
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null)
    +                    continue;
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]), Integer.valueOf(split[3]).intValue() == 1 ? true
    +                        : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error ", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem)
    +                return true;
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            for (SubSystemType type : hierarchy.getSubSystems()) {
    +                if (type == subsystem)
    +                    return hierarchy;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subsystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subsystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy mounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (CgroupUtils.dirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy))
    +                    return h;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.mounted(hierarchy) != null) {
    +            LOG.error(hierarchy.getDir() + " is mounted");
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52647214
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,210 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The storm.supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{} is not existing.", file.getPath());
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * initalize subsystems
    +     */
    +    private void prepareSubSystem(Map conf) throws IOException {
    +        List<SubSystemType> subSystemTypes = new LinkedList<>();
    +        for (String resource : Config.getCgroupStormResources(conf)) {
    +            subSystemTypes.add(SubSystemType.getSubSystem(resource));
    +        }
    +
    +        this.hierarchy = center.getHierarchyWithSubSystems(subSystemTypes);
    +
    +        if (this.hierarchy == null) {
    +            Set<SubSystemType> types = new HashSet<SubSystemType>();
    +            types.add(SubSystemType.cpu);
    +            this.hierarchy = new Hierarchy(Config.getCgroupStormHierarchyName(conf), types, Config.getCgroupStormHierarchyDir(conf));
    +        }
    +        this.rootCgroup = new CgroupCommon(this.rootDir, this.hierarchy, this.hierarchy.getRootCgroups());
    +
    +        // set upper limit to how much cpu can be used by all workers running on supervisor node.
    +        // This is done so that some cpu cycles will remain free to run the daemons and other miscellaneous OS operations.
    +        CpuCore supervisorRootCPU = (CpuCore) this.rootCgroup.getCores().get(SubSystemType.cpu);
    +        setCpuUsageUpperLimit(supervisorRootCPU, ((Number) this.conf.get(Config.SUPERVISOR_CPU_CAPACITY)).intValue());
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g.
    +     * If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    --- End diff --
    
    Is this supposed to be a number literal?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52767645
  
    --- Diff: conf/defaults.yaml ---
    @@ -263,7 +263,7 @@ topology.state.checkpoint.interval.ms: 1000
     # topology priority describing the importance of the topology in decreasing importance starting from 0 (i.e. 0 is the highest priority and the priority importance decreases as the priority number increases).
     # Recommended range of 0-29 but no hard limit set.
     topology.priority: 29
    -topology.component.resources.onheap.memory.mb: 128.0
    +topology.component.resources.onheap.memory.mb: 256.0
    --- End diff --
    
    First of all RAS should be keeping scheduling ackers just like any other bolt.  The system will have some overhead, and so will the JVM itself. If we assume that the JVM will only ever use the amount of memory specified in the HEAP we are going to be in trouble when someone starts to use their entire heap.
    
    I would say initially that we want a two configs that are for worker system overhead, both on heap and off heap.  Then we reserve that many MBs * number_of_slots from being used for scheduling.  Increasing the default does not solve this problem, except in the case where we are not using all of the heap. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52421574
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/DevicesCore.java ---
    @@ -0,0 +1,185 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.Device;
    +
    +import java.io.IOException;
    +import java.util.List;
    +
    +public class DevicesCore implements CgroupCore {
    +
    +    private final String dir;
    +
    +    private static final String DEVICES_ALLOW = "/devices.allow";
    +    private static final String DEVICES_DENY = "/devices.deny";
    +    private static final String DEVICES_LIST = "/devices.list";
    +
    +    private static final char TYPE_ALL = 'a';
    +    private static final char TYPE_BLOCK = 'b';
    +    private static final char TYPE_CHAR = 'c';
    +
    +    private static final int ACCESS_READ = 1;
    +    private static final int ACCESS_WRITE = 2;
    +    private static final int ACCESS_CREATE = 4;
    +
    +    private static final char ACCESS_READ_CH = 'r';
    +    private static final char ACCESS_WRITE_CH = 'w';
    +    private static final char ACCESS_CREATE_CH = 'm';
    +
    +    public DevicesCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.devices;
    +    }
    +
    +    public static class Record {
    +        Device device;
    +        char type;
    +        int accesses;
    +
    +        public Record(char type, Device device, int accesses) {
    +            this.type = type;
    +            this.device = device;
    +            this.accesses = accesses;
    +        }
    +
    +        public Record(String output) {
    +            if (output.contains("*")) {
    +                System.out.println("Pre:" + output);
    --- End diff --
    
    will replace


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52071474
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1088,8 +1104,26 @@
                                                 (str "file:///" storm-log4j2-conf-dir))
                                               storm-log4j2-conf-dir)
                                          file-path-separator "worker.xml")
    +
    +          cgroup-command (if (conf STORM-CGROUP-ENABLE)
    +                           (str/split
    +                             (.startNewWorker (:cgroup-manager supervisor) worker-id
    +                               (merge
    +                                 ;; The manually set CGROUP-WORKER-CPU-LIMIT config on supervisor will overwrite resources assigned by RAS (Resource Aware Scheduler)
    --- End diff --
    
    Check comment; seems it should refer to memory.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51917161
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    --- End diff --
    
    since users can submit a value representing the amount of resources as an integer or double from RAS


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51916794
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    --- End diff --
    
    will rename


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51786262
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    --- End diff --
    
    why are we using a Number, are we not sure of its type


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#issuecomment-184985755
  
    +1 looks good. Thank you.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51134752
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null)
    +            instance = new CgroupCenter();
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup"))
    +                    continue;
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error", e);
    --- End diff --
    
    I realize a lot of this code came directly from JStorm, but I really would prefer to not eat exceptions.  If CGroups is supposed to be enabled I would rather have the supervisor crash if there is an issue running instead of disabling it and outputting an error message.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52076438
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    +        File file = new File(dir);
    +        return file.exists();
    +    }
    +
    +    public static boolean dirExists(String dir) {
    +        File file = new File(dir);
    +        return file.isDirectory();
    +    }
    +
    +    public static Set<SubSystemType> analyse(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static String reAnalyse(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return CgroupUtils.fileExists(Constants.CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    +        List<String> result = new ArrayList<String>();
    +        File file = new File(fileDir);
    +        try (FileReader fileReader = new FileReader(file);
    +             BufferedReader reader = new BufferedReader(fileReader)) {
    +            String tempString = null;
    +            while ((tempString = reader.readLine()) != null) {
    +                result.add(tempString);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static void writeFileByLine(String fileDir, List<String> strings) throws IOException {
    +        File file = new File(fileDir);
    +        if (!file.exists()) {
    +            LOG.error("{} is no existed", fileDir);
    +            return;
    +        }
    +        try (FileWriter writer = new FileWriter(file, true);
    +             BufferedWriter bw = new BufferedWriter(writer)) {
    +            for (String string : strings) {
    +                bw.write(string);
    +                bw.newLine();
    +                bw.flush();
    +            }
    +        }
    +    }
    +
    +    public static void writeFileByLine(String fileDir, String string) throws IOException {
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51916552
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,225 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.init();
    +        cores = CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.init();
    +        cores = CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(Constants.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(Constants.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!this.cores.keySet().contains(SubSystemType.cpuset)) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getCgroupCloneChildren() throws IOException {
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, CGROUP_CLONE_CHILDREN)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(eventFd);
    +        sb.append(' ');
    +        sb.append(controlFd);
    +        for (String arg : args) {
    +            sb.append(' ');
    +            sb.append(arg);
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_EVENT_CONTROL), sb.toString());
    +    }
    +
    +    public Hierarchy getHierarchy() {
    +        return hierarchy;
    +    }
    +
    +    public String getName() {
    +        return name;
    +    }
    +
    +    public String getDir() {
    +        return dir;
    +    }
    +
    +    public CgroupCommon getParent() {
    +        return parent;
    +    }
    +
    +    public Set<CgroupCommon> getChildren() {
    +        return children;
    +    }
    +
    +    public boolean isRoot() {
    +        return isRoot;
    +    }
    +
    +    public Map<SubSystemType, CgroupCore> getCores() {
    +        return cores;
    +    }
    +
    +    public void delete() throws IOException {
    +        this.free();
    +        if (!this.isRoot) {
    +            this.parent.getChildren().remove(this);
    +        }
    +    }
    +
    +    private void free() throws IOException {
    +        for (CgroupCommon child : this.children)
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52061674
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    };
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ").append(type).append(" -o ").append(data).append(" ").append(name).append(" ").append(target);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static void umount(String name) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("umount ").append(name);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static String exec(String cmd) throws IOException {
    +        LOG.debug("Shell cmd: {}", cmd);
    +        Process process = new ProcessBuilder(new String[] { "/bin/bash", "-c", cmd }).start();
    +        try {
    +            process.waitFor();
    +            String output = IOUtils.toString(process.getInputStream());
    +            String errorOutput = IOUtils.toString(process.getErrorStream());
    +            LOG.debug("Shell Output: {}", output);
    +            if (errorOutput.length() != 0) {
    +                LOG.error("Shell Error Output: {}", errorOutput);
    +                throw new IOException(errorOutput);
    +            }
    +            return output;
    +        } catch (InterruptedException ie) {
    +            throw new IOException(ie.toString());
    +        }
    +    }
    +}
    --- End diff --
    
    newline


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52336431
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,128 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    /**
    +     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
    +     */
    +    public static Set<SubSystemType> getSubSystemsFromString(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    /**
    +     * Get a string that is a comma delimited list of subsystems
    +     */
    +    public static String subSystemsToString(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return Utils.checkFileExists(CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    +        List<String> result = new ArrayList<String>();
    +        File file = new File(fileDir);
    +        try (FileReader fileReader = new FileReader(file);
    +             BufferedReader reader = new BufferedReader(fileReader)) {
    +            String tempString = null;
    +            while ((tempString = reader.readLine()) != null) {
    +                result.add(tempString);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static void writeFileByLine(String fileDir, List<String> strings) throws IOException {
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52278203
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    --- End diff --
    
    There is new line such as this in other functions as well. It can be removed. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51140226
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,174 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        try {
    +            String cmd = "rmdir " + dir;
    +            SystemOperation.exec(cmd);
    +        } catch (IOException e) {
    +            // TODO Auto-generated catch block
    +            LOG.error("rm " + dir + " fail!", e);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    +        File file = new File(dir);
    +        return file.exists();
    +    }
    +
    +    public static boolean dirExists(String dir) {
    +        File file = new File(dir);
    +        return file.isDirectory();
    +    }
    +
    +    public static Set<SubSystemType> analyse(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null)
    +                result.add(type);
    +        }
    +        return result;
    +    }
    +
    +    public static String reAnalyse(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0)
    +            return sb.toString();
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return CgroupUtils.fileExists(Constants.CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    +        List<String> result = new ArrayList<String>();
    +        FileReader fileReader = null;
    +        BufferedReader reader = null;
    +        try {
    +            File file = new File(fileDir);
    +            fileReader = new FileReader(file);
    +            reader = new BufferedReader(fileReader);
    +            String tempString = null;
    +            while ((tempString = reader.readLine()) != null) {
    +                result.add(tempString);
    +            }
    +        } finally {
    +            CgroupUtils.close(fileReader, reader);
    +        }
    +        return result;
    +    }
    +
    +    public static void writeFileByLine(String fileDir, List<String> strings) throws IOException {
    +        FileWriter writer = null;
    +        BufferedWriter bw = null;
    +        try {
    +            File file = new File(fileDir);
    +            if (!file.exists()) {
    +                LOG.error(fileDir + " is no existed");
    +                return;
    +            }
    +            writer = new FileWriter(file, true);
    +            bw = new BufferedWriter(writer);
    +            for (String string : strings) {
    +                bw.write(string);
    +                bw.newLine();
    +                bw.flush();
    +            }
    +        } finally {
    +            CgroupUtils.close(writer, bw);
    +        }
    +    }
    +
    +    public static void writeFileByLine(String fileDir, String string) throws IOException {
    +        LOG.info("For CGroups - writing {} to {} ", string, fileDir);
    --- End diff --
    
    Can we make this 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52277354
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    --- End diff --
    
    should there be a check on length of strSplit 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52286842
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    --- End diff --
    
    this operation should override the existing file, isn't 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52355251
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SubSystemType.java ---
    @@ -0,0 +1,52 @@
    +/**
    + * 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.container.cgroup;
    +
    +/**
    + * A enum class to described the subsystems that can be used
    + */
    +public enum SubSystemType {
    +
    +    // net_cls,ns is not supported in ubuntu
    +    blkio, cpu, cpuacct, cpuset, devices, freezer, memory, perf_event, net_cls, net_prio;
    +
    +    public static SubSystemType getSubSystem(String str) {
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52417035
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/DevicesCore.java ---
    @@ -0,0 +1,185 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.Device;
    +
    +import java.io.IOException;
    +import java.util.List;
    +
    +public class DevicesCore implements CgroupCore {
    +
    +    private final String dir;
    +
    +    private static final String DEVICES_ALLOW = "/devices.allow";
    +    private static final String DEVICES_DENY = "/devices.deny";
    +    private static final String DEVICES_LIST = "/devices.list";
    +
    +    private static final char TYPE_ALL = 'a';
    +    private static final char TYPE_BLOCK = 'b';
    +    private static final char TYPE_CHAR = 'c';
    +
    +    private static final int ACCESS_READ = 1;
    +    private static final int ACCESS_WRITE = 2;
    +    private static final int ACCESS_CREATE = 4;
    +
    +    private static final char ACCESS_READ_CH = 'r';
    +    private static final char ACCESS_WRITE_CH = 'w';
    +    private static final char ACCESS_CREATE_CH = 'm';
    +
    +    public DevicesCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.devices;
    +    }
    +
    +    public static class Record {
    +        Device device;
    +        char type;
    +        int accesses;
    +
    +        public Record(char type, Device device, int accesses) {
    +            this.type = type;
    +            this.device = device;
    +            this.accesses = accesses;
    +        }
    +
    +        public Record(String output) {
    +            if (output.contains("*")) {
    +                System.out.println("Pre:" + output);
    --- End diff --
    
    replace with logger


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51373442
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    +        this.center.create(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    --- End diff --
    
    I am OK for filing follow-up JIRA of hard CPU limit or CPU guarantee. But we still need a full testing on cpu.share, whether the worker which come back from idle to busy can retrieve enough cpu resource again instead of starving, when the other worker has occupied all cpu resources. Actually, we has met this kind problem on online jobs. That is reason why we choose a hard limit finally.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51286585
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52341213
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!getCores().keySet().contains(SubSystemType.cpuset)) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
    --- End diff --
    
    not necessary, not generic enough


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52056377
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            for (SubSystemType type : hierarchy.getSubSystems()) {
    +                if (type == subsystem) {
    +                    return hierarchy;
    +                }
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subsystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subsystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    --- End diff --
    
    So if the last Hierarchy element in heirarchies has a set of subsystems that doesn't contain all of `subSystems`, then this 'center' is busy?
    
    Why loop through all the hierarchies? 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52111863
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1062,9 +1072,15 @@
                             (add-to-classpath [stormjar])
                             (add-to-classpath topo-classpath))
               top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
    -          mem-onheap (if (and mem-onheap (> mem-onheap 0)) ;; not nil and not zero
    -                       (int (Math/ceil mem-onheap)) ;; round up
    +          mem-onheap (if (and (.get_mem_on_heap resources) (> (.get_mem_on_heap resources) 0)) ;; not nil and not zero
    +                       (int (Math/ceil (.get_mem_on_heap resources))) ;; round up
                            (storm-conf WORKER-HEAP-MEMORY-MB)) ;; otherwise use default value
    +          mem-offheap (if (.get_mem_off_heap resources)
    +                        (int (Math/ceil (.get_mem_off_heap resources))) ;; round up
    +                        0)
    +
    +          cpu (int (Math/ceil (.get_cpu resources)))
    --- End diff --
    
    we actually don't need to check any of the resource configs since the configs are validated in StormSubmitter when the topology is submitted


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52124602
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/Hierarchy.java ---
    @@ -0,0 +1,117 @@
    +/**
    + * 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.container.cgroup;
    +
    +import java.util.Set;
    +
    +public class Hierarchy {
    +
    +    private final String name;
    +
    +    private final Set<SubSystemType> subSystems;
    +
    +    private final String type;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon rootCgroups;
    +
    +    public Hierarchy(String name, Set<SubSystemType> subSystems, String dir) {
    +        this.name = name;
    +        this.subSystems = subSystems;
    +        this.dir = dir;
    +        this.rootCgroups = new CgroupCommon(this, dir);
    +        this.type = CgroupUtils.reAnalyse(subSystems);
    +    }
    +
    +    public Set<SubSystemType> getSubSystems() {
    +        return subSystems;
    +    }
    +
    +    public String getType() {
    +        return type;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        final int prime = 31;
    +        int result = 1;
    +        result = prime * result + ((dir == null) ? 0 : dir.hashCode());
    +        result = prime * result + ((name == null) ? 0 : name.hashCode());
    +        result = prime * result + ((type == null) ? 0 : type.hashCode());
    +        return result;
    +    }
    --- End diff --
    
    the member variable "type" is just a comma delimited string of subsystems that the hierarchy uses.  So in terms of identification, "type" and "subSystems" 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by jerrypeng <gi...@git.apache.org>.
Github user jerrypeng commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-182071058
  
    @abhishekagarwal87 Thanks for the detailed review! much appreciated!  I have addressed your comments
    @zhuoliu thanks for your review as well.  I have address your comments 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by jerrypeng <gi...@git.apache.org>.
Github user jerrypeng commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-176444835
  
    @revans2 thank you for the review.  I added the apache headers to files that were missing them and addressed all of your 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51767827
  
    --- Diff: storm-core/test/jvm/org/apache/storm/TestCGroup.java ---
    @@ -0,0 +1,118 @@
    +/**
    --- End diff --
    
    Can we have the filenames also to be consistent, TestCgroup.java. 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52339354
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    --- End diff --
    
    we can put a warning message


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52124713
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/Utils.java ---
    @@ -1367,12 +1367,67 @@ public static int toPositive(int number) {
             return number & Integer.MAX_VALUE;
         }
     
    -    public static RuntimeException wrapInRuntime(Exception e){
    -        if (e instanceof RuntimeException){
    -            return (RuntimeException)e;
    -        }else {
    +    public static RuntimeException wrapInRuntime(Exception e) {
    +        if (e instanceof RuntimeException) {
    +            return (RuntimeException) e;
    +        } else {
                 return new RuntimeException(e);
             }
         }
    +
    +    public static void ensure_process_killed(Integer pid) {
    +        // in this function, just kill the process 5 times
    +        // make sure the process be killed definitely
    +        for (int i = 0; i < 5; i++) {
    +            try {
    +                exec_command("kill -9 " + pid);
    +                LOG.info("kill -9 process " + pid);
    +                sleepMs(100);
    +            } catch (ExecuteException e) {
    +                LOG.info("Error when trying to kill " + pid + ". Process has been killed");
    +                return;
    +            } catch (Exception e) {
    +                LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +            }
    +        }
    +    }
    +
    +    public static void process_killed(Integer pid) {
    +        try {
    +            exec_command("kill " + pid);
    +            LOG.info("kill process " + pid);
    +        } catch (ExecuteException e) {
    +            LOG.info("Error when trying to kill " + pid + ". Process has been killed. ");
    +        } catch (Exception e) {
    +            LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +        }
    +    }
    +
    +    public static void kill(Integer pid) {
    --- End diff --
    
    get rid of


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51917264
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    +        this.center.create(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    +            } catch (IOException e) {
    +                throw new RuntimeException("Cannot set cpu.shares! Exception: " + e);
    +            }
    +        }
    +
    +        if (totalMem != null) {
    +            MemoryCore memCore = (MemoryCore) workerGroup.getCores().get(SubSystemType.memory);
    +            try {
    +                memCore.setPhysicalUsageLimit(Long.valueOf(totalMem.longValue() * 1024 * 1024));
    +            } catch (IOException e) {
    +                throw new RuntimeException("Cannot set memory.limit_in_bytes! Exception: " + e);
    +            }
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        sb.append(this.conf.get(Config.STORM_CGROUP_CGEXEC_CMD)).append(" -g ");
    +
    +        Iterator<SubSystemType> it = h.getSubSystems().iterator();
    --- End diff --
    
    renamed


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52053581
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    +        File file = new File(dir);
    +        return file.exists();
    +    }
    +
    +    public static boolean dirExists(String dir) {
    --- End diff --
    
    And 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51134075
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    --- End diff --
    
    Again do we want all of these to be CGroup or Cgroup.  I don't really care so long as we are consistent everywhere.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52068474
  
    --- Diff: conf/cgconfig.conf.example ---
    @@ -0,0 +1,41 @@
    +# 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.
    +
    +mount {
    +	cpuset	= /cgroup/cpuset;
    +	cpu	= /cgroup/storm_resources;
    +	cpuacct	= /cgroup/cpuacct;
    +	memory	= /cgroup/storm_resources;
    +	devices	= /cgroup/devices;
    +	freezer	= /cgroup/freezer;
    +	net_cls	= /cgroup/net_cls;
    +	blkio	= /cgroup/blkio;
    +}
    +
    +group storm {
    +       perm {
    +               task {
    +                      uid = 500;
    +                      gid = 500;
    +               }
    +               admin {
    +                      uid = 500;
    +                      gid = 500;
    +               }
    +       }
    +       cpu {
    +       }
    +}
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51156699
  
    --- Diff: storm-core/test/jvm/org/apache/storm/TestCGroup.java ---
    @@ -0,0 +1,94 @@
    +package org.apache.storm;
    +
    +import org.junit.Assert;
    +import org.junit.Assume;
    +import org.apache.storm.container.cgroup.CgroupManager;
    +import org.apache.storm.utils.Utils;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.nio.file.Files;
    +import java.nio.file.Paths;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.UUID;
    +
    +/**
    + * Created by jerrypeng on 1/19/16.
    + */
    +public class TestCGroup {
    --- End diff --
    
    will fix and 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52075177
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52286274
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,128 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    /**
    +     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
    +     */
    +    public static Set<SubSystemType> getSubSystemsFromString(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    /**
    +     * Get a string that is a comma delimited list of subsystems
    +     */
    +    public static String subSystemsToString(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return Utils.checkFileExists(CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    +        List<String> result = new ArrayList<String>();
    +        File file = new File(fileDir);
    +        try (FileReader fileReader = new FileReader(file);
    +             BufferedReader reader = new BufferedReader(fileReader)) {
    +            String tempString = null;
    +            while ((tempString = reader.readLine()) != null) {
    +                result.add(tempString);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static void writeFileByLine(String fileDir, List<String> strings) throws IOException {
    --- End diff --
    
    fileDir can be renamed to filepath. strings can be renames to lines. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52287708
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    --- End diff --
    
    should be storm.supervisor.cgroup.rootdir is 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52068522
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -42,7 +42,9 @@
       (:require [metrics.gauges :refer [defgauge]])
       (:require [metrics.meters :refer [defmeter mark!]])
       (:gen-class
    -    :methods [^{:static true} [launch [org.apache.storm.scheduler.ISupervisor] void]]))
    +    :methods [^{:static true} [launch [org.apache.storm.scheduler.ISupervisor] void]])
    +  (:import [org.apache.storm.container.cgroup CgroupManager])
    --- End diff --
    
    CgroupManager unused?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51133883
  
    --- Diff: storm-core/src/clj/org/apache/storm/util.clj ---
    @@ -541,6 +541,7 @@
         (.redirectErrorStream builder true)
         (doseq [[k v] environment]
           (.put process-env k v))
    +    (log-message "Starting process with command: " command)
    --- End diff --
    
    Can we make this debug or remove it if debugging is 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52289072
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,128 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    /**
    +     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
    +     */
    +    public static Set<SubSystemType> getSubSystemsFromString(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    /**
    +     * Get a string that is a comma delimited list of subsystems
    +     */
    +    public static String subSystemsToString(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return Utils.checkFileExists(CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    --- End diff --
    
    readFileByLine can be replaced with Files.readlines(new File(fileDir)) available in guava and already added as dependency


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52097950
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,226 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.init();
    +        cores = CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.init();
    +        cores = CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), this.dir);
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(Constants.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(Constants.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, RELEASE_AGENT), command);
    +    }
    +
    +    @Override
    +    public String getReleaseAgent() throws IOException {
    +        if (!this.isRoot) {
    +            return null;
    +        }
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, RELEASE_AGENT)).get(0);
    +    }
    +
    +    @Override
    +    public void setCgroupCloneChildren(boolean flag) throws IOException {
    +        if (!this.cores.keySet().contains(SubSystemType.cpuset)) {
    +            return;
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_CLONE_CHILDREN), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getCgroupCloneChildren() throws IOException {
    +        return CgroupUtils.readFileByLine(Constants.getDir(this.dir, CGROUP_CLONE_CHILDREN)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setEventControl(String eventFd, String controlFd, String... args) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(eventFd);
    +        sb.append(' ');
    +        sb.append(controlFd);
    +        for (String arg : args) {
    +            sb.append(' ');
    +            sb.append(arg);
    +        }
    +        CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_EVENT_CONTROL), sb.toString());
    +    }
    +
    +    public Hierarchy getHierarchy() {
    +        return hierarchy;
    +    }
    +
    +    public String getName() {
    +        return name;
    +    }
    +
    +    public String getDir() {
    +        return dir;
    +    }
    +
    +    public CgroupCommon getParent() {
    +        return parent;
    +    }
    +
    +    public Set<CgroupCommon> getChildren() {
    +        return children;
    +    }
    +
    +    public boolean isRoot() {
    +        return isRoot;
    +    }
    +
    +    public Map<SubSystemType, CgroupCore> getCores() {
    +        return cores;
    +    }
    +
    +    public void delete() throws IOException {
    +        this.free();
    +        if (!this.isRoot) {
    +            this.parent.getChildren().remove(this);
    +        }
    +    }
    +
    +    private void free() throws IOException {
    +        for (CgroupCommon child : this.children) {
    +            child.free();
    +        }
    +        if (this.isRoot) {
    +            return;
    +        }
    +        Set<Integer> tasks = this.getTasks();
    +        if (tasks != null) {
    +            for (Integer task : tasks) {
    +                this.parent.addTask(task);
    +            }
    +        }
    +        CgroupUtils.deleteDir(this.dir);
    +    }
    +
    +    private void init() {
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52286719
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    +        this.hierarchy = hierarchy;
    +        this.parent = null;
    +        this.dir = dir;
    +        this.isRoot = true;
    +    }
    +
    +    @Override
    +    public void addTask(int taskId) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, TASKS), String.valueOf(taskId));
    +    }
    +
    +    @Override
    +    public Set<Integer> getTasks() throws IOException {
    +        List<String> stringTasks = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, TASKS));
    +        Set<Integer> tasks = new HashSet<Integer>();
    +        for (String task : stringTasks) {
    +            tasks.add(Integer.valueOf(task));
    +        }
    +        return tasks;
    +    }
    +
    +    @Override
    +    public void addProcs(int pid) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS), String.valueOf(pid));
    +    }
    +
    +    @Override
    +    public Set<Integer> getPids() throws IOException {
    +        List<String> stringPids = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, CGROUP_PROCS));
    +        Set<Integer> pids = new HashSet<Integer>();
    +        for (String task : stringPids) {
    +            pids.add(Integer.valueOf(task));
    +        }
    +        return pids;
    +    }
    +
    +    @Override
    +    public void setNotifyOnRelease(boolean flag) throws IOException {
    +
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE), flag ? "1" : "0");
    +    }
    +
    +    @Override
    +    public boolean getNotifyOnRelease() throws IOException {
    +        return CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, NOTIFY_ON_RELEASE)).get(0).equals("1") ? true : false;
    +    }
    +
    +    @Override
    +    public void setReleaseAgent(String command) throws IOException {
    +        if (!this.isRoot) {
    +            return;
    --- End diff --
    
    should you log an error/warn if it is called from non-root cgroup


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51225283
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    --- End diff --
    
    public -> private


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52370991
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return true;
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.isMounted(hierarchy)) {
    +            LOG.error("{} is already mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subSystems) {
    +            if (this.getHierarchyWithSubSystem(type) != null) {
    +                LOG.error("subSystem: {} is busy", type.name());
    +                subSystems.remove(type);
    +            }
    +        }
    +        if (subSystems.size() == 0) {
    +            return;
    +        }
    +        if (!Utils.CheckDirExists(hierarchy.getDir())) {
    +            new File(hierarchy.getDir()).mkdirs();
    +        }
    +        String subSystemsName = CgroupUtils.subSystemsToString(subSystems);
    +        SystemOperation.mount(subSystemsName, hierarchy.getDir(), "cgroup", subSystemsName);
    +
    +    }
    +
    +    @Override
    +    public void umount(Hierarchy hierarchy) throws IOException {
    +        if (this.isMounted(hierarchy)) {
    +            hierarchy.getRootCgroups().delete();
    +            SystemOperation.umount(hierarchy.getDir());
    +            CgroupUtils.deleteDir(hierarchy.getDir());
    +        } else {
    +            LOG.error("{} is not mounted", hierarchy.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void createCgroup(CgroupCommon cgroup) throws SecurityException {
    +        if (cgroup.isRoot()) {
    +            LOG.error("You can't create rootCgroup in this function");
    +            return;
    +        }
    +        CgroupCommon parent = cgroup.getParent();
    +        while (parent != null) {
    +            if (!Utils.CheckDirExists(parent.getDir())) {
    +                LOG.error("Parent {} does not exist", parent.getDir());
    +                return;
    --- End diff --
    
    actually this is a important function, lets throw an exception


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by jerrypeng <gi...@git.apache.org>.
Github user jerrypeng commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-181586732
  
    @d2r @knusbaum @kishorvpatil Thank you for your reviews.  I have addressed all your comments.
    
    I have also added additional documentation in the code.
    
    @kishorvpatil I rewrote the ResourceIsolationInterface as per our discussion


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52333311
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52111775
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -381,7 +391,7 @@
                     (:storm-id assignment)
                     port
                     id
    -                mem-onheap)
    +                resources)
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52277714
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    --- End diff --
    
    again length check on split?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51225598
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            for (SubSystemType type : hierarchy.getSubSystems()) {
    +                if (type == subsystem) {
    +                    return hierarchy;
    +                }
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subsystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subsystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy mounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (CgroupUtils.dirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return h;
    +                }
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.mounted(hierarchy) != null) {
    +            LOG.error("{} is mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subsystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subsystems) {
    +            if (this.busy(type) != null) {
    +                LOG.error("subsystem: {} is busy", type.name());
    +                subsystems.remove(type);
    +            }
    +        }
    +        if (subsystems.size() == 0) {
    +            return;
    +        }
    +        if (!CgroupUtils.dirExists(hierarchy.getDir())) {
    +            new File(hierarchy.getDir()).mkdirs();
    +        }
    +        String subSystems = CgroupUtils.reAnalyse(subsystems);
    +        SystemOperation.mount(subSystems, hierarchy.getDir(), "cgroup", subSystems);
    +
    +    }
    +
    +    @Override
    +    public void umount(Hierarchy hierarchy) throws IOException {
    +        if (this.mounted(hierarchy) != null) {
    +            hierarchy.getRootCgroups().delete();
    +            SystemOperation.umount(hierarchy.getDir());
    +            CgroupUtils.deleteDir(hierarchy.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void create(CgroupCommon cgroup) throws SecurityException {
    +        if (cgroup.isRoot()) {
    +            LOG.error("You can't create rootCgroup in this function");
    +            return;
    +        }
    +        CgroupCommon parent = cgroup.getParent();
    +        while (parent != null) {
    +            if (!CgroupUtils.dirExists(parent.getDir())) {
    +                LOG.error(" {} is not existed", parent.getDir());
    +                return;
    +            }
    +            parent = parent.getParent();
    +        }
    +        Hierarchy h = cgroup.getHierarchy();
    +        if (mounted(h) == null) {
    +            LOG.error("{} is not mounted", h.getDir());
    +            return;
    +        }
    +        if (CgroupUtils.dirExists(cgroup.getDir())) {
    +            LOG.error("{} is existed", cgroup.getDir());
    +            return;
    +        }
    +
    +        //Todo perhaps thrown exception or print out error message is dir is not created successfully
    +        if (!(new File(cgroup.getDir())).mkdir()) {
    +            LOG.error("Could not create cgroup dir at {}", cgroup.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void delete(CgroupCommon cgroup) throws IOException {
    +
    +        cgroup.delete();
    +    }
    +
    +    public static void main(String args[]) {
    +        System.out.println(CgroupCenter.getInstance().getHierarchies().get(0).getRootCgroups().getChildren().size());
    --- End diff --
    
    Maybe we can add some unit Tests for CGroups and remove the `main` method 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51135424
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null)
    +            instance = new CgroupCenter();
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup"))
    +                    continue;
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +        FileReader reader = null;
    +        BufferedReader br = null;
    +        try {
    +            reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +            br = new BufferedReader(reader);
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null)
    +                    continue;
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]), Integer.valueOf(split[3]).intValue() == 1 ? true
    +                        : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error ", e);
    +        } finally {
    +            CgroupUtils.close(reader, br);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem)
    +                return true;
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            for (SubSystemType type : hierarchy.getSubSystems()) {
    +                if (type == subsystem)
    +                    return hierarchy;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subsystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subsystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy mounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (CgroupUtils.dirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy))
    +                    return h;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.mounted(hierarchy) != null) {
    +            LOG.error(hierarchy.getDir() + " is mounted");
    --- End diff --
    
    Can we please update the log messages to use "{}" substitution.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52083925
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            for (SubSystemType type : hierarchy.getSubSystems()) {
    +                if (type == subsystem) {
    +                    return hierarchy;
    +                }
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subsystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subsystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    --- End diff --
    
    It would be good to rename `busy` and perhaps add a javadoc explaining what it does.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51225292
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class SystemOperation {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    --- End diff --
    
    public -> private


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52124733
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    --- End diff --
    
    will rename isSubsystemEnabled


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52367000
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    //private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    +
    +    public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) {
    +        this.name = parent.getName() + "/" + name;
    +        this.hierarchy = hierarchy;
    +        this.parent = parent;
    +        this.dir = parent.getDir() + "/" + name;
    +        this.isRoot = false;
    +    }
    +
    +    /**
    +     * rootCgroup
    +     */
    +    public CgroupCommon(Hierarchy hierarchy, String dir) {
    +        this.name = "";
    --- End diff --
    
    the name variable holds the path relative to storm base cgroup directory while dir holds the complete 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52355713
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,207 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * initalize subsystems
    +     */
    +    private void prepareSubSystem(Map conf) throws IOException {
    +        List<SubSystemType> subSystemTypes = new LinkedList<>();
    +        for (String resource : Config.getCgroupStormResources(conf)) {
    +            subSystemTypes.add(SubSystemType.getSubSystem(resource));
    +        }
    +
    +        this.hierarchy = center.getHierarchyWithSubSystems(subSystemTypes);
    +
    +        if (this.hierarchy == null) {
    +            Set<SubSystemType> types = new HashSet<SubSystemType>();
    +            types.add(SubSystemType.cpu);
    +            this.hierarchy = new Hierarchy(Config.getCgroupStormHierarchyName(conf), types, Config.getCgroupStormHierarchyDir(conf));
    +        }
    +        this.rootCgroup = new CgroupCommon(this.rootDir, this.hierarchy, this.hierarchy.getRootCgroups());
    +
    +        // set upper limit to how much cpu can be used by all workers running on supervisor node.
    +        // This is done so that some cpu cycles will remain free to run the daemons and other miscellaneous OS operations.
    +        CpuCore supervisorRootCPU = (CpuCore)  this.rootCgroup.getCores().get(SubSystemType.cpu);
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52627119
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -344,6 +347,12 @@
        :sync-retry (atom 0)
        :download-lock (Object.)
        :stormid->profiler-actions (atom {})
    +   :resource-isolation-manager (if (conf STORM-RESOURCE-ISOLATION-PLUGIN-ENABLE)
    +                     (let [resource-isolation-manager (Utils/newInstance (conf STORM-RESOURCE-ISOLATION-PLUGIN))]
    --- End diff --
    
    Indentation here is off.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52334956
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return true;
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.isMounted(hierarchy)) {
    +            LOG.error("{} is already mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subSystems) {
    +            if (this.getHierarchyWithSubSystem(type) != null) {
    +                LOG.error("subSystem: {} is busy", type.name());
    +                subSystems.remove(type);
    +            }
    +        }
    +        if (subSystems.size() == 0) {
    +            return;
    +        }
    +        if (!Utils.CheckDirExists(hierarchy.getDir())) {
    +            new File(hierarchy.getDir()).mkdirs();
    +        }
    +        String subSystemsName = CgroupUtils.subSystemsToString(subSystems);
    +        SystemOperation.mount(subSystemsName, hierarchy.getDir(), "cgroup", subSystemsName);
    +
    +    }
    +
    +    @Override
    +    public void umount(Hierarchy hierarchy) throws IOException {
    +        if (this.isMounted(hierarchy)) {
    +            hierarchy.getRootCgroups().delete();
    +            SystemOperation.umount(hierarchy.getDir());
    +            CgroupUtils.deleteDir(hierarchy.getDir());
    +        } else {
    +            LOG.error("{} is not mounted", hierarchy.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void createCgroup(CgroupCommon cgroup) throws SecurityException {
    +        if (cgroup.isRoot()) {
    +            LOG.error("You can't create rootCgroup in this function");
    +            return;
    +        }
    +        CgroupCommon parent = cgroup.getParent();
    +        while (parent != null) {
    +            if (!Utils.CheckDirExists(parent.getDir())) {
    +                LOG.error("Parent {} does not exist", parent.getDir());
    +                return;
    --- End diff --
    
    I think printing out an error message is ok for now.  Since we are not sure if we want to crash the supervisor if something in cgroups goes wrong.  Though this is something to investigate in a follow 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52278401
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    --- End diff --
    
    Also, getHierarchies is going and reading from the file system. If getHierarchies is being called too often, you should consider caching. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51786510
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    --- End diff --
    
    h here is hard to read what it is exactly


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51225510
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class SystemOperation {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    };
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ").append(type).append(" -o ").append(data).append(" ").append(name).append(" ").append(target);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static void umount(String name) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("umount ").append(name);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static String exec(String cmd) throws IOException {
    +        LOG.debug("Shell cmd: {}", cmd);
    +        Process process = new ProcessBuilder(new String[] { "/bin/bash", "-c", cmd }).start();
    +        try {
    +            process.waitFor();
    +            String output = IOUtils.toString(process.getInputStream());
    +            String errorOutput = IOUtils.toString(process.getErrorStream());
    +            LOG.debug("Shell Output: {}", output);
    +            if (errorOutput.length() != 0) {
    +                LOG.error("Shell Error Output: {}", errorOutput);
    +                throw new IOException(errorOutput);
    +            }
    +            return output;
    +        } catch (InterruptedException ie) {
    +            throw new IOException(ie.toString());
    +        }
    +
    +    }
    +
    +    public static void main(String[] args) throws IOException {
    +        SystemOperation.mount("test", "/cgroup/cpu", "cgroup", "cpu");
    --- End diff --
    
    Maybe we can add some unit Tests for CGroups and remove the `main` method 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52124299
  
    --- Diff: conf/defaults.yaml ---
    @@ -281,3 +281,16 @@ pacemaker.thread.timeout: 10
     pacemaker.childopts: "-Xmx1024m"
     pacemaker.auth.method: "NONE"
     pacemaker.kerberos.users: []
    +
    +storm.resource.isolation.plugin: "org.apache.storm.container.cgroup.CgroupManager"
    +
    +# Configs for CGroup support
    +storm.cgroup.hierarchy.dir: "/cgroup/storm_resources"
    +storm.cgroup.resources:
    +    - cpu
    +    - memory
    --- End diff --
    
    will add quotes


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52083786
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,133 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    public static boolean fileExists(String dir) {
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52335379
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    --- End diff --
    
    these are not just arbitrary strings but actual names for files used in cgroup


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52285373
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    //private final Map<SubSystemType, CgroupCore> cores;
    --- End diff --
    
    remove the commented 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52421120
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/BlkioCore.java ---
    @@ -0,0 +1,226 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.Device;
    +
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +public class BlkioCore implements CgroupCore {
    +
    +    public static final String BLKIO_WEIGHT = "/blkio.weight";
    +    public static final String BLKIO_WEIGHT_DEVICE = "/blkio.weight_device";
    +    public static final String BLKIO_RESET_STATS = "/blkio.reset_stats";
    +
    +    public static final String BLKIO_THROTTLE_READ_BPS_DEVICE = "/blkio.throttle.read_bps_device";
    +    public static final String BLKIO_THROTTLE_WRITE_BPS_DEVICE = "/blkio.throttle.write_bps_device";
    +    public static final String BLKIO_THROTTLE_READ_IOPS_DEVICE = "/blkio.throttle.read_iops_device";
    +    public static final String BLKIO_THROTTLE_WRITE_IOPS_DEVICE = "/blkio.throttle.write_iops_device";
    +
    +    public static final String BLKIO_THROTTLE_IO_SERVICED = "/blkio.throttle.io_serviced";
    +    public static final String BLKIO_THROTTLE_IO_SERVICE_BYTES = "/blkio.throttle.io_service_bytes";
    +
    +    public static final String BLKIO_TIME = "/blkio.time";
    +    public static final String BLKIO_SECTORS = "/blkio.sectors";
    +    public static final String BLKIO_IO_SERVICED = "/blkio.io_serviced";
    +    public static final String BLKIO_IO_SERVICE_BYTES = "/blkio.io_service_bytes";
    +    public static final String BLKIO_IO_SERVICE_TIME = "/blkio.io_service_time";
    +    public static final String BLKIO_IO_WAIT_TIME = "/blkio.io_wait_time";
    +    public static final String BLKIO_IO_MERGED = "/blkio.io_merged";
    +    public static final String BLKIO_IO_QUEUED = "/blkio.io_queued";
    +
    +    private final String dir;
    +
    +    public BlkioCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.blkio;
    +    }
    +
    +    /* weight: 100-1000 */
    +    public void setBlkioWeight(int weight) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT), String.valueOf(weight));
    +    }
    +
    +    public int getBlkioWeight() throws IOException {
    +        return Integer.valueOf(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT)).get(0)).intValue();
    +    }
    +
    +    public void setBlkioWeightDevice(Device device, int weight) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT_DEVICE), makeContext(device, weight));
    +    }
    +
    +    public Map<Device, Integer> getBlkioWeightDevice() throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_WEIGHT_DEVICE));
    +        Map<Device, Integer> result = new HashMap<Device, Integer>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Integer weight = Integer.valueOf(strArgs[1]);
    +            result.put(device, weight);
    +        }
    +        return result;
    +    }
    +
    +    public void setReadBps(Device device, long bps) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_READ_BPS_DEVICE), makeContext(device, bps));
    +    }
    +
    +    public Map<Device, Long> getReadBps() throws IOException {
    +        return parseConfig(BLKIO_THROTTLE_READ_BPS_DEVICE);
    +    }
    +
    +    public void setWriteBps(Device device, long bps) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_WRITE_BPS_DEVICE), makeContext(device, bps));
    +    }
    +
    +    public Map<Device, Long> getWriteBps() throws IOException {
    +        return parseConfig(BLKIO_THROTTLE_WRITE_BPS_DEVICE);
    +    }
    +
    +    public void setReadIOps(Device device, long iops) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE), makeContext(device, iops));
    +    }
    +
    +    public Map<Device, Long> getReadIOps() throws IOException {
    +        return parseConfig(BLKIO_THROTTLE_READ_IOPS_DEVICE);
    +    }
    +
    +    public void setWriteIOps(Device device, long iops) throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE), makeContext(device, iops));
    +    }
    +
    +    public Map<Device, Long> getWriteIOps() throws IOException {
    +        return parseConfig(BLKIO_THROTTLE_WRITE_IOPS_DEVICE);
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getThrottleIOServiced() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICED)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getThrottleIOServiceByte() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_THROTTLE_IO_SERVICE_BYTES)));
    +    }
    +
    +    public Map<Device, Long> getBlkioTime() throws IOException {
    +        return parseConfig(BLKIO_TIME);
    +    }
    +
    +    public Map<Device, Long> getBlkioSectors() throws IOException {
    +        return parseConfig(BLKIO_SECTORS);
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOServiced() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICED)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOServiceBytes() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICE_BYTES)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOServiceTime() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_SERVICE_TIME)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOWaitTime() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_WAIT_TIME)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOMerged() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_MERGED)));
    +    }
    +
    +    public Map<Device, Map<RecordType, Long>> getIOQueued() throws IOException {
    +        return this.analyseRecord(CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, BLKIO_IO_QUEUED)));
    +    }
    +
    +    public void resetStats() throws IOException {
    +        CgroupUtils.writeFileByLine(CgroupUtils.getDir(this.dir, BLKIO_RESET_STATS), "1");
    +    }
    +
    +    private String makeContext(Device device, Object data) {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append(device.toString()).append(" ").append(data);
    +        return sb.toString();
    +    }
    +
    +    private Map<Device, Long> parseConfig(String config) throws IOException {
    +        List<String> strings = CgroupUtils.readFileByLine(CgroupUtils.getDir(this.dir, config));
    +        Map<Device, Long> result = new HashMap<Device, Long>();
    +        for (String string : strings) {
    +            String[] strArgs = string.split(" ");
    +            Device device = new Device(strArgs[0]);
    +            Long value = Long.valueOf(strArgs[1]);
    +            result.put(device, value);
    +        }
    +        return result;
    +    }
    +
    +    private Map<Device, Map<RecordType, Long>> analyseRecord(List<String> strs) {
    +        Map<Device, Map<RecordType, Long>> result = new HashMap<Device, Map<RecordType, Long>>();
    +        for (String str : strs) {
    +            String[] strArgs = str.split(" ");
    +            if (strArgs.length != 3) {
    +                continue;
    +            }
    +            Device device = new Device(strArgs[0]);
    +            RecordType key = RecordType.getType(strArgs[1]);
    +            Long value = Long.parseLong(strArgs[2]);
    +            Map<RecordType, Long> record = result.get(device);
    +            if (record == null) {
    +                record = new HashMap<RecordType, Long>();
    +                result.put(device, record);
    +            }
    +            record.put(key, value);
    +        }
    +        return result;
    +    }
    +
    +    public enum RecordType {
    +        read, write, sync, async, total;
    +
    +        public static RecordType getType(String type) {
    +            if (type.equals("Read")) {
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52061437
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    };
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ").append(type).append(" -o ").append(data).append(" ").append(name).append(" ").append(target);
    --- End diff --
    
    Can we break this up into lines? I think it would actually be easier to tell what's being appended if the things being appended are all aligned vertically.
    
    ```
    sb.append("mount -t ")
        .append(type)
        .append(" -o ")
        .append(data)
        .append(" ")
        .append(name)
        .append(" ")
        .append(target);
    ```


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51286785
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    --- End diff --
    
    will fix


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51225264
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,225 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    +
    +    private final Hierarchy hierarchy;
    +
    +    private final String name;
    +
    +    private final String dir;
    +
    +    private final CgroupCommon parent;
    +
    +    private final Map<SubSystemType, CgroupCore> cores;
    +
    +    private final boolean isRoot;
    +
    +    private final Set<CgroupCommon> children = new HashSet<CgroupCommon>();
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupCommon.class);
    --- End diff --
    
    public -> private


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52056829
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean enabled(SubSystemType subsystem) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subsystem) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy busy(SubSystemType subsystem) {
    --- End diff --
    
    ```
    {
        return busy(Arrays.toList(subsystem));
    }
    ```


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52045220
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -344,6 +348,12 @@
        :sync-retry (atom 0)
        :download-lock (Object.)
        :stormid->profiler-actions (atom {})
    +   :cgroup-manager (if (conf STORM-CGROUP-ENABLE)
    +                     (let [cgroup-manager (.newInstance (Class/forName (conf STORM-RESOURCE-ISOLATION-PLUGIN)))]
    --- End diff --
    
    If the resource isolation plugin is pluggable, do we want to call this `:cgroup-manager`?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52289184
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/Device.java ---
    @@ -0,0 +1,75 @@
    +/**
    + * 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.container.cgroup;
    +
    +/**
    + * a class that represents a device in linux
    + */
    +public class Device {
    +
    +    public final int major;
    +    public final int minor;
    +
    +    public Device(int major, int minor) {
    +        this.major = major;
    +        this.minor = minor;
    +    }
    +
    +    public Device(String str) {
    +        String[] strArgs = str.split(":");
    +        this.major = Integer.valueOf(strArgs[0]);
    +        this.minor = Integer.valueOf(strArgs[1]);
    +    }
    +
    +    @Override
    +    public String toString() {
    --- End diff --
    
    lot of boilerplate code can be done away with using lombok annotations. 


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52278797
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = new Hierarchy(name, CgroupUtils.getSubSystemsFromString(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(CgroupUtils.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2]),
    +                        Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    +            }
    +            return subSystems;
    +        } catch (Exception e) {
    +            LOG.error("Get subSystems error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isSubSystemEnabled(SubSystemType subSystemType) {
    +
    +        Set<SubSystem> subSystems = this.getSubSystems();
    +        for (SubSystem subSystem : subSystems) {
    +            if (subSystem.getType() == subSystemType) {
    +                return true;
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystem(SubSystemType subSystem) {
    +        return getHierarchyWithSubSystems(Arrays.asList(subSystem));
    +    }
    +
    +    @Override
    +    public Hierarchy getHierarchyWithSubSystems(List<SubSystemType> subSystems) {
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        for (Hierarchy hierarchy : hierarchies) {
    +            Hierarchy ret = hierarchy;
    +            for (SubSystemType subSystem : subSystems) {
    +                if (!hierarchy.getSubSystems().contains(subSystem)) {
    +                    ret = null;
    +                    break;
    +                }
    +            }
    +            if (ret != null) {
    +                return ret;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean isMounted(Hierarchy hierarchy) {
    +
    +        List<Hierarchy> hierarchies = this.getHierarchies();
    +        if (Utils.CheckDirExists(hierarchy.getDir())) {
    +            for (Hierarchy h : hierarchies) {
    +                if (h.equals(hierarchy)) {
    +                    return true;
    +                }
    +            }
    +        }
    +        return false;
    +    }
    +
    +    @Override
    +    public void mount(Hierarchy hierarchy) throws IOException {
    +
    +        if (this.isMounted(hierarchy)) {
    +            LOG.error("{} is already mounted", hierarchy.getDir());
    +            return;
    +        }
    +        Set<SubSystemType> subSystems = hierarchy.getSubSystems();
    +        for (SubSystemType type : subSystems) {
    +            if (this.getHierarchyWithSubSystem(type) != null) {
    +                LOG.error("subSystem: {} is busy", type.name());
    +                subSystems.remove(type);
    +            }
    +        }
    +        if (subSystems.size() == 0) {
    +            return;
    +        }
    +        if (!Utils.CheckDirExists(hierarchy.getDir())) {
    +            new File(hierarchy.getDir()).mkdirs();
    +        }
    +        String subSystemsName = CgroupUtils.subSystemsToString(subSystems);
    +        SystemOperation.mount(subSystemsName, hierarchy.getDir(), "cgroup", subSystemsName);
    +
    +    }
    +
    +    @Override
    +    public void umount(Hierarchy hierarchy) throws IOException {
    +        if (this.isMounted(hierarchy)) {
    +            hierarchy.getRootCgroups().delete();
    +            SystemOperation.umount(hierarchy.getDir());
    +            CgroupUtils.deleteDir(hierarchy.getDir());
    +        } else {
    +            LOG.error("{} is not mounted", hierarchy.getDir());
    +        }
    +    }
    +
    +    @Override
    +    public void createCgroup(CgroupCommon cgroup) throws SecurityException {
    +        if (cgroup.isRoot()) {
    +            LOG.error("You can't create rootCgroup in this function");
    +            return;
    +        }
    +        CgroupCommon parent = cgroup.getParent();
    +        while (parent != null) {
    +            if (!Utils.CheckDirExists(parent.getDir())) {
    +                LOG.error("Parent {} does not exist", parent.getDir());
    +                return;
    --- End diff --
    
    should it rather throw some exception or return a boolean result?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51138787
  
    --- Diff: storm-core/src/clj/org/apache/storm/util.clj ---
    @@ -541,6 +541,7 @@
         (.redirectErrorStream builder true)
         (doseq [[k v] environment]
           (.put process-env k v))
    +    (log-message "Starting process with command: " command)
    --- End diff --
    
    will 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52775213
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1178,7 +1212,7 @@
               (FileUtils/copyDirectory (File. (.getFile url)) (File. target-dir)))))))
     
     (defmethod launch-worker
    -    :local [supervisor storm-id port worker-id mem-onheap]
    +    :local [supervisor storm-id port worker-id resources]
    --- End diff --
    
    I think when I made my comment I thought we had simply added an unused parameter, but that was not the case. Yes, we should keep things consistent as you have done, so disregard.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52367469
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCommon.java ---
    @@ -0,0 +1,247 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCommon implements CgroupCommonOperation {
    +
    +    public static final String TASKS = "/tasks";
    +    public static final String NOTIFY_ON_RELEASE = "/notify_on_release";
    +    public static final String RELEASE_AGENT = "/release_agent";
    +    public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children";
    +    public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control";
    +    public static final String CGROUP_PROCS = "/cgroup.procs";
    --- End diff --
    
    My bad. I definitely need to read up more on cgroups to understand it better.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51225788
  
    --- Diff: storm-core/src/jvm/org/apache/storm/utils/Utils.java ---
    @@ -1382,5 +1385,60 @@ public static TopologyInfo getTopologyInfo(String name, String asUser, Map storm
         public static int toPositive(int number) {
             return number & Integer.MAX_VALUE;
         }
    +
    +    public static void ensure_process_killed(Integer pid) {
    +        // in this function, just kill the process 5 times
    +        // make sure the process be killed definitely
    +        for (int i = 0; i < 5; i++) {
    +            try {
    +                exec_command("kill -9 " + pid);
    +                LOG.info("kill -9 process " + pid);
    +                sleepMs(100);
    +            } catch (ExecuteException e) {
    +                LOG.info("Error when trying to kill " + pid + ". Process has been killed");
    +                return;
    +            } catch (Exception e) {
    +                LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +            }
    +        }
    +    }
    +
    +    public static void process_killed(Integer pid) {
    +        try {
    +            exec_command("kill " + pid);
    +            LOG.info("kill process " + pid);
    +        } catch (ExecuteException e) {
    +            LOG.info("Error when trying to kill " + pid + ". Process has been killed. ");
    +        } catch (Exception e) {
    +            LOG.info("Error when trying to kill " + pid + ".Exception ", e);
    +        }
    +    }
    +
    +    public static void kill(Integer pid) {
    +        process_killed(pid);
    +
    +        sleepMs(1000);
    +
    +        ensure_process_killed(pid);
    +    }
    +
    +    public static void exec_command(String command) throws ExecuteException, IOException {
    +        String[] cmdlist = command.split(" ");
    +        CommandLine cmd = new CommandLine(cmdlist[0]);
    +        for (int i = 1; i < cmdlist.length; i++) {
    +            cmd.addArgument(cmdlist[i]);
    +        }
    +
    +        DefaultExecutor exec = new DefaultExecutor();
    +        exec.execute(cmd);
    +    }
    +
    +    public static void sleepMs(long ms) {
    +        try {
    +            Thread.sleep(ms);
    +        } catch (InterruptedException e) {
    +
    --- End diff --
    
    Do we need: `throw new RuntimeException(e);`


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52098122
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -42,7 +42,9 @@
       (:require [metrics.gauges :refer [defgauge]])
       (:require [metrics.meters :refer [defmeter mark!]])
       (:gen-class
    -    :methods [^{:static true} [launch [org.apache.storm.scheduler.ISupervisor] void]]))
    +    :methods [^{:static true} [launch [org.apache.storm.scheduler.ISupervisor] void]])
    +  (:import [org.apache.storm.container.cgroup CgroupManager])
    --- End diff --
    
    removed


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51225274
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    --- End diff --
    
    public -> private


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51790298
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/core/MemoryCore.java ---
    @@ -0,0 +1,189 @@
    +/**
    + * 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.container.cgroup.core;
    +
    +import org.apache.storm.container.cgroup.CgroupUtils;
    +import org.apache.storm.container.cgroup.Constants;
    +import org.apache.storm.container.cgroup.SubSystemType;
    +
    +import java.io.IOException;
    +
    +public class MemoryCore implements CgroupCore {
    +
    +    public static final String MEMORY_STAT = "/memory.stat";
    +    public static final String MEMORY_USAGE_IN_BYTES = "/memory.usage_in_bytes";
    +    public static final String MEMORY_MEMSW_USAGE_IN_BYTES = "/memory.memsw.usage_in_bytes";
    +    public static final String MEMORY_MAX_USAGE_IN_BYTES = "/memory.max_usage_in_bytes";
    +    public static final String MEMORY_MEMSW_MAX_USAGE_IN_BYTES = "/memory.memsw.max_usage_in_bytes";
    +    public static final String MEMORY_LIMIT_IN_BYTES = "/memory.limit_in_bytes";
    +    public static final String MEMORY_MEMSW_LIMIT_IN_BYTES = "/memory.memsw.limit_in_bytes";
    +    public static final String MEMORY_FAILCNT = "/memory.failcnt";
    +    public static final String MEMORY_MEMSW_FAILCNT = "/memory.memsw.failcnt";
    +    public static final String MEMORY_FORCE_EMPTY = "/memory.force_empty";
    +    public static final String MEMORY_SWAPPINESS = "/memory.swappiness";
    +    public static final String MEMORY_USE_HIERARCHY = "/memory.use_hierarchy";
    +    public static final String MEMORY_OOM_CONTROL = "/memory.oom_control";
    +
    +    private final String dir;
    +
    +    public MemoryCore(String dir) {
    +        this.dir = dir;
    +    }
    +
    +    @Override
    +    public SubSystemType getType() {
    +        return SubSystemType.memory;
    +    }
    +
    +    public static class Stat {
    +        public final long cacheSize;
    +        public final long rssSize;
    +        public final long mappedFileSize;
    +        public final long pgpginNum;
    +        public final long pgpgoutNum;
    +        public final long swapSize;
    +        public final long activeAnonSize;
    +        public final long inactiveAnonSize;
    +        public final long activeFileSize;
    +        public final long inactiveFileSize;
    +        public final long unevictableSize;
    +        public final long hierarchicalMemoryLimitSize;
    +        public final long hierarchicalMemswLimitSize;
    +        public final long totalCacheSize;
    +        public final long totalRssSize;
    +        public final long totalMappedFileSize;
    +        public final long totalPgpginNum;
    +        public final long totalPgpgoutNum;
    +        public final long totalSwapSize;
    +        public final long totalActiveAnonSize;
    +        public final long totalInactiveAnonSize;
    +        public final long totalActiveFileSize;
    +        public final long totalInactiveFileSize;
    +        public final long totalUnevictableSize;
    +        public final long totalHierarchicalMemoryLimitSize;
    +        public final long totalHierarchicalMemswLimitSize;
    +
    +        public Stat(String output) {
    +            String[] splits = output.split("\n");
    +            this.cacheSize = Long.parseLong(splits[0]);
    +            this.rssSize = Long.parseLong(splits[1]);
    +            this.mappedFileSize = Long.parseLong(splits[2]);
    +            this.pgpginNum = Long.parseLong(splits[3]);
    +            this.pgpgoutNum = Long.parseLong(splits[4]);
    +            this.swapSize = Long.parseLong(splits[5]);
    +            this.inactiveAnonSize = Long.parseLong(splits[6]);
    +            this.activeAnonSize = Long.parseLong(splits[7]);
    +            this.inactiveFileSize = Long.parseLong(splits[8]);
    +            this.activeFileSize = Long.parseLong(splits[9]);
    +            this.unevictableSize = Long.parseLong(splits[10]);
    +            this.hierarchicalMemoryLimitSize = Long.parseLong(splits[11]);
    +            this.hierarchicalMemswLimitSize = Long.parseLong(splits[12]);
    +            this.totalCacheSize = Long.parseLong(splits[13]);
    +            this.totalRssSize = Long.parseLong(splits[14]);
    +            this.totalMappedFileSize = Long.parseLong(splits[15]);
    +            this.totalPgpginNum = Long.parseLong(splits[16]);
    +            this.totalPgpgoutNum = Long.parseLong(splits[17]);
    +            this.totalSwapSize = Long.parseLong(splits[18]);
    +            this.totalInactiveAnonSize = Long.parseLong(splits[19]);
    +            this.totalActiveAnonSize = Long.parseLong(splits[20]);
    +            this.totalInactiveFileSize = Long.parseLong(splits[21]);
    +            this.totalActiveFileSize = Long.parseLong(splits[22]);
    +            this.totalUnevictableSize = Long.parseLong(splits[23]);
    +            this.totalHierarchicalMemoryLimitSize = Long.parseLong(splits[24]);
    +            this.totalHierarchicalMemswLimitSize = Long.parseLong(splits[25]);
    --- End diff --
    
    Just a few minor comments, the core implementation of cgroup manager seems fine, not sure how other subsytems come into the picture if we are using only cpu and memory. @revans2 might want to take a look at it before it gets merged.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51785738
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    --- End diff --
    
    Can we take split the comment to next line for readability


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52072760
  
    --- Diff: conf/defaults.yaml ---
    @@ -281,3 +281,16 @@ pacemaker.thread.timeout: 10
     pacemaker.childopts: "-Xmx1024m"
     pacemaker.auth.method: "NONE"
     pacemaker.kerberos.users: []
    +
    +storm.resource.isolation.plugin: "org.apache.storm.container.cgroup.CgroupManager"
    +
    +# Configs for CGroup support
    +storm.cgroup.hierarchy.dir: "/cgroup/storm_resources"
    +storm.cgroup.resources:
    +    - cpu
    +    - memory
    --- End diff --
    
    With few exceptions, I see string values are quoted. Not sure this would make a difference with these values, but maybe we should be 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52342637
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,128 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    /**
    +     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
    +     */
    +    public static Set<SubSystemType> getSubSystemsFromString(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    /**
    +     * Get a string that is a comma delimited list of subsystems
    +     */
    +    public static String subSystemsToString(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return Utils.checkFileExists(CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    +        List<String> result = new ArrayList<String>();
    +        File file = new File(fileDir);
    +        try (FileReader fileReader = new FileReader(file);
    +             BufferedReader reader = new BufferedReader(fileReader)) {
    +            String tempString = null;
    +            while ((tempString = reader.readLine()) != null) {
    +                result.add(tempString);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static void writeFileByLine(String fileDir, List<String> strings) throws IOException {
    +        LOG.debug("For CGroups - writing {} to {} ", strings, fileDir);
    +        File file = new File(fileDir);
    +        if (!file.exists()) {
    +            LOG.error("{} is no existed", fileDir);
    --- End diff --
    
    not => not


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52679346
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,210 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.lang.ArrayUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Class that implements ResourceIsolationInterface that manages cgroups
    + */
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy hierarchy;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    /**
    +     * initialize intial data structures
    +     * @param conf storm confs
    +     */
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The storm.supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{} is not existing.", file.getPath());
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * initalize subsystems
    +     */
    +    private void prepareSubSystem(Map conf) throws IOException {
    +        List<SubSystemType> subSystemTypes = new LinkedList<>();
    +        for (String resource : Config.getCgroupStormResources(conf)) {
    +            subSystemTypes.add(SubSystemType.getSubSystem(resource));
    +        }
    +
    +        this.hierarchy = center.getHierarchyWithSubSystems(subSystemTypes);
    +
    +        if (this.hierarchy == null) {
    +            Set<SubSystemType> types = new HashSet<SubSystemType>();
    +            types.add(SubSystemType.cpu);
    +            this.hierarchy = new Hierarchy(Config.getCgroupStormHierarchyName(conf), types, Config.getCgroupStormHierarchyDir(conf));
    +        }
    +        this.rootCgroup = new CgroupCommon(this.rootDir, this.hierarchy, this.hierarchy.getRootCgroups());
    +
    +        // set upper limit to how much cpu can be used by all workers running on supervisor node.
    +        // This is done so that some cpu cycles will remain free to run the daemons and other miscellaneous OS operations.
    +        CpuCore supervisorRootCPU = (CpuCore) this.rootCgroup.getCores().get(SubSystemType.cpu);
    +        setCpuUsageUpperLimit(supervisorRootCPU, ((Number) this.conf.get(Config.SUPERVISOR_CPU_CAPACITY)).intValue());
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g.
    +     * If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    --- End diff --
    
    yes


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52098057
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class SystemOperation {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    };
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ").append(type).append(" -o ").append(data).append(" ").append(name).append(" ").append(target);
    --- End diff --
    
    will refactor


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by jerrypeng <gi...@git.apache.org>.
Github user jerrypeng commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-184984971
  
    @dr2 thanks for review, I have addressed your 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52050012
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,232 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    /**
    +     * Thread unsafe
    +     * 
    +     * @return
    +     */
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    +    }
    +
    +    @Override
    +    public List<Hierarchy> getHierarchies() {
    +
    +        Map<String, Hierarchy> hierarchies = new HashMap<String, Hierarchy>();
    +
    +        try (FileReader reader = new FileReader(Constants.MOUNT_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)) {
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] strSplit = str.split(" ");
    +                if (!strSplit[2].equals("cgroup")) {
    +                    continue;
    +                }
    +                String name = strSplit[0];
    +                String type = strSplit[3];
    +                String dir = strSplit[1];
    +                Hierarchy h = hierarchies.get(type);
    +                h = new Hierarchy(name, CgroupUtils.analyse(type), dir);
    +                hierarchies.put(type, h);
    +            }
    +            return new ArrayList<Hierarchy>(hierarchies.values());
    +        } catch (Exception e) {
    +            LOG.error("Get hierarchies error {}", e);
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public Set<SubSystem> getSubSystems() {
    +
    +        Set<SubSystem> subSystems = new HashSet<SubSystem>();
    +
    +        try (FileReader reader = new FileReader(Constants.CGROUP_STATUS_FILE);
    +             BufferedReader br = new BufferedReader(reader)){
    +            String str = null;
    +            while ((str = br.readLine()) != null) {
    +                String[] split = str.split("\t");
    +                SubSystemType type = SubSystemType.getSubSystem(split[0]);
    +                if (type == null) {
    +                    continue;
    +                }
    +                subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), Integer.valueOf(split[2])
    +                        , Integer.valueOf(split[3]).intValue() == 1 ? true : false));
    --- End diff --
    
    Move comma next to previous argument.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52067649
  
    --- Diff: conf/defaults.yaml ---
    @@ -281,3 +281,16 @@ pacemaker.thread.timeout: 10
     pacemaker.childopts: "-Xmx1024m"
     pacemaker.auth.method: "NONE"
     pacemaker.kerberos.users: []
    +
    +storm.resource.isolation.plugin: "org.apache.storm.container.cgroup.CgroupManager"
    +
    +# Configs for CGroup support
    +storm.cgroup.hierarchy.dir: "/cgroup/storm_resources"
    +storm.cgroup.resources:
    +    - cpu
    +    - memory
    +storm.cgroup.hierarchy.name: "storm"
    +# Also determines whether the unit tests for cgroup runs.  If cgroup.enable is set to false the unit tests for cgroups will not run
    --- End diff --
    
    I think it would be better if this comment were part of the javadoc for the config definition in Config.java.  If we remove `storm.cgroup.enable` as @knusbaum [suggests](https://github.com/apache/storm/pull/1053/files#r52045815), then this comment should move to the plugin config 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51287904
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/SystemOperation.java ---
    @@ -0,0 +1,70 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class SystemOperation {
    +
    +    public static final Logger LOG = LoggerFactory.getLogger(SystemOperation.class);
    +
    +    public static boolean isRoot() throws IOException {
    +        String result = SystemOperation.exec("echo $EUID").substring(0, 1);
    +        return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true : false;
    +    };
    +
    +    public static void mount(String name, String target, String type, String data) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("mount -t ").append(type).append(" -o ").append(data).append(" ").append(name).append(" ").append(target);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static void umount(String name) throws IOException {
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("umount ").append(name);
    +        SystemOperation.exec(sb.toString());
    +    }
    +
    +    public static String exec(String cmd) throws IOException {
    +        LOG.debug("Shell cmd: {}", cmd);
    +        Process process = new ProcessBuilder(new String[] { "/bin/bash", "-c", cmd }).start();
    +        try {
    +            process.waitFor();
    +            String output = IOUtils.toString(process.getInputStream());
    +            String errorOutput = IOUtils.toString(process.getErrorStream());
    +            LOG.debug("Shell Output: {}", output);
    +            if (errorOutput.length() != 0) {
    +                LOG.error("Shell Error Output: {}", errorOutput);
    +                throw new IOException(errorOutput);
    +            }
    +            return output;
    +        } catch (InterruptedException ie) {
    +            throw new IOException(ie.toString());
    +        }
    +
    +    }
    +
    +    public static void main(String[] args) throws IOException {
    +        SystemOperation.mount("test", "/cgroup/cpu", "cgroup", "cpu");
    --- End diff --
    
    will remove main method, but perhaps we can file a follow up jira to cover all these helper functions since this PR is pretty big as it is and I have a include a unit test that tests the end to end functionality of CGroups.  A lot of the CGroups code came from JStorm, but I was unable to find any unit tests for them in JStorm.  Perhaps some that worked on JStorm can shed some light on whether there are individual unit tests for these functions


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52288658
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,128 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    /**
    +     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
    +     */
    +    public static Set<SubSystemType> getSubSystemsFromString(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    /**
    +     * Get a string that is a comma delimited list of subsystems
    +     */
    +    public static String subSystemsToString(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return Utils.checkFileExists(CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    --- End diff --
    
    fileDir can be renamed to filepath


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52325204
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupCenter.java ---
    @@ -0,0 +1,222 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupCenter implements CgroupOperation {
    +
    +    private static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class);
    +
    +    private static CgroupCenter instance;
    +
    +    private CgroupCenter() {
    +
    +    }
    +
    +    public synchronized static CgroupCenter getInstance() {
    +        if (instance == null) {
    +            instance = new CgroupCenter();
    +        }
    +        return CgroupUtils.enabled() ? instance : null;
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52355545
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupUtils.java ---
    @@ -0,0 +1,128 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.BufferedReader;
    +import java.io.BufferedWriter;
    +import java.io.File;
    +import java.io.FileReader;
    +import java.io.FileWriter;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class CgroupUtils {
    +
    +    public static final String CGROUP_STATUS_FILE = "/proc/cgroups";
    +    public static final String MOUNT_STATUS_FILE = "/proc/mounts";
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class);
    +
    +    public static void deleteDir(String dir) {
    +        File d = new File(dir);
    +        if (d.exists()) {
    +            if (d.isDirectory()) {
    +                if (!d.delete()) {
    +                    throw new RuntimeException("Cannot delete dir " + dir);
    +                }
    +            } else {
    +                throw new RuntimeException("dir " + dir + " is not a directory!");
    +            }
    +        } else {
    +            LOG.warn("dir {} does not exist!", dir);
    +        }
    +    }
    +
    +    /**
    +     * Get a set of SubSystemType objects from a comma delimited list of subsystem names
    +     */
    +    public static Set<SubSystemType> getSubSystemsFromString(String str) {
    +        Set<SubSystemType> result = new HashSet<SubSystemType>();
    +        String[] subSystems = str.split(",");
    +        for (String subSystem : subSystems) {
    +            SubSystemType type = SubSystemType.getSubSystem(subSystem);
    +            if (type != null) {
    +                result.add(type);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    /**
    +     * Get a string that is a comma delimited list of subsystems
    +     */
    +    public static String subSystemsToString(Set<SubSystemType> subSystems) {
    +        StringBuilder sb = new StringBuilder();
    +        if (subSystems.size() == 0) {
    +            return sb.toString();
    +        }
    +        for (SubSystemType type : subSystems) {
    +            sb.append(type.name()).append(",");
    +        }
    +        return sb.toString().substring(0, sb.length() - 1);
    +    }
    +
    +    public static boolean enabled() {
    +        return Utils.checkFileExists(CGROUP_STATUS_FILE);
    +    }
    +
    +    public static List<String> readFileByLine(String fileDir) throws IOException {
    +        List<String> result = new ArrayList<String>();
    +        File file = new File(fileDir);
    +        try (FileReader fileReader = new FileReader(file);
    +             BufferedReader reader = new BufferedReader(fileReader)) {
    +            String tempString = null;
    +            while ((tempString = reader.readLine()) != null) {
    +                result.add(tempString);
    +            }
    +        }
    +        return result;
    +    }
    +
    +    public static void writeFileByLine(String fileDir, List<String> strings) throws IOException {
    +        LOG.debug("For CGroups - writing {} to {} ", strings, fileDir);
    +        File file = new File(fileDir);
    +        if (!file.exists()) {
    +            LOG.error("{} is no existed", fileDir);
    --- End diff --
    
    will 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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51220119
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -1088,8 +1104,26 @@
                                                 (str "file:///" storm-log4j2-conf-dir))
                                               storm-log4j2-conf-dir)
                                          file-path-separator "worker.xml")
    +
    +          cgroup-command (if (conf STORM-CGROUP-ENABLE)
    --- End diff --
    
    Maybe it is better to disable cgroup in local mode since the cgroup is not supported in Windows or some linux with old core version


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

Posted by jerrypeng <gi...@git.apache.org>.
Github user jerrypeng commented on the pull request:

    https://github.com/apache/storm/pull/1053#issuecomment-180040777
  
    @redsanket Thanks for your review.  I addressed all your 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 pull request: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51787164
  
    --- Diff: storm-core/src/jvm/org/apache/storm/container/cgroup/CgroupManager.java ---
    @@ -0,0 +1,176 @@
    +/**
    + * 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.container.cgroup;
    +
    +import org.apache.storm.Config;
    +import org.apache.storm.container.ResourceIsolationInterface;
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class CgroupManager implements ResourceIsolationInterface {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(CgroupManager.class);
    +
    +    private CgroupCenter center;
    +
    +    private Hierarchy h;
    +
    +    private CgroupCommon rootCgroup;
    +
    +    private static String rootDir;
    +
    +    private Map conf;
    +
    +    public void prepare(Map conf) throws IOException {
    +        this.conf = conf;
    +        this.rootDir = Config.getCgroupRootDir(this.conf);
    +        if (this.rootDir == null) {
    +            throw new RuntimeException("Check configuration file. The supervisor.cgroup.rootdir is missing.");
    +        }
    +
    +        File file = new File(Config.getCgroupStormHierarchyDir(conf) + "/" + this.rootDir);
    +        if (!file.exists()) {
    +            LOG.error("{}/{} is not existing.", Config.getCgroupStormHierarchyDir(conf), this.rootDir);
    +            throw new RuntimeException("Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file.");
    +        }
    +        this.center = CgroupCenter.getInstance();
    +        if (this.center == null) {
    +            throw new RuntimeException("Cgroup error, please check /proc/cgroups");
    +        }
    +        this.prepareSubSystem(this.conf);
    +    }
    +
    +    /**
    +     * User cfs_period & cfs_quota to control the upper limit use of cpu core e.g. If making a process to fully use two cpu cores, set cfs_period_us to
    +     * 100000 and set cfs_quota_us to 200000
    +     */
    +    private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) throws IOException {
    +
    +        if (cpuCoreUpperLimit == -1) {
    +            // No control of cpu usage
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit);
    +        } else {
    +            cpuCore.setCpuCfsPeriodUs(100000);
    +            cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 1000);
    +        }
    +    }
    +
    +    public String startNewWorker(String workerId, Map resourcesMap) throws SecurityException {
    +        Number cpuNum = (Number) resourcesMap.get("cpu");
    +        Number totalMem = null;
    +        if (resourcesMap.get("memory") != null) {
    +            totalMem = (Number) resourcesMap.get("memory");
    +        }
    +
    +        CgroupCommon workerGroup = new CgroupCommon(workerId, h, this.rootCgroup);
    +        this.center.create(workerGroup);
    +
    +        if (cpuNum != null) {
    +            CpuCore cpuCore = (CpuCore) workerGroup.getCores().get(SubSystemType.cpu);
    +            try {
    +                cpuCore.setCpuShares(cpuNum.intValue());
    +            } catch (IOException e) {
    +                throw new RuntimeException("Cannot set cpu.shares! Exception: " + e);
    +            }
    +        }
    +
    +        if (totalMem != null) {
    +            MemoryCore memCore = (MemoryCore) workerGroup.getCores().get(SubSystemType.memory);
    +            try {
    +                memCore.setPhysicalUsageLimit(Long.valueOf(totalMem.longValue() * 1024 * 1024));
    +            } catch (IOException e) {
    +                throw new RuntimeException("Cannot set memory.limit_in_bytes! Exception: " + e);
    +            }
    +        }
    +
    +        StringBuilder sb = new StringBuilder();
    +
    +        sb.append(this.conf.get(Config.STORM_CGROUP_CGEXEC_CMD)).append(" -g ");
    +
    +        Iterator<SubSystemType> it = h.getSubSystems().iterator();
    --- End diff --
    
    same here h?


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r52052852
  
    --- Diff: storm-core/src/clj/org/apache/storm/daemon/supervisor.clj ---
    @@ -301,7 +303,9 @@
             (try
               (rmpath (ConfigUtils/workerPidPath conf id pid))
               (catch Exception e)))) ;; on windows, the supervisor may still holds the lock on the worker directory
    -    (try-cleanup-worker conf id))
    +    (try-cleanup-worker conf id)
    +    (if (conf STORM-CGROUP-ENABLE)
    +      (.shutDownWorker (:cgroup-manager supervisor) id false)))
    --- End diff --
    
    I would move check and call for `shutdownWorker` under `try-cleanup-worker` method itself. Also, we can check if `(not-nil? (:cgroup-manger supervisor) )`.


---
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: [STORM-1336] - Evalute/Port JStorm cgroup supp...

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

    https://github.com/apache/storm/pull/1053#discussion_r51182779
  
    --- Diff: storm-core/src/jvm/org/apache/storm/Config.java ---
    @@ -2359,4 +2417,24 @@ public void setTopologyStrategy(Class<? extends IStrategy> clazz) {
                 this.put(Config.TOPOLOGY_SCHEDULER_STRATEGY, clazz.getName());
             }
         }
    +
    +    public static String getCgroupRootDir(Map conf) {
    --- End diff --
    
    fixed


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