Skip to content
Merged
Show file tree
Hide file tree
Changes from 66 commits
Commits
Show all changes
69 commits
Select commit Hold shift + click to select a range
05c4dbc
[core] (cgroups 1/n) Adding a sys/fs filesystem driver
israbbani Jul 24, 2025
645f9a0
adding the copyright
israbbani Jul 24, 2025
2bb2c5b
Adding a fallback for creating processes inside cgroups with fork/exec
israbbani Jul 24, 2025
4793094
adding a pause in the tests to see what's up with the container
israbbani Jul 25, 2025
85d0ebf
Update src/ray/common/cgroup2/cgroup_driver_interface.h
israbbani Jul 25, 2025
3a5a020
Comments
israbbani Jul 25, 2025
68b0c93
Merge branch 'irabbani/cgroups-1' of github.com:ray-project/ray into …
israbbani Jul 25, 2025
f52354b
Putting the cgroupv2 tests into a separate target
israbbani Jul 29, 2025
148d04d
removing test sleep
israbbani Jul 29, 2025
d3f8b79
Removing a docstring
israbbani Jul 29, 2025
d76ff15
enabling CI tests
israbbani Jul 29, 2025
2798ea5
fixing absl imports
israbbani Jul 29, 2025
3fda505
commenting local
israbbani Jul 29, 2025
9e1e931
doxygen formatting
israbbani Jul 29, 2025
f066f34
Merge branch 'master' into irabbani/cgroups-1
israbbani Jul 30, 2025
e6b4926
removing integration tests
israbbani Jul 30, 2025
f4e0cb2
final cleanup
israbbani Jul 30, 2025
544ba83
iwyu
israbbani Jul 30, 2025
669ba99
Merge branch 'master' into irabbani/cgroups-1
israbbani Jul 30, 2025
2e341d6
we cpplintin!
israbbani Jul 30, 2025
9e46ce6
Update src/ray/common/cgroup2/sysfs_cgroup_driver.cc
israbbani Jul 30, 2025
7c745c5
Apply suggestions from code review
israbbani Jul 30, 2025
d7eb863
bug
israbbani Jul 30, 2025
ff64534
Merge branch 'irabbani/cgroups-1' of github.com:ray-project/ray into …
israbbani Jul 30, 2025
da4b475
[core] Integration tests for SysFsCgroupDriver.
israbbani Jul 30, 2025
37e205f
Cleaning up cgroup_test_utils and attempting to
israbbani Jul 30, 2025
7b83932
broken
israbbani Jul 31, 2025
b911d25
up
israbbani Jul 31, 2025
63506dc
upup
israbbani Jul 31, 2025
e6f1ae9
Merge branch 'master' into irabbani/cgroups-2
israbbani Jul 31, 2025
ead9de1
up
israbbani Jul 31, 2025
d0bcf4d
Adding shell scripts to do cgroup setup/teardown
israbbani Aug 27, 2025
08c36d8
Merge branch 'master' into irabbani/cgroups-2
israbbani Aug 27, 2025
758955a
Merged and fixed a few issues
israbbani Aug 27, 2025
e59ac62
fixing test target for CI
israbbani Aug 27, 2025
8866592
maybe this will trigger tests
israbbani Aug 27, 2025
5364a1d
runforever
israbbani Aug 27, 2025
c77e1f8
up
israbbani Aug 27, 2025
fe54541
up
israbbani Aug 27, 2025
67b21d5
cleaning up todos and docs
israbbani Aug 27, 2025
6e6bc32
one more
israbbani Aug 27, 2025
c399d45
adding separate target for unit tests now
israbbani Aug 27, 2025
2cb4f6e
typo
israbbani Aug 27, 2025
dd25a97
come unit and integration test targets
israbbani Aug 27, 2025
4a95598
missing flag
israbbani Aug 27, 2025
cc51788
plz work
israbbani Aug 28, 2025
d31eb1a
one more
israbbani Aug 28, 2025
d43a5d3
[core] Adding CgroupManager to create, modify, and delete the cgroup
israbbani Sep 3, 2025
a458406
disabling cgroup test
israbbani Sep 3, 2025
01023b9
Addressing feedback
israbbani Sep 3, 2025
bb5d866
ci change
israbbani Sep 3, 2025
f4a8553
Begrudingly using the random id generator from id.h
israbbani Sep 3, 2025
17d1008
instructions for running locally
israbbani Sep 3, 2025
3423eab
adding instructions to run locally
israbbani Sep 3, 2025
5357ea3
Merge branch 'master' into irabbani/cgroups-2
israbbani Sep 3, 2025
1ecfdda
Merge branch 'irabbani/cgroups-2' into irabbani/cgroups-3
israbbani Sep 3, 2025
17c07da
Cleaning up comments
israbbani Sep 3, 2025
e044fcd
fixing ci
israbbani Sep 3, 2025
b59dbc4
Merge branch 'irabbani/cgroups-2' of github.com:ray-project/ray into …
israbbani Sep 3, 2025
13eee38
Merge branch 'irabbani/cgroups-2' into irabbani/cgroups-3
israbbani Sep 3, 2025
f698183
ci
israbbani Sep 3, 2025
3b37051
Removing the no_windows tags and replacing it with the bazel
israbbani Sep 3, 2025
946ec90
Merge branch 'irabbani/cgroups-3' of github.com:ray-project/ray into …
israbbani Sep 4, 2025
ca63baa
Merge branch 'irabbani/cgroups-2' into irabbani/cgroups-3
israbbani Sep 4, 2025
0fe9113
Merge branch 'master' into irabbani/cgroups-3
israbbani Sep 4, 2025
ca83426
Merge branch 'master' into irabbani/cgroups-3
israbbani Sep 4, 2025
2457558
Merge branch 'master' into irabbani/cgroups-3
israbbani Sep 4, 2025
b5f6c5e
Addressing feedback.
israbbani Sep 5, 2025
03c731e
Merge branch 'master' into irabbani/cgroups-3
edoakes Sep 5, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
48 changes: 48 additions & 0 deletions src/ray/common/cgroup2/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,40 @@ ray_cc_library(
],
)

ray_cc_library(
name = "cgroup_manager_interface",
hdrs = [
"cgroup_manager_interface.h",
],
target_compatible_with = [
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🫶

"@platforms//os:linux",
],
deps = [
"//src/ray/common:status",
"//src/ray/common:status_or",
],
)

ray_cc_library(
name = "cgroup_manager",
srcs = ["cgroup_manager.cc"],
hdrs = [
"cgroup_manager.h",
"scoped_cgroup_operation.h",
],
target_compatible_with = [
"@platforms//os:linux",
],
deps = [
":cgroup_driver_interface",
":cgroup_manager_interface",
"//src/ray/common:status",
"//src/ray/common:status_or",
"//src/ray/util:logging",
"@com_google_absl//absl/strings",
],
)

ray_cc_library(
name = "sysfs_cgroup_driver",
srcs = ["sysfs_cgroup_driver.cc"],
Expand All @@ -32,6 +66,20 @@ ray_cc_library(
],
)

ray_cc_library(
name = "fake_cgroup_driver",
hdrs = [
"fake_cgroup_driver.h",
],
target_compatible_with = [
"@platforms//os:linux",
],
deps = [
":cgroup_driver_interface",
"//src/ray/common:status",
],
)

ray_cc_library(
name = "cgroup_test_utils",
srcs = ["cgroup_test_utils.cc"],
Expand Down
229 changes: 229 additions & 0 deletions src/ray/common/cgroup2/cgroup_manager.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,229 @@
// Copyright 2025 The Ray Authors.
//
// Licensed 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 "ray/common/cgroup2/cgroup_manager.h"

#include <filesystem>
#include <memory>
#include <string>
#include <unordered_set>
#include <utility>

#include "absl/strings/str_format.h"
#include "absl/strings/str_join.h"
#include "ray/common/cgroup2/cgroup_driver_interface.h"
#include "ray/common/cgroup2/scoped_cgroup_operation.h"
#include "ray/common/status_or.h"

namespace ray {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe should have a namespace cgroup ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm open to trying it. I haven't really wrapped my head around what best practices should be around namespaces. I've added an item to #54703. I'll play around with it at the end.


CgroupManager::CgroupManager(std::string base_cgroup_path,
const std::string &node_id,
std::unique_ptr<CgroupDriverInterface> cgroup_driver)
: base_cgroup_path_(std::move(base_cgroup_path)),
cgroup_driver_(std::move(cgroup_driver)) {
node_cgroup_path_ = base_cgroup_path_ + std::filesystem::path::preferred_separator +
absl::StrFormat("%s_%s", kNodeCgroupName, node_id);
system_cgroup_path_ =
node_cgroup_path_ + std::filesystem::path::preferred_separator + kSystemCgroupName;

application_cgroup_path_ = node_cgroup_path_ +
std::filesystem::path::preferred_separator +
kApplicationCgroupName;
}

CgroupManager::~CgroupManager() {
while (!cleanup_operations_.empty()) {
cleanup_operations_.pop_back();
}
}

StatusOr<std::unique_ptr<CgroupManager>> CgroupManager::Create(
std::string base_cgroup_path,
const std::string &node_id,
const int64_t system_reserved_cpu_weight,
const int64_t system_reserved_memory_bytes,
std::unique_ptr<CgroupDriverInterface> cgroup_driver) {
// TODO(#54703): Add bounds checking for system_reserved_cpu_weight
// and system_reserved_memory_bytes.
RAY_RETURN_NOT_OK(cgroup_driver->CheckCgroupv2Enabled());
RAY_RETURN_NOT_OK(cgroup_driver->CheckCgroup(base_cgroup_path));
StatusOr<std::unordered_set<std::string>> available_controllers =
cgroup_driver->GetAvailableControllers(base_cgroup_path);

if (!available_controllers.ok()) {
return available_controllers.status();
}

std::string supported_controllers_str =
absl::StrCat("[", absl::StrJoin(supported_controllers_, ", "), "]");

for (const auto &ctrl : supported_controllers_) {
if (available_controllers->find(ctrl) == available_controllers->end()) {
std::string available_controllers_str =
absl::StrCat("[", absl::StrJoin(*available_controllers, ", "), "]");
return Status::Invalid(absl::StrFormat(
"Failed to initialize resource isolation "
"because required controllers are not available in the cgroup %s. "
"To make controllers available in %s, you need to enable for its ancestor "
"cgroups."
"\nSee "
"https://docs.kernel.org/admin-guide/cgroup-v2.html#controlling-controllers "
"for more details."
"\nAvailable controllers: %s."
"\nRequired controllers: %s.",
base_cgroup_path,
base_cgroup_path,
available_controllers_str,
supported_controllers_str));
}
}

std::unique_ptr<CgroupManager> cgroup_manager = std::unique_ptr<CgroupManager>(
new CgroupManager(std::move(base_cgroup_path), node_id, std::move(cgroup_driver)));

RAY_RETURN_NOT_OK(cgroup_manager->Initialize(system_reserved_cpu_weight,
system_reserved_memory_bytes));

return cgroup_manager;
}

// TODO(#54703): This is a placeholder for cleanup. This will call
// CgroupDriver::DeleteCgroup.
void CgroupManager::RegisterDeleteCgroup(const std::string &cgroup_path) {
cleanup_operations_.emplace_back([cgroup = cgroup_path]() {
RAY_LOG(INFO) << absl::StrFormat("Deleting all cgroup %s.", cgroup);
});
}

// TODO(#54703): This is a placeholder for cleanup. This will call
// CgroupDriver::MoveAllProcesses.
void CgroupManager::RegisterMoveAllProcesses(const std::string &from,
const std::string &to) {
cleanup_operations_.emplace_back([from_cgroup = from, to_cgroup = to]() {
RAY_LOG(INFO) << absl::StrFormat(
"Moved All Processes from %s to %s.", from_cgroup, to_cgroup);
});
}

// TODO(#54703): This is a placeholder for cleanup. This will call
// CgroupDriver::AddConstraint(cgroup, constraint, default_value).
void CgroupManager::RegisterRemoveConstraint(const std::string &cgroup,
const std::string &constraint) {
cleanup_operations_.emplace_back(
[constrained_cgroup = cgroup, constraint_to_remove = constraint]() {
auto constraint_metadata = supported_constraints_.find(constraint_to_remove);
RAY_CHECK(constraint_metadata != supported_constraints_.end());
RAY_LOG(INFO) << absl::StrFormat(
"Setting constraint %s to default value %lld for cgroup %s",
constraint_to_remove,
constraint_metadata->second.default_value,
constrained_cgroup);
});
}

// TODO(#54703): This is a placeholder for cleanup. This will call
// CgroupDriver::DisableController.
void CgroupManager::RegisterDisableController(const std::string &cgroup,
const std::string &controller) {
cleanup_operations_.emplace_back([cgroup_to_clean = cgroup,
controller_to_disable = controller]() {
RAY_LOG(INFO) << absl::StrFormat(
"Disabling controller %s for cgroup %s.", controller_to_disable, cgroup_to_clean);
});
}

Status CgroupManager::Initialize(int64_t system_reserved_cpu_weight,
int64_t system_reserved_memory_bytes) {
std::string supported_controllers =
absl::StrCat("[", absl::StrJoin(supported_controllers_, ", "), "]");

// The cpu.weight is distributed between the system and application cgroups.
// The application cgroup gets whatever is leftover from the system cgroup.
int64_t max_cpu_weight = supported_constraints_.at(kCPUWeightConstraint).Max();
int64_t application_cgroup_cpu_weight = max_cpu_weight - system_reserved_cpu_weight;

RAY_LOG(INFO) << absl::StrFormat(
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

seems a little noisy for an info level, but only once per raylet startup so should be ok

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As an SRE, I found log lines like this one very useful when debugging issues. As a rule of thumb I think we should log the configuration each component starts up with (especially if it's only created once in the lifecycle of the application).

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good. This one is nice that all of the info is logged in one place. We have some other startup logs that are noisy because we log each bit in a separate log line from different components.

"Initializing CgroupManager at base cgroup path at %s. Ray's cgroup "
"hierarchy will under the node cgroup %s. The %s controllers will be "
"enabled. "
"System cgroup %s will have constraints [%s=%lld, %s=%lld]. "
"Application cgroup %s will have constraints [%s=%lld].",
base_cgroup_path_,
node_cgroup_path_,
supported_controllers,
system_cgroup_path_,
kCPUWeightConstraint,
system_reserved_cpu_weight,
kMemoryMinConstraint,
system_reserved_memory_bytes,
application_cgroup_path_,
kCPUWeightConstraint,
application_cgroup_cpu_weight);

// Create the cgroup heirarchy:
// base_cgroup_path (e.g. /sys/fs/cgroup)
// |
// ray_node_<node_id>
// | |
// system application
RAY_RETURN_NOT_OK(cgroup_driver_->CreateCgroup(node_cgroup_path_));
RegisterDeleteCgroup(node_cgroup_path_);

RAY_RETURN_NOT_OK(cgroup_driver_->CreateCgroup(system_cgroup_path_));
RegisterDeleteCgroup(system_cgroup_path_);

RAY_RETURN_NOT_OK(cgroup_driver_->CreateCgroup(application_cgroup_path_));
RegisterDeleteCgroup(application_cgroup_path_);

// Move all processes from the base_cgroup into the system_cgroup to make sure
// that the no internal process constraint is not violated. This is relevant
// when the base_cgroup_path is not a root cgroup for the system. This is likely
// the case if Ray is running inside a container.
RAY_RETURN_NOT_OK(
cgroup_driver_->MoveAllProcesses(base_cgroup_path_, system_cgroup_path_));
RegisterMoveAllProcesses(system_cgroup_path_, base_cgroup_path_);

for (const auto &ctrl : supported_controllers_) {
RAY_RETURN_NOT_OK(cgroup_driver_->EnableController(base_cgroup_path_, ctrl));
RegisterDisableController(base_cgroup_path_, ctrl);
RAY_RETURN_NOT_OK(cgroup_driver_->EnableController(node_cgroup_path_, ctrl));
RegisterDisableController(node_cgroup_path_, ctrl);
RAY_RETURN_NOT_OK(cgroup_driver_->EnableController(system_cgroup_path_, ctrl));
RegisterDisableController(system_cgroup_path_, ctrl);
RAY_RETURN_NOT_OK(cgroup_driver_->EnableController(application_cgroup_path_, ctrl));
RegisterDisableController(application_cgroup_path_, ctrl);
}

RAY_RETURN_NOT_OK(
cgroup_driver_->AddConstraint(system_cgroup_path_,
kMemoryMinConstraint,
std::to_string(system_reserved_memory_bytes)));
RegisterRemoveConstraint(system_cgroup_path_, kMemoryMinConstraint);

RAY_RETURN_NOT_OK(
cgroup_driver_->AddConstraint(system_cgroup_path_,
kCPUWeightConstraint,
std::to_string(system_reserved_cpu_weight)));
RegisterRemoveConstraint(system_cgroup_path_, kCPUWeightConstraint);

RAY_RETURN_NOT_OK(
cgroup_driver_->AddConstraint(application_cgroup_path_,
kCPUWeightConstraint,
std::to_string(application_cgroup_cpu_weight)));
RegisterRemoveConstraint(application_cgroup_path_, kCPUWeightConstraint);

return Status::OK();
}
} // namespace ray
Loading