Skip to content

Commit d61fdcd

Browse files
israbbaniedoakes
authored andcommitted
[core] (cgroups 10/n) Adding support in CgroupManager and CgroupDriver to move processes into system cgroup (#56446)
This PR stacks on #56352 . For more details about the resource isolation project see #54703. This PR the following functions to move a process into the system cgroup: * CgroupManagerInterface::AddProcessToSystemCgroup * CgroupDriverInterface::AddProcessToCgroup I've also added integration tests for SysFsCgroupDriver and unit tests for CgroupManager. Let me explain how these APIs will be used. In the next PR, the raylet will * be passed a list of pids of system processes that are started before the raylet starts and need to be moved into the system cgroup (e.g. gcs_server) * call CgroupManagerInterface::AddProcessToSystemCgroup for each of these pids to move them into the system cgroup. --------- Signed-off-by: Ibrahim Rabbani <[email protected]> Co-authored-by: Edward Oakes <[email protected]> Signed-off-by: elliot-barn <[email protected]>
1 parent ede2bdf commit d61fdcd

File tree

9 files changed

+291
-1
lines changed

9 files changed

+291
-1
lines changed

src/ray/common/cgroup2/cgroup_driver_interface.h

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -206,5 +206,26 @@ class CgroupDriverInterface {
206206
*/
207207
virtual StatusOr<std::unordered_set<std::string>> GetEnabledControllers(
208208
const std::string &cgroup) = 0;
209+
210+
/**
211+
Adds the process to the specified cgroup.
212+
213+
To move the pid, the process must have read, write, and execute permissions for the
214+
1) the cgroup the pid is currently in i.e. the source cgroup.
215+
2) the destination cgroup.
216+
3) the lowest common ancestor of the source and destination cgroups.
217+
218+
@param cgroup to move the process into.
219+
@param pid of the process that will be moved.
220+
221+
@return Status::OK if the process was moved successfully into the cgroup.
222+
@return Status::NotFound if the cgroup does not exist.
223+
@return Status::PermissionDenied if process doesn't have read, write, and execute
224+
permissions for the cgroup.
225+
@return Status::InvalidArgument if the pid is invalid, does not exist, or any other
226+
error.
227+
*/
228+
virtual Status AddProcessToCgroup(const std::string &cgroup,
229+
const std::string &pid) = 0;
209230
};
210231
} // namespace ray

src/ray/common/cgroup2/cgroup_manager.cc

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
#include "ray/common/cgroup2/cgroup_driver_interface.h"
2727
#include "ray/common/cgroup2/scoped_cgroup_operation.h"
2828
#include "ray/common/status_or.h"
29+
#include "ray/util/logging.h"
2930

3031
namespace ray {
3132

@@ -291,4 +292,22 @@ Status CgroupManager::Initialize(int64_t system_reserved_cpu_weight,
291292

292293
return Status::OK();
293294
}
295+
296+
Status CgroupManager::AddProcessToSystemCgroup(const std::string &pid) {
297+
Status s = cgroup_driver_->AddProcessToCgroup(system_leaf_cgroup_, pid);
298+
// TODO(#54703): Add link to OSS documentation once available.
299+
RAY_CHECK(!s.IsNotFound()) << "Failed to move process " << pid << " into system cgroup "
300+
<< system_leaf_cgroup_
301+
<< "because the cgroup was not found. "
302+
"If resource isolation is enabled, Ray's cgroup "
303+
"hierarchy must not be modified "
304+
"while Ray is running.";
305+
RAY_CHECK(!s.IsPermissionDenied())
306+
<< "Failed to move process " << pid << " into system cgroup " << system_leaf_cgroup_
307+
<< " because Ray does not have read, write, and execute "
308+
"permissions for the cgroup. If resource isolation is enabled, Ray's cgroup "
309+
"hierarchy must not be modified while Ray is running.";
310+
311+
return s;
312+
}
294313
} // namespace ray

src/ray/common/cgroup2/cgroup_manager.h

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -69,6 +69,28 @@ class CgroupManager : public CgroupManagerInterface {
6969
CgroupManager(CgroupManager &&);
7070
CgroupManager &operator=(CgroupManager &&);
7171

72+
/**
73+
Moves the process into the system leaf cgroup (@see
74+
CgroupManagerInterface::kLeafCgroupName).
75+
76+
To move the pid, the process must have read, write, and execute permissions for the
77+
1) the cgroup the pid is currently in i.e. the source cgroup.
78+
2) the system leaf cgroup i.e. the destination cgroup.
79+
3) the lowest common ancestor of the source and destination cgroups.
80+
81+
TODO(#54703): There currently is not a good way to signal to the caller that
82+
the method can cause a FATAL error. Revisit this once we've settled on a pattern.
83+
84+
NOTE: If the process does not have adequate cgroup permissions or the system leaf
85+
cgroup does not exist, this will fail a RAY_CHECK.
86+
87+
@param pid of the process to move into the system leaf cgroup.
88+
89+
@return Status::OK if pid moved successfully.
90+
@return Status::NotFound if the system cgroup does not exist.
91+
*/
92+
Status AddProcessToSystemCgroup(const std::string &pid) override;
93+
7294
/**
7395
Performs cleanup in reverse order from the Initialize function:
7496
1. remove resource constraints to the system and application cgroups.

src/ray/common/cgroup2/cgroup_manager_interface.h

Lines changed: 21 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,27 @@ class CgroupManagerInterface {
4545
// TODO(#54703): These will be implemented in a later PR to move processes
4646
// into a cgroup.
4747
// virtual Status AddProcessToApplicationCgroup(int) = 0;
48-
// virtual Status AddProcessToSystemCgroup(int) = 0;
48+
49+
/**
50+
Moves the process into the system leaf cgroup (@see kLeafCgroupName).
51+
52+
To move the pid, the process must have read, write, and execute permissions for the
53+
1) the cgroup the pid is currently in i.e. the source cgroup.
54+
2) the system leaf cgroup i.e. the destination cgroup.
55+
3) the lowest common ancestor of the source and destination cgroups.
56+
57+
TODO(#54703): There currently is not a good way to signal to the caller that
58+
the method can cause a FATAL error. Revisit this once we've settled on a pattern.
59+
60+
NOTE: If the process does not have adequate cgroup permissions or the system leaf
61+
cgroup does not exist, this will fail a RAY_CHECK.
62+
63+
@param pid of the process to move into the system leaf cgroup.
64+
65+
@return Status::OK if pid moved successfully.
66+
@return Status::NotFound if the system cgroup does not exist.
67+
*/
68+
virtual Status AddProcessToSystemCgroup(const std::string &pid) = 0;
4969

5070
/**
5171
Cleans up the cgroup hierarchy, disables all controllers and removes all

src/ray/common/cgroup2/fake_cgroup_driver.h

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -126,6 +126,7 @@ class FakeCgroupDriver : public CgroupDriverInterface {
126126
Status add_constraint_s_ = Status::OK();
127127
Status available_controllers_s_ = Status::OK();
128128
Status enabled_controllers_s_ = Status::OK();
129+
Status add_process_to_cgroup_s_ = Status::OK();
129130

130131
// These have no side-effects.
131132
Status CheckCgroupv2Enabled() override { return check_cgroup_enabled_s_; }
@@ -222,6 +223,10 @@ class FakeCgroupDriver : public CgroupDriverInterface {
222223
}
223224
return (*cgroups_)[cgroup].enabled_controllers_;
224225
}
226+
227+
Status AddProcessToCgroup(const std::string &cgroup, const std::string &pid) override {
228+
return add_process_to_cgroup_s_;
229+
}
225230
};
226231

227232
} // namespace ray

src/ray/common/cgroup2/integration_tests/sysfs_cgroup_driver_integration_test.cc

Lines changed: 68 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -622,4 +622,72 @@ TEST_F(SysFsCgroupDriverIntegrationTest, AddResourceConstraintSucceeds) {
622622
Status s = driver.AddConstraint(cgroup->GetPath(), "cpu", "cpu.weight", "500");
623623
ASSERT_TRUE(s.ok()) << s.ToString();
624624
}
625+
626+
TEST_F(SysFsCgroupDriverIntegrationTest, AddProcessToCgroupFailsIfCgroupDoesNotExist) {
627+
auto cgroup_or_status = TempCgroupDirectory::Create(test_cgroup_path_, S_IRWXU);
628+
ASSERT_TRUE(cgroup_or_status.ok()) << cgroup_or_status.ToString();
629+
auto cgroup = std::move(cgroup_or_status.value());
630+
std::string non_existent_path =
631+
cgroup->GetPath() + std::filesystem::path::preferred_separator + "nope";
632+
SysFsCgroupDriver driver;
633+
Status s = driver.AddProcessToCgroup(non_existent_path, "123");
634+
ASSERT_TRUE(s.IsNotFound()) << s.ToString();
635+
}
636+
637+
TEST_F(SysFsCgroupDriverIntegrationTest,
638+
AddProcessToCgroupFailsIfCNotReadWriteExecPermissionsForCgroup) {
639+
auto cgroup_or_status = TempCgroupDirectory::Create(test_cgroup_path_, S_IREAD);
640+
ASSERT_TRUE(cgroup_or_status.ok()) << cgroup_or_status.ToString();
641+
auto cgroup = std::move(cgroup_or_status.value());
642+
SysFsCgroupDriver driver;
643+
Status s = driver.AddProcessToCgroup(cgroup->GetPath(), "123");
644+
ASSERT_TRUE(s.IsPermissionDenied()) << s.ToString();
645+
}
646+
647+
TEST_F(SysFsCgroupDriverIntegrationTest, AddProcessToCgroupFailsIfProcessDoesNotExist) {
648+
auto cgroup_or_status = TempCgroupDirectory::Create(test_cgroup_path_, S_IRWXU);
649+
ASSERT_TRUE(cgroup_or_status.ok()) << cgroup_or_status.ToString();
650+
auto cgroup = std::move(cgroup_or_status.value());
651+
SysFsCgroupDriver driver;
652+
Status s = driver.AddProcessToCgroup(cgroup->GetPath(), "123");
653+
ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
654+
}
655+
656+
TEST_F(SysFsCgroupDriverIntegrationTest,
657+
AddProcessToCgroupSucceedsIfProcessExistsAndCorrectPermissions) {
658+
auto cgroup_or_status = TempCgroupDirectory::Create(test_cgroup_path_, S_IRWXU);
659+
ASSERT_TRUE(cgroup_or_status.ok()) << cgroup_or_status.ToString();
660+
auto cgroup = std::move(cgroup_or_status.value());
661+
auto child_cgroup_or_status = TempCgroupDirectory::Create(cgroup->GetPath(), S_IRWXU);
662+
ASSERT_TRUE(child_cgroup_or_status.ok()) << child_cgroup_or_status.ToString();
663+
auto child_cgroup = std::move(child_cgroup_or_status.value());
664+
StatusOr<std::pair<pid_t, int>> child_process_s =
665+
StartChildProcessInCgroup(cgroup->GetPath());
666+
ASSERT_TRUE(child_process_s.ok()) << child_process_s.ToString();
667+
auto [child_pid, child_pidfd] = child_process_s.value();
668+
SysFsCgroupDriver driver;
669+
Status s =
670+
driver.AddProcessToCgroup(child_cgroup->GetPath(), std::to_string(child_pid));
671+
ASSERT_TRUE(s.ok()) << s.ToString();
672+
// Assert that the child's pid is actually in the new file.
673+
std::string child_cgroup_procs_file_path = child_cgroup->GetPath() +
674+
std::filesystem::path::preferred_separator +
675+
"cgroup.procs";
676+
std::ifstream child_cgroup_procs_file(child_cgroup_procs_file_path);
677+
ASSERT_TRUE(child_cgroup_procs_file.is_open())
678+
<< "Could not open file " << child_cgroup_procs_file_path << ".";
679+
std::unordered_set<int> child_cgroup_pids;
680+
int pid = -1;
681+
while (child_cgroup_procs_file >> pid) {
682+
ASSERT_FALSE(child_cgroup_procs_file.fail())
683+
<< "Unable to read pid from file " << child_cgroup_procs_file_path;
684+
child_cgroup_pids.emplace(pid);
685+
}
686+
EXPECT_EQ(child_cgroup_pids.size(), 1);
687+
EXPECT_TRUE(child_cgroup_pids.find(child_pid) != child_cgroup_pids.end());
688+
Status terminate_s =
689+
TerminateChildProcessAndWaitForTimeout(child_pid, child_pidfd, 5000);
690+
ASSERT_TRUE(terminate_s.ok()) << terminate_s.ToString();
691+
}
692+
625693
} // namespace ray

src/ray/common/cgroup2/sysfs_cgroup_driver.cc

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -423,4 +423,35 @@ StatusOr<std::unordered_set<std::string>> SysFsCgroupDriver::ReadControllerFile(
423423
return StatusOr<std::unordered_set<std::string>>(controllers);
424424
}
425425

426+
Status SysFsCgroupDriver::AddProcessToCgroup(const std::string &cgroup,
427+
const std::string &process) {
428+
RAY_RETURN_NOT_OK(CheckCgroup(cgroup));
429+
std::filesystem::path cgroup_procs_file_path =
430+
cgroup / std::filesystem::path(kCgroupProcsFilename);
431+
432+
int fd = open(cgroup_procs_file_path.c_str(), O_RDWR);
433+
434+
if (fd == -1) {
435+
return Status::InvalidArgument(absl::StrFormat(
436+
"Failed to write pid %s to cgroup.procs for cgroup %s with error %s",
437+
process,
438+
cgroup,
439+
strerror(errno)));
440+
}
441+
442+
ssize_t bytes_written = write(fd, process.c_str(), process.size());
443+
444+
if (bytes_written != static_cast<ssize_t>(process.size())) {
445+
close(fd);
446+
return Status::InvalidArgument(absl::StrFormat(
447+
"Failed to write pid %s to cgroup.procs for cgroup %s with error %s",
448+
process,
449+
cgroup,
450+
strerror(errno)));
451+
}
452+
453+
close(fd);
454+
return Status::OK();
455+
}
456+
426457
} // namespace ray

src/ray/common/cgroup2/sysfs_cgroup_driver.h

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -255,6 +255,29 @@ class SysFsCgroupDriver : public CgroupDriverInterface {
255255
const std::string &constraint,
256256
const std::string &constraint_value) override;
257257

258+
/**
259+
Attempts to write pid to the cgroup.procs file of the specified cgroup.
260+
261+
To write a pid to a cgroup.procs file, the process must have read, write, and execute
262+
to the source, destination, and lowest-common ancestor of source and destination
263+
cgroups.
264+
265+
For more details, see the documentation:
266+
- @see https://docs.kernel.org/admin-guide/cgroup-v2.html#delegation-containment
267+
- @see https://docs.kernel.org/admin-guide/cgroup-v2.html#core-interface-files
268+
269+
@param cgroup to move the process into.
270+
@param pid pid of the process that will be moved.
271+
272+
@return Status::OK if the process was moved successfully into the cgroup.
273+
@return Status::NotFound if the cgroup does not exist.
274+
@return Status::PermissionDenied if current user doesn't have read, write, and execute
275+
permissions for the cgroup.
276+
@return Status::InvalidArgument if the pid is invalid, does not exist, or any other
277+
error.
278+
*/
279+
Status AddProcessToCgroup(const std::string &cgroup, const std::string &pid) override;
280+
258281
private:
259282
/**
260283
@param controller_file_path the absolute path of the controller file to read which is

src/ray/common/cgroup2/tests/cgroup_manager_test.cc

Lines changed: 81 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -283,4 +283,85 @@ TEST(CgroupManagerTest, CreateSucceedsWithCleanupInOrder) {
283283
ASSERT_EQ((*deleted_cgroups)[4].second, node_cgroup_path);
284284
}
285285

286+
TEST(CgroupManagerTest, AddProcessToSystemCgroupFailsIfInvalidProcess) {
287+
std::shared_ptr<std::unordered_map<std::string, FakeCgroup>> cgroups =
288+
std::make_shared<std::unordered_map<std::string, FakeCgroup>>();
289+
cgroups->emplace("/sys/fs/cgroup",
290+
FakeCgroup{"/sys/fs/cgroup", {5}, {}, {"cpu", "memory"}, {}});
291+
FakeCgroup base_cgroup{"/sys/fs/cgroup"};
292+
293+
std::unique_ptr<FakeCgroupDriver> driver = FakeCgroupDriver::Create(cgroups);
294+
driver->add_process_to_cgroup_s_ = Status::InvalidArgument("");
295+
296+
auto cgroup_manager_s = CgroupManager::Create(
297+
"/sys/fs/cgroup", "node_id_123", 100, 1000000, std::move(driver));
298+
ASSERT_TRUE(cgroup_manager_s.ok()) << cgroup_manager_s.ToString();
299+
300+
std::unique_ptr<CgroupManager> cgroup_manager = std::move(cgroup_manager_s.value());
301+
Status s = cgroup_manager->AddProcessToSystemCgroup("-1");
302+
ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
303+
}
304+
305+
TEST(CgroupManagerTest, AddProcessToSystemCgroupIsFatalIfSystemCgroupDoesNotExist) {
306+
std::shared_ptr<std::unordered_map<std::string, FakeCgroup>> cgroups =
307+
std::make_shared<std::unordered_map<std::string, FakeCgroup>>();
308+
cgroups->emplace("/sys/fs/cgroup",
309+
FakeCgroup{"/sys/fs/cgroup", {5}, {}, {"cpu", "memory"}, {}});
310+
FakeCgroup base_cgroup{"/sys/fs/cgroup"};
311+
312+
std::unique_ptr<FakeCgroupDriver> driver = FakeCgroupDriver::Create(cgroups);
313+
driver->add_process_to_cgroup_s_ = Status::NotFound("");
314+
315+
auto cgroup_manager_s = CgroupManager::Create(
316+
"/sys/fs/cgroup", "node_id_123", 100, 1000000, std::move(driver));
317+
ASSERT_TRUE(cgroup_manager_s.ok()) << cgroup_manager_s.ToString();
318+
319+
std::unique_ptr<CgroupManager> cgroup_manager = std::move(cgroup_manager_s.value());
320+
321+
EXPECT_DEATH((void)cgroup_manager->AddProcessToSystemCgroup("-1"),
322+
"Failed to move.*not found");
323+
}
324+
325+
TEST(CgroupManagerTest,
326+
AddProcessToSystemCgroupIsFatalIfProcessDoesNotHavePermissionsForSystemCgroup) {
327+
std::shared_ptr<std::unordered_map<std::string, FakeCgroup>> cgroups =
328+
std::make_shared<std::unordered_map<std::string, FakeCgroup>>();
329+
cgroups->emplace("/sys/fs/cgroup",
330+
FakeCgroup{"/sys/fs/cgroup", {5}, {}, {"cpu", "memory"}, {}});
331+
FakeCgroup base_cgroup{"/sys/fs/cgroup"};
332+
333+
std::unique_ptr<FakeCgroupDriver> driver = FakeCgroupDriver::Create(cgroups);
334+
driver->add_process_to_cgroup_s_ = Status::PermissionDenied("");
335+
336+
auto cgroup_manager_s = CgroupManager::Create(
337+
"/sys/fs/cgroup", "node_id_123", 100, 1000000, std::move(driver));
338+
ASSERT_TRUE(cgroup_manager_s.ok()) << cgroup_manager_s.ToString();
339+
340+
std::unique_ptr<CgroupManager> cgroup_manager = std::move(cgroup_manager_s.value());
341+
342+
EXPECT_DEATH((void)cgroup_manager->AddProcessToSystemCgroup("-1"),
343+
"Failed to move.*permissions");
344+
}
345+
346+
TEST(
347+
CgroupManagerTest,
348+
AddProcessToSystemCgroupSucceedsIfSystemCgroupExistsWithCorrectPermissionsAndValidProcess) {
349+
std::shared_ptr<std::unordered_map<std::string, FakeCgroup>> cgroups =
350+
std::make_shared<std::unordered_map<std::string, FakeCgroup>>();
351+
cgroups->emplace("/sys/fs/cgroup",
352+
FakeCgroup{"/sys/fs/cgroup", {5}, {}, {"cpu", "memory"}, {}});
353+
FakeCgroup base_cgroup{"/sys/fs/cgroup"};
354+
355+
std::unique_ptr<FakeCgroupDriver> driver = FakeCgroupDriver::Create(cgroups);
356+
357+
auto cgroup_manager_s = CgroupManager::Create(
358+
"/sys/fs/cgroup", "node_id_123", 100, 1000000, std::move(driver));
359+
ASSERT_TRUE(cgroup_manager_s.ok()) << cgroup_manager_s.ToString();
360+
361+
std::unique_ptr<CgroupManager> cgroup_manager = std::move(cgroup_manager_s.value());
362+
363+
Status s = cgroup_manager->AddProcessToSystemCgroup("5");
364+
ASSERT_TRUE(s.ok()) << s.ToString();
365+
}
366+
286367
} // namespace ray

0 commit comments

Comments
 (0)