Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
92 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
398ef88
[core] cgroups (4/n) adding constraint bounds checking to the
israbbani Sep 4, 2025
ca83426
Merge branch 'master' into irabbani/cgroups-3
israbbani Sep 4, 2025
f7f04db
Merge branch 'irabbani/cgroups-3' into irabbani/cgroups-4
israbbani Sep 4, 2025
dfd9b07
Build with clang to find bugs locally!
israbbani Sep 4, 2025
1884da5
Merge branch 'irabbani/cgroups-4' of github.com:ray-project/ray into …
israbbani Sep 4, 2025
e0bbac8
[core] (cgroups 5/n) Adding methods the following methods to
israbbani Sep 4, 2025
2457558
Merge branch 'master' into irabbani/cgroups-3
israbbani Sep 4, 2025
36101f4
[core] (cgroups 6/n) CgroupManager cleans up the entire cgroup hierarchy
israbbani Sep 4, 2025
a145a81
Adding a very long happy path test
israbbani Sep 5, 2025
fc85704
Merge branch 'irabbani/cgroups-3' into irabbani/cgroups-4
israbbani Sep 5, 2025
b5f6c5e
Addressing feedback.
israbbani Sep 5, 2025
03c731e
Merge branch 'master' into irabbani/cgroups-3
edoakes Sep 5, 2025
4aeabf4
Merge branch 'irabbani/cgroups-3' into irabbani/cgroups-4
israbbani Sep 5, 2025
44a5844
Merge branch 'irabbani/cgroups-4' into irabbani/cgroups-5
israbbani Sep 5, 2025
4de334c
Merge branch 'irabbani/cgroups-5' into irabbani/cgroups-6
israbbani Sep 5, 2025
89f49e5
cleaning up and adding comments
israbbani Sep 5, 2025
dd0bf98
Merge branch 'irabbani/cgroups-6' of github.com:ray-project/ray into …
israbbani Sep 5, 2025
eee8982
Merge branch 'master' into irabbani/cgroups-5
israbbani Sep 6, 2025
fd9ef0d
Merge branch 'irabbani/cgroups-5' into irabbani/cgroups-6
israbbani Sep 6, 2025
1c17e3f
Merge branch 'master' into irabbani/cgroups-6
israbbani Sep 8, 2025
4a581d7
[core] (cgroups 7/n) cleaning up old cgroup integration code for rayl…
israbbani Sep 9, 2025
df8f925
Merge branch 'master' into irabbani/cgroups-6
israbbani Sep 9, 2025
0059039
Merge branch 'master' into irabbani/cgroups-6
israbbani Sep 9, 2025
e386cb5
Merge branch 'master' into irabbani/cgroups-6
israbbani Sep 9, 2025
cc3af83
Merge branch 'master' into irabbani/cgroups-6
israbbani Sep 11, 2025
161dd95
[core] (cgroups 8/n) Wiring CgroupManager into the raylet. (#56297)
israbbani Sep 11, 2025
92bb7ce
Merge branch 'master' into irabbani/cgroups-6
israbbani Sep 11, 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
4 changes: 0 additions & 4 deletions python/ray/_private/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -2380,7 +2380,6 @@ def is_initialized() -> bool:
return ray._private.worker.global_worker.connected


# TODO(hjiang): Add cgroup path along with [enable_resource_isolation].
@with_connect_or_shutdown_lock
def connect(
node,
Expand All @@ -2399,7 +2398,6 @@ def connect(
worker_launch_time_ms: int = -1,
worker_launched_time_ms: int = -1,
debug_source: str = "",
enable_resource_isolation: bool = False,
):
"""Connect this worker to the raylet, to Plasma, and to GCS.

Expand Down Expand Up @@ -2428,7 +2426,6 @@ def connect(
finshes launching. If the worker is not launched by raylet (e.g.,
driver), this must be -1 (default value).
debug_source: Source information for `CoreWorker`, used for debugging and informational purpose, rather than functional purpose.
enable_resource_isolation: If true, core worker enables resource isolation by adding itself into appropriate cgroup.
"""
# Do some basic checking to make sure we didn't call ray.init twice.
error_message = "Perhaps you called ray.init twice by accident?"
Expand Down Expand Up @@ -2607,7 +2604,6 @@ def connect(
worker_launch_time_ms,
worker_launched_time_ms,
debug_source,
enable_resource_isolation,
)

if mode == SCRIPT_MODE:
Expand Down
1 change: 0 additions & 1 deletion python/ray/_private/workers/default_worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -271,7 +271,6 @@
ray_debugger_external=args.ray_debugger_external,
worker_launch_time_ms=args.worker_launch_time_ms,
worker_launched_time_ms=worker_launched_time_ms,
enable_resource_isolation=args.enable_resource_isolation,
)

worker = ray._private.worker.global_worker
Expand Down
3 changes: 1 addition & 2 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -3002,7 +3002,7 @@ cdef class CoreWorker:
local_mode, driver_name,
serialized_job_config, metrics_agent_port, runtime_env_hash,
startup_token, session_name, cluster_id, entrypoint,
worker_launch_time_ms, worker_launched_time_ms, debug_source, enable_resource_isolation):
worker_launch_time_ms, worker_launched_time_ms, debug_source):
self.is_local_mode = local_mode

cdef CCoreWorkerOptions options = CCoreWorkerOptions()
Expand Down Expand Up @@ -3058,7 +3058,6 @@ cdef class CoreWorker:
options.worker_launch_time_ms = worker_launch_time_ms
options.worker_launched_time_ms = worker_launched_time_ms
options.debug_source = debug_source
options.enable_resource_isolation = enable_resource_isolation
CCoreWorkerProcess.Initialize(options)

self.cgname_to_eventloop_dict = None
Expand Down
1 change: 0 additions & 1 deletion python/ray/includes/libcoreworker.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -438,7 +438,6 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
int64_t worker_launch_time_ms
int64_t worker_launched_time_ms
c_string debug_source
c_bool enable_resource_isolation

cdef cppclass CCoreWorkerProcess "ray::core::CoreWorkerProcess":
@staticmethod
Expand Down
62 changes: 41 additions & 21 deletions src/ray/common/cgroup2/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,13 +1,40 @@
load("//bazel:ray.bzl", "ray_cc_library")

config_setting(
name = "is_linux",
constraint_values = ["@platforms//os:linux"],
)

# Public targets.
ray_cc_library(
name = "cgroup_manager",
srcs = select({
":is_linux": ["cgroup_manager.cc"],
"//conditions:default": ["noop_cgroup_manager.cc"],
}),
hdrs = ["cgroup_manager.h"],
visibility = ["//visibility:public"],
deps = [
":cgroup_driver_interface",
":cgroup_manager_interface",
"//src/ray/common:status",
"//src/ray/common:status_or",
] + select({
":is_linux": [
":scoped_cgroup_operation",
"//src/ray/util:logging",
"@com_google_absl//absl/strings",
],
"//conditions:default": [],
}),
)

ray_cc_library(
name = "cgroup_driver_interface",
hdrs = [
"cgroup_driver_interface.h",
],
target_compatible_with = [
"@platforms//os:linux",
],
visibility = ["//visibility:public"],
deps = [
"//src/ray/common:status",
"//src/ray/common:status_or",
Expand All @@ -19,51 +46,42 @@ ray_cc_library(
hdrs = [
"cgroup_manager_interface.h",
],
target_compatible_with = [
"@platforms//os:linux",
],
visibility = ["//visibility:public"],
deps = [
"//src/ray/common:status",
"//src/ray/common:status_or",
],
)

ray_cc_library(
name = "cgroup_manager",
srcs = ["cgroup_manager.cc"],
name = "sysfs_cgroup_driver",
srcs = ["sysfs_cgroup_driver.cc"],
hdrs = [
"cgroup_manager.h",
"scoped_cgroup_operation.h",
"sysfs_cgroup_driver.h",
],
target_compatible_with = [
"@platforms//os:linux",
],
visibility = ["//visibility:public"],
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",
],
)

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

ray_cc_library(
Expand All @@ -74,6 +92,7 @@ ray_cc_library(
target_compatible_with = [
"@platforms//os:linux",
],
visibility = [":__subpackages__"],
deps = [
":cgroup_driver_interface",
"//src/ray/common:status",
Expand All @@ -87,6 +106,7 @@ ray_cc_library(
target_compatible_with = [
"@platforms//os:linux",
],
visibility = [":__subpackages__"],
deps = [
"//src/ray/common:id",
"//src/ray/common:status",
Expand Down
75 changes: 46 additions & 29 deletions src/ray/common/cgroup2/cgroup_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -113,48 +113,65 @@ StatusOr<std::unique_ptr<CgroupManager>> CgroupManager::Create(
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);
Copy link
Contributor

Choose a reason for hiding this comment

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

Out of curiosity, why remove the log? (I won't insist on bringing the log back, just curious to the motivation)

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 left it in by mistake from the previous PR. I was using it to debug a test.

cleanup_operations_.emplace_back([this, cgroup = cgroup_path]() {
Status s = this->cgroup_driver_->DeleteCgroup(cgroup);
if (!s.ok()) {
RAY_LOG(WARNING) << absl::StrFormat(
"Failed to delete cgroup %s with error %s.", cgroup, s.ToString());
}
});
}

// 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);
cleanup_operations_.emplace_back([this, from_cgroup = from, to_cgroup = to]() {
Status s = this->cgroup_driver_->MoveAllProcesses(from_cgroup, to_cgroup);
if (!s.ok()) {
RAY_LOG(WARNING) << absl::StrFormat(
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: WARNING vs. ERROR?

Also, what is the potential implication of this failing? If we're unable to move processes into their appropriate cgroup, then.... what, they'll just run without the resource limits we'd otherwise like imposted? Should we keep running in that scenario or bail out?

Copy link
Contributor

Choose a reason for hiding this comment

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

I guess this line of questioning applies to delete too, but.... process move seems more concerning.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is during clean up. When CgroupManager starts up with a base_cgroup, it moves all processes in that cgroup into the <base_cgroup_path>/system/leaf cgroup. This is reversing by moving those processes back into the base_cgroup.

The reason for doing this is that the default cgroup path, and the most commonly anticipated pattern, will be to run this with /sys/fs/cgroup inside a container which is the root cgroup for the container.

However, the root cgroup for the container is not the actual root cgroup of the operating system. This means that it is subject to the No Internal Processes constraint. It cannot have child cgroups and a controller enabled if it has processes inside it.

The implication for clean up is that the cgroup hierarchy will not be deleted properly. This is mostly fine when it's running inside a container. Given this, maybe ERROR makes more sense?

"Failed to move all processes from %s to %s with error %s",
from_cgroup,
to_cgroup,
s.ToString());
}
});
}

// TODO(#54703): This is a placeholder for cleanup. This will call
// CgroupDriver::AddConstraint(cgroup, constraint, default_value).
template <typename T>
void CgroupManager::RegisterRemoveConstraint(const std::string &cgroup,
const Constraint<T> &constraint) {
cleanup_operations_.emplace_back(
[constrained_cgroup = cgroup, constraint_to_remove = constraint]() {
RAY_LOG(INFO) << absl::StrFormat(
"Setting constraint %s to default value %lld for cgroup %s",
constraint_to_remove.name_,
constraint_to_remove.default_value_,
constrained_cgroup);
[this, constrained_cgroup = cgroup, constraint_to_remove = constraint]() {
std::string default_value = std::to_string(constraint_to_remove.default_value_);
Status s = this->cgroup_driver_->AddConstraint(constrained_cgroup,
constraint_to_remove.controller_,
constraint_to_remove.name_,
default_value);
if (!s.ok()) {
RAY_LOG(WARNING) << absl::StrFormat(
"Failed to set constraint %s=%s to default value for cgroup %s with error "
"%s.",
constraint_to_remove.name_,
default_value,
constrained_cgroup,
s.ToString());
}
});
}

// TODO(#54703): This is a placeholder for cleanup. This will call
// CgroupDriver::DisableController.
void CgroupManager::RegisterDisableController(const std::string &cgroup,
void CgroupManager::RegisterDisableController(const std::string &cgroup_path,
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);
});
cleanup_operations_.emplace_back(
[this, cgroup = cgroup_path, controller_to_disable = controller]() {
Status s = this->cgroup_driver_->DisableController(cgroup, controller_to_disable);
if (!s.ok()) {
RAY_LOG(WARNING) << absl::StrFormat(
"Failed to disable controller %s for cgroup %s with error %s",
controller_to_disable,
cgroup,
s.ToString());
}
});
}

Status CgroupManager::Initialize(int64_t system_reserved_cpu_weight,
Expand All @@ -168,11 +185,11 @@ Status CgroupManager::Initialize(int64_t system_reserved_cpu_weight,
cpu_weight_constraint_.Max() - system_reserved_cpu_weight;

RAY_LOG(INFO) << absl::StrFormat(
"Initializing CgroupManager at base cgroup path at %s. Ray's cgroup "
"hierarchy will under the node cgroup %s. The %s controllers will be "
"Initializing CgroupManager at base cgroup at '%s'. Ray's cgroup "
"hierarchy will under the node cgroup at '%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].",
"The system cgroup at '%s' will have constraints [%s=%lld, %s=%lld]. "
"The application cgroup '%s' will have constraints [%s=%lld].",
base_cgroup_path_,
node_cgroup_path_,
supported_controllers,
Expand Down
7 changes: 4 additions & 3 deletions src/ray/common/cgroup2/cgroup_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -111,9 +111,10 @@ class CgroupManager : public CgroupManagerInterface {
Status Initialize(const int64_t system_reserved_cpu_weight,
const int64_t system_reserved_memory_bytes);

// TODO(#54703): This is a placeholder for cleanup. This will be implemented in the a
// future PR.
void RegisterDeleteCgroup(const std::string &cgroup_path);
// The Register* methods register a callback that will execute in the destructor
// in FILO order. All callbacks required the cgroup_driver_ to be available to
// remove the cgroup hierarchy.
void RegisterDeleteCgroup(const std::string &cgroup);
void RegisterMoveAllProcesses(const std::string &from, const std::string &to);
template <typename T>
void RegisterRemoveConstraint(const std::string &cgroup,
Expand Down
Loading