diff --git a/BUILD.bazel b/BUILD.bazel index f6b20a7f48d1..4d11c44a714f 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -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"], diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 6c90a88bb711..ae8d409de4c1 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -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, diff --git a/python/ray/tests/test_unreconstructable_errors.py b/python/ray/tests/test_unreconstructable_errors.py deleted file mode 100644 index 24be89b94297..000000000000 --- a/python/ray/tests/test_unreconstructable_errors.py +++ /dev/null @@ -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__])) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index ddda57a17c42..0bd255012bd0 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -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) diff --git a/src/ray/common/task/task_spec.h b/src/ray/common/task/task_spec.h index bee66464cda4..28acb3ac49ab 100644 --- a/src/ray/common/task/task_spec.h +++ b/src/ray/common/task/task_spec.h @@ -175,8 +175,6 @@ class TaskSpecification : public MessageWrapper { ObjectID PreviousActorTaskDummyObjectId() const; - bool IsDirectCall() const; - int MaxActorConcurrency() const; bool IsAsyncioActor() const; diff --git a/src/ray/object_manager/ownership_based_object_directory.cc b/src/ray/object_manager/ownership_based_object_directory.cc index a390bbb40271..e619512ccdbc 100644 --- a/src/ray/object_manager/ownership_based_object_directory.cc +++ b/src/ray/object_manager/ownership_based_object_directory.cc @@ -17,8 +17,11 @@ namespace ray { OwnershipBasedObjectDirectory::OwnershipBasedObjectDirectory( - boost::asio::io_service &io_service, std::shared_ptr &gcs_client) - : ObjectDirectory(io_service, gcs_client), client_call_manager_(io_service) {} + boost::asio::io_service &io_service, std::shared_ptr &gcs_client, + std::function 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_client, bool UpdateObjectLocations(const rpc::GetObjectLocationsOwnerReply &location_reply, const Status &status, const ObjectID &object_id, std::shared_ptr gcs_client, + std::function mark_as_failed, std::unordered_set *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 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::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 diff --git a/src/ray/object_manager/ownership_based_object_directory.h b/src/ray/object_manager/ownership_based_object_directory.h index 5b07f79998cd..d7ca0fe511da 100644 --- a/src/ray/object_manager/ownership_based_object_directory.h +++ b/src/ray/object_manager/ownership_based_object_directory.h @@ -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_client); + std::shared_ptr &gcs_client, + std::function 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 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 diff --git a/src/ray/raylet/dependency_manager.cc b/src/ray/raylet/dependency_manager.cc index 9481419987f5..bf260f9ccc0b 100644 --- a/src/ray/raylet/dependency_manager.cc +++ b/src/ray/raylet/dependency_manager.cc @@ -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 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 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); } diff --git a/src/ray/raylet/dependency_manager.h b/src/ray/raylet/dependency_manager.h index da8b244b3433..471de1ae4134 100644 --- a/src/ray/raylet/dependency_manager.h +++ b/src/ray/raylet/dependency_manager.h @@ -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. diff --git a/src/ray/raylet/dependency_manager_test.cc b/src/ray/raylet/dependency_manager_test.cc index 524bd99985ef..6f3fe9d96502 100644 --- a/src/ray/raylet/dependency_manager_test.cc +++ b/src/ray/raylet/dependency_manager_test.cc @@ -46,19 +46,10 @@ class MockObjectManager : public ObjectManagerInterface { std::unordered_set 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 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 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 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 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 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(); } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 5f183fbe44e9..2ce61d70182f 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -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(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(new ClusterTaskManager( - self_node_id_, - std::dynamic_pointer_cast(cluster_resource_scheduler_), - dependency_manager_, is_owner_alive, get_node_info_func, announce_infeasible_task, - worker_pool_, leased_workers_, - [this](const std::vector &object_ids, - std::vector> *results) { - return GetObjectsFromPlasma(object_ids, results); - })); - placement_group_resource_manager_ = - std::make_shared( - std::dynamic_pointer_cast( - cluster_resource_scheduler_)); - } else { - cluster_resource_scheduler_ = std::make_shared( - self_node_id_, local_available_resources_, cluster_resource_map_, - gcs_client_->NodeResources().GetLastResourceUsage()); - cluster_task_manager_ = std::shared_ptr(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( - local_available_resources_, cluster_resource_map_, self_node_id_); - } + SchedulingResources local_resources(config.resource_config); + cluster_resource_scheduler_ = + std::shared_ptr(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(new ClusterTaskManager( + self_node_id_, + std::dynamic_pointer_cast(cluster_resource_scheduler_), + dependency_manager_, is_owner_alive, get_node_info_func, announce_infeasible_task, + worker_pool_, leased_workers_, + [this](const std::vector &object_ids, + std::vector> *results) { + return GetObjectsFromPlasma(object_ids, results); + })); + placement_group_resource_manager_ = std::make_shared( + std::dynamic_pointer_cast(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 decision = - scheduling_policy_.SpillOverInfeasibleTasks(new_local_resources); - - std::unordered_set 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 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> NodeManager::MakeTasksByClass( - const std::vector &tasks) const { - std::unordered_map> 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> &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> *> 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> *a, - const std::pair> *b) { - return local_queues_.NumRunning(a->first) < local_queues_.NumRunning(b->first); - }); - } - std::vector> 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 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 &client, int64_t message_type, const uint8_t *message_data) { @@ -1004,15 +890,6 @@ void NodeManager::ProcessClientMessage(const std::shared_ptr & // 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 & 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(message_data); AsyncResolveObjectsFinish(client, from_flatbuf(*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 & 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 &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 &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(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 &client, const uint8_t *message_data) { - // Read the SetResource message - auto message = flatbuffers::GetRoot(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(*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> data_map; - auto resource_table_data = std::make_shared(); - 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 &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 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 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 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 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 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 &worker, bool release_resources) { if (!worker || worker->IsBlocked() || worker->GetAssignedTaskId().IsNil() || @@ -1829,40 +1542,13 @@ void NodeManager::AsyncResolveObjects( const std::vector &required_object_refs, const TaskID ¤t_task_id, bool ray_get, bool mark_worker_blocked) { std::shared_ptr 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 &client, const TaskID ¤t_task_id, bool was_blocked) { std::shared_ptr 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,86 +1579,7 @@ 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 &worker, - const Task &task, - std::vector> *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 &worker_ptr) { @@ -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( - 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 &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 &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(); } diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 7d515c97502e..067bc141e1cd 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -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 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 &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 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 &worker, const Task &task, - std::vector> *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 &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 &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> &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 &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 &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> MakeTasksByClass( - const std::vector &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 &object_ids, std::vector> *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 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 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 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 &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 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 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 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 &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 &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 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_; diff --git a/src/ray/raylet/node_manager.task.cc b/src/ray/raylet/node_manager.task.cc deleted file mode 100644 index 2fec7360b354..000000000000 --- a/src/ray/raylet/node_manager.task.cc +++ /dev/null @@ -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 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 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 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 &ready_ids) { - if (ready_ids.empty()) { - return; - } - - std::unordered_set 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 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 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 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); - mutable_task.OnDispatchInstead( - [this, owner_address, reply, send_reply_callback]( - const std::shared_ptr granted, const std::string &address, int port, - const WorkerID &worker_id, const ResourceIdSet &resource_ids) { - auto worker = std::static_pointer_cast(granted); - uint32_t worker_pid = static_cast(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 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 &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 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 diff --git a/src/ray/raylet/placement_group_resource_manager.cc b/src/ray/raylet/placement_group_resource_manager.cc index fedcc607de38..8e95d6dd7b6f 100644 --- a/src/ray/raylet/placement_group_resource_manager.cc +++ b/src/ray/raylet/placement_group_resource_manager.cc @@ -34,120 +34,6 @@ void PlacementGroupResourceManager::ReturnUnusedBundle( } } -OldPlacementGroupResourceManager::OldPlacementGroupResourceManager( - ResourceIdSet &local_available_resources_, - std::unordered_map &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(); - 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(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 cluster_resource_scheduler_) : cluster_resource_scheduler_(cluster_resource_scheduler_) {} diff --git a/src/ray/raylet/placement_group_resource_manager.h b/src/ray/raylet/placement_group_resource_manager.h index 20ef5325aa3c..ab688fe5e69a 100644 --- a/src/ray/raylet/placement_group_resource_manager.h +++ b/src/ray/raylet/placement_group_resource_manager.h @@ -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 std::size_t operator()(const std::pair &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 &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 &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, pair_hash> - bundle_state_map_; -}; - /// Associated with new scheduler. class NewPlacementGroupResourceManager : public PlacementGroupResourceManager { public: diff --git a/src/ray/raylet/placement_group_resource_manager_test.cc b/src/ray/raylet/placement_group_resource_manager_test.cc index e78a21b40a7c..f6a81edb19a9 100644 --- a/src/ray/raylet/placement_group_resource_manager_test.cc +++ b/src/ray/raylet/placement_group_resource_manager_test.cc @@ -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 - old_placement_group_resource_manager_; - - void InitLocalAvailableResource( - std::unordered_map &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 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 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 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 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 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 resource_labels = {"CPU_group_" + group_id.Hex(), - "CPU_group_1_" + group_id.Hex()}; - std::vector 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 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 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 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 resource_labels = {"CPU_group_" + group_id.Hex(), - "CPU_group_1_" + group_id.Hex(), - "CPU_group_2_" + group_id.Hex()}; - std::vector 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 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 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 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 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 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 resource_labels = {"CPU_group_" + group_id.Hex(), - "CPU_group_1_" + group_id.Hex(), "CPU"}; - std::vector 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 diff --git a/src/ray/raylet/raylet.cc b/src/ray/raylet/raylet.cc index 687b68409e84..19381f27f63e 100644 --- a/src/ray/raylet/raylet.cc +++ b/src/ray/raylet/raylet.cc @@ -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( - std::make_shared(main_service, - gcs_client_)) + std::make_shared( + 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( std::make_shared(main_service, gcs_client_))), object_manager_( diff --git a/src/ray/raylet/reconstruction_policy.cc b/src/ray/raylet/reconstruction_policy.cc deleted file mode 100644 index 1da422529cda..000000000000 --- a/src/ray/raylet/reconstruction_policy.cc +++ /dev/null @@ -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 reconstruction_handler, - int64_t initial_reconstruction_timeout_ms, const NodeID &node_id, - std::shared_ptr gcs_client, - std::shared_ptr 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::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 &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 &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(); - 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 &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 diff --git a/src/ray/raylet/reconstruction_policy.h b/src/ray/raylet/reconstruction_policy.h deleted file mode 100644 index e221faffe910..000000000000 --- a/src/ray/raylet/reconstruction_policy.h +++ /dev/null @@ -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 -#include -#include -#include - -#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 reconstruction_handler, - int64_t initial_reconstruction_timeout_ms, const NodeID &node_id, - std::shared_ptr gcs_client, - std::shared_ptr 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 created_objects; - // Owner addresses of created objects. - std::unordered_map 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 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::iterator task_it, - int64_t timeout_ms); - - /// Handle task lease notification from GCS. - void OnTaskLeaseNotification(const TaskID &task_id, - const boost::optional &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 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_client_; - /// The object directory used to lookup object locations. - std::shared_ptr object_directory_; - /// The tasks that we are currently subscribed to in the GCS. - std::unordered_map listening_tasks_; -}; - -} // namespace raylet - -} // namespace ray diff --git a/src/ray/raylet/reconstruction_policy_test.cc b/src/ray/raylet/reconstruction_policy_test.cc deleted file mode 100644 index d4eb387a3ac0..000000000000 --- a/src/ray/raylet/reconstruction_policy_test.cc +++ /dev/null @@ -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 -#include - -#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(), "", 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 &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()); - 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> callbacks_; - std::unordered_map> 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> &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 result; - subscribe(task_id, result); - } else { - boost::optional 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 &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 result(*task_lease_data); - subscribe_callback_(task_id, result); - } - return Status::OK(); - } - - Status AsyncGetTaskLease( - const TaskID &task_id, - const gcs::OptionalItemCallback &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 &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(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> subscribe_callback_; - std::unordered_map> task_lease_table_; - std::unordered_set subscribed_tasks_; - std::unordered_map> - 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()), - reconstruction_timeout_ms_(50), - reconstruction_policy_(std::make_shared( - 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 &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 &handler, - std::shared_ptr 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 &handler) { - timer_canceled_ = false; - auto timer_period = boost::posix_time::milliseconds(period_ms); - auto timer = std::make_shared(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(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 mock_gcs_; - MockTaskInfoAccessor *task_accessor_; - MockNodeInfoAccessor *node_accessor_; - gcs::SubscribeCallback> subscribe_callback_; - std::shared_ptr mock_object_directory_; - uint64_t reconstruction_timeout_ms_; - std::shared_ptr reconstruction_policy_; - bool timer_canceled_; - std::unordered_map 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()); - // 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(); - 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(); - 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(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(); - 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(); -} diff --git a/src/ray/raylet/scheduling/cluster_task_manager.cc b/src/ray/raylet/scheduling/cluster_task_manager.cc index fb044d12c6b7..d0b2e62189c4 100644 --- a/src/ray/raylet/scheduling/cluster_task_manager.cc +++ b/src/ray/raylet/scheduling/cluster_task_manager.cc @@ -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 &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 diff --git a/src/ray/raylet/scheduling/cluster_task_manager.h b/src/ray/raylet/scheduling/cluster_task_manager.h index 7e9814f2ad21..2b05fb96fbe8 100644 --- a/src/ray/raylet/scheduling/cluster_task_manager.h +++ b/src/ray/raylet/scheduling/cluster_task_manager.h @@ -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 &waiting_task_ids) override; - /// The helper to dump the debug state of the cluster task manater. std::string DebugStr() const override; diff --git a/src/ray/raylet/scheduling/cluster_task_manager_interface.h b/src/ray/raylet/scheduling/cluster_task_manager_interface.h index 950741e54be3..eae8b08ec4b6 100644 --- a/src/ray/raylet/scheduling/cluster_task_manager_interface.h +++ b/src/ray/raylet/scheduling/cluster_task_manager_interface.h @@ -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 &waiting_task_ids) = 0; - /// The helper to dump the debug state of the cluster task manater. /// /// As the NodeManager inherites from ClusterTaskManager and the diff --git a/src/ray/raylet/scheduling/old_cluster_resource_scheduler.cc b/src/ray/raylet/scheduling/old_cluster_resource_scheduler.cc deleted file mode 100644 index 9801e57c6311..000000000000 --- a/src/ray/raylet/scheduling/old_cluster_resource_scheduler.cc +++ /dev/null @@ -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 &cluster_resource_map, - std::shared_ptr 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 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 diff --git a/src/ray/raylet/scheduling/old_cluster_resource_scheduler.h b/src/ray/raylet/scheduling/old_cluster_resource_scheduler.h deleted file mode 100644 index 927442c6c078..000000000000 --- a/src/ray/raylet/scheduling/old_cluster_resource_scheduler.h +++ /dev/null @@ -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 &cluster_resource_map, - const std::shared_ptr 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 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 &cluster_resource_map_; - const std::shared_ptr last_heartbeat_resources_; -}; -} // namespace ray diff --git a/src/ray/raylet/scheduling_policy.cc b/src/ray/raylet/scheduling_policy.cc deleted file mode 100644 index d4d2ffc11f84..000000000000 --- a/src/ray/raylet/scheduling_policy.cc +++ /dev/null @@ -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 -#include -#include - -#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 SchedulingPolicy::Schedule( - std::unordered_map &cluster_resources, - const NodeID &local_node_id) { - // The policy decision to be returned. - std::unordered_map 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 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 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 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 SchedulingPolicy::SpillOverInfeasibleTasks( - SchedulingResources &node_resources) const { - // The policy decision to be returned. - std::vector 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 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 diff --git a/src/ray/raylet/scheduling_policy.h b/src/ray/raylet/scheduling_policy.h deleted file mode 100644 index f719a1f6ef92..000000000000 --- a/src/ray/raylet/scheduling_policy.h +++ /dev/null @@ -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 -#include - -#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 Schedule( - std::unordered_map &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 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 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 diff --git a/src/ray/raylet/scheduling_queue.cc b/src/ray/raylet/scheduling_queue.cc deleted file mode 100644 index 5bdd0528d238..000000000000 --- a/src/ray/raylet/scheduling_queue.cc +++ /dev/null @@ -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 - -#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(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(task_state)]; -} - -// Helper function to get tasks for a job from a given state. -template -inline void GetTasksForJobFromQueue(const TaskQueue &queue, const ray::JobID &job_id, - std::unordered_set &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 -inline void GetActorTasksFromQueue(const TaskQueue &queue, const ray::ActorID &actor_id, - std::unordered_set &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 *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 &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 &TaskQueue::GetResourceLoadByShape() - const { - return resource_load_by_shape_; -} - -const std::unordered_map &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 *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> - &ReadyQueue::GetTasksByClass() const { - return tasks_by_class_; -} - -const std::list &SchedulingQueue::GetTasks(TaskState task_state) const { - const auto &queue = GetTaskQueue(task_state); - return queue->GetTasks(); -} - -const std::unordered_map> - &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(TaskState::INFEASIBLE)]->GetTotalResourceLoad()); - return load; -} - -rpc::ResourceLoad SchedulingQueue::GetResourceLoadByShape( - int64_t max_shapes, bool report_worker_backlog) const { - std::unordered_map load; - auto infeasible_queue_load = - task_queues_[static_cast(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({{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 &SchedulingQueue::GetBlockedTaskIds() const { - return blocked_task_ids_; -} - -void SchedulingQueue::FilterStateFromQueue(std::unordered_set &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 &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::type>(filter_state); - } -} - -const std::shared_ptr &SchedulingQueue::GetTaskQueue( - TaskState task_state) const { - RAY_CHECK(task_state < TaskState::kNumTaskQueues) - << static_cast(task_state) << "Task state " << static_cast(task_state) - << " does not correspond to a task queue"; - return task_queues_[static_cast(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 &task_ids, - std::vector *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 SchedulingQueue::RemoveTasks(std::unordered_set &task_ids) { - // List of removed tasks to be returned. - std::vector 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 removed_tasks; - std::unordered_set 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 &task_ids, TaskState src_state, - TaskState dst_state) { - std::vector 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::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::type>(dst_state); - } -} - -void SchedulingQueue::QueueTasks(const std::vector &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 SchedulingQueue::GetTaskIdsForJob(const JobID &job_id) const { - std::unordered_set 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 &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(ray::raylet::TaskState::kNumTaskQueues); i++) { - TaskState task_state = static_cast(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 diff --git a/src/ray/raylet/scheduling_queue.h b/src/ray/raylet/scheduling_queue.h deleted file mode 100644 index 062b411b0931..000000000000 --- a/src/ray/raylet/scheduling_queue.h +++ /dev/null @@ -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 -#include -#include -#include -#include - -#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 *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 &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 &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 &GetRequestBacklogByShape() const; - - protected: - /// A list of tasks. - std::list task_list_; - /// A hash to speed up looking up a task. - std::unordered_map::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 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 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 *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> &GetTasksByClass() const; - - private: - /// Index from task description to tasks queued of that type. - std::unordered_map> 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()) { - for (const auto &task_state : { - TaskState::PLACEABLE, - TaskState::WAITING, - TaskState::READY, - TaskState::RUNNING, - TaskState::INFEASIBLE, - }) { - if (task_state == TaskState::READY) { - task_queues_[static_cast(task_state)] = ready_queue_; - } else { - task_queues_[static_cast(task_state)] = std::make_shared(); - } - } - } - - /// 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 &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> &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 &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 &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 RemoveTasks(std::unordered_set &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 &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 &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 &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 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 &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 &task_ids, - std::vector *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 &task_ids, - TaskState task_state) const; - - // A pointer to the ready queue. - const std::shared_ptr ready_queue_; - /// Track the breakdown of tasks by class in the RUNNING queue. - std::unordered_map num_running_tasks_; - // A pointer to the task queues. These contain all tasks that have a task - // state < TaskState::kNumTaskQueues. - std::array, static_cast(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 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 driver_task_ids_; -}; - -} // namespace raylet - -} // namespace ray