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 2020/03/20 09:03:04 UTC

[mesos] 07/21: Set container's `memory.limit_in_bytes` to its memory limit.

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

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

commit c35bfbf998407d3abab453e5db1ff3ff7a7ab658
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Thu Jan 2 09:02:43 2020 +0800

    Set container's `memory.limit_in_bytes` to its memory limit.
    
    Review: https://reviews.apache.org/r/71943
---
 .../mesos/isolators/cgroups/subsystems/memory.cpp  | 132 ++++++++++++++-------
 .../mesos/isolators/cgroups/subsystems/memory.hpp  |   4 +
 2 files changed, 95 insertions(+), 41 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/memory.cpp b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/memory.cpp
index 88329f7..4102985 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/memory.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/memory.cpp
@@ -17,6 +17,7 @@
 #include <sys/types.h>
 
 #include <climits>
+#include <cmath>
 #include <sstream>
 
 #include <process/collect.hpp>
@@ -123,6 +124,7 @@ Future<Nothing> MemorySubsystemProcess::recover(
   }
 
   infos.put(containerId, Owned<Info>(new Info));
+  infos[containerId]->hardLimitUpdated = true;
 
   oomListen(containerId, cgroup);
   pressureListen(containerId, cgroup);
@@ -140,6 +142,7 @@ Future<Nothing> MemorySubsystemProcess::prepare(
   }
 
   infos.put(containerId, Owned<Info>(new Info));
+  infos[containerId]->hardLimitUpdated = false;
 
   oomListen(containerId, cgroup);
   pressureListen(containerId, cgroup);
@@ -182,13 +185,13 @@ Future<Nothing> MemorySubsystemProcess::update(
 
   // New limit.
   Bytes mem = resourceRequests.mem().get();
-  Bytes limit = std::max(mem, MIN_MEMORY);
+  Bytes softLimit = std::max(mem, MIN_MEMORY);
 
   // Always set the soft limit.
   Try<Nothing> write = cgroups::memory::soft_limit_in_bytes(
       hierarchy,
       cgroup,
-      limit);
+      softLimit);
 
   if (write.isError()) {
     return Failure(
@@ -197,60 +200,113 @@ Future<Nothing> MemorySubsystemProcess::update(
   }
 
   LOG(INFO) << "Updated 'memory.soft_limit_in_bytes' to "
-            << limit << " for container " << containerId;
+            << softLimit << " for container " << containerId;
 
-  // Read the existing limit.
-  Try<Bytes> currentLimit = cgroups::memory::limit_in_bytes(hierarchy, cgroup);
+  // Read the existing hard limit.
+  Try<Bytes> currentHardLimit =
+    cgroups::memory::limit_in_bytes(hierarchy, cgroup);
 
-  // NOTE: If `flags.cgroups_limit_swap` is (has been) used then both
-  // 'limit_in_bytes' and 'memsw.limit_in_bytes' will always be set to
-  // the same value.
-  if (currentLimit.isError()) {
+  if (currentHardLimit.isError()) {
     return Failure(
         "Failed to read 'memory.limit_in_bytes'"
-        ": " + currentLimit.error());
+        ": " + currentHardLimit.error());
+  }
+
+  Option<double> memLimit = None();
+  foreach (auto&& limit, resourceLimits) {
+    if (limit.first == "mem") {
+      memLimit = limit.second.value();
+    }
   }
 
+  // Rather than trying to represent an infinite limit with the `Bytes`
+  // type, we represent the infinite case by setting `isInfiniteLimit`
+  // to `true` and letting `hardLimit` be NONE.
+  bool isInfiniteLimit = false;
+  Option<Bytes> hardLimit = None();
+  if (memLimit.isSome() && std::isinf(memLimit.get())) {
+    isInfiniteLimit = true;
+  } else {
+    // Set hard limit to memory limit (if any) or to memory request.
+    hardLimit = memLimit.isSome() ?
+        std::max(Megabytes(static_cast<uint64_t>(memLimit.get())), MIN_MEMORY) :
+        softLimit;
+  }
+
+  // NOTE: If `flags.cgroups_limit_swap` is (has been) used then both
+  // 'limit_in_bytes' and 'memsw.limit_in_bytes' will always be set to
+  // the same value.
   bool limitSwap = flags.cgroups_limit_swap;
 
   auto setLimitInBytes = [=]() -> Try<Nothing> {
-    Try<Nothing> write = cgroups::memory::limit_in_bytes(
-        hierarchy,
-        cgroup,
-        limit);
+    if (isInfiniteLimit) {
+      Try<Nothing> write =
+        cgroups::write(hierarchy, cgroup, "memory.limit_in_bytes", "-1");
 
-    if (write.isError()) {
-      return Error(
-          "Failed to set 'memory.limit_in_bytes'"
-          ": " + write.error());
-    }
-
-    LOG(INFO) << "Updated 'memory.limit_in_bytes' to " << limit
-              << " for container " << containerId;
+      if (write.isError()) {
+        return Error(
+            "Failed to update 'memory.limit_in_bytes': " + write.error());
+      }
 
-    return Nothing();
-  };
+      LOG(INFO) << "Updated 'memory.limit_in_bytes' to -1"
+                << " for container " << containerId;
+    } else {
+      CHECK_SOME(hardLimit);
 
-  auto setMemswLimitInBytes = [=]() -> Try<Nothing> {
-    if (limitSwap) {
-      Try<bool> write = cgroups::memory::memsw_limit_in_bytes(
+      Try<Nothing> write = cgroups::memory::limit_in_bytes(
           hierarchy,
           cgroup,
-          limit);
+          hardLimit.get());
 
       if (write.isError()) {
         return Error(
-            "Failed to set 'memory.memsw.limit_in_bytes'"
+            "Failed to set 'memory.limit_in_bytes'"
             ": " + write.error());
       }
 
-      LOG(INFO) << "Updated 'memory.memsw.limit_in_bytes' to " << limit
+      LOG(INFO) << "Updated 'memory.limit_in_bytes' to " << hardLimit.get()
                 << " for container " << containerId;
     }
 
     return Nothing();
   };
 
+  auto setMemswLimitInBytes = [=]() -> Try<Nothing> {
+    if (limitSwap) {
+      if (isInfiniteLimit) {
+        Try<Nothing> write = cgroups::write(
+            hierarchy, cgroup, "memory.memsw.limit_in_bytes", "-1");
+
+        if (write.isError()) {
+          return Error(
+              "Failed to update 'memory.memsw.limit_in_bytes'"
+              ": " + write.error());
+        }
+
+        LOG(INFO) << "Updated 'memory.memsw.limit_in_bytes' to -1"
+                  << " for container " << containerId;
+      } else {
+        CHECK_SOME(hardLimit);
+
+        Try<bool> write = cgroups::memory::memsw_limit_in_bytes(
+            hierarchy,
+            cgroup,
+            hardLimit.get());
+
+        if (write.isError()) {
+          return Error(
+              "Failed to set 'memory.memsw.limit_in_bytes'"
+              ": " + write.error());
+        }
+
+        LOG(INFO) << "Updated 'memory.memsw.limit_in_bytes' to "
+                  << hardLimit.get() << " for container " << containerId;
+      }
+    }
+
+    return Nothing();
+  };
+
   vector<lambda::function<Try<Nothing>(void)>> setFunctions;
 
   // Now, determine whether to set the hard limit. We only update the
@@ -266,26 +322,18 @@ Future<Nothing> MemorySubsystemProcess::update(
   // discrepancy between usage and soft limit and introduces a "manual
   // oom" if necessary.
   //
-  // If this is the first time, 'memory.limit_in_bytes' is unlimited
-  // which may be one of following possible values:
-  //   * LONG_MAX (Linux Kernel Version < 3.12)
-  //   * ULONG_MAX (3.12 <= Linux Kernel Version < 3.19)
-  //   * LONG_MAX / pageSize * pageSize (Linux Kernel Version >= 3.19)
-  static const size_t pageSize = os::pagesize();
-  Bytes unlimited(static_cast<uint64_t>(LONG_MAX / pageSize * pageSize));
-
   // NOTE: It's required by the Linux kernel that
   // 'memory.limit_in_bytes' should be less than or equal to
   // 'memory.memsw.limit_in_bytes'. Otherwise, the kernel will fail
   // the cgroup write with EINVAL. As a result, the order of setting
   // these two control files is important. See MESOS-7237 for details.
-  if (currentLimit.get() >= unlimited) {
-    // This is the first time memory limit is being set. So
+  if (!infos[containerId]->hardLimitUpdated) {
+    // This is the first time memory hard limit is being set. So
     // effectively we are reducing the memory limits because of which
     // we need to set the 'memory.limit_in_bytes' before setting
     // 'memory.memsw.limit_in_bytes'
     setFunctions = {setLimitInBytes, setMemswLimitInBytes};
-  } else if (limit > currentLimit.get()) {
+  } else if (isInfiniteLimit || hardLimit.get() > currentHardLimit.get()) {
     setFunctions = {setMemswLimitInBytes, setLimitInBytes};
   }
 
@@ -296,6 +344,8 @@ Future<Nothing> MemorySubsystemProcess::update(
     }
   }
 
+  infos[containerId]->hardLimitUpdated = true;
+
   return Nothing();
 }
 
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/memory.hpp b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/memory.hpp
index 3bd22c5..ed34df8 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/subsystems/memory.hpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/subsystems/memory.hpp
@@ -93,6 +93,10 @@ private:
         process::Owned<cgroups::memory::pressure::Counter>> pressureCounters;
 
     process::Promise<mesos::slave::ContainerLimitation> limitation;
+
+    // Indicate whether the memory hard limit of this container has
+    // already been updated.
+    bool hardLimitUpdated;
   };
 
   MemorySubsystemProcess(const Flags& flags, const std::string& hierarchy);