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

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

GitHub user revans2 opened a pull request:

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

    STORM-2333: CGroup memory and CPU metrics

    

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

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

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

    https://github.com/apache/storm/pull/1934.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 #1934
    
----
commit ecbd21c00030f66783834fc00781bd13b49d7196
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Date:   2017-01-31T22:05:22Z

    STORM-2333: CGroup memory and CPU metrics

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934
  
    @revans2 this is a really cool feature and thanks for adding it.  My only question is that I see you have implemented some parsers/getters for the cgroup metrics.  There is already a set of them implemented in the cgroup package.  Is there a reason why we shouldn't reuse 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 #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r100951524
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java ---
    @@ -0,0 +1,35 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +/**
    + * Reports the current memory limit of the cgroup for this worker
    + */
    +public class CGroupMemoryLimit extends CGroupMetricsBase<Long> {
    +
    +    public CGroupMemoryLimit(Map<String, Object> conf) {
    +        super(conf, "memory.limit_in_bytes");
    +    }
    +
    +    @Override
    +    public Long parseFileContents(String contents) {
    +        return Long.parseLong(contents.trim());
    --- End diff --
    
    Many of these cgroup getting functions have been implemented in the cgroup package:
    
    https://github.com/apache/storm/tree/master/storm-core/src/jvm/org/apache/storm/container/cgroup/core
    
    
    for example there is already a method of get memory.limit_in_bytes:
    
    https://github.com/apache/storm/blob/master/storm-core/src/jvm/org/apache/storm/container/cgroup/core/MemoryCore.java#L133


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934
  
    Thanks @revans2 +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 issue #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934
  
    @jerrypeng Fixed the issue.  We were only checking if the SybSystem was enabled, but not if it was mounted.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r100485119
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/SystemBolt.java ---
    @@ -85,14 +87,15 @@ public Object getValueAndReset() {
             }
         }
     
    +    @SuppressWarnings({ "unchecked", "rawtypes" })
         @Override
    -    public void prepare(final Map stormConf, TopologyContext context, OutputCollector collector) {
    -        if(_prepareWasCalled && !"local".equals(stormConf.get(Config.STORM_CLUSTER_MODE))) {
    +    public void prepare(final Map topoConf, TopologyContext context, OutputCollector collector) {
    +        if(_prepareWasCalled && !"local".equals(topoConf.get(Config.STORM_CLUSTER_MODE))) {
                 throw new RuntimeException("A single worker should have 1 SystemBolt instance.");
             }
             _prepareWasCalled = true;
    --- End diff --
    
    is storm prepare method thread safe?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r100821485
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.java ---
    @@ -0,0 +1,68 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +/**
    + * Report CPU used in the cgroup
    + */
    +public class CGroupCpu extends CGroupMetricsBase<Map<String,Long>> {
    +    long previousSystem = 0;
    +    long previousUser = 0;
    +    
    +    public CGroupCpu(Map<String, Object> conf) {
    +        super(conf, "cpuacct.stat");
    +    }
    +
    +    public int getUserHZ() {
    +        return 100; // On most systems (x86) this is fine.
    +        // If someone really does want full support
    +        // we need to run `getconf CLK_TCK` and cache the result.
    +    }
    +
    +    @Override
    +    public Map<String, Long> parseFileContents(String contents) {
    +        try {
    +            long systemHz = 0;
    +            long userHz = 0;
    +            for (String line: contents.split("\n")) {
    +                if (!line.isEmpty()) {
    +                    String [] parts = line.toLowerCase().split("\\s+");
    +                    if (parts[0].contains("system")) {
    +                        systemHz = Long.parseLong(parts[1].trim());
    +                    } else if (parts[0].contains("user")) {
    +                        userHz = Long.parseLong(parts[1].trim());
    +                    }
    +                }   
    +            }
    +            long user = userHz - previousUser;
    +            long sys = systemHz - previousSystem;
    +            previousUser = userHz;
    +            previousSystem = systemHz;
    +            long hz = getUserHZ();
    +            HashMap<String, Long> ret = new HashMap<>();
    +            ret.put("user-ms", user * 1000/hz); //Convert to millis
    --- End diff --
    
    Makes sense, thanks for the explanation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r100951683
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupCpuGuarantee.java ---
    @@ -0,0 +1,48 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +import org.apache.storm.container.cgroup.core.CpuCore;
    +
    +/**
    + * Report the guaranteed number of ms this worker has requested.
    + */
    +public class CGroupCpuGuarantee extends CGroupMetricsBase<Long> {
    +    long previousTime = -1;
    +    
    +    public CGroupCpuGuarantee(Map<String, Object> conf) {
    +        super(conf, CpuCore.CPU_SHARES);
    +    }
    +
    +    @Override
    +    public Long parseFileContents(String contents) {
    --- End diff --
    
    Function already defined:
    
    https://github.com/apache/storm/blob/master/storm-core/src/jvm/org/apache/storm/container/cgroup/core/CpuCore.java#L50


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r102484535
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.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.metric.cgroup;
    +
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.apache.storm.container.cgroup.core.CpuacctCore;
    +import org.apache.storm.container.cgroup.core.CpuacctCore.StatType;
    +
    +/**
    + * Report CPU used in the cgroup
    + */
    +public class CGroupCpu extends CGroupMetricsBase<Map<String,Long>> {
    +    long previousSystem = 0;
    +    long previousUser = 0;
    +    private int userHz = -1;
    +    
    +    public CGroupCpu(Map<String, Object> conf) {
    +        super(conf, SubSystemType.cpuacct);
    +    }
    +
    +    public synchronized int getUserHZ() throws IOException {
    +        if (userHz < 0) {
    +            ProcessBuilder pb = new ProcessBuilder("getconf", "CLK_TCK");
    +            Process p = pb.start();
    +            BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()));
    +            String line = in.readLine().trim();
    +            userHz = Integer.valueOf(line);
    +        }
    +        return userHz;
    +    }
    +
    +    @Override
    +    public Map<String, Long> getDataFrom(CgroupCore core) throws IOException {
    +        CpuacctCore cpu = (CpuacctCore) core;
    +        Map<StatType, Long> stat = cpu.getCpuStat();
    --- End diff --
    
    All of the CGroupCore subclasses throw an IOException.  I thought it would be better to have the parent class handle it rather then have the children each have the same code copied and pasted.   On some I guess I forgot to clean it up.  Will 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 #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r100814219
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.java ---
    @@ -0,0 +1,68 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +/**
    + * Report CPU used in the cgroup
    + */
    +public class CGroupCpu extends CGroupMetricsBase<Map<String,Long>> {
    +    long previousSystem = 0;
    +    long previousUser = 0;
    +    
    +    public CGroupCpu(Map<String, Object> conf) {
    +        super(conf, "cpuacct.stat");
    +    }
    +
    +    public int getUserHZ() {
    +        return 100; // On most systems (x86) this is fine.
    +        // If someone really does want full support
    +        // we need to run `getconf CLK_TCK` and cache the result.
    +    }
    +
    +    @Override
    +    public Map<String, Long> parseFileContents(String contents) {
    +        try {
    +            long systemHz = 0;
    +            long userHz = 0;
    +            for (String line: contents.split("\n")) {
    +                if (!line.isEmpty()) {
    +                    String [] parts = line.toLowerCase().split("\\s+");
    +                    if (parts[0].contains("system")) {
    +                        systemHz = Long.parseLong(parts[1].trim());
    +                    } else if (parts[0].contains("user")) {
    +                        userHz = Long.parseLong(parts[1].trim());
    +                    }
    +                }   
    +            }
    +            long user = userHz - previousUser;
    +            long sys = systemHz - previousSystem;
    +            previousUser = userHz;
    +            previousSystem = systemHz;
    +            long hz = getUserHZ();
    +            HashMap<String, Long> ret = new HashMap<>();
    +            ret.put("user-ms", user * 1000/hz); //Convert to millis
    --- End diff --
    
    I did this in part to match the https://github.com/apache/storm/blob/master/external/storm-metrics/src/main/java/org/apache/storm/metrics/sigar/CPUMetric.java format, which offers ms level metrics, not floating point, but just for the JVM, not for the entire container. 
    
    I'll try to add documentation to explain the level of granularity that this supports and some of it's limitations.  I am missing documentation all together after all.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r102484623
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryUsage.java ---
    @@ -0,0 +1,39 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +
    +/**
    + * Reports the current memory usage of the cgroup for this worker
    + */
    +public class CGroupMemoryUsage extends CGroupMetricsBase<Long> {
    +
    +    public CGroupMemoryUsage(Map<String, Object> conf) {
    +        super(conf, SubSystemType.memory);
    +    }
    +
    +    @Override
    +    public Long getDataFrom(CgroupCore core) throws Exception {
    --- End diff --
    
    ^C^V


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934
  
    @jerrypeng I added in the extra documentation you wanted, and I spell checked the whole thing.
    
    I also updated the CGroupCpuGuarantee metric to let the parent handle the 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 #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r102382461
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java ---
    @@ -0,0 +1,39 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +
    +/**
    + * Reports the current memory limit of the cgroup for this worker
    + */
    +public class CGroupMemoryLimit extends CGroupMetricsBase<Long> {
    +
    +    public CGroupMemoryLimit(Map<String, Object> conf) {
    +        super(conf, SubSystemType.memory);
    +    }
    +
    +    @Override
    +    public Long getDataFrom(CgroupCore core) throws Exception {
    --- End diff --
    
    Same question here but also this function throws an Exception except for IOException


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r101365418
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java ---
    @@ -0,0 +1,35 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +/**
    + * Reports the current memory limit of the cgroup for this worker
    + */
    +public class CGroupMemoryLimit extends CGroupMetricsBase<Long> {
    +
    +    public CGroupMemoryLimit(Map<String, Object> conf) {
    +        super(conf, "memory.limit_in_bytes");
    +    }
    +
    +    @Override
    +    public Long parseFileContents(String contents) {
    +        return Long.parseLong(contents.trim());
    --- End diff --
    
    I'll look into possibly using it, but right now most of the code that would be replaced is the code in CGroupMetricsBase that slurps 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 #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r101368038
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java ---
    @@ -0,0 +1,35 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +/**
    + * Reports the current memory limit of the cgroup for this worker
    + */
    +public class CGroupMemoryLimit extends CGroupMetricsBase<Long> {
    +
    +    public CGroupMemoryLimit(Map<String, Object> conf) {
    +        super(conf, "memory.limit_in_bytes");
    +    }
    +
    +    @Override
    +    public Long parseFileContents(String contents) {
    +        return Long.parseLong(contents.trim());
    --- End diff --
    
    I have been looking into it and it would be a lot of change just to save a few lines of code.  If you really think it is important to make the code common I will, but I would prefer to leave it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r102484597
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java ---
    @@ -0,0 +1,39 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +
    +/**
    + * Reports the current memory limit of the cgroup for this worker
    + */
    +public class CGroupMemoryLimit extends CGroupMetricsBase<Long> {
    +
    +    public CGroupMemoryLimit(Map<String, Object> conf) {
    +        super(conf, SubSystemType.memory);
    +    }
    +
    +    @Override
    +    public Long getDataFrom(CgroupCore core) throws Exception {
    --- End diff --
    
    ^C^V


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934
  
    Thanks @revans2 +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 #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r100691777
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.java ---
    @@ -0,0 +1,68 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +/**
    + * Report CPU used in the cgroup
    + */
    +public class CGroupCpu extends CGroupMetricsBase<Map<String,Long>> {
    +    long previousSystem = 0;
    +    long previousUser = 0;
    +    
    +    public CGroupCpu(Map<String, Object> conf) {
    +        super(conf, "cpuacct.stat");
    +    }
    +
    +    public int getUserHZ() {
    +        return 100; // On most systems (x86) this is fine.
    +        // If someone really does want full support
    +        // we need to run `getconf CLK_TCK` and cache the result.
    +    }
    +
    +    @Override
    +    public Map<String, Long> parseFileContents(String contents) {
    +        try {
    +            long systemHz = 0;
    +            long userHz = 0;
    +            for (String line: contents.split("\n")) {
    +                if (!line.isEmpty()) {
    +                    String [] parts = line.toLowerCase().split("\\s+");
    +                    if (parts[0].contains("system")) {
    +                        systemHz = Long.parseLong(parts[1].trim());
    +                    } else if (parts[0].contains("user")) {
    +                        userHz = Long.parseLong(parts[1].trim());
    +                    }
    +                }   
    +            }
    +            long user = userHz - previousUser;
    +            long sys = systemHz - previousSystem;
    +            previousUser = userHz;
    +            previousSystem = systemHz;
    +            long hz = getUserHZ();
    +            HashMap<String, Long> ret = new HashMap<>();
    +            ret.put("user-ms", user * 1000/hz); //Convert to millis
    --- End diff --
    
    This is a integer division during ms conversion. A float division would produce better resolution. It is surely not a problem with getUserHz() returning 100, but might cause issues if 100 is changed in the future.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934
  
    thanks @revans2 for taking the time to refactor some of the code!  I reviewed it once more.  If there is an example of how to user the metrics in the docs I think that would be extremely helpful to users.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r102381366
  
    --- Diff: docs/cgroups_in_storm.md ---
    @@ -43,13 +43,24 @@ group storm {
            }
            cpu {
            }
    +       memory {
    +       }
    +       cpuacct {
    +       }
     }
     ```
     
     For a more detailed explanation of the format and configs for the cgconfig.conf file, please visit:
     
     https://access.redhat.com/documentation/en-US/Red_Hat_Enterprise_Linux/6/html/Resource_Management_Guide/ch-Using_Control_Groups.html#The_cgconfig.conf_File
     
    +To let storm manage the cgroups for indavidual workers you need to make sure that the resources you want to control are mounted under the same directory as in the example above.
    +If they are not in the same directory the supervisor will throw an exception.
    +
    +The perm section needs to be configured so that the user the supervisor is running as can modify the group.
    +
    +If run as user is enabled so the supervisor spawns other processes as the user that launched the topology make sure that the permissions are such that indavidual users have read access but not write access.
    --- End diff --
    
    perhaps more clean 
    
    if "run as user"is enabled so **that** the supervisor spawns other...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934
  
    @jerrypeng Please take another look.  I refactored the code to reuse the CgroupManager and such.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r101369654
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryLimit.java ---
    @@ -0,0 +1,35 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +/**
    + * Reports the current memory limit of the cgroup for this worker
    + */
    +public class CGroupMemoryLimit extends CGroupMetricsBase<Long> {
    +
    +    public CGroupMemoryLimit(Map<String, Object> conf) {
    +        super(conf, "memory.limit_in_bytes");
    +    }
    +
    +    @Override
    +    public Long parseFileContents(String contents) {
    +        return Long.parseLong(contents.trim());
    --- End diff --
    
    Actually it might be worth it, just to allow it to support different CGroup hierarchies better.  I'll keep trying


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r102382308
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupCpu.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.metric.cgroup;
    +
    +import java.io.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.apache.storm.container.cgroup.core.CpuacctCore;
    +import org.apache.storm.container.cgroup.core.CpuacctCore.StatType;
    +
    +/**
    + * Report CPU used in the cgroup
    + */
    +public class CGroupCpu extends CGroupMetricsBase<Map<String,Long>> {
    +    long previousSystem = 0;
    +    long previousUser = 0;
    +    private int userHz = -1;
    +    
    +    public CGroupCpu(Map<String, Object> conf) {
    +        super(conf, SubSystemType.cpuacct);
    +    }
    +
    +    public synchronized int getUserHZ() throws IOException {
    +        if (userHz < 0) {
    +            ProcessBuilder pb = new ProcessBuilder("getconf", "CLK_TCK");
    +            Process p = pb.start();
    +            BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()));
    +            String line = in.readLine().trim();
    +            userHz = Integer.valueOf(line);
    +        }
    +        return userHz;
    +    }
    +
    +    @Override
    +    public Map<String, Long> getDataFrom(CgroupCore core) throws IOException {
    +        CpuacctCore cpu = (CpuacctCore) core;
    +        Map<StatType, Long> stat = cpu.getCpuStat();
    --- End diff --
    
    is there a reason why we not doing a try catch of an IOException here similar for cpu.getCpuShares() below but just propagating the 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 issue #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934
  
    @jerrypeng Actually just found a bug working on a fix 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 #1934: STORM-2333: CGroup memory and CPU metrics

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

    https://github.com/apache/storm/pull/1934#discussion_r102382536
  
    --- Diff: storm-core/src/jvm/org/apache/storm/metric/cgroup/CGroupMemoryUsage.java ---
    @@ -0,0 +1,39 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.metric.cgroup;
    +
    +import java.util.Map;
    +
    +import org.apache.storm.container.cgroup.SubSystemType;
    +import org.apache.storm.container.cgroup.core.CgroupCore;
    +import org.apache.storm.container.cgroup.core.MemoryCore;
    +
    +/**
    + * Reports the current memory usage of the cgroup for this worker
    + */
    +public class CGroupMemoryUsage extends CGroupMetricsBase<Long> {
    +
    +    public CGroupMemoryUsage(Map<String, Object> conf) {
    +        super(conf, SubSystemType.memory);
    +    }
    +
    +    @Override
    +    public Long getDataFrom(CgroupCore core) throws Exception {
    --- End diff --
    
    same question 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.
---