[core] avoid scheduling on gpu nodes by default (#18743)

* [core] avoid scheduling on gpu nodes by default

* Fix cluster_task_manager_test tests.

Made most tests in cluster_task_manager_test not use GPU on the head
node.

Also added another test to scheduling_policy_test.

Co-authored-by: Sasha Sobol <sasha@asobol.com>
This commit is contained in:
Ameer Haj Ali 2021-09-20 17:38:40 -07:00 committed by GitHub
parent 65c1c8bb9e
commit 9efbd80733
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
3 changed files with 19 additions and 6 deletions

View file

@ -473,4 +473,4 @@ RAY_CONFIG(bool, event_log_reporter_enabled, false)
RAY_CONFIG(std::string, event_level, "warning")
/// Whether to avoid scheduling cpu requests on gpu nodes
RAY_CONFIG(bool, scheduler_avoid_gpu_nodes, false)
RAY_CONFIG(bool, scheduler_avoid_gpu_nodes, true)

View file

@ -102,10 +102,10 @@ class MockWorkerPool : public WorkerPoolInterface {
};
std::shared_ptr<ClusterResourceScheduler> CreateSingleNodeScheduler(
const std::string &id) {
const std::string &id, double num_gpus = 0.0) {
std::unordered_map<std::string, double> local_node_resources;
local_node_resources[ray::kCPU_ResourceLabel] = 8;
local_node_resources[ray::kGPU_ResourceLabel] = 4;
local_node_resources[ray::kGPU_ResourceLabel] = num_gpus;
local_node_resources[ray::kMemory_ResourceLabel] = 128;
auto scheduler = std::make_shared<ClusterResourceScheduler>(
@ -177,9 +177,9 @@ class MockTaskDependencyManager : public TaskDependencyManagerInterface {
class ClusterTaskManagerTest : public ::testing::Test {
public:
ClusterTaskManagerTest()
ClusterTaskManagerTest(double num_gpus_at_head = 0.0)
: id_(NodeID::FromRandom()),
scheduler_(CreateSingleNodeScheduler(id_.Binary())),
scheduler_(CreateSingleNodeScheduler(id_.Binary(), num_gpus_at_head)),
is_owner_alive_(true),
node_info_calls_(0),
announce_infeasible_task_calls_(0),
@ -284,6 +284,12 @@ class ClusterTaskManagerTest : public ::testing::Test {
ClusterTaskManager task_manager_;
};
// Same as ClusterTaskManagerTest, but the head node starts with 4.0 num gpus.
class ClusterTaskManagerTestWithGPUsAtHead : public ClusterTaskManagerTest {
public:
ClusterTaskManagerTestWithGPUsAtHead() : ClusterTaskManagerTest(4.0) {}
};
TEST_F(ClusterTaskManagerTest, BasicTest) {
/*
Test basic scheduler functionality:
@ -1196,7 +1202,7 @@ TEST_F(ClusterTaskManagerTest, FeasibleToNonFeasible) {
task1.GetTaskSpecification().TaskId());
}
TEST_F(ClusterTaskManagerTest, RleaseAndReturnWorkerCpuResources) {
TEST_F(ClusterTaskManagerTestWithGPUsAtHead, RleaseAndReturnWorkerCpuResources) {
const NodeResources &node_resources = scheduler_->GetLocalNodeResources();
ASSERT_EQ(node_resources.predefined_resources[PredefinedResources::CPU].available, 8);
ASSERT_EQ(node_resources.predefined_resources[PredefinedResources::GPU].available, 4);

View file

@ -269,6 +269,13 @@ TEST_F(SchedulingPolicyTest, AvoidSchedulingCPURequestsOnGPUNodes) {
req, local_node, nodes, 0.51, false, true, true);
ASSERT_EQ(to_schedule, local_node);
}
{
// A CPU request can be be scheduled on a CPU node.
const ResourceRequest req = ResourceMapToResourceRequest(map, {{"CPU", 1}}, false);
const int to_schedule = raylet_scheduling_policy::HybridPolicy(
req, local_node, nodes, 0.51, false, true, true);
ASSERT_EQ(to_schedule, remote_node);
}
{
// A mixed CPU/GPU request should be scheduled on a GPU node.
const ResourceRequest req =