You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2014/08/05 00:09:47 UTC

[26/43] git commit: Used the new cgroups helpers when updating Docker container resources.

Used the new cgroups helpers when updating Docker container resources.

Rather than "prepare" the cgroups for Docker, now we just introspect
and determine the necessary hierarchies and cgroups we need based on
how Docker (or an operator) set up the cgroups mounts.


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

Branch: refs/heads/master
Commit: ddc775aa373f3d25f47bf1973d76df146fc08f2a
Parents: 283665b
Author: Benjamin Hindman <be...@gmail.com>
Authored: Wed Jul 9 11:30:08 2014 -0700
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Mon Aug 4 15:08:17 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp       | 127 ++++++++++++++++----------
 src/tests/docker_containerizer_tests.cpp |   1 -
 2 files changed, 79 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ddc775aa/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 5a68d94..f7cc630 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -144,7 +144,7 @@ private:
   process::Future<Nothing> _update(
       const ContainerID& containerId,
       const Resources& resources,
-      const Future<Docker::Container>& future);
+      const Docker::Container& container);
 
   Future<ResourceStatistics> _usage(
     const ContainerID& containerId,
@@ -181,30 +181,6 @@ private:
 };
 
 
-Try<Nothing> DockerContainerizer::prepareCgroups(const Flags& flags)
-{
-#ifdef __linux__
-  std::vector<string> subsystems;
-  subsystems.push_back("cpu");
-  subsystems.push_back("cpuacct");
-  subsystems.push_back("memory");
-
-  foreach (const string& subsystem, subsystems) {
-    // We're assuming docker is under cgroup directory "docker".
-    Try<string> hierarchy =
-      cgroups::prepare(flags.cgroups_hierarchy, subsystem, "docker");
-
-    if (hierarchy.isError()) {
-      return Error(
-          "Failed to prepare cgroup hierarchy " + flags.cgroups_hierarchy +
-          " subsystem '" + subsystem + "' for Docker: " + hierarchy.error());
-    }
-  }
-#endif // __linux__
-  return Nothing();
-}
-
-
 Try<DockerContainerizer*> DockerContainerizer::create(
     const Flags& flags,
     bool local)
@@ -215,11 +191,6 @@ Try<DockerContainerizer*> DockerContainerizer::create(
     return Error(validation.error());
   }
 
-  Try<Nothing> prepare = prepareCgroups(flags);
-  if (prepare.isError()) {
-    return Error(prepare.error());
-  }
-
   return new DockerContainerizer(flags, local, docker);
 }
 
@@ -594,7 +565,7 @@ Future<bool> DockerContainerizerProcess::_launch(
   // Docker containers not created by Mesos).
   // TODO(benh): Get full path to 'docker'.
   string override =
-    "docker wait " + DOCKER_NAME_PREFIX + stringify(containerId);
+    flags.docker + " wait " + DOCKER_NAME_PREFIX + stringify(containerId);
 
   Try<Subprocess> s = subprocess(
       executorInfo.command().value() + " --override " + override,
@@ -690,21 +661,64 @@ Future<Nothing> DockerContainerizerProcess::update(
 Future<Nothing> DockerContainerizerProcess::_update(
     const ContainerID& containerId,
     const Resources& _resources,
-    const Future<Docker::Container>& future)
+    const Docker::Container& container)
 {
 #ifdef __linux__
-  const string& id = path::join("docker", future.get().id());
+  // Determine the 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());
+  }
 
-  // Update CPU shares.
-  if (_resources.cpus().isSome()) {
+  // 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
+  // update the proper cgroup control files.
+
+  // First check that this container still appears to be running.
+  Option<pid_t> pid = container.pid();
+  if (pid.isNone()) {
+    return Nothing();
+  }
+
+  // Determine the cgroup for the 'cpu' subsystem (based on the
+  // container's pid).
+  Result<string> cpuCgroup = cgroups::cpu::cgroup(pid.get());
+
+  if (cpuCgroup.isError()) {
+    return Failure("Failed to determine cgroup for the 'cpu' subsystem: " +
+                   cpuCgroup.error());
+  } else if (cpuCgroup.isNone()) {
+    LOG(WARNING)
+      << "Container " << containerId
+      << " does not appear to be a member of a cgroup "
+      << "where the 'cpu' subsystem is mounted";
+  }
+
+  // And update the CPU shares (if applicable).
+  if (cpuHierarchy.isSome() &&
+      cpuCgroup.isSome() &&
+      _resources.cpus().isSome()) {
     double cpuShares = _resources.cpus().get();
 
     uint64_t shares =
       std::max((uint64_t) (CPU_SHARES_PER_CPU * cpuShares), MIN_CPU_SHARES);
 
     Try<Nothing> write =
-      cgroups::cpu::shares(
-          path::join(flags.cgroups_hierarchy, "cpu"), id, shares);
+      cgroups::cpu::shares(cpuHierarchy.get(), cpuCgroup.get(), shares);
 
     if (write.isError()) {
       return Failure("Failed to update 'cpu.shares': " + write.error());
@@ -712,24 +726,38 @@ Future<Nothing> DockerContainerizerProcess::_update(
 
     LOG(INFO)
       << "Updated 'cpu.shares' to " << shares
+      << " at " << path::join(cpuHierarchy.get(), cpuCgroup.get())
       << " for container " << containerId;
   }
 
-  // Update Memory.
-  if (_resources.mem().isSome()) {
+  // Now determine the cgroup for the 'memory' subsystem.
+  Result<string> memoryCgroup = cgroups::memory::cgroup(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";
+  }
+
+  // And update the memory limits (if applicable).
+  if (memoryHierarchy.isSome() &&
+      memoryCgroup.isSome() &&
+      _resources.mem().isSome()) {
     Bytes mem = _resources.mem().get();
     Bytes limit = std::max(mem, MIN_MEMORY);
 
-    std::string memHierarchy =
-      path::join(flags.cgroups_hierarchy, "memory");
-
     // Always set the soft limit.
     Try<Nothing> write =
-      cgroups::memory::soft_limit_in_bytes(memHierarchy, id, limit);
+      cgroups::memory::soft_limit_in_bytes(
+          memoryHierarchy.get(), memoryCgroup.get(), limit);
 
     if (write.isError()) {
       return Failure("Failed to set 'memory.soft_limit_in_bytes': " +
-          write.error());
+                     write.error());
     }
 
     LOG(INFO)
@@ -738,24 +766,27 @@ Future<Nothing> DockerContainerizerProcess::_update(
 
     // Read the existing limit.
     Try<Bytes> currentLimit =
-      cgroups::memory::limit_in_bytes(memHierarchy, id);
+      cgroups::memory::limit_in_bytes(
+          memoryHierarchy.get(), memoryCgroup.get());
 
     if (currentLimit.isError()) {
       return Failure("Failed to read 'memory.limit_in_bytes': " +
-          currentLimit.error());
+                     currentLimit.error());
     }
 
     // Only update if new limit is higher.
     if (limit > currentLimit.get()) {
-      write = cgroups::memory::limit_in_bytes(memHierarchy, id, limit);
+      write = cgroups::memory::limit_in_bytes(
+          memoryHierarchy.get(), memoryCgroup.get(), limit);
 
       if (write.isError()) {
         return Failure("Failed to set 'memory.limit_in_bytes': " +
-            write.error());
+                       write.error());
       }
 
       LOG(INFO)
         << "Updated 'memory.limit_in_bytes' to " << limit
+        << " at " << path::join(memoryHierarchy.get(), memoryCgroup.get())
         << " for container " << containerId;
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/ddc775aa/src/tests/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/docker_containerizer_tests.cpp b/src/tests/docker_containerizer_tests.cpp
index f772eb0..3d02e62 100644
--- a/src/tests/docker_containerizer_tests.cpp
+++ b/src/tests/docker_containerizer_tests.cpp
@@ -64,7 +64,6 @@ public:
       const Docker& docker)
     : DockerContainerizer(flags, local, docker)
   {
-    DockerContainerizer::prepareCgroups(flags);
     EXPECT_CALL(*this, launch(_, _, _, _, _, _, _, _))
       .WillRepeatedly(Invoke(this, &MockDockerContainerizer::_launch));
   }