Skip to content
Merged
Show file tree
Hide file tree
Changes from 14 commits
Commits
Show all changes
20 commits
Select commit Hold shift + click to select a range
599f968
[wip] [core] (cgroups 14/n) Clean up bazel targets and expose just the
israbbani Sep 30, 2025
77f6b64
[core] Cleaning up Cgroup related bazel targets. CgroupManagerInteface
israbbani Oct 6, 2025
9fd1160
[core] (cgroups 14/n) Clean up bazel targets and enable cross-platform
israbbani Oct 6, 2025
a5f4b5a
Merge branch 'master' into irabbani/cgroups-14
israbbani Oct 6, 2025
762b5cf
Merge branch 'irabbani/cgroups-14' of github.com:ray-project/ray into…
israbbani Oct 6, 2025
8d07f6f
Merge branch 'master' into irabbani/cgroups-14
israbbani Oct 6, 2025
b92677e
Forgot to use clang locally
israbbani Oct 7, 2025
a870d5a
Merge branch 'irabbani/cgroups-14' of github.com:ray-project/ray into…
israbbani Oct 7, 2025
e34f19b
Unused imports
israbbani Oct 7, 2025
55f1ec8
unused includes breaking the build
israbbani Oct 7, 2025
4c7545e
fixing the macos build
israbbani Oct 7, 2025
aef6bd8
Merge branch 'master' into irabbani/cgroups-14
israbbani Oct 7, 2025
59366ce
move operators for NoopCgroupManager
israbbani Oct 7, 2025
44ab09e
Merge branch 'irabbani/cgroups-14' of github.com:ray-project/ray into…
israbbani Oct 7, 2025
0c8d8e3
Update src/ray/common/cgroup2/cgroup_manager_factory.h
israbbani Oct 7, 2025
6dc39ad
feedback
israbbani Oct 7, 2025
60d77bb
up
israbbani Oct 8, 2025
bfd2482
Merge branch 'master' into irabbani/cgroups-14
israbbani Oct 9, 2025
9174357
Merge branch 'master' into irabbani/cgroups-14
israbbani Oct 10, 2025
ab526bf
Merge branch 'master' into irabbani/cgroups-14
israbbani Oct 10, 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
96 changes: 51 additions & 45 deletions src/ray/common/cgroup2/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,104 +5,110 @@ config_setting(
constraint_values = ["@platforms//os:linux"],
)

# Public targets.
# The module exposes only two public targets.
# "cgroup_manager_factory" to create a CgroupManager
# "cgroup_manager_interface" to use the public API of CgroupManager.
ray_cc_library(
name = "cgroup_manager",
name = "cgroup_manager_factory",
srcs = select({
":is_linux": ["cgroup_manager.cc"],
"//conditions:default": ["noop_cgroup_manager.cc"],
":is_linux": [
"linux_cgroup_manager_factory.cc",
],
"//conditions:default": [
"noop_cgroup_manager_factory.cc",
],
}),
hdrs = [
"cgroup_manager.h",
"scoped_cgroup_operation.h",
"cgroup_manager_factory.h",
],
visibility = ["//visibility:public"],
deps = [
":cgroup_driver_interface",
":cgroup_manager_interface",
"//src/ray/common:status",
"//src/ray/common:status_or",
":noop_cgroup_manager",
"//src/ray/util:logging",
] + select({
":is_linux": [
"//src/ray/util:logging",
":cgroup_driver_interface",
":cgroup_manager",
":sysfs_cgroup_driver",
"//src/ray/common:status",
"//src/ray/common:status_or",
"@com_google_absl//absl/strings",
],
"//conditions:default": [],
}),
)

ray_cc_library(
name = "cgroup_driver_interface",
name = "cgroup_manager_interface",
hdrs = [
"cgroup_driver_interface.h",
"cgroup_manager_interface.h",
],
visibility = ["//visibility:public"],
deps = [
":cgroup_driver_interface",
"//src/ray/common:status",
"//src/ray/common:status_or",
],
)

# Private targets
ray_cc_library(
name = "cgroup_manager_interface",
name = "cgroup_manager",
srcs = [
"cgroup_manager.cc",
],
hdrs = [
"cgroup_manager_interface.h",
"cgroup_manager.h",
"scoped_cgroup_operation.h",
],
visibility = ["//visibility:public"],
visibility = [":__subpackages__"],
deps = [
":cgroup_driver_interface",
":cgroup_manager_interface",
"//src/ray/common:status",
"//src/ray/common:status_or",
"//src/ray/util:logging",
],
)

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

# Private targets
#
# TODO(#54703): This target builds the noop implementations.
# There's a corressponding test that runs on Linux and Non-Linux
# CI so breakages are caught in premerge before these targets are
# cleaned up at the end of the resource isolation milestone 1
# project.
ray_cc_library(
name = "noop_cgroup_targets",
srcs = [
"noop_cgroup_manager.cc",
"noop_sysfs_cgroup_driver.cc",
],
name = "cgroup_driver_interface",
hdrs = [
"cgroup_manager.h",
"scoped_cgroup_operation.h",
"sysfs_cgroup_driver.h",
"cgroup_driver_interface.h",
],
visibility = [":__subpackages__"],
deps = [
"//src/ray/common:status",
"//src/ray/common:status_or",
],
)

ray_cc_library(
name = "sysfs_cgroup_driver",
srcs = ["sysfs_cgroup_driver.cc"],
hdrs = ["sysfs_cgroup_driver.h"],
visibility = [":__subpackages__"],
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",
],
)

Expand Down
5 changes: 1 addition & 4 deletions src/ray/common/cgroup2/cgroup_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -78,10 +78,7 @@ class CgroupManager : public CgroupManagerInterface {
2) the system leaf cgroup i.e. the destination cgroup.
3) the lowest common ancestor of the source and destination cgroups.

TODO(#54703): There currently is not a good way to signal to the caller that
the method can cause a FATAL error. Revisit this once we've settled on a pattern.

NOTE: If the process does not have adequate cgroup permissions or the application leaf
@note If the process does not have adequate cgroup permissions or the application leaf
cgroup does not exist, this will fail a RAY_CHECK.

@param pid of the process to move into the application leaf cgroup.
Expand Down
69 changes: 69 additions & 0 deletions src/ray/common/cgroup2/cgroup_manager_factory.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
// 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.
#pragma once

#include <memory>
#include <string>

#include "ray/common/cgroup2/cgroup_manager_interface.h"

namespace ray {

// TODO(54703): Refactor the configs into a struct called CgroupManagerConfig
// and delegate input validation and error messages to it.
Comment on lines +23 to +24
Copy link
Collaborator

Choose a reason for hiding this comment

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

yeah!

class CgroupManagerFactory {
public:
/**

This feature is only enabled in Linux. If using Linux, validates inputs, creates the
ray's cgroup heirarchy, enables constraints, and moves all system processes into the
system cgroup.

On non-Linux platforms, this will return a noop implementation.

@param enable_resource_isolation if true, will create process isolation with using
cgroups (@see CgroupManager::Create for more information).
@param cgroup_path the cgroup that the process will take ownership of.
@param node_id used to create a unique cgroup subtree per running ray node.
@param system_reserved_cpu_weight a value between [1,10000] to assign to the cgroup
for system processes. The cgroup for application processes gets 10000 -
system_reserved_cpu_weight.
@param system_reserved_memory_bytes used to reserve memory for the system cgroup.
@param system_pids a comma-separated list of pids of ray system processes to move into
the system cgroup.

For more information about the parameters, see @ref CgroupManager::Create.

@note any of the following is undefined behavior and will cause a RAY_CHECK to fail
1. enable_resource_isolation is true and either
a. cgroup_path is empty
b. system_reserved_cpu_weight or system_reserved_memory_bytes are -1.
2. The CgroupManager's precondition checks fail
a. cgroupv2 is not mounted correctly in unified mode (see @ref
CgroupDriverInterface::CheckCgroupv2Enabled).
b. the current process does not adequate permissions (see @ref
CgroupManager::Create).
c. supported cgroup controllers are not available (see @ref
CgroupManager::supported_controllers_).
3. if a process in system_pids cannot be moved into the system cgroup.
*/
static std::unique_ptr<CgroupManagerInterface> Create(
bool enable_resource_isolation,
std::string cgroup_path,
const std::string &node_id,
const int64_t system_reserved_cpu_weight,
const int64_t system_reserved_memory_bytes,
const std::string &system_pids);
};
} // namespace ray
7 changes: 1 addition & 6 deletions src/ray/common/cgroup2/cgroup_manager_interface.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,6 @@
// limitations under the License.
#pragma once

#include <sys/types.h>

#include <limits>
#include <memory>
#include <string>
Expand Down Expand Up @@ -50,10 +48,7 @@ class CgroupManagerInterface {
2) the system leaf cgroup i.e. the destination cgroup.
3) the lowest common ancestor of the source and destination cgroups.

TODO(#54703): There currently is not a good way to signal to the caller that
the method can cause a FATAL error. Revisit this once we've settled on a pattern.

NOTE: If the process does not have adequate cgroup permissions or the application leaf
@note If the process does not have adequate cgroup permissions or the application leaf
cgroup does not exist, this will fail a RAY_CHECK.

@param pid of the process to move into the system leaf cgroup.
Expand Down
83 changes: 83 additions & 0 deletions src/ray/common/cgroup2/linux_cgroup_manager_factory.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
// 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 <sys/types.h>
#include <unistd.h>

#include <memory>
#include <string>
#include <utility>
#include <vector>

#include "absl/strings/str_format.h"
#include "absl/strings/str_split.h"
#include "ray/common/cgroup2/cgroup_driver_interface.h"
#include "ray/common/cgroup2/cgroup_manager.h"
#include "ray/common/cgroup2/cgroup_manager_factory.h"
#include "ray/common/cgroup2/cgroup_manager_interface.h"
#include "ray/common/cgroup2/noop_cgroup_manager.h"
#include "ray/common/cgroup2/sysfs_cgroup_driver.h"

namespace ray {

std::unique_ptr<CgroupManagerInterface> CgroupManagerFactory::Create(
bool enable_resource_isolation,
std::string cgroup_path,
const std::string &node_id,
const int64_t system_reserved_cpu_weight,
const int64_t system_reserved_memory_bytes,
const std::string &system_pids) {
if (!enable_resource_isolation) {
return std::make_unique<NoopCgroupManager>();
}

RAY_CHECK(!cgroup_path.empty())
<< "Failed to start CgroupManager. If enable_resource_isolation is set to true, "
"cgroup_path cannot be empty.";
Comment on lines +44 to +46
Copy link
Collaborator

Choose a reason for hiding this comment

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

in the general case, should we structure the factories so that they RAY_CHECK on initialization failure or return a status code? the paranoid/perfectionist version would be the latter. practically I'm not sure that it makes much of a difference.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Capturing our discussion for posterity.

If there's a panic or a fatal error, I would keep the actual RAY_CHECK as close to the error as possible. You're not worried about clean up and you can dump as much context as possible.

I would return a Status is if there's a chance that the caller can either recover provide more useful information.
However, this has the unpleasant side-effect of having landmines buried deep in the call stack.. which makes me sad.

The antidote to this sadness is to have as few FATAL errors as possible and most failures should be recoverable.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Invariant checking on component startups or detecting misconfigurations is a valid use of RAY_CHECK.


RAY_CHECK_NE(system_reserved_cpu_weight, -1)
<< "Failed to start CgroupManager. If enable_resource_isolation is set to true, "
"system_reserved_cpu_weight must be set to a value between [1,10000]";

RAY_CHECK_NE(system_reserved_memory_bytes, -1)
<< "Failed to start CgroupManager. If enable_resource_isolation is set to true, "
"system_reserved_memory_bytes must be set to a value > 0";

StatusOr<std::unique_ptr<CgroupManagerInterface>> cgroup_manager_s =
CgroupManager::Create(cgroup_path,
node_id,
system_reserved_cpu_weight,
system_reserved_memory_bytes,
std::make_unique<SysFsCgroupDriver>());

RAY_CHECK(cgroup_manager_s.ok()) << absl::StrFormat(
"Failed to start CgroupManager due to %s.", cgroup_manager_s.ToString());

std::unique_ptr<CgroupManagerInterface> cgroup_manager =
std::move(cgroup_manager_s.value());

std::vector<std::string> system_pids_to_move;
if (!system_pids.empty()) {
system_pids_to_move = std::move(absl::StrSplit(system_pids, ","));
}

system_pids_to_move.emplace_back(std::to_string(getpid()));

for (const auto &pid : system_pids_to_move) {
RAY_CHECK_OK(cgroup_manager->AddProcessToSystemCgroup(pid))
<< absl::StrFormat("Failed to move process with pid %s into system cgroup.", pid);
}

return cgroup_manager;
}
} // namespace ray
48 changes: 0 additions & 48 deletions src/ray/common/cgroup2/noop_cgroup_manager.cc

This file was deleted.

Loading