You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by qi...@apache.org on 2019/08/23 07:42:45 UTC

[mesos] 01/02: Used cached cgroups for updating resources in Docker containerizer.

This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.8.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit ee01c8d479b34ced35ee6bd172108a128086277e
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Wed Aug 21 16:50:06 2019 +0800

    Used cached cgroups for updating resources in Docker containerizer.
    
    Review: https://reviews.apache.org/r/71335
---
 src/slave/containerizer/docker.cpp                 | 125 ++++++++++++---------
 src/slave/containerizer/docker.hpp                 |   8 +-
 .../containerizer/docker_containerizer_tests.cpp   |   2 +-
 3 files changed, 76 insertions(+), 59 deletions(-)

diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index e4ad945..2a9b2ff 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -1693,9 +1693,9 @@ Future<Nothing> DockerContainerizerProcess::update(
     return Nothing();
   }
 
-  // Skip inspecting the docker container if we already have the pid.
-  if (container->pid.isSome()) {
-    return __update(containerId, _resources, container->pid.get());
+  // Skip inspecting the docker container if we already have the cgroups.
+  if (container->cpuCgroup.isSome() && container->memoryCgroup.isSome()) {
+    return __update(containerId, _resources);
   }
 
   string containerName = containers_.at(containerId)->containerName;
@@ -1746,6 +1746,7 @@ Future<Nothing> DockerContainerizerProcess::update(
 }
 
 
+#ifdef __linux__
 Future<Nothing> DockerContainerizerProcess::_update(
     const ContainerID& containerId,
     const Resources& _resources,
@@ -1763,23 +1764,6 @@ Future<Nothing> DockerContainerizerProcess::_update(
 
   containers_.at(containerId)->pid = container.pid.get();
 
-  return __update(containerId, _resources, container.pid.get());
-}
-
-
-Future<Nothing> DockerContainerizerProcess::__update(
-    const ContainerID& containerId,
-    const Resources& _resources,
-    pid_t pid)
-{
-#ifdef __linux__
-  // Determine the cgroups hierarchies where the 'cpu' and
-  // 'memory' subsystems are mounted (they may be the same). Note that
-  // we make these static so we can reuse the result for subsequent
-  // calls.
-  static Result<string> cpuHierarchy = cgroups::hierarchy("cpu");
-  static Result<string> memoryHierarchy = cgroups::hierarchy("memory");
-
   // NOTE: Normally, a Docker container should be in its own cgroup.
   // However, a zombie process (exited but not reaped) will be
   // temporarily moved into the system root cgroup. We add some
@@ -1787,18 +1771,6 @@ Future<Nothing> DockerContainerizerProcess::__update(
   // in the root cgroup. See MESOS-8480 for details.
   const string systemRootCgroup = stringify(os::PATH_SEPARATOR);
 
-  if (cpuHierarchy.isError()) {
-    return Failure("Failed to determine the cgroup hierarchy "
-                   "where the 'cpu' subsystem is mounted: " +
-                   cpuHierarchy.error());
-  }
-
-  if (memoryHierarchy.isError()) {
-    return Failure("Failed to determine the cgroup hierarchy "
-                   "where the 'memory' subsystem is mounted: " +
-                   memoryHierarchy.error());
-  }
-
   // We need to find the cgroup(s) this container is currently running
   // in for both the hierarchy with the 'cpu' subsystem attached and
   // the hierarchy with the 'memory' subsystem attached so we can
@@ -1806,8 +1778,7 @@ Future<Nothing> DockerContainerizerProcess::__update(
 
   // Determine the cgroup for the 'cpu' subsystem (based on the
   // container's pid).
-  Result<string> cpuCgroup = cgroups::cpu::cgroup(pid);
-
+  Result<string> cpuCgroup = cgroups::cpu::cgroup(container.pid.get());
   if (cpuCgroup.isError()) {
     return Failure("Failed to determine cgroup for the 'cpu' subsystem: " +
                    cpuCgroup.error());
@@ -1817,14 +1788,73 @@ Future<Nothing> DockerContainerizerProcess::__update(
                  << " where the 'cpu' subsystem is mounted";
   } else if (cpuCgroup.get() == systemRootCgroup) {
     LOG(WARNING)
-        << "Process '" << pid
+        << "Process '" << container.pid.get()
+        << "' should not be in the system root cgroup (being destroyed?)";
+  } else {
+    // Cache the CPU cgroup.
+    containers_.at(containerId)->cpuCgroup = cpuCgroup.get();
+  }
+
+  // Now determine the cgroup for the 'memory' subsystem.
+  Result<string> memoryCgroup = cgroups::memory::cgroup(container.pid.get());
+  if (memoryCgroup.isError()) {
+    return Failure("Failed to determine cgroup for the 'memory' subsystem: " +
+                   memoryCgroup.error());
+  } else if (memoryCgroup.isNone()) {
+    LOG(WARNING) << "Container " << containerId
+                 << " does not appear to be a member of a cgroup"
+                 << " where the 'memory' subsystem is mounted";
+  } else if (memoryCgroup.get() == systemRootCgroup) {
+    LOG(WARNING)
+        << "Process '" << container.pid.get()
         << "' should not be in the system root cgroup (being destroyed?)";
+  } else {
+    // Cache the memory cgroup.
+    containers_.at(containerId)->memoryCgroup = memoryCgroup.get();
+  }
+
+  if (containers_.at(containerId)->cpuCgroup.isNone() &&
+      containers_.at(containerId)->memoryCgroup.isNone()) {
+    return Nothing();
+  }
+
+  return __update(containerId, _resources);
+}
+
+
+Future<Nothing> DockerContainerizerProcess::__update(
+    const ContainerID& containerId,
+    const Resources& _resources)
+{
+  CHECK(containers_.contains(containerId));
+
+  Container* container = containers_.at(containerId);
+
+  // Determine the cgroups hierarchies where the 'cpu' and
+  // 'memory' subsystems are mounted (they may be the same). Note that
+  // we make these static so we can reuse the result for subsequent
+  // calls.
+  static Result<string> cpuHierarchy = cgroups::hierarchy("cpu");
+  static Result<string> memoryHierarchy = cgroups::hierarchy("memory");
+
+  if (cpuHierarchy.isError()) {
+    return Failure("Failed to determine the cgroup hierarchy "
+                   "where the 'cpu' subsystem is mounted: " +
+                   cpuHierarchy.error());
+  }
+
+  if (memoryHierarchy.isError()) {
+    return Failure("Failed to determine the cgroup hierarchy "
+                   "where the 'memory' subsystem is mounted: " +
+                   memoryHierarchy.error());
   }
 
-  // And update the CPU shares (if applicable).
+  Option<string> cpuCgroup = container->cpuCgroup;
+  Option<string> memoryCgroup = container->memoryCgroup;
+
+  // Update the CPU shares (if applicable).
   if (cpuHierarchy.isSome() &&
       cpuCgroup.isSome() &&
-      cpuCgroup.get() != systemRootCgroup &&
       _resources.cpus().isSome()) {
     double cpuShares = _resources.cpus().get();
 
@@ -1872,26 +1902,9 @@ Future<Nothing> DockerContainerizerProcess::__update(
     }
   }
 
-  // Now determine the cgroup for the 'memory' subsystem.
-  Result<string> memoryCgroup = cgroups::memory::cgroup(pid);
-
-  if (memoryCgroup.isError()) {
-    return Failure("Failed to determine cgroup for the 'memory' subsystem: " +
-                   memoryCgroup.error());
-  } else if (memoryCgroup.isNone()) {
-    LOG(WARNING) << "Container " << containerId
-                 << " does not appear to be a member of a cgroup"
-                 << " where the 'memory' subsystem is mounted";
-  } else if (memoryCgroup.get() == systemRootCgroup) {
-    LOG(WARNING)
-        << "Process '" << pid
-        << "' should not be in the system root cgroup (being destroyed?)";
-  }
-
-  // And update the memory limits (if applicable).
+  // Update the memory limits (if applicable).
   if (memoryHierarchy.isSome() &&
       memoryCgroup.isSome() &&
-      memoryCgroup.get() != systemRootCgroup &&
       _resources.mem().isSome()) {
     // TODO(tnachen): investigate and handle OOM with docker.
     Bytes mem = _resources.mem().get();
@@ -1938,10 +1951,10 @@ Future<Nothing> DockerContainerizerProcess::__update(
                 << " for container " << containerId;
     }
   }
-#endif // __linux__
 
   return Nothing();
 }
+#endif // __linux__
 
 
 Future<ResourceStatistics> DockerContainerizerProcess::usage(
diff --git a/src/slave/containerizer/docker.hpp b/src/slave/containerizer/docker.hpp
index ca41f3b..0349f53 100644
--- a/src/slave/containerizer/docker.hpp
+++ b/src/slave/containerizer/docker.hpp
@@ -250,6 +250,7 @@ private:
       const ContainerID& containerId,
       process::Future<Nothing> future);
 
+#ifdef __linux__
   process::Future<Nothing> _update(
       const ContainerID& containerId,
       const Resources& resources,
@@ -257,8 +258,8 @@ private:
 
   process::Future<Nothing> __update(
       const ContainerID& containerId,
-      const Resources& resources,
-      pid_t pid);
+      const Resources& resources);
+#endif // __linux__
 
   process::Future<Nothing> mountPersistentVolumes(
       const ContainerID& containerId);
@@ -519,6 +520,9 @@ private:
 #ifdef __linux__
     // GPU resources allocated to the container.
     std::set<Gpu> gpus;
+
+    Option<std::string> cpuCgroup;
+    Option<std::string> memoryCgroup;
 #endif // __linux__
 
     // Marks if this container launches an executor in a docker
diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp b/src/tests/containerizer/docker_containerizer_tests.cpp
index a621758..df06f60 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -1217,7 +1217,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
 
   newResources = Resources::parse("cpus:1;mem:144");
 
-  // Issue second update that uses the cached pid instead of inspect.
+  // Issue second update that uses the cached cgroups instead of inspect.
   update = dockerContainerizer.update(containerId.get(), newResources.get());
 
   AWAIT_READY(update);