You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2024/04/19 00:02:11 UTC

(mesos) branch master updated: [cgroups2] Introduced API to listen for OOM events.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new a4b725bed [cgroups2] Introduced API to listen for OOM events.
a4b725bed is described below

commit a4b725bed40352de8a2c14c17be9234e354c02e0
Author: Devin Leamy <dl...@twitter.com>
AuthorDate: Thu Apr 18 19:23:21 2024 -0400

    [cgroups2] Introduced API to listen for OOM events.
    
    Introduces `cgroups2::memory::events::oom` which returns a future
    that resolves when the cgroup reaches its memory limit and allocation
    was about to fail.
    
    In cgroups v1, there was a bespoke notification API.
    
    Cgroups v2 provides the 'memory.events' control which contains key-value
    pairs of events and the number of times they took place [1]. For OOMs, we
    look at the value of the `oom` field. In `cgroups2::memory::events::oom`
    we watch for changes to 'memory.events' (via polling every 100ms for now,
    and later via inotify) and resolve a future when `events.oom > 0`.
    
    [1] https://docs.kernel.org/admin-guide/cgroup-v2.html#memory
    
    This closes #563
---
 src/linux/cgroups2.cpp                     | 77 ++++++++++++++++++++++++++++++
 src/linux/cgroups2.hpp                     | 44 +++++++++++++++++
 src/tests/containerizer/cgroups2_tests.cpp | 42 ++++++++++++++++
 3 files changed, 163 insertions(+)

diff --git a/src/linux/cgroups2.cpp b/src/linux/cgroups2.cpp
index 35eaf1f24..2dd197c4c 100644
--- a/src/linux/cgroups2.cpp
+++ b/src/linux/cgroups2.cpp
@@ -24,6 +24,10 @@
 #include <string>
 #include <vector>
 
+#include <process/after.hpp>
+#include <process/loop.hpp>
+#include <process/pid.hpp>
+
 #include <stout/none.hpp>
 #include <stout/numify.hpp>
 #include <stout/os.hpp>
@@ -40,6 +44,13 @@ using std::set;
 using std::string;
 using std::vector;
 
+using process::Break;
+using process::Continue;
+using process::ControlFlow;
+using process::Failure;
+using process::Future;
+using process::loop;
+
 using mesos::internal::fs::MountTable;
 
 namespace cgroups2 {
@@ -795,6 +806,7 @@ Result<Bytes> parse_bytelimit(const string& value)
 namespace control {
 
 const string CURRENT = "memory.current";
+const string EVENTS = "memory.events";
 const string LOW = "memory.low";
 const string HIGH = "memory.high";
 const string MAX = "memory.max";
@@ -802,6 +814,71 @@ const string MIN = "memory.min";
 
 } // namespace control {
 
+namespace events {
+
+Try<Events> parse(const string& content)
+{
+  Events events;
+
+  foreach (const string& line, strings::split(content, "\n")) {
+    if (line.empty()) {
+      continue;
+    }
+
+    vector<string> tokens = strings::split(line, " ");
+    if (tokens.size() != 2) {
+      return Error("Invalid line format in 'memory.events' expected "
+                   "<key> <value> received: '" + line + "'");
+    }
+
+    const string& field = tokens[0];
+    const string& value = tokens[1];
+
+    Try<uint64_t> count = numify<uint64_t>(value);
+    if (count.isError()) {
+      return Error("Failed to numify '" + value + "': " + count.error());
+    }
+
+    if      (field == "low")            { events.low            = *count; }
+    else if (field == "high")           { events.high           = *count; }
+    else if (field == "max")            { events.max            = *count; }
+    else if (field == "oom")            { events.oom            = *count; }
+    else if (field == "oom_kill")       { events.oom_kill       = *count; }
+    else if (field == "oom_group_kill") { events.oom_group_kill = *count; }
+  }
+
+  return events;
+}
+
+} // namespace events {
+
+
+Future<Nothing> oom(const string& cgroup)
+{
+  // TODO(dleamy): Update this to use inotify, rather than polling.
+  return loop(
+      []() {
+        return process::after(Milliseconds(100));
+      },
+      [=](const Nothing&) -> Future<ControlFlow<Nothing>> {
+        Try<string> content = cgroups2::read<string>(cgroup, control::EVENTS);
+        if (content.isError()) {
+          return Failure("Failed to read 'memory.events': " + content.error());
+        }
+
+        Try<Events> events = events::parse(strings::trim(*content));
+        if (events.isError()) {
+          return Failure("Failed to parse 'memory.events': " + events.error());
+        }
+
+        if (events->oom > 0) {
+          return Break(Nothing());
+        }
+        return Continue();
+      });
+}
+
+
 Try<Bytes> usage(const string& cgroup)
 {
   Try<uint64_t> contents = cgroups2::read<uint64_t>(
diff --git a/src/linux/cgroups2.hpp b/src/linux/cgroups2.hpp
index 22cfaeecf..91f8d65e6 100644
--- a/src/linux/cgroups2.hpp
+++ b/src/linux/cgroups2.hpp
@@ -21,6 +21,8 @@
 #include <string>
 #include <vector>
 
+#include <process/future.hpp>
+
 #include <stout/bytes.hpp>
 #include <stout/duration.hpp>
 #include <stout/nothing.hpp>
@@ -241,6 +243,48 @@ Try<BandwidthLimit> max(const std::string& cgroup);
 // See: https://docs.kernel.org/admin-guide/cgroup-v2.html
 namespace memory {
 
+// Cgroup memory controller events.
+//
+// Snapshot of the 'memory.events' or 'memory.local.events' control files.
+struct Events
+{
+  // The number of times the cgroup is reclaimed due to high memory pressure
+  // even though its usage is under the low boundary. This usually indicates
+  // that the low boundary is over-committed.
+  uint64_t low;
+
+  // The number of times processes of the cgroup are throttled and routed to
+  // perform direct memory reclaim because the high memory boundary was
+  // exceeded. For a cgroup whose memory usage is capped by the high limit
+  // rather than global memory pressure, this event’s occurrences are expected.
+  uint64_t high;
+
+  // The number of times the cgroup’s memory usage was about to go over the
+  // max boundary. If direct reclaim fails to bring it down, the cgroup goes
+  // to OOM state.
+  uint64_t max;
+
+  // The number of times the cgroup’s memory usage was reached the limit and
+  // allocation was about to fail. This event is not raised if the OOM killer
+  // is not considered as an option, e.g. for failed high-order allocations
+  // or if caller asked to not retry attempts.
+  uint64_t oom;
+
+  // The number of processes belonging to this cgroup killed by any kind of
+  // OOM killer.
+  uint64_t oom_kill;
+
+  // The number of times a group OOM has occurred.
+  uint64_t oom_group_kill;
+};
+
+
+// Listen for an OOM event for the cgroup or any descendants.
+//
+// Cannot be used for the root cgroup.
+process::Future<Nothing> oom(const std::string& cgroup);
+
+
 // Current memory usage of a cgroup and its descendants in bytes.
 Try<Bytes> usage(const std::string& cgroup);
 
diff --git a/src/tests/containerizer/cgroups2_tests.cpp b/src/tests/containerizer/cgroups2_tests.cpp
index ac2b0e0f2..4cffa80f1 100644
--- a/src/tests/containerizer/cgroups2_tests.cpp
+++ b/src/tests/containerizer/cgroups2_tests.cpp
@@ -22,6 +22,7 @@
 #include <utility>
 #include <vector>
 
+#include <process/future.hpp>
 #include <process/reap.hpp>
 #include <process/gmock.hpp>
 #include <process/gtest.hpp>
@@ -38,6 +39,10 @@
 #include "linux/cgroups2.hpp"
 #include "linux/ebpf.hpp"
 
+#include "tests/containerizer/memory_test_helper.hpp"
+
+using process::Future;
+
 using std::pair;
 using std::set;
 using std::string;
@@ -484,6 +489,43 @@ TEST_F(Cgroups2Test, ROOT_CGROUPS2_GetCgroups)
 }
 
 
+TEST_F(Cgroups2Test, ROOT_CGROUPS2_OomDetection)
+{
+  // Check that exceeding the hard memory limit will trigger an OOM event.
+  //
+  // We set a hard memory limit on the test cgroup, move a process inside of
+  // the test cgroup subtree, listen for an OOM event, deliberately trigger an
+  // OOM event by exceeding the hard limit, and then check that an event was
+  // triggered.
+  ASSERT_SOME(enable_controllers({"memory"}));
+
+  ASSERT_SOME(cgroups2::create(TEST_CGROUP));
+  ASSERT_SOME(cgroups2::controllers::enable(TEST_CGROUP, {"memory"}));
+
+  const string leaf_cgroup = TEST_CGROUP + "/leaf";
+  ASSERT_SOME(cgroups2::create(leaf_cgroup));
+
+  MemoryTestHelper helper;
+  ASSERT_SOME(helper.spawn());
+  ASSERT_SOME(helper.pid());
+
+  Bytes limit = Megabytes(64);
+
+  ASSERT_SOME(cgroups2::assign(leaf_cgroup, *helper.pid()));
+  ASSERT_SOME(cgroups2::memory::set_max(TEST_CGROUP, limit));
+
+  Future<Nothing> oomEvent = cgroups2::memory::oom(TEST_CGROUP);
+
+  // Assert that the OOM event has not been triggered.
+  EXPECT_FALSE(oomEvent.isReady());
+
+  // Increase memory usage beyond the limit.
+  ASSERT_ERROR(helper.increaseRSS(limit * 2));
+
+  AWAIT_EXPECT_READY(oomEvent);
+}
+
+
 // Arguments for os::open(). Combination of a path and an access type.
 typedef pair<string, int> OpenArgs;