You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2020/01/08 11:58:50 UTC

[GitHub] [incubator-doris] yangzhg opened a new pull request #2710: use cgroups memory limit and cpu cores in container

yangzhg opened a new pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710
 
 
   #2688 
   In docker container doris will get the memory of host not the memory of cgroups limit, this may cause oom

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364203466
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/filesystem.hpp>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "util/error_util.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using boost::algorithm::is_any_of;
+using boost::algorithm::split;
+using boost::algorithm::token_compress_on;
+using strings::CUnescape;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) continue;
+        vector<string> fields;
+        split(fields, line, is_any_of(":"));
 
 Review comment:
   You can try split in gutil which is easier to use

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364573968
 
 

 ##########
 File path: be/src/util/cgroup_util.h
 ##########
 @@ -0,0 +1,63 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef DORIS_BE_SRC_UTIL_CGROUPING_UTIL_H
+#define DORIS_BE_SRC_UTIL_CGROUPING_UTIL_H
 
 Review comment:
   ```suggestion
   #pragma once
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364575041
 
 

 ##########
 File path: be/src/util/mem_info.cpp
 ##########
 @@ -59,6 +60,11 @@ void MemInfo::init() {
         if (result == StringParser::PARSE_SUCCESS) {
             // Entries in /proc/meminfo are in KB.
             _s_physical_mem = mem_total_kb * 1024L;
+            int64_t cgroup_mem_limit;
 
 Review comment:
   seems better to move this out of while block.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364527789
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
 
 Review comment:
   some functions from impala

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364581437
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,229 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/split.h"
+#include "util/error_util.h"
+#include "util/file_utils.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using strings::CUnescape;
+using strings::Split;
+using strings::SkipWhitespace;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        // so field size will be 3
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) {
+            continue;
+        }
+        vector<string> fields = Split(line, ":");
+        DCHECK_GE(fields.size(), 3);
+        // ":" in the path does not appear to be escaped - bail in the unusual case that
+        // we get too many tokens.
+        if (fields.size() != 3) {
+            return Status::InvalidArgument(Substitute(
+                                               "Could not parse line from /proc/self/cgroup - had $0 > 3 tokens: '$1'",
+                                               fields.size(), line));
+        }
+        vector<string> subsystems = Split(fields[1], ",");
+        auto it = std::find(subsystems.begin(), subsystems.end(), subsystem);
+        if (it != subsystems.end()) {
+            *path = move(fields[2]);
+            return Status::OK();
+        }
+    }
+}
+
+static Status unescape_path(const string& escaped, string* unescaped) {
+    string err;
+    if (!CUnescape(escaped, unescaped, &err)) {
+        return Status::InvalidArgument(Substitute("Could not unescape path '$0': $1", escaped, err));
+    }
+    return Status::OK();
+}
+
+static Status read_cgroup_value(const string& limit_file_path, int64_t* val) {
+    ifstream limit_file(limit_file_path, ios::in);
+    string line;
+    getline(limit_file, line);
+    if (limit_file.fail() || limit_file.bad()) {
+        return Status::IOError(Substitute("Error reading $0: $1", limit_file_path, get_str_err_msg()));
+    }
+    StringParser::ParseResult pr;
+    // Parse into an an int64_t If it overflows, returning the max value of int64_t is ok because that
+    // is effectively unlimited.
+    *val = StringParser::string_to_int<int64_t>(line.c_str(), line.size(), &pr);
+    if ((pr != StringParser::PARSE_SUCCESS && pr != StringParser::PARSE_OVERFLOW)) {
+        return Status::InvalidArgument(
+                   Substitute("Failed to parse $0 as int64: '$1'", limit_file_path, line));
+    }
+    return Status::OK();
+}
+
+Status CGroupUtil::find_cgroup_mounts(
+    const string& subsystem, pair<string, string>* result) {
+    ifstream mountinfo("/proc/self/mountinfo", ios::in);
 
 Review comment:
   this is done automatically by the ifstream destructor.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364573882
 
 

 ##########
 File path: be/src/util/cgroup_util.h
 ##########
 @@ -0,0 +1,63 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef DORIS_BE_SRC_UTIL_CGROUPING_UTIL_H
+#define DORIS_BE_SRC_UTIL_CGROUPING_UTIL_H
+#include <cstdint>
+#include <string>
+#include <utility>
+
+#include "common/status.h"
+namespace doris {
+class CGroupUtil {
+  public:
 
 Review comment:
   ```suggestion
   public:
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364574598
 
 

 ##########
 File path: be/src/util/mem_info.cpp
 ##########
 @@ -59,6 +60,11 @@ void MemInfo::init() {
         if (result == StringParser::PARSE_SUCCESS) {
             // Entries in /proc/meminfo are in KB.
             _s_physical_mem = mem_total_kb * 1024L;
+            int64_t cgroup_mem_limit;
 
 Review comment:
   ```suggestion
               int64_t cgroup_mem_limit = 0;
   ```
   better to give a default value

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364203803
 
 

 ##########
 File path: be/src/util/cgroup_util.h
 ##########
 @@ -0,0 +1,62 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef DORIS_BE_SRC_UTIL_CGROUPING_UTIL_H
+#define DORIS_BE_SRC_UTIL_CGROUPING_UTIL_H
+#include <cstdint>
+#include <string>
+#include <utility>
+
+#include "common/status.h"
+namespace doris {
+class CGroupUtil {
+  public:
+    // Determines the CGroup memory limit from the current processes' cgroup.
+    // If the limit is more than INT64_MAX, INT64_MAX is returned (since that is
+    // effectively unlimited anyway). Does not take into account memory limits
+    // set on any ancestor CGroups.
+    static Status find_cgroup_mem_limit(int64_t* bytes);
+
+    // Determines the CGroup cpu cores limit from the current processes' cgroup.
+    static Status find_cgroup_cpu_limit(float* cpu_count);
+
+    // Returns a human-readable string with information about CGroups.
+    static std::string debug_string();
+
+    // detect if cgroup is enabled
+    static bool enable();
+
+  private:
+    static Status find_global_cgroup(const std::string& subsystem, std::string* path);
 
 Review comment:
   give some comments for this function? What does it do

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364209801
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/filesystem.hpp>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "util/error_util.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using boost::algorithm::is_any_of;
+using boost::algorithm::split;
+using boost::algorithm::token_compress_on;
+using strings::CUnescape;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) continue;
+        vector<string> fields;
+        split(fields, line, is_any_of(":"));
+        DCHECK_GE(fields.size(), 3);
+        // ":" in the path does not appear to be escaped - bail in the unusual case that
+        // we get too many tokens.
+        if (fields.size() > 3) {
+            return Status::InvalidArgument(Substitute(
+                                               "Could not parse line from /proc/self/cgroup - had $0 > 3 tokens: '$1'",
+                                               fields.size(), line));
+        }
+        vector<string> subsystems;
+        split(subsystems, fields[1], is_any_of(","));
+        auto it = std::find(subsystems.begin(), subsystems.end(), subsystem);
+        if (it != subsystems.end()) {
+            *path = move(fields[2]);
+            return Status::OK();
+        }
+    }
+}
+
+static Status unescape_path(const string& escaped, string* unescaped) {
+    string err;
+    if (!CUnescape(escaped, unescaped, &err)) {
+        return Status::InvalidArgument(Substitute("Could not unescape path '$0': $1", escaped, err));
+    }
+    return Status::OK();
+}
+
+static Status read_cgroup_value(const string &limit_file_path, int64_t* val) {
 
 Review comment:
   give some comments for this function.
   ```suggestion
   static Status read_cgroup_value(const string& limit_file_path, int64_t* val) {
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364573348
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,229 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/split.h"
+#include "util/error_util.h"
+#include "util/file_utils.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using strings::CUnescape;
+using strings::Split;
+using strings::SkipWhitespace;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        // so field size will be 3
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) {
+            continue;
+        }
+        vector<string> fields = Split(line, ":");
+        DCHECK_GE(fields.size(), 3);
+        // ":" in the path does not appear to be escaped - bail in the unusual case that
+        // we get too many tokens.
+        if (fields.size() != 3) {
+            return Status::InvalidArgument(Substitute(
+                                               "Could not parse line from /proc/self/cgroup - had $0 > 3 tokens: '$1'",
+                                               fields.size(), line));
+        }
+        vector<string> subsystems = Split(fields[1], ",");
+        auto it = std::find(subsystems.begin(), subsystems.end(), subsystem);
+        if (it != subsystems.end()) {
+            *path = move(fields[2]);
+            return Status::OK();
+        }
+    }
+}
+
+static Status unescape_path(const string& escaped, string* unescaped) {
+    string err;
+    if (!CUnescape(escaped, unescaped, &err)) {
+        return Status::InvalidArgument(Substitute("Could not unescape path '$0': $1", escaped, err));
+    }
+    return Status::OK();
+}
+
+static Status read_cgroup_value(const string& limit_file_path, int64_t* val) {
+    ifstream limit_file(limit_file_path, ios::in);
+    string line;
+    getline(limit_file, line);
+    if (limit_file.fail() || limit_file.bad()) {
+        return Status::IOError(Substitute("Error reading $0: $1", limit_file_path, get_str_err_msg()));
+    }
+    StringParser::ParseResult pr;
+    // Parse into an an int64_t If it overflows, returning the max value of int64_t is ok because that
+    // is effectively unlimited.
+    *val = StringParser::string_to_int<int64_t>(line.c_str(), line.size(), &pr);
+    if ((pr != StringParser::PARSE_SUCCESS && pr != StringParser::PARSE_OVERFLOW)) {
+        return Status::InvalidArgument(
+                   Substitute("Failed to parse $0 as int64: '$1'", limit_file_path, line));
+    }
+    return Status::OK();
+}
+
+Status CGroupUtil::find_cgroup_mounts(
+    const string& subsystem, pair<string, string>* result) {
+    ifstream mountinfo("/proc/self/mountinfo", ios::in);
+    string line;
+    while (true) {
+        if (mountinfo.fail() || mountinfo.bad()) {
+            return Status::IOError(Substitute("Error reading /proc/self/mountinfo: $0", get_str_err_msg()));
+        } else if (mountinfo.eof()) {
+            return Status::NotFound(
+                       Substitute("Could not find subsystem $0 in /proc/self/mountinfo", subsystem));
+        }
+        // The relevant lines look like below (see proc manpage for full documentation). The
+        // first example is running outside of a container, the second example is running
+        // inside a docker container. Field 3 is the path relative to the root CGroup on
+        // the host and Field 4 is the mount point from this process's point of view.
+        // 34 29 0:28 / /sys/fs/cgroup/memory rw,nosuid,nodev,noexec,relatime shared:15 -
+        //    cgroup cgroup rw,memory
+        // 275 271 0:28 /docker/f23eee6f88c2ba99fcce /sys/fs/cgroup/memory
+        //    ro,nosuid,nodev,noexec,relatime master:15 - cgroup cgroup rw,memory
+        getline(mountinfo, line);
+        if (!mountinfo.good()) continue;
+        vector<string> fields = Split(line, " ", SkipWhitespace());
+        DCHECK_GE(fields.size(), 7);
 
 Review comment:
   No DCHECK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364571178
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,229 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/split.h"
+#include "util/error_util.h"
+#include "util/file_utils.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using strings::CUnescape;
+using strings::Split;
+using strings::SkipWhitespace;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        // so field size will be 3
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) {
+            continue;
+        }
+        vector<string> fields = Split(line, ":");
+        DCHECK_GE(fields.size(), 3);
 
 Review comment:
   Better to remove this, because we should not assert the input content.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] yangzhg commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
yangzhg commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364529002
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/filesystem.hpp>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "util/error_util.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using boost::algorithm::is_any_of;
+using boost::algorithm::split;
+using boost::algorithm::token_compress_on;
+using strings::CUnescape;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) continue;
+        vector<string> fields;
+        split(fields, line, is_any_of(":"));
+        DCHECK_GE(fields.size(), 3);
+        // ":" in the path does not appear to be escaped - bail in the unusual case that
+        // we get too many tokens.
+        if (fields.size() > 3) {
 
 Review comment:
   line formate like `4:memory:/user.slice` split by `:`, so size  is 3

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364206192
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
 
 Review comment:
   Is this file written by yourself? If it is not, is it OK to use this license?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364202784
 
 

 ##########
 File path: be/src/util/mem_info.cpp
 ##########
 @@ -59,6 +60,11 @@ void MemInfo::init() {
         if (result == StringParser::PARSE_SUCCESS) {
             // Entries in /proc/meminfo are in KB.
             _s_physical_mem = mem_total_kb * 1024L;
+	    int64_t cgroup_mem_limit;
 
 Review comment:
   there may be tab

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay merged pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay merged pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364206933
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/filesystem.hpp>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "util/error_util.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using boost::algorithm::is_any_of;
+using boost::algorithm::split;
+using boost::algorithm::token_compress_on;
+using strings::CUnescape;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) continue;
+        vector<string> fields;
+        split(fields, line, is_any_of(":"));
+        DCHECK_GE(fields.size(), 3);
+        // ":" in the path does not appear to be escaped - bail in the unusual case that
+        // we get too many tokens.
+        if (fields.size() > 3) {
 
 Review comment:
   != 3?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364209627
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/filesystem.hpp>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "util/error_util.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using boost::algorithm::is_any_of;
+using boost::algorithm::split;
+using boost::algorithm::token_compress_on;
+using strings::CUnescape;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) continue;
+        vector<string> fields;
+        split(fields, line, is_any_of(":"));
+        DCHECK_GE(fields.size(), 3);
+        // ":" in the path does not appear to be escaped - bail in the unusual case that
+        // we get too many tokens.
+        if (fields.size() > 3) {
+            return Status::InvalidArgument(Substitute(
+                                               "Could not parse line from /proc/self/cgroup - had $0 > 3 tokens: '$1'",
+                                               fields.size(), line));
+        }
+        vector<string> subsystems;
+        split(subsystems, fields[1], is_any_of(","));
+        auto it = std::find(subsystems.begin(), subsystems.end(), subsystem);
+        if (it != subsystems.end()) {
+            *path = move(fields[2]);
+            return Status::OK();
+        }
+    }
+}
+
+static Status unescape_path(const string& escaped, string* unescaped) {
+    string err;
+    if (!CUnescape(escaped, unescaped, &err)) {
+        return Status::InvalidArgument(Substitute("Could not unescape path '$0': $1", escaped, err));
+    }
+    return Status::OK();
+}
+
+static Status read_cgroup_value(const string &limit_file_path, int64_t* val) {
+    ifstream limit_file(limit_file_path, ios::in);
+    string line;
+    getline(limit_file, line);
+    if (limit_file.fail() || limit_file.bad()) {
+        return Status::IOError(Substitute("Error reading $0: $1", limit_file_path, get_str_err_msg()));
+    }
+    StringParser::ParseResult pr;
+    // Parse into an an int64_t If it overflows, returning the max value of int64_t is ok because that
+    // is effectively unlimited.
+    *val = StringParser::string_to_int<int64_t>(line.c_str(), line.size(), &pr);
+    if ((pr != StringParser::PARSE_SUCCESS && pr != StringParser::PARSE_OVERFLOW)) {
+        return Status::InvalidArgument(
+                   Substitute("Failed to parse $0 as int64: '$1'", limit_file_path, line));
+    }
+    return Status::OK();
+}
+
+Status CGroupUtil::find_cgroup_mounts(
+    const string& subsystem, pair<string, string>* result) {
+    ifstream mountinfo("/proc/self/mountinfo", ios::in);
+    string line;
+    while (true) {
+        if (mountinfo.fail() || mountinfo.bad()) {
+            return Status::IOError(Substitute("Error reading /proc/self/mountinfo: $0", get_str_err_msg()));
+        } else if (mountinfo.eof()) {
+            return Status::NotFound(
+                       Substitute("Could not find subsystem $0 in /proc/self/mountinfo", subsystem));
+        }
+        // The relevant lines look like below (see proc manpage for full documentation). The
+        // first example is running outside of a container, the second example is running
+        // inside a docker container. Field 3 is the path relative to the root CGroup on
+        // the host and Field 4 is the mount point from this process's point of view.
+        // 34 29 0:28 / /sys/fs/cgroup/memory rw,nosuid,nodev,noexec,relatime shared:15 -
+        //    cgroup cgroup rw,memory
+        // 275 271 0:28 /docker/f23eee6f88c2ba99fcce /sys/fs/cgroup/memory
+        //    ro,nosuid,nodev,noexec,relatime master:15 - cgroup cgroup rw,memory
+        getline(mountinfo, line);
+        if (!mountinfo.good()) continue;
+        vector<string> fields;
+        split(fields, line, is_any_of(" "), token_compress_on);
+        DCHECK_GE(fields.size(), 7);
+
+        if (fields[fields.size() - 3] != "cgroup") continue;
+        // This is a cgroup mount. Check if it's the mount we're looking for.
+        vector<string> cgroup_opts;
+        split(cgroup_opts, fields[fields.size() - 1], is_any_of(","), token_compress_on);
+        auto it = std::find(cgroup_opts.begin(), cgroup_opts.end(), subsystem);
+        if (it == cgroup_opts.end()) continue;
+        // This is the right mount.
+        string mount_path, system_path;
+        RETURN_IF_ERROR(unescape_path(fields[4], &mount_path));
+        RETURN_IF_ERROR(unescape_path(fields[3], &system_path));
+        // Strip trailing "/" so that both returned paths match in whether they have a
+        // trailing "/".
+        if (system_path[system_path.size() - 1] == '/') system_path.pop_back();
+        *result = {mount_path, system_path};
+        return Status::OK();
+    }
+}
+
+Status CGroupUtil::find_abs_cgroup_path(const string& subsystem, string* path) {
+    RETURN_IF_ERROR(find_global_cgroup(subsystem, path));
+    pair<string, string> paths;
+    RETURN_IF_ERROR(find_cgroup_mounts(subsystem, &paths));
+    const string& mount_path = paths.first;
+    const string& system_path = paths.second;
+    if (path->compare(0, system_path.size(), system_path) != 0) {
+        return Status::InvalidArgument(
+                   Substitute("Expected CGroup path '$0' to start with '$1'",
+                              *path, system_path));
+    }
+    path->replace(0, system_path.size(), mount_path);
+    return Status::OK();
+}
+
+Status CGroupUtil::find_cgroup_mem_limit(int64_t* bytes) {
+    if (!enable()) {
+        *bytes = INT64_MAX;
+        return Status::InvalidArgument("cgroup is not enabled!");
 
 Review comment:
   If return error, why do you change the return value?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364575310
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,229 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "gutil/strings/split.h"
+#include "util/error_util.h"
+#include "util/file_utils.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using strings::CUnescape;
+using strings::Split;
+using strings::SkipWhitespace;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        // so field size will be 3
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) {
+            continue;
+        }
+        vector<string> fields = Split(line, ":");
+        DCHECK_GE(fields.size(), 3);
+        // ":" in the path does not appear to be escaped - bail in the unusual case that
+        // we get too many tokens.
+        if (fields.size() != 3) {
+            return Status::InvalidArgument(Substitute(
+                                               "Could not parse line from /proc/self/cgroup - had $0 > 3 tokens: '$1'",
+                                               fields.size(), line));
+        }
+        vector<string> subsystems = Split(fields[1], ",");
+        auto it = std::find(subsystems.begin(), subsystems.end(), subsystem);
+        if (it != subsystems.end()) {
+            *path = move(fields[2]);
+            return Status::OK();
+        }
+    }
+}
+
+static Status unescape_path(const string& escaped, string* unescaped) {
+    string err;
+    if (!CUnescape(escaped, unescaped, &err)) {
+        return Status::InvalidArgument(Substitute("Could not unescape path '$0': $1", escaped, err));
+    }
+    return Status::OK();
+}
+
+static Status read_cgroup_value(const string& limit_file_path, int64_t* val) {
+    ifstream limit_file(limit_file_path, ios::in);
+    string line;
+    getline(limit_file, line);
+    if (limit_file.fail() || limit_file.bad()) {
+        return Status::IOError(Substitute("Error reading $0: $1", limit_file_path, get_str_err_msg()));
+    }
+    StringParser::ParseResult pr;
+    // Parse into an an int64_t If it overflows, returning the max value of int64_t is ok because that
+    // is effectively unlimited.
+    *val = StringParser::string_to_int<int64_t>(line.c_str(), line.size(), &pr);
+    if ((pr != StringParser::PARSE_SUCCESS && pr != StringParser::PARSE_OVERFLOW)) {
+        return Status::InvalidArgument(
+                   Substitute("Failed to parse $0 as int64: '$1'", limit_file_path, line));
+    }
+    return Status::OK();
+}
+
+Status CGroupUtil::find_cgroup_mounts(
+    const string& subsystem, pair<string, string>* result) {
+    ifstream mountinfo("/proc/self/mountinfo", ios::in);
 
 Review comment:
   Is there any resouce leak if ifstream.close is not called?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #2710: use cgroups memory limit and cpu cores in container
URL: https://github.com/apache/incubator-doris/pull/2710#discussion_r364207522
 
 

 ##########
 File path: be/src/util/cgroup_util.cpp
 ##########
 @@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/cgroup_util.h"
+
+#include <algorithm>
+#include <fstream>
+#include <iostream>
+#include <utility>
+
+#include <boost/algorithm/string.hpp>
+#include <boost/filesystem.hpp>
+
+#include "gutil/strings/escaping.h"
+#include "gutil/strings/substitute.h"
+#include "util/error_util.h"
+#include "util/string_parser.hpp"
+
+#include "common/names.h"
+
+using boost::algorithm::is_any_of;
+using boost::algorithm::split;
+using boost::algorithm::token_compress_on;
+using strings::CUnescape;
+using std::pair;
+
+namespace doris {
+
+Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
+    ifstream proc_cgroups("/proc/self/cgroup", ios::in);
+    string line;
+    while (true) {
+        if (proc_cgroups.fail()) {
+            return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+        } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
+            return Status::NotFound(Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+        }
+        // The line format looks like this:
+        // 4:memory:/user.slice
+        // 9:cpu,cpuacct:/user.slice
+        getline(proc_cgroups, line);
+        if (!proc_cgroups.good()) continue;
+        vector<string> fields;
+        split(fields, line, is_any_of(":"));
+        DCHECK_GE(fields.size(), 3);
+        // ":" in the path does not appear to be escaped - bail in the unusual case that
+        // we get too many tokens.
+        if (fields.size() > 3) {
+            return Status::InvalidArgument(Substitute(
+                                               "Could not parse line from /proc/self/cgroup - had $0 > 3 tokens: '$1'",
+                                               fields.size(), line));
+        }
+        vector<string> subsystems;
+        split(subsystems, fields[1], is_any_of(","));
+        auto it = std::find(subsystems.begin(), subsystems.end(), subsystem);
+        if (it != subsystems.end()) {
+            *path = move(fields[2]);
+            return Status::OK();
+        }
+    }
+}
+
+static Status unescape_path(const string& escaped, string* unescaped) {
+    string err;
+    if (!CUnescape(escaped, unescaped, &err)) {
+        return Status::InvalidArgument(Substitute("Could not unescape path '$0': $1", escaped, err));
+    }
+    return Status::OK();
+}
+
+static Status read_cgroup_value(const string &limit_file_path, int64_t* val) {
+    ifstream limit_file(limit_file_path, ios::in);
+    string line;
+    getline(limit_file, line);
+    if (limit_file.fail() || limit_file.bad()) {
+        return Status::IOError(Substitute("Error reading $0: $1", limit_file_path, get_str_err_msg()));
+    }
+    StringParser::ParseResult pr;
+    // Parse into an an int64_t If it overflows, returning the max value of int64_t is ok because that
+    // is effectively unlimited.
+    *val = StringParser::string_to_int<int64_t>(line.c_str(), line.size(), &pr);
+    if ((pr != StringParser::PARSE_SUCCESS && pr != StringParser::PARSE_OVERFLOW)) {
+        return Status::InvalidArgument(
+                   Substitute("Failed to parse $0 as int64: '$1'", limit_file_path, line));
+    }
+    return Status::OK();
+}
+
+Status CGroupUtil::find_cgroup_mounts(
+    const string& subsystem, pair<string, string>* result) {
+    ifstream mountinfo("/proc/self/mountinfo", ios::in);
+    string line;
+    while (true) {
+        if (mountinfo.fail() || mountinfo.bad()) {
+            return Status::IOError(Substitute("Error reading /proc/self/mountinfo: $0", get_str_err_msg()));
+        } else if (mountinfo.eof()) {
+            return Status::NotFound(
+                       Substitute("Could not find subsystem $0 in /proc/self/mountinfo", subsystem));
+        }
+        // The relevant lines look like below (see proc manpage for full documentation). The
+        // first example is running outside of a container, the second example is running
+        // inside a docker container. Field 3 is the path relative to the root CGroup on
+        // the host and Field 4 is the mount point from this process's point of view.
+        // 34 29 0:28 / /sys/fs/cgroup/memory rw,nosuid,nodev,noexec,relatime shared:15 -
+        //    cgroup cgroup rw,memory
+        // 275 271 0:28 /docker/f23eee6f88c2ba99fcce /sys/fs/cgroup/memory
+        //    ro,nosuid,nodev,noexec,relatime master:15 - cgroup cgroup rw,memory
+        getline(mountinfo, line);
+        if (!mountinfo.good()) continue;
+        vector<string> fields;
+        split(fields, line, is_any_of(" "), token_compress_on);
+        DCHECK_GE(fields.size(), 7);
+
+        if (fields[fields.size() - 3] != "cgroup") continue;
+        // This is a cgroup mount. Check if it's the mount we're looking for.
+        vector<string> cgroup_opts;
+        split(cgroup_opts, fields[fields.size() - 1], is_any_of(","), token_compress_on);
+        auto it = std::find(cgroup_opts.begin(), cgroup_opts.end(), subsystem);
+        if (it == cgroup_opts.end()) continue;
+        // This is the right mount.
+        string mount_path, system_path;
+        RETURN_IF_ERROR(unescape_path(fields[4], &mount_path));
+        RETURN_IF_ERROR(unescape_path(fields[3], &system_path));
+        // Strip trailing "/" so that both returned paths match in whether they have a
+        // trailing "/".
+        if (system_path[system_path.size() - 1] == '/') system_path.pop_back();
+        *result = {mount_path, system_path};
+        return Status::OK();
+    }
+}
+
+Status CGroupUtil::find_abs_cgroup_path(const string& subsystem, string* path) {
+    RETURN_IF_ERROR(find_global_cgroup(subsystem, path));
+    pair<string, string> paths;
+    RETURN_IF_ERROR(find_cgroup_mounts(subsystem, &paths));
+    const string& mount_path = paths.first;
+    const string& system_path = paths.second;
+    if (path->compare(0, system_path.size(), system_path) != 0) {
+        return Status::InvalidArgument(
+                   Substitute("Expected CGroup path '$0' to start with '$1'",
+                              *path, system_path));
+    }
+    path->replace(0, system_path.size(), mount_path);
+    return Status::OK();
+}
+
+Status CGroupUtil::find_cgroup_mem_limit(int64_t* bytes) {
+    if (!enable()) {
+        *bytes = INT64_MAX;
+        return Status::InvalidArgument("cgroup is not enabled!");
+    }
+    string cgroup_path;
+    RETURN_IF_ERROR(find_abs_cgroup_path("memory", &cgroup_path));
+    string limit_file_path = cgroup_path + "/memory.limit_in_bytes";
+    return read_cgroup_value(limit_file_path, bytes);
+}
+Status CGroupUtil::find_cgroup_cpu_limit(float* cpu_count) {
+    if (!enable()) {
+        *cpu_count = FLT_MAX;
+        return Status::InvalidArgument("cgroup is not enabled!");
+    }
+    int64_t quota;
+    int64_t period;
+    string cgroup_path;
+    if (!find_abs_cgroup_path("cpu", &cgroup_path).ok()) {
+        RETURN_IF_ERROR(find_abs_cgroup_path("cpuacct", &cgroup_path));
+    }
+    string cfs_quota_filename = cgroup_path + "/cpu.cfs_quota_us";
+    RETURN_IF_ERROR(read_cgroup_value(cfs_quota_filename, &quota));
+    if (quota <= 0) {
+        *cpu_count = -1;
+        return Status::OK();
+    }
+    string cfs_period_filename = cgroup_path + "/cfs_period_us";
+    RETURN_IF_ERROR(read_cgroup_value(cfs_period_filename, &period));
+    if (quota <= period) {
+        *cpu_count = -1;
+        return Status::InvalidArgument("quota <= period");
+    }
+    *cpu_count = float(quota) / float(period);
+    if (*cpu_count >= FLT_MAX) {
+        *cpu_count = -1;
+        return Status::InvalidArgument("unknown");
+    }
+    return Status::OK();
+}
+
+std::string CGroupUtil::debug_string() {
+    if (!enable()) {
+        return std::string("cgroup is not enabled!");
+    }
+    string mem_limit_str;
+    int64_t mem_limit;
+    Status status = find_cgroup_mem_limit(&mem_limit);
+    if (status.ok()) {
+        mem_limit_str = Substitute("$0", mem_limit);
+    } else {
+        mem_limit_str = status.get_error_msg();
+    }
+    string cpu_limit_str;
+    float cpu_limit;
+    status = find_cgroup_cpu_limit(&cpu_limit);
+    if (status.ok()) {
+        if (cpu_limit >0) {
+            std::stringstream stream;
+            stream << std::fixed << std::setprecision(1) << cpu_limit;
+            cpu_limit_str = stream.str();
+        } else {
+            cpu_limit_str = "unlimited";
+        }
+    } else {
+        cpu_limit_str = status.get_error_msg();
+    }
+    return Substitute("Process CGroup Info: memory.limit_in_bytes=$0, cpu cfs limits: $1", mem_limit_str, cpu_limit_str);
+}
+
+bool CGroupUtil::enable() {
+    return boost::filesystem::exists("/proc/cgroups");
 
 Review comment:
   better to use FileUtils

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org