Remove old scheduler and friends (#14184)

This commit is contained in:
Eric Liang 2021-03-03 18:29:15 -08:00 committed by GitHub
parent 3f6c23e3cc
commit 99a63b3dd1
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
29 changed files with 89 additions and 4160 deletions

View file

@ -866,25 +866,6 @@ cc_test(
],
)
cc_test(
name = "reconstruction_policy_test",
srcs = ["src/ray/raylet/reconstruction_policy_test.cc"],
copts = COPTS + select({
"//:msvc-cl": [
],
"//conditions:default": [
# Ignore this warning since it's impractical to fix in the relevant headers
"-Wno-inconsistent-missing-override",
],
}),
deps = [
":node_manager_fbs",
":object_manager",
":raylet_lib",
"@com_google_googletest//:gtest_main",
],
)
cc_test(
name = "worker_pool_test",
srcs = ["src/ray/raylet/worker_pool_test.cc"],

View file

@ -66,7 +66,6 @@ py_test_module_list(
"test_stress.py",
"test_stress_sharded.py",
"test_tensorflow.py",
"test_unreconstructable_errors.py",
],
size = "medium",
extra_srcs = SRCS,

View file

@ -1,30 +0,0 @@
import numpy as np
import unittest
import ray
from ray.test_utils import put_unpinned_object
class TestObjectLostErrors(unittest.TestCase):
def setUp(self):
ray.init(
num_cpus=1,
object_store_memory=150 * 1024 * 1024,
_redis_max_memory=10**8)
def tearDown(self):
ray.shutdown()
def testDriverPutEvictedCannotReconstruct(self):
x_id = put_unpinned_object(np.zeros(1 * 1024 * 1024))
ray.get(x_id)
for _ in range(20):
ray.put(np.zeros(10 * 1024 * 1024))
self.assertRaises(ray.exceptions.ObjectLostError,
lambda: ray.get(x_id))
if __name__ == "__main__":
import pytest
import sys
sys.exit(pytest.main(["-v", __file__]))

View file

@ -100,13 +100,6 @@ RAY_CONFIG(size_t, free_objects_batch_size, 100)
RAY_CONFIG(bool, lineage_pinning_enabled, false)
/// Whether to enable the new scheduler. The new scheduler is designed
/// only to work with direct calls. Once direct calls are becoming
/// the default, this scheduler will also become the default.
RAY_CONFIG(bool, new_scheduler_enabled,
getenv("RAY_ENABLE_NEW_SCHEDULER") == nullptr ||
getenv("RAY_ENABLE_NEW_SCHEDULER") == std::string("1"))
// The max allowed size in bytes of a return object from direct actor calls.
// Objects larger than this size will be spilled/promoted to plasma.
RAY_CONFIG(int64_t, max_direct_call_object_size, 100 * 1024)

View file

@ -175,8 +175,6 @@ class TaskSpecification : public MessageWrapper<rpc::TaskSpec> {
ObjectID PreviousActorTaskDummyObjectId() const;
bool IsDirectCall() const;
int MaxActorConcurrency() const;
bool IsAsyncioActor() const;

View file

@ -17,8 +17,11 @@
namespace ray {
OwnershipBasedObjectDirectory::OwnershipBasedObjectDirectory(
boost::asio::io_service &io_service, std::shared_ptr<gcs::GcsClient> &gcs_client)
: ObjectDirectory(io_service, gcs_client), client_call_manager_(io_service) {}
boost::asio::io_service &io_service, std::shared_ptr<gcs::GcsClient> &gcs_client,
std::function<void(const ObjectID &)> mark_as_failed)
: ObjectDirectory(io_service, gcs_client),
client_call_manager_(io_service),
mark_as_failed_(mark_as_failed) {}
namespace {
@ -38,6 +41,7 @@ void FilterRemovedNodes(std::shared_ptr<gcs::GcsClient> gcs_client,
bool UpdateObjectLocations(const rpc::GetObjectLocationsOwnerReply &location_reply,
const Status &status, const ObjectID &object_id,
std::shared_ptr<gcs::GcsClient> gcs_client,
std::function<void(const ObjectID &)> mark_as_failed,
std::unordered_set<NodeID> *node_ids, std::string *spilled_url,
NodeID *spilled_node_id, size_t *object_size) {
bool is_updated = false;
@ -45,12 +49,14 @@ bool UpdateObjectLocations(const rpc::GetObjectLocationsOwnerReply &location_rep
std::unordered_set<NodeID> new_node_ids;
if (!status.ok()) {
RAY_LOG(INFO) << "Failed to return location updates to subscribers for " << object_id
RAY_LOG(INFO) << "Failed to return location updates to subscribers for " << object_id
<< ": " << status.ToString()
<< ", assuming that the object was freed or evicted.";
// When we can't get location updates from the owner, we assume that the object was
// freed or evicted, so we send an empty location update to all subscribers.
// When we can't get location updates from the owner, we assume that the object
// was freed or evicted, so we send an empty location update to all subscribers.
*node_ids = new_node_ids;
// Mark the object as failed immediately here since we know it can never appear.
mark_as_failed(object_id);
is_updated = true;
} else {
// The size can be 0 if the update was a deletion. This assumes that an
@ -193,7 +199,7 @@ void OwnershipBasedObjectDirectory::SubscriptionCallback(
it->second.subscribed = true;
// Update entries for this object.
if (UpdateObjectLocations(reply, status, object_id, gcs_client_,
if (UpdateObjectLocations(reply, status, object_id, gcs_client_, mark_as_failed_,
&it->second.current_object_locations, &it->second.spilled_url,
&it->second.spilled_node_id, &it->second.object_size)) {
RAY_LOG(DEBUG) << "Pushing location updates to subscribers for object " << object_id
@ -275,9 +281,9 @@ ray::Status OwnershipBasedObjectDirectory::SubscribeObjectLocations(
<< " locations, spilled_url: " << spilled_url
<< ", spilled node ID: " << spilled_node_id
<< ", object size: " << object_size;
// We post the callback to the event loop in order to avoid mutating data structures
// shared with the caller and potentially invalidating caller iterators.
// See https://github.com/ray-project/ray/issues/2959.
// We post the callback to the event loop in order to avoid mutating data
// structures shared with the caller and potentially invalidating caller
// iterators. See https://github.com/ray-project/ray/issues/2959.
io_service_.post(
[callback, locations, spilled_url, spilled_node_id, object_size, object_id]() {
callback(object_id, locations, spilled_url, spilled_node_id, object_size);
@ -312,9 +318,9 @@ ray::Status OwnershipBasedObjectDirectory::LookupLocations(
auto &spilled_url = it->second.spilled_url;
auto &spilled_node_id = it->second.spilled_node_id;
auto object_size = it->second.object_size;
// We post the callback to the event loop in order to avoid mutating data structures
// shared with the caller and potentially invalidating caller iterators.
// See https://github.com/ray-project/ray/issues/2959.
// We post the callback to the event loop in order to avoid mutating data
// structures shared with the caller and potentially invalidating caller
// iterators. See https://github.com/ray-project/ray/issues/2959.
io_service_.post(
[callback, object_id, locations, spilled_url, spilled_node_id, object_size]() {
callback(object_id, locations, spilled_url, spilled_node_id, object_size);
@ -325,9 +331,9 @@ ray::Status OwnershipBasedObjectDirectory::LookupLocations(
if (rpc_client == nullptr) {
RAY_LOG(WARNING) << "Object " << object_id << " does not have owner. "
<< "LookupLocations returns an empty list of locations.";
// We post the callback to the event loop in order to avoid mutating data structures
// shared with the caller and potentially invalidating caller iterators.
// See https://github.com/ray-project/ray/issues/2959.
// We post the callback to the event loop in order to avoid mutating data
// structures shared with the caller and potentially invalidating caller
// iterators. See https://github.com/ray-project/ray/issues/2959.
io_service_.post([callback, object_id]() {
callback(object_id, std::unordered_set<NodeID>(), "", NodeID::Nil(), 0);
});
@ -350,8 +356,8 @@ ray::Status OwnershipBasedObjectDirectory::LookupLocations(
std::string spilled_url;
NodeID spilled_node_id;
size_t object_size = 0;
UpdateObjectLocations(reply, status, object_id, gcs_client_, &node_ids,
&spilled_url, &spilled_node_id, &object_size);
UpdateObjectLocations(reply, status, object_id, gcs_client_, mark_as_failed_,
&node_ids, &spilled_url, &spilled_node_id, &object_size);
RAY_LOG(DEBUG) << "Looked up locations for " << object_id
<< ", returning: " << node_ids.size()
<< " locations, spilled_url: " << spilled_url

View file

@ -40,7 +40,8 @@ class OwnershipBasedObjectDirectory : public ObjectDirectory {
/// \param gcs_client A Ray GCS client to request object and node
/// information from.
OwnershipBasedObjectDirectory(boost::asio::io_service &io_service,
std::shared_ptr<gcs::GcsClient> &gcs_client);
std::shared_ptr<gcs::GcsClient> &gcs_client,
std::function<void(const ObjectID &)> mark_as_failed);
virtual ~OwnershipBasedObjectDirectory() {}
@ -70,6 +71,8 @@ class OwnershipBasedObjectDirectory : public ObjectDirectory {
private:
/// The client call manager used to create the RPC clients.
rpc::ClientCallManager client_call_manager_;
/// The callback used to mark an object as failed.
std::function<void(const ObjectID &)> mark_as_failed_;
/// Cache of gRPC clients to workers (not necessarily running on this node).
/// Also includes the number of inflight requests to each worker - when this
/// reaches zero, the client will be deleted and a new one will need to be created

View file

@ -30,9 +30,6 @@ void DependencyManager::RemoveObjectIfNotNeeded(
<< " request: " << required_object_it->second.wait_request_id;
object_manager_.CancelPull(required_object_it->second.wait_request_id);
}
if (!local_objects_.count(object_id)) {
reconstruction_policy_.Cancel(object_id);
}
required_objects_.erase(required_object_it);
}
}
@ -43,9 +40,6 @@ DependencyManager::GetOrInsertRequiredObject(const ObjectID &object_id,
auto it = required_objects_.find(object_id);
if (it == required_objects_.end()) {
it = required_objects_.emplace(object_id, ref).first;
if (local_objects_.count(object_id) == 0) {
reconstruction_policy_.ListenAndMaybeReconstruct(object_id, ref.owner_address());
}
}
return it;
}
@ -233,10 +227,6 @@ std::vector<TaskID> DependencyManager::HandleObjectMissing(
}
task_entry.num_missing_dependencies++;
}
// The object is missing and needed so wait for a possible failure again.
reconstruction_policy_.ListenAndMaybeReconstruct(object_entry->first,
object_entry->second.owner_address);
}
// Process callbacks for all of the tasks dependent on the object that are
@ -285,7 +275,6 @@ std::vector<TaskID> DependencyManager::HandleObjectLocal(const ray::ObjectID &ob
object_manager_.CancelPull(object_entry->second.wait_request_id);
object_entry->second.wait_request_id = 0;
}
reconstruction_policy_.Cancel(object_entry->first);
RemoveObjectIfNotNeeded(object_entry);
}

View file

@ -19,7 +19,6 @@
#include "ray/common/id.h"
#include "ray/common/task/task.h"
#include "ray/object_manager/object_manager.h"
#include "ray/raylet/reconstruction_policy.h"
// clang-format on
namespace ray {
@ -28,8 +27,6 @@ namespace raylet {
using rpc::TaskLeaseData;
class ReconstructionPolicy;
/// Used for unit-testing the ClusterTaskManager, which requests dependencies
/// for queued tasks.
class TaskDependencyManagerInterface {
@ -53,9 +50,8 @@ class TaskDependencyManagerInterface {
class DependencyManager : public TaskDependencyManagerInterface {
public:
/// Create a task dependency manager.
DependencyManager(ObjectManagerInterface &object_manager,
ReconstructionPolicyInterface &reconstruction_policy)
: object_manager_(object_manager), reconstruction_policy_(reconstruction_policy) {}
DependencyManager(ObjectManagerInterface &object_manager)
: object_manager_(object_manager) {}
/// Check whether an object is locally available.
///
@ -225,13 +221,6 @@ class DependencyManager : public TaskDependencyManagerInterface {
/// The object manager, used to fetch required objects from remote nodes.
ObjectManagerInterface &object_manager_;
/// The reconstruction policy, used to reconstruct required objects that no
/// longer exist on any live nodes.
/// TODO(swang): This class is no longer needed for reconstruction, since the
/// object's owner handles reconstruction. We use this class as a timer to
/// detect the owner's failure. Remove this class and move the timer logic
/// into this class.
ReconstructionPolicyInterface &reconstruction_policy_;
/// A map from the ID of a queued task to metadata about whether the task's
/// dependencies are all local or not.

View file

@ -46,19 +46,10 @@ class MockObjectManager : public ObjectManagerInterface {
std::unordered_set<uint64_t> active_requests;
};
class MockReconstructionPolicy : public ReconstructionPolicyInterface {
public:
MOCK_METHOD2(ListenAndMaybeReconstruct,
void(const ObjectID &object_id, const rpc::Address &owner_address));
MOCK_METHOD1(Cancel, void(const ObjectID &object_id));
};
class DependencyManagerTest : public ::testing::Test {
public:
DependencyManagerTest()
: object_manager_mock_(),
reconstruction_policy_mock_(),
dependency_manager_(object_manager_mock_, reconstruction_policy_mock_) {}
: object_manager_mock_(), dependency_manager_(object_manager_mock_) {}
void AssertNoLeaks() {
ASSERT_TRUE(dependency_manager_.required_objects_.empty());
@ -70,7 +61,6 @@ class DependencyManagerTest : public ::testing::Test {
}
MockObjectManager object_manager_mock_;
MockReconstructionPolicy reconstruction_policy_mock_;
DependencyManager dependency_manager_;
};
@ -84,11 +74,6 @@ TEST_F(DependencyManagerTest, TestSimpleTask) {
arguments.push_back(ObjectID::FromRandom());
}
TaskID task_id = RandomTaskId();
// No objects have been registered in the task dependency manager, so all
// arguments should be remote.
for (const auto &argument_id : arguments) {
EXPECT_CALL(reconstruction_policy_mock_, ListenAndMaybeReconstruct(argument_id, _));
}
bool ready =
dependency_manager_.RequestTaskDependencies(task_id, ObjectIdsToRefs(arguments));
ASSERT_FALSE(ready);
@ -96,9 +81,6 @@ TEST_F(DependencyManagerTest, TestSimpleTask) {
// For each argument, tell the task dependency manager that the argument is
// local. All arguments should be canceled as they become available locally.
for (const auto &argument_id : arguments) {
EXPECT_CALL(reconstruction_policy_mock_, Cancel(argument_id));
}
auto ready_task_ids = dependency_manager_.HandleObjectLocal(arguments[0]);
ASSERT_TRUE(ready_task_ids.empty());
ready_task_ids = dependency_manager_.HandleObjectLocal(arguments[1]);
@ -120,7 +102,6 @@ TEST_F(DependencyManagerTest, TestMultipleTasks) {
ObjectID argument_id = ObjectID::FromRandom();
std::vector<TaskID> dependent_tasks;
int num_dependent_tasks = 3;
EXPECT_CALL(reconstruction_policy_mock_, ListenAndMaybeReconstruct(argument_id, _));
for (int i = 0; i < num_dependent_tasks; i++) {
TaskID task_id = RandomTaskId();
dependent_tasks.push_back(task_id);
@ -132,7 +113,6 @@ TEST_F(DependencyManagerTest, TestMultipleTasks) {
}
// Tell the task dependency manager that the object is local.
EXPECT_CALL(reconstruction_policy_mock_, Cancel(argument_id));
auto ready_task_ids = dependency_manager_.HandleObjectLocal(argument_id);
// Check that all tasks are now ready to run.
std::unordered_set<TaskID> added_tasks(dependent_tasks.begin(), dependent_tasks.end());
@ -158,18 +138,12 @@ TEST_F(DependencyManagerTest, TestTaskArgEviction) {
arguments.push_back(ObjectID::FromRandom());
}
TaskID task_id = RandomTaskId();
for (const auto &argument_id : arguments) {
EXPECT_CALL(reconstruction_policy_mock_, ListenAndMaybeReconstruct(argument_id, _));
}
bool ready =
dependency_manager_.RequestTaskDependencies(task_id, ObjectIdsToRefs(arguments));
ASSERT_FALSE(ready);
// Tell the task dependency manager that each of the arguments is now
// available.
for (const auto &argument_id : arguments) {
EXPECT_CALL(reconstruction_policy_mock_, Cancel(argument_id));
}
for (size_t i = 0; i < arguments.size(); i++) {
std::vector<TaskID> ready_tasks;
ready_tasks = dependency_manager_.HandleObjectLocal(arguments[i]);
@ -183,9 +157,6 @@ TEST_F(DependencyManagerTest, TestTaskArgEviction) {
// Simulate each of the arguments getting evicted. Each object should now be
// considered remote.
for (const auto &argument_id : arguments) {
EXPECT_CALL(reconstruction_policy_mock_, ListenAndMaybeReconstruct(argument_id, _));
}
for (size_t i = 0; i < arguments.size(); i++) {
std::vector<TaskID> waiting_tasks;
waiting_tasks = dependency_manager_.HandleObjectMissing(arguments[i]);
@ -203,9 +174,6 @@ TEST_F(DependencyManagerTest, TestTaskArgEviction) {
// Tell the task dependency manager that each of the arguments is available
// again.
for (const auto &argument_id : arguments) {
EXPECT_CALL(reconstruction_policy_mock_, Cancel(argument_id));
}
for (size_t i = 0; i < arguments.size(); i++) {
std::vector<TaskID> ready_tasks;
ready_tasks = dependency_manager_.HandleObjectLocal(arguments[i]);
@ -234,7 +202,6 @@ TEST_F(DependencyManagerTest, TestGet) {
// Subscribe to the task's dependencies. All arguments except the last are
// duplicates of previous subscription calls. Each argument should only be
// requested from the node manager once.
EXPECT_CALL(reconstruction_policy_mock_, ListenAndMaybeReconstruct(argument_id, _));
auto prev_pull_reqs = object_manager_mock_.active_requests;
dependency_manager_.StartOrUpdateGetRequest(worker_id, ObjectIdsToRefs(arguments));
// Previous pull request for this worker should be canceled upon each new
@ -248,11 +215,6 @@ TEST_F(DependencyManagerTest, TestGet) {
dependency_manager_.StartOrUpdateGetRequest(worker_id, ObjectIdsToRefs(arguments));
ASSERT_EQ(object_manager_mock_.active_requests, prev_pull_reqs);
// All arguments should be canceled as they become available locally.
for (const auto &argument_id : arguments) {
EXPECT_CALL(reconstruction_policy_mock_, Cancel(argument_id));
}
// Cancel the pull request once the worker cancels the `ray.get`.
dependency_manager_.CancelGetRequest(worker_id);
AssertNoLeaks();
@ -268,15 +230,11 @@ TEST_F(DependencyManagerTest, TestWait) {
for (int i = 0; i < num_objects; i++) {
oids.push_back(ObjectID::FromRandom());
}
// Simulate a worker calling `ray.wait` on some objects.
EXPECT_CALL(reconstruction_policy_mock_, ListenAndMaybeReconstruct(_, _))
.Times(num_objects);
dependency_manager_.StartOrUpdateWaitRequest(worker_id, ObjectIdsToRefs(oids));
ASSERT_EQ(object_manager_mock_.active_requests.size(), num_objects);
for (int i = 0; i < num_objects; i++) {
// Object is local.
EXPECT_CALL(reconstruction_policy_mock_, Cancel(oids[i]));
auto ready_task_ids = dependency_manager_.HandleObjectLocal(oids[i]);
// Local object gets evicted. The `ray.wait` call should not be
@ -300,8 +258,6 @@ TEST_F(DependencyManagerTest, TestWaitThenCancel) {
oids.push_back(ObjectID::FromRandom());
}
// Simulate a worker calling `ray.wait` on some objects.
EXPECT_CALL(reconstruction_policy_mock_, ListenAndMaybeReconstruct(_, _))
.Times(num_objects);
dependency_manager_.StartOrUpdateWaitRequest(worker_id, ObjectIdsToRefs(oids));
ASSERT_EQ(object_manager_mock_.active_requests.size(), num_objects);
auto prev_pull_reqs = object_manager_mock_.active_requests;
@ -310,7 +266,6 @@ TEST_F(DependencyManagerTest, TestWaitThenCancel) {
dependency_manager_.StartOrUpdateWaitRequest(worker_id, ObjectIdsToRefs(oids));
ASSERT_EQ(object_manager_mock_.active_requests, prev_pull_reqs);
// Cancel the worker's `ray.wait`.
EXPECT_CALL(reconstruction_policy_mock_, Cancel(_)).Times(num_objects);
dependency_manager_.CancelWaitRequest(worker_id);
AssertNoLeaks();
}
@ -331,14 +286,6 @@ TEST_F(DependencyManagerTest, TestWaitObjectLocal) {
const ObjectID local_object_id = std::move(oids.back());
auto ready_task_ids = dependency_manager_.HandleObjectLocal(local_object_id);
ASSERT_TRUE(ready_task_ids.empty());
// Simulate a worker calling `ray.wait` on the objects. It should only make
// requests for the objects that are not local.
for (const auto &object_id : oids) {
if (object_id != local_object_id) {
EXPECT_CALL(reconstruction_policy_mock_, ListenAndMaybeReconstruct(object_id, _));
}
}
dependency_manager_.StartOrUpdateWaitRequest(worker_id, ObjectIdsToRefs(oids));
ASSERT_EQ(object_manager_mock_.active_requests.size(), num_objects - 1);
// Simulate the local object getting evicted. The `ray.wait` call should not
@ -347,11 +294,6 @@ TEST_F(DependencyManagerTest, TestWaitObjectLocal) {
ASSERT_TRUE(waiting_task_ids.empty());
ASSERT_EQ(object_manager_mock_.active_requests.size(), num_objects - 1);
// Cancel the worker's `ray.wait`.
for (const auto &object_id : oids) {
if (object_id != local_object_id) {
EXPECT_CALL(reconstruction_policy_mock_, Cancel(object_id));
}
}
dependency_manager_.CancelWaitRequest(worker_id);
AssertNoLeaks();
}

View file

@ -180,7 +180,6 @@ NodeManager::NodeManager(boost::asio::io_service &io_service, const NodeID &self
fair_queueing_enabled_(config.fair_queueing_enabled),
temp_dir_(config.temp_dir),
initial_config_(config),
local_available_resources_(config.resource_config),
worker_pool_(io_service, self_node_id_, config.node_manager_address,
config.num_workers_soft_limit,
config.num_initial_python_workers_for_first_job,
@ -190,15 +189,7 @@ NodeManager::NodeManager(boost::asio::io_service &io_service, const NodeID &self
/*starting_worker_timeout_callback=*/
[this] { cluster_task_manager_->ScheduleAndDispatchTasks(); },
/*get_time=*/[]() { return absl::GetCurrentTimeNanos() / 1e6; }),
scheduling_policy_(local_queues_),
reconstruction_policy_(
io_service_,
[this](const TaskID &task_id, const ObjectID &required_object_id) {
HandleTaskReconstruction(task_id, required_object_id);
},
RayConfig::instance().object_timeout_milliseconds(), self_node_id_, gcs_client_,
object_directory_),
dependency_manager_(object_manager, reconstruction_policy_),
dependency_manager_(object_manager),
node_manager_server_("NodeManager", config.node_manager_port),
node_manager_service_(io_service, *this),
agent_manager_service_handler_(
@ -234,9 +225,6 @@ NodeManager::NodeManager(boost::asio::io_service &io_service, const NodeID &self
record_metrics_period_ms_(config.record_metrics_period_ms) {
RAY_LOG(INFO) << "Initializing NodeManager with ID " << self_node_id_;
RAY_CHECK(RayConfig::instance().raylet_heartbeat_period_milliseconds() > 0);
// Initialize the resource map with own cluster resource configuration.
cluster_resource_map_.emplace(self_node_id_,
SchedulingResources(config.resource_config));
RAY_CHECK_OK(object_manager_.SubscribeObjAdded(
[this](const object_manager::protocol::ObjectInfoT &object_info) {
@ -246,50 +234,34 @@ NodeManager::NodeManager(boost::asio::io_service &io_service, const NodeID &self
RAY_CHECK_OK(object_manager_.SubscribeObjDeleted(
[this](const ObjectID &object_id) { HandleObjectMissing(object_id); }));
if (RayConfig::instance().new_scheduler_enabled()) {
SchedulingResources &local_resources = cluster_resource_map_[self_node_id_];
cluster_resource_scheduler_ =
std::shared_ptr<ClusterResourceScheduler>(new ClusterResourceScheduler(
self_node_id_.Binary(), local_resources.GetTotalResources().GetResourceMap(),
[this]() { return object_manager_.GetUsedMemory(); }));
SchedulingResources local_resources(config.resource_config);
cluster_resource_scheduler_ =
std::shared_ptr<ClusterResourceScheduler>(new ClusterResourceScheduler(
self_node_id_.Binary(), local_resources.GetTotalResources().GetResourceMap(),
[this]() { return object_manager_.GetUsedMemory(); }));
auto get_node_info_func = [this](const NodeID &node_id) {
return gcs_client_->Nodes().Get(node_id);
};
auto is_owner_alive = [this](const WorkerID &owner_worker_id,
const NodeID &owner_node_id) {
return !(failed_workers_cache_.count(owner_worker_id) > 0 ||
failed_nodes_cache_.count(owner_node_id) > 0);
};
auto announce_infeasible_task = [this](const Task &task) {
PublishInfeasibleTaskError(task);
};
cluster_task_manager_ = std::shared_ptr<ClusterTaskManager>(new ClusterTaskManager(
self_node_id_,
std::dynamic_pointer_cast<ClusterResourceScheduler>(cluster_resource_scheduler_),
dependency_manager_, is_owner_alive, get_node_info_func, announce_infeasible_task,
worker_pool_, leased_workers_,
[this](const std::vector<ObjectID> &object_ids,
std::vector<std::unique_ptr<RayObject>> *results) {
return GetObjectsFromPlasma(object_ids, results);
}));
placement_group_resource_manager_ =
std::make_shared<NewPlacementGroupResourceManager>(
std::dynamic_pointer_cast<ClusterResourceScheduler>(
cluster_resource_scheduler_));
} else {
cluster_resource_scheduler_ = std::make_shared<OldClusterResourceScheduler>(
self_node_id_, local_available_resources_, cluster_resource_map_,
gcs_client_->NodeResources().GetLastResourceUsage());
cluster_task_manager_ = std::shared_ptr<NodeManager>(this, [](NodeManager *) {
// If new scheduler is not enabled, the `cluster_task_manager_` is this NodeManager
// itself. So we create a `shared_ptr` that points to `this` and the deleter does
// nothing.
});
placement_group_resource_manager_ =
std::make_shared<OldPlacementGroupResourceManager>(
local_available_resources_, cluster_resource_map_, self_node_id_);
}
auto get_node_info_func = [this](const NodeID &node_id) {
return gcs_client_->Nodes().Get(node_id);
};
auto is_owner_alive = [this](const WorkerID &owner_worker_id,
const NodeID &owner_node_id) {
return !(failed_workers_cache_.count(owner_worker_id) > 0 ||
failed_nodes_cache_.count(owner_node_id) > 0);
};
auto announce_infeasible_task = [this](const Task &task) {
PublishInfeasibleTaskError(task);
};
cluster_task_manager_ = std::shared_ptr<ClusterTaskManager>(new ClusterTaskManager(
self_node_id_,
std::dynamic_pointer_cast<ClusterResourceScheduler>(cluster_resource_scheduler_),
dependency_manager_, is_owner_alive, get_node_info_func, announce_infeasible_task,
worker_pool_, leased_workers_,
[this](const std::vector<ObjectID> &object_ids,
std::vector<std::unique_ptr<RayObject>> *results) {
return GetObjectsFromPlasma(object_ids, results);
}));
placement_group_resource_manager_ = std::make_shared<NewPlacementGroupResourceManager>(
std::dynamic_pointer_cast<ClusterResourceScheduler>(cluster_resource_scheduler_));
RAY_CHECK_OK(store_client_.Connect(config.store_socket_name.c_str()));
// Run the node manger rpc server.
@ -694,9 +666,7 @@ void NodeManager::NodeAdded(const GcsNodeInfo &node_info) {
const NodeID node_id = NodeID::FromBinary(node_info.node_id());
RAY_LOG(DEBUG) << "[NodeAdded] Received callback from node id " << node_id;
if (1 == cluster_resource_map_.count(node_id)) {
RAY_LOG(DEBUG) << "Received notification of a new node that already exists: "
<< node_id;
if (node_id == self_node_id_) {
return;
}
@ -845,27 +815,6 @@ void NodeManager::ResourceDeleted(const NodeID &node_id,
return;
}
void NodeManager::TryLocalInfeasibleTaskScheduling() {
RAY_LOG(DEBUG) << "[LocalResourceUpdateRescheduler] The resource update is on the "
"local node, check if we can reschedule tasks";
SchedulingResources &new_local_resources = cluster_resource_map_[self_node_id_];
// SpillOver locally to figure out which infeasible tasks can be placed now
std::vector<TaskID> decision =
scheduling_policy_.SpillOverInfeasibleTasks(new_local_resources);
std::unordered_set<TaskID> local_task_ids(decision.begin(), decision.end());
// Transition locally placed tasks to waiting or ready for dispatch.
if (local_task_ids.size() > 0) {
std::vector<Task> tasks = local_queues_.RemoveTasks(local_task_ids);
for (const auto &t : tasks) {
EnqueuePlaceableTask(t);
}
}
}
void NodeManager::UpdateResourceUsage(const NodeID &node_id,
const rpc::ResourcesData &resource_data) {
if (!cluster_resource_scheduler_->UpdateNode(node_id.Binary(), resource_data)) {
@ -882,7 +831,7 @@ void NodeManager::UpdateResourceUsage(const NodeID &node_id,
// If light resource usage report enabled, we update remote resources only when related
// resources map in heartbeat is not empty.
cluster_task_manager_->OnNodeResourceUsageUpdated(node_id, resource_data);
cluster_task_manager_->ScheduleAndDispatchTasks();
}
void NodeManager::ResourceUsageBatchReceived(
@ -903,69 +852,6 @@ void NodeManager::ProcessNewClient(ClientConnection &client) {
client.ProcessMessages();
}
// A helper function to create a mapping from task scheduling class to
// tasks with that class from a given list of tasks.
std::unordered_map<SchedulingClass, ordered_set<TaskID>> NodeManager::MakeTasksByClass(
const std::vector<Task> &tasks) const {
std::unordered_map<SchedulingClass, ordered_set<TaskID>> result;
for (const auto &task : tasks) {
auto spec = task.GetTaskSpecification();
result[spec.GetSchedulingClass()].push_back(spec.TaskId());
}
return result;
}
void NodeManager::DispatchTasks(
const std::unordered_map<SchedulingClass, ordered_set<TaskID>> &tasks_by_class) {
// Dispatch tasks in priority order by class. This avoids starvation problems where
// one class of tasks become stuck behind others in the queue, causing Ray to start
// many workers. See #3644 for a more detailed description of this issue.
std::vector<const std::pair<const SchedulingClass, ordered_set<TaskID>> *> fair_order;
for (auto &it : tasks_by_class) {
fair_order.emplace_back(&it);
}
// Prioritize classes that have fewer currently running tasks. Note that we only
// sort once per round of task dispatch, which is less fair then it could be, but
// is simpler and faster.
if (fair_queueing_enabled_) {
std::sort(
std::begin(fair_order), std::end(fair_order),
[this](const std::pair<const SchedulingClass, ordered_set<ray::TaskID>> *a,
const std::pair<const SchedulingClass, ordered_set<ray::TaskID>> *b) {
return local_queues_.NumRunning(a->first) < local_queues_.NumRunning(b->first);
});
}
std::vector<std::function<void()>> post_assign_callbacks;
// Approximate fair round robin between classes.
for (const auto &it : fair_order) {
const auto &task_resources =
TaskSpecification::GetSchedulingClassDescriptor(it->first);
// FIFO order within each class.
for (const auto &task_id : it->second) {
const auto &task = local_queues_.GetTaskOfState(task_id, TaskState::READY);
if (!local_available_resources_.Contains(task_resources)) {
// All the tasks in it.second have the same resource shape, so
// once the first task is not feasible, we can break out of this loop
break;
}
// Try to get an idle worker to execute this task. If nullptr, there
// aren't any available workers so we can't assign the task.
std::shared_ptr<WorkerInterface> worker =
worker_pool_.PopWorker(task.GetTaskSpecification());
if (worker != nullptr) {
AssignTask(worker, task, &post_assign_callbacks);
}
}
}
// Call the callbacks from the AssignTask calls above. These need to be called
// after the above loop, as they may alter the scheduling queues and invalidate
// the loop iterator.
for (auto &func : post_assign_callbacks) {
func();
}
}
void NodeManager::ProcessClientMessage(const std::shared_ptr<ClientConnection> &client,
int64_t message_type,
const uint8_t *message_data) {
@ -1004,15 +890,6 @@ void NodeManager::ProcessClientMessage(const std::shared_ptr<ClientConnection> &
// because it's already disconnected.
return;
} break;
case protocol::MessageType::SubmitTask: {
// For tasks submitted via the raylet path, we must make sure to order the
// task submission so that tasks are always submitted after the tasks that
// they depend on.
ProcessSubmitTaskMessage(message_data);
} break;
case protocol::MessageType::SetResourceRequest: {
ProcessSetResourceRequest(client, message_data);
} break;
case protocol::MessageType::FetchOrReconstruct: {
ProcessFetchOrReconstructMessage(client, message_data);
} break;
@ -1024,6 +901,8 @@ void NodeManager::ProcessClientMessage(const std::shared_ptr<ClientConnection> &
HandleDirectCallTaskUnblocked(worker);
} break;
case protocol::MessageType::NotifyUnblocked: {
// TODO(ekl) this is still used from core worker even in direct call mode to
// finish up get requests.
auto message = flatbuffers::GetRoot<protocol::NotifyUnblocked>(message_data);
AsyncResolveObjectsFinish(client, from_flatbuf<TaskID>(*message->task_id()),
/*was_blocked*/ true);
@ -1128,7 +1007,6 @@ void NodeManager::ProcessRegisterClientRequestMessage(
job_config.ParseFromString(message->serialized_job_config()->str());
Status status = worker_pool_.RegisterDriver(worker, job_config, send_reply_callback);
if (status.ok()) {
local_queues_.AddDriverTaskId(driver_task_id);
auto job_data_ptr =
gcs::CreateJobTableData(job_id, /*is_dead*/ false, std::time(nullptr),
worker_ip_address, pid, job_config);
@ -1189,10 +1067,6 @@ void NodeManager::HandleWorkerAvailable(const std::shared_ptr<WorkerInterface> &
worker_pool_.PushWorker(worker);
}
// Local resource availability changed: invoke scheduling policy for local node.
cluster_resource_map_[self_node_id_].SetLoadResources(
local_queues_.GetTotalResourceLoad());
cluster_task_manager_->ScheduleAndDispatchTasks();
}
@ -1224,14 +1098,8 @@ void NodeManager::DisconnectClient(const std::shared_ptr<ClientConnection> &clie
// Because in this case, its task is already cleaned up.
RAY_LOG(DEBUG) << "Skip unblocking worker because it's already dead.";
} else {
// Clean up any open ray.get calls that the worker made.
while (!worker->GetBlockedTaskIds().empty()) {
// NOTE(swang): AsyncResolveObjectsFinish will modify the worker, so it is
// not safe to pass in the iterator directly.
const TaskID task_id = *worker->GetBlockedTaskIds().begin();
AsyncResolveObjectsFinish(client, task_id, true);
}
// Clean up any open ray.wait calls that the worker made.
dependency_manager_.CancelGetRequest(worker->WorkerId());
dependency_manager_.CancelWaitRequest(worker->WorkerId());
}
@ -1285,8 +1153,6 @@ void NodeManager::DisconnectClient(const std::shared_ptr<ClientConnection> &clie
const auto job_id = worker->GetAssignedJobId();
RAY_CHECK(!job_id.IsNil());
RAY_CHECK_OK(gcs_client_->Jobs().AsyncMarkFinished(job_id, nullptr));
const auto driver_id = ComputeDriverIdFromJob(job_id);
local_queues_.RemoveDriverTaskId(TaskID::ComputeDriverTaskId(driver_id));
worker_pool_.DisconnectDriver(worker);
RAY_LOG(INFO) << "Driver (pid=" << worker->GetProcess().GetId()
@ -1449,18 +1315,6 @@ void NodeManager::ProcessPushErrorRequestMessage(const uint8_t *message_data) {
RAY_CHECK_OK(gcs_client_->Errors().AsyncReportJobError(error_data_ptr, nullptr));
}
void NodeManager::ProcessSubmitTaskMessage(const uint8_t *message_data) {
// Read the task submitted by the client.
auto fbs_message = flatbuffers::GetRoot<protocol::SubmitTaskRequest>(message_data);
rpc::Task task_message;
RAY_CHECK(task_message.mutable_task_spec()->ParseFromArray(
fbs_message->task_spec()->data(), fbs_message->task_spec()->size()));
// Submit the task to the raylet. Since the task was submitted
// locally, there is no uncommitted lineage.
SubmitTask(Task(task_message));
}
void NodeManager::HandleRequestWorkerLease(const rpc::RequestWorkerLeaseRequest &request,
rpc::RequestWorkerLeaseReply *reply,
rpc::SendReplyCallback send_reply_callback) {
@ -1621,114 +1475,6 @@ void NodeManager::HandleCancelWorkerLease(const rpc::CancelWorkerLeaseRequest &r
send_reply_callback(Status::OK(), nullptr, nullptr);
}
void NodeManager::ProcessSetResourceRequest(
const std::shared_ptr<ClientConnection> &client, const uint8_t *message_data) {
// Read the SetResource message
auto message = flatbuffers::GetRoot<protocol::SetResourceRequest>(message_data);
auto const &resource_name = string_from_flatbuf(*message->resource_name());
double const &capacity = message->capacity();
bool is_deletion = capacity <= 0;
NodeID node_id = from_flatbuf<NodeID>(*message->node_id());
// If the python arg was null, set node_id to the local node id.
if (node_id.IsNil()) {
node_id = self_node_id_;
}
if (is_deletion &&
cluster_resource_map_[node_id].GetTotalResources().GetResourceMap().count(
resource_name) == 0) {
// Resource does not exist in the cluster resource map, thus nothing to delete.
// Return..
RAY_LOG(INFO) << "[ProcessDeleteResourceRequest] Trying to delete resource "
<< resource_name << ", but it does not exist. Doing nothing..";
return;
}
// Submit to the resource table. This calls the ResourceCreateUpdated or ResourceDeleted
// callback, which updates cluster_resource_map_.
if (is_deletion) {
RAY_CHECK_OK(gcs_client_->NodeResources().AsyncDeleteResources(
node_id, {resource_name}, nullptr));
} else {
std::unordered_map<std::string, std::shared_ptr<rpc::ResourceTableData>> data_map;
auto resource_table_data = std::make_shared<rpc::ResourceTableData>();
resource_table_data->set_resource_capacity(capacity);
data_map.emplace(resource_name, resource_table_data);
RAY_CHECK_OK(
gcs_client_->NodeResources().AsyncUpdateResources(node_id, data_map, nullptr));
}
}
void NodeManager::ScheduleTasks(
std::unordered_map<NodeID, SchedulingResources> &resource_map) {
// If the resource map contains the local raylet, update load before calling policy.
if (resource_map.count(self_node_id_) > 0) {
resource_map[self_node_id_].SetLoadResources(local_queues_.GetTotalResourceLoad());
}
// Invoke the scheduling policy.
auto policy_decision = scheduling_policy_.Schedule(resource_map, self_node_id_);
#ifndef NDEBUG
RAY_LOG(DEBUG) << "[NM ScheduleTasks] policy decision:";
for (const auto &task_client_pair : policy_decision) {
TaskID task_id = task_client_pair.first;
NodeID node_id = task_client_pair.second;
RAY_LOG(DEBUG) << task_id << " --> " << node_id;
}
#endif
// Extract decision for this raylet.
std::unordered_set<TaskID> local_task_ids;
// Iterate over (taskid, nodeid) pairs, extract tasks assigned to the local node.
for (const auto &task_client_pair : policy_decision) {
const TaskID &task_id = task_client_pair.first;
const NodeID &node_id = task_client_pair.second;
if (node_id == self_node_id_) {
local_task_ids.insert(task_id);
} else {
// TODO(atumanov): need a better interface for task exit on forward.
// (See design_docs/task_states.rst for the state transition diagram.)
Task task;
if (local_queues_.RemoveTask(task_id, &task)) {
// Attempt to forward the task. If this fails to forward the task,
// the task will be resubmit locally.
ForwardTaskOrResubmit(task, node_id);
}
}
}
// Transition locally placed tasks to waiting or ready for dispatch.
if (local_task_ids.size() > 0) {
std::vector<Task> tasks = local_queues_.RemoveTasks(local_task_ids);
for (const auto &t : tasks) {
EnqueuePlaceableTask(t);
}
}
// All remaining placeable tasks should be registered with the task dependency
// manager. TaskDependencyManager::TaskPending() is assumed to be idempotent.
// TODO(atumanov): evaluate performance implications of registering all new tasks on
// submission vs. registering remaining queued placeable tasks here.
std::unordered_set<TaskID> move_task_set;
for (const auto &task : local_queues_.GetTasks(TaskState::PLACEABLE)) {
move_task_set.insert(task.GetTaskSpecification().TaskId());
PublishInfeasibleTaskError(task);
// Assert that this placeable task is not feasible locally (necessary but not
// sufficient).
RAY_CHECK(!task.GetTaskSpecification().GetRequiredPlacementResources().IsSubset(
cluster_resource_map_[self_node_id_].GetTotalResources()));
}
// Assumption: all remaining placeable tasks are infeasible and are moved to the
// infeasible task queue. Infeasible task queue is checked when new nodes join.
local_queues_.MoveTasks(move_task_set, TaskState::PLACEABLE, TaskState::INFEASIBLE);
// Check the invariant that no placeable tasks remain after a call to the policy.
RAY_CHECK(local_queues_.GetTasks(TaskState::PLACEABLE).size() == 0);
}
void NodeManager::MarkObjectsAsFailed(
const ErrorType &error_type, const std::vector<rpc::ObjectReference> objects_to_fail,
const JobID &job_id) {
@ -1760,39 +1506,6 @@ void NodeManager::MarkObjectsAsFailed(
}
}
void NodeManager::SubmitTask(const Task &task) {
const TaskSpecification &spec = task.GetTaskSpecification();
// Actor tasks should be no longer submitted to raylet.
RAY_CHECK(!spec.IsActorTask());
const TaskID &task_id = spec.TaskId();
RAY_LOG(DEBUG) << "Submitting task: " << task.DebugString();
if (local_queues_.HasTask(task_id)) {
if (spec.IsActorCreationTask()) {
// NOTE(hchen): Normally when raylet receives a duplicated actor creation task
// from GCS, raylet should just ignore the task. However, due to the hack that
// we save the RPC reply in task's OnDispatch callback, we have to remove the
// old task and re-add the new task, to make sure the RPC reply callback is correct.
RAY_LOG(WARNING) << "Submitted actor creation task " << task_id
<< " is already queued. This is most likely due to a GCS restart. "
"We will remove "
"the old one from the queue, and enqueue the new one.";
std::unordered_set<TaskID> task_ids{task_id};
local_queues_.RemoveTasks(task_ids);
} else {
RAY_LOG(WARNING) << "Submitted task " << task_id
<< " is already queued and will not be restarted. This is most "
"likely due to spurious reconstruction.";
return;
}
}
// (See design_docs/task_states.rst for the state transition diagram.)
local_queues_.QueueTasks({task}, TaskState::PLACEABLE);
ScheduleTasks(cluster_resource_map_);
// TODO(atumanov): assert that !placeable.isempty() => insufficient available
// resources locally.
}
void NodeManager::HandleDirectCallTaskBlocked(
const std::shared_ptr<WorkerInterface> &worker, bool release_resources) {
if (!worker || worker->IsBlocked() || worker->GetAssignedTaskId().IsNil() ||
@ -1829,40 +1542,13 @@ void NodeManager::AsyncResolveObjects(
const std::vector<rpc::ObjectReference> &required_object_refs,
const TaskID &current_task_id, bool ray_get, bool mark_worker_blocked) {
std::shared_ptr<WorkerInterface> worker = worker_pool_.GetRegisteredWorker(client);
if (worker) {
// The client is a worker. If the worker is not already blocked and the
// blocked task matches the one assigned to the worker, then mark the
// worker as blocked. This temporarily releases any resources that the
// worker holds while it is blocked.
if (mark_worker_blocked && !worker->IsBlocked() &&
current_task_id == worker->GetAssignedTaskId()) {
Task task;
RAY_CHECK(local_queues_.RemoveTask(current_task_id, &task));
local_queues_.QueueTasks({task}, TaskState::RUNNING);
// Get the CPU resources required by the running task.
// Release the CPU resources.
auto const cpu_resource_ids = worker->ReleaseTaskCpuResources();
local_available_resources_.Release(cpu_resource_ids);
cluster_resource_map_[self_node_id_].Release(cpu_resource_ids.ToResourceSet());
worker->MarkBlocked();
// Try dispatching tasks since we may have released some resources.
DispatchTasks(local_queues_.GetReadyTasksByClass());
}
} else {
if (!worker) {
// The client is a driver. Drivers do not hold resources, so we simply mark
// the task as blocked.
worker = worker_pool_.GetRegisteredDriver(client);
}
RAY_CHECK(worker);
// Mark the task as blocked.
if (mark_worker_blocked) {
worker->AddBlockedTaskId(current_task_id);
if (local_queues_.GetBlockedTaskIds().count(current_task_id) == 0) {
local_queues_.AddBlockedTaskId(current_task_id);
}
}
// Subscribe to the objects required by the task. These objects will be
// fetched and/or restarted as necessary, until the objects become local
// or are unsubscribed.
@ -1878,45 +1564,7 @@ void NodeManager::AsyncResolveObjectsFinish(
const std::shared_ptr<ClientConnection> &client, const TaskID &current_task_id,
bool was_blocked) {
std::shared_ptr<WorkerInterface> worker = worker_pool_.GetRegisteredWorker(client);
// TODO(swang): Because the object dependencies are tracked in the task
// dependency manager, we could actually remove this message entirely and
// instead unblock the worker once all the objects become available.
if (worker) {
// The client is a worker. If the worker is not already unblocked and the
// unblocked task matches the one assigned to the worker, then mark the
// worker as unblocked. This returns the temporarily released resources to
// the worker. Workers that have been marked dead have already been cleaned
// up.
if (was_blocked && worker->IsBlocked() &&
current_task_id == worker->GetAssignedTaskId() && !worker->IsDead()) {
// (See design_docs/task_states.rst for the state transition diagram.)
Task task;
RAY_CHECK(local_queues_.RemoveTask(current_task_id, &task));
local_queues_.QueueTasks({task}, TaskState::RUNNING);
// Get the CPU resources required by the running task.
const auto required_resources = task.GetTaskSpecification().GetRequiredResources();
const ResourceSet cpu_resources = required_resources.GetNumCpus();
// Check if we can reacquire the CPU resources.
bool oversubscribed = !local_available_resources_.Contains(cpu_resources);
if (!oversubscribed) {
// Reacquire the CPU resources for the worker. Note that care needs to be
// taken if the user is using the specific CPU IDs since the IDs that we
// reacquire here may be different from the ones that the task started with.
auto const resource_ids = local_available_resources_.Acquire(cpu_resources);
worker->AcquireTaskCpuResources(resource_ids);
cluster_resource_map_[self_node_id_].Acquire(cpu_resources);
} else {
// In this case, we simply don't reacquire the CPU resources for the worker.
// The worker can keep running and when the task finishes, it will simply
// not have any CPU resources to release.
RAY_LOG(WARNING)
<< "Resources oversubscribed: "
<< cluster_resource_map_[self_node_id_].GetAvailableResources().ToString();
}
worker->MarkUnblocked();
}
} else {
if (!worker) {
// The client is a driver. Drivers do not hold resources, so we simply
// mark the driver as unblocked.
worker = worker_pool_.GetRegisteredDriver(client);
@ -1931,88 +1579,9 @@ void NodeManager::AsyncResolveObjectsFinish(
// Mark the task as unblocked.
if (was_blocked) {
worker->RemoveBlockedTaskId(current_task_id);
local_queues_.RemoveBlockedTaskId(current_task_id);
}
}
void NodeManager::EnqueuePlaceableTask(const Task &task) {
// TODO(atumanov): add task lookup hashmap and change EnqueuePlaceableTask to take
// a vector of TaskIDs. Trigger MoveTask internally.
// Subscribe to the task's dependencies.
bool args_ready = dependency_manager_.RequestTaskDependencies(
task.GetTaskSpecification().TaskId(), task.GetDependencies());
// Enqueue the task. If all dependencies are available, then the task is queued
// in the READY state, else the WAITING state.
// (See design_docs/task_states.rst for the state transition diagram.)
if (args_ready) {
local_queues_.QueueTasks({task}, TaskState::READY);
DispatchTasks(MakeTasksByClass({task}));
} else {
local_queues_.QueueTasks({task}, TaskState::WAITING);
}
}
void NodeManager::AssignTask(const std::shared_ptr<WorkerInterface> &worker,
const Task &task,
std::vector<std::function<void()>> *post_assign_callbacks) {
// TODO(sang): Modify method names.
const TaskSpecification &spec = task.GetTaskSpecification();
RAY_CHECK(post_assign_callbacks);
RAY_LOG(DEBUG) << "Assigning task " << spec.TaskId() << " to worker with pid "
<< worker->GetProcess().GetId() << ", worker id: " << worker->WorkerId();
flatbuffers::FlatBufferBuilder fbb;
// Resource accounting: acquire resources for the assigned task.
auto acquired_resources =
local_available_resources_.Acquire(spec.GetRequiredResources());
cluster_resource_map_[self_node_id_].Acquire(spec.GetRequiredResources());
if (spec.IsActorCreationTask()) {
// Check that the actor's placement resource requirements are satisfied.
RAY_CHECK(spec.GetRequiredPlacementResources().IsSubset(
cluster_resource_map_[self_node_id_].GetTotalResources()));
worker->SetLifetimeResourceIds(acquired_resources);
} else {
worker->SetTaskResourceIds(acquired_resources);
}
auto task_id = spec.TaskId();
RAY_CHECK(task.OnDispatch() != nullptr);
if (task.GetTaskSpecification().IsDetachedActor()) {
worker->MarkDetachedActor();
}
worker->SetBundleId(spec.PlacementGroupBundleId());
const auto owner_worker_id = WorkerID::FromBinary(spec.CallerAddress().worker_id());
const auto owner_node_id = NodeID::FromBinary(spec.CallerAddress().raylet_id());
RAY_CHECK(!owner_worker_id.IsNil());
RAY_LOG(DEBUG) << "Worker lease request DISPATCH " << task_id << " to worker "
<< worker->WorkerId() << ", owner ID " << owner_worker_id;
task.OnDispatch()(worker, initial_config_.node_manager_address, worker->Port(),
worker->WorkerId(),
spec.IsActorCreationTask() ? worker->GetLifetimeResourceIds()
: worker->GetTaskResourceIds());
// If the owner has died since this task was queued, cancel the task by
// killing the worker (unless this task is for a detached actor).
if (!worker->IsDetachedActor() && (failed_workers_cache_.count(owner_worker_id) > 0 ||
failed_nodes_cache_.count(owner_node_id) > 0)) {
// TODO(swang): Skip assigning this task to this worker instead of
// killing the worker?
RAY_LOG(INFO) << "Owner of assigned task " << task.GetTaskSpecification().TaskId()
<< " died, killing leased worker " << worker->WorkerId();
KillWorker(worker);
}
post_assign_callbacks->push_back([this, worker, task_id]() {
RAY_LOG(DEBUG) << "Finished assigning task " << task_id << " to worker "
<< worker->WorkerId();
FinishAssignTask(worker, task_id, /*success=*/true);
});
}
bool NodeManager::FinishAssignedTask(const std::shared_ptr<WorkerInterface> &worker_ptr) {
// TODO (Alex): We should standardize to pass
// std::shared_ptr<WorkerInterface> instead of refs.
@ -2063,63 +1632,6 @@ void NodeManager::FinishAssignedActorCreationTask(WorkerInterface &worker,
}
}
void NodeManager::HandleTaskReconstruction(const TaskID &task_id,
const ObjectID &required_object_id) {
// Get the owner's address.
rpc::Address owner_addr;
bool has_owner = dependency_manager_.GetOwnerAddress(required_object_id, &owner_addr);
if (has_owner) {
RAY_LOG(DEBUG) << "Required object " << required_object_id
<< " fetch timed out, asking owner "
<< WorkerID::FromBinary(owner_addr.worker_id());
// The owner's address exists. Poll the owner to check if the object is
// still in scope. If not, mark the object as failed.
// TODO(swang): If the owner has died, we could also mark the object as
// failed as soon as we hear about the owner's failure from the GCS,
// avoiding the raylet's reconstruction timeout.
auto client = std::unique_ptr<rpc::CoreWorkerClient>(
new rpc::CoreWorkerClient(owner_addr, client_call_manager_));
rpc::GetObjectStatusRequest request;
request.set_object_id(required_object_id.Binary());
request.set_owner_worker_id(owner_addr.worker_id());
client->GetObjectStatus(
request, [this, required_object_id, owner_addr](
Status status, const rpc::GetObjectStatusReply &reply) {
if (!status.ok() || reply.status() == rpc::GetObjectStatusReply::OUT_OF_SCOPE ||
reply.status() == rpc::GetObjectStatusReply::FREED) {
// The owner is gone, or the owner replied that the object has
// gone out of scope (this is an edge case in the distributed ref
// counting protocol where a borrower dies before it can notify
// the owner of another borrower), or the object value has been
// freed. Store an error in the local plasma store so that an
// exception will be thrown when the worker tries to get the
// value.
rpc::ObjectReference ref;
ref.set_object_id(required_object_id.Binary());
ref.mutable_owner_address()->CopyFrom(owner_addr);
MarkObjectsAsFailed(ErrorType::OBJECT_UNRECONSTRUCTABLE, {ref}, JobID::Nil());
}
// Do nothing if the owner replied that the object is available. The
// object manager will continue trying to fetch the object, and this
// handler will get triggered again if the object is still
// unavailable after another timeout.
});
} else {
RAY_LOG(WARNING)
<< "Ray cannot get the value of ObjectIDs that are generated "
"randomly (ObjectID.from_random()) or out-of-band "
"(ObjectID.from_binary(...)) because Ray "
"does not know which task will create them. "
"If this was not how your object ID was generated, please file an "
"issue "
"at https://github.com/ray-project/ray/issues/";
rpc::ObjectReference ref;
ref.set_object_id(required_object_id.Binary());
MarkObjectsAsFailed(ErrorType::OBJECT_UNRECONSTRUCTABLE, {ref}, JobID::Nil());
}
}
void NodeManager::HandleObjectLocal(const ObjectID &object_id) {
// Notify the task dependency manager that this object is local.
const auto ready_task_ids = dependency_manager_.HandleObjectLocal(object_id);
@ -2153,81 +1665,6 @@ void NodeManager::HandleObjectMissing(const ObjectID &object_id) {
}
}
RAY_LOG(DEBUG) << result.str();
cluster_task_manager_->OnObjectMissing(object_id, waiting_task_ids);
}
void NodeManager::ForwardTaskOrResubmit(const Task &task, const NodeID &node_manager_id) {
// Attempt to forward the task.
// TODO(sang): Modify method names.
ForwardTask(task, node_manager_id,
[this, node_manager_id](ray::Status error, const Task &task) {
const TaskID task_id = task.GetTaskSpecification().TaskId();
RAY_LOG(INFO) << "Failed to forward task " << task_id
<< " to node manager " << node_manager_id;
// The task is not for an actor and may therefore be placed on another
// node immediately. Send it to the scheduling policy to be placed again.
local_queues_.QueueTasks({task}, TaskState::PLACEABLE);
ScheduleTasks(cluster_resource_map_);
});
}
void NodeManager::ForwardTask(
const Task &task, const NodeID &node_id,
const std::function<void(const ray::Status &, const Task &)> &on_error) {
// This method spillbacks lease requests to other nodes.
// TODO(sang): Modify method names.
RAY_CHECK(task.OnSpillback() != nullptr);
auto node_info = gcs_client_->Nodes().Get(node_id);
RAY_CHECK(node_info)
<< "Spilling back to a node manager, but no GCS info found for node " << node_id;
task.OnSpillback()(node_id, node_info->node_manager_address(),
node_info->node_manager_port());
}
void NodeManager::FinishAssignTask(const std::shared_ptr<WorkerInterface> &worker,
const TaskID &task_id, bool success) {
// TODO(sang): Modify method names.
RAY_LOG(DEBUG) << "FinishAssignTask: " << task_id;
// Remove the ASSIGNED task from the READY queue.
Task assigned_task;
TaskState state;
if (!local_queues_.RemoveTask(task_id, &assigned_task, &state)) {
// TODO(edoakes): should we be failing silently here?
return;
}
RAY_CHECK(state == TaskState::READY);
if (success) {
auto spec = assigned_task.GetTaskSpecification();
// We successfully assigned the task to the worker.
worker->AssignTaskId(spec.TaskId());
worker->SetOwnerAddress(spec.CallerAddress());
RAY_CHECK(worker->GetAssignedJobId() == spec.JobId());
// TODO(swang): For actors with multiple actor handles, to
// guarantee that tasks are replayed in the same order after a
// failure, we must update the task's execution dependency to be
// the actor's current execution dependency.
// Mark the task as running.
// (See design_docs/task_states.rst for the state transition diagram.)
assigned_task.OnDispatchInstead(nullptr);
assigned_task.OnSpillbackInstead(nullptr);
local_queues_.QueueTasks({assigned_task}, TaskState::RUNNING);
// Notify the task dependency manager that we no longer need this task's
// object dependencies.
dependency_manager_.RemoveTaskDependencies(spec.TaskId());
} else {
RAY_LOG(WARNING) << "Failed to send task to worker, disconnecting client";
// We failed to send the task to the worker, so disconnect the worker.
DisconnectClient(worker->Connection());
// Queue this task for future assignment. We need to do this since
// DispatchTasks() removed it from the ready queue. The task will be
// assigned to a worker once one becomes available.
// (See design_docs/task_states.rst for the state transition diagram.)
local_queues_.QueueTasks({assigned_task}, TaskState::READY);
DispatchTasks(MakeTasksByClass({assigned_task}));
}
}
void NodeManager::ProcessSubscribePlasmaReady(
@ -2328,15 +1765,10 @@ std::string NodeManager::DebugString() const {
result << cluster_task_manager_->DebugStr();
}
result << "\nClusterResources:";
for (auto &pair : cluster_resource_map_) {
result << "\n" << pair.first.Hex() << ": " << pair.second.DebugString();
}
result << "\n" << local_object_manager_.DebugString();
result << "\n" << object_manager_.DebugString();
result << "\n" << gcs_client_->DebugString();
result << "\n" << worker_pool_.DebugString();
result << "\n" << local_queues_.DebugString();
result << "\n" << reconstruction_policy_.DebugString();
result << "\n" << dependency_manager_.DebugString();
{
absl::MutexLock guard(&plasma_object_notification_lock_);
@ -2425,21 +1857,6 @@ void NodeManager::HandleGetNodeStats(const rpc::GetNodeStatsRequest &node_stats_
rpc::GetNodeStatsReply *reply,
rpc::SendReplyCallback send_reply_callback) {
cluster_task_manager_->FillPendingActorInfo(reply);
for (const auto &task : local_queues_.GetTasks(TaskState::INFEASIBLE)) {
if (task.GetTaskSpecification().IsActorCreationTask()) {
auto infeasible_task = reply->add_infeasible_tasks();
infeasible_task->ParseFromString(task.GetTaskSpecification().Serialize());
}
}
// Report tasks that are not scheduled because
// resources are occupied by other actors/tasks.
for (const auto &task : local_queues_.GetTasks(TaskState::READY)) {
if (task.GetTaskSpecification().IsActorCreationTask()) {
auto ready_task = reply->add_ready_tasks();
ready_task->ParseFromString(task.GetTaskSpecification().Serialize());
}
}
// Report object spilling stats.
local_object_manager_.FillObjectSpillingStats(reply);
// Report object store stats.
@ -2703,21 +2120,6 @@ void NodeManager::RecordMetrics() {
uint64_t current_time = current_time_ms();
uint64_t duration_ms = current_time - metrics_last_recorded_time_ms_;
// Record available resources of this node.
const auto &available_resources =
cluster_resource_map_.at(self_node_id_).GetAvailableResources().GetResourceMap();
for (const auto &pair : available_resources) {
stats::LocalAvailableResource().Record(pair.second,
{{stats::ResourceNameKey, pair.first}});
}
// Record total resources of this node.
const auto &total_resources =
cluster_resource_map_.at(self_node_id_).GetTotalResources().GetResourceMap();
for (const auto &pair : total_resources) {
stats::LocalTotalResource().Record(pair.second,
{{stats::ResourceNameKey, pair.first}});
}
// Record average number of tasks information per second.
stats::AvgNumScheduledTasks.Record((double)metrics_num_task_scheduled_ *
(1000.0 / (double)duration_ms));
@ -2730,7 +2132,6 @@ void NodeManager::RecordMetrics() {
metrics_num_task_spilled_back_ = 0;
object_manager_.RecordMetrics();
local_queues_.RecordMetrics();
local_object_manager_.RecordObjectSpillingStats();
}

View file

@ -31,11 +31,7 @@
#include "ray/raylet/scheduling/scheduling_ids.h"
#include "ray/raylet/scheduling/cluster_resource_scheduler.h"
#include "ray/raylet/scheduling/cluster_task_manager.h"
#include "ray/raylet/scheduling/old_cluster_resource_scheduler.h"
#include "ray/raylet/scheduling/cluster_task_manager_interface.h"
#include "ray/raylet/scheduling_policy.h"
#include "ray/raylet/scheduling_queue.h"
#include "ray/raylet/reconstruction_policy.h"
#include "ray/raylet/dependency_manager.h"
#include "ray/raylet/worker_pool.h"
#include "ray/rpc/worker/core_worker_client_pool.h"
@ -134,8 +130,7 @@ class HeartbeatSender {
uint64_t last_heartbeat_at_ms_;
};
class NodeManager : public rpc::NodeManagerServiceHandler,
public ClusterTaskManagerInterface {
class NodeManager : public rpc::NodeManagerServiceHandler {
public:
/// Create a node manager.
///
@ -189,6 +184,15 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
/// object ids.
void TriggerGlobalGC();
/// Mark the specified objects as failed with the given error type.
///
/// \param error_type The type of the error that caused this task to fail.
/// \param object_ids The object ids to store error messages into.
/// \param job_id The optional job to push errors to if the writes fail.
void MarkObjectsAsFailed(const ErrorType &error_type,
const std::vector<rpc::ObjectReference> object_ids,
const JobID &job_id);
/// Stop this node manager.
void Stop();
@ -225,6 +229,9 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
void ResourceDeleted(const NodeID &node_id,
const std::vector<std::string> &resource_names);
/// Send heartbeats to the GCS.
void Heartbeat();
/// Evaluates the local infeasible queue to check if any tasks can be scheduled.
/// This is called whenever there's an update to the resources on the local node.
/// \return Void.
@ -257,35 +264,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
/// \param resource_usage_batch The batch of resource usage data.
void ResourceUsageBatchReceived(const ResourceUsageBatchData &resource_usage_batch);
/// Methods for task scheduling.
/// Enqueue a placeable task to wait on object dependencies or be ready for
/// dispatch.
///
/// \param task The task in question.
/// \return Void.
void EnqueuePlaceableTask(const Task &task);
/// Mark the specified objects as failed with the given error type.
///
/// \param error_type The type of the error that caused this task to fail.
/// \param object_ids The object ids to store error messages into.
/// \param job_id The optional job to push errors to if the writes fail.
void MarkObjectsAsFailed(const ErrorType &error_type,
const std::vector<rpc::ObjectReference> object_ids,
const JobID &job_id);
/// Handle specified task's submission to the local node manager.
///
/// \param task The task being submitted.
/// \return Void.
void SubmitTask(const Task &task);
/// Assign a task to a worker. The task is assumed to not be queued in local_queues_.
///
/// \param[in] worker The worker to assign the task to.
/// \param[in] task The task in question.
/// \param[out] post_assign_callbacks Vector of callbacks that will be appended
/// to with any logic that should run after the DispatchTasks loop runs.
void AssignTask(const std::shared_ptr<WorkerInterface> &worker, const Task &task,
std::vector<std::function<void()>> *post_assign_callbacks);
/// Handle a worker finishing its assigned task.
///
/// \param worker The worker that finished the task.
@ -306,58 +284,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
/// \param task The actor task or actor creation task.
/// \return Void.
void FinishAssignedActorCreationTask(WorkerInterface &worker, const Task &task);
/// Make a placement decision for placeable tasks given the resource_map
/// provided. This will perform task state transitions and task forwarding.
///
/// \param resource_map A mapping from node manager ID to an estimate of the
/// resources available to that node manager. Scheduling decisions will only
/// consider the local node manager and the node managers in the keys of the
/// resource_map argument.
/// \return Void.
void ScheduleTasks(std::unordered_map<NodeID, SchedulingResources> &resource_map);
/// Handle a task whose return value(s) must be reconstructed.
///
/// \param task_id The relevant task ID.
/// \param required_object_id The object id we are reconstructing for.
/// \return Void.
void HandleTaskReconstruction(const TaskID &task_id,
const ObjectID &required_object_id);
/// Attempt to forward a task to a remote different node manager. If this
/// fails, the task will be resubmit locally.
///
/// \param task The task in question.
/// \param node_manager_id The ID of the remote node manager.
/// \return Void.
void ForwardTaskOrResubmit(const Task &task, const NodeID &node_manager_id);
/// Forward a task to another node to execute. The task is assumed to not be
/// queued in local_queues_.
///
/// \param task The task to forward.
/// \param node_id The ID of the node to forward the task to.
/// \param on_error Callback on run on non-ok status.
void ForwardTask(
const Task &task, const NodeID &node_id,
const std::function<void(const ray::Status &, const Task &)> &on_error);
/// Dispatch locally scheduled tasks. This attempts the transition from "scheduled" to
/// "running" task state.
///
/// This function is called in the following cases:
/// (1) A set of new tasks is added to the ready queue.
/// (2) New resources are becoming available on the local node.
/// (3) A new worker becomes available.
/// Note in case (1) we only need to look at the new tasks added to the
/// ready queue, as we know that the old tasks in the ready queue cannot
/// be scheduled (We checked those tasks last time new resources or
/// workers became available, and nothing changed since then.) In this case,
/// tasks_with_resources contains only the newly added tasks to the
/// ready queue. Otherwise, tasks_with_resources points to entire ready queue.
/// \param tasks_with_resources Mapping from resource shapes to tasks with
/// that resource shape.
void DispatchTasks(
const std::unordered_map<SchedulingClass, ordered_set<TaskID>> &tasks_by_class);
/// Handle blocking gets of objects. This could be a task assigned to a worker,
/// an out-of-band task (e.g., a thread created by the application), or a
@ -535,22 +461,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
/// \return Void.
void ProcessPushErrorRequestMessage(const uint8_t *message_data);
/// Process client message of SetResourceRequest
/// \param client The client that sent the message.
/// \param message_data A pointer to the message data.
/// \return Void.
void ProcessSetResourceRequest(const std::shared_ptr<ClientConnection> &client,
const uint8_t *message_data);
/// Finish assigning a task to a worker.
///
/// \param worker Worker that the task is assigned to.
/// \param task_id Id of the task.
/// \param success Whether or not assigning the task was successful.
/// \return void.
void FinishAssignTask(const std::shared_ptr<WorkerInterface> &worker,
const TaskID &task_id, bool success);
/// Process worker subscribing to a given plasma object become available. This handler
/// makes sure that the plasma object is local and calls core worker's PlasmaObjectReady
/// gRPC endpoint.
@ -665,9 +575,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
const std::string &spilled_url,
const NodeID &node_id);
std::unordered_map<SchedulingClass, ordered_set<TaskID>> MakeTasksByClass(
const std::vector<Task> &tasks) const;
/// Get pointers to objects stored in plasma. They will be
/// released once the returned references go out of scope.
///
@ -678,112 +585,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
bool GetObjectsFromPlasma(const std::vector<ObjectID> &object_ids,
std::vector<std::unique_ptr<RayObject>> *results);
///////////////////////////////////////////////////////////////////////////////////////
//////////////////// Begin of the override methods of ClusterTaskManager //////////////
// The following methods are defined in node_manager.task.cc instead of node_manager.cc
/// Return the resources that were being used by this worker.
void ReleaseWorkerResources(std::shared_ptr<WorkerInterface> worker) override;
/// When a task is blocked in ray.get or ray.wait, the worker who is executing the task
/// should give up the CPU resources allocated for the running task for the time being
/// and the worker itself should also be marked as blocked.
///
/// \param worker The worker who will give up the CPU resources.
/// \return true if the cpu resources of the specified worker are released successfully,
/// else false.
bool ReleaseCpuResourcesFromUnblockedWorker(
std::shared_ptr<WorkerInterface> worker) override;
/// When a task is no longer blocked in a ray.get or ray.wait, the CPU resources that
/// the worker gave up should be returned to it.
///
/// \param worker The blocked worker.
/// \return true if the cpu resources are returned back to the specified worker, else
/// false.
bool ReturnCpuResourcesToBlockedWorker(
std::shared_ptr<WorkerInterface> worker) override;
// Schedule and dispatch tasks.
void ScheduleAndDispatchTasks() override;
/// Move tasks from waiting to ready for dispatch. Called when a task's
/// dependencies are resolved.
///
/// \param readyIds: The tasks which are now ready to be dispatched.
void TasksUnblocked(const std::vector<TaskID> &ready_ids) override;
/// Populate the relevant parts of the resource usage. This is intended for
/// sending resource usage from raylet to gcs. In particular, this should fill in
/// resource_load and resource_load_by_shape.
///
/// \param Output parameter. `resource_load` and `resource_load_by_shape` are the only
/// fields used.
void FillResourceUsage(std::shared_ptr<rpc::ResourcesData> data) override;
/// Populate the list of pending or infeasible actor tasks for node stats.
///
/// \param Output parameter.
void FillPendingActorInfo(rpc::GetNodeStatsReply *reply) const override;
/// Return the finished task and relase the worker resources.
/// This method will be removed and can be replaced by `ReleaseWorkerResources` directly
/// once we remove the legacy scheduler.
///
/// \param worker: The worker which was running the task.
/// \param task: Output parameter.
void TaskFinished(std::shared_ptr<WorkerInterface> worker, Task *task) override;
/// Return worker resources.
/// This method will be removed and can be replaced by `ReleaseWorkerResources` directly
/// once we remove the legacy scheduler.
///
/// \param worker: The worker which was running the task.
void ReturnWorkerResources(std::shared_ptr<WorkerInterface> worker) override;
/// Attempt to cancel an already queued task.
///
/// \param task_id: The id of the task to remove.
///
/// \return True if task was successfully removed. This function will return
/// false if the task is already running.
bool CancelTask(const TaskID &task_id) override;
/// Queue task and schedule. This hanppens when processing the worker lease request.
///
/// \param task: The incoming task to be queued and scheduled.
/// \param reply: The reply of the lease request.
/// \param send_reply_callback: The function used during dispatching.
void QueueAndScheduleTask(const Task &task, rpc::RequestWorkerLeaseReply *reply,
rpc::SendReplyCallback send_reply_callback) override;
/// Schedule infeasible tasks.
void ScheduleInfeasibleTasks() override;
/// Return if any tasks are pending resource acquisition.
///
/// \param[in] exemplar An example task that is deadlocking.
/// \param[in] num_pending_actor_creation Number of pending actor creation tasks.
/// \param[in] num_pending_tasks Number of pending tasks.
/// \param[in] any_pending True if there's any pending exemplar.
/// \return True if any progress is any tasks are pending.
bool AnyPendingTasks(Task *exemplar, bool *any_pending, int *num_pending_actor_creation,
int *num_pending_tasks) const override;
/// Handle the resource usage updated event of the specified node.
///
/// \param node_id ID of the node which resources are updated.
/// \param resource_data The node resources.
void OnNodeResourceUsageUpdated(const NodeID &node_id,
const rpc::ResourcesData &resource_data) override;
/// Handle the object missing event.
///
/// \param object_id ID of the missing object.
/// \param waiting_task_ids IDs of tasks that are waitting for the specified missing
/// object.
void OnObjectMissing(const ObjectID &object_id,
const std::vector<TaskID> &waiting_task_ids) override;
/// Disconnect a client.
///
/// \param client The client that sent the message.
@ -792,11 +593,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
void DisconnectClient(
const std::shared_ptr<ClientConnection> &client,
rpc::WorkerExitType disconnect_type = rpc::WorkerExitType::SYSTEM_ERROR_EXIT);
/// The helper to dump the debug state of the cluster task manater.
std::string DebugStr() const override;
//////////////////// End of the Override of ClusterTaskManager //////////////////////
///////////////////////////////////////////////////////////////////////////////////////
/// ID of this node.
NodeID self_node_id_;
@ -827,19 +623,9 @@ class NodeManager : public rpc::NodeManagerServiceHandler,
std::string temp_dir_;
/// Initial node manager configuration.
const NodeManagerConfig initial_config_;
/// The resources (and specific resource IDs) that are currently available.
/// These two resource container is shared with `PlacementGroupResourceManager`.
ResourceIdSet local_available_resources_;
std::unordered_map<NodeID, SchedulingResources> cluster_resource_map_;
/// A pool of workers.
WorkerPool worker_pool_;
/// A set of queues to maintain tasks.
SchedulingQueue local_queues_;
/// The scheduling policy in effect for this raylet.
SchedulingPolicy scheduling_policy_;
/// The reconstruction policy for deciding when to re-execute a task.
ReconstructionPolicy reconstruction_policy_;
/// A manager to resolve objects needed by queued tasks and workers that
/// called `ray.get` or `ray.wait`.
DependencyManager dependency_manager_;

View file

@ -1,355 +0,0 @@
// Copyright 2017 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/raylet/node_manager.h"
namespace ray {
namespace raylet {
void NodeManager::ReleaseWorkerResources(std::shared_ptr<WorkerInterface> worker) {
auto const &task_resources = worker->GetTaskResourceIds();
local_available_resources_.ReleaseConstrained(
task_resources, cluster_resource_map_[self_node_id_].GetTotalResources());
cluster_resource_map_[self_node_id_].Release(task_resources.ToResourceSet());
worker->ResetTaskResourceIds();
auto const &lifetime_resources = worker->GetLifetimeResourceIds();
local_available_resources_.ReleaseConstrained(
lifetime_resources, cluster_resource_map_[self_node_id_].GetTotalResources());
cluster_resource_map_[self_node_id_].Release(lifetime_resources.ToResourceSet());
worker->ResetLifetimeResourceIds();
}
bool NodeManager::ReleaseCpuResourcesFromUnblockedWorker(
std::shared_ptr<WorkerInterface> worker) {
if (!worker || worker->IsBlocked()) {
return false;
}
auto const cpu_resource_ids = worker->ReleaseTaskCpuResources();
local_available_resources_.Release(cpu_resource_ids);
cluster_resource_map_[self_node_id_].Release(cpu_resource_ids.ToResourceSet());
return true;
}
bool NodeManager::ReturnCpuResourcesToBlockedWorker(
std::shared_ptr<WorkerInterface> worker) {
if (!worker || !worker->IsBlocked()) {
return false;
}
const TaskID &task_id = worker->GetAssignedTaskId();
const Task &task = local_queues_.GetTaskOfState(task_id, TaskState::RUNNING);
const auto &required_resources = task.GetTaskSpecification().GetRequiredResources();
const ResourceSet cpu_resources = required_resources.GetNumCpus();
bool oversubscribed = !local_available_resources_.Contains(cpu_resources);
if (!oversubscribed) {
// Reacquire the CPU resources for the worker. Note that care needs to be
// taken if the user is using the specific CPU IDs since the IDs that we
// reacquire here may be different from the ones that the task started with.
auto const resource_ids = local_available_resources_.Acquire(cpu_resources);
worker->AcquireTaskCpuResources(resource_ids);
cluster_resource_map_[self_node_id_].Acquire(cpu_resources);
} else {
// In this case, we simply don't reacquire the CPU resources for the worker.
// The worker can keep running and when the task finishes, it will simply
// not have any CPU resources to release.
RAY_LOG(WARNING)
<< "Resources oversubscribed: "
<< cluster_resource_map_[self_node_id_].GetAvailableResources().ToString();
}
return true;
}
void NodeManager::ScheduleAndDispatchTasks() {
DispatchTasks(local_queues_.GetReadyTasksByClass());
}
void NodeManager::TasksUnblocked(const std::vector<TaskID> &ready_ids) {
if (ready_ids.empty()) {
return;
}
std::unordered_set<TaskID> ready_task_id_set(ready_ids.begin(), ready_ids.end());
// First filter out the tasks that should not be moved to READY.
local_queues_.FilterState(ready_task_id_set, TaskState::BLOCKED);
local_queues_.FilterState(ready_task_id_set, TaskState::RUNNING);
local_queues_.FilterState(ready_task_id_set, TaskState::DRIVER);
// Make sure that the remaining tasks are all WAITING or direct call
// actors.
auto ready_task_id_set_copy = ready_task_id_set;
local_queues_.FilterState(ready_task_id_set_copy, TaskState::WAITING);
// Filter out direct call actors. These are not tracked by the raylet and
// their assigned task ID is the actor ID.
for (const auto &id : ready_task_id_set_copy) {
ready_task_id_set.erase(id);
}
// Queue and dispatch the tasks that are ready to run (i.e., WAITING).
auto ready_tasks = local_queues_.RemoveTasks(ready_task_id_set);
local_queues_.QueueTasks(ready_tasks, TaskState::READY);
DispatchTasks(MakeTasksByClass(ready_tasks));
}
void NodeManager::FillResourceUsage(std::shared_ptr<rpc::ResourcesData> resources_data) {
SchedulingResources &local_resources = cluster_resource_map_[self_node_id_];
local_resources.SetLoadResources(local_queues_.GetTotalResourceLoad());
auto last_heartbeat_resources = gcs_client_->NodeResources().GetLastResourceUsage();
if (!last_heartbeat_resources->GetLoadResources().IsEqual(
local_resources.GetLoadResources())) {
resources_data->set_resource_load_changed(true);
for (const auto &resource_pair :
local_resources.GetLoadResources().GetResourceMap()) {
(*resources_data->mutable_resource_load())[resource_pair.first] =
resource_pair.second;
}
}
// Add resource load by shape. This will be used by the new autoscaler.
auto resource_load = local_queues_.GetResourceLoadByShape(
RayConfig::instance().max_resource_shapes_per_load_report());
resources_data->mutable_resource_load_by_shape()->Swap(&resource_load);
}
void NodeManager::TaskFinished(std::shared_ptr<WorkerInterface> worker, Task *task) {
RAY_CHECK(worker != nullptr && task != nullptr);
const auto &task_id = worker->GetAssignedTaskId();
// (See design_docs/task_states.rst for the state transition diagram.)
RAY_CHECK(local_queues_.RemoveTask(task_id, task)) << task_id;
// Release task's resources. The worker's lifetime resources are still held.
auto const &task_resources = worker->GetTaskResourceIds();
local_available_resources_.ReleaseConstrained(
task_resources, cluster_resource_map_[self_node_id_].GetTotalResources());
cluster_resource_map_[self_node_id_].Release(task_resources.ToResourceSet());
worker->ResetTaskResourceIds();
}
void NodeManager::ReturnWorkerResources(std::shared_ptr<WorkerInterface> worker) {
// Do nothing.
}
bool NodeManager::CancelTask(const TaskID &task_id) {
Task removed_task;
TaskState removed_task_state;
bool canceled = local_queues_.RemoveTask(task_id, &removed_task, &removed_task_state);
if (!canceled) {
// We do not have the task. This could be because we haven't received the
// lease request yet, or because we already granted the lease request and
// it has already been returned.
} else {
if (removed_task.OnDispatch()) {
// We have not yet granted the worker lease. Cancel it now.
removed_task.OnCancellation()();
if (removed_task_state == TaskState::WAITING) {
dependency_manager_.RemoveTaskDependencies(task_id);
}
} else {
// We already granted the worker lease and sent the reply. Re-queue the
// task and wait for the requester to return the leased worker.
local_queues_.QueueTasks({removed_task}, removed_task_state);
}
}
return canceled;
}
void NodeManager::QueueAndScheduleTask(const Task &task,
rpc::RequestWorkerLeaseReply *reply,
rpc::SendReplyCallback send_reply_callback) {
// Override the task dispatch to call back to the client instead of executing the
// task directly on the worker.
TaskID task_id = task.GetTaskSpecification().TaskId();
rpc::Address owner_address = task.GetTaskSpecification().CallerAddress();
Task &mutable_task = const_cast<Task &>(task);
mutable_task.OnDispatchInstead(
[this, owner_address, reply, send_reply_callback](
const std::shared_ptr<void> granted, const std::string &address, int port,
const WorkerID &worker_id, const ResourceIdSet &resource_ids) {
auto worker = std::static_pointer_cast<Worker>(granted);
uint32_t worker_pid = static_cast<uint32_t>(worker->GetProcess().GetId());
reply->mutable_worker_address()->set_ip_address(address);
reply->mutable_worker_address()->set_port(port);
reply->mutable_worker_address()->set_worker_id(worker_id.Binary());
reply->mutable_worker_address()->set_raylet_id(self_node_id_.Binary());
reply->set_worker_pid(worker_pid);
for (const auto &mapping : resource_ids.AvailableResources()) {
auto resource = reply->add_resource_mapping();
resource->set_name(mapping.first);
for (const auto &id : mapping.second.WholeIds()) {
auto rid = resource->add_resource_ids();
rid->set_index(id);
rid->set_quantity(1.0);
}
for (const auto &id : mapping.second.FractionalIds()) {
auto rid = resource->add_resource_ids();
rid->set_index(id.first);
rid->set_quantity(id.second.ToDouble());
}
}
auto reply_failure_handler = [this, worker_id]() {
RAY_LOG(WARNING)
<< "Failed to reply to GCS server, because it might have restarted. GCS "
"cannot obtain the information of the leased worker, so we need to "
"release the leased worker to avoid leakage.";
leased_workers_.erase(worker_id);
metrics_num_task_executed_ -= 1;
};
metrics_num_task_executed_ += 1;
send_reply_callback(Status::OK(), nullptr, reply_failure_handler);
RAY_CHECK(leased_workers_.find(worker_id) == leased_workers_.end())
<< "Worker is already leased out " << worker_id;
leased_workers_[worker_id] = worker;
});
mutable_task.OnSpillbackInstead(
[this, reply, task_id, send_reply_callback](const NodeID &spillback_to,
const std::string &address, int port) {
RAY_LOG(DEBUG) << "Worker lease request SPILLBACK " << task_id;
reply->mutable_retry_at_raylet_address()->set_ip_address(address);
reply->mutable_retry_at_raylet_address()->set_port(port);
reply->mutable_retry_at_raylet_address()->set_raylet_id(spillback_to.Binary());
metrics_num_task_spilled_back_ += 1;
send_reply_callback(Status::OK(), nullptr, nullptr);
});
mutable_task.OnCancellationInstead([reply, task_id, send_reply_callback]() {
RAY_LOG(DEBUG) << "Task lease request canceled " << task_id;
reply->set_canceled(true);
send_reply_callback(Status::OK(), nullptr, nullptr);
});
SubmitTask(task);
}
void NodeManager::ScheduleInfeasibleTasks() { TryLocalInfeasibleTaskScheduling(); }
/// Return if any tasks are pending resource acquisition.
///
/// \param[in] exemplar An example task that is deadlocking.
/// \param[in] num_pending_actor_creation Number of pending actor creation tasks.
/// \param[in] num_pending_tasks Number of pending tasks.
/// \param[in] any_pending True if there's any pending exemplar.
/// \return True if any progress is any tasks are pending.
bool NodeManager::AnyPendingTasks(Task *exemplar, bool *any_pending,
int *num_pending_actor_creation,
int *num_pending_tasks) const {
// See if any tasks are blocked trying to acquire resources.
for (const auto &task : local_queues_.GetTasks(TaskState::READY)) {
const TaskSpecification &spec = task.GetTaskSpecification();
if (spec.IsActorCreationTask()) {
*num_pending_actor_creation += 1;
} else {
*num_pending_tasks += 1;
}
if (!any_pending) {
*exemplar = task;
*any_pending = true;
}
}
return *any_pending;
}
void NodeManager::OnNodeResourceUsageUpdated(const NodeID &node_id,
const rpc::ResourcesData &resource_data) {
auto it = cluster_resource_map_.find(node_id);
if (it == cluster_resource_map_.end()) {
return;
}
// Extract decision for this raylet.
auto decision =
scheduling_policy_.SpillOver(it->second, cluster_resource_map_[self_node_id_]);
std::unordered_set<TaskID> local_task_ids;
for (const auto &task_id : decision) {
// (See design_docs/task_states.rst for the state transition diagram.)
Task task;
TaskState state;
if (!local_queues_.RemoveTask(task_id, &task, &state)) {
return;
}
// Since we are spilling back from the ready and waiting queues, we need
// to unsubscribe the dependencies.
if (state != TaskState::INFEASIBLE) {
// Don't unsubscribe for infeasible tasks because we never subscribed in
// the first place.
dependency_manager_.RemoveTaskDependencies(task_id);
}
// Attempt to forward the task. If this fails to forward the task,
// the task will be resubmit locally.
ForwardTaskOrResubmit(task, node_id);
}
}
void NodeManager::FillPendingActorInfo(rpc::GetNodeStatsReply *reply) const {
// TODO(Shanly): Implement.
}
void NodeManager::OnObjectMissing(const ObjectID &object_id,
const std::vector<TaskID> &waiting_task_ids) {
RAY_UNUSED(object_id);
// Transition any tasks that were in the runnable state and are dependent on
// this object to the waiting state.
if (!waiting_task_ids.empty()) {
std::unordered_set<TaskID> waiting_task_id_set(waiting_task_ids.begin(),
waiting_task_ids.end());
// NOTE(zhijunfu): For direct actors, the worker is initially assigned actor
// creation task ID, which will not be reset after the task finishes. And later
// tasks of this actor will reuse this task ID to require objects from plasma with
// FetchOrReconstruct, since direct actor task IDs are not known to raylet.
// To support actor reconstruction for direct actor, raylet marks actor creation
// task as completed and removes it from `local_queues_` when it receives `TaskDone`
// message from worker. This is necessary because the actor creation task will be
// re-submitted during reconstruction, if the task is not removed previously, the
// new submitted task will be marked as duplicate and thus ignored. So here we check
// for direct actor creation task explicitly to allow this case.
auto iter = waiting_task_id_set.begin();
while (iter != waiting_task_id_set.end()) {
if (IsActorCreationTask(*iter)) {
RAY_LOG(DEBUG) << "Ignoring direct actor creation task " << *iter
<< " when handling object missing for " << object_id;
iter = waiting_task_id_set.erase(iter);
} else {
++iter;
}
}
// First filter out any tasks that can't be transitioned to READY. These
// are running workers or drivers, now blocked in a get.
local_queues_.FilterState(waiting_task_id_set, TaskState::RUNNING);
local_queues_.FilterState(waiting_task_id_set, TaskState::DRIVER);
// Transition the tasks back to the waiting state. They will be made
// runnable once the deleted object becomes available again.
local_queues_.MoveTasks(waiting_task_id_set, TaskState::READY, TaskState::WAITING);
RAY_CHECK(waiting_task_id_set.empty());
// Moving ready tasks to waiting may have changed the load, making space for placing
// new tasks locally.
ScheduleTasks(cluster_resource_map_);
}
}
std::string NodeManager::DebugStr() const {
// As the NodeManager inherites from ClusterTaskManager and the
// `cluster_task_manager_->DebugString()` is invoked inside
// `NodeManager::DebugString()`, which will leads to infinite loop and cause stack
// overflow, so we should rename `ClusterTaskManager::DebugString` to
// `ClusterTaskManager::DebugStr` to avoid this.
return "";
}
} // namespace raylet
} // namespace ray

View file

@ -34,120 +34,6 @@ void PlacementGroupResourceManager::ReturnUnusedBundle(
}
}
OldPlacementGroupResourceManager::OldPlacementGroupResourceManager(
ResourceIdSet &local_available_resources_,
std::unordered_map<NodeID, SchedulingResources> &cluster_resource_map_,
const NodeID &self_node_id_)
: local_available_resources_(local_available_resources_),
cluster_resource_map_(cluster_resource_map_),
self_node_id_(self_node_id_) {}
bool OldPlacementGroupResourceManager::PrepareBundle(
const BundleSpecification &bundle_spec) {
// We will first delete the existing bundle to ensure idempotence.
// The reason why we do this is: after GCS restarts, placement group can be rescheduled
// directly without rolling back the operations performed before the restart.
const auto &bundle_id = bundle_spec.BundleId();
auto iter = bundle_state_map_.find(bundle_id);
if (iter != bundle_state_map_.end()) {
if (iter->second->state == CommitState::COMMITTED) {
// If the bundle state is already committed, it means that prepare request is just
// stale.
RAY_LOG(INFO) << "Duplicate prepare bundle request, skip it directly.";
return true;
} else {
// If there was a bundle in prepare state, it already locked resources, we will
// return bundle resources.
ReturnBundle(bundle_spec);
}
}
auto &local_resource_set = cluster_resource_map_[self_node_id_];
auto bundle_state = std::make_shared<BundleState>();
bool local_resource_enough = bundle_spec.GetRequiredResources().IsSubset(
local_resource_set.GetAvailableResources());
if (local_resource_enough) {
// Register states.
auto it = bundle_state_map_.find(bundle_id);
// Same bundle cannot be rescheduled.
RAY_CHECK(it == bundle_state_map_.end());
// Prepare resources. This shouldn't create formatted placement group resources
// because that'll be done at the commit phase.
bundle_state->acquired_resources =
local_available_resources_.Acquire(bundle_spec.GetRequiredResources());
local_resource_set.Acquire(bundle_spec.GetRequiredResources());
// Register bundle state.
bundle_state->state = CommitState::PREPARED;
bundle_state_map_.emplace(bundle_id, bundle_state);
bundle_spec_map_.emplace(
bundle_id, std::make_shared<BundleSpecification>(bundle_spec.GetMessage()));
}
return bundle_state->acquired_resources.AvailableResources().size() > 0;
}
void OldPlacementGroupResourceManager::CommitBundle(
const BundleSpecification &bundle_spec) {
const auto &bundle_id = bundle_spec.BundleId();
auto it = bundle_state_map_.find(bundle_id);
// When bundle is committed, it should've been prepared already.
// If GCS call `CommitBundleResources` after `CancelResourceReserve`, we will skip it
// directly.
if (it == bundle_state_map_.end()) {
RAY_LOG(INFO) << "The bundle has been cancelled. Skip it directly. Bundle info is "
<< bundle_spec.DebugString();
return;
} else {
// Ignore request If the bundle state is already committed.
if (it->second->state == CommitState::COMMITTED) {
RAY_LOG(INFO) << "Duplicate committ bundle request, skip it directly.";
return;
}
}
const auto &bundle_state = it->second;
bundle_state->state = CommitState::COMMITTED;
const auto &acquired_resources = bundle_state->acquired_resources;
const auto &bundle_resource_labels = bundle_spec.GetFormattedResources();
const auto &formatted_resource_set = ResourceSet(bundle_resource_labels);
local_available_resources_.Release(ResourceIdSet(formatted_resource_set));
cluster_resource_map_[self_node_id_].AddResource(ResourceSet(bundle_resource_labels));
RAY_CHECK(acquired_resources.AvailableResources().size() > 0)
<< "Prepare should've been failed if there were no acquireable resources.";
}
void OldPlacementGroupResourceManager::ReturnBundle(
const BundleSpecification &bundle_spec) {
// We should commit resources if it weren't because
// ReturnBundle requires resources to be committed when it is called.
auto it = bundle_state_map_.find(bundle_spec.BundleId());
if (it == bundle_state_map_.end()) {
RAY_LOG(INFO) << "Duplicate cancel request, skip it directly.";
return;
}
const auto &bundle_state = it->second;
if (bundle_state->state == CommitState::PREPARED) {
CommitBundle(bundle_spec);
}
bundle_state_map_.erase(it);
const auto &resource_set = bundle_spec.GetRequiredResources();
const auto &placement_group_resource_labels = bundle_spec.GetFormattedResources();
// Return resources to ResourceIdSet.
local_available_resources_.Release(ResourceIdSet(resource_set));
local_available_resources_.Acquire(ResourceSet(placement_group_resource_labels));
// Return resources to SchedulingResources.
cluster_resource_map_[self_node_id_].Release(resource_set);
cluster_resource_map_[self_node_id_].Acquire(
ResourceSet(placement_group_resource_labels));
}
NewPlacementGroupResourceManager::NewPlacementGroupResourceManager(
std::shared_ptr<ClusterResourceScheduler> cluster_resource_scheduler_)
: cluster_resource_scheduler_(cluster_resource_scheduler_) {}

View file

@ -31,13 +31,6 @@ enum CommitState {
COMMITTED
};
struct BundleState {
/// Leasing state for 2PC protocol.
CommitState state;
/// Resources that are acquired at preparation stage.
ResourceIdSet acquired_resources;
};
struct pair_hash {
template <class T1, class T2>
std::size_t operator()(const std::pair<T1, T2> &pair) const {
@ -87,53 +80,6 @@ class PlacementGroupResourceManager {
bundle_spec_map_;
};
/// Associated with old scheduler.
class OldPlacementGroupResourceManager : public PlacementGroupResourceManager {
public:
/// Create a old placement group resource manager.
///
/// \param local_available_resources_: The resources (IDs specificed) that are currently
/// available.
/// \param cluster_resource_map_: The resources (without IDs specificed) that
/// are currently available.
/// \param self_node_id_: The related raylet with current
/// placement group manager.
OldPlacementGroupResourceManager(
ResourceIdSet &local_available_resources_,
std::unordered_map<NodeID, SchedulingResources> &cluster_resource_map_,
const NodeID &self_node_id_);
virtual ~OldPlacementGroupResourceManager() = default;
bool PrepareBundle(const BundleSpecification &bundle_spec);
void CommitBundle(const BundleSpecification &bundle_spec);
void ReturnBundle(const BundleSpecification &bundle_spec);
/// Get all local available resource(IDs specificed).
const ResourceIdSet &GetAllResourceIdSet() const { return local_available_resources_; };
/// Get all local available resource(without IDs specificed).
const SchedulingResources &GetAllResourceSetWithoutId() const {
return cluster_resource_map_[self_node_id_];
}
private:
/// The resources (and specific resource IDs) that are currently available.
/// These two resource container is shared with `NodeManager`.
ResourceIdSet &local_available_resources_;
std::unordered_map<NodeID, SchedulingResources> &cluster_resource_map_;
/// Related raylet with current placement group manager.
NodeID self_node_id_;
/// This map represents the commit state of 2PC protocol for atomic placement group
/// creation.
absl::flat_hash_map<BundleID, std::shared_ptr<BundleState>, pair_hash>
bundle_state_map_;
};
/// Associated with new scheduler.
class NewPlacementGroupResourceManager : public PlacementGroupResourceManager {
public:

View file

@ -24,244 +24,6 @@
namespace ray {
class OldPlacementGroupResourceManagerTest : public ::testing::Test {
public:
OldPlacementGroupResourceManagerTest() {
old_placement_group_resource_manager_.reset(
new raylet::OldPlacementGroupResourceManager(
local_available_resources_, cluster_resource_map_, self_node_id_));
}
std::unique_ptr<raylet::OldPlacementGroupResourceManager>
old_placement_group_resource_manager_;
void InitLocalAvailableResource(
std::unordered_map<std::string, double> &unit_resource) {
ResourceSet init_resourece(unit_resource);
cluster_resource_map_[self_node_id_] = SchedulingResources(init_resourece);
local_available_resources_ = ResourceIdSet(init_resourece);
}
void CheckRemainingResourceCorrect(ResourceSet &result_resource) {
auto &remaining_resource =
old_placement_group_resource_manager_->GetAllResourceSetWithoutId();
ASSERT_EQ(1, remaining_resource.GetAvailableResources().IsEqual(result_resource))
<< remaining_resource.GetAvailableResources().ToString() << " vs "
<< result_resource.ToString();
ASSERT_EQ(1, local_available_resources_.ToResourceSet().IsEqual(result_resource))
<< local_available_resources_.ToResourceSet().ToString() << " vs "
<< result_resource.ToString();
}
protected:
ResourceIdSet local_available_resources_;
std::unordered_map<NodeID, SchedulingResources> cluster_resource_map_;
NodeID self_node_id_ = NodeID::FromRandom();
};
TEST_F(OldPlacementGroupResourceManagerTest, TestPrepareBundleResource) {
// 1. create bundle spec.
auto group_id = PlacementGroupID::FromRandom();
std::unordered_map<std::string, double> unit_resource;
unit_resource.insert({"CPU", 1.0});
auto bundle_spec = Mocker::GenBundleCreation(group_id, 1, unit_resource);
/// 2. init local available resource.
InitLocalAvailableResource(unit_resource);
/// 3. prepare bundle resource.
old_placement_group_resource_manager_->PrepareBundle(bundle_spec);
/// 4. check remaining resources is correct.
auto &remaining_resource =
old_placement_group_resource_manager_->GetAllResourceSetWithoutId();
ResourceSet result_resource;
ASSERT_EQ(0, local_available_resources_.AvailableResources().size());
ASSERT_EQ(1, remaining_resource.GetAvailableResources().IsEqual(result_resource))
<< remaining_resource.GetAvailableResources().ToString() << " vs "
<< result_resource.ToString();
ASSERT_EQ(1, local_available_resources_.ToResourceSet().IsEqual(result_resource))
<< local_available_resources_.ToResourceSet().ToString() << " vs "
<< result_resource.ToString();
}
TEST_F(OldPlacementGroupResourceManagerTest, TestPrepareBundleWithInsufficientResource) {
// 1. create bundle spec.
auto group_id = PlacementGroupID::FromRandom();
std::unordered_map<std::string, double> unit_resource;
unit_resource.insert({"CPU", 2.0});
auto bundle_spec = Mocker::GenBundleCreation(group_id, 1, unit_resource);
/// 2. init local available resource.
std::unordered_map<std::string, double> init_unit_resource;
init_unit_resource.insert({"CPU", 1.0});
InitLocalAvailableResource(init_unit_resource);
/// 3. prepare bundle resource.
ASSERT_FALSE(old_placement_group_resource_manager_->PrepareBundle(bundle_spec));
}
TEST_F(OldPlacementGroupResourceManagerTest, TestCommitBundleResource) {
// 1. create bundle spec.
auto group_id = PlacementGroupID::FromRandom();
std::unordered_map<std::string, double> unit_resource;
unit_resource.insert({"CPU", 1.0});
auto bundle_spec = Mocker::GenBundleCreation(group_id, 1, unit_resource);
/// 2. init local available resource.
InitLocalAvailableResource(unit_resource);
/// 3. prepare and commit bundle resource.
old_placement_group_resource_manager_->PrepareBundle(bundle_spec);
old_placement_group_resource_manager_->CommitBundle(bundle_spec);
/// 4. check remaining resources is correct.
auto &remaining_resource =
old_placement_group_resource_manager_->GetAllResourceSetWithoutId();
std::vector<std::string> resource_labels = {"CPU_group_" + group_id.Hex(),
"CPU_group_1_" + group_id.Hex()};
std::vector<double> resource_capacity = {1.0, 1.0};
ResourceSet result_resource(resource_labels, resource_capacity);
ASSERT_EQ(2, local_available_resources_.AvailableResources().size());
ASSERT_EQ(1, remaining_resource.GetAvailableResources().IsEqual(result_resource))
<< remaining_resource.GetAvailableResources().ToString() << " vs "
<< result_resource.ToString();
ASSERT_EQ(1, local_available_resources_.ToResourceSet().IsEqual(result_resource))
<< local_available_resources_.ToResourceSet().ToString() << " vs "
<< result_resource.ToString();
}
TEST_F(OldPlacementGroupResourceManagerTest, TestReturnBundleResource) {
// 1. create bundle spec.
auto group_id = PlacementGroupID::FromRandom();
std::unordered_map<std::string, double> unit_resource;
unit_resource.insert({"CPU", 1.0});
auto bundle_spec = Mocker::GenBundleCreation(group_id, 1, unit_resource);
/// 2. init local available resource.
InitLocalAvailableResource(unit_resource);
/// 3. prepare and commit bundle resource.
old_placement_group_resource_manager_->PrepareBundle(bundle_spec);
old_placement_group_resource_manager_->CommitBundle(bundle_spec);
/// 4. return bundle resource.
old_placement_group_resource_manager_->ReturnBundle(bundle_spec);
/// 5. check remaining resources is correct.
auto &remaining_resource =
old_placement_group_resource_manager_->GetAllResourceSetWithoutId();
ResourceSet result_resource(unit_resource);
ASSERT_EQ(1, local_available_resources_.AvailableResources().size());
ASSERT_EQ(1, remaining_resource.GetAvailableResources().IsEqual(result_resource))
<< remaining_resource.GetAvailableResources().ToString() << " vs "
<< result_resource.ToString();
ASSERT_EQ(1, local_available_resources_.ToResourceSet().IsEqual(result_resource))
<< local_available_resources_.ToResourceSet().ToString() << " vs "
<< result_resource.ToString();
}
TEST_F(OldPlacementGroupResourceManagerTest, TestMultipleBundlesCommitAndReturn) {
// 1. create two bundles spec.
auto group_id = PlacementGroupID::FromRandom();
std::unordered_map<std::string, double> unit_resource;
unit_resource.insert({"CPU", 1.0});
auto first_bundle_spec = Mocker::GenBundleCreation(group_id, 1, unit_resource);
auto second_bundle_spec = Mocker::GenBundleCreation(group_id, 2, unit_resource);
/// 2. init local available resource.
std::unordered_map<std::string, double> init_unit_resource;
init_unit_resource.insert({"CPU", 2.0});
InitLocalAvailableResource(init_unit_resource);
/// 3. prepare and commit two bundle resource.
old_placement_group_resource_manager_->PrepareBundle(first_bundle_spec);
old_placement_group_resource_manager_->PrepareBundle(second_bundle_spec);
old_placement_group_resource_manager_->CommitBundle(first_bundle_spec);
old_placement_group_resource_manager_->CommitBundle(second_bundle_spec);
/// 4. check remaining resources is correct after commit phase.
auto &remaining_resource =
old_placement_group_resource_manager_->GetAllResourceSetWithoutId();
std::vector<std::string> resource_labels = {"CPU_group_" + group_id.Hex(),
"CPU_group_1_" + group_id.Hex(),
"CPU_group_2_" + group_id.Hex()};
std::vector<double> resource_capacity = {2.0, 1.0, 1.0};
ResourceSet result_resource(resource_labels, resource_capacity);
ASSERT_EQ(3, local_available_resources_.AvailableResources().size());
ASSERT_EQ(1, remaining_resource.GetAvailableResources().IsEqual(result_resource))
<< remaining_resource.GetAvailableResources().ToString() << " vs "
<< result_resource.ToString();
ASSERT_EQ(1, local_available_resources_.ToResourceSet().IsEqual(result_resource))
<< local_available_resources_.ToResourceSet().ToString() << " vs "
<< result_resource.ToString();
/// 5. return second bundle.
old_placement_group_resource_manager_->ReturnBundle(second_bundle_spec);
/// 6. check remaining resources is correct after return second bundle.
resource_labels = {"CPU", "CPU_group_" + group_id.Hex(),
"CPU_group_1_" + group_id.Hex()};
resource_capacity = {1.0, 1.0, 1.0};
result_resource = ResourceSet(resource_labels, resource_capacity);
ASSERT_EQ(1, remaining_resource.GetAvailableResources().IsEqual(result_resource))
<< remaining_resource.GetAvailableResources().ToString() << " vs "
<< result_resource.ToString();
ASSERT_EQ(1, local_available_resources_.ToResourceSet().IsEqual(result_resource))
<< local_available_resources_.ToResourceSet().ToString() << " vs "
<< result_resource.ToString();
/// 7. return first bundel.
old_placement_group_resource_manager_->ReturnBundle(first_bundle_spec);
/// 8. check remaining resources is correct after all bundle returned.
result_resource = ResourceSet(init_unit_resource);
ASSERT_EQ(1, remaining_resource.GetAvailableResources().IsEqual(result_resource))
<< remaining_resource.GetAvailableResources().ToString() << " vs "
<< result_resource.ToString();
ASSERT_EQ(1, local_available_resources_.ToResourceSet().IsEqual(result_resource))
<< local_available_resources_.ToResourceSet().ToString() << " vs "
<< result_resource.ToString();
}
TEST_F(OldPlacementGroupResourceManagerTest, TestIdempotencyWithMultiPrepare) {
// 1. create one bundle spec.
auto group_id = PlacementGroupID::FromRandom();
std::unordered_map<std::string, double> unit_resource;
unit_resource.insert({"CPU", 1.0});
auto bundle_spec = Mocker::GenBundleCreation(group_id, 1, unit_resource);
/// 2. init local available resource.
std::unordered_map<std::string, double> available_resource = {
std::make_pair("CPU", 3.0)};
InitLocalAvailableResource(available_resource);
/// 3. prepare bundle resource 10 times.
for (int i = 0; i < 10; i++) {
old_placement_group_resource_manager_->PrepareBundle(bundle_spec);
}
/// 4. check remaining resources is correct.
std::unordered_map<std::string, double> result_resource_map = {
std::make_pair("CPU", 2.0)};
ResourceSet result_resource(result_resource_map);
CheckRemainingResourceCorrect(result_resource);
}
TEST_F(OldPlacementGroupResourceManagerTest, TestIdempotencyWithRandomOrder) {
// 1. create one bundle spec.
auto group_id = PlacementGroupID::FromRandom();
std::unordered_map<std::string, double> unit_resource;
unit_resource.insert({"CPU", 1.0});
auto bundle_spec = Mocker::GenBundleCreation(group_id, 1, unit_resource);
/// 2. init local available resource.
std::unordered_map<std::string, double> available_resource = {
std::make_pair("CPU", 3.0)};
InitLocalAvailableResource(available_resource);
/// 3. prepare bundle -> commit bundle -> prepare bundle.
old_placement_group_resource_manager_->PrepareBundle(bundle_spec);
old_placement_group_resource_manager_->CommitBundle(bundle_spec);
old_placement_group_resource_manager_->PrepareBundle(bundle_spec);
/// 4. check remaining resources is correct.
std::vector<std::string> resource_labels = {"CPU_group_" + group_id.Hex(),
"CPU_group_1_" + group_id.Hex(), "CPU"};
std::vector<double> resource_capacity = {1.0, 1.0, 2.0};
ResourceSet result_resource(resource_labels, resource_capacity);
CheckRemainingResourceCorrect(result_resource);
old_placement_group_resource_manager_->ReturnBundle(bundle_spec);
// 5. prepare bundle -> commit bundle -> commit bundle.
old_placement_group_resource_manager_->PrepareBundle(bundle_spec);
old_placement_group_resource_manager_->CommitBundle(bundle_spec);
old_placement_group_resource_manager_->CommitBundle(bundle_spec);
// 6. check remaining resources is correct.
CheckRemainingResourceCorrect(result_resource);
old_placement_group_resource_manager_->ReturnBundle(bundle_spec);
// 7. prepare bundle -> return bundle -> commit bundle.
old_placement_group_resource_manager_->PrepareBundle(bundle_spec);
old_placement_group_resource_manager_->ReturnBundle(bundle_spec);
old_placement_group_resource_manager_->CommitBundle(bundle_spec);
result_resource = ResourceSet(available_resource);
CheckRemainingResourceCorrect(result_resource);
}
class NewPlacementGroupResourceManagerTest : public ::testing::Test {
public:
std::unique_ptr<raylet::NewPlacementGroupResourceManager>

View file

@ -66,8 +66,17 @@ Raylet::Raylet(boost::asio::io_service &main_service, const std::string &socket_
object_directory_(
RayConfig::instance().ownership_based_object_directory_enabled()
? std::dynamic_pointer_cast<ObjectDirectoryInterface>(
std::make_shared<OwnershipBasedObjectDirectory>(main_service,
gcs_client_))
std::make_shared<OwnershipBasedObjectDirectory>(
main_service, gcs_client_,
[this](const ObjectID &obj_id) {
rpc::ObjectReference ref;
ref.set_object_id(obj_id.Binary());
node_manager_.MarkObjectsAsFailed(
ErrorType::OBJECT_UNRECONSTRUCTABLE, {ref}, JobID::Nil());
}
))
: std::dynamic_pointer_cast<ObjectDirectoryInterface>(
std::make_shared<ObjectDirectory>(main_service, gcs_client_))),
object_manager_(

View file

@ -1,259 +0,0 @@
// Copyright 2017 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/raylet/reconstruction_policy.h"
#include "ray/stats/stats.h"
namespace ray {
namespace raylet {
ReconstructionPolicy::ReconstructionPolicy(
boost::asio::io_service &io_service,
std::function<void(const TaskID &, const ObjectID &)> reconstruction_handler,
int64_t initial_reconstruction_timeout_ms, const NodeID &node_id,
std::shared_ptr<gcs::GcsClient> gcs_client,
std::shared_ptr<ObjectDirectoryInterface> object_directory)
: io_service_(io_service),
reconstruction_handler_(reconstruction_handler),
initial_reconstruction_timeout_ms_(initial_reconstruction_timeout_ms),
node_id_(node_id),
gcs_client_(gcs_client),
object_directory_(std::move(object_directory)) {}
void ReconstructionPolicy::SetTaskTimeout(
std::unordered_map<TaskID, ReconstructionTask>::iterator task_it,
int64_t timeout_ms) {
task_it->second.expires_at = current_time_ms() + timeout_ms;
auto timeout = boost::posix_time::milliseconds(timeout_ms);
task_it->second.reconstruction_timer->expires_from_now(timeout);
const TaskID task_id = task_it->first;
task_it->second.reconstruction_timer->async_wait(
[this, task_id](const boost::system::error_code &error) {
if (!error) {
auto it = listening_tasks_.find(task_id);
if (it == listening_tasks_.end()) {
return;
}
if (it->second.subscribed) {
// If the timer expired and we were subscribed to notifications,
// then this means that we did not receive a task lease
// notification within the lease period. Otherwise, the timer
// would have been reset when the most recent notification was
// received. The current lease is now considered expired.
HandleTaskLeaseExpired(task_id);
} else {
const auto task_lease_notification_callback =
[this](const TaskID &task_id,
const boost::optional<rpc::TaskLeaseData> &task_lease) {
OnTaskLeaseNotification(task_id, task_lease);
};
// This task is still required, so subscribe to task lease
// notifications. Reconstruction will be triggered if the current
// task lease expires, or if no one has acquired the task lease.
// NOTE(swang): When reconstruction for a task is first requested,
// we do not initially subscribe to task lease notifications, which
// requires at least one GCS operation. This is in case the objects
// required by the task are no longer needed soon after. If the
// task is still required after this initial period, then we now
// subscribe to task lease notifications.
RAY_CHECK_OK(gcs_client_->Tasks().AsyncSubscribeTaskLease(
task_id, task_lease_notification_callback, /*done*/ nullptr));
it->second.subscribed = true;
}
} else {
// Check that the error was due to the timer being canceled.
RAY_CHECK(error == boost::asio::error::operation_aborted);
}
});
}
void ReconstructionPolicy::OnTaskLeaseNotification(
const TaskID &task_id, const boost::optional<rpc::TaskLeaseData> &task_lease) {
if (!task_lease) {
// Task lease not exist.
HandleTaskLeaseNotification(task_id, 0);
return;
}
const NodeID node_manager_id = NodeID::FromBinary(task_lease->node_manager_id());
if (gcs_client_->Nodes().IsRemoved(node_manager_id)) {
// The node manager that added the task lease is already removed. The
// lease is considered inactive.
HandleTaskLeaseNotification(task_id, 0);
} else {
// NOTE(swang): The task_lease.timeout is an overestimate of the
// lease's expiration period since the entry may have been in the GCS
// for some time already. For a more accurate estimate, the age of the
// entry in the GCS should be subtracted from task_lease.timeout.
HandleTaskLeaseNotification(task_id, task_lease->timeout());
}
}
void ReconstructionPolicy::HandleReconstructionLogAppend(
const TaskID &task_id, const ObjectID &required_object_id, bool success) {
auto it = listening_tasks_.find(task_id);
if (it == listening_tasks_.end()) {
return;
}
// Reset the timer to wait for task lease notifications again. NOTE(swang):
// The timer should already be set here, but we extend it to give some time
// for the reconstructed task to propagate notifications.
SetTaskTimeout(it, initial_reconstruction_timeout_ms_);
if (success) {
reconstruction_handler_(task_id, required_object_id);
}
}
void ReconstructionPolicy::AttemptReconstruction(const TaskID &task_id,
const ObjectID &required_object_id,
int reconstruction_attempt) {
// If we are no longer listening for objects created by this task, give up.
auto it = listening_tasks_.find(task_id);
if (it == listening_tasks_.end()) {
return;
}
// If the object is no longer required, give up.
if (it->second.created_objects.count(required_object_id) == 0) {
return;
}
// Suppress duplicate reconstructions of the same task. This can happen if,
// for example, a task creates two different objects that both require
// reconstruction.
if (reconstruction_attempt != it->second.reconstruction_attempt) {
// Through some other path, reconstruction was already attempted more than
// reconstruction_attempt many times.
return;
}
// Attempt to reconstruct the task by inserting an entry into the task
// reconstruction log. This will fail if another node has already inserted
// an entry for this reconstruction.
auto reconstruction_entry = std::make_shared<TaskReconstructionData>();
reconstruction_entry->set_task_id(task_id.Binary());
reconstruction_entry->set_num_reconstructions(reconstruction_attempt);
reconstruction_entry->set_node_manager_id(node_id_.Binary());
RAY_CHECK_OK(gcs_client_->Tasks().AttemptTaskReconstruction(
reconstruction_entry,
/*done=*/
[this, task_id, required_object_id](Status status) {
if (status.ok()) {
HandleReconstructionLogAppend(task_id, required_object_id, /*success=*/true);
} else {
HandleReconstructionLogAppend(task_id, required_object_id, /*success=*/false);
}
}));
// Increment the number of times reconstruction has been attempted. This is
// used to suppress duplicate reconstructions of the same task. If
// reconstruction is attempted again, the next attempt will try to insert a
// task reconstruction entry at the next index in the log.
it->second.reconstruction_attempt++;
}
void ReconstructionPolicy::HandleTaskLeaseExpired(const TaskID &task_id) {
auto it = listening_tasks_.find(task_id);
RAY_CHECK(it != listening_tasks_.end());
int reconstruction_attempt = it->second.reconstruction_attempt;
// Lookup the objects created by this task in the object directory. If any
// objects no longer exist on any live nodes, then reconstruction will be
// attempted asynchronously.
for (const auto &created_object_id : it->second.created_objects) {
RAY_CHECK_OK(object_directory_->LookupLocations(
created_object_id, it->second.owner_addresses[created_object_id],
[this, task_id, reconstruction_attempt](
const ray::ObjectID &object_id, const std::unordered_set<ray::NodeID> &nodes,
const std::string &spilled_url, const ray::NodeID &spilled_node_id,
size_t object_size) {
if (nodes.empty() && spilled_url.empty()) {
// The required object no longer exists on any live nodes. Attempt
// reconstruction.
AttemptReconstruction(task_id, object_id, reconstruction_attempt);
}
}));
}
// Reset the timer to wait for task lease notifications again.
SetTaskTimeout(it, initial_reconstruction_timeout_ms_);
}
void ReconstructionPolicy::HandleTaskLeaseNotification(const TaskID &task_id,
int64_t lease_timeout_ms) {
auto it = listening_tasks_.find(task_id);
if (it == listening_tasks_.end()) {
// We are no longer listening for this task, so ignore the notification.
return;
}
if (lease_timeout_ms == 0) {
HandleTaskLeaseExpired(task_id);
} else if ((current_time_ms() + lease_timeout_ms) > it->second.expires_at) {
// The current lease is longer than the timer's current expiration time.
// Reset the timer according to the current lease.
SetTaskTimeout(it, lease_timeout_ms);
}
}
void ReconstructionPolicy::ListenAndMaybeReconstruct(const ObjectID &object_id,
const rpc::Address &owner_address) {
RAY_LOG(DEBUG) << "Listening and maybe reconstructing object " << object_id;
TaskID task_id = object_id.TaskId();
auto it = listening_tasks_.find(task_id);
// Add this object to the list of objects created by the same task.
if (it == listening_tasks_.end()) {
auto inserted = listening_tasks_.emplace(task_id, ReconstructionTask(io_service_));
it = inserted.first;
// Set a timer for the task that created the object. If the lease for that
// task expires, then reconstruction of that task will be triggered.
SetTaskTimeout(it, initial_reconstruction_timeout_ms_);
}
it->second.created_objects.insert(object_id);
it->second.owner_addresses.emplace(object_id, owner_address);
}
void ReconstructionPolicy::Cancel(const ObjectID &object_id) {
RAY_LOG(DEBUG) << "Reconstruction for object " << object_id << " canceled";
TaskID task_id = object_id.TaskId();
auto it = listening_tasks_.find(task_id);
if (it == listening_tasks_.end()) {
// We already stopped listening for this task.
return;
}
it->second.created_objects.erase(object_id);
// If there are no more needed objects created by this task, stop listening
// for notifications.
if (it->second.created_objects.empty()) {
// Cancel notifications for the task lease if we were subscribed to them.
if (it->second.subscribed) {
RAY_CHECK_OK(gcs_client_->Tasks().AsyncUnsubscribeTaskLease(task_id));
}
listening_tasks_.erase(it);
}
}
std::string ReconstructionPolicy::DebugString() const {
std::stringstream result;
result << "ReconstructionPolicy:";
result << "\n- num reconstructing: " << listening_tasks_.size();
return result.str();
}
} // namespace raylet
} // end namespace ray

View file

@ -1,166 +0,0 @@
// Copyright 2017 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 <boost/asio.hpp>
#include <functional>
#include <unordered_map>
#include <unordered_set>
#include "ray/common/id.h"
#include "ray/gcs/gcs_client.h"
#include "ray/object_manager/object_directory.h"
namespace ray {
namespace raylet {
using rpc::TaskReconstructionData;
class ReconstructionPolicyInterface {
public:
virtual void ListenAndMaybeReconstruct(const ObjectID &object_id,
const rpc::Address &owner_address) = 0;
virtual void Cancel(const ObjectID &object_id) = 0;
virtual ~ReconstructionPolicyInterface(){};
};
class ReconstructionPolicy : public ReconstructionPolicyInterface {
public:
/// Create the reconstruction policy.
///
/// \param io_service The event loop to attach reconstruction timers to.
/// \param reconstruction_handler The handler to call if a task needs to be
/// re-executed.
/// \param initial_reconstruction_timeout_ms The initial timeout within which
/// a task lease notification must be received. Otherwise, reconstruction
/// will be triggered.
/// \param node_id The node ID to use when requesting notifications from
/// the GCS.
/// \param gcs_client The Client of GCS.
/// lease notifications from.
ReconstructionPolicy(
boost::asio::io_service &io_service,
std::function<void(const TaskID &, const ObjectID &)> reconstruction_handler,
int64_t initial_reconstruction_timeout_ms, const NodeID &node_id,
std::shared_ptr<gcs::GcsClient> gcs_client,
std::shared_ptr<ObjectDirectoryInterface> object_directory);
/// Listen for task lease notifications about an object that may require
/// reconstruction. If no notifications are received within the initial
/// timeout, then the registered task reconstruction handler will be called
/// for the task that created the object.
///
/// \param object_id The object to check for reconstruction.
void ListenAndMaybeReconstruct(const ObjectID &object_id,
const rpc::Address &owner_address);
/// Cancel listening for an object. Notifications for the object will be
/// ignored. This does not cancel a reconstruction attempt that is already in
/// progress.
///
/// \param object_id The object to cancel.
void Cancel(const ObjectID &object_id);
/// Handle a notification for a task lease. This handler should be called to
/// indicate that a task is currently being executed, so any objects that it
/// creates should not be reconstructed.
///
/// \param task_id The task ID of the task being executed.
/// \param lease_timeout_ms After this timeout, the task's lease is
/// guaranteed to be expired. If a second notification is not received within
/// this timeout, then objects that the task creates may be reconstructed.
void HandleTaskLeaseNotification(const TaskID &task_id, int64_t lease_timeout_ms);
/// Returns debug string for class.
///
/// \return string.
std::string DebugString() const;
private:
struct ReconstructionTask {
ReconstructionTask(boost::asio::io_service &io_service)
: expires_at(INT64_MAX),
subscribed(false),
reconstruction_attempt(0),
reconstruction_timer(new boost::asio::deadline_timer(io_service)) {}
// The objects created by this task that we are listening for notifications for.
std::unordered_set<ObjectID> created_objects;
// Owner addresses of created objects.
std::unordered_map<ObjectID, rpc::Address> owner_addresses;
// The time at which the timer for this task expires, according to this
// node's steady clock.
int64_t expires_at;
// Whether we are subscribed to lease notifications for this task.
bool subscribed;
// The number of times we've attempted reconstructing this task so far.
int reconstruction_attempt;
// The task's reconstruction timer. If this expires before a lease
// notification is received, then the task will be reconstructed.
std::unique_ptr<boost::asio::deadline_timer> reconstruction_timer;
};
/// Set the reconstruction timer for a task. If no task lease notifications
/// are received within the timeout, then reconstruction will be triggered.
/// If the timer was previously set, this method will cancel it and reset the
/// timer to the new timeout.
void SetTaskTimeout(std::unordered_map<TaskID, ReconstructionTask>::iterator task_it,
int64_t timeout_ms);
/// Handle task lease notification from GCS.
void OnTaskLeaseNotification(const TaskID &task_id,
const boost::optional<rpc::TaskLeaseData> &task_lease);
/// Attempt to re-execute a task to reconstruct the required object.
///
/// \param task_id The task to attempt to re-execute.
/// \param required_object_id The object created by the task that requires
/// reconstruction.
/// \param reconstruction_attempt What number attempt this is at
/// reconstructing the task. This is used to suppress duplicate
/// reconstructions of the same task (e.g., if a task creates two objects
/// that both require reconstruction).
void AttemptReconstruction(const TaskID &task_id, const ObjectID &required_object_id,
int reconstruction_attempt);
/// Handle expiration of a task lease.
void HandleTaskLeaseExpired(const TaskID &task_id);
/// Handle the response for an attempt at adding an entry to the task
/// reconstruction log.
void HandleReconstructionLogAppend(const TaskID &task_id, const ObjectID &object_id,
bool success);
/// The event loop.
boost::asio::io_service &io_service_;
/// The handler to call for tasks that require reconstruction.
const std::function<void(const TaskID &, const ObjectID &)> reconstruction_handler_;
/// The initial timeout within which a task lease notification must be
/// received. Otherwise, reconstruction will be triggered.
const int64_t initial_reconstruction_timeout_ms_;
/// The node ID to use when requesting notifications from the GCS.
const NodeID node_id_;
/// A client connection to the GCS.
std::shared_ptr<gcs::GcsClient> gcs_client_;
/// The object directory used to lookup object locations.
std::shared_ptr<ObjectDirectoryInterface> object_directory_;
/// The tasks that we are currently subscribed to in the GCS.
std::unordered_map<TaskID, ReconstructionTask> listening_tasks_;
};
} // namespace raylet
} // namespace ray

View file

@ -1,490 +0,0 @@
// Copyright 2017 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/raylet/reconstruction_policy.h"
#include <boost/asio.hpp>
#include <list>
#include "absl/time/clock.h"
#include "gmock/gmock.h"
#include "gtest/gtest.h"
#include "ray/gcs/callback.h"
#include "ray/gcs/gcs_client/service_based_accessor.h"
#include "ray/gcs/gcs_client/service_based_gcs_client.h"
#include "ray/object_manager/object_directory.h"
#include "ray/raylet/format/node_manager_generated.h"
#include "ray/raylet/reconstruction_policy.h"
namespace ray {
namespace raylet {
using rpc::TaskLeaseData;
// A helper function to get a normal task id.
inline TaskID ForNormalTask() {
const static JobID job_id = JobID::FromInt(1);
const static TaskID driver_task_id = TaskID::ForDriverTask(job_id);
static TaskID task_id =
TaskID::ForNormalTask(job_id, driver_task_id, /*parent_task_counter=*/1);
return task_id;
}
class MockObjectDirectory : public ObjectDirectoryInterface {
public:
MockObjectDirectory() {}
ray::Status LookupLocations(const ObjectID &object_id,
const rpc::Address &owner_address,
const OnLocationsFound &callback) override {
callbacks_.push_back({object_id, callback});
return ray::Status::OK();
}
void FlushCallbacks() {
for (const auto &callback : callbacks_) {
const ObjectID object_id = callback.first;
auto it = locations_.find(object_id);
if (it == locations_.end()) {
callback.second(object_id, std::unordered_set<ray::NodeID>(), "", NodeID::Nil(),
0);
} else {
callback.second(object_id, it->second, "", NodeID::Nil(), 0);
}
}
callbacks_.clear();
}
void SetObjectLocations(const ObjectID &object_id,
const std::unordered_set<NodeID> &locations) {
locations_[object_id] = locations;
}
void HandleNodeRemoved(const NodeID &node_id) override {
for (auto &locations : locations_) {
locations.second.erase(node_id);
}
}
std::string DebugString() const override { return ""; }
MOCK_METHOD0(GetLocalNodeID, ray::NodeID());
MOCK_CONST_METHOD1(LookupRemoteConnectionInfo, void(RemoteConnectionInfo &));
MOCK_CONST_METHOD0(LookupAllRemoteConnections, std::vector<RemoteConnectionInfo>());
MOCK_METHOD4(SubscribeObjectLocations,
ray::Status(const ray::UniqueID &, const ObjectID &,
const rpc::Address &owner_address, const OnLocationsFound &));
MOCK_METHOD2(UnsubscribeObjectLocations,
ray::Status(const ray::UniqueID &, const ObjectID &));
MOCK_METHOD3(ReportObjectAdded,
ray::Status(const ObjectID &, const NodeID &,
const object_manager::protocol::ObjectInfoT &));
MOCK_METHOD3(ReportObjectRemoved,
ray::Status(const ObjectID &, const NodeID &,
const object_manager::protocol::ObjectInfoT &));
private:
std::vector<std::pair<ObjectID, OnLocationsFound>> callbacks_;
std::unordered_map<ObjectID, std::unordered_set<NodeID>> locations_;
};
class MockNodeInfoAccessor : public gcs::ServiceBasedNodeInfoAccessor {
public:
MockNodeInfoAccessor(gcs::ServiceBasedGcsClient *client)
: gcs::ServiceBasedNodeInfoAccessor(client) {}
bool IsRemoved(const NodeID &node_id) const override { return false; }
};
class MockTaskInfoAccessor : public gcs::ServiceBasedTaskInfoAccessor {
public:
MockTaskInfoAccessor(gcs::ServiceBasedGcsClient *client)
: ServiceBasedTaskInfoAccessor(client) {}
Status AsyncSubscribeTaskLease(
const TaskID &task_id,
const gcs::SubscribeCallback<TaskID, boost::optional<TaskLeaseData>> &subscribe,
const gcs::StatusCallback &done) override {
subscribe_callback_ = subscribe;
subscribed_tasks_.insert(task_id);
auto entry = task_lease_table_.find(task_id);
if (entry == task_lease_table_.end()) {
boost::optional<TaskLeaseData> result;
subscribe(task_id, result);
} else {
boost::optional<TaskLeaseData> result(*entry->second);
subscribe(task_id, result);
}
return ray::Status::OK();
}
Status AsyncUnsubscribeTaskLease(const TaskID &task_id) override {
subscribed_tasks_.erase(task_id);
return ray::Status::OK();
}
Status AsyncAddTaskLease(const std::shared_ptr<TaskLeaseData> &task_lease_data,
const gcs::StatusCallback &done) override {
TaskID task_id = TaskID::FromBinary(task_lease_data->task_id());
task_lease_table_[task_id] = task_lease_data;
if (subscribed_tasks_.count(task_id) == 1) {
boost::optional<TaskLeaseData> result(*task_lease_data);
subscribe_callback_(task_id, result);
}
return Status::OK();
}
Status AsyncGetTaskLease(
const TaskID &task_id,
const gcs::OptionalItemCallback<rpc::TaskLeaseData> &callback) override {
auto iter = task_lease_table_.find(task_id);
if (iter != task_lease_table_.end()) {
callback(Status::OK(), *iter->second);
} else {
callback(Status::OK(), boost::none);
}
return Status::OK();
}
Status AttemptTaskReconstruction(
const std::shared_ptr<TaskReconstructionData> &task_data,
const gcs::StatusCallback &done) override {
int log_index = task_data->num_reconstructions();
TaskID task_id = TaskID::FromBinary(task_data->task_id());
if (task_reconstruction_log_[task_id].size() == static_cast<size_t>(log_index)) {
task_reconstruction_log_[task_id].push_back(*task_data);
if (done != nullptr) {
done(Status::OK());
}
} else {
if (done != nullptr) {
done(Status::Invalid("Updating task reconstruction failed."));
}
}
return Status::OK();
}
private:
gcs::SubscribeCallback<TaskID, boost::optional<TaskLeaseData>> subscribe_callback_;
std::unordered_map<TaskID, std::shared_ptr<TaskLeaseData>> task_lease_table_;
std::unordered_set<TaskID> subscribed_tasks_;
std::unordered_map<TaskID, std::vector<TaskReconstructionData>>
task_reconstruction_log_;
};
class MockGcs : public gcs::ServiceBasedGcsClient {
public:
MockGcs() : gcs::ServiceBasedGcsClient(gcs::GcsClientOptions("", 0, "")){};
void Init(gcs::TaskInfoAccessor *task_accessor, gcs::NodeInfoAccessor *node_accessor) {
task_accessor_.reset(task_accessor);
node_accessor_.reset(node_accessor);
}
};
class ReconstructionPolicyTest : public ::testing::Test {
public:
ReconstructionPolicyTest()
: io_service_(),
mock_gcs_(new MockGcs()),
task_accessor_(new MockTaskInfoAccessor(mock_gcs_.get())),
node_accessor_(new MockNodeInfoAccessor(mock_gcs_.get())),
mock_object_directory_(std::make_shared<MockObjectDirectory>()),
reconstruction_timeout_ms_(50),
reconstruction_policy_(std::make_shared<ReconstructionPolicy>(
io_service_,
[this](const TaskID &task_id, const ObjectID &obj) {
TriggerReconstruction(task_id);
},
reconstruction_timeout_ms_, NodeID::FromRandom(), mock_gcs_,
mock_object_directory_)),
timer_canceled_(false) {
subscribe_callback_ = [this](const TaskID &task_id,
const boost::optional<TaskLeaseData> &task_lease) {
if (task_lease) {
reconstruction_policy_->HandleTaskLeaseNotification(task_id,
task_lease->timeout());
} else {
reconstruction_policy_->HandleTaskLeaseNotification(task_id, 0);
}
};
mock_gcs_->Init(task_accessor_, node_accessor_);
}
void TriggerReconstruction(const TaskID &task_id) { reconstructed_tasks_[task_id]++; }
void Tick(const std::function<void(void)> &handler,
std::shared_ptr<boost::asio::deadline_timer> timer,
boost::posix_time::milliseconds timer_period,
const boost::system::error_code &error) {
if (timer_canceled_) {
return;
}
ASSERT_FALSE(error);
handler();
// Fire the timer again after another period.
timer->expires_from_now(timer_period);
timer->async_wait(
[this, handler, timer, timer_period](const boost::system::error_code &error) {
Tick(handler, timer, timer_period, error);
});
}
void SetPeriodicTimer(uint64_t period_ms, const std::function<void(void)> &handler) {
timer_canceled_ = false;
auto timer_period = boost::posix_time::milliseconds(period_ms);
auto timer = std::make_shared<boost::asio::deadline_timer>(io_service_, timer_period);
timer->async_wait(
[this, handler, timer, timer_period](const boost::system::error_code &error) {
Tick(handler, timer, timer_period, error);
});
}
void CancelPeriodicTimer() { timer_canceled_ = true; }
void Run(uint64_t reconstruction_timeout_ms) {
auto timer_period = boost::posix_time::milliseconds(reconstruction_timeout_ms);
auto timer = std::make_shared<boost::asio::deadline_timer>(io_service_, timer_period);
timer->async_wait([this, timer](const boost::system::error_code &error) {
ASSERT_FALSE(error);
io_service_.stop();
});
io_service_.run();
io_service_.reset();
mock_object_directory_->FlushCallbacks();
}
protected:
boost::asio::io_service io_service_;
std::shared_ptr<MockGcs> mock_gcs_;
MockTaskInfoAccessor *task_accessor_;
MockNodeInfoAccessor *node_accessor_;
gcs::SubscribeCallback<TaskID, boost::optional<TaskLeaseData>> subscribe_callback_;
std::shared_ptr<MockObjectDirectory> mock_object_directory_;
uint64_t reconstruction_timeout_ms_;
std::shared_ptr<ReconstructionPolicy> reconstruction_policy_;
bool timer_canceled_;
std::unordered_map<TaskID, int> reconstructed_tasks_;
};
TEST_F(ReconstructionPolicyTest, TestReconstructionSimple) {
TaskID task_id = ForNormalTask();
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
// Listen for an object.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
// Run the test for longer than the reconstruction timeout.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction was triggered for the task that created the
// object.
ASSERT_EQ(reconstructed_tasks_[task_id], 1);
// Run the test again.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction was triggered again.
ASSERT_EQ(reconstructed_tasks_[task_id], 2);
}
TEST_F(ReconstructionPolicyTest, TestReconstructionEvicted) {
TaskID task_id = ForNormalTask();
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
mock_object_directory_->SetObjectLocations(object_id, {NodeID::FromRandom()});
// Listen for both objects.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
// Run the test for longer than the reconstruction timeout.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction was not triggered, since the objects still
// exist on a live node.
ASSERT_EQ(reconstructed_tasks_[task_id], 0);
// Simulate evicting one of the objects.
mock_object_directory_->SetObjectLocations(object_id,
std::unordered_set<ray::NodeID>());
// Run the test again.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction was triggered, since one of the objects was
// evicted.
ASSERT_EQ(reconstructed_tasks_[task_id], 1);
}
TEST_F(ReconstructionPolicyTest, TestReconstructionObjectLost) {
TaskID task_id = ForNormalTask();
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
NodeID node_id = NodeID::FromRandom();
mock_object_directory_->SetObjectLocations(object_id, {node_id});
// Listen for both objects.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
// Run the test for longer than the reconstruction timeout.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction was not triggered, since the objects still
// exist on a live node.
ASSERT_EQ(reconstructed_tasks_[task_id], 0);
// Simulate evicting one of the objects.
mock_object_directory_->HandleNodeRemoved(node_id);
// Run the test again.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction was triggered, since one of the objects was
// evicted.
ASSERT_EQ(reconstructed_tasks_[task_id], 1);
}
TEST_F(ReconstructionPolicyTest, TestDuplicateReconstruction) {
// Create two object IDs produced by the same task.
TaskID task_id = ForNormalTask();
ObjectID object_id1 = ObjectID::FromIndex(task_id, /*index=*/1);
ObjectID object_id2 = ObjectID::FromIndex(task_id, /*index=*/2);
// Listen for both objects.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id1, rpc::Address());
reconstruction_policy_->ListenAndMaybeReconstruct(object_id2, rpc::Address());
// Run the test for longer than the reconstruction timeout.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction is only triggered once for the task that created
// both objects.
ASSERT_EQ(reconstructed_tasks_[task_id], 1);
// Run the test again.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction is again only triggered once.
ASSERT_EQ(reconstructed_tasks_[task_id], 2);
}
TEST_F(ReconstructionPolicyTest, TestReconstructionSuppressed) {
TaskID task_id = ForNormalTask();
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
// Run the test for much longer than the reconstruction timeout.
int64_t test_period = 2 * reconstruction_timeout_ms_;
// Acquire the task lease for a period longer than the test period.
auto task_lease_data = std::make_shared<TaskLeaseData>();
task_lease_data->set_node_manager_id(NodeID::FromRandom().Binary());
task_lease_data->set_acquired_at(absl::GetCurrentTimeNanos() / 1000000);
task_lease_data->set_timeout(2 * test_period);
task_lease_data->set_task_id(task_id.Binary());
RAY_CHECK_OK(mock_gcs_->Tasks().AsyncAddTaskLease(task_lease_data, nullptr));
// Listen for an object.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
// Run the test.
Run(test_period);
// Check that reconstruction is suppressed by the active task lease.
ASSERT_TRUE(reconstructed_tasks_.empty());
// Run the test again past the expiration time of the lease.
Run(task_lease_data->timeout() * 1.1);
// Check that this time, reconstruction is triggered.
ASSERT_EQ(reconstructed_tasks_[task_id], 1);
}
TEST_F(ReconstructionPolicyTest, TestReconstructionContinuallySuppressed) {
TaskID task_id = ForNormalTask();
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
// Listen for an object.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
// Send the reconstruction manager heartbeats about the object.
SetPeriodicTimer(reconstruction_timeout_ms_ / 2, [this, task_id]() {
auto task_lease_data = std::make_shared<TaskLeaseData>();
task_lease_data->set_node_manager_id(NodeID::FromRandom().Binary());
task_lease_data->set_acquired_at(absl::GetCurrentTimeNanos() / 1000000);
task_lease_data->set_timeout(reconstruction_timeout_ms_);
task_lease_data->set_task_id(task_id.Binary());
RAY_CHECK_OK(mock_gcs_->Tasks().AsyncAddTaskLease(task_lease_data, nullptr));
});
// Run the test for much longer than the reconstruction timeout.
Run(reconstruction_timeout_ms_ * 2);
// Check that reconstruction is suppressed.
ASSERT_TRUE(reconstructed_tasks_.empty());
// Cancel the heartbeats to the reconstruction manager.
CancelPeriodicTimer();
// Run the test again.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that this time, reconstruction is triggered.
ASSERT_EQ(reconstructed_tasks_[task_id], 1);
}
TEST_F(ReconstructionPolicyTest, TestReconstructionCanceled) {
TaskID task_id = ForNormalTask();
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
// Listen for an object.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
// Halfway through the reconstruction timeout, cancel the object
// reconstruction.
auto timer_period = boost::posix_time::milliseconds(reconstruction_timeout_ms_);
auto timer = std::make_shared<boost::asio::deadline_timer>(io_service_, timer_period);
timer->async_wait([this, timer, object_id](const boost::system::error_code &error) {
ASSERT_FALSE(error);
reconstruction_policy_->Cancel(object_id);
});
Run(reconstruction_timeout_ms_ * 2);
// Check that reconstruction is suppressed.
ASSERT_TRUE(reconstructed_tasks_.empty());
// Listen for the object again.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
// Run the test again.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that this time, reconstruction is triggered.
ASSERT_EQ(reconstructed_tasks_[task_id], 1);
}
TEST_F(ReconstructionPolicyTest, TestSimultaneousReconstructionSuppressed) {
TaskID task_id = ForNormalTask();
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
// Log a reconstruction attempt to simulate a different node attempting the
// reconstruction first. This should suppress this node's first attempt at
// reconstruction.
auto task_reconstruction_data = std::make_shared<TaskReconstructionData>();
task_reconstruction_data->set_task_id(task_id.Binary());
task_reconstruction_data->set_node_manager_id(NodeID::FromRandom().Binary());
task_reconstruction_data->set_num_reconstructions(0);
RAY_CHECK_OK(mock_gcs_->Tasks().AttemptTaskReconstruction(
task_reconstruction_data,
/*done=*/
[](Status status) { ASSERT_TRUE(status.ok()); }));
// Listen for an object.
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
// Run the test for longer than the reconstruction timeout.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that reconstruction is suppressed by the reconstruction attempt
// logged by the other node.
ASSERT_TRUE(reconstructed_tasks_.empty());
// Run the test for longer than the reconstruction timeout again.
Run(reconstruction_timeout_ms_ * 1.1);
// Check that this time, reconstruction is triggered, since we did not
// receive a task lease notification from the other node yet and our next
// attempt to reconstruct adds an entry at the next index in the
// TaskReconstructionLog.
ASSERT_EQ(reconstructed_tasks_[task_id], 1);
}
} // namespace raylet
} // namespace ray
int main(int argc, char **argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View file

@ -961,20 +961,5 @@ void ClusterTaskManager::SpillWaitingTasks() {
}
}
void ClusterTaskManager::OnNodeResourceUsageUpdated(
const NodeID &node_id, const rpc::ResourcesData &resource_data) {
// TODO(Shanly): This method will be removed and can be replaced by
// `ScheduleAndDispatchTasks` directly once we remove the legacy scheduler.
ScheduleAndDispatchTasks();
}
void ClusterTaskManager::OnObjectMissing(const ObjectID &object_id,
const std::vector<TaskID> &waiting_task_ids) {
// We don't need to do anything if the new scheduler is enabled because tasks
// will get moved back to waiting once they reach the front of the dispatch
// queue.
// TODO(Shanly): This method will be removed once we remove the legacy scheduler.
}
} // namespace raylet
} // namespace ray

View file

@ -157,21 +157,6 @@ class ClusterTaskManager : public ClusterTaskManagerInterface {
// Schedule and dispatch tasks.
void ScheduleAndDispatchTasks() override;
/// Handle the resource usage updated event of the specified node.
///
/// \param node_id ID of the node which resources are updated.
/// \param resource_data The node resources.
void OnNodeResourceUsageUpdated(const NodeID &node_id,
const rpc::ResourcesData &resource_data) override;
/// Handle the object missing event.
///
/// \param object_id ID of the missing object.
/// \param waiting_task_ids IDs of tasks that are waitting for the specified missing
/// object.
void OnObjectMissing(const ObjectID &object_id,
const std::vector<TaskID> &waiting_task_ids) override;
/// The helper to dump the debug state of the cluster task manater.
std::string DebugStr() const override;

View file

@ -113,21 +113,6 @@ class ClusterTaskManagerInterface {
int *num_pending_actor_creation,
int *num_pending_tasks) const = 0;
/// Handle the resource usage updated event of the specified node.
///
/// \param node_id ID of the node which resources are updated.
/// \param resource_data The node resources.
virtual void OnNodeResourceUsageUpdated(const NodeID &node_id,
const rpc::ResourcesData &resource_data) = 0;
/// Handle the object missing event.
///
/// \param object_id ID of the missing object.
/// \param waiting_task_ids IDs of tasks that are waitting for the specified missing
/// object.
virtual void OnObjectMissing(const ObjectID &object_id,
const std::vector<TaskID> &waiting_task_ids) = 0;
/// The helper to dump the debug state of the cluster task manater.
///
/// As the NodeManager inherites from ClusterTaskManager and the

View file

@ -1,118 +0,0 @@
// Copyright 2017 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/raylet/scheduling/old_cluster_resource_scheduler.h"
#include "ray/common/grpc_util.h"
#include "ray/common/ray_config.h"
namespace ray {
OldClusterResourceScheduler::OldClusterResourceScheduler(
const NodeID &self_node_id, ResourceIdSet &local_available_resources,
std::unordered_map<NodeID, SchedulingResources> &cluster_resource_map,
std::shared_ptr<SchedulingResources> last_heartbeat_resources)
: self_node_id_string_(self_node_id.Binary()),
local_available_resources_(local_available_resources),
cluster_resource_map_(cluster_resource_map),
last_heartbeat_resources_(last_heartbeat_resources) {}
bool OldClusterResourceScheduler::RemoveNode(const std::string &node_id_string) {
return cluster_resource_map_.erase(NodeID::FromBinary(node_id_string)) != 0;
}
void OldClusterResourceScheduler::UpdateResourceCapacity(
const std::string &node_id_string, const std::string &resource_name,
double resource_total) {
if (node_id_string == self_node_id_string_) {
local_available_resources_.AddOrUpdateResource(resource_name, resource_total);
}
auto iter = cluster_resource_map_.find(NodeID::FromBinary(node_id_string));
if (iter != cluster_resource_map_.end()) {
auto &scheduling_resources = iter->second;
scheduling_resources.UpdateResourceCapacity(resource_name, resource_total);
}
}
void OldClusterResourceScheduler::DeleteResource(const std::string &node_id_string,
const std::string &resource_name) {
if (node_id_string == self_node_id_string_) {
local_available_resources_.DeleteResource(resource_name);
}
auto iter = cluster_resource_map_.find(NodeID::FromBinary(node_id_string));
if (iter != cluster_resource_map_.end()) {
auto &scheduling_resources = iter->second;
scheduling_resources.DeleteResource(resource_name);
}
}
void OldClusterResourceScheduler::FillResourceUsage(
std::shared_ptr<rpc::ResourcesData> resources_data) {
// TODO(atumanov): modify the heartbeat table protocol to use the ResourceSet
// directly.
// TODO(atumanov): implement a ResourceSet const_iterator.
// If light resource usage report enabled, we only set filed that represent resources
// changed.
auto &local_resources = cluster_resource_map_[NodeID::FromBinary(self_node_id_string_)];
if (!last_heartbeat_resources_->GetTotalResources().IsEqual(
local_resources.GetTotalResources())) {
for (const auto &resource_pair :
local_resources.GetTotalResources().GetResourceMap()) {
(*resources_data->mutable_resources_total())[resource_pair.first] =
resource_pair.second;
}
}
if (!last_heartbeat_resources_->GetAvailableResources().IsEqual(
local_resources.GetAvailableResources())) {
resources_data->set_resources_available_changed(true);
for (const auto &resource_pair :
local_resources.GetAvailableResources().GetResourceMap()) {
(*resources_data->mutable_resources_available())[resource_pair.first] =
resource_pair.second;
}
}
}
bool OldClusterResourceScheduler::UpdateNode(const std::string &node_id_string,
const rpc::ResourcesData &resource_data) {
NodeID node_id = NodeID::FromBinary(node_id_string);
auto iter = cluster_resource_map_.find(node_id);
if (iter == cluster_resource_map_.end()) {
return false;
}
SchedulingResources &remote_resources = iter->second;
if (resource_data.resources_total_size() > 0) {
ResourceSet remote_total(MapFromProtobuf(resource_data.resources_total()));
remote_resources.SetTotalResources(std::move(remote_total));
}
if (resource_data.resources_available_changed()) {
ResourceSet remote_available(MapFromProtobuf(resource_data.resources_available()));
remote_resources.SetAvailableResources(std::move(remote_available));
}
if (resource_data.resource_load_changed()) {
ResourceSet remote_load(MapFromProtobuf(resource_data.resource_load()));
// Extract the load information and save it locally.
remote_resources.SetLoadResources(std::move(remote_load));
}
return true;
}
std::string OldClusterResourceScheduler::GetLocalResourceViewString() const {
SchedulingResources &local_resources =
cluster_resource_map_[NodeID::FromBinary(self_node_id_string_)];
return local_resources.GetAvailableResources().ToString();
}
} // namespace ray

View file

@ -1,72 +0,0 @@
// Copyright 2017 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 "ray/common/task/scheduling_resources.h"
#include "ray/raylet/scheduling/cluster_resource_scheduler_interface.h"
namespace ray {
class OldClusterResourceScheduler : public ClusterResourceSchedulerInterface {
public:
explicit OldClusterResourceScheduler(
const NodeID &self_node_id, ResourceIdSet &local_available_resources,
std::unordered_map<NodeID, SchedulingResources> &cluster_resource_map,
const std::shared_ptr<SchedulingResources> last_heartbeat_resources);
/// Remove node from the cluster data structure. This happens
/// when a node fails or it is removed from the cluster.
///
/// \param node_id_string ID of the node to be removed.
bool RemoveNode(const std::string &node_id_string) override;
/// Update node resources. This hanppens when a node resource usage udpated.
///
/// \param node_id_string ID of the node which resoruces need to be udpated.
/// \param resource_data The node resource data.
bool UpdateNode(const std::string &node_id_string,
const rpc::ResourcesData &resource_data) override;
/// \param node_name: Node whose resource we want to update.
/// \param resource_name: Resource which we want to update.
/// \param resource_total: New capacity of the resource.
void UpdateResourceCapacity(const std::string &node_id_string,
const std::string &resource_name,
double resource_total) override;
/// Delete a given resource from a given node.
///
/// \param node_name: Node whose resource we want to delete.
/// \param resource_name: Resource we want to delete
void DeleteResource(const std::string &node_id_string,
const std::string &resource_name) override;
/// Populate the relevant parts of the heartbeat table. This is intended for
/// sending raylet <-> gcs heartbeats. In particular, this should fill in
/// resources_available and resources_total.
///
/// \param Output parameter. `resources_available` and `resources_total` are the only
/// fields used.
void FillResourceUsage(std::shared_ptr<rpc::ResourcesData> data) override;
/// Return local resources in human-readable string form.
std::string GetLocalResourceViewString() const override;
private:
std::string self_node_id_string_;
ResourceIdSet &local_available_resources_;
std::unordered_map<NodeID, SchedulingResources> &cluster_resource_map_;
const std::shared_ptr<SchedulingResources> last_heartbeat_resources_;
};
} // namespace ray

View file

@ -1,223 +0,0 @@
// Copyright 2017 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/raylet/scheduling_policy.h"
#include <algorithm>
#include <chrono>
#include <random>
#include "ray/util/logging.h"
namespace ray {
namespace raylet {
SchedulingPolicy::SchedulingPolicy(const SchedulingQueue &scheduling_queue)
: scheduling_queue_(scheduling_queue),
gen_(std::chrono::high_resolution_clock::now().time_since_epoch().count()) {}
std::unordered_map<TaskID, NodeID> SchedulingPolicy::Schedule(
std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
const NodeID &local_node_id) {
// The policy decision to be returned.
std::unordered_map<TaskID, NodeID> decision;
#ifndef NDEBUG
RAY_LOG(DEBUG) << "Cluster resource map: ";
for (const auto &node_resource_pair : cluster_resources) {
// pair = NodeID, SchedulingResources
const NodeID &node_id = node_resource_pair.first;
const SchedulingResources &resources = node_resource_pair.second;
RAY_LOG(DEBUG) << "node_id: " << node_id << " "
<< resources.GetAvailableResources().ToString();
}
#endif
// We expect all placeable tasks to be placed on exit from this policy method.
RAY_CHECK(scheduling_queue_.GetTasks(TaskState::PLACEABLE).size() <= 1);
// Iterate over running tasks, get their resource demand and try to schedule.
for (const auto &t : scheduling_queue_.GetTasks(TaskState::PLACEABLE)) {
// Get task's resource demand
const auto &spec = t.GetTaskSpecification();
const auto &resource_demand = spec.GetRequiredPlacementResources();
const TaskID &task_id = spec.TaskId();
// Try to place tasks locally first.
const auto &local_resources = cluster_resources[local_node_id];
ResourceSet available_local_resources =
ResourceSet(local_resources.GetAvailableResources());
// We have to subtract the current "load" because we set the current "load"
// to be the resources used by tasks that are in the
// `SchedulingQueue::ready_queue_` in NodeManager::HandleWorkerAvailable's
// call to SchedulingQueue::GetResourceLoad.
available_local_resources.SubtractResources(local_resources.GetLoadResources());
if (resource_demand.IsSubset(available_local_resources)) {
// This node is a feasible candidate.
decision[task_id] = local_node_id;
ResourceSet new_load(cluster_resources[local_node_id].GetLoadResources());
new_load.AddResources(resource_demand);
cluster_resources[local_node_id].SetLoadResources(std::move(new_load));
continue;
}
// Construct a set of viable node candidates and randomly pick between them.
// Get all the node id keys and randomly pick.
std::vector<NodeID> node_keys;
for (const auto &node_resource_pair : cluster_resources) {
// pair = NodeID, SchedulingResources
NodeID node_id = node_resource_pair.first;
const auto &node_resources = node_resource_pair.second;
ResourceSet available_node_resources =
ResourceSet(node_resources.GetAvailableResources());
// We have to subtract the current "load" because we set the current "load"
// to be the resources used by tasks that are in the
// `SchedulingQueue::ready_queue_` in NodeManager::HandleWorkerAvailable's
// call to SchedulingQueue::GetResourceLoad.
available_node_resources.SubtractResources(node_resources.GetLoadResources());
RAY_LOG(DEBUG) << "node_id " << node_id
<< " avail: " << node_resources.GetAvailableResources().ToString()
<< " load: " << node_resources.GetLoadResources().ToString();
if (resource_demand.IsSubset(available_node_resources)) {
// This node is a feasible candidate.
node_keys.push_back(node_id);
}
}
if (!node_keys.empty()) {
// Choose index at random.
// Initialize a uniform integer distribution over the key space.
// TODO(atumanov): change uniform random to discrete, weighted by resource capacity.
std::uniform_int_distribution<int> distribution(0, node_keys.size() - 1);
int node_key_index = distribution(gen_);
const NodeID &dst_node_id = node_keys[node_key_index];
decision[task_id] = dst_node_id;
// Update dst_node_id's load to keep track of remote task load until
// the next heartbeat.
ResourceSet new_load(cluster_resources[dst_node_id].GetLoadResources());
new_load.AddResources(resource_demand);
cluster_resources[dst_node_id].SetLoadResources(std::move(new_load));
} else {
// If the task doesn't fit, place randomly subject to hard constraints.
for (const auto &node_resource_pair2 : cluster_resources) {
// pair = NodeID, SchedulingResources
NodeID node_id = node_resource_pair2.first;
const auto &node_resources = node_resource_pair2.second;
if (resource_demand.IsSubset(node_resources.GetTotalResources())) {
// This node is a feasible candidate.
node_keys.push_back(node_id);
}
}
// node candidate list constructed, pick randomly.
if (!node_keys.empty()) {
// Choose index at random.
// Initialize a uniform integer distribution over the key space.
// TODO(atumanov): change uniform random to discrete, weighted by resource
// capacity.
std::uniform_int_distribution<int> distribution(0, node_keys.size() - 1);
int node_key_index = distribution(gen_);
const NodeID &dst_node_id = node_keys[node_key_index];
decision[task_id] = dst_node_id;
// Update dst_node_id's load to keep track of remote task load until
// the next heartbeat.
ResourceSet new_load(cluster_resources[dst_node_id].GetLoadResources());
new_load.AddResources(resource_demand);
cluster_resources[dst_node_id].SetLoadResources(std::move(new_load));
} else {
// There are no nodes that can feasibly execute this task. The task remains
// placeable until cluster capacity becomes available.
// TODO(rkn): Propagate a warning to the user.
RAY_LOG(INFO) << "The task with ID " << task_id << " requires "
<< spec.GetRequiredResources().ToString() << " for execution and "
<< spec.GetRequiredPlacementResources().ToString()
<< " for placement, but no nodes have the necessary resources. "
<< "Check the node table to view node resources.";
}
}
}
return decision;
}
std::vector<TaskID> SchedulingPolicy::SpillOverInfeasibleTasks(
SchedulingResources &node_resources) const {
// The policy decision to be returned.
std::vector<TaskID> decision;
ResourceSet new_load(node_resources.GetLoadResources());
// Check if we can accommodate infeasible tasks.
for (const auto &task : scheduling_queue_.GetTasks(TaskState::INFEASIBLE)) {
const auto &spec = task.GetTaskSpecification();
const auto &placement_resources = spec.GetRequiredPlacementResources();
if (placement_resources.IsSubset(node_resources.GetTotalResources())) {
decision.push_back(spec.TaskId());
new_load.AddResources(spec.GetRequiredResources());
}
}
node_resources.SetLoadResources(std::move(new_load));
return decision;
}
std::vector<TaskID> SchedulingPolicy::SpillOver(
SchedulingResources &remote_resources, SchedulingResources &local_resources) const {
// First try to spill infeasible tasks.
auto decision = SpillOverInfeasibleTasks(remote_resources);
// Get local available resources.
ResourceSet available_local_resources =
ResourceSet(local_resources.GetAvailableResources());
available_local_resources.SubtractResources(local_resources.GetLoadResources());
// Try to accommodate up to a single ready task.
bool task_spilled = false;
for (const auto &queue : scheduling_queue_.GetReadyTasksByClass()) {
// Skip tasks for which there are resources available locally.
const auto &task_resources =
TaskSpecification::GetSchedulingClassDescriptor(queue.first);
if (task_resources.IsSubset(available_local_resources)) {
continue;
}
// Try to spill one task.
for (const auto &task_id : queue.second) {
const auto &task = scheduling_queue_.GetTaskOfState(task_id, TaskState::READY);
const auto &spec = task.GetTaskSpecification();
// Make sure the node has enough available resources to prevent forwarding cycles.
if (spec.GetRequiredPlacementResources().IsSubset(
remote_resources.GetAvailableResources())) {
// Update the scheduling resources.
ResourceSet new_remote_load(remote_resources.GetLoadResources());
new_remote_load.AddResources(spec.GetRequiredResources());
remote_resources.SetLoadResources(std::move(new_remote_load));
ResourceSet new_local_load(local_resources.GetLoadResources());
new_local_load.SubtractResources(spec.GetRequiredResources());
local_resources.SetLoadResources(std::move(new_local_load));
decision.push_back(spec.TaskId());
task_spilled = true;
break;
}
}
if (task_spilled) {
break;
}
}
return decision;
}
SchedulingPolicy::~SchedulingPolicy() {}
} // namespace raylet
} // namespace ray

View file

@ -1,83 +0,0 @@
// Copyright 2017 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 <random>
#include <unordered_map>
#include "ray/common/bundle_spec.h"
#include "ray/common/task/scheduling_resources.h"
#include "ray/raylet/scheduling_queue.h"
namespace ray {
namespace raylet {
/// \class SchedulingPolicy
/// \brief Implements a scheduling policy for the node manager.
class SchedulingPolicy {
public:
/// \brief SchedulingPolicy constructor.
///
/// \param scheduling_queue: reference to a scheduler queues object for access to
/// tasks.
/// \return Void.
SchedulingPolicy(const SchedulingQueue &scheduling_queue);
/// \brief Perform a scheduling operation, given a set of cluster resources and
/// producing a mapping of tasks to raylets.
///
/// \param cluster_resources: a set of cluster resources containing resource and load
/// information for some subset of the cluster. For all node IDs in the returned
/// placement map, the corresponding SchedulingResources::resources_load_ is
/// incremented by the aggregate resource demand of the tasks assigned to it.
/// \param local_node_id The ID of the node manager that owns this
/// SchedulingPolicy object.
/// \return Scheduling decision, mapping tasks to raylets for placement.
std::unordered_map<TaskID, NodeID> Schedule(
std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
const NodeID &local_node_id);
/// \brief Given a set of cluster resources, try to spillover infeasible tasks.
///
/// \param node_resources The resource information for a node. This may be
/// the local node.
/// \return Tasks that should be spilled to this node.
std::vector<TaskID> SpillOverInfeasibleTasks(SchedulingResources &node_resources) const;
/// \brief Given a set of cluster resources perform a spill-over scheduling operation.
///
/// \param remote_resources The resource information for a remote node. This
/// is guaranteed to not be the local node. The load info is updated if a
/// task is spilled.
/// \param local_resources The resource information for the local node. The
/// load info is updated if a task is spilled.
/// \return Tasks that should be spilled to this node.
std::vector<TaskID> SpillOver(SchedulingResources &remote_resources,
SchedulingResources &local_resources) const;
/// \brief SchedulingPolicy destructor.
virtual ~SchedulingPolicy();
private:
/// An immutable reference to the scheduling task queues.
const SchedulingQueue &scheduling_queue_;
/// Internally maintained random number generator.
std::mt19937_64 gen_;
};
} // namespace raylet
} // namespace ray

View file

@ -1,546 +0,0 @@
// Copyright 2017 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/raylet/scheduling_queue.h"
#include <sstream>
#include "ray/common/status.h"
#include "ray/stats/stats.h"
namespace {
static constexpr const char *task_state_strings[] = {"placeable", "waiting", "ready",
"running", "infeasible"};
static_assert(sizeof(task_state_strings) / sizeof(const char *) ==
static_cast<int>(ray::raylet::TaskState::kNumTaskQueues),
"Must specify a TaskState name for every task queue");
inline const char *GetTaskStateString(ray::raylet::TaskState task_state) {
return task_state_strings[static_cast<int>(task_state)];
}
// Helper function to get tasks for a job from a given state.
template <typename TaskQueue>
inline void GetTasksForJobFromQueue(const TaskQueue &queue, const ray::JobID &job_id,
std::unordered_set<ray::TaskID> &task_ids) {
const auto &tasks = queue.GetTasks();
for (const auto &task : tasks) {
auto const &spec = task.GetTaskSpecification();
if (job_id == spec.JobId()) {
task_ids.insert(spec.TaskId());
}
}
}
// Helper function to get tasks for an actor from a given state.
template <typename TaskQueue>
inline void GetActorTasksFromQueue(const TaskQueue &queue, const ray::ActorID &actor_id,
std::unordered_set<ray::TaskID> &task_ids) {
const auto &tasks = queue.GetTasks();
for (const auto &task : tasks) {
auto const &spec = task.GetTaskSpecification();
if (spec.IsActorTask() && actor_id == spec.ActorId()) {
task_ids.insert(spec.TaskId());
}
}
}
} // namespace
namespace ray {
namespace raylet {
bool TaskQueue::AppendTask(const TaskID &task_id, const Task &task) {
RAY_CHECK(task_map_.find(task_id) == task_map_.end());
auto list_iterator = task_list_.insert(task_list_.end(), task);
task_map_[task_id] = list_iterator;
// Resource bookkeeping
total_resource_load_.AddResources(task.GetTaskSpecification().GetRequiredResources());
const auto &scheduling_class = task.GetTaskSpecification().GetSchedulingClass();
resource_load_by_shape_[scheduling_class]++;
int64_t backlog_size = task.BacklogSize();
if (backlog_size >
0) { // Poor man's version of RayConfig::instance().report_worker_backlog()
request_backlog_by_shape_[scheduling_class] += task.BacklogSize();
}
return true;
}
bool TaskQueue::RemoveTask(const TaskID &task_id, std::vector<Task> *removed_tasks) {
auto task_found_iterator = task_map_.find(task_id);
if (task_found_iterator == task_map_.end()) {
return false;
}
auto it = task_found_iterator->second;
// Resource bookkeeping
total_resource_load_.SubtractResourcesStrict(
it->GetTaskSpecification().GetRequiredResources());
auto scheduling_class = it->GetTaskSpecification().GetSchedulingClass();
resource_load_by_shape_[scheduling_class]--;
if (resource_load_by_shape_[scheduling_class] == 0) {
resource_load_by_shape_.erase(scheduling_class);
}
request_backlog_by_shape_[scheduling_class] -= it->BacklogSize();
if (request_backlog_by_shape_[scheduling_class] <= 0) {
request_backlog_by_shape_.erase(scheduling_class);
}
if (removed_tasks) {
removed_tasks->push_back(std::move(*it));
}
task_map_.erase(task_found_iterator);
task_list_.erase(it);
return true;
}
bool TaskQueue::HasTask(const TaskID &task_id) const {
return task_map_.find(task_id) != task_map_.end();
}
const std::list<Task> &TaskQueue::GetTasks() const { return task_list_; }
const Task &TaskQueue::GetTask(const TaskID &task_id) const {
auto it = task_map_.find(task_id);
RAY_CHECK(it != task_map_.end());
return *it->second;
}
const ResourceSet &TaskQueue::GetTotalResourceLoad() const {
return total_resource_load_;
}
const std::unordered_map<SchedulingClass, uint64_t> &TaskQueue::GetResourceLoadByShape()
const {
return resource_load_by_shape_;
}
const std::unordered_map<SchedulingClass, int64_t> &TaskQueue::GetRequestBacklogByShape()
const {
return request_backlog_by_shape_;
}
bool ReadyQueue::AppendTask(const TaskID &task_id, const Task &task) {
const auto &scheduling_class = task.GetTaskSpecification().GetSchedulingClass();
tasks_by_class_[scheduling_class].push_back(task_id);
return TaskQueue::AppendTask(task_id, task);
}
bool ReadyQueue::RemoveTask(const TaskID &task_id, std::vector<Task> *removed_tasks) {
if (task_map_.find(task_id) != task_map_.end()) {
const auto &scheduling_class =
task_map_[task_id]->GetTaskSpecification().GetSchedulingClass();
tasks_by_class_[scheduling_class].erase(task_id);
}
return TaskQueue::RemoveTask(task_id, removed_tasks);
}
const std::unordered_map<SchedulingClass, ordered_set<TaskID>>
&ReadyQueue::GetTasksByClass() const {
return tasks_by_class_;
}
const std::list<Task> &SchedulingQueue::GetTasks(TaskState task_state) const {
const auto &queue = GetTaskQueue(task_state);
return queue->GetTasks();
}
const std::unordered_map<SchedulingClass, ordered_set<TaskID>>
&SchedulingQueue::GetReadyTasksByClass() const {
return ready_queue_->GetTasksByClass();
}
const Task &SchedulingQueue::GetTaskOfState(const TaskID &task_id,
TaskState task_state) const {
const auto &queue = GetTaskQueue(task_state);
return queue->GetTask(task_id);
}
ResourceSet SchedulingQueue::GetTotalResourceLoad() const {
auto load = ready_queue_->GetTotalResourceLoad();
// Also take into account infeasible tasks so they show up for autoscaling.
load.AddResources(
task_queues_[static_cast<int>(TaskState::INFEASIBLE)]->GetTotalResourceLoad());
return load;
}
rpc::ResourceLoad SchedulingQueue::GetResourceLoadByShape(
int64_t max_shapes, bool report_worker_backlog) const {
std::unordered_map<SchedulingClass, rpc::ResourceDemand> load;
auto infeasible_queue_load =
task_queues_[static_cast<int>(TaskState::INFEASIBLE)]->GetResourceLoadByShape();
auto ready_queue_load = ready_queue_->GetResourceLoadByShape();
auto backlog_size_load = ready_queue_->GetRequestBacklogByShape();
size_t max_shapes_to_add = ready_queue_load.size() + infeasible_queue_load.size();
if (max_shapes >= 0) {
max_shapes_to_add = max_shapes;
}
// Always collect the 1-CPU resource shape stats, if the specified max shapes
// allows.
static const ResourceSet one_cpu_resource_set(
std::unordered_map<std::string, double>({{kCPU_ResourceLabel, 1}}));
static const SchedulingClass one_cpu_scheduling_cls(
TaskSpecification::GetSchedulingClass(one_cpu_resource_set));
if (max_shapes_to_add > 0) {
if (infeasible_queue_load.count(one_cpu_scheduling_cls) > 0) {
load[one_cpu_scheduling_cls].set_num_infeasible_requests_queued(
infeasible_queue_load.at(one_cpu_scheduling_cls));
}
if (ready_queue_load.count(one_cpu_scheduling_cls) > 0) {
load[one_cpu_scheduling_cls].set_num_ready_requests_queued(
ready_queue_load.at(one_cpu_scheduling_cls));
}
if (report_worker_backlog) {
if (backlog_size_load.count(one_cpu_scheduling_cls) > 0) {
load[one_cpu_scheduling_cls].set_backlog_size(
backlog_size_load.at(one_cpu_scheduling_cls));
}
}
}
// Collect the infeasible queue's load.
auto infeasible_it = infeasible_queue_load.begin();
while (infeasible_it != infeasible_queue_load.end() &&
load.size() < max_shapes_to_add) {
load[infeasible_it->first].set_num_infeasible_requests_queued(infeasible_it->second);
infeasible_it++;
}
// Collect the ready queue's load.
auto ready_it = ready_queue_load.begin();
while (ready_it != ready_queue_load.end() && load.size() < max_shapes_to_add) {
load[ready_it->first].set_num_ready_requests_queued(ready_it->second);
ready_it++;
}
if (report_worker_backlog) {
// Collect the backlog size.
auto backlog_it = backlog_size_load.begin();
while (backlog_it != backlog_size_load.end() && load.size() < max_shapes_to_add) {
load[backlog_it->first].set_backlog_size(backlog_it->second);
backlog_it++;
}
}
// Set the resource shapes.
rpc::ResourceLoad load_proto;
for (auto &demand : load) {
auto demand_proto = load_proto.add_resource_demands();
demand_proto->Swap(&demand.second);
const auto &resource_map =
TaskSpecification::GetSchedulingClassDescriptor(demand.first).GetResourceMap();
for (const auto &resource_pair : resource_map) {
(*demand_proto->mutable_shape())[resource_pair.first] = resource_pair.second;
}
}
return load_proto;
}
const std::unordered_set<TaskID> &SchedulingQueue::GetBlockedTaskIds() const {
return blocked_task_ids_;
}
void SchedulingQueue::FilterStateFromQueue(std::unordered_set<ray::TaskID> &task_ids,
TaskState task_state) const {
auto &queue = GetTaskQueue(task_state);
for (auto it = task_ids.begin(); it != task_ids.end();) {
if (queue->HasTask(*it)) {
it = task_ids.erase(it);
} else {
it++;
}
}
}
void SchedulingQueue::FilterState(std::unordered_set<TaskID> &task_ids,
TaskState filter_state) const {
switch (filter_state) {
case TaskState::PLACEABLE:
FilterStateFromQueue(task_ids, TaskState::PLACEABLE);
break;
case TaskState::WAITING:
FilterStateFromQueue(task_ids, TaskState::WAITING);
break;
case TaskState::READY:
FilterStateFromQueue(task_ids, TaskState::READY);
break;
case TaskState::RUNNING:
FilterStateFromQueue(task_ids, TaskState::RUNNING);
break;
case TaskState::INFEASIBLE:
FilterStateFromQueue(task_ids, TaskState::INFEASIBLE);
break;
case TaskState::BLOCKED: {
const auto blocked_ids = GetBlockedTaskIds();
for (auto it = task_ids.begin(); it != task_ids.end();) {
if (blocked_ids.count(*it) == 1) {
it = task_ids.erase(it);
} else {
it++;
}
}
} break;
case TaskState::DRIVER: {
const auto driver_task_ids = GetDriverTaskIds();
for (auto it = task_ids.begin(); it != task_ids.end();) {
if (driver_task_ids.count(*it) == 1) {
it = task_ids.erase(it);
} else {
it++;
}
}
} break;
default:
RAY_LOG(FATAL) << "Attempting to filter tasks on unrecognized state "
<< static_cast<std::underlying_type<TaskState>::type>(filter_state);
}
}
const std::shared_ptr<TaskQueue> &SchedulingQueue::GetTaskQueue(
TaskState task_state) const {
RAY_CHECK(task_state < TaskState::kNumTaskQueues)
<< static_cast<int>(task_state) << "Task state " << static_cast<int>(task_state)
<< " does not correspond to a task queue";
return task_queues_[static_cast<int>(task_state)];
}
// Helper function to remove tasks in the given set of task_ids from a
// queue, and append them to the given vector removed_tasks.
void SchedulingQueue::RemoveTasksFromQueue(ray::raylet::TaskState task_state,
std::unordered_set<ray::TaskID> &task_ids,
std::vector<ray::Task> *removed_tasks) {
auto &queue = GetTaskQueue(task_state);
for (auto it = task_ids.begin(); it != task_ids.end();) {
const auto &task_id = *it;
if (queue->RemoveTask(task_id, removed_tasks)) {
RAY_LOG(DEBUG) << "Removed task " << task_id << " from "
<< GetTaskStateString(task_state) << " queue";
if (task_state == TaskState::RUNNING) {
num_running_tasks_
[removed_tasks->back().GetTaskSpecification().GetSchedulingClass()] -= 1;
}
it = task_ids.erase(it);
} else {
it++;
}
}
}
std::vector<Task> SchedulingQueue::RemoveTasks(std::unordered_set<TaskID> &task_ids) {
// List of removed tasks to be returned.
std::vector<Task> removed_tasks;
// Try to find the tasks to remove from the queues.
for (const auto &task_state : {
TaskState::PLACEABLE,
TaskState::WAITING,
TaskState::READY,
TaskState::RUNNING,
TaskState::INFEASIBLE,
}) {
RemoveTasksFromQueue(task_state, task_ids, &removed_tasks);
}
RAY_CHECK(task_ids.size() == 0);
return removed_tasks;
}
bool SchedulingQueue::RemoveTask(const TaskID &task_id, Task *removed_task,
TaskState *removed_task_state) {
std::vector<Task> removed_tasks;
std::unordered_set<TaskID> task_id_set = {task_id};
// Try to find the task to remove in the queues.
for (const auto &task_state : {
TaskState::PLACEABLE,
TaskState::WAITING,
TaskState::READY,
TaskState::RUNNING,
TaskState::INFEASIBLE,
}) {
RemoveTasksFromQueue(task_state, task_id_set, &removed_tasks);
if (task_id_set.empty()) {
// The task was removed from the current queue.
if (removed_task_state != nullptr) {
// If the state of the removed task was requested, then set it with the
// current queue's state.
*removed_task_state = task_state;
}
break;
}
}
// Make sure we got the removed task.
if (removed_tasks.size() == 1) {
*removed_task = removed_tasks.front();
RAY_CHECK(removed_task->GetTaskSpecification().TaskId() == task_id);
return true;
}
RAY_LOG(DEBUG) << "Task " << task_id
<< " that is to be removed could not be found any more."
<< " Probably its driver was removed.";
return false;
}
void SchedulingQueue::MoveTasks(std::unordered_set<TaskID> &task_ids, TaskState src_state,
TaskState dst_state) {
std::vector<Task> removed_tasks;
// Remove the tasks from the specified source queue.
switch (src_state) {
case TaskState::PLACEABLE:
RemoveTasksFromQueue(TaskState::PLACEABLE, task_ids, &removed_tasks);
break;
case TaskState::WAITING:
RemoveTasksFromQueue(TaskState::WAITING, task_ids, &removed_tasks);
break;
case TaskState::READY:
RemoveTasksFromQueue(TaskState::READY, task_ids, &removed_tasks);
break;
case TaskState::RUNNING:
RemoveTasksFromQueue(TaskState::RUNNING, task_ids, &removed_tasks);
break;
case TaskState::INFEASIBLE:
RemoveTasksFromQueue(TaskState::INFEASIBLE, task_ids, &removed_tasks);
break;
default:
RAY_LOG(FATAL) << "Attempting to move tasks from unrecognized state "
<< static_cast<std::underlying_type<TaskState>::type>(src_state);
}
// Make sure that all tasks were able to be moved.
RAY_CHECK(task_ids.empty());
// Add the tasks to the specified destination queue.
switch (dst_state) {
case TaskState::PLACEABLE:
QueueTasks(removed_tasks, TaskState::PLACEABLE);
break;
case TaskState::WAITING:
QueueTasks(removed_tasks, TaskState::WAITING);
break;
case TaskState::READY:
QueueTasks(removed_tasks, TaskState::READY);
break;
case TaskState::RUNNING:
QueueTasks(removed_tasks, TaskState::RUNNING);
break;
case TaskState::INFEASIBLE:
QueueTasks(removed_tasks, TaskState::INFEASIBLE);
break;
default:
RAY_LOG(FATAL) << "Attempting to move tasks to unrecognized state "
<< static_cast<std::underlying_type<TaskState>::type>(dst_state);
}
}
void SchedulingQueue::QueueTasks(const std::vector<Task> &tasks, TaskState task_state) {
auto &queue = GetTaskQueue(task_state);
for (const auto &task : tasks) {
RAY_LOG(DEBUG) << "Added task " << task.GetTaskSpecification().TaskId() << " to "
<< GetTaskStateString(task_state) << " queue";
if (task_state == TaskState::RUNNING) {
num_running_tasks_[task.GetTaskSpecification().GetSchedulingClass()] += 1;
}
queue->AppendTask(task.GetTaskSpecification().TaskId(), task);
}
}
bool SchedulingQueue::HasTask(const TaskID &task_id) const {
for (const auto &task_queue : task_queues_) {
if (task_queue->HasTask(task_id)) {
return true;
}
}
return false;
}
std::unordered_set<TaskID> SchedulingQueue::GetTaskIdsForJob(const JobID &job_id) const {
std::unordered_set<TaskID> task_ids;
for (const auto &task_queue : task_queues_) {
GetTasksForJobFromQueue(*task_queue, job_id, task_ids);
}
return task_ids;
}
void SchedulingQueue::AddBlockedTaskId(const TaskID &task_id) {
RAY_LOG(DEBUG) << "Added blocked task " << task_id;
auto inserted = blocked_task_ids_.insert(task_id);
RAY_CHECK(inserted.second);
}
void SchedulingQueue::RemoveBlockedTaskId(const TaskID &task_id) {
RAY_LOG(DEBUG) << "Removed blocked task " << task_id;
auto erased = blocked_task_ids_.erase(task_id);
RAY_CHECK(erased == 1);
}
void SchedulingQueue::AddDriverTaskId(const TaskID &task_id) {
RAY_LOG(DEBUG) << "Added driver task " << task_id;
auto inserted = driver_task_ids_.insert(task_id);
RAY_CHECK(inserted.second);
}
void SchedulingQueue::RemoveDriverTaskId(const TaskID &task_id) {
RAY_LOG(DEBUG) << "Removed driver task " << task_id;
auto erased = driver_task_ids_.erase(task_id);
RAY_CHECK(erased == 1);
}
const std::unordered_set<TaskID> &SchedulingQueue::GetDriverTaskIds() const {
return driver_task_ids_;
}
int SchedulingQueue::NumRunning(const SchedulingClass &cls) const {
auto it = num_running_tasks_.find(cls);
if (it == num_running_tasks_.end()) {
return 0;
} else {
return it->second;
}
}
std::string SchedulingQueue::DebugString() const {
std::stringstream result;
result << "SchedulingQueue:";
for (size_t i = 0; i < static_cast<int>(ray::raylet::TaskState::kNumTaskQueues); i++) {
TaskState task_state = static_cast<TaskState>(i);
result << "\n- num " << GetTaskStateString(task_state)
<< " tasks: " << GetTaskQueue(task_state)->GetTasks().size();
}
result << "\n- num tasks blocked: " << blocked_task_ids_.size();
result << "\nScheduledTaskCounts:";
size_t total = 0;
for (const auto &pair : num_running_tasks_) {
result << "\n- ";
auto desc = TaskSpecification::GetSchedulingClassDescriptor(pair.first);
result << desc.ToString();
result << ": " << pair.second;
total += pair.second;
}
RAY_CHECK(total == GetTaskQueue(TaskState::RUNNING)->GetTasks().size())
<< total << " vs " << GetTaskQueue(TaskState::RUNNING)->GetTasks().size();
return result.str();
}
void SchedulingQueue::RecordMetrics() const {
stats::NumInfeasibleTasks().Record(
GetTaskQueue(TaskState::INFEASIBLE)->GetTasks().size());
}
} // namespace raylet
} // namespace ray

View file

@ -1,374 +0,0 @@
// Copyright 2017 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 <array>
#include <list>
#include <unordered_map>
#include <unordered_set>
#include <vector>
#include "ray/common/task/task.h"
#include "ray/util/logging.h"
#include "ray/util/ordered_set.h"
#include "src/ray/protobuf/gcs.pb.h"
namespace ray {
namespace raylet {
enum class TaskState {
// The task may be placed on a node.
PLACEABLE,
// The task has been placed on a node and is waiting for some object
// dependencies to become local.
WAITING,
// The task has been placed on a node, all dependencies are satisfied, and is
// waiting for resources to run.
READY,
// The task is running on a worker. The task may also be blocked in a ray.get
// or ray.wait call, in which case it also has state BLOCKED.
RUNNING,
// The task has resources that cannot be satisfied by any node, as far as we
// know.
INFEASIBLE,
// The number of task queues. All states that precede this enum must have an
// associated TaskQueue in SchedulingQueue. All states that succeed
// this enum do not have an associated TaskQueue, since the tasks
// in those states may not have any associated task data.
kNumTaskQueues,
// The task is running but blocked in a ray.get or ray.wait call. Tasks that
// were explicitly assigned by us may be both BLOCKED and RUNNING, while
// tasks that were created out-of-band (e.g., the application created
// multiple threads) are only BLOCKED.
BLOCKED,
// The task is a driver task.
DRIVER,
};
class TaskQueue {
public:
/// TaskQueue destructor.
virtual ~TaskQueue() {}
/// \brief Append a task to queue.
///
/// \param task_id The task ID for the task to append.
/// \param task The task to append to the queue.
/// \return Whether the append operation succeeds.
virtual bool AppendTask(const TaskID &task_id, const Task &task);
/// \brief Remove a task from queue.
///
/// \param task_id The task ID for the task to remove from the queue.
/// \param removed_tasks If the task specified by task_id is successfully
/// removed from the queue, the task data is appended to the vector. Can
/// be a nullptr, in which case nothing is appended.
/// \return Whether the removal succeeds.
virtual bool RemoveTask(const TaskID &task_id,
std::vector<Task> *removed_tasks = nullptr);
/// \brief Check if the queue contains a specific task id.
///
/// \param task_id The task ID for the task.
/// \return Whether the task_id exists in this queue.
bool HasTask(const TaskID &task_id) const;
/// \brief Return the task list of the queue.
///
/// \return A list of tasks contained in this queue.
const std::list<Task> &GetTasks() const;
/// Get a task from the queue. The caller must ensure that the task is in
/// the queue.
///
/// \return The task.
const Task &GetTask(const TaskID &task_id) const;
/// \brief Return all resource demand associated with the ready queue.
///
/// \return Aggregate resource demand from ready tasks.
const ResourceSet &GetTotalResourceLoad() const;
/// \brief Get the resources required by the tasks in the queue.
///
/// \return A map from resource shape key to the number of tasks queued that
/// require that shape.
const std::unordered_map<SchedulingClass, uint64_t> &GetResourceLoadByShape() const;
/// \brief Get the resources required by the tasks queued in CoreWorkers.
///
/// \return A map from resource shape key to the number of tasks queued that
/// require that shape.
const std::unordered_map<SchedulingClass, int64_t> &GetRequestBacklogByShape() const;
protected:
/// A list of tasks.
std::list<Task> task_list_;
/// A hash to speed up looking up a task.
std::unordered_map<TaskID, std::list<Task>::iterator> task_map_;
/// Aggregate resources of all the tasks in this queue.
ResourceSet total_resource_load_;
/// Required resources for all the tasks in this queue. This is a
/// map from resource shape key to number of tasks queued that require that
/// shape.
std::unordered_map<SchedulingClass, uint64_t> resource_load_by_shape_;
/// Required resources for all the tasks that are queued in core workers
/// still.. This is a map from resource shape key to number of tasks queued
/// on any worker requesting a lease from this raylet that require that
/// shape.
std::unordered_map<SchedulingClass, int64_t> request_backlog_by_shape_;
};
class ReadyQueue : public TaskQueue {
public:
ReadyQueue(){};
ReadyQueue(const ReadyQueue &other) = delete;
/// ReadyQueue destructor.
virtual ~ReadyQueue() {}
/// \brief Append a task to queue.
///
/// \param task_id The task ID for the task to append.
/// \param task The task to append to the queue.
/// \return Whether the append operation succeeds.
bool AppendTask(const TaskID &task_id, const Task &task) override;
/// \brief Remove a task from queue.
///
/// \param task_id The task ID for the task to remove from the queue.
/// \return Whether the removal succeeds.
bool RemoveTask(const TaskID &task_id, std::vector<Task> *removed_tasks) override;
/// \brief Get a mapping from resource shape to tasks.
///
/// \return Mapping from resource set to task IDs with these resource requirements.
const std::unordered_map<SchedulingClass, ordered_set<TaskID>> &GetTasksByClass() const;
private:
/// Index from task description to tasks queued of that type.
std::unordered_map<SchedulingClass, ordered_set<TaskID>> tasks_by_class_;
};
/// \class SchedulingQueue
///
/// Encapsulates task queues.
// (See design_docs/task_states.rst for the state transition diagram.)
class SchedulingQueue {
public:
/// Create a scheduling queue.
SchedulingQueue() : ready_queue_(std::make_shared<ReadyQueue>()) {
for (const auto &task_state : {
TaskState::PLACEABLE,
TaskState::WAITING,
TaskState::READY,
TaskState::RUNNING,
TaskState::INFEASIBLE,
}) {
if (task_state == TaskState::READY) {
task_queues_[static_cast<int>(task_state)] = ready_queue_;
} else {
task_queues_[static_cast<int>(task_state)] = std::make_shared<TaskQueue>();
}
}
}
/// SchedulingQueue destructor.
virtual ~SchedulingQueue() {}
/// \brief Check if the queue contains a specific task id.
///
/// \param task_id The task ID for the task.
/// \return Whether the task_id exists in the queue.
bool HasTask(const TaskID &task_id) const;
/// \brief Get all tasks in the given state.
///
/// \param task_state The requested task state. This must correspond to one
/// of the task queues (has value < TaskState::kNumTaskQueues).
const std::list<Task> &GetTasks(TaskState task_state) const;
/// Get a reference to the queue of ready tasks.
///
/// \return A reference to the queue of ready tasks.
const std::unordered_map<SchedulingClass, ordered_set<TaskID>> &GetReadyTasksByClass()
const;
/// Get a task from the queue of a given state. The caller must ensure that
/// the task has the given state.
///
/// \param task_id The task to get.
/// \param task_state The state that the requested task should be in.
/// \return The task.
const Task &GetTaskOfState(const TaskID &task_id, TaskState task_state) const;
/// \brief Return an aggregate resource set for all tasks exerting load on this raylet.
///
/// \return A resource set with aggregate resource information about resource load on
/// this raylet.
ResourceSet GetTotalResourceLoad() const;
/// \brief Return a summary of the requests in the ready and infeasible
/// queues.
///
/// \return A message summarizing the number of requests, sorted by shape, in
/// the ready and infeasible queues.
rpc::ResourceLoad GetResourceLoadByShape(int64_t max_shapes = -1,
bool report_worker_backlog = true) const;
/// Get the tasks in the blocked state.
///
/// \return A const reference to the tasks that are are blocked on a data
/// dependency discovered to be missing at runtime. These include RUNNING
/// tasks that were explicitly assigned to a worker by us, as well as tasks
/// that were created out-of-band (e.g., the application created
// multiple threads) are only BLOCKED.
const std::unordered_set<TaskID> &GetBlockedTaskIds() const;
/// Get the set of driver task IDs.
///
/// \return A const reference to the set of driver task IDs. These are empty
/// tasks used to represent drivers.
const std::unordered_set<TaskID> &GetDriverTaskIds() const;
/// Remove tasks from the task queue.
///
/// \param task_ids The set of task IDs to remove from the queue. The
/// corresponding tasks must be contained in the queue. The IDs of removed
/// tasks will be erased from the set.
/// \return A vector of the tasks that were removed.
std::vector<Task> RemoveTasks(std::unordered_set<TaskID> &task_ids);
/// Remove a task from the task queue.
///
/// \param task_id The task ID to remove from the queue. The corresponding
/// task must be contained in the queue.
/// \param task The removed task will be written here, if any.
/// \param task_state If this is not nullptr, then the state of the removed
/// task will be written here.
/// \return true if the task was removed, false if it is not in the queue.
bool RemoveTask(const TaskID &task_id, Task *removed_task,
TaskState *removed_task_state = nullptr);
/// Remove a driver task ID. This is an empty task used to represent a driver.
///
/// \param The driver task ID to remove.
void RemoveDriverTaskId(const TaskID &task_id);
/// Add tasks to the given queue.
///
/// \param tasks The tasks to queue.
/// \param task_state The state of the tasks to queue. The requested task
/// state must correspond to one of the task queues (has value <
/// TaskState::kNumTaskQueues).
void QueueTasks(const std::vector<Task> &tasks, TaskState task_state);
/// Add a task ID in the blocked state. These are tasks that have been
/// dispatched to a worker but are blocked on a data dependency that was
/// discovered to be missing at runtime.
///
/// \param task_id The task to mark as blocked.
void AddBlockedTaskId(const TaskID &task_id);
/// Remove a task ID in the blocked state. These are tasks that have been
/// dispatched to a worker but were blocked on a data dependency that was
/// discovered to be missing at runtime.
///
/// \param task_id The task to mark as unblocked.
void RemoveBlockedTaskId(const TaskID &task_id);
/// Add a driver task ID. This is an empty task used to represent a driver.
///
/// \param The driver task ID to add.
void AddDriverTaskId(const TaskID &task_id);
/// \brief Move the specified tasks from the source state to the destination
/// state.
///
/// \param tasks The set of task IDs to move. The IDs of successfully moved
/// tasks will be erased from the set.
/// \param src_state Source state, which corresponds to one of the internal
/// task queues.
/// \param dst_state Destination state, corresponding to one of the internal
/// task queues.
void MoveTasks(std::unordered_set<TaskID> &tasks, TaskState src_state,
TaskState dst_state);
/// \brief Filter out task IDs based on their scheduling state.
///
/// \param task_ids The set of task IDs to filter. All tasks that have the
/// given filter_state will be removed from this set.
/// \param filter_state The task state to filter out.
void FilterState(std::unordered_set<TaskID> &task_ids, TaskState filter_state) const;
/// \brief Get all the task IDs for a job.
///
/// \param job_id All the tasks that have the given job_id are returned.
/// \return All the tasks that have the given job ID.
std::unordered_set<TaskID> GetTaskIdsForJob(const JobID &job_id) const;
/// Returns the number of running tasks in this class.
///
/// \return int.
int NumRunning(const SchedulingClass &cls) const;
/// Returns debug string for class.
///
/// \return string.
std::string DebugString() const;
/// Record metrics.
void RecordMetrics() const;
private:
/// Get the task queue in the given state. The requested task state must
/// correspond to one of the task queues (has value <
/// TaskState::kNumTaskQueues).
const std::shared_ptr<TaskQueue> &GetTaskQueue(TaskState task_state) const;
/// A helper function to remove tasks from a given queue. The requested task
/// state must correspond to one of the task queues (has value <
/// TaskState::kNumTaskQueues).
void RemoveTasksFromQueue(ray::raylet::TaskState task_state,
std::unordered_set<ray::TaskID> &task_ids,
std::vector<ray::Task> *removed_tasks);
/// A helper function to filter out tasks of a given state from the set of
/// task IDs. The requested task state must correspond to one of the task
/// queues (has value < TaskState::kNumTaskQueues).
void FilterStateFromQueue(std::unordered_set<ray::TaskID> &task_ids,
TaskState task_state) const;
// A pointer to the ready queue.
const std::shared_ptr<ReadyQueue> ready_queue_;
/// Track the breakdown of tasks by class in the RUNNING queue.
std::unordered_map<SchedulingClass, int32_t> num_running_tasks_;
// A pointer to the task queues. These contain all tasks that have a task
// state < TaskState::kNumTaskQueues.
std::array<std::shared_ptr<TaskQueue>, static_cast<int>(TaskState::kNumTaskQueues)>
task_queues_;
/// Tasks that were dispatched to a worker but are blocked on a data
/// dependency that was missing at runtime.
std::unordered_set<TaskID> blocked_task_ids_;
/// The set of currently running driver tasks. These are empty tasks that are
/// started by a driver process on initialization.
std::unordered_set<TaskID> driver_task_ids_;
};
} // namespace raylet
} // namespace ray