You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2016/08/20 16:34:15 UTC

mesos git commit: Implemented `CpuacctSubsystem`.

Repository: mesos
Updated Branches:
  refs/heads/master 6698cf866 -> ee4f8a251


Implemented `CpuacctSubsystem`.

Review: https://reviews.apache.org/r/49850/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/ee4f8a25
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/ee4f8a25
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/ee4f8a25

Branch: refs/heads/master
Commit: ee4f8a251f9b5d8be222f7033610059b2485e039
Parents: 6698cf8
Author: haosdent huang <ha...@gmail.com>
Authored: Sat Aug 20 09:30:48 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Sat Aug 20 09:34:07 2016 -0700

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   2 +
 .../mesos/isolators/cgroups/cgroups.cpp         |   1 +
 .../mesos/isolators/cgroups/constants.hpp       |   1 +
 .../mesos/isolators/cgroups/subsystem.cpp       |   2 +
 .../isolators/cgroups/subsystems/cpuacct.cpp    | 118 +++++++++++++++++++
 .../isolators/cgroups/subsystems/cpuacct.hpp    |  63 ++++++++++
 7 files changed, 188 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ee4f8a25/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index a83dde9..85eb2b8 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -168,6 +168,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/cgroups/perf_event.cpp
   slave/containerizer/mesos/isolators/cgroups/subsystem.cpp
   slave/containerizer/mesos/isolators/cgroups/subsystems/cpu.cpp
+  slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.cpp
   slave/containerizer/mesos/isolators/docker/runtime.cpp
   slave/containerizer/mesos/isolators/docker/volume/driver.cpp
   slave/containerizer/mesos/isolators/docker/volume/isolator.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/ee4f8a25/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index bfda83d..d0f937d 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1052,6 +1052,7 @@ MESOS_LINUX_FILES =							\
   slave/containerizer/mesos/isolators/cgroups/perf_event.cpp		\
   slave/containerizer/mesos/isolators/cgroups/subsystem.cpp		\
   slave/containerizer/mesos/isolators/cgroups/subsystems/cpu.cpp	\
+  slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.cpp	\
   slave/containerizer/mesos/isolators/docker/runtime.cpp		\
   slave/containerizer/mesos/isolators/docker/volume/isolator.cpp	\
   slave/containerizer/mesos/isolators/filesystem/linux.cpp		\
@@ -1086,6 +1087,7 @@ MESOS_LINUX_FILES +=							\
   slave/containerizer/mesos/isolators/cgroups/perf_event.hpp		\
   slave/containerizer/mesos/isolators/cgroups/subsystem.hpp		\
   slave/containerizer/mesos/isolators/cgroups/subsystems/cpu.hpp	\
+  slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.hpp	\
   slave/containerizer/mesos/isolators/docker/runtime.hpp		\
   slave/containerizer/mesos/isolators/docker/volume/isolator.hpp	\
   slave/containerizer/mesos/isolators/filesystem/linux.hpp		\

http://git-wip-us.apache.org/repos/asf/mesos/blob/ee4f8a25/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
index cae642a..1b84a56 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
@@ -76,6 +76,7 @@ Try<Isolator*> CgroupsIsolatorProcess::create(const Flags& flags)
   // Multimap: isolator name -> subsystem name.
   multihashmap<string, string> isolatorMap = {
     {"cpu", CGROUP_SUBSYSTEM_CPU_NAME},
+    {"cpu", CGROUP_SUBSYSTEM_CPUACCT_NAME},
   };
 
   foreach (string isolator, strings::tokenize(flags.isolation, ",")) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ee4f8a25/src/slave/containerizer/mesos/isolators/cgroups/constants.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/constants.hpp b/src/slave/containerizer/mesos/isolators/cgroups/constants.hpp
index cc402b4..3146cc6 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/constants.hpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/constants.hpp
@@ -40,6 +40,7 @@ const Bytes MIN_MEMORY = Megabytes(32);
 
 // Subsystem names.
 const std::string CGROUP_SUBSYSTEM_CPU_NAME = "cpu";
+const std::string CGROUP_SUBSYSTEM_CPUACCT_NAME = "cpuacct";
 
 } // namespace slave {
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ee4f8a25/src/slave/containerizer/mesos/isolators/cgroups/subsystem.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/subsystem.cpp b/src/slave/containerizer/mesos/isolators/cgroups/subsystem.cpp
index 767c56d..9789cf6 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/subsystem.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/subsystem.cpp
@@ -21,6 +21,7 @@
 #include "slave/containerizer/mesos/isolators/cgroups/subsystem.hpp"
 
 #include "slave/containerizer/mesos/isolators/cgroups/subsystems/cpu.hpp"
+#include "slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.hpp"
 
 using process::Future;
 using process::Owned;
@@ -39,6 +40,7 @@ Try<Owned<Subsystem>> Subsystem::create(
   hashmap<string, Try<Owned<Subsystem>>(*)(const Flags&, const string&)>
     creators = {
     {CGROUP_SUBSYSTEM_CPU_NAME, &CpuSubsystem::create},
+    {CGROUP_SUBSYSTEM_CPUACCT_NAME, &CpuacctSubsystem::create},
   };
 
   if (!creators.contains(name)) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ee4f8a25/src/slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.cpp b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.cpp
new file mode 100644
index 0000000..a35ecb0
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.cpp
@@ -0,0 +1,118 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <process/id.hpp>
+
+#include "linux/cgroups.hpp"
+
+#include "slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.hpp"
+
+using process::Failure;
+using process::Future;
+using process::Owned;
+
+using std::set;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<Owned<Subsystem>> CpuacctSubsystem::create(
+    const Flags& flags,
+    const string& hierarchy)
+{
+  return Owned<Subsystem>(new CpuacctSubsystem(flags, hierarchy));
+}
+
+
+CpuacctSubsystem::CpuacctSubsystem(
+    const Flags& _flags,
+    const string& _hierarchy)
+  : ProcessBase(process::ID::generate("cgroups-cpuacct-subsystem")),
+    Subsystem(_flags, _hierarchy) {}
+
+
+Future<ResourceStatistics> CpuacctSubsystem::usage(
+    const ContainerID& containerId)
+{
+  ResourceStatistics result;
+
+  // TODO(chzhcn): Getting the number of processes and threads is
+  // available as long as any cgroup subsystem is used so this best
+  // not be tied to a specific cgroup subsystem. A better place is
+  // probably Linux Launcher, which uses the cgroup freezer subsystem.
+  // That requires some change for it to adopt the new semantics of
+  // reporting subsystem-independent cgroup usage.
+  // Note: The complexity of this operation is linear to the number of
+  // processes and threads in a container: the kernel has to allocate
+  // memory to contain the list of pids or tids; the userspace has to
+  // parse the cgroup files to get the size. If this proves to be a
+  // performance bottleneck, some kind of rate limiting mechanism
+  // needs to be employed.
+  if (flags.cgroups_cpu_enable_pids_and_tids_count) {
+    Try<set<pid_t>> pids = cgroups::processes(
+        hierarchy,
+        path::join(flags.cgroups_root, containerId.value()));
+
+    if (pids.isError()) {
+      return Failure("Failed to get number of processes: " + pids.error());
+    }
+
+    result.set_processes(pids.get().size());
+
+    Try<set<pid_t>> tids = cgroups::threads(
+        hierarchy,
+        path::join(flags.cgroups_root, containerId.value()));
+
+    if (tids.isError()) {
+      return Failure("Failed to get number of threads: " + tids.error());
+    }
+
+    result.set_threads(tids.get().size());
+  }
+
+  // Get the number of clock ticks, used for cpu accounting.
+  static long ticks = sysconf(_SC_CLK_TCK);
+
+  PCHECK(ticks > 0) << "Failed to get sysconf(_SC_CLK_TCK)";
+
+  // Add the cpuacct.stat information.
+  Try<hashmap<string, uint64_t>> stat = cgroups::stat(
+      hierarchy,
+      path::join(flags.cgroups_root, containerId.value()),
+      "cpuacct.stat");
+
+  if (stat.isError()) {
+    return Failure("Failed to read 'cpuacct.stat': " + stat.error());
+  }
+
+  // TODO(bmahler): Add namespacing to cgroups to enforce the expected
+  // structure, e.g., cgroups::cpuacct::stat.
+  Option<uint64_t> user = stat.get().get("user");
+  Option<uint64_t> system = stat.get().get("system");
+
+  if (user.isSome() && system.isSome()) {
+    result.set_cpus_user_time_secs((double) user.get() / (double) ticks);
+    result.set_cpus_system_time_secs((double) system.get() / (double) ticks);
+  }
+
+  return result;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ee4f8a25/src/slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.hpp b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.hpp
new file mode 100644
index 0000000..10fbbc9
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/cpuacct.hpp
@@ -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 __CGROUPS_ISOLATOR_SUBSYSTEMS_CPUACCT_HPP__
+#define __CGROUPS_ISOLATOR_SUBSYSTEMS_CPUACCT_HPP__
+
+#include <string>
+
+#include <process/owned.hpp>
+
+#include <stout/try.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolators/cgroups/constants.hpp"
+#include "slave/containerizer/mesos/isolators/cgroups/subsystem.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+/**
+ * Represent cgroups cpuacct subsystem.
+ */
+class CpuacctSubsystem : public Subsystem
+{
+public:
+  static Try<process::Owned<Subsystem>> create(
+      const Flags& flags,
+      const std::string& hierarchy);
+
+  virtual ~CpuacctSubsystem() {}
+
+  virtual std::string name() const
+  {
+    return CGROUP_SUBSYSTEM_CPUACCT_NAME;
+  }
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+private:
+  CpuacctSubsystem(const Flags& flags, const std::string& hierarchy);
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __CGROUPS_ISOLATOR_SUBSYSTEMS_CPUACCT_HPP__