From 0e18ca7b4c06683f8fdfdc6c6a327ec97a9f2bd0 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 16 May 2018 16:16:42 -0700 Subject: [PATCH 01/42] Use pubsub instead of timeout. --- src/ray/object_manager/object_directory.cc | 145 +++++++++++------- src/ray/object_manager/object_directory.h | 69 +++++---- src/ray/object_manager/object_manager.cc | 35 ++--- src/ray/object_manager/object_manager.h | 16 +- .../test/object_manager_stress_test.cc | 4 +- .../test/object_manager_test.cc | 4 +- src/ray/raylet/node_manager.cc | 2 + 7 files changed, 149 insertions(+), 126 deletions(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index 990727e403ef..56093e5948c1 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -2,9 +2,34 @@ namespace ray { -ObjectDirectory::ObjectDirectory(std::shared_ptr gcs_client) { +ObjectDirectory::ObjectDirectory(std::shared_ptr &gcs_client) { gcs_client_ = gcs_client; -}; +} + +void ObjectDirectory::RegisterBackend() { + auto object_notification_callback = [this](gcs::AsyncGcsClient *client, + const ObjectID &object_id, + const std::vector data) { + // Objects are added to this map in SubscribeObjectLocations. + auto entry = listeners_.find(object_id); + // Do nothing for objects we are not listening for. + if (entry == listeners_.end()) { + return; + } + // Obtain reported client ids. + std::vector client_ids; + for (auto item : data) { + if (!item.is_eviction) { + ClientID client_id = ClientID::from_binary(item.manager); + client_ids.push_back(client_id); + } + } + entry->second.locations_found_callback(client_ids, object_id); + }; + gcs_client_->object_table().Subscribe(UniqueID::nil(), + gcs_client_->client_table().GetLocalClientId(), + object_notification_callback, nullptr); +} ray::Status ObjectDirectory::ReportObjectAdded(const ObjectID &object_id, const ClientID &client_id, @@ -21,13 +46,13 @@ ray::Status ObjectDirectory::ReportObjectAdded(const ObjectID &object_id, // Do nothing. }); return status; -}; +} ray::Status ObjectDirectory::ReportObjectRemoved(const ObjectID &object_id, const ClientID &client_id) { // TODO(hme): Need corresponding remove method in GCS. return ray::Status::NotImplemented("ObjectTable.Remove is not implemented"); -}; +} ray::Status ObjectDirectory::GetInformation(const ClientID &client_id, const InfoSuccessCallback &success_callback, @@ -42,63 +67,77 @@ ray::Status ObjectDirectory::GetInformation(const ClientID &client_id, success_callback(info); } return ray::Status::OK(); -}; +} -ray::Status ObjectDirectory::GetLocations(const ObjectID &object_id, - const OnLocationsSuccess &success_callback, - const OnLocationsFailure &fail_callback) { - ray::Status status_code = ray::Status::OK(); - if (existing_requests_.count(object_id) == 0) { - existing_requests_[object_id] = ODCallbacks({success_callback, fail_callback}); - status_code = ExecuteGetLocations(object_id); - } else { - // Do nothing. A request is in progress. +ray::Status ObjectDirectory::SubscribeObjectLocations(const ObjectID &object_id, + const OnLocationsFound &callback) { + ray::Status status = ray::Status::OK(); + if (listeners_.find(object_id) != listeners_.end()) { + RAY_LOG(ERROR) << "Duplicate calls to SubscribeObjectLocations for " << object_id; + return ray::Status::Invalid("Cannot do things."); } - return status_code; -}; - -ray::Status ObjectDirectory::ExecuteGetLocations(const ObjectID &object_id) { - JobID job_id = JobID::nil(); - // Note: Lookup must be synchronous for thread-safe access. - // For now, this is only accessed by the main thread. - ray::Status status = gcs_client_->object_table().Lookup( - job_id, object_id, [this](gcs::AsyncGcsClient *client, const ObjectID &object_id, - const std::vector &data) { - GetLocationsComplete(object_id, data); + listeners_.emplace(object_id, LocationListenerState(callback)); + GetLocations( + object_id, + [this, callback](const std::vector &v, + const ray::ObjectID &object_id) { + if (listeners_.count(object_id) > 0) { + // Make sure we're still interested in this object's locations. + callback(v, object_id); + } + }, + [this](const ray::ObjectID &object_id) { + auto entry = listeners_.find(object_id); + if (entry == listeners_.end()) { + return; + } + entry->second.listening = true; + RAY_CHECK_OK(gcs_client_->object_table().RequestNotifications( + JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId())); }); return status; -}; +} -void ObjectDirectory::GetLocationsComplete( - const ObjectID &object_id, const std::vector &location_entries) { - auto request = existing_requests_.find(object_id); - // Do not invoke a callback if the request was cancelled. - if (request == existing_requests_.end()) { - return; - } - // Build the set of current locations based on the entries in the log. - std::unordered_set locations; - for (auto entry : location_entries) { - ClientID client_id = ClientID::from_binary(entry.manager); - if (!entry.is_eviction) { - locations.insert(client_id); - } else { - locations.erase(client_id); - } +ray::Status ObjectDirectory::UnsubscribeObjectLocations(const ObjectID &object_id) { + auto entry = listeners_.find(object_id); + if (entry == listeners_.end()) { + return ray::Status::OK(); } - // Invoke the callback. - std::vector locations_vector(locations.begin(), locations.end()); - if (locations_vector.empty()) { - request->second.fail_cb(object_id); - } else { - request->second.success_cb(locations_vector, object_id); + if (entry->second.listening) { + RAY_CHECK_OK(gcs_client_->object_table().CancelNotifications( + JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId())); } - existing_requests_.erase(request); + listeners_.erase(entry); + return ray::Status::OK(); } -ray::Status ObjectDirectory::Cancel(const ObjectID &object_id) { - existing_requests_.erase(object_id); - return ray::Status::OK(); -}; +ray::Status ObjectDirectory::GetLocations(const ObjectID &object_id, + const OnLocationsFound &success_callback, + const OnLocationsFailure &fail_callback) { + JobID job_id = JobID::nil(); + ray::Status status = gcs_client_->object_table().Lookup( + job_id, object_id, [this, success_callback, fail_callback]( + gcs::AsyncGcsClient *client, const ObjectID &object_id, + const std::vector &location_entries) { + // Build the set of current locations based on the entries in the log. + std::unordered_set locations; + for (auto entry : location_entries) { + ClientID client_id = ClientID::from_binary(entry.manager); + if (!entry.is_eviction) { + locations.insert(client_id); + } else { + locations.erase(client_id); + } + } + // Invoke the callback. + std::vector locations_vector(locations.begin(), locations.end()); + if (locations_vector.empty()) { + fail_callback(object_id); + } else { + success_callback(locations_vector, object_id); + } + }); + return status; +} } // namespace ray diff --git a/src/ray/object_manager/object_directory.h b/src/ray/object_manager/object_directory.h index 7eca8c550526..802dbff3b995 100644 --- a/src/ray/object_manager/object_directory.h +++ b/src/ray/object_manager/object_directory.h @@ -33,6 +33,8 @@ class ObjectDirectoryInterface { using InfoSuccessCallback = std::function; using InfoFailureCallback = std::function; + virtual void RegisterBackend() = 0; + /// This is used to establish object manager client connections. /// /// \param client_id The client for which information is required. @@ -43,27 +45,25 @@ class ObjectDirectoryInterface { const InfoSuccessCallback &success_cb, const InfoFailureCallback &fail_cb) = 0; - // Callbacks for GetLocations. - using OnLocationsSuccess = std::function &v, - const ray::ObjectID &object_id)>; - using OnLocationsFailure = std::function; + /// Callback for object location notifications. + using OnLocationsFound = std::function &v, + const ray::ObjectID &object_id)>; - /// Asynchronously obtain the locations of an object by ObjectID. - /// This is used to handle object pulls. + /// Subscribe to be notified of locations (ClientID) of the given object. + /// The callback will be invoked whenever locations are obtained for the + /// specified object. /// /// \param object_id The required object's ObjectID. - /// \param success_cb Invoked upon success with list of remote connection info. - /// \param fail_cb Invoked upon failure with ray status and object id. - /// \return Status of whether this asynchronous request succeeded. - virtual ray::Status GetLocations(const ObjectID &object_id, - const OnLocationsSuccess &success_cb, - const OnLocationsFailure &fail_cb) = 0; + /// \param success_cb Invoked with non-empty list of client ids and object_id. + /// \return Status of whether subscription succeeded. + virtual ray::Status SubscribeObjectLocations(const ObjectID &object_id, + const OnLocationsFound &callback) = 0; - /// Cancels the invocation of the callback associated with callback_id. + /// Unsubscribe to object location notifications. /// - /// \param object_id The object id invoked with GetLocations. - /// \return Status of whether this method succeeded. - virtual ray::Status Cancel(const ObjectID &object_id) = 0; + /// \param object_id The object id invoked with Subscribe. + /// \return + virtual ray::Status UnsubscribeObjectLocations(const ObjectID &object_id) = 0; /// Report objects added to this node's store to the object directory. /// @@ -90,40 +90,43 @@ class ObjectDirectory : public ObjectDirectoryInterface { ObjectDirectory() = default; ~ObjectDirectory() override = default; + void RegisterBackend() override; + ray::Status GetInformation(const ClientID &client_id, const InfoSuccessCallback &success_callback, const InfoFailureCallback &fail_callback) override; - ray::Status GetLocations(const ObjectID &object_id, - const OnLocationsSuccess &success_callback, - const OnLocationsFailure &fail_callback) override; - ray::Status Cancel(const ObjectID &object_id) override; + + ray::Status SubscribeObjectLocations(const ObjectID &object_id, + const OnLocationsFound &callback) override; + ray::Status UnsubscribeObjectLocations(const ObjectID &object_id) override; + ray::Status ReportObjectAdded(const ObjectID &object_id, const ClientID &client_id, const ObjectInfoT &object_info) override; ray::Status ReportObjectRemoved(const ObjectID &object_id, const ClientID &client_id) override; /// Ray only (not part of the OD interface). - ObjectDirectory(std::shared_ptr gcs_client); + ObjectDirectory(std::shared_ptr &gcs_client); /// ObjectDirectory should not be copied. RAY_DISALLOW_COPY_AND_ASSIGN(ObjectDirectory); private: + using OnLocationsFailure = std::function; + /// Callbacks associated with a call to GetLocations. - // TODO(hme): I think these can be removed. - struct ODCallbacks { - OnLocationsSuccess success_cb; - OnLocationsFailure fail_cb; + struct LocationListenerState { + LocationListenerState(const OnLocationsFound &locations_found_callback) + : locations_found_callback(locations_found_callback), listening(false) {} + OnLocationsFound locations_found_callback; + bool listening; }; - /// GetLocations registers a request for locations. - /// This function actually carries out that request. - ray::Status ExecuteGetLocations(const ObjectID &object_id); - /// Invoked when call to ExecuteGetLocations completes. - void GetLocationsComplete(const ObjectID &object_id, - const std::vector &location_entries); + ray::Status GetLocations(const ObjectID &object_id, + const OnLocationsFound &success_callback, + const OnLocationsFailure &fail_callback); - /// Maintain map of in-flight GetLocation requests. - std::unordered_map existing_requests_; + /// Info about subscribers to object locations. + std::unordered_map listeners_; /// Reference to the gcs client. std::shared_ptr gcs_client_; }; diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 42ef8164aee7..e4295e9add89 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -57,6 +57,8 @@ ObjectManager::ObjectManager(asio::io_service &main_service, ObjectManager::~ObjectManager() { StopIOService(); } +void ObjectManager::RegisterGcs() { object_directory_->RegisterBackend(); } + void ObjectManager::StartIOService() { for (int i = 0; i < config_.max_sends; ++i) { send_threads_.emplace_back(std::thread(&ObjectManager::RunSendService, this)); @@ -109,23 +111,13 @@ ray::Status ObjectManager::Pull(const ObjectID &object_id) { return PullGetLocations(object_id); } -void ObjectManager::SchedulePull(const ObjectID &object_id, int wait_ms) { - pull_requests_[object_id] = std::make_shared( - *main_service_, boost::posix_time::milliseconds(wait_ms)); - pull_requests_[object_id]->async_wait( - [this, object_id](const boost::system::error_code &error_code) { - pull_requests_.erase(object_id); - RAY_CHECK_OK(PullGetLocations(object_id)); - }); -} - ray::Status ObjectManager::PullGetLocations(const ObjectID &object_id) { - ray::Status status_code = object_directory_->GetLocations( + ray::Status status_code = object_directory_->SubscribeObjectLocations( object_id, [this](const std::vector &client_ids, const ObjectID &object_id) { - return GetLocationsSuccess(client_ids, object_id); - }, - [this](const ObjectID &object_id) { return GetLocationsFailed(object_id); }); + object_directory_->UnsubscribeObjectLocations(object_id); + GetLocationsSuccess(client_ids, object_id); + }); return status_code; } @@ -136,10 +128,6 @@ void ObjectManager::GetLocationsSuccess(const std::vector &client ray::Status status_code = Pull(object_id, client_id); } -void ObjectManager::GetLocationsFailed(const ObjectID &object_id) { - SchedulePull(object_id, config_.pull_timeout_ms); -} - ray::Status ObjectManager::Pull(const ObjectID &object_id, const ClientID &client_id) { return PullEstablishConnection(object_id, client_id); }; @@ -158,10 +146,7 @@ ray::Status ObjectManager::PullEstablishConnection(const ObjectID &object_id, status = connection_pool_.GetSender(ConnectionPool::ConnectionType::MESSAGE, client_id, &conn); if (!status.ok()) { - // TODO(hme): Keep track of retries, - // and only retry on object not local - // for now. - SchedulePull(object_id, config_.pull_timeout_ms); + // TODO (hme): Accept changes in error cleanup PR. return status; } if (conn == nullptr) { @@ -175,7 +160,8 @@ ray::Status ObjectManager::PullEstablishConnection(const ObjectID &object_id, RAY_CHECK_OK(PullSendRequest(object_id, async_conn)); }, [this, object_id](const Status &status) { - SchedulePull(object_id, config_.pull_timeout_ms); + RAY_LOG(ERROR) << "Failed to establish connection with remote object manager."; + RAY_CHECK_OK(status); }); } else { RAY_CHECK_OK(PullSendRequest(object_id, conn)); @@ -299,8 +285,7 @@ ray::Status ObjectManager::SendObjectData(const ObjectID &object_id, } ray::Status ObjectManager::Cancel(const ObjectID &object_id) { - // TODO(hme): Account for pull timers. - ray::Status status = object_directory_->Cancel(object_id); + ray::Status status = object_directory_->UnsubscribeObjectLocations(object_id); return ray::Status::OK(); } diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index d34d50762f24..ae2144925877 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -68,6 +68,9 @@ class ObjectManager { ~ObjectManager(); + /// Register GCS-related functionality. + void RegisterGcs(); + /// Subscribe to notifications of objects added to local store. /// Upon subscribing, the callback will be invoked for all objects that /// @@ -178,10 +181,6 @@ class ObjectManager { /// Connection pool for reusing outgoing connections to remote object managers. ConnectionPool connection_pool_; - /// Timeout for failed pull requests. - std::unordered_map> - pull_requests_; - /// Cache of locally available objects. std::unordered_map local_objects_; @@ -197,11 +196,6 @@ class ObjectManager { /// Register object remove with directory. void NotifyDirectoryObjectDeleted(const ObjectID &object_id); - /// Wait wait_ms milliseconds before triggering a pull request for object_id. - /// This is invoked when a pull fails. Only point of failure currently considered - /// is GetLocationsFailed. - void SchedulePull(const ObjectID &object_id, int wait_ms); - /// Part of an asynchronous sequence of Pull methods. /// Gets the location of an object before invoking PullEstablishConnection. /// Guaranteed to execute on main_service_ thread. @@ -219,10 +213,6 @@ class ObjectManager { void GetLocationsSuccess(const std::vector &client_ids, const ray::ObjectID &object_id); - /// Private callback implementation for failure on get location. Called from - /// ObjectDirectory. - void GetLocationsFailed(const ObjectID &object_id); - /// Synchronously send a pull request via remote object manager connection. /// Executes on main_service_ thread. ray::Status PullSendRequest(const ObjectID &object_id, diff --git a/src/ray/object_manager/test/object_manager_stress_test.cc b/src/ray/object_manager/test/object_manager_stress_test.cc index 827d19818979..c6078cf471ad 100644 --- a/src/ray/object_manager/test/object_manager_stress_test.cc +++ b/src/ray/object_manager/test/object_manager_stress_test.cc @@ -54,7 +54,9 @@ class MockServer { client_info.node_manager_address = ip; client_info.node_manager_port = object_manager_port; client_info.object_manager_port = object_manager_port; - return gcs_client_->client_table().Connect(client_info); + ray::Status status = gcs_client_->client_table().Connect(client_info); + object_manager_.RegisterGcs(); + return status; } void DoAcceptObjectManager() { diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index faef6850a465..c9ec618dff3f 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -45,7 +45,9 @@ class MockServer { client_info.node_manager_address = ip; client_info.node_manager_port = object_manager_port; client_info.object_manager_port = object_manager_port; - return gcs_client_->client_table().Connect(client_info); + ray::Status status = gcs_client_->client_table().Connect(client_info); + object_manager_.RegisterGcs(); + return status; } void DoAcceptObjectManager() { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 3fc99a2a6080..2edebe67e0dd 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -91,6 +91,8 @@ NodeManager::NodeManager(boost::asio::io_service &io_service, } ray::Status NodeManager::RegisterGcs() { + object_manager_.RegisterGcs(); + // Subscribe to task entry commits in the GCS. These notifications are // forwarded to the lineage cache, which requests notifications about tasks // that were executed remotely. From fc2572c2635f324b9a0e5682d11492ea3025d66a Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 16 May 2018 17:01:12 -0700 Subject: [PATCH 02/42] Correct status message. --- src/ray/object_manager/object_directory.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index 56093e5948c1..b8538b30f664 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -74,7 +74,7 @@ ray::Status ObjectDirectory::SubscribeObjectLocations(const ObjectID &object_id, ray::Status status = ray::Status::OK(); if (listeners_.find(object_id) != listeners_.end()) { RAY_LOG(ERROR) << "Duplicate calls to SubscribeObjectLocations for " << object_id; - return ray::Status::Invalid("Cannot do things."); + return ray::Status::Invalid("Unable to subscribe to the same object twice."); } listeners_.emplace(object_id, LocationListenerState(callback)); GetLocations( From 2e8af607b2d8d58ff36755702c0a09c0a9e58c93 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Thu, 17 May 2018 11:01:35 -0700 Subject: [PATCH 03/42] eric's feedback! --- src/ray/object_manager/object_directory.cc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index b8538b30f664..d3b56f1d2db0 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -9,7 +9,7 @@ ObjectDirectory::ObjectDirectory(std::shared_ptr &gcs_clien void ObjectDirectory::RegisterBackend() { auto object_notification_callback = [this](gcs::AsyncGcsClient *client, const ObjectID &object_id, - const std::vector data) { + const std::vector &data) { // Objects are added to this map in SubscribeObjectLocations. auto entry = listeners_.find(object_id); // Do nothing for objects we are not listening for. @@ -18,7 +18,7 @@ void ObjectDirectory::RegisterBackend() { } // Obtain reported client ids. std::vector client_ids; - for (auto item : data) { + for (auto &item : data) { if (!item.is_eviction) { ClientID client_id = ClientID::from_binary(item.manager); client_ids.push_back(client_id); @@ -121,7 +121,7 @@ ray::Status ObjectDirectory::GetLocations(const ObjectID &object_id, const std::vector &location_entries) { // Build the set of current locations based on the entries in the log. std::unordered_set locations; - for (auto entry : location_entries) { + for (auto &entry : location_entries) { ClientID client_id = ClientID::from_binary(entry.manager); if (!entry.is_eviction) { locations.insert(client_id); From b57a548bdd13c523cd7dd8bb35b646444c2c1a93 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Thu, 17 May 2018 15:44:26 -0700 Subject: [PATCH 04/42] Changes from Stephanie's review. --- src/ray/object_manager/object_directory.cc | 76 +++++----------------- src/ray/object_manager/object_directory.h | 12 ++-- src/ray/object_manager/object_manager.cc | 6 +- src/ray/object_manager/object_manager.h | 6 -- 4 files changed, 20 insertions(+), 80 deletions(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index d3b56f1d2db0..ac46fd538749 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -16,15 +16,18 @@ void ObjectDirectory::RegisterBackend() { if (entry == listeners_.end()) { return; } - // Obtain reported client ids. - std::vector client_ids; - for (auto &item : data) { - if (!item.is_eviction) { - ClientID client_id = ClientID::from_binary(item.manager); - client_ids.push_back(client_id); + // Update entries for this object. + auto client_id_set = entry->second.client_ids; + for (auto &object_table_data : data) { + ClientID client_id = ClientID::from_binary(object_table_data.manager); + if (!object_table_data.is_eviction) { + client_id_set.insert(client_id); + } else { + client_id_set.erase(client_id); } } - entry->second.locations_found_callback(client_ids, object_id); + std::vector client_id_vec(client_id_set.begin(), client_id_set.end()); + entry->second.locations_found_callback(client_id_vec, object_id); }; gcs_client_->object_table().Subscribe(UniqueID::nil(), gcs_client_->client_table().GetLocalClientId(), @@ -71,31 +74,13 @@ ray::Status ObjectDirectory::GetInformation(const ClientID &client_id, ray::Status ObjectDirectory::SubscribeObjectLocations(const ObjectID &object_id, const OnLocationsFound &callback) { - ray::Status status = ray::Status::OK(); if (listeners_.find(object_id) != listeners_.end()) { RAY_LOG(ERROR) << "Duplicate calls to SubscribeObjectLocations for " << object_id; - return ray::Status::Invalid("Unable to subscribe to the same object twice."); + return ray::Status::OK(); } listeners_.emplace(object_id, LocationListenerState(callback)); - GetLocations( - object_id, - [this, callback](const std::vector &v, - const ray::ObjectID &object_id) { - if (listeners_.count(object_id) > 0) { - // Make sure we're still interested in this object's locations. - callback(v, object_id); - } - }, - [this](const ray::ObjectID &object_id) { - auto entry = listeners_.find(object_id); - if (entry == listeners_.end()) { - return; - } - entry->second.listening = true; - RAY_CHECK_OK(gcs_client_->object_table().RequestNotifications( - JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId())); - }); - return status; + return gcs_client_->object_table().RequestNotifications( + JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId()); } ray::Status ObjectDirectory::UnsubscribeObjectLocations(const ObjectID &object_id) { @@ -103,40 +88,9 @@ ray::Status ObjectDirectory::UnsubscribeObjectLocations(const ObjectID &object_i if (entry == listeners_.end()) { return ray::Status::OK(); } - if (entry->second.listening) { - RAY_CHECK_OK(gcs_client_->object_table().CancelNotifications( - JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId())); - } + ray::Status status = gcs_client_->object_table().CancelNotifications( + JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId()); listeners_.erase(entry); - return ray::Status::OK(); -} - -ray::Status ObjectDirectory::GetLocations(const ObjectID &object_id, - const OnLocationsFound &success_callback, - const OnLocationsFailure &fail_callback) { - JobID job_id = JobID::nil(); - ray::Status status = gcs_client_->object_table().Lookup( - job_id, object_id, [this, success_callback, fail_callback]( - gcs::AsyncGcsClient *client, const ObjectID &object_id, - const std::vector &location_entries) { - // Build the set of current locations based on the entries in the log. - std::unordered_set locations; - for (auto &entry : location_entries) { - ClientID client_id = ClientID::from_binary(entry.manager); - if (!entry.is_eviction) { - locations.insert(client_id); - } else { - locations.erase(client_id); - } - } - // Invoke the callback. - std::vector locations_vector(locations.begin(), locations.end()); - if (locations_vector.empty()) { - fail_callback(object_id); - } else { - success_callback(locations_vector, object_id); - } - }); return status; } diff --git a/src/ray/object_manager/object_directory.h b/src/ray/object_manager/object_directory.h index 802dbff3b995..027f442391ae 100644 --- a/src/ray/object_manager/object_directory.h +++ b/src/ray/object_manager/object_directory.h @@ -111,20 +111,16 @@ class ObjectDirectory : public ObjectDirectoryInterface { RAY_DISALLOW_COPY_AND_ASSIGN(ObjectDirectory); private: - using OnLocationsFailure = std::function; - /// Callbacks associated with a call to GetLocations. struct LocationListenerState { LocationListenerState(const OnLocationsFound &locations_found_callback) - : locations_found_callback(locations_found_callback), listening(false) {} + : locations_found_callback(locations_found_callback) {} + /// The callback to invoke when object locations are found. OnLocationsFound locations_found_callback; - bool listening; + /// The current set of known locations of this object. + std::unordered_set client_ids; }; - ray::Status GetLocations(const ObjectID &object_id, - const OnLocationsFound &success_callback, - const OnLocationsFailure &fail_callback); - /// Info about subscribers to object locations. std::unordered_map listeners_; /// Reference to the gcs client. diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index e4295e9add89..47f1d0139e2f 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -108,10 +108,6 @@ ray::Status ObjectManager::SubscribeObjDeleted( } ray::Status ObjectManager::Pull(const ObjectID &object_id) { - return PullGetLocations(object_id); -} - -ray::Status ObjectManager::PullGetLocations(const ObjectID &object_id) { ray::Status status_code = object_directory_->SubscribeObjectLocations( object_id, [this](const std::vector &client_ids, const ObjectID &object_id) { @@ -286,7 +282,7 @@ ray::Status ObjectManager::SendObjectData(const ObjectID &object_id, ray::Status ObjectManager::Cancel(const ObjectID &object_id) { ray::Status status = object_directory_->UnsubscribeObjectLocations(object_id); - return ray::Status::OK(); + return status; } ray::Status ObjectManager::Wait(const std::vector &object_ids, diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index ae2144925877..e12d569d16cf 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -196,12 +196,6 @@ class ObjectManager { /// Register object remove with directory. void NotifyDirectoryObjectDeleted(const ObjectID &object_id); - /// Part of an asynchronous sequence of Pull methods. - /// Gets the location of an object before invoking PullEstablishConnection. - /// Guaranteed to execute on main_service_ thread. - /// Executes on main_service_ thread. - ray::Status PullGetLocations(const ObjectID &object_id); - /// Part of an asynchronous sequence of Pull methods. /// Uses an existing connection or creates a connection to ClientID. /// Executes on main_service_ thread. From a128698c8274bc22be69b32a322f40f86bdb8996 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 18 May 2018 10:21:51 -0700 Subject: [PATCH 05/42] object directory changes for ray.wait. --- src/ray/object_manager/object_directory.cc | 71 +++++++++++++++------- src/ray/object_manager/object_directory.h | 28 ++++++--- src/ray/object_manager/object_manager.cc | 6 +- 3 files changed, 70 insertions(+), 35 deletions(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index ac46fd538749..090616bdd361 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -7,27 +7,38 @@ ObjectDirectory::ObjectDirectory(std::shared_ptr &gcs_clien } void ObjectDirectory::RegisterBackend() { - auto object_notification_callback = [this](gcs::AsyncGcsClient *client, - const ObjectID &object_id, - const std::vector &data) { + auto object_notification_callback = [this]( + gcs::AsyncGcsClient *client, const ObjectID &object_id, + const std::vector &object_location_ids) { // Objects are added to this map in SubscribeObjectLocations. - auto entry = listeners_.find(object_id); + auto object_id_listener_pair = listeners_.find(object_id); // Do nothing for objects we are not listening for. - if (entry == listeners_.end()) { + if (object_id_listener_pair == listeners_.end()) { return; } // Update entries for this object. - auto client_id_set = entry->second.client_ids; - for (auto &object_table_data : data) { + auto &location_client_id_set = object_id_listener_pair->second.location_client_ids; + // object_location_ids has the history of locations of the object: + // client1.is_eviction = false + // client1.is_eviction = true + // client2.is_eviction = false + for (const auto &object_table_data : object_location_ids) { ClientID client_id = ClientID::from_binary(object_table_data.manager); if (!object_table_data.is_eviction) { - client_id_set.insert(client_id); + location_client_id_set.insert(client_id); } else { - client_id_set.erase(client_id); + location_client_id_set.erase(client_id); } } - std::vector client_id_vec(client_id_set.begin(), client_id_set.end()); - entry->second.locations_found_callback(client_id_vec, object_id); + std::vector client_id_vec(location_client_id_set.begin(), + location_client_id_set.end()); + // Copy the callbacks so that the callbacks can unsubscribe without interrupting + // looping over the callbacks. + auto callbacks = object_id_listener_pair->second.callbacks; + // Call all callbacks associated with the object id locations we have received. + for (const auto &callback_pair : callbacks) { + callback_pair.second(client_id_vec, object_id); + } }; gcs_client_->object_table().Subscribe(UniqueID::nil(), gcs_client_->client_table().GetLocalClientId(), @@ -72,25 +83,41 @@ ray::Status ObjectDirectory::GetInformation(const ClientID &client_id, return ray::Status::OK(); } -ray::Status ObjectDirectory::SubscribeObjectLocations(const ObjectID &object_id, +ray::Status ObjectDirectory::SubscribeObjectLocations(const std::string &callback_id, + const ObjectID &object_id, const OnLocationsFound &callback) { - if (listeners_.find(object_id) != listeners_.end()) { - RAY_LOG(ERROR) << "Duplicate calls to SubscribeObjectLocations for " << object_id; + ray::Status status = ray::Status::OK(); + if (listeners_.find(object_id) == listeners_.end()) { + listeners_.emplace(object_id, LocationListenerState()); + status = gcs_client_->object_table().RequestNotifications( + JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId()); + } + if (listeners_[object_id].callbacks.count(callback_id) > 0) { return ray::Status::OK(); } - listeners_.emplace(object_id, LocationListenerState(callback)); - return gcs_client_->object_table().RequestNotifications( - JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId()); + listeners_[object_id].callbacks.emplace(callback_id, callback); + // Immediately notify of found object locations. + if (!listeners_[object_id].location_client_ids.empty()) { + std::vector client_id_vec(listeners_[object_id].location_client_ids.begin(), + listeners_[object_id].location_client_ids.end()); + callback(client_id_vec, object_id); + } + return status; } -ray::Status ObjectDirectory::UnsubscribeObjectLocations(const ObjectID &object_id) { +ray::Status ObjectDirectory::UnsubscribeObjectLocations(const std::string &label, + const ObjectID &object_id) { + ray::Status status = ray::Status::OK(); auto entry = listeners_.find(object_id); if (entry == listeners_.end()) { - return ray::Status::OK(); + return status; + } + entry->second.callbacks.erase(label); + if (entry->second.callbacks.empty()) { + status = gcs_client_->object_table().CancelNotifications( + JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId()); + listeners_.erase(entry); } - ray::Status status = gcs_client_->object_table().CancelNotifications( - JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId()); - listeners_.erase(entry); return status; } diff --git a/src/ray/object_manager/object_directory.h b/src/ray/object_manager/object_directory.h index 027f442391ae..ddf4ff3a8117 100644 --- a/src/ray/object_manager/object_directory.h +++ b/src/ray/object_manager/object_directory.h @@ -46,24 +46,32 @@ class ObjectDirectoryInterface { const InfoFailureCallback &fail_cb) = 0; /// Callback for object location notifications. - using OnLocationsFound = std::function &v, + using OnLocationsFound = std::function &, const ray::ObjectID &object_id)>; /// Subscribe to be notified of locations (ClientID) of the given object. /// The callback will be invoked whenever locations are obtained for the /// specified object. /// + /// \param callback_id The label associated with this subscription. This is + /// tied to the callback method and is needed when UnsubscribeObjectLocations + /// is called. /// \param object_id The required object's ObjectID. /// \param success_cb Invoked with non-empty list of client ids and object_id. /// \return Status of whether subscription succeeded. - virtual ray::Status SubscribeObjectLocations(const ObjectID &object_id, + virtual ray::Status SubscribeObjectLocations(const std::string &callback_id, + const ObjectID &object_id, const OnLocationsFound &callback) = 0; /// Unsubscribe to object location notifications. /// + /// \param label The label associated with this subscription. This was given + /// at subscription time, and unsubscribes the corresponding callback from + /// further notifications about the given object's location. /// \param object_id The object id invoked with Subscribe. - /// \return - virtual ray::Status UnsubscribeObjectLocations(const ObjectID &object_id) = 0; + /// \return Status of unsubscribing from object location notifications. + virtual ray::Status UnsubscribeObjectLocations(const std::string &label, + const ObjectID &object_id) = 0; /// Report objects added to this node's store to the object directory. /// @@ -96,9 +104,11 @@ class ObjectDirectory : public ObjectDirectoryInterface { const InfoSuccessCallback &success_callback, const InfoFailureCallback &fail_callback) override; - ray::Status SubscribeObjectLocations(const ObjectID &object_id, + ray::Status SubscribeObjectLocations(const std::string &callback_id, + const ObjectID &object_id, const OnLocationsFound &callback) override; - ray::Status UnsubscribeObjectLocations(const ObjectID &object_id) override; + ray::Status UnsubscribeObjectLocations(const std::string &label, + const ObjectID &object_id) override; ray::Status ReportObjectAdded(const ObjectID &object_id, const ClientID &client_id, const ObjectInfoT &object_info) override; @@ -113,12 +123,10 @@ class ObjectDirectory : public ObjectDirectoryInterface { private: /// Callbacks associated with a call to GetLocations. struct LocationListenerState { - LocationListenerState(const OnLocationsFound &locations_found_callback) - : locations_found_callback(locations_found_callback) {} /// The callback to invoke when object locations are found. - OnLocationsFound locations_found_callback; + std::unordered_map callbacks; /// The current set of known locations of this object. - std::unordered_set client_ids; + std::unordered_set location_client_ids; }; /// Info about subscribers to object locations. diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 47f1d0139e2f..c9a6c00dfc7c 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -109,9 +109,9 @@ ray::Status ObjectManager::SubscribeObjDeleted( ray::Status ObjectManager::Pull(const ObjectID &object_id) { ray::Status status_code = object_directory_->SubscribeObjectLocations( - object_id, + "pull", object_id, [this](const std::vector &client_ids, const ObjectID &object_id) { - object_directory_->UnsubscribeObjectLocations(object_id); + object_directory_->UnsubscribeObjectLocations("pull", object_id); GetLocationsSuccess(client_ids, object_id); }); return status_code; @@ -281,7 +281,7 @@ ray::Status ObjectManager::SendObjectData(const ObjectID &object_id, } ray::Status ObjectManager::Cancel(const ObjectID &object_id) { - ray::Status status = object_directory_->UnsubscribeObjectLocations(object_id); + ray::Status status = object_directory_->UnsubscribeObjectLocations("pull", object_id); return status; } From f9a9e169b947a5299a3044ebafd6f2195210f2c6 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 18 May 2018 12:24:26 -0700 Subject: [PATCH 06/42] wait without testing or timeout=0. --- src/ray/object_manager/object_directory.cc | 6 +- src/ray/object_manager/object_directory.h | 17 ++-- src/ray/object_manager/object_manager.cc | 91 ++++++++++++++++++++-- src/ray/object_manager/object_manager.h | 41 ++++++++-- 4 files changed, 129 insertions(+), 26 deletions(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index 7c088ee6188f..758089190f04 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -93,7 +93,7 @@ ray::Status ObjectDirectory::GetInformation(const ClientID &client_id, return ray::Status::OK(); } -ray::Status ObjectDirectory::SubscribeObjectLocations(const std::string &callback_id, +ray::Status ObjectDirectory::SubscribeObjectLocations(const UniqueID &callback_id, const ObjectID &object_id, const OnLocationsFound &callback) { ray::Status status = ray::Status::OK(); @@ -115,14 +115,14 @@ ray::Status ObjectDirectory::SubscribeObjectLocations(const std::string &callbac return status; } -ray::Status ObjectDirectory::UnsubscribeObjectLocations(const std::string &label, +ray::Status ObjectDirectory::UnsubscribeObjectLocations(const UniqueID &callback_id, const ObjectID &object_id) { ray::Status status = ray::Status::OK(); auto entry = listeners_.find(object_id); if (entry == listeners_.end()) { return status; } - entry->second.callbacks.erase(label); + entry->second.callbacks.erase(callback_id); if (entry->second.callbacks.empty()) { status = gcs_client_->object_table().CancelNotifications( JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId()); diff --git a/src/ray/object_manager/object_directory.h b/src/ray/object_manager/object_directory.h index 9e8afbf086dd..31acfe32c8da 100644 --- a/src/ray/object_manager/object_directory.h +++ b/src/ray/object_manager/object_directory.h @@ -53,24 +53,23 @@ class ObjectDirectoryInterface { /// The callback will be invoked whenever locations are obtained for the /// specified object. /// - /// \param callback_id The label associated with this subscription. This is - /// tied to the callback method and is needed when UnsubscribeObjectLocations - /// is called. + /// \param callback_id The id associated with the specified callback. This is + /// needed when UnsubscribeObjectLocations is called. /// \param object_id The required object's ObjectID. /// \param success_cb Invoked with non-empty list of client ids and object_id. /// \return Status of whether subscription succeeded. - virtual ray::Status SubscribeObjectLocations(const std::string &callback_id, + virtual ray::Status SubscribeObjectLocations(const UniqueID &callback_id, const ObjectID &object_id, const OnLocationsFound &callback) = 0; /// Unsubscribe to object location notifications. /// - /// \param label The label associated with this subscription. This was given + /// \param callback_id The id associated with a callback. This was given /// at subscription time, and unsubscribes the corresponding callback from /// further notifications about the given object's location. /// \param object_id The object id invoked with Subscribe. /// \return Status of unsubscribing from object location notifications. - virtual ray::Status UnsubscribeObjectLocations(const std::string &label, + virtual ray::Status UnsubscribeObjectLocations(const UniqueID &callback_id, const ObjectID &object_id) = 0; /// Report objects added to this node's store to the object directory. @@ -104,10 +103,10 @@ class ObjectDirectory : public ObjectDirectoryInterface { const InfoSuccessCallback &success_callback, const InfoFailureCallback &fail_callback) override; - ray::Status SubscribeObjectLocations(const std::string &callback_id, + ray::Status SubscribeObjectLocations(const UniqueID &callback_id, const ObjectID &object_id, const OnLocationsFound &callback) override; - ray::Status UnsubscribeObjectLocations(const std::string &label, + ray::Status UnsubscribeObjectLocations(const UniqueID &callback_id, const ObjectID &object_id) override; ray::Status ReportObjectAdded(const ObjectID &object_id, const ClientID &client_id, @@ -124,7 +123,7 @@ class ObjectDirectory : public ObjectDirectoryInterface { /// Callbacks associated with a call to GetLocations. struct LocationListenerState { /// The callback to invoke when object locations are found. - std::unordered_map callbacks; + std::unordered_map callbacks; /// The current set of known locations of this object. std::unordered_set location_client_ids; }; diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 51bdbd4ebf89..5a481a8567ff 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -114,9 +114,10 @@ ray::Status ObjectManager::Pull(const ObjectID &object_id) { return ray::Status::OK(); } ray::Status status_code = object_directory_->SubscribeObjectLocations( - "pull", object_id, + object_directory_pull_callback_id_, object_id, [this](const std::vector &client_ids, const ObjectID &object_id) { - object_directory_->UnsubscribeObjectLocations("pull", object_id); + object_directory_->UnsubscribeObjectLocations(object_directory_pull_callback_id_, + object_id); GetLocationsSuccess(client_ids, object_id); }); return status_code; @@ -296,17 +297,95 @@ ray::Status ObjectManager::SendObjectData(const ObjectID &object_id, } ray::Status ObjectManager::Cancel(const ObjectID &object_id) { - ray::Status status = object_directory_->UnsubscribeObjectLocations("pull", object_id); + ray::Status status = object_directory_->UnsubscribeObjectLocations( + object_directory_pull_callback_id_, object_id); return status; } -ray::Status ObjectManager::Wait(const std::vector &object_ids, - uint64_t timeout_ms, int num_ready_objects, +ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t wait_ms, + uint64_t num_required_objects, bool wait_local, const WaitCallback &callback) { - // TODO: Implement wait. + UniqueID wait_id = UniqueID::from_random(); + + if (wait_ms < 0) { + return ray::Status::Invalid("Unable to wait negative wait time."); + } + + if (wait_local) { + return ray::Status::NotImplemented("Wait for local objects is not yet implemented."); + } + + if (num_required_objects == 0) { + num_required_objects = object_ids.size(); + } + + if (wait_ms == 0) { + // Need GetLocations. + } else { + // Initialize fields. + active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); + auto &wait_state = active_wait_requests_.find(wait_id)->second; + wait_state.num_required_objects = num_required_objects; + wait_state.start_time = boost::posix_time::second_clock::local_time(); + for (auto &oid : object_ids) { + if (local_objects_.count(oid) > 0) { + wait_state.found.insert(oid); + } else { + wait_state.remaining.insert(oid); + } + } + if (wait_state.found.size() >= wait_state.num_required_objects) { + // Requirements already satisfied. + WaitComplete(wait_id); + } else { + for (auto &oid : wait_state.remaining) { + // Subscribe to object notifications. + wait_state.subscribed_objects.insert(oid); + object_directory_->SubscribeObjectLocations( + wait_id, oid, [this, wait_id](const std::vector &client_ids, + const ObjectID &object_id) { + auto &wait_state = active_wait_requests_.find(wait_id)->second; + if (wait_state.remaining.count(object_id) != 0) { + wait_state.remaining.erase(object_id); + wait_state.found.insert(object_id); + } + wait_state.subscribed_objects.erase(object_id); + object_directory_->UnsubscribeObjectLocations(wait_id, object_id); + if (wait_state.found.size() >= wait_state.num_required_objects) { + WaitComplete(wait_id); + } + }); + } + // Set timeout. + wait_state.timeout_timer->async_wait( + [this, wait_id](const boost::system::error_code &error_code) { + if (error_code.value() != 0) { + return; + } + WaitComplete(wait_id); + }); + } + } return ray::Status::OK(); } +void ObjectManager::WaitComplete(const UniqueID &wait_id) { + auto &wait_state = active_wait_requests_.find(wait_id)->second; + // Unsubscribe to any objects that weren't found in the time allotted. + for (auto &object_id : wait_state.subscribed_objects) { + object_directory_->UnsubscribeObjectLocations(wait_id, object_id); + } + // Cancel the timer. This is okay even if the timer hasn't been started. + // The timer handler will be given a non-zero error code. The handler + // will do nothing on non-zero error codes. + wait_state.timeout_timer->cancel(); + // Invoke the wait handler. + int64_t time_taken = + (boost::posix_time::second_clock::local_time() - wait_state.start_time) + .total_milliseconds(); + wait_state.callback(time_taken, wait_state.found, wait_state.remaining); +} + std::shared_ptr ObjectManager::CreateSenderConnection( ConnectionPool::ConnectionType type, RemoteConnectionInfo info) { std::shared_ptr conn = diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index f5ea78a54e3c..d92f9d29e630 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -139,21 +139,22 @@ class ObjectManager : public ObjectManagerInterface { ray::Status Cancel(const ObjectID &object_id); /// Callback definition for wait. - using WaitCallback = std::function &)>; - /// Wait for timeout_ms before invoking the provided callback. - /// If num_ready_objects is satisfied before the timeout, then - /// invoke the callback. + using WaitCallback = std::function &found, + const std::unordered_set &remaining)>; + /// Wait until either num_required_objects are located or wait_ms has elapsed, + /// then invoke the provided callback. /// /// \param object_ids The object ids to wait on. /// \param timeout_ms The time in milliseconds to wait before invoking the callback. - /// \param num_ready_objects The minimum number of objects required before + /// \param num_required_objects The minimum number of objects required before /// invoking the callback. + /// \param wait_local Whether to wait until objects arrive to this node's store. /// \param callback Invoked when either timeout_ms is satisfied OR num_ready_objects /// is satisfied. /// \return Status of whether the wait successfully initiated. - ray::Status Wait(const std::vector &object_ids, uint64_t timeout_ms, - int num_ready_objects, const WaitCallback &callback); + ray::Status Wait(const std::vector &object_ids, int64_t wait_ms, + uint64_t num_required_objects, bool wait_local, const WaitCallback &callback); private: ClientID client_id_; @@ -191,6 +192,30 @@ class ObjectManager : public ObjectManagerInterface { /// Cache of locally available objects. std::unordered_map local_objects_; + UniqueID object_directory_pull_callback_id_ = UniqueID::from_random(); + + struct WaitState { + WaitState(asio::io_service &service, int64_t wait_ms, const WaitCallback &callback) : + wait_ms(wait_ms), + timeout_timer(std::make_shared(service, boost::posix_time::milliseconds(wait_ms))), + callback(callback) + {} + int64_t wait_ms; + std::shared_ptr timeout_timer; + WaitCallback callback; + std::unordered_set remaining; + std::unordered_set found; + std::unordered_set subscribed_objects; + uint64_t num_required_objects; + boost::posix_time::ptime start_time; + }; + + /// A set of active wait requests. + std::unordered_map active_wait_requests_; + + /// Completion handler for Wait. + void WaitComplete(const UniqueID &wait_id); + /// Handle starting, running, and stopping asio io_service. void StartIOService(); void RunSendService(); From 15b7f61e6f05799f17105e874183e564e557e7ca Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 18 May 2018 14:17:22 -0700 Subject: [PATCH 07/42] Handle remaining cases for wait. --- src/ray/object_manager/object_directory.cc | 24 +++++++++ src/ray/object_manager/object_directory.h | 11 ++++ src/ray/object_manager/object_manager.cc | 58 +++++++++++++++------- src/ray/object_manager/object_manager.h | 2 +- 4 files changed, 75 insertions(+), 20 deletions(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index 758089190f04..0f884e9790b9 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -131,4 +131,28 @@ ray::Status ObjectDirectory::UnsubscribeObjectLocations(const UniqueID &callback return status; } +ray::Status ObjectDirectory::LookupLocations(const ObjectID &object_id, + const OnLocationsFound &callback) { + JobID job_id = JobID::nil(); + ray::Status status = gcs_client_->object_table().Lookup( + job_id, object_id, [this, callback]( + gcs::AsyncGcsClient *client, const ObjectID &object_id, + const std::vector &location_entries) { + // Build the set of current locations based on the entries in the log. + std::unordered_set locations; + for (auto entry : location_entries) { + ClientID client_id = ClientID::from_binary(entry.manager); + if (!entry.is_eviction) { + locations.insert(client_id); + } else { + locations.erase(client_id); + } + } + // Invoke the callback. + std::vector locations_vector(locations.begin(), locations.end()); + callback(locations_vector, object_id); + }); + return status; +} + } // namespace ray diff --git a/src/ray/object_manager/object_directory.h b/src/ray/object_manager/object_directory.h index 31acfe32c8da..755ff7f82529 100644 --- a/src/ray/object_manager/object_directory.h +++ b/src/ray/object_manager/object_directory.h @@ -49,6 +49,14 @@ class ObjectDirectoryInterface { using OnLocationsFound = std::function &, const ray::ObjectID &object_id)>; + /// Lookup object locations. Callback may be invoked with empty list of client ids. + /// + /// \param object_id The object's ObjectID. + /// \param callback Invoked with (possibly empty) list of client ids and object_id. + /// \return Status of whether async call to backend succeeded. + virtual ray::Status LookupLocations(const ObjectID &object_id, + const OnLocationsFound &callback) = 0; + /// Subscribe to be notified of locations (ClientID) of the given object. /// The callback will be invoked whenever locations are obtained for the /// specified object. @@ -103,6 +111,9 @@ class ObjectDirectory : public ObjectDirectoryInterface { const InfoSuccessCallback &success_callback, const InfoFailureCallback &fail_callback) override; + ray::Status LookupLocations(const ObjectID &object_id, + const OnLocationsFound &callback) override; + ray::Status SubscribeObjectLocations(const UniqueID &callback_id, const ObjectID &object_id, const OnLocationsFound &callback) override; diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 5a481a8567ff..1ba79bfa378d 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -316,31 +316,49 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t } if (num_required_objects == 0) { + // TODO: Confirm this is the default value for waiting for all objects. num_required_objects = object_ids.size(); } - if (wait_ms == 0) { - // Need GetLocations. + // Initialize fields. + active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); + auto &wait_state = active_wait_requests_.find(wait_id)->second; + wait_state.num_required_objects = num_required_objects; + wait_state.start_time = boost::posix_time::second_clock::local_time(); + for (auto &oid : object_ids) { + if (local_objects_.count(oid) > 0) { + wait_state.found.insert(oid); + } else { + wait_state.remaining.insert(oid); + } + } + + if (wait_state.found.size() >= wait_state.num_required_objects) { + // Requirements already satisfied. + WaitComplete(wait_id); } else { - // Initialize fields. - active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); - auto &wait_state = active_wait_requests_.find(wait_id)->second; - wait_state.num_required_objects = num_required_objects; - wait_state.start_time = boost::posix_time::second_clock::local_time(); - for (auto &oid : object_ids) { - if (local_objects_.count(oid) > 0) { - wait_state.found.insert(oid); - } else { - wait_state.remaining.insert(oid); + if (wait_ms == 0) { + for (auto &oid : wait_state.remaining) { + // Subscribe to object notifications. + wait_state.requested_objects.insert(oid); + object_directory_->LookupLocations( + oid, [this, wait_id](const std::vector &client_ids, + const ObjectID &object_id) { + auto &wait_state = active_wait_requests_.find(wait_id)->second; + if (!client_ids.empty()) { + wait_state.remaining.erase(object_id); + wait_state.found.insert(object_id); + } + wait_state.requested_objects.erase(object_id); + if (wait_state.requested_objects.empty()) { + WaitComplete(wait_id); + } + }); } - } - if (wait_state.found.size() >= wait_state.num_required_objects) { - // Requirements already satisfied. - WaitComplete(wait_id); } else { for (auto &oid : wait_state.remaining) { // Subscribe to object notifications. - wait_state.subscribed_objects.insert(oid); + wait_state.requested_objects.insert(oid); object_directory_->SubscribeObjectLocations( wait_id, oid, [this, wait_id](const std::vector &client_ids, const ObjectID &object_id) { @@ -349,7 +367,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t wait_state.remaining.erase(object_id); wait_state.found.insert(object_id); } - wait_state.subscribed_objects.erase(object_id); + wait_state.requested_objects.erase(object_id); object_directory_->UnsubscribeObjectLocations(wait_id, object_id); if (wait_state.found.size() >= wait_state.num_required_objects) { WaitComplete(wait_id); @@ -371,8 +389,10 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t void ObjectManager::WaitComplete(const UniqueID &wait_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; + // If we complete with outstanding requests, then wait_ms should be non-zero. + RAY_CHECK(!(wait_state.requested_objects.size() > 0) || wait_state.wait_ms > 0); // Unsubscribe to any objects that weren't found in the time allotted. - for (auto &object_id : wait_state.subscribed_objects) { + for (auto &object_id : wait_state.requested_objects) { object_directory_->UnsubscribeObjectLocations(wait_id, object_id); } // Cancel the timer. This is okay even if the timer hasn't been started. diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index d92f9d29e630..9a308b07ac9f 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -205,7 +205,7 @@ class ObjectManager : public ObjectManagerInterface { WaitCallback callback; std::unordered_set remaining; std::unordered_set found; - std::unordered_set subscribed_objects; + std::unordered_set requested_objects; uint64_t num_required_objects; boost::posix_time::ptime start_time; }; From a22263b1960848505bcc64c2a1ef0e08f3dbd9b9 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 18 May 2018 14:20:31 -0700 Subject: [PATCH 08/42] linting --- src/ray/object_manager/object_directory.cc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index 0f884e9790b9..a3bd73ee18e4 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -135,9 +135,9 @@ ray::Status ObjectDirectory::LookupLocations(const ObjectID &object_id, const OnLocationsFound &callback) { JobID job_id = JobID::nil(); ray::Status status = gcs_client_->object_table().Lookup( - job_id, object_id, [this, callback]( - gcs::AsyncGcsClient *client, const ObjectID &object_id, - const std::vector &location_entries) { + job_id, object_id, + [this, callback](gcs::AsyncGcsClient *client, const ObjectID &object_id, + const std::vector &location_entries) { // Build the set of current locations based on the entries in the log. std::unordered_set locations; for (auto entry : location_entries) { From 8ab41f053201b22d6d2a5e53cc321ae030dee88e Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 18 May 2018 15:01:21 -0700 Subject: [PATCH 09/42] added tests of om wait imp. --- src/ray/object_manager/object_manager.cc | 5 +- .../test/object_manager_test.cc | 62 +++++++++++++++++++ 2 files changed, 66 insertions(+), 1 deletion(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 1ba79bfa378d..231d5c1404dc 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -375,6 +375,9 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t }); } // Set timeout. + // TODO (hme): If we need to just wait for all objects independent of time, + // determine what the value of wait_ms should be and skip this call. + // WaitComplete will be invoked when all objects have locations. wait_state.timeout_timer->async_wait( [this, wait_id](const boost::system::error_code &error_code) { if (error_code.value() != 0) { @@ -400,7 +403,7 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { // will do nothing on non-zero error codes. wait_state.timeout_timer->cancel(); // Invoke the wait handler. - int64_t time_taken = + int64_t time_taken = wait_state.wait_ms == 0? 0 : (boost::posix_time::second_clock::local_time() - wait_state.start_time) .total_milliseconds(); wait_state.callback(time_taken, wait_state.found, wait_state.remaining); diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index c9ec618dff3f..b9306e7d22e3 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -187,6 +187,8 @@ class TestObjectManager : public ::testing::Test { class TestObjectManagerCommands : public TestObjectManager { public: + + int current_wait_test = -1; int num_connected_clients = 0; uint num_expected_objects; ClientID client_id_1; @@ -233,6 +235,66 @@ class TestObjectManagerCommands : public TestObjectManager { void NotificationTestComplete(ObjectID object_id_1, ObjectID object_id_2) { ASSERT_EQ(object_id_1, object_id_2); + NextWaitTest(); + } + + void NextWaitTest() { + current_wait_test += 1; + switch(current_wait_test){ + case 0: { + TestWait(100, 5, 3, 0, false); + } break; + case 1: { + TestWait(100, 5, 3, 1000, false); + } break; + case 2: { + TestWait(100, 5, 6, 1000, true); + } break; + } + } + + void TestWait(int data_size, int num_objects, uint64_t required_objects, int wait_ms, bool include_nonexistent) { + std::vector object_ids; + for (int i = -1; ++i < num_objects;) { + ObjectID oid = WriteDataToClient(client2, data_size); + object_ids.push_back(oid); + } + if (include_nonexistent) { + num_objects += 1; + object_ids.push_back(ObjectID::from_random()); + } + server1->object_manager_.Wait( + object_ids, wait_ms, required_objects, false, + [this, num_objects, wait_ms, required_objects](int64_t elapsed, + const std::unordered_set &found, + const std::unordered_set &remaining) { + switch(current_wait_test){ + case 0: { + ASSERT_TRUE(elapsed == 0); + ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + NextWaitTest(); + } break; + case 1: { + RAY_LOG(DEBUG) << "elapsed " << elapsed; + RAY_LOG(DEBUG) << "found " << found.size(); + RAY_LOG(DEBUG) << "remaining " << remaining.size(); + ASSERT_TRUE(found.size() >= required_objects); + ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + NextWaitTest(); + } break; + case 2: { + RAY_LOG(DEBUG) << "elapsed " << elapsed; + RAY_LOG(DEBUG) << "found " << found.size(); + RAY_LOG(DEBUG) << "remaining " << remaining.size(); + ASSERT_TRUE(elapsed >= wait_ms); + ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + TestWaitComplete(); + } break; + } + }); + } + + void TestWaitComplete() { main_service.stop(); } From 98bacfa5004d308dc5057a2c4ca874b760bdda8a Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 18 May 2018 15:14:58 -0700 Subject: [PATCH 10/42] add local test. --- .../test/object_manager_test.cc | 26 +++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index b9306e7d22e3..e88f1c05035b 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -242,21 +242,29 @@ class TestObjectManagerCommands : public TestObjectManager { current_wait_test += 1; switch(current_wait_test){ case 0: { - TestWait(100, 5, 3, 0, false); + TestWait(100, 5, 3, 0, false, false); } break; case 1: { - TestWait(100, 5, 3, 1000, false); + TestWait(100, 5, 3, 1000, false, true); } break; case 2: { - TestWait(100, 5, 6, 1000, true); + TestWait(100, 5, 3, 1000, false, false); + } break; + case 3: { + TestWait(100, 5, 6, 1000, true, false); } break; } } - void TestWait(int data_size, int num_objects, uint64_t required_objects, int wait_ms, bool include_nonexistent) { + void TestWait(int data_size, int num_objects, uint64_t required_objects, int wait_ms, bool include_nonexistent, bool test_local) { std::vector object_ids; for (int i = -1; ++i < num_objects;) { - ObjectID oid = WriteDataToClient(client2, data_size); + ObjectID oid; + if (test_local) { + oid = WriteDataToClient(client1, data_size); + } else { + oid = WriteDataToClient(client2, data_size); + } object_ids.push_back(oid); } if (include_nonexistent) { @@ -283,6 +291,14 @@ class TestObjectManagerCommands : public TestObjectManager { NextWaitTest(); } break; case 2: { + RAY_LOG(DEBUG) << "elapsed " << elapsed; + RAY_LOG(DEBUG) << "found " << found.size(); + RAY_LOG(DEBUG) << "remaining " << remaining.size(); + ASSERT_TRUE(found.size() >= required_objects); + ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + NextWaitTest(); + } break; + case 3: { RAY_LOG(DEBUG) << "elapsed " << elapsed; RAY_LOG(DEBUG) << "found " << found.size(); RAY_LOG(DEBUG) << "remaining " << remaining.size(); From 53f33e0302fc9292828a431247bcdc70dd09a71c Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 23 May 2018 17:37:40 -0700 Subject: [PATCH 11/42] plasma imp. --- python/ray/worker.py | 6 +-- src/ray/raylet/format/node_manager.fbs | 51 +++++++++++++++++++++++++- src/ray/raylet/node_manager.cc | 39 ++++++++++++++++++++ 3 files changed, 91 insertions(+), 5 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 9b66141d5196..b663a42fc80b 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2097,7 +2097,8 @@ def connect(info, worker.plasma_client = plasma.connect(info["store_socket_name"], info["manager_socket_name"], 64) else: - worker.plasma_client = plasma.connect(info["store_socket_name"], "", + worker.plasma_client = plasma.connect(info["store_socket_name"], + info["raylet_socket_name"], 64) if not worker.use_raylet: @@ -2526,9 +2527,6 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): A list of object IDs that are ready and a list of the remaining object IDs. """ - if worker.use_raylet: - print("plasma_client.wait has not been implemented yet") - return if isinstance(object_ids, ray.ObjectID): raise TypeError( diff --git a/src/ray/raylet/format/node_manager.fbs b/src/ray/raylet/format/node_manager.fbs index 90ee05db254e..5c50d2c542fd 100644 --- a/src/ray/raylet/format/node_manager.fbs +++ b/src/ray/raylet/format/node_manager.fbs @@ -53,7 +53,11 @@ enum MessageType:int { // making their execution dependencies available. SetActorFrontier, // A node manager request to process a task forwarded from another node manager. - ForwardTaskRequest + ForwardTaskRequest, + + // Wait for objects to be ready either from local or remote Plasma stores. + PlasmaWaitRequest=20, + PlasmaWaitReply=21 } table TaskExecutionSpecification { @@ -117,3 +121,48 @@ table ReconstructObject { // Object ID of the object that needs to be reconstructed. object_id: string; } + +enum ObjectStatus:int { + // Object is stored in the local Plasma Store. + Local = 1, + // Object is stored on a remote Plasma store, and it is not stored on the + // local Plasma Store. + Remote, + // Object is not stored in the system. + Nonexistent, + // Object is currently transferred from a remote Plasma store to the local + // Plasma Store. + Transfer +} + +table ObjectRequestSpec { + // ID of the object. + object_id: string; + // The type of the object. This specifies whether we + // will be waiting for an object store in the local or + // global Plasma store. + type: int; +} + +table PlasmaWaitRequest { + // Array of object requests whose status we are asking for. + object_requests: [ObjectRequestSpec]; + // Number of objects expected to be returned, if available. + num_ready_objects: int; + // timeout + timeout: long; +} + +table ObjectReply { + // ID of the object. + object_id: string; + // The object status. This specifies where the object is stored. + status: int; +} + +table PlasmaWaitReply { + // Array of object requests being returned. + object_requests: [ObjectReply]; + // Number of objects expected to be returned, if available. + num_ready_objects: int; +} diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index d63adb773fea..a9734c7e2bf2 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -447,6 +447,45 @@ void NodeManager::ProcessClientMessage( worker->MarkUnblocked(); } } break; + case protocol::MessageType_PlasmaWaitRequest: { + // Read the data. + auto message = flatbuffers::GetRoot(message_data); + auto object_requests = message->object_requests(); + int num_requests = object_requests->size(); + std::vector object_ids; + // Does it make sense to specify local/non-local for each object? + std::vector wait_local; + for (int i = 0; i < num_requests; i++) { + ObjectID object_id = ObjectID::from_binary( + message->object_requests()->Get(i)->object_id()->str()); + object_ids.push_back(object_id); + wait_local.push_back(message->object_requests()->Get(i)->type() == 1); + } + int64_t wait_ms = message->timeout(); + uint64_t num_required_objects = static_cast(message->num_ready_objects()); + + object_manager_.Wait(object_ids, wait_ms, num_required_objects, false, + [this, client](int64_t time_taken, std::unordered_set found, std::unordered_set remaining){ + // Write the data. + flatbuffers::FlatBufferBuilder fbb; + std::vector> object_replies; + for (const auto &object_id : found) { + object_replies.push_back(protocol::CreateObjectReply( + fbb, fbb.CreateString(object_id.binary()), + protocol::ObjectStatus_Remote)); + } + for (const auto &object_id : remaining) { + object_replies.push_back(protocol::CreateObjectReply( + fbb, fbb.CreateString(object_id.binary()), + protocol::ObjectStatus_Nonexistent)); + } + auto send_message = protocol::CreatePlasmaWaitReply( + fbb, fbb.CreateVector(object_replies.data(), found.size()), static_cast(found.size())); + fbb.Finish(send_message); + client->WriteMessage(protocol::MessageType_PlasmaWaitReply, + fbb.GetSize(), fbb.GetBufferPointer()); + }); + } break; default: RAY_LOG(FATAL) << "Received unexpected message type " << message_type; From 8ef35f7092c8bed670985ddbd4e7ca5e2465fff5 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Tue, 29 May 2018 15:38:25 -0700 Subject: [PATCH 12/42] block worker as with pull. --- src/ray/raylet/node_manager.cc | 50 ++++++++++++++++++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index a9734c7e2bf2..e695e68fc599 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -448,6 +448,31 @@ void NodeManager::ProcessClientMessage( } } break; case protocol::MessageType_PlasmaWaitRequest: { + // Block the worker. + std::shared_ptr worker = worker_pool_.GetRegisteredWorker(client); + if (worker && !worker->IsBlocked()) { + RAY_CHECK(!worker->GetAssignedTaskId().is_nil()); + auto tasks = local_queues_.RemoveTasks({worker->GetAssignedTaskId()}); + const auto &task = tasks.front(); + // Get the CPU resources required by the running task. + const auto required_resources = task.GetTaskSpecification().GetRequiredResources(); + double required_cpus = 0; + RAY_CHECK(required_resources.GetResource(kCPU_ResourceLabel, &required_cpus)); + const std::unordered_map cpu_resources = { + {kCPU_ResourceLabel, required_cpus}}; + // Release the CPU resources. + RAY_CHECK( + cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release( + ResourceSet(cpu_resources))); + // Mark the task as blocked. + local_queues_.QueueBlockedTasks(tasks); + worker->MarkBlocked(); + + // Try to dispatch more tasks since the blocked worker released some + // resources. + DispatchTasks(); + } + // Read the data. auto message = flatbuffers::GetRoot(message_data); auto object_requests = message->object_requests(); @@ -466,6 +491,31 @@ void NodeManager::ProcessClientMessage( object_manager_.Wait(object_ids, wait_ms, num_required_objects, false, [this, client](int64_t time_taken, std::unordered_set found, std::unordered_set remaining){ + // Unblock the worker. + std::shared_ptr worker = worker_pool_.GetRegisteredWorker(client); + if (worker && !worker->IsBlocked()) { + RAY_CHECK(!worker->GetAssignedTaskId().is_nil()); + auto tasks = local_queues_.RemoveTasks({worker->GetAssignedTaskId()}); + const auto &task = tasks.front(); + // Get the CPU resources required by the running task. + const auto required_resources = task.GetTaskSpecification().GetRequiredResources(); + double required_cpus = 0; + RAY_CHECK(required_resources.GetResource(kCPU_ResourceLabel, &required_cpus)); + const std::unordered_map cpu_resources = { + {kCPU_ResourceLabel, required_cpus}}; + // Release the CPU resources. + RAY_CHECK( + cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release( + ResourceSet(cpu_resources))); + // Mark the task as blocked. + local_queues_.QueueBlockedTasks(tasks); + worker->MarkBlocked(); + + // Try to dispatch more tasks since the blocked worker released some + // resources. + DispatchTasks(); + } + // Write the data. flatbuffers::FlatBufferBuilder fbb; std::vector> object_replies; From 6e10f9e8c1e3de4ea7335d8cdc13f6c5f60e3773 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Tue, 29 May 2018 18:33:28 -0700 Subject: [PATCH 13/42] local scheduler implementation of wait. --- python/ray/worker.py | 32 +++--- .../lib/python/local_scheduler_extension.cc | 53 +++++++++ src/local_scheduler/local_scheduler_client.cc | 35 ++++++ src/local_scheduler/local_scheduler_client.h | 16 +++ src/ray/raylet/format/node_manager.fbs | 52 +++------ src/ray/raylet/node_manager.cc | 102 ++++-------------- 6 files changed, 155 insertions(+), 135 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index b663a42fc80b..2e26edd90986 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2097,8 +2097,7 @@ def connect(info, worker.plasma_client = plasma.connect(info["store_socket_name"], info["manager_socket_name"], 64) else: - worker.plasma_client = plasma.connect(info["store_socket_name"], - info["raylet_socket_name"], + worker.plasma_client = plasma.connect(info["store_socket_name"], "", 64) if not worker.use_raylet: @@ -2558,18 +2557,23 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): if len(object_ids) == 0: return [], [] - object_id_strs = [ - plasma.ObjectID(object_id.id()) for object_id in object_ids - ] - timeout = timeout if timeout is not None else 2**30 - ready_ids, remaining_ids = worker.plasma_client.wait( - object_id_strs, timeout, num_returns) - ready_ids = [ - ray.ObjectID(object_id.binary()) for object_id in ready_ids - ] - remaining_ids = [ - ray.ObjectID(object_id.binary()) for object_id in remaining_ids - ] + if worker.use_raylet: + timeout = timeout if timeout is not None else 2**30 + ready_ids, remaining_ids = worker.local_scheduler_client.wait( + object_ids, timeout, num_returns, False) + else: + object_id_strs = [ + plasma.ObjectID(object_id.id()) for object_id in object_ids + ] + timeout = timeout if timeout is not None else 2**30 + ready_ids, remaining_ids = worker.plasma_client.wait( + object_id_strs, timeout, num_returns) + ready_ids = [ + ray.ObjectID(object_id.binary()) for object_id in ready_ids + ] + remaining_ids = [ + ray.ObjectID(object_id.binary()) for object_id in remaining_ids + ] return ready_ids, remaining_ids diff --git a/src/local_scheduler/lib/python/local_scheduler_extension.cc b/src/local_scheduler/lib/python/local_scheduler_extension.cc index acd3613ea863..a547a76ccfac 100644 --- a/src/local_scheduler/lib/python/local_scheduler_extension.cc +++ b/src/local_scheduler/lib/python/local_scheduler_extension.cc @@ -179,6 +179,56 @@ static PyObject *PyLocalSchedulerClient_set_actor_frontier(PyObject *self, Py_RETURN_NONE; } +static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { + int timeout_ms; + int num_returns; + bool wait_local; + PyObject *py_object_ids; + if (!PyArg_ParseTuple(args, "Oiip", &py_object_ids, &timeout_ms, &num_returns, + &wait_local)) { + return NULL; + } + // Convert to milliseconds. + timeout_ms *= 1000; + PyObject *iter = PyObject_GetIter(py_object_ids); + if (!iter) { + return NULL; + } + // Convert object ids. + std::vector object_ids; + while (true) { + PyObject *next = PyIter_Next(iter); + ObjectID object_id; + if (!next) { + break; + } + if (!PyObjectToUniqueID(next, &object_id)) { + // Error parsing object id. + return NULL; + } + object_ids.push_back(object_id); + } + + // Invoke wait. + std::pair, std::vector> result = + local_scheduler_wait( + ((PyLocalSchedulerClient *) self)->local_scheduler_connection, + object_ids, num_returns, timeout_ms, wait_local); + + // Convert result to py object. + PyObject *py_found = + PyList_New((Py_ssize_t) result.first.size()); + for (uint i = 0; i < result.first.size(); ++i) { + PyList_SetItem(py_found, i, PyObjectID_make(result.first[i])); + } + PyObject *py_remaining = + PyList_New((Py_ssize_t) result.second.size()); + for (uint i = 0; i < result.second.size(); ++i) { + PyList_SetItem(py_remaining, i, PyObjectID_make(result.second[i])); + } + return Py_BuildValue("(OO)", py_found, py_remaining); +} + static PyMethodDef PyLocalSchedulerClient_methods[] = { {"disconnect", (PyCFunction) PyLocalSchedulerClient_disconnect, METH_NOARGS, "Notify the local scheduler that this client is exiting gracefully."}, @@ -201,6 +251,9 @@ static PyMethodDef PyLocalSchedulerClient_methods[] = { (PyCFunction) PyLocalSchedulerClient_get_actor_frontier, METH_VARARGS, ""}, {"set_actor_frontier", (PyCFunction) PyLocalSchedulerClient_set_actor_frontier, METH_VARARGS, ""}, + {"wait", + (PyCFunction) PyLocalSchedulerClient_wait, METH_VARARGS, + "Wait for a list of objects to be created."}, {NULL} /* Sentinel */ }; diff --git a/src/local_scheduler/local_scheduler_client.cc b/src/local_scheduler/local_scheduler_client.cc index 2d134d9a9b8d..be6be1e07159 100644 --- a/src/local_scheduler/local_scheduler_client.cc +++ b/src/local_scheduler/local_scheduler_client.cc @@ -2,6 +2,7 @@ #include "common_protocol.h" #include "format/local_scheduler_generated.h" +#include "ray/raylet/format/node_manager_generated.h" #include "common/io.h" #include "common/task.h" @@ -186,3 +187,37 @@ void local_scheduler_set_actor_frontier(LocalSchedulerConnection *conn, write_message(conn->conn, MessageType_SetActorFrontier, frontier.size(), const_cast(frontier.data())); } + +std::pair, std::vector> local_scheduler_wait( + LocalSchedulerConnection *conn, + const std::vector &object_ids, + int num_returns, + int timeout, + bool wait_local) { + // Write request. + flatbuffers::FlatBufferBuilder fbb; + auto message = ray::protocol::CreateWaitRequest( + fbb, to_flatbuf(fbb, object_ids), num_returns, timeout, wait_local); + fbb.Finish(message); + write_message(conn->conn, ray::protocol::MessageType_WaitRequest, + fbb.GetSize(), fbb.GetBufferPointer()); + // Read result. + int64_t type; + int64_t reply_size; + uint8_t *reply; + read_message(conn->conn, &type, &reply_size, &reply); + auto reply_message = flatbuffers::GetRoot(reply); + // Convert result. + std::pair, std::vector> result; + auto found = reply_message->found(); + for (uint i = 0; i < found->size(); i++) { + ObjectID object_id = ObjectID::from_binary(found->Get(i)->str()); + result.first.push_back(object_id); + } + auto remaining = reply_message->remaining(); + for (uint i = 0; i < remaining->size(); i++) { + ObjectID object_id = ObjectID::from_binary(remaining->Get(i)->str()); + result.second.push_back(object_id); + } + return result; +} diff --git a/src/local_scheduler/local_scheduler_client.h b/src/local_scheduler/local_scheduler_client.h index 7b834a09c7a2..d9cb502fd7c7 100644 --- a/src/local_scheduler/local_scheduler_client.h +++ b/src/local_scheduler/local_scheduler_client.h @@ -169,4 +169,20 @@ const std::vector local_scheduler_get_actor_frontier( void local_scheduler_set_actor_frontier(LocalSchedulerConnection *conn, const std::vector &frontier); +/// Wait for the given objects until timeout expires or num_return objects are found. +/// +/// \param conn The connection information. +/// \param object_ids The objects to wait for. +/// \param num_returns The number of objects to wait for. +/// \param timeout The duration to wait before returning. +/// \param wait_local Whether to wait for objects to appear on this node. +/// \return A pair with the first element containing the object ids that were found, +/// and the second element the objects that were not found. +std::pair, std::vector> local_scheduler_wait( + LocalSchedulerConnection *conn, + const std::vector &object_ids, + int num_returns, + int timeout, + bool wait_local); + #endif diff --git a/src/ray/raylet/format/node_manager.fbs b/src/ray/raylet/format/node_manager.fbs index 5c50d2c542fd..fb8591b88ad8 100644 --- a/src/ray/raylet/format/node_manager.fbs +++ b/src/ray/raylet/format/node_manager.fbs @@ -54,10 +54,9 @@ enum MessageType:int { SetActorFrontier, // A node manager request to process a task forwarded from another node manager. ForwardTaskRequest, - // Wait for objects to be ready either from local or remote Plasma stores. - PlasmaWaitRequest=20, - PlasmaWaitReply=21 + WaitRequest, + WaitReply } table TaskExecutionSpecification { @@ -122,47 +121,20 @@ table ReconstructObject { object_id: string; } -enum ObjectStatus:int { - // Object is stored in the local Plasma Store. - Local = 1, - // Object is stored on a remote Plasma store, and it is not stored on the - // local Plasma Store. - Remote, - // Object is not stored in the system. - Nonexistent, - // Object is currently transferred from a remote Plasma store to the local - // Plasma Store. - Transfer -} - -table ObjectRequestSpec { - // ID of the object. - object_id: string; - // The type of the object. This specifies whether we - // will be waiting for an object store in the local or - // global Plasma store. - type: int; -} - -table PlasmaWaitRequest { - // Array of object requests whose status we are asking for. - object_requests: [ObjectRequestSpec]; +table WaitRequest { + // List of object ids we'll be waiting on. + object_ids: [string]; // Number of objects expected to be returned, if available. num_ready_objects: int; // timeout timeout: long; + // Whether to wait until objects appear locally. + wait_local: bool; } -table ObjectReply { - // ID of the object. - object_id: string; - // The object status. This specifies where the object is stored. - status: int; -} - -table PlasmaWaitReply { - // Array of object requests being returned. - object_requests: [ObjectReply]; - // Number of objects expected to be returned, if available. - num_ready_objects: int; +table WaitReply { + // List of object ids found. + found: [string]; + // List of object ids not found. + remaining: [string]; } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index e695e68fc599..cb61c2c466f7 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -447,94 +447,34 @@ void NodeManager::ProcessClientMessage( worker->MarkUnblocked(); } } break; - case protocol::MessageType_PlasmaWaitRequest: { - // Block the worker. - std::shared_ptr worker = worker_pool_.GetRegisteredWorker(client); - if (worker && !worker->IsBlocked()) { - RAY_CHECK(!worker->GetAssignedTaskId().is_nil()); - auto tasks = local_queues_.RemoveTasks({worker->GetAssignedTaskId()}); - const auto &task = tasks.front(); - // Get the CPU resources required by the running task. - const auto required_resources = task.GetTaskSpecification().GetRequiredResources(); - double required_cpus = 0; - RAY_CHECK(required_resources.GetResource(kCPU_ResourceLabel, &required_cpus)); - const std::unordered_map cpu_resources = { - {kCPU_ResourceLabel, required_cpus}}; - // Release the CPU resources. - RAY_CHECK( - cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release( - ResourceSet(cpu_resources))); - // Mark the task as blocked. - local_queues_.QueueBlockedTasks(tasks); - worker->MarkBlocked(); - - // Try to dispatch more tasks since the blocked worker released some - // resources. - DispatchTasks(); - } - + case protocol::MessageType_WaitRequest: { // Read the data. - auto message = flatbuffers::GetRoot(message_data); - auto object_requests = message->object_requests(); - int num_requests = object_requests->size(); + auto message = flatbuffers::GetRoot(message_data); + auto object_id_strings = message->object_ids(); + int num_requests = object_id_strings->size(); std::vector object_ids; - // Does it make sense to specify local/non-local for each object? - std::vector wait_local; for (int i = 0; i < num_requests; i++) { - ObjectID object_id = ObjectID::from_binary( - message->object_requests()->Get(i)->object_id()->str()); + ObjectID object_id = ObjectID::from_binary(object_id_strings->Get(i)->str()); object_ids.push_back(object_id); - wait_local.push_back(message->object_requests()->Get(i)->type() == 1); } int64_t wait_ms = message->timeout(); uint64_t num_required_objects = static_cast(message->num_ready_objects()); - - object_manager_.Wait(object_ids, wait_ms, num_required_objects, false, - [this, client](int64_t time_taken, std::unordered_set found, std::unordered_set remaining){ - // Unblock the worker. - std::shared_ptr worker = worker_pool_.GetRegisteredWorker(client); - if (worker && !worker->IsBlocked()) { - RAY_CHECK(!worker->GetAssignedTaskId().is_nil()); - auto tasks = local_queues_.RemoveTasks({worker->GetAssignedTaskId()}); - const auto &task = tasks.front(); - // Get the CPU resources required by the running task. - const auto required_resources = task.GetTaskSpecification().GetRequiredResources(); - double required_cpus = 0; - RAY_CHECK(required_resources.GetResource(kCPU_ResourceLabel, &required_cpus)); - const std::unordered_map cpu_resources = { - {kCPU_ResourceLabel, required_cpus}}; - // Release the CPU resources. - RAY_CHECK( - cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release( - ResourceSet(cpu_resources))); - // Mark the task as blocked. - local_queues_.QueueBlockedTasks(tasks); - worker->MarkBlocked(); - - // Try to dispatch more tasks since the blocked worker released some - // resources. - DispatchTasks(); - } - - // Write the data. - flatbuffers::FlatBufferBuilder fbb; - std::vector> object_replies; - for (const auto &object_id : found) { - object_replies.push_back(protocol::CreateObjectReply( - fbb, fbb.CreateString(object_id.binary()), - protocol::ObjectStatus_Remote)); - } - for (const auto &object_id : remaining) { - object_replies.push_back(protocol::CreateObjectReply( - fbb, fbb.CreateString(object_id.binary()), - protocol::ObjectStatus_Nonexistent)); - } - auto send_message = protocol::CreatePlasmaWaitReply( - fbb, fbb.CreateVector(object_replies.data(), found.size()), static_cast(found.size())); - fbb.Finish(send_message); - client->WriteMessage(protocol::MessageType_PlasmaWaitReply, - fbb.GetSize(), fbb.GetBufferPointer()); - }); + bool wait_local = message->wait_local(); + + object_manager_.Wait( + object_ids, wait_ms, num_required_objects, wait_local, + [this, client](int64_t time_taken, std::unordered_set found, + std::unordered_set remaining) { + // Write the data. + std::vector found_vec(found.begin(), found.end()); + std::vector remaining_vec(remaining.begin(), remaining.end()); + flatbuffers::FlatBufferBuilder fbb; + flatbuffers::Offset wait_reply = protocol::CreateWaitReply( + fbb, to_flatbuf(fbb, found_vec), to_flatbuf(fbb, remaining_vec)); + fbb.Finish(wait_reply); + client->WriteMessage(protocol::MessageType_WaitReply, fbb.GetSize(), + fbb.GetBufferPointer()); + }); } break; default: From 9a95c65bef02c2ef581e89609f930b213cd712af Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 30 May 2018 00:39:22 -0700 Subject: [PATCH 14/42] with passing tests. --- python/ray/worker.py | 11 +++++++++- .../lib/python/local_scheduler_extension.cc | 21 ++++++++----------- src/local_scheduler/local_scheduler_client.cc | 2 +- src/local_scheduler/local_scheduler_client.h | 8 ++++--- src/ray/object_manager/object_manager.cc | 11 ++++++---- src/ray/raylet/node_manager.cc | 3 ++- test/runtest.py | 3 --- 7 files changed, 34 insertions(+), 25 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 2e26edd90986..b0d5aacd4197 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2558,9 +2558,18 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): return [], [] if worker.use_raylet: + if len(object_ids) != len(set(object_ids)): + raise Exception("Wait requires a list of unique object IDs.") + if len(object_ids) <= 0: + raise Exception("Invalid number of objects %d." % len(object_ids)) + if num_returns <= 0: + raise Exception("Invalid number of objects to return %d." % num_returns) + if num_returns > len(object_ids): + raise Exception("num_returns cannot be greater than the number " + "of objects provided to ray.wait.") timeout = timeout if timeout is not None else 2**30 ready_ids, remaining_ids = worker.local_scheduler_client.wait( - object_ids, timeout, num_returns, False) + object_ids, num_returns, timeout, False) else: object_id_strs = [ plasma.ObjectID(object_id.id()) for object_id in object_ids diff --git a/src/local_scheduler/lib/python/local_scheduler_extension.cc b/src/local_scheduler/lib/python/local_scheduler_extension.cc index a547a76ccfac..018abcef5620 100644 --- a/src/local_scheduler/lib/python/local_scheduler_extension.cc +++ b/src/local_scheduler/lib/python/local_scheduler_extension.cc @@ -180,16 +180,16 @@ static PyObject *PyLocalSchedulerClient_set_actor_frontier(PyObject *self, } static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { - int timeout_ms; - int num_returns; - bool wait_local; PyObject *py_object_ids; - if (!PyArg_ParseTuple(args, "Oiip", &py_object_ids, &timeout_ms, &num_returns, + int num_returns; + int64_t timeout_ms; + int wait_local; + + if (!PyArg_ParseTuple(args, "Oili", &py_object_ids, &num_returns, &timeout_ms, &wait_local)) { return NULL; } // Convert to milliseconds. - timeout_ms *= 1000; PyObject *iter = PyObject_GetIter(py_object_ids); if (!iter) { return NULL; @@ -213,16 +213,14 @@ static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { std::pair, std::vector> result = local_scheduler_wait( ((PyLocalSchedulerClient *) self)->local_scheduler_connection, - object_ids, num_returns, timeout_ms, wait_local); + object_ids, num_returns, timeout_ms, static_cast(wait_local)); // Convert result to py object. - PyObject *py_found = - PyList_New((Py_ssize_t) result.first.size()); + PyObject *py_found = PyList_New((Py_ssize_t) result.first.size()); for (uint i = 0; i < result.first.size(); ++i) { PyList_SetItem(py_found, i, PyObjectID_make(result.first[i])); } - PyObject *py_remaining = - PyList_New((Py_ssize_t) result.second.size()); + PyObject *py_remaining = PyList_New((Py_ssize_t) result.second.size()); for (uint i = 0; i < result.second.size(); ++i) { PyList_SetItem(py_remaining, i, PyObjectID_make(result.second[i])); } @@ -251,8 +249,7 @@ static PyMethodDef PyLocalSchedulerClient_methods[] = { (PyCFunction) PyLocalSchedulerClient_get_actor_frontier, METH_VARARGS, ""}, {"set_actor_frontier", (PyCFunction) PyLocalSchedulerClient_set_actor_frontier, METH_VARARGS, ""}, - {"wait", - (PyCFunction) PyLocalSchedulerClient_wait, METH_VARARGS, + {"wait", (PyCFunction) PyLocalSchedulerClient_wait, METH_VARARGS, "Wait for a list of objects to be created."}, {NULL} /* Sentinel */ }; diff --git a/src/local_scheduler/local_scheduler_client.cc b/src/local_scheduler/local_scheduler_client.cc index be6be1e07159..a148cf0b5789 100644 --- a/src/local_scheduler/local_scheduler_client.cc +++ b/src/local_scheduler/local_scheduler_client.cc @@ -192,7 +192,7 @@ std::pair, std::vector> local_scheduler_wait( LocalSchedulerConnection *conn, const std::vector &object_ids, int num_returns, - int timeout, + int64_t timeout, bool wait_local) { // Write request. flatbuffers::FlatBufferBuilder fbb; diff --git a/src/local_scheduler/local_scheduler_client.h b/src/local_scheduler/local_scheduler_client.h index d9cb502fd7c7..a68608db24ee 100644 --- a/src/local_scheduler/local_scheduler_client.h +++ b/src/local_scheduler/local_scheduler_client.h @@ -169,20 +169,22 @@ const std::vector local_scheduler_get_actor_frontier( void local_scheduler_set_actor_frontier(LocalSchedulerConnection *conn, const std::vector &frontier); -/// Wait for the given objects until timeout expires or num_return objects are found. +/// Wait for the given objects until timeout expires or num_return objects are +/// found. /// /// \param conn The connection information. /// \param object_ids The objects to wait for. /// \param num_returns The number of objects to wait for. /// \param timeout The duration to wait before returning. /// \param wait_local Whether to wait for objects to appear on this node. -/// \return A pair with the first element containing the object ids that were found, +/// \return A pair with the first element containing the object ids that were +/// found, /// and the second element the objects that were not found. std::pair, std::vector> local_scheduler_wait( LocalSchedulerConnection *conn, const std::vector &object_ids, int num_returns, - int timeout, + int64_t timeout, bool wait_local); #endif diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 41ede8894069..059f0309b260 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -116,7 +116,8 @@ ray::Status ObjectManager::Pull(const ObjectID &object_id) { ray::Status status_code = object_directory_->SubscribeObjectLocations( object_directory_pull_callback_id_, object_id, [this](const std::vector &client_ids, const ObjectID &object_id) { - RAY_CHECK_OK(object_directory_->UnsubscribeObjectLocations(object_directory_pull_callback_id_, object_id)); + RAY_CHECK_OK(object_directory_->UnsubscribeObjectLocations( + object_directory_pull_callback_id_, object_id)); GetLocationsSuccess(client_ids, object_id); }); return status_code; @@ -402,9 +403,11 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { // will do nothing on non-zero error codes. wait_state.timeout_timer->cancel(); // Invoke the wait handler. - int64_t time_taken = wait_state.wait_ms == 0? 0 : - (boost::posix_time::second_clock::local_time() - wait_state.start_time) - .total_milliseconds(); + int64_t time_taken = + wait_state.wait_ms == 0 + ? 0 + : (boost::posix_time::second_clock::local_time() - wait_state.start_time) + .total_milliseconds(); wait_state.callback(time_taken, wait_state.found, wait_state.remaining); } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index cb61c2c466f7..2161a5e54e74 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -461,7 +461,7 @@ void NodeManager::ProcessClientMessage( uint64_t num_required_objects = static_cast(message->num_ready_objects()); bool wait_local = message->wait_local(); - object_manager_.Wait( + ray::Status status = object_manager_.Wait( object_ids, wait_ms, num_required_objects, wait_local, [this, client](int64_t time_taken, std::unordered_set found, std::unordered_set remaining) { @@ -475,6 +475,7 @@ void NodeManager::ProcessClientMessage( client->WriteMessage(protocol::MessageType_WaitReply, fbb.GetSize(), fbb.GetBufferPointer()); }); + RAY_CHECK_OK(status); } break; default: diff --git a/test/runtest.py b/test/runtest.py index beb2ed0c32e0..550984ea5da7 100644 --- a/test/runtest.py +++ b/test/runtest.py @@ -758,9 +758,6 @@ def testGetMultiple(self): results = ray.get([object_ids[i] for i in indices]) self.assertEqual(results, indices) - @unittest.skipIf( - os.environ.get("RAY_USE_XRAY") == "1", - "This test does not work with xray yet.") def testWait(self): self.init_ray(num_cpus=1) From aa12bd7927f5afb43abece2bcf316907ae71d217 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 30 May 2018 00:59:26 -0700 Subject: [PATCH 15/42] minor adjustments. --- .../lib/python/local_scheduler_extension.cc | 3 +-- src/ray/object_manager/object_directory.cc | 11 +++++++---- src/ray/object_manager/object_manager.cc | 13 ++++++------- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/local_scheduler/lib/python/local_scheduler_extension.cc b/src/local_scheduler/lib/python/local_scheduler_extension.cc index 018abcef5620..4d09e82c02ee 100644 --- a/src/local_scheduler/lib/python/local_scheduler_extension.cc +++ b/src/local_scheduler/lib/python/local_scheduler_extension.cc @@ -189,12 +189,11 @@ static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { &wait_local)) { return NULL; } - // Convert to milliseconds. + // Convert object ids. PyObject *iter = PyObject_GetIter(py_object_ids); if (!iter) { return NULL; } - // Convert object ids. std::vector object_ids; while (true) { PyObject *next = PyIter_Next(iter); diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index a06c1872815e..eaa5aaa9eb84 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -18,10 +18,13 @@ void ObjectDirectory::RegisterBackend() { } // Update entries for this object. auto &location_client_id_set = object_id_listener_pair->second.location_client_ids; - // object_location_ids has the history of locations of the object: - // client1.is_eviction = false - // client1.is_eviction = true - // client2.is_eviction = false + // object_location_ids contains the history of locations of the object (it is a log), + // which might look like the following: + // client1.is_eviction = false + // client1.is_eviction = true + // client2.is_eviction = false + // In such a scenario, we want to indicate client2 is the only client that contains + // the object, which the following code achieves. for (const auto &object_table_data : object_location_ids) { ClientID client_id = ClientID::from_binary(object_table_data.manager); if (!object_table_data.is_eviction) { diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 059f0309b260..eedcb9e46935 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -315,10 +315,9 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t return ray::Status::NotImplemented("Wait for local objects is not yet implemented."); } - if (num_required_objects == 0) { - // TODO: Confirm this is the default value for waiting for all objects. - num_required_objects = object_ids.size(); - } + RAY_CHECK(object_ids.size() != 0); + RAY_CHECK(num_required_objects != 0); + RAY_CHECK(num_required_objects <= object_ids.size()); // Initialize fields. active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); @@ -375,9 +374,9 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t }); } // Set timeout. - // TODO (hme): If we need to just wait for all objects independent of time, - // determine what the value of wait_ms should be and skip this call. - // WaitComplete will be invoked when all objects have locations. + // TODO (hme): If we need to just wait for all objects independent of time + // (i.e. infinite wait time), determine what the value of wait_ms should be and + // skip this call. WaitComplete will be invoked when all objects have locations. wait_state.timeout_timer->async_wait( [this, wait_id](const boost::system::error_code &error_code) { if (error_code.value() != 0) { From 304b39c887bb54ae34c484ff67700069bd2f047f Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 30 May 2018 13:41:32 -0700 Subject: [PATCH 16/42] handle return statuses. --- src/ray/object_manager/object_manager.cc | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index c290db3a7b84..0378357e5d80 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -350,7 +350,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t for (auto &oid : wait_state.remaining) { // Subscribe to object notifications. wait_state.requested_objects.insert(oid); - object_directory_->LookupLocations( + RAY_CHECK_OK(object_directory_->LookupLocations( oid, [this, wait_id](const std::vector &client_ids, const ObjectID &object_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; @@ -362,13 +362,13 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t if (wait_state.requested_objects.empty()) { WaitComplete(wait_id); } - }); + })); } } else { for (auto &oid : wait_state.remaining) { // Subscribe to object notifications. wait_state.requested_objects.insert(oid); - object_directory_->SubscribeObjectLocations( + RAY_CHECK_OK(object_directory_->SubscribeObjectLocations( wait_id, oid, [this, wait_id](const std::vector &client_ids, const ObjectID &object_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; @@ -377,11 +377,12 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t wait_state.found.insert(object_id); } wait_state.requested_objects.erase(object_id); - object_directory_->UnsubscribeObjectLocations(wait_id, object_id); + RAY_CHECK_OK( + object_directory_->UnsubscribeObjectLocations(wait_id, object_id)); if (wait_state.found.size() >= wait_state.num_required_objects) { WaitComplete(wait_id); } - }); + })); } // Set timeout. // TODO (hme): If we need to just wait for all objects independent of time @@ -405,7 +406,7 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { RAY_CHECK(!(wait_state.requested_objects.size() > 0) || wait_state.wait_ms > 0); // Unsubscribe to any objects that weren't found in the time allotted. for (auto &object_id : wait_state.requested_objects) { - object_directory_->UnsubscribeObjectLocations(wait_id, object_id); + RAY_CHECK_OK(object_directory_->UnsubscribeObjectLocations(wait_id, object_id)); } // Cancel the timer. This is okay even if the timer hasn't been started. // The timer handler will be given a non-zero error code. The handler From 5d63bb3cbb68b588a07ce08e578ac4873d26ef82 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 30 May 2018 16:22:06 -0700 Subject: [PATCH 17/42] enable more tests. --- test/runtest.py | 3 --- test/stress_tests.py | 3 --- 2 files changed, 6 deletions(-) diff --git a/test/runtest.py b/test/runtest.py index ab1692bcedd1..59ea43b6b5f3 100644 --- a/test/runtest.py +++ b/test/runtest.py @@ -823,9 +823,6 @@ def f(delay): with self.assertRaises(TypeError): ray.wait([1]) - @unittest.skipIf( - os.environ.get("RAY_USE_XRAY") == "1", - "This test does not work with xray yet.") def testMultipleWaitsAndGets(self): # It is important to use three workers here, so that the three tasks # launched in this experiment can run at the same time. diff --git a/test/stress_tests.py b/test/stress_tests.py index 12e0e1aa6693..dea955a42eff 100644 --- a/test/stress_tests.py +++ b/test/stress_tests.py @@ -121,9 +121,6 @@ def f(): self.assertTrue(ray.services.all_processes_alive()) ray.worker.cleanup() - @unittest.skipIf( - os.environ.get("RAY_USE_XRAY") == "1", - "This test does not work with xray yet.") def testWait(self): for num_local_schedulers in [1, 4]: for num_workers_per_scheduler in [4]: From cf1fdb277fc149d2c922a1afbd47cda116c46171 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 30 May 2018 17:21:24 -0700 Subject: [PATCH 18/42] add test for existing num_returns semantics, and maintain existing num_returns semantics. --- src/ray/object_manager/object_manager.cc | 13 ++++++++++++- test/runtest.py | 6 ++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 0378357e5d80..c869a76d50fd 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -348,7 +348,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t } else { if (wait_ms == 0) { for (auto &oid : wait_state.remaining) { - // Subscribe to object notifications. + // Lookup remaining objects. wait_state.requested_objects.insert(oid); RAY_CHECK_OK(object_directory_->LookupLocations( oid, [this, wait_id](const std::vector &client_ids, @@ -418,7 +418,18 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { ? 0 : (boost::posix_time::second_clock::local_time() - wait_state.start_time) .total_milliseconds(); + // Wait semantics require marking at most num_required_objects as found. + int64_t num_move = wait_state.found.size() - wait_state.num_required_objects; + if (num_move > 0) { + auto iter = wait_state.found.begin(); + while (num_move > 0) { + num_move -= 1; + wait_state.remaining.insert(*iter); + iter = wait_state.found.erase(iter); + } + } wait_state.callback(time_taken, wait_state.found, wait_state.remaining); + active_wait_requests_.erase(wait_id); } std::shared_ptr ObjectManager::CreateSenderConnection( diff --git a/test/runtest.py b/test/runtest.py index 59ea43b6b5f3..90e2c461c160 100644 --- a/test/runtest.py +++ b/test/runtest.py @@ -814,6 +814,12 @@ def f(delay): self.assertEqual(ready_ids, []) self.assertEqual(remaining_ids, []) + # Test semantics of num_returns with no timeout. + oids = [ray.put(i) for i in range(10)] + (found, rest) = ray.wait(oids, num_returns=2) + self.assertEqual(len(found), 2) + self.assertEqual(len(rest), 8) + # Verify that incorrect usage raises a TypeError. x = ray.put(1) with self.assertRaises(TypeError): From 531d0249f4791ec2f45eaaa3e472d9c35f53277e Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 30 May 2018 17:22:15 -0700 Subject: [PATCH 19/42] move error handling to both code paths. --- python/ray/worker.py | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 8efa4380c4c7..3dc9de3426ad 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2571,24 +2571,23 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): if len(object_ids) == 0: return [], [] + if len(object_ids) != len(set(object_ids)): + raise Exception("Wait requires a list of unique object IDs.") + if len(object_ids) == 0: + raise Exception("Invalid number of objects %d." % len(object_ids)) + if num_returns <= 0: + raise Exception("Invalid number of objects to return %d." % num_returns) + if num_returns > len(object_ids): + raise Exception("num_returns cannot be greater than the number " + "of objects provided to ray.wait.") + timeout = timeout if timeout is not None else 2**30 if worker.use_raylet: - if len(object_ids) != len(set(object_ids)): - raise Exception("Wait requires a list of unique object IDs.") - if len(object_ids) <= 0: - raise Exception("Invalid number of objects %d." % len(object_ids)) - if num_returns <= 0: - raise Exception("Invalid number of objects to return %d." % num_returns) - if num_returns > len(object_ids): - raise Exception("num_returns cannot be greater than the number " - "of objects provided to ray.wait.") - timeout = timeout if timeout is not None else 2**30 ready_ids, remaining_ids = worker.local_scheduler_client.wait( object_ids, num_returns, timeout, False) else: object_id_strs = [ plasma.ObjectID(object_id.id()) for object_id in object_ids ] - timeout = timeout if timeout is not None else 2**30 ready_ids, remaining_ids = worker.plasma_client.wait( object_id_strs, timeout, num_returns) ready_ids = [ From d0d3ea4548032aac6834f6df54f68266d2bfc7a1 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 30 May 2018 18:45:34 -0700 Subject: [PATCH 20/42] implementing another round of feedback. --- src/local_scheduler/local_scheduler_client.h | 3 +- src/ray/object_manager/object_manager.cc | 9 +- src/ray/object_manager/object_manager.h | 30 ++++-- .../test/object_manager_test.cc | 100 +++++++++--------- src/ray/raylet/format/node_manager.fbs | 2 + src/ray/raylet/node_manager.cc | 4 +- 6 files changed, 78 insertions(+), 70 deletions(-) diff --git a/src/local_scheduler/local_scheduler_client.h b/src/local_scheduler/local_scheduler_client.h index a68608db24ee..4291bbe6e3df 100644 --- a/src/local_scheduler/local_scheduler_client.h +++ b/src/local_scheduler/local_scheduler_client.h @@ -178,8 +178,7 @@ void local_scheduler_set_actor_frontier(LocalSchedulerConnection *conn, /// \param timeout The duration to wait before returning. /// \param wait_local Whether to wait for objects to appear on this node. /// \return A pair with the first element containing the object ids that were -/// found, -/// and the second element the objects that were not found. +/// found, and the second element the objects that were not found. std::pair, std::vector> local_scheduler_wait( LocalSchedulerConnection *conn, const std::vector &object_ids, diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index c869a76d50fd..dd9fd60941e2 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -317,17 +317,16 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t const WaitCallback &callback) { UniqueID wait_id = UniqueID::from_random(); - if (wait_ms < 0) { - return ray::Status::Invalid("Unable to wait negative wait time."); - } - if (wait_local) { return ray::Status::NotImplemented("Wait for local objects is not yet implemented."); } - RAY_CHECK(object_ids.size() != 0); + RAY_CHECK(wait_ms >= 0); RAY_CHECK(num_required_objects != 0); RAY_CHECK(num_required_objects <= object_ids.size()); + if (object_ids.size() == 0) { + callback(0, std::unordered_set(), std::unordered_set()); + } // Initialize fields. active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index e73824929e29..e16b6cb51dff 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -142,9 +142,9 @@ class ObjectManager : public ObjectManagerInterface { ray::Status Cancel(const ObjectID &object_id); /// Callback definition for wait. - using WaitCallback = std::function &found, - const std::unordered_set &remaining)>; + using WaitCallback = + std::function &found, + const std::unordered_set &remaining)>; /// Wait until either num_required_objects are located or wait_ms has elapsed, /// then invoke the provided callback. /// @@ -157,7 +157,8 @@ class ObjectManager : public ObjectManagerInterface { /// is satisfied. /// \return Status of whether the wait successfully initiated. ray::Status Wait(const std::vector &object_ids, int64_t wait_ms, - uint64_t num_required_objects, bool wait_local, const WaitCallback &callback); + uint64_t num_required_objects, bool wait_local, + const WaitCallback &callback); private: ClientID client_id_; @@ -198,18 +199,27 @@ class ObjectManager : public ObjectManagerInterface { UniqueID object_directory_pull_callback_id_ = UniqueID::from_random(); struct WaitState { - WaitState(asio::io_service &service, int64_t wait_ms, const WaitCallback &callback) : - wait_ms(wait_ms), - timeout_timer(std::make_shared(service, boost::posix_time::milliseconds(wait_ms))), - callback(callback) - {} + WaitState(asio::io_service &service, int64_t wait_ms, const WaitCallback &callback) + : wait_ms(wait_ms), + timeout_timer(std::unique_ptr( + new boost::asio::deadline_timer(service, + boost::posix_time::milliseconds(wait_ms)))), + callback(callback) {} + /// The period of time to wait before invoking the callback. int64_t wait_ms; - std::shared_ptr timeout_timer; + /// The timer used whenever wait_ms > 0. + std::unique_ptr timeout_timer; + /// The callback invoked when WaitCallback is complete. WaitCallback callback; + /// The objects that have not yet been found. std::unordered_set remaining; + /// The objects that have been found. std::unordered_set found; + /// Objects that have been requested either by Lookup or Subscribe. std::unordered_set requested_objects; + /// The number of required objects. uint64_t num_required_objects; + /// The time at which the wait operation begins. boost::posix_time::ptime start_time; }; diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index 94787c7a2b6a..b65d04500955 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -190,7 +190,6 @@ class TestObjectManager : public ::testing::Test { class TestObjectManagerCommands : public TestObjectManager { public: - int current_wait_test = -1; int num_connected_clients = 0; uint num_expected_objects; @@ -243,23 +242,24 @@ class TestObjectManagerCommands : public TestObjectManager { void NextWaitTest() { current_wait_test += 1; - switch(current_wait_test){ - case 0: { - TestWait(100, 5, 3, 0, false, false); - } break; - case 1: { - TestWait(100, 5, 3, 1000, false, true); - } break; - case 2: { - TestWait(100, 5, 3, 1000, false, false); - } break; - case 3: { - TestWait(100, 5, 6, 1000, true, false); - } break; + switch (current_wait_test) { + case 0: { + TestWait(100, 5, 3, 0, false, false); + } break; + case 1: { + TestWait(100, 5, 3, 1000, false, true); + } break; + case 2: { + TestWait(100, 5, 3, 1000, false, false); + } break; + case 3: { + TestWait(100, 5, 6, 1000, true, false); + } break; } } - void TestWait(int data_size, int num_objects, uint64_t required_objects, int wait_ms, bool include_nonexistent, bool test_local) { + void TestWait(int data_size, int num_objects, uint64_t required_objects, int wait_ms, + bool include_nonexistent, bool test_local) { std::vector object_ids; for (int i = -1; ++i < num_objects;) { ObjectID oid; @@ -276,46 +276,44 @@ class TestObjectManagerCommands : public TestObjectManager { } server1->object_manager_.Wait( object_ids, wait_ms, required_objects, false, - [this, num_objects, wait_ms, required_objects](int64_t elapsed, - const std::unordered_set &found, - const std::unordered_set &remaining) { - switch(current_wait_test){ - case 0: { - ASSERT_TRUE(elapsed == 0); - ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); - NextWaitTest(); - } break; - case 1: { - RAY_LOG(DEBUG) << "elapsed " << elapsed; - RAY_LOG(DEBUG) << "found " << found.size(); - RAY_LOG(DEBUG) << "remaining " << remaining.size(); - ASSERT_TRUE(found.size() >= required_objects); - ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); - NextWaitTest(); - } break; - case 2: { - RAY_LOG(DEBUG) << "elapsed " << elapsed; - RAY_LOG(DEBUG) << "found " << found.size(); - RAY_LOG(DEBUG) << "remaining " << remaining.size(); - ASSERT_TRUE(found.size() >= required_objects); - ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); - NextWaitTest(); - } break; - case 3: { - RAY_LOG(DEBUG) << "elapsed " << elapsed; - RAY_LOG(DEBUG) << "found " << found.size(); - RAY_LOG(DEBUG) << "remaining " << remaining.size(); - ASSERT_TRUE(elapsed >= wait_ms); - ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); - TestWaitComplete(); - } break; + [this, num_objects, wait_ms, required_objects]( + int64_t elapsed, const std::unordered_set &found, + const std::unordered_set &remaining) { + switch (current_wait_test) { + case 0: { + ASSERT_TRUE(elapsed == 0); + ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + NextWaitTest(); + } break; + case 1: { + RAY_LOG(DEBUG) << "elapsed " << elapsed; + RAY_LOG(DEBUG) << "found " << found.size(); + RAY_LOG(DEBUG) << "remaining " << remaining.size(); + ASSERT_TRUE(found.size() >= required_objects); + ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + NextWaitTest(); + } break; + case 2: { + RAY_LOG(DEBUG) << "elapsed " << elapsed; + RAY_LOG(DEBUG) << "found " << found.size(); + RAY_LOG(DEBUG) << "remaining " << remaining.size(); + ASSERT_TRUE(found.size() >= required_objects); + ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + NextWaitTest(); + } break; + case 3: { + RAY_LOG(DEBUG) << "elapsed " << elapsed; + RAY_LOG(DEBUG) << "found " << found.size(); + RAY_LOG(DEBUG) << "remaining " << remaining.size(); + ASSERT_TRUE(elapsed >= wait_ms); + ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + TestWaitComplete(); + } break; } }); } - void TestWaitComplete() { - main_service.stop(); - } + void TestWaitComplete() { main_service.stop(); } void TestConnections() { RAY_LOG(DEBUG) << "\n" diff --git a/src/ray/raylet/format/node_manager.fbs b/src/ray/raylet/format/node_manager.fbs index fb8591b88ad8..ba3dda4cb813 100644 --- a/src/ray/raylet/format/node_manager.fbs +++ b/src/ray/raylet/format/node_manager.fbs @@ -56,6 +56,8 @@ enum MessageType:int { ForwardTaskRequest, // Wait for objects to be ready either from local or remote Plasma stores. WaitRequest, + // The response message to WaitRequest; replies with the objects found and objects + // remaining. WaitReply } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 2161a5e54e74..7ec264d97895 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -472,8 +472,8 @@ void NodeManager::ProcessClientMessage( flatbuffers::Offset wait_reply = protocol::CreateWaitReply( fbb, to_flatbuf(fbb, found_vec), to_flatbuf(fbb, remaining_vec)); fbb.Finish(wait_reply); - client->WriteMessage(protocol::MessageType_WaitReply, fbb.GetSize(), - fbb.GetBufferPointer()); + RAY_CHECK_OK(client->WriteMessage(protocol::MessageType_WaitReply, + fbb.GetSize(), fbb.GetBufferPointer())); }); RAY_CHECK_OK(status); } break; From 62ae832c9e8cb208712075624619c7d43d1da5ed Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 30 May 2018 19:12:59 -0700 Subject: [PATCH 21/42] Comment on OM tests. --- src/ray/object_manager/test/object_manager_test.cc | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index b65d04500955..46361e2ddddf 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -281,11 +281,14 @@ class TestObjectManagerCommands : public TestObjectManager { const std::unordered_set &remaining) { switch (current_wait_test) { case 0: { + // Ensure wait_ms = 0 returns immediately after lookup when objects are + // remote. ASSERT_TRUE(elapsed == 0); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); NextWaitTest(); } break; case 1: { + // Ensure lookup succeeds as expected when objects are local. RAY_LOG(DEBUG) << "elapsed " << elapsed; RAY_LOG(DEBUG) << "found " << found.size(); RAY_LOG(DEBUG) << "remaining " << remaining.size(); @@ -294,6 +297,7 @@ class TestObjectManagerCommands : public TestObjectManager { NextWaitTest(); } break; case 2: { + // Ensure lookup succeeds as expected when objects are remote. RAY_LOG(DEBUG) << "elapsed " << elapsed; RAY_LOG(DEBUG) << "found " << found.size(); RAY_LOG(DEBUG) << "remaining " << remaining.size(); @@ -302,6 +306,7 @@ class TestObjectManagerCommands : public TestObjectManager { NextWaitTest(); } break; case 3: { + // Ensure lookup returns after wait_ms elapses when one object doesn't exist. RAY_LOG(DEBUG) << "elapsed " << elapsed; RAY_LOG(DEBUG) << "found " << found.size(); RAY_LOG(DEBUG) << "remaining " << remaining.size(); From 67eef677fe90982066a798a1a3a8caa11fe35f83 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Thu, 31 May 2018 18:36:13 -0700 Subject: [PATCH 22/42] remove check for length zero list. --- python/ray/worker.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 3dc9de3426ad..39a1f0c2461c 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2573,8 +2573,6 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): if len(object_ids) != len(set(object_ids)): raise Exception("Wait requires a list of unique object IDs.") - if len(object_ids) == 0: - raise Exception("Invalid number of objects %d." % len(object_ids)) if num_returns <= 0: raise Exception("Invalid number of objects to return %d." % num_returns) if num_returns > len(object_ids): From 0796a1740e4bd02aca94346fe136b97c152130bc Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 11:57:45 -0700 Subject: [PATCH 23/42] remove elapsed. --- src/ray/object_manager/object_manager.cc | 11 ++------- src/ray/object_manager/object_manager.h | 4 +--- .../test/object_manager_test.cc | 23 ++++++++----------- src/ray/raylet/node_manager.cc | 2 +- 4 files changed, 13 insertions(+), 27 deletions(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index dd9fd60941e2..a6df60195421 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -325,14 +325,13 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t RAY_CHECK(num_required_objects != 0); RAY_CHECK(num_required_objects <= object_ids.size()); if (object_ids.size() == 0) { - callback(0, std::unordered_set(), std::unordered_set()); + callback(std::unordered_set(), std::unordered_set()); } // Initialize fields. active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); auto &wait_state = active_wait_requests_.find(wait_id)->second; wait_state.num_required_objects = num_required_objects; - wait_state.start_time = boost::posix_time::second_clock::local_time(); for (auto &oid : object_ids) { if (local_objects_.count(oid) > 0) { wait_state.found.insert(oid); @@ -411,12 +410,6 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { // The timer handler will be given a non-zero error code. The handler // will do nothing on non-zero error codes. wait_state.timeout_timer->cancel(); - // Invoke the wait handler. - int64_t time_taken = - wait_state.wait_ms == 0 - ? 0 - : (boost::posix_time::second_clock::local_time() - wait_state.start_time) - .total_milliseconds(); // Wait semantics require marking at most num_required_objects as found. int64_t num_move = wait_state.found.size() - wait_state.num_required_objects; if (num_move > 0) { @@ -427,7 +420,7 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { iter = wait_state.found.erase(iter); } } - wait_state.callback(time_taken, wait_state.found, wait_state.remaining); + wait_state.callback(wait_state.found, wait_state.remaining); active_wait_requests_.erase(wait_id); } diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index e16b6cb51dff..bebb9cf895b3 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -143,7 +143,7 @@ class ObjectManager : public ObjectManagerInterface { /// Callback definition for wait. using WaitCallback = - std::function &found, + std::function &found, const std::unordered_set &remaining)>; /// Wait until either num_required_objects are located or wait_ms has elapsed, /// then invoke the provided callback. @@ -219,8 +219,6 @@ class ObjectManager : public ObjectManagerInterface { std::unordered_set requested_objects; /// The number of required objects. uint64_t num_required_objects; - /// The time at which the wait operation begins. - boost::posix_time::ptime start_time; }; /// A set of active wait requests. diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index 46361e2ddddf..d27ce4def592 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -274,42 +274,37 @@ class TestObjectManagerCommands : public TestObjectManager { num_objects += 1; object_ids.push_back(ObjectID::from_random()); } + boost::posix_time::ptime start_time = boost::posix_time::second_clock::local_time(); server1->object_manager_.Wait( object_ids, wait_ms, required_objects, false, - [this, num_objects, wait_ms, required_objects]( - int64_t elapsed, const std::unordered_set &found, + [this, num_objects, wait_ms, required_objects, start_time]( + const std::unordered_set &found, const std::unordered_set &remaining) { + int64_t elapsed = (boost::posix_time::second_clock::local_time() - start_time).total_milliseconds(); + RAY_LOG(DEBUG) << "elapsed " << elapsed; + RAY_LOG(DEBUG) << "found " << found.size(); + RAY_LOG(DEBUG) << "remaining " << remaining.size(); switch (current_wait_test) { case 0: { - // Ensure wait_ms = 0 returns immediately after lookup when objects are - // remote. - ASSERT_TRUE(elapsed == 0); + // Ensure wait_ms = 0 returns expected number of found / remaining objects. + ASSERT_TRUE(found.size() <= required_objects); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); NextWaitTest(); } break; case 1: { // Ensure lookup succeeds as expected when objects are local. - RAY_LOG(DEBUG) << "elapsed " << elapsed; - RAY_LOG(DEBUG) << "found " << found.size(); - RAY_LOG(DEBUG) << "remaining " << remaining.size(); ASSERT_TRUE(found.size() >= required_objects); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); NextWaitTest(); } break; case 2: { // Ensure lookup succeeds as expected when objects are remote. - RAY_LOG(DEBUG) << "elapsed " << elapsed; - RAY_LOG(DEBUG) << "found " << found.size(); - RAY_LOG(DEBUG) << "remaining " << remaining.size(); ASSERT_TRUE(found.size() >= required_objects); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); NextWaitTest(); } break; case 3: { // Ensure lookup returns after wait_ms elapses when one object doesn't exist. - RAY_LOG(DEBUG) << "elapsed " << elapsed; - RAY_LOG(DEBUG) << "found " << found.size(); - RAY_LOG(DEBUG) << "remaining " << remaining.size(); ASSERT_TRUE(elapsed >= wait_ms); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); TestWaitComplete(); diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 7ec264d97895..7c9a3b476ff0 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -463,7 +463,7 @@ void NodeManager::ProcessClientMessage( ray::Status status = object_manager_.Wait( object_ids, wait_ms, num_required_objects, wait_local, - [this, client](int64_t time_taken, std::unordered_set found, + [this, client](std::unordered_set found, std::unordered_set remaining) { // Write the data. std::vector found_vec(found.begin(), found.end()); From dd9f0dbae08160c6e69e2f9e5a4f84cf9186d5ff Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 13:46:38 -0700 Subject: [PATCH 24/42] Preserve input/output order. --- python/ray/worker.py | 4 ++ src/ray/object_manager/object_manager.cc | 37 ++++++++++++++----- src/ray/object_manager/object_manager.h | 7 ++-- .../test/object_manager_test.cc | 35 ++++++++++++++++-- src/ray/raylet/node_manager.cc | 7 +--- 5 files changed, 69 insertions(+), 21 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 39a1f0c2461c..8b4c5e65faec 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2529,6 +2529,10 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): correspond to objects that are stored in the object store. The second list corresponds to the rest of the object IDs (which may or may not be ready). + Ordering of the input list of object IDs is preserved: if A precedes B in + the input list, and both are in the ready list, then A will precede B in + the ready list. + Args: object_ids (List[ObjectID]): List of object IDs for objects that may or may not be ready. Note that these IDs must be unique. diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index a6df60195421..4b94e7f69cc0 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -325,12 +325,13 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t RAY_CHECK(num_required_objects != 0); RAY_CHECK(num_required_objects <= object_ids.size()); if (object_ids.size() == 0) { - callback(std::unordered_set(), std::unordered_set()); + callback(std::vector(), std::vector()); } // Initialize fields. active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); auto &wait_state = active_wait_requests_.find(wait_id)->second; + wait_state.object_id_order = object_ids; wait_state.num_required_objects = num_required_objects; for (auto &oid : object_ids) { if (local_objects_.count(oid) > 0) { @@ -340,6 +341,8 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t } } + // TODO: Debias returning local objects first by doing a lookup on remaining + // objects before subscribing. if (wait_state.found.size() >= wait_state.num_required_objects) { // Requirements already satisfied. WaitComplete(wait_id); @@ -411,16 +414,32 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { // will do nothing on non-zero error codes. wait_state.timeout_timer->cancel(); // Wait semantics require marking at most num_required_objects as found. - int64_t num_move = wait_state.found.size() - wait_state.num_required_objects; - if (num_move > 0) { - auto iter = wait_state.found.begin(); - while (num_move > 0) { - num_move -= 1; - wait_state.remaining.insert(*iter); - iter = wait_state.found.erase(iter); + + // int64_t num_move = wait_state.found.size() - wait_state.num_required_objects; + // if (num_move > 0) { + // auto iter = wait_state.found.begin(); + // while (num_move > 0) { + // num_move -= 1; + // wait_state.remaining.insert(*iter); + // iter = wait_state.found.erase(iter); + // } + // } + // std::vector found_vec(wait_state.found.begin(), wait_state.found.end()); + // std::vector remaining_vec(wait_state.remaining.begin(), + // wait_state.remaining.end()); + + std::vector found_vec; + std::vector remaining_vec; + + for (auto item : wait_state.object_id_order) { + if (found_vec.size() < wait_state.num_required_objects && + wait_state.found.count(item) > 0) { + found_vec.push_back(item); + } else { + remaining_vec.push_back(item); } } - wait_state.callback(wait_state.found, wait_state.remaining); + wait_state.callback(found_vec, remaining_vec); active_wait_requests_.erase(wait_id); } diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index bebb9cf895b3..c2e5e297cee2 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -142,9 +142,8 @@ class ObjectManager : public ObjectManagerInterface { ray::Status Cancel(const ObjectID &object_id); /// Callback definition for wait. - using WaitCallback = - std::function &found, - const std::unordered_set &remaining)>; + using WaitCallback = std::function &found, + const std::vector &remaining)>; /// Wait until either num_required_objects are located or wait_ms has elapsed, /// then invoke the provided callback. /// @@ -211,6 +210,8 @@ class ObjectManager : public ObjectManagerInterface { std::unique_ptr timeout_timer; /// The callback invoked when WaitCallback is complete. WaitCallback callback; + /// Ordered input object_ids. + std::vector object_id_order; /// The objects that have not yet been found. std::unordered_set remaining; /// The objects that have been found. diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index d27ce4def592..ba7e401d9687 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -277,13 +277,40 @@ class TestObjectManagerCommands : public TestObjectManager { boost::posix_time::ptime start_time = boost::posix_time::second_clock::local_time(); server1->object_manager_.Wait( object_ids, wait_ms, required_objects, false, - [this, num_objects, wait_ms, required_objects, start_time]( - const std::unordered_set &found, - const std::unordered_set &remaining) { - int64_t elapsed = (boost::posix_time::second_clock::local_time() - start_time).total_milliseconds(); + [this, object_ids, num_objects, wait_ms, required_objects, start_time]( + const std::vector &found, + const std::vector &remaining) { + int64_t elapsed = (boost::posix_time::second_clock::local_time() - start_time) + .total_milliseconds(); RAY_LOG(DEBUG) << "elapsed " << elapsed; RAY_LOG(DEBUG) << "found " << found.size(); RAY_LOG(DEBUG) << "remaining " << remaining.size(); + + // Ensure object order is preserved for all invocations. + uint j = 0; + uint k = 0; + for (uint i = 0; i < object_ids.size(); ++i) { + ObjectID oid = object_ids[i]; + // Make sure the object is in either the found vector or the remaining vector. + if (!((!found.empty() && found[j] == oid) || + (!remaining.empty() && remaining[k] == oid))) { + // If we're at the end of both arrays, then an object is missing. + ASSERT_TRUE(j < found.size() || k < remaining.size()); + if (j < found.size() && oid == found[j + 1]) { + j += 1; + } + if (k < remaining.size() && oid == remaining[k + 1]) { + k += 1; + } + } + } + if (!found.empty()) { + ASSERT_EQ(j + 1, found.size()); + } + if (!remaining.empty()) { + ASSERT_EQ(k + 1, remaining.size()); + } + switch (current_wait_test) { case 0: { // Ensure wait_ms = 0 returns expected number of found / remaining objects. diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 7c9a3b476ff0..3342c886360b 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -463,14 +463,11 @@ void NodeManager::ProcessClientMessage( ray::Status status = object_manager_.Wait( object_ids, wait_ms, num_required_objects, wait_local, - [this, client](std::unordered_set found, - std::unordered_set remaining) { + [this, client](std::vector found, std::vector remaining) { // Write the data. - std::vector found_vec(found.begin(), found.end()); - std::vector remaining_vec(remaining.begin(), remaining.end()); flatbuffers::FlatBufferBuilder fbb; flatbuffers::Offset wait_reply = protocol::CreateWaitReply( - fbb, to_flatbuf(fbb, found_vec), to_flatbuf(fbb, remaining_vec)); + fbb, to_flatbuf(fbb, found), to_flatbuf(fbb, remaining)); fbb.Finish(wait_reply); RAY_CHECK_OK(client->WriteMessage(protocol::MessageType_WaitReply, fbb.GetSize(), fbb.GetBufferPointer())); From 9d4ed2b36e8c7fbc85f13f607da6dcdad7e568b8 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 14:00:09 -0700 Subject: [PATCH 25/42] debias local objects. --- src/ray/object_manager/object_manager.cc | 138 +++++++++++------------ src/ray/object_manager/object_manager.h | 2 + 2 files changed, 66 insertions(+), 74 deletions(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 4b94e7f69cc0..19f043c2165a 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -332,6 +332,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); auto &wait_state = active_wait_requests_.find(wait_id)->second; wait_state.object_id_order = object_ids; + wait_state.wait_ms = wait_ms; wait_state.num_required_objects = num_required_objects; for (auto &oid : object_ids) { if (local_objects_.count(oid) > 0) { @@ -341,66 +342,70 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t } } - // TODO: Debias returning local objects first by doing a lookup on remaining - // objects before subscribing. - if (wait_state.found.size() >= wait_state.num_required_objects) { - // Requirements already satisfied. + if (wait_state.remaining.empty()) { WaitComplete(wait_id); } else { - if (wait_ms == 0) { - for (auto &oid : wait_state.remaining) { - // Lookup remaining objects. - wait_state.requested_objects.insert(oid); - RAY_CHECK_OK(object_directory_->LookupLocations( - oid, [this, wait_id](const std::vector &client_ids, - const ObjectID &object_id) { - auto &wait_state = active_wait_requests_.find(wait_id)->second; - if (!client_ids.empty()) { - wait_state.remaining.erase(object_id); - wait_state.found.insert(object_id); - } - wait_state.requested_objects.erase(object_id); - if (wait_state.requested_objects.empty()) { - WaitComplete(wait_id); - } - })); - } - } else { - for (auto &oid : wait_state.remaining) { - // Subscribe to object notifications. - wait_state.requested_objects.insert(oid); - RAY_CHECK_OK(object_directory_->SubscribeObjectLocations( - wait_id, oid, [this, wait_id](const std::vector &client_ids, - const ObjectID &object_id) { - auto &wait_state = active_wait_requests_.find(wait_id)->second; - if (wait_state.remaining.count(object_id) != 0) { - wait_state.remaining.erase(object_id); - wait_state.found.insert(object_id); - } - wait_state.requested_objects.erase(object_id); - RAY_CHECK_OK( - object_directory_->UnsubscribeObjectLocations(wait_id, object_id)); - if (wait_state.found.size() >= wait_state.num_required_objects) { - WaitComplete(wait_id); - } - })); - } - // Set timeout. - // TODO (hme): If we need to just wait for all objects independent of time - // (i.e. infinite wait time), determine what the value of wait_ms should be and - // skip this call. WaitComplete will be invoked when all objects have locations. - wait_state.timeout_timer->async_wait( - [this, wait_id](const boost::system::error_code &error_code) { - if (error_code.value() != 0) { - return; + for (auto &oid : wait_state.remaining) { + // Lookup remaining objects. + wait_state.requested_objects.insert(oid); + RAY_CHECK_OK(object_directory_->LookupLocations( + oid, [this, wait_id](const std::vector &client_ids, + const ObjectID &object_id) { + auto &wait_state = active_wait_requests_.find(wait_id)->second; + if (!client_ids.empty()) { + wait_state.remaining.erase(object_id); + wait_state.found.insert(object_id); } - WaitComplete(wait_id); - }); + wait_state.requested_objects.erase(object_id); + if (wait_state.requested_objects.empty()) { + WaitLookupComplete(wait_id); + } + })); } } return ray::Status::OK(); } +void ObjectManager::WaitLookupComplete(const UniqueID &wait_id) { + auto &wait_state = active_wait_requests_.find(wait_id)->second; + if (wait_state.found.size() >= wait_state.num_required_objects || + wait_state.wait_ms == 0) { + // Requirements already satisfied. + WaitComplete(wait_id); + } else { + for (auto &oid : wait_state.remaining) { + // Subscribe to object notifications. + wait_state.requested_objects.insert(oid); + RAY_CHECK_OK(object_directory_->SubscribeObjectLocations( + wait_id, oid, [this, wait_id](const std::vector &client_ids, + const ObjectID &object_id) { + auto &wait_state = active_wait_requests_.find(wait_id)->second; + if (wait_state.remaining.count(object_id) != 0) { + wait_state.remaining.erase(object_id); + wait_state.found.insert(object_id); + } + wait_state.requested_objects.erase(object_id); + RAY_CHECK_OK( + object_directory_->UnsubscribeObjectLocations(wait_id, object_id)); + if (wait_state.found.size() >= wait_state.num_required_objects) { + WaitComplete(wait_id); + } + })); + } + // Set timeout. + // TODO (hme): If we need to just wait for all objects independent of time + // (i.e. infinite wait time), determine what the value of wait_ms should be and + // skip this call. WaitComplete will be invoked when all objects have locations. + wait_state.timeout_timer->async_wait( + [this, wait_id](const boost::system::error_code &error_code) { + if (error_code.value() != 0) { + return; + } + WaitComplete(wait_id); + }); + } +} + void ObjectManager::WaitComplete(const UniqueID &wait_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; // If we complete with outstanding requests, then wait_ms should be non-zero. @@ -413,33 +418,18 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { // The timer handler will be given a non-zero error code. The handler // will do nothing on non-zero error codes. wait_state.timeout_timer->cancel(); - // Wait semantics require marking at most num_required_objects as found. - - // int64_t num_move = wait_state.found.size() - wait_state.num_required_objects; - // if (num_move > 0) { - // auto iter = wait_state.found.begin(); - // while (num_move > 0) { - // num_move -= 1; - // wait_state.remaining.insert(*iter); - // iter = wait_state.found.erase(iter); - // } - // } - // std::vector found_vec(wait_state.found.begin(), wait_state.found.end()); - // std::vector remaining_vec(wait_state.remaining.begin(), - // wait_state.remaining.end()); - - std::vector found_vec; - std::vector remaining_vec; - + // Order objects according to input order. + std::vector found; + std::vector remaining; for (auto item : wait_state.object_id_order) { - if (found_vec.size() < wait_state.num_required_objects && + if (found.size() < wait_state.num_required_objects && wait_state.found.count(item) > 0) { - found_vec.push_back(item); + found.push_back(item); } else { - remaining_vec.push_back(item); + remaining.push_back(item); } } - wait_state.callback(found_vec, remaining_vec); + wait_state.callback(found, remaining); active_wait_requests_.erase(wait_id); } diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index c2e5e297cee2..a5ecd81be6a2 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -225,6 +225,8 @@ class ObjectManager : public ObjectManagerInterface { /// A set of active wait requests. std::unordered_map active_wait_requests_; + /// Invoked when lookup for remaining objects has been invoked. + void WaitLookupComplete(const UniqueID &wait_id); /// Completion handler for Wait. void WaitComplete(const UniqueID &wait_id); From 58af7390137a3c26a1832ad66f5299b1c07c5708 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 14:34:43 -0700 Subject: [PATCH 26/42] use common helper function in object directory. --- src/ray/object_manager/object_directory.cc | 74 +++++++++++----------- src/ray/object_manager/object_directory.h | 2 +- 2 files changed, 37 insertions(+), 39 deletions(-) diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index eaa5aaa9eb84..d1d6d8e3da55 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -6,10 +6,31 @@ ObjectDirectory::ObjectDirectory(std::shared_ptr &gcs_clien gcs_client_ = gcs_client; } +std::vector UpdateObjectLocations( + std::unordered_set &client_ids, + const std::vector &location_history) { + // location_history contains the history of locations of the object (it is a log), + // which might look like the following: + // client1.is_eviction = false + // client1.is_eviction = true + // client2.is_eviction = false + // In such a scenario, we want to indicate client2 is the only client that contains + // the object, which the following code achieves. + for (const auto &object_table_data : location_history) { + ClientID client_id = ClientID::from_binary(object_table_data.manager); + if (!object_table_data.is_eviction) { + client_ids.insert(client_id); + } else { + client_ids.erase(client_id); + } + } + return std::vector(client_ids.begin(), client_ids.end()); +} + void ObjectDirectory::RegisterBackend() { auto object_notification_callback = [this]( gcs::AsyncGcsClient *client, const ObjectID &object_id, - const std::vector &object_location_ids) { + const std::vector &location_history) { // Objects are added to this map in SubscribeObjectLocations. auto object_id_listener_pair = listeners_.find(object_id); // Do nothing for objects we are not listening for. @@ -17,25 +38,9 @@ void ObjectDirectory::RegisterBackend() { return; } // Update entries for this object. - auto &location_client_id_set = object_id_listener_pair->second.location_client_ids; - // object_location_ids contains the history of locations of the object (it is a log), - // which might look like the following: - // client1.is_eviction = false - // client1.is_eviction = true - // client2.is_eviction = false - // In such a scenario, we want to indicate client2 is the only client that contains - // the object, which the following code achieves. - for (const auto &object_table_data : object_location_ids) { - ClientID client_id = ClientID::from_binary(object_table_data.manager); - if (!object_table_data.is_eviction) { - location_client_id_set.insert(client_id); - } else { - location_client_id_set.erase(client_id); - } - } - if (!location_client_id_set.empty()) { - std::vector client_id_vec(location_client_id_set.begin(), - location_client_id_set.end()); + std::vector client_id_vec = UpdateObjectLocations( + object_id_listener_pair->second.current_object_locations, location_history); + if (!client_id_vec.empty()) { // Copy the callbacks so that the callbacks can unsubscribe without interrupting // looping over the callbacks. auto callbacks = object_id_listener_pair->second.callbacks; @@ -45,9 +50,9 @@ void ObjectDirectory::RegisterBackend() { } } }; - RAY_CHECK_OK(gcs_client_->object_table().Subscribe(UniqueID::nil(), - gcs_client_->client_table().GetLocalClientId(), - object_notification_callback, nullptr)); + RAY_CHECK_OK(gcs_client_->object_table().Subscribe( + UniqueID::nil(), gcs_client_->client_table().GetLocalClientId(), + object_notification_callback, nullptr)); } ray::Status ObjectDirectory::ReportObjectAdded(const ObjectID &object_id, @@ -112,9 +117,10 @@ ray::Status ObjectDirectory::SubscribeObjectLocations(const UniqueID &callback_i } listeners_[object_id].callbacks.emplace(callback_id, callback); // Immediately notify of found object locations. - if (!listeners_[object_id].location_client_ids.empty()) { - std::vector client_id_vec(listeners_[object_id].location_client_ids.begin(), - listeners_[object_id].location_client_ids.end()); + if (!listeners_[object_id].current_object_locations.empty()) { + std::vector client_id_vec( + listeners_[object_id].current_object_locations.begin(), + listeners_[object_id].current_object_locations.end()); callback(client_id_vec, object_id); } return status; @@ -142,19 +148,11 @@ ray::Status ObjectDirectory::LookupLocations(const ObjectID &object_id, ray::Status status = gcs_client_->object_table().Lookup( job_id, object_id, [this, callback](gcs::AsyncGcsClient *client, const ObjectID &object_id, - const std::vector &location_entries) { + const std::vector &location_history) { // Build the set of current locations based on the entries in the log. - std::unordered_set locations; - for (auto entry : location_entries) { - ClientID client_id = ClientID::from_binary(entry.manager); - if (!entry.is_eviction) { - locations.insert(client_id); - } else { - locations.erase(client_id); - } - } - // Invoke the callback. - std::vector locations_vector(locations.begin(), locations.end()); + std::unordered_set client_ids; + std::vector locations_vector = + UpdateObjectLocations(client_ids, location_history); callback(locations_vector, object_id); }); return status; diff --git a/src/ray/object_manager/object_directory.h b/src/ray/object_manager/object_directory.h index 755ff7f82529..4e395c3eb6c1 100644 --- a/src/ray/object_manager/object_directory.h +++ b/src/ray/object_manager/object_directory.h @@ -136,7 +136,7 @@ class ObjectDirectory : public ObjectDirectoryInterface { /// The callback to invoke when object locations are found. std::unordered_map callbacks; /// The current set of known locations of this object. - std::unordered_set location_client_ids; + std::unordered_set current_object_locations; }; /// Info about subscribers to object locations. From d9ef29bbd8d2cf5002e91ca456c48accc4cbf8fa Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 14:42:20 -0700 Subject: [PATCH 27/42] updated documentation --- src/ray/object_manager/object_manager.cc | 16 ++++++++-------- src/ray/object_manager/object_manager.h | 10 +++++----- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 72bedb244e9b..5ca0fff45128 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -88,10 +88,10 @@ void ObjectManager::NotifyDirectoryObjectAdd(const ObjectInfoT &object_info) { local_objects_[object_id] = object_info; ray::Status status = object_directory_->ReportObjectAdded(object_id, client_id_, object_info); - // Handle the unfulfilled_push_tasks_ which contains the push request that is not + // Handle the unfulfilled_push_requests_ which contains the push request that is not // completed due to unsatisfied local objects. - auto iter = unfulfilled_push_tasks_.find(object_id); - if (iter != unfulfilled_push_tasks_.end()) { + auto iter = unfulfilled_push_requests_.find(object_id); + if (iter != unfulfilled_push_requests_.end()) { for (auto &pair : iter->second) { auto &client_id = pair.first; main_service_->post( @@ -101,7 +101,7 @@ void ObjectManager::NotifyDirectoryObjectAdd(const ObjectInfoT &object_info) { pair.second->cancel(); } } - unfulfilled_push_tasks_.erase(iter); + unfulfilled_push_requests_.erase(iter); } } @@ -214,19 +214,19 @@ void ObjectManager::HandlePushTaskTimeout(const ObjectID &object_id, const ClientID &client_id) { RAY_LOG(WARNING) << "Invalid Push request ObjectID: " << object_id << " after waiting for " << config_.push_timeout_ms << " ms."; - auto iter = unfulfilled_push_tasks_.find(object_id); - RAY_CHECK(iter != unfulfilled_push_tasks_.end()); + auto iter = unfulfilled_push_requests_.find(object_id); + RAY_CHECK(iter != unfulfilled_push_requests_.end()); uint num_erased = iter->second.erase(client_id); RAY_CHECK(num_erased == 1); if (iter->second.size() == 0) { - unfulfilled_push_tasks_.erase(iter); + unfulfilled_push_requests_.erase(iter); } } ray::Status ObjectManager::Push(const ObjectID &object_id, const ClientID &client_id) { if (local_objects_.count(object_id) == 0) { // Avoid setting duplicated timer for the same object and client pair. - auto &clients = unfulfilled_push_tasks_[object_id]; + auto &clients = unfulfilled_push_requests_[object_id]; if (clients.count(client_id) == 0) { // If config_.push_timeout_ms < 0, we give an empty timer // and the task will be kept infinitely. diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index e6b4fa78cdff..b83f286a5521 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -226,18 +226,18 @@ class ObjectManager : public ObjectManagerInterface { /// A set of active wait requests. std::unordered_map active_wait_requests_; - - /// Invoked when lookup for remaining objects has been invoked. + /// Invoked when lookup for remaining objects has been invoked. This method subscribes + /// to any remaining objects if wait conditions have not yet been satisfied. void WaitLookupComplete(const UniqueID &wait_id); /// Completion handler for Wait. void WaitComplete(const UniqueID &wait_id); - /// Unfulfilled Push tasks. - /// The timer is for removing a push task due to unsatisfied local object. + /// Maintains a map of push requests that have not been fulfilled due to an object not + /// being local. Objects are removed from this map after push_timeout_ms have elapsed. std::unordered_map< ObjectID, std::unordered_map>> - unfulfilled_push_tasks_; + unfulfilled_push_requests_; /// Handle starting, running, and stopping asio io_service. void StartIOService(); From fa1928b5cbba462d4b5255db30def6563b25c009 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 14:47:51 -0700 Subject: [PATCH 28/42] linting. --- python/ray/worker.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 8b4c5e65faec..8d0696403226 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2578,7 +2578,8 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): if len(object_ids) != len(set(object_ids)): raise Exception("Wait requires a list of unique object IDs.") if num_returns <= 0: - raise Exception("Invalid number of objects to return %d." % num_returns) + raise Exception( + "Invalid number of objects to return %d." % num_returns) if num_returns > len(object_ids): raise Exception("num_returns cannot be greater than the number " "of objects provided to ray.wait.") From d41b1d0b4eb4eff58800cb07f3badc20caae4c9b Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 15:58:02 -0700 Subject: [PATCH 29/42] handle return status. --- src/ray/object_manager/test/object_manager_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index 8d275bd55924..721dc22ab059 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -309,7 +309,7 @@ class TestObjectManagerCommands : public TestObjectManager { object_ids.push_back(ObjectID::from_random()); } boost::posix_time::ptime start_time = boost::posix_time::second_clock::local_time(); - server1->object_manager_.Wait( + RAY_CHECK_OK(server1->object_manager_.Wait( object_ids, wait_ms, required_objects, false, [this, object_ids, num_objects, wait_ms, required_objects, start_time]( const std::vector &found, @@ -371,7 +371,7 @@ class TestObjectManagerCommands : public TestObjectManager { TestWaitComplete(); } break; } - }); + })); } void TestWaitComplete() { main_service.stop(); } From aeaab5b36b51c4a18e4583e7cfce1565fd267d7a Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 16:21:11 -0700 Subject: [PATCH 30/42] simplify order preservation test + fix valgrind test error. --- .../test/object_manager_test.cc | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index 721dc22ab059..5df03eaf344f 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -326,23 +326,18 @@ class TestObjectManagerCommands : public TestObjectManager { for (uint i = 0; i < object_ids.size(); ++i) { ObjectID oid = object_ids[i]; // Make sure the object is in either the found vector or the remaining vector. - if (!((!found.empty() && found[j] == oid) || - (!remaining.empty() && remaining[k] == oid))) { - // If we're at the end of both arrays, then an object is missing. - ASSERT_TRUE(j < found.size() || k < remaining.size()); - if (j < found.size() && oid == found[j + 1]) { - j += 1; - } - if (k < remaining.size() && oid == remaining[k + 1]) { - k += 1; - } + if (j < found.size() && found[j] == oid) { + j += 1; + } + if (k < remaining.size() && remaining[k] == oid) { + k += 1; } } if (!found.empty()) { - ASSERT_EQ(j + 1, found.size()); + ASSERT_EQ(j, found.size()); } if (!remaining.empty()) { - ASSERT_EQ(k + 1, remaining.size()); + ASSERT_EQ(k, remaining.size()); } switch (current_wait_test) { From 048f45ff4b11e6e874a8ca1d4e484f43d4102cff Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 17:15:56 -0700 Subject: [PATCH 31/42] update name of final Lookup callback. remove c-style casts where possible. update worker.py documentation. --- python/ray/worker.py | 2 +- src/local_scheduler/lib/python/local_scheduler_extension.cc | 4 ++-- src/ray/object_manager/object_manager.cc | 4 ++-- src/ray/object_manager/object_manager.h | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 8d0696403226..7a8a5f4e839d 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2531,7 +2531,7 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): Ordering of the input list of object IDs is preserved: if A precedes B in the input list, and both are in the ready list, then A will precede B in - the ready list. + the ready list. This also holds true if A and B are both in the remaining list. Args: object_ids (List[ObjectID]): List of object IDs for objects that may or diff --git a/src/local_scheduler/lib/python/local_scheduler_extension.cc b/src/local_scheduler/lib/python/local_scheduler_extension.cc index 4d09e82c02ee..23d9747a79f8 100644 --- a/src/local_scheduler/lib/python/local_scheduler_extension.cc +++ b/src/local_scheduler/lib/python/local_scheduler_extension.cc @@ -215,11 +215,11 @@ static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { object_ids, num_returns, timeout_ms, static_cast(wait_local)); // Convert result to py object. - PyObject *py_found = PyList_New((Py_ssize_t) result.first.size()); + PyObject *py_found = PyList_New(static_cast(result.first.size())); for (uint i = 0; i < result.first.size(); ++i) { PyList_SetItem(py_found, i, PyObjectID_make(result.first[i])); } - PyObject *py_remaining = PyList_New((Py_ssize_t) result.second.size()); + PyObject *py_remaining = PyList_New(static_cast(result.second.size())); for (uint i = 0; i < result.second.size(); ++i) { PyList_SetItem(py_remaining, i, PyObjectID_make(result.second[i])); } diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 5ca0fff45128..488277e37070 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -401,7 +401,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t } wait_state.requested_objects.erase(object_id); if (wait_state.requested_objects.empty()) { - WaitLookupComplete(wait_id); + AllWaitLookupsComplete(wait_id); } })); } @@ -409,7 +409,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t return ray::Status::OK(); } -void ObjectManager::WaitLookupComplete(const UniqueID &wait_id) { +void ObjectManager::AllWaitLookupsComplete(const UniqueID &wait_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; if (wait_state.found.size() >= wait_state.num_required_objects || wait_state.wait_ms == 0) { diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index b83f286a5521..b7545ae87a8c 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -228,7 +228,7 @@ class ObjectManager : public ObjectManagerInterface { std::unordered_map active_wait_requests_; /// Invoked when lookup for remaining objects has been invoked. This method subscribes /// to any remaining objects if wait conditions have not yet been satisfied. - void WaitLookupComplete(const UniqueID &wait_id); + void AllWaitLookupsComplete(const UniqueID &wait_id); /// Completion handler for Wait. void WaitComplete(const UniqueID &wait_id); From 833939f3d0131c47c99efb294be4f8fa6e8f1826 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 17:28:13 -0700 Subject: [PATCH 32/42] linting --- src/local_scheduler/lib/python/local_scheduler_extension.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/local_scheduler/lib/python/local_scheduler_extension.cc b/src/local_scheduler/lib/python/local_scheduler_extension.cc index 23d9747a79f8..94824dcb774e 100644 --- a/src/local_scheduler/lib/python/local_scheduler_extension.cc +++ b/src/local_scheduler/lib/python/local_scheduler_extension.cc @@ -219,7 +219,8 @@ static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { for (uint i = 0; i < result.first.size(); ++i) { PyList_SetItem(py_found, i, PyObjectID_make(result.first[i])); } - PyObject *py_remaining = PyList_New(static_cast(result.second.size())); + PyObject *py_remaining = + PyList_New(static_cast(result.second.size())); for (uint i = 0; i < result.second.size(); ++i) { PyList_SetItem(py_remaining, i, PyObjectID_make(result.second[i])); } From 8e1947c6809ee6c97f066f66b7ee5f59788eb389 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Fri, 1 Jun 2018 17:48:31 -0700 Subject: [PATCH 33/42] c++ style casting. --- src/local_scheduler/lib/python/local_scheduler_extension.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/local_scheduler/lib/python/local_scheduler_extension.cc b/src/local_scheduler/lib/python/local_scheduler_extension.cc index 94824dcb774e..3845fc89bf70 100644 --- a/src/local_scheduler/lib/python/local_scheduler_extension.cc +++ b/src/local_scheduler/lib/python/local_scheduler_extension.cc @@ -211,7 +211,7 @@ static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { // Invoke wait. std::pair, std::vector> result = local_scheduler_wait( - ((PyLocalSchedulerClient *) self)->local_scheduler_connection, + reinterpret_cast(self)->local_scheduler_connection, object_ids, num_returns, timeout_ms, static_cast(wait_local)); // Convert result to py object. From 83d04ddce084e4252f8b5199440c70f3e672bbca Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Sat, 2 Jun 2018 12:26:53 -0700 Subject: [PATCH 34/42] linting. --- .../lib/python/local_scheduler_extension.cc | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/local_scheduler/lib/python/local_scheduler_extension.cc b/src/local_scheduler/lib/python/local_scheduler_extension.cc index 3845fc89bf70..482d74dea0af 100644 --- a/src/local_scheduler/lib/python/local_scheduler_extension.cc +++ b/src/local_scheduler/lib/python/local_scheduler_extension.cc @@ -210,9 +210,10 @@ static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { // Invoke wait. std::pair, std::vector> result = - local_scheduler_wait( - reinterpret_cast(self)->local_scheduler_connection, - object_ids, num_returns, timeout_ms, static_cast(wait_local)); + local_scheduler_wait(reinterpret_cast(self) + ->local_scheduler_connection, + object_ids, num_returns, timeout_ms, + static_cast(wait_local)); // Convert result to py object. PyObject *py_found = PyList_New(static_cast(result.first.size())); From 080282f12528ceb3bbb54323799ab317b707f39f Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Sat, 2 Jun 2018 13:51:52 -0700 Subject: [PATCH 35/42] linting. --- python/ray/worker.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/worker.py b/python/ray/worker.py index 7a8a5f4e839d..2a5e3fc84b36 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -2531,7 +2531,8 @@ def wait(object_ids, num_returns=1, timeout=None, worker=global_worker): Ordering of the input list of object IDs is preserved: if A precedes B in the input list, and both are in the ready list, then A will precede B in - the ready list. This also holds true if A and B are both in the remaining list. + the ready list. This also holds true if A and B are both in the remaining + list. Args: object_ids (List[ObjectID]): List of object IDs for objects that may or From a58f5c914586a4587e3c0db729b979d364950926 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Mon, 4 Jun 2018 17:33:37 -0700 Subject: [PATCH 36/42] incorporate second round of feedback. --- src/local_scheduler/local_scheduler_client.cc | 8 +- src/local_scheduler/local_scheduler_client.h | 5 +- src/ray/object_manager/object_directory.cc | 12 +-- src/ray/object_manager/object_manager.cc | 88 ++++++++++--------- src/ray/object_manager/object_manager.h | 15 ++-- .../test/object_manager_test.cc | 24 +++-- src/ray/raylet/node_manager.cc | 8 +- 7 files changed, 90 insertions(+), 70 deletions(-) diff --git a/src/local_scheduler/local_scheduler_client.cc b/src/local_scheduler/local_scheduler_client.cc index a148cf0b5789..ddbf9a7580e4 100644 --- a/src/local_scheduler/local_scheduler_client.cc +++ b/src/local_scheduler/local_scheduler_client.cc @@ -192,12 +192,13 @@ std::pair, std::vector> local_scheduler_wait( LocalSchedulerConnection *conn, const std::vector &object_ids, int num_returns, - int64_t timeout, + int64_t timeout_milliseconds, bool wait_local) { // Write request. flatbuffers::FlatBufferBuilder fbb; auto message = ray::protocol::CreateWaitRequest( - fbb, to_flatbuf(fbb, object_ids), num_returns, timeout, wait_local); + fbb, to_flatbuf(fbb, object_ids), num_returns, timeout_milliseconds, + wait_local); fbb.Finish(message); write_message(conn->conn, ray::protocol::MessageType_WaitRequest, fbb.GetSize(), fbb.GetBufferPointer()); @@ -206,6 +207,7 @@ std::pair, std::vector> local_scheduler_wait( int64_t reply_size; uint8_t *reply; read_message(conn->conn, &type, &reply_size, &reply); + RAY_CHECK(type == ray::protocol::MessageType_WaitReply); auto reply_message = flatbuffers::GetRoot(reply); // Convert result. std::pair, std::vector> result; @@ -219,5 +221,7 @@ std::pair, std::vector> local_scheduler_wait( ObjectID object_id = ObjectID::from_binary(remaining->Get(i)->str()); result.second.push_back(object_id); } + /* Free the original message from the local scheduler. */ + free(reply); return result; } diff --git a/src/local_scheduler/local_scheduler_client.h b/src/local_scheduler/local_scheduler_client.h index 4291bbe6e3df..ed6f3916e2ed 100644 --- a/src/local_scheduler/local_scheduler_client.h +++ b/src/local_scheduler/local_scheduler_client.h @@ -175,7 +175,8 @@ void local_scheduler_set_actor_frontier(LocalSchedulerConnection *conn, /// \param conn The connection information. /// \param object_ids The objects to wait for. /// \param num_returns The number of objects to wait for. -/// \param timeout The duration to wait before returning. +/// \param timeout_milliseconds Duration, in milliseconds, to wait before +/// returning. /// \param wait_local Whether to wait for objects to appear on this node. /// \return A pair with the first element containing the object ids that were /// found, and the second element the objects that were not found. @@ -183,7 +184,7 @@ std::pair, std::vector> local_scheduler_wait( LocalSchedulerConnection *conn, const std::vector &object_ids, int num_returns, - int64_t timeout, + int64_t timeout_milliseconds, bool wait_local); #endif diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index d1d6d8e3da55..59ba6b29334d 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -112,15 +112,17 @@ ray::Status ObjectDirectory::SubscribeObjectLocations(const UniqueID &callback_i status = gcs_client_->object_table().RequestNotifications( JobID::nil(), object_id, gcs_client_->client_table().GetLocalClientId()); } - if (listeners_[object_id].callbacks.count(callback_id) > 0) { + auto &listener_state = listeners_.find(object_id)->second; + // TODO(hme): Make this fatal after implementing Pull suppression. + if (listener_state.callbacks.count(callback_id) > 0) { return ray::Status::OK(); } - listeners_[object_id].callbacks.emplace(callback_id, callback); + listener_state.callbacks.emplace(callback_id, callback); // Immediately notify of found object locations. - if (!listeners_[object_id].current_object_locations.empty()) { + if (!listener_state.current_object_locations.empty()) { std::vector client_id_vec( - listeners_[object_id].current_object_locations.begin(), - listeners_[object_id].current_object_locations.end()); + listener_state.current_object_locations.begin(), + listener_state.current_object_locations.end()); callback(client_id_vec, object_id); } return status; diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 488277e37070..38de325e73e7 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -355,16 +355,16 @@ ray::Status ObjectManager::Cancel(const ObjectID &object_id) { return status; } -ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t wait_ms, - uint64_t num_required_objects, bool wait_local, - const WaitCallback &callback) { +ray::Status ObjectManager::Wait(const std::vector &object_ids, + int64_t timeout_ms, uint64_t num_required_objects, + bool wait_local, const WaitCallback &callback) { UniqueID wait_id = UniqueID::from_random(); if (wait_local) { return ray::Status::NotImplemented("Wait for local objects is not yet implemented."); } - RAY_CHECK(wait_ms >= 0); + RAY_CHECK(timeout_ms >= 0 || timeout_ms == -1); RAY_CHECK(num_required_objects != 0); RAY_CHECK(num_required_objects <= object_ids.size()); if (object_ids.size() == 0) { @@ -372,34 +372,39 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t } // Initialize fields. - active_wait_requests_.emplace(wait_id, WaitState(*main_service_, wait_ms, callback)); + active_wait_requests_.emplace(wait_id, WaitState(*main_service_, timeout_ms, callback)); auto &wait_state = active_wait_requests_.find(wait_id)->second; wait_state.object_id_order = object_ids; - wait_state.wait_ms = wait_ms; + wait_state.timeout_ms = timeout_ms; wait_state.num_required_objects = num_required_objects; - for (auto &oid : object_ids) { - if (local_objects_.count(oid) > 0) { - wait_state.found.insert(oid); + for (auto &object_id : object_ids) { + if (local_objects_.count(object_id) > 0) { + wait_state.found.insert(object_id); } else { - wait_state.remaining.insert(oid); + wait_state.remaining.insert(object_id); } } if (wait_state.remaining.empty()) { WaitComplete(wait_id); } else { - for (auto &oid : wait_state.remaining) { + // We invoke lookup calls immediately after checking which objects are local to + // obtain current information about the location of remote objects. Thus, + // we obtain information about all given objects, regardless of their location. + // This is required to ensure we do not bias returning locally available objects + // as ready whenever Wait is invoked with a mixture of local and remote objects. + for (auto &object_id : wait_state.remaining) { // Lookup remaining objects. - wait_state.requested_objects.insert(oid); + wait_state.requested_objects.insert(object_id); RAY_CHECK_OK(object_directory_->LookupLocations( - oid, [this, wait_id](const std::vector &client_ids, - const ObjectID &object_id) { + object_id, [this, wait_id](const std::vector &client_ids, + const ObjectID &lookup_object_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; if (!client_ids.empty()) { - wait_state.remaining.erase(object_id); - wait_state.found.insert(object_id); + wait_state.remaining.erase(lookup_object_id); + wait_state.found.insert(lookup_object_id); } - wait_state.requested_objects.erase(object_id); + wait_state.requested_objects.erase(lookup_object_id); if (wait_state.requested_objects.empty()) { AllWaitLookupsComplete(wait_id); } @@ -412,47 +417,46 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t void ObjectManager::AllWaitLookupsComplete(const UniqueID &wait_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; if (wait_state.found.size() >= wait_state.num_required_objects || - wait_state.wait_ms == 0) { + wait_state.timeout_ms == 0) { // Requirements already satisfied. WaitComplete(wait_id); } else { - for (auto &oid : wait_state.remaining) { + for (auto &object_id : wait_state.remaining) { // Subscribe to object notifications. - wait_state.requested_objects.insert(oid); + wait_state.requested_objects.insert(object_id); RAY_CHECK_OK(object_directory_->SubscribeObjectLocations( - wait_id, oid, [this, wait_id](const std::vector &client_ids, - const ObjectID &object_id) { + wait_id, object_id, [this, wait_id](const std::vector &client_ids, + const ObjectID &subscribe_object_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; - if (wait_state.remaining.count(object_id) != 0) { - wait_state.remaining.erase(object_id); - wait_state.found.insert(object_id); - } - wait_state.requested_objects.erase(object_id); - RAY_CHECK_OK( - object_directory_->UnsubscribeObjectLocations(wait_id, object_id)); + RAY_CHECK(wait_state.remaining.erase(subscribe_object_id)); + wait_state.found.insert(subscribe_object_id); + wait_state.requested_objects.erase(subscribe_object_id); + RAY_CHECK_OK(object_directory_->UnsubscribeObjectLocations( + wait_id, subscribe_object_id)); if (wait_state.found.size() >= wait_state.num_required_objects) { WaitComplete(wait_id); } })); } - // Set timeout. - // TODO (hme): If we need to just wait for all objects independent of time - // (i.e. infinite wait time), determine what the value of wait_ms should be and - // skip this call. WaitComplete will be invoked when all objects have locations. - wait_state.timeout_timer->async_wait( - [this, wait_id](const boost::system::error_code &error_code) { - if (error_code.value() != 0) { - return; - } - WaitComplete(wait_id); - }); + if (wait_state.timeout_ms != -1) { + wait_state.timeout_timer->async_wait( + [this, wait_id](const boost::system::error_code &error_code) { + if (error_code.value() != 0) { + return; + } + WaitComplete(wait_id); + }); + } } } void ObjectManager::WaitComplete(const UniqueID &wait_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; - // If we complete with outstanding requests, then wait_ms should be non-zero. - RAY_CHECK(!(wait_state.requested_objects.size() > 0) || wait_state.wait_ms > 0); + // If we complete with outstanding requests, then timeout_ms should be non-zero or -1 + // (infinite wait time). + if (!wait_state.requested_objects.empty()) { + RAY_CHECK(wait_state.timeout_ms > 0 || wait_state.timeout_ms == -1); + } // Unsubscribe to any objects that weren't found in the time allotted. for (auto &object_id : wait_state.requested_objects) { RAY_CHECK_OK(object_directory_->UnsubscribeObjectLocations(wait_id, object_id)); diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index b7545ae87a8c..5a6be5883fb9 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -157,7 +157,7 @@ class ObjectManager : public ObjectManagerInterface { /// \param callback Invoked when either timeout_ms is satisfied OR num_ready_objects /// is satisfied. /// \return Status of whether the wait successfully initiated. - ray::Status Wait(const std::vector &object_ids, int64_t wait_ms, + ray::Status Wait(const std::vector &object_ids, int64_t timeout_ms, uint64_t num_required_objects, bool wait_local, const WaitCallback &callback); @@ -197,17 +197,20 @@ class ObjectManager : public ObjectManagerInterface { /// Cache of locally available objects. std::unordered_map local_objects_; + /// This is used as the callback identifier in Pull for + /// SubscribeObjectLocations. We only need one identifier because we never need to + /// subscribe multiple times to the same object during Pull. UniqueID object_directory_pull_callback_id_ = UniqueID::from_random(); struct WaitState { - WaitState(asio::io_service &service, int64_t wait_ms, const WaitCallback &callback) - : wait_ms(wait_ms), + WaitState(asio::io_service &service, int64_t timeout_ms, const WaitCallback &callback) + : timeout_ms(timeout_ms), timeout_timer(std::unique_ptr( - new boost::asio::deadline_timer(service, - boost::posix_time::milliseconds(wait_ms)))), + new boost::asio::deadline_timer( + service, boost::posix_time::milliseconds(timeout_ms)))), callback(callback) {} /// The period of time to wait before invoking the callback. - int64_t wait_ms; + int64_t timeout_ms; /// The timer used whenever wait_ms > 0. std::unique_ptr timeout_timer; /// The callback invoked when WaitCallback is complete. diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index 5df03eaf344f..bbb5176ed34c 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -278,6 +278,8 @@ class TestObjectManagerCommands : public TestObjectManager { current_wait_test += 1; switch (current_wait_test) { case 0: { + // Simple test with timeout = 0 to ensure timeout_ms = 0 is handled correctly. + // Out of 5 objects, we expect 3 ready objects and 2 remaining objects. TestWait(100, 5, 3, 0, false, false); } break; case 1: { @@ -289,10 +291,13 @@ class TestObjectManagerCommands : public TestObjectManager { case 3: { TestWait(100, 5, 6, 1000, true, false); } break; + case 4: { + TestWait(100, 5, 5, -1, false, false); + } break; } } - void TestWait(int data_size, int num_objects, uint64_t required_objects, int wait_ms, + void TestWait(int data_size, int num_objects, uint64_t required_objects, int timeout_ms, bool include_nonexistent, bool test_local) { std::vector object_ids; for (int i = -1; ++i < num_objects;) { @@ -310,8 +315,8 @@ class TestObjectManagerCommands : public TestObjectManager { } boost::posix_time::ptime start_time = boost::posix_time::second_clock::local_time(); RAY_CHECK_OK(server1->object_manager_.Wait( - object_ids, wait_ms, required_objects, false, - [this, object_ids, num_objects, wait_ms, required_objects, start_time]( + object_ids, timeout_ms, required_objects, false, + [this, object_ids, num_objects, timeout_ms, required_objects, start_time]( const std::vector &found, const std::vector &remaining) { int64_t elapsed = (boost::posix_time::second_clock::local_time() - start_time) @@ -342,7 +347,7 @@ class TestObjectManagerCommands : public TestObjectManager { switch (current_wait_test) { case 0: { - // Ensure wait_ms = 0 returns expected number of found / remaining objects. + // Ensure timeout_ms = 0 returns expected number of found / remaining objects. ASSERT_TRUE(found.size() <= required_objects); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); NextWaitTest(); @@ -360,9 +365,16 @@ class TestObjectManagerCommands : public TestObjectManager { NextWaitTest(); } break; case 3: { - // Ensure lookup returns after wait_ms elapses when one object doesn't exist. - ASSERT_TRUE(elapsed >= wait_ms); + // Ensure lookup returns after timeout_ms elapses when one object doesn't + // exist. + ASSERT_TRUE(elapsed >= timeout_ms); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); + NextWaitTest(); + } break; + case 4: { + // Ensure timeout_ms = -1 works properly. + ASSERT_TRUE(static_cast(found.size()) == num_objects); + ASSERT_TRUE(remaining.size() == 0); TestWaitComplete(); } break; } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index e34d68f0fea1..0975e9f9447f 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -451,13 +451,7 @@ void NodeManager::ProcessClientMessage( case protocol::MessageType_WaitRequest: { // Read the data. auto message = flatbuffers::GetRoot(message_data); - auto object_id_strings = message->object_ids(); - int num_requests = object_id_strings->size(); - std::vector object_ids; - for (int i = 0; i < num_requests; i++) { - ObjectID object_id = ObjectID::from_binary(object_id_strings->Get(i)->str()); - object_ids.push_back(object_id); - } + std::vector object_ids = from_flatbuf(*message->object_ids()); int64_t wait_ms = message->timeout(); uint64_t num_required_objects = static_cast(message->num_ready_objects()); bool wait_local = message->wait_local(); From c6d8ba5946bcef791227dbd700db396afc70af76 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Tue, 5 Jun 2018 13:35:07 -0700 Subject: [PATCH 37/42] correct python tests. Add test for ObjectManager.Wait during subscribe to a common same object. --- src/ray/object_manager/object_manager.cc | 14 +++++- src/ray/object_manager/object_manager.h | 2 + .../test/object_manager_test.cc | 49 +++++++++++++++++-- test/runtest.py | 26 ++++++++-- 4 files changed, 84 insertions(+), 7 deletions(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 38de325e73e7..0f329eac8903 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -421,12 +421,24 @@ void ObjectManager::AllWaitLookupsComplete(const UniqueID &wait_id) { // Requirements already satisfied. WaitComplete(wait_id); } else { - for (auto &object_id : wait_state.remaining) { + // Subscribe to objects in order to ensure Wait-related tests are deterministic. + for (auto &object_id : wait_state.object_id_order) { + if (wait_state.remaining.count(object_id) == 0) { + continue; + } // Subscribe to object notifications. + if (active_wait_requests_.find(wait_id) == active_wait_requests_.end()) { + // This is possible if an object's location is obtained immediately, + // within the current callstack. In this case, WaitComplete has been + // invoked already, so we're done. + return; + } wait_state.requested_objects.insert(object_id); RAY_CHECK_OK(object_directory_->SubscribeObjectLocations( wait_id, object_id, [this, wait_id](const std::vector &client_ids, const ObjectID &subscribe_object_id) { + auto object_id_wait_state = active_wait_requests_.find(wait_id); + RAY_CHECK(object_id_wait_state != active_wait_requests_.end()); auto &wait_state = active_wait_requests_.find(wait_id)->second; RAY_CHECK(wait_state.remaining.erase(subscribe_object_id)); wait_state.found.insert(subscribe_object_id); diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index 5a6be5883fb9..a0697743d0b6 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -162,6 +162,8 @@ class ObjectManager : public ObjectManagerInterface { const WaitCallback &callback); private: + friend class TestObjectManager; + ClientID client_id_; const ObjectManagerConfig config_; std::unique_ptr object_directory_; diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index bbb5176ed34c..25cdff48f6db 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -70,6 +70,7 @@ class MockServer { DoAcceptObjectManager(); } + friend class TestObjectManager; friend class TestObjectManagerCommands; boost::asio::ip::tcp::acceptor object_manager_acceptor_; @@ -124,7 +125,6 @@ class TestObjectManager : public ::testing::Test { om_config_1.max_sends = max_sends; om_config_1.max_receives = max_receives; om_config_1.object_chunk_size = object_chunk_size; - // Push will stop immediately if local object is not satisfied. om_config_1.push_timeout_ms = push_timeout_ms; server1.reset(new MockServer(main_service, om_config_1, gcs_client_1)); @@ -136,7 +136,6 @@ class TestObjectManager : public ::testing::Test { om_config_2.max_sends = max_sends; om_config_2.max_receives = max_receives; om_config_2.object_chunk_size = object_chunk_size; - // Push will wait infinitely until local object is satisfied. om_config_2.push_timeout_ms = push_timeout_ms; server2.reset(new MockServer(main_service, om_config_2, gcs_client_2)); @@ -177,6 +176,10 @@ class TestObjectManager : public ::testing::Test { void object_added_handler_2(ObjectID object_id) { v2.push_back(object_id); }; + ObjectDirectoryInterface &get_object_directory(const MockServer &server) { + return *server.object_manager_.object_directory_; + } + protected: std::thread p; boost::asio::io_service main_service; @@ -270,10 +273,50 @@ class TestObjectManagerCommands : public TestObjectManager { uint num_expected_objects1 = 1; uint num_expected_objects2 = 2; if (v1.size() == num_expected_objects1 && v2.size() == num_expected_objects2) { - NextWaitTest(); + TestWaitCallbacks1(); } } + void TestWaitCallbacks1() { + int data_size = 100; + // Test to ensure Wait works properly during an active subscription to the same + // object. + ObjectID object_1 = WriteDataToClient(client2, data_size); + ObjectID object_2 = WriteDataToClient(client2, data_size); + UniqueID sub_id = ray::ObjectID::from_random(); + + get_object_directory(*server1).SubscribeObjectLocations( + sub_id, object_1, + [this, sub_id, object_1, object_2](const std::vector &, + const ray::ObjectID &object_id) { + TestWaitCallbacks2(sub_id, object_1, object_2); + }); + } + + void TestWaitCallbacks2(UniqueID sub_id, ObjectID object_1, ObjectID object_2) { + int num_objects = 2; + int required_objects = 1; + int timeout_ms = 1000; + + std::vector object_ids = {object_1, object_2}; + boost::posix_time::ptime start_time = boost::posix_time::second_clock::local_time(); + RAY_CHECK_OK(server1->object_manager_.Wait( + object_ids, timeout_ms, required_objects, false, + [this, sub_id, object_1, object_ids, num_objects, start_time]( + const std::vector &found, + const std::vector &remaining) { + int64_t elapsed = (boost::posix_time::second_clock::local_time() - start_time) + .total_milliseconds(); + RAY_LOG(INFO) << "elapsed " << elapsed; + RAY_LOG(INFO) << "found " << found.size(); + RAY_LOG(INFO) << "remaining " << remaining.size(); + // There's nothing more to test. The process will exit if something goes wrong. + RAY_CHECK(found.size() == 1); + get_object_directory(*server1).UnsubscribeObjectLocations(sub_id, object_1); + NextWaitTest(); + })); + } + void NextWaitTest() { current_wait_test += 1; switch (current_wait_test) { diff --git a/test/runtest.py b/test/runtest.py index 83211ef87ced..e9d05b77c4ba 100644 --- a/test/runtest.py +++ b/test/runtest.py @@ -850,9 +850,29 @@ def f(delay): with self.assertRaises(TypeError): ray.wait([1]) - @unittest.skipIf( - os.environ.get("RAY_USE_XRAY") == "1", - "This test does not work with xray yet.") + def testWaitIterables(self): + self.init_ray(num_cpus=1) + + @ray.remote + def f(delay): + time.sleep(delay) + return 1 + + objectids = (f.remote(1.0), f.remote(0.5), f.remote(0.5), + f.remote(0.5)) + ready_ids, remaining_ids = ray.experimental.wait(objectids) + self.assertEqual(len(ready_ids), 1) + self.assertEqual(len(remaining_ids), 3) + + objectids = np.array( + [f.remote(1.0), + f.remote(0.5), + f.remote(0.5), + f.remote(0.5)]) + ready_ids, remaining_ids = ray.experimental.wait(objectids) + self.assertEqual(len(ready_ids), 1) + self.assertEqual(len(remaining_ids), 3) + def testMultipleWaitsAndGets(self): # It is important to use three workers here, so that the three tasks # launched in this experiment can run at the same time. From 7d8d756ed57c953e454e0413b59c74e97320b82f Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Tue, 5 Jun 2018 13:48:10 -0700 Subject: [PATCH 38/42] test comments. --- .../test/object_manager_test.cc | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index 25cdff48f6db..e6c00d74f8be 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -307,11 +307,12 @@ class TestObjectManagerCommands : public TestObjectManager { const std::vector &remaining) { int64_t elapsed = (boost::posix_time::second_clock::local_time() - start_time) .total_milliseconds(); - RAY_LOG(INFO) << "elapsed " << elapsed; - RAY_LOG(INFO) << "found " << found.size(); - RAY_LOG(INFO) << "remaining " << remaining.size(); - // There's nothing more to test. The process will exit if something goes wrong. + RAY_LOG(DEBUG) << "elapsed " << elapsed; + RAY_LOG(DEBUG) << "found " << found.size(); + RAY_LOG(DEBUG) << "remaining " << remaining.size(); RAY_CHECK(found.size() == 1); + // There's nothing more to test. A check will fail if unexpected behavior is + // triggered. get_object_directory(*server1).UnsubscribeObjectLocations(sub_id, object_1); NextWaitTest(); })); @@ -321,21 +322,29 @@ class TestObjectManagerCommands : public TestObjectManager { current_wait_test += 1; switch (current_wait_test) { case 0: { - // Simple test with timeout = 0 to ensure timeout_ms = 0 is handled correctly. + // Ensure timeout_ms = 0 is handled correctly. // Out of 5 objects, we expect 3 ready objects and 2 remaining objects. - TestWait(100, 5, 3, 0, false, false); + TestWait(100, 5, 3, /*timeout_ms=*/0, false, false); } break; case 1: { - TestWait(100, 5, 3, 1000, false, true); + // Ensure timeout_ms = 1000 is handled correctly. + // Out of 5 objects, we expect 3 ready objects and 2 remaining objects. + TestWait(100, 5, 3, /*timeout_ms=*/1000, false, false); } break; case 2: { - TestWait(100, 5, 3, 1000, false, false); + // Generate objects locally to ensure local object code-path works properly. + // Out of 5 objects, we expect 3 ready objects and 2 remaining objects. + TestWait(100, 5, 3, 1000, false, /*test_local=*/true); } break; case 3: { - TestWait(100, 5, 6, 1000, true, false); + // Wait on an object that's never registered with GCS to ensure timeout works + // properly. + TestWait(100, /*num_objects=*/5, /*required_objects=*/6, 1000, + /*include_nonexistent=*/true, false); } break; case 4: { - TestWait(100, 5, 5, -1, false, false); + // Ensure infinite time code-path works properly. + TestWait(100, 5, 5, /*timeout_ms=*/-1, false, false); } break; } } @@ -396,13 +405,13 @@ class TestObjectManagerCommands : public TestObjectManager { NextWaitTest(); } break; case 1: { - // Ensure lookup succeeds as expected when objects are local. + // Ensure lookup succeeds as expected when timeout_ms = 1000. ASSERT_TRUE(found.size() >= required_objects); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); NextWaitTest(); } break; case 2: { - // Ensure lookup succeeds as expected when objects are remote. + // Ensure lookup succeeds as expected when objects are local. ASSERT_TRUE(found.size() >= required_objects); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); NextWaitTest(); From 6b6e2f3518b45846ba198d5e2f9b7839dcd7c52e Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Tue, 5 Jun 2018 17:14:26 -0700 Subject: [PATCH 39/42] incorporate reviews. --- .../lib/python/local_scheduler_extension.cc | 9 ++++++--- src/ray/object_manager/object_directory.cc | 5 ++--- src/ray/object_manager/object_manager.cc | 10 +++++----- src/ray/object_manager/test/object_manager_test.cc | 7 ++++--- 4 files changed, 17 insertions(+), 14 deletions(-) diff --git a/src/local_scheduler/lib/python/local_scheduler_extension.cc b/src/local_scheduler/lib/python/local_scheduler_extension.cc index 482d74dea0af..89ae40259c18 100644 --- a/src/local_scheduler/lib/python/local_scheduler_extension.cc +++ b/src/local_scheduler/lib/python/local_scheduler_extension.cc @@ -183,12 +183,15 @@ static PyObject *PyLocalSchedulerClient_wait(PyObject *self, PyObject *args) { PyObject *py_object_ids; int num_returns; int64_t timeout_ms; - int wait_local; + PyObject *py_wait_local; - if (!PyArg_ParseTuple(args, "Oili", &py_object_ids, &num_returns, &timeout_ms, - &wait_local)) { + if (!PyArg_ParseTuple(args, "OilO", &py_object_ids, &num_returns, &timeout_ms, + &py_wait_local)) { return NULL; } + + bool wait_local = PyObject_IsTrue(py_wait_local); + // Convert object ids. PyObject *iter = PyObject_GetIter(py_object_ids); if (!iter) { diff --git a/src/ray/object_manager/object_directory.cc b/src/ray/object_manager/object_directory.cc index 59ba6b29334d..f98630cfc74c 100644 --- a/src/ray/object_manager/object_directory.cc +++ b/src/ray/object_manager/object_directory.cc @@ -120,9 +120,8 @@ ray::Status ObjectDirectory::SubscribeObjectLocations(const UniqueID &callback_i listener_state.callbacks.emplace(callback_id, callback); // Immediately notify of found object locations. if (!listener_state.current_object_locations.empty()) { - std::vector client_id_vec( - listener_state.current_object_locations.begin(), - listener_state.current_object_locations.end()); + std::vector client_id_vec(listener_state.current_object_locations.begin(), + listener_state.current_object_locations.end()); callback(client_id_vec, object_id); } return status; diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 0f329eac8903..4ac6e1fe1b4c 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -377,7 +377,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, wait_state.object_id_order = object_ids; wait_state.timeout_ms = timeout_ms; wait_state.num_required_objects = num_required_objects; - for (auto &object_id : object_ids) { + for (const auto &object_id : object_ids) { if (local_objects_.count(object_id) > 0) { wait_state.found.insert(object_id); } else { @@ -393,7 +393,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, // we obtain information about all given objects, regardless of their location. // This is required to ensure we do not bias returning locally available objects // as ready whenever Wait is invoked with a mixture of local and remote objects. - for (auto &object_id : wait_state.remaining) { + for (const auto &object_id : wait_state.remaining) { // Lookup remaining objects. wait_state.requested_objects.insert(object_id); RAY_CHECK_OK(object_directory_->LookupLocations( @@ -422,7 +422,7 @@ void ObjectManager::AllWaitLookupsComplete(const UniqueID &wait_id) { WaitComplete(wait_id); } else { // Subscribe to objects in order to ensure Wait-related tests are deterministic. - for (auto &object_id : wait_state.object_id_order) { + for (const auto &object_id : wait_state.object_id_order) { if (wait_state.remaining.count(object_id) == 0) { continue; } @@ -470,7 +470,7 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { RAY_CHECK(wait_state.timeout_ms > 0 || wait_state.timeout_ms == -1); } // Unsubscribe to any objects that weren't found in the time allotted. - for (auto &object_id : wait_state.requested_objects) { + for (const auto &object_id : wait_state.requested_objects) { RAY_CHECK_OK(object_directory_->UnsubscribeObjectLocations(wait_id, object_id)); } // Cancel the timer. This is okay even if the timer hasn't been started. @@ -480,7 +480,7 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { // Order objects according to input order. std::vector found; std::vector remaining; - for (auto item : wait_state.object_id_order) { + for (const auto item : wait_state.object_id_order) { if (found.size() < wait_state.num_required_objects && wait_state.found.count(item) > 0) { found.push_back(item); diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index e6c00d74f8be..fa57bbb49dc4 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -285,12 +285,12 @@ class TestObjectManagerCommands : public TestObjectManager { ObjectID object_2 = WriteDataToClient(client2, data_size); UniqueID sub_id = ray::ObjectID::from_random(); - get_object_directory(*server1).SubscribeObjectLocations( + RAY_CHECK_OK(get_object_directory(*server1).SubscribeObjectLocations( sub_id, object_1, [this, sub_id, object_1, object_2](const std::vector &, const ray::ObjectID &object_id) { TestWaitCallbacks2(sub_id, object_1, object_2); - }); + })); } void TestWaitCallbacks2(UniqueID sub_id, ObjectID object_1, ObjectID object_2) { @@ -313,7 +313,8 @@ class TestObjectManagerCommands : public TestObjectManager { RAY_CHECK(found.size() == 1); // There's nothing more to test. A check will fail if unexpected behavior is // triggered. - get_object_directory(*server1).UnsubscribeObjectLocations(sub_id, object_1); + RAY_CHECK_OK(get_object_directory(*server1).UnsubscribeObjectLocations( + sub_id, object_1)); NextWaitTest(); })); } From 3a86c938044c24925b7d34500112bfa81e16345a Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Tue, 5 Jun 2018 18:38:55 -0700 Subject: [PATCH 40/42] Fixes with regression tests. --- src/ray/object_manager/object_manager.cc | 41 +++++++++++++++---- src/ray/object_manager/object_manager.h | 13 +++++- .../test/object_manager_test.cc | 41 ++++++++++--------- 3 files changed, 67 insertions(+), 28 deletions(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 4ac6e1fe1b4c..d51eeb2bcb6a 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -359,7 +359,19 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, int64_t timeout_ms, uint64_t num_required_objects, bool wait_local, const WaitCallback &callback) { UniqueID wait_id = UniqueID::from_random(); + RAY_RETURN_NOT_OK(AddWaitRequest(wait_id, object_ids, timeout_ms, num_required_objects, + wait_local, callback)); + RAY_RETURN_NOT_OK(LookupRemainingWaitObjects(wait_id)); + // LookupRemainingWaitObjects invokes SubscribeRemainingWaitObjects once lookup has + // been performed on all remaining objects. + return ray::Status::OK(); +} +ray::Status ObjectManager::AddWaitRequest(const UniqueID &wait_id, + const std::vector &object_ids, + int64_t timeout_ms, + uint64_t num_required_objects, bool wait_local, + const WaitCallback &callback) { if (wait_local) { return ray::Status::NotImplemented("Wait for local objects is not yet implemented."); } @@ -385,6 +397,12 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, } } + return ray::Status::OK(); +} + +ray::Status ObjectManager::LookupRemainingWaitObjects(const UniqueID &wait_id) { + auto &wait_state = active_wait_requests_.find(wait_id)->second; + if (wait_state.remaining.empty()) { WaitComplete(wait_id); } else { @@ -396,7 +414,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, for (const auto &object_id : wait_state.remaining) { // Lookup remaining objects. wait_state.requested_objects.insert(object_id); - RAY_CHECK_OK(object_directory_->LookupLocations( + RAY_RETURN_NOT_OK(object_directory_->LookupLocations( object_id, [this, wait_id](const std::vector &client_ids, const ObjectID &lookup_object_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; @@ -406,7 +424,7 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, } wait_state.requested_objects.erase(lookup_object_id); if (wait_state.requested_objects.empty()) { - AllWaitLookupsComplete(wait_id); + SubscribeRemainingWaitObjects(wait_id); } })); } @@ -414,19 +432,23 @@ ray::Status ObjectManager::Wait(const std::vector &object_ids, return ray::Status::OK(); } -void ObjectManager::AllWaitLookupsComplete(const UniqueID &wait_id) { +void ObjectManager::SubscribeRemainingWaitObjects(const UniqueID &wait_id) { auto &wait_state = active_wait_requests_.find(wait_id)->second; if (wait_state.found.size() >= wait_state.num_required_objects || wait_state.timeout_ms == 0) { // Requirements already satisfied. WaitComplete(wait_id); } else { - // Subscribe to objects in order to ensure Wait-related tests are deterministic. + // Wait may complete during the execution of any one of the following calls to + // SubscribeObjectLocations, so copy the object ids that need to be iterated over. + // Order matters for test purposes. + std::vector ordered_remaining_object_ids; for (const auto &object_id : wait_state.object_id_order) { - if (wait_state.remaining.count(object_id) == 0) { - continue; + if (wait_state.remaining.count(object_id) > 0) { + ordered_remaining_object_ids.push_back(object_id); } - // Subscribe to object notifications. + } + for (const auto &object_id : ordered_remaining_object_ids) { if (active_wait_requests_.find(wait_id) == active_wait_requests_.end()) { // This is possible if an object's location is obtained immediately, // within the current callstack. In this case, WaitComplete has been @@ -434,12 +456,15 @@ void ObjectManager::AllWaitLookupsComplete(const UniqueID &wait_id) { return; } wait_state.requested_objects.insert(object_id); + // Subscribe to object notifications. RAY_CHECK_OK(object_directory_->SubscribeObjectLocations( wait_id, object_id, [this, wait_id](const std::vector &client_ids, const ObjectID &subscribe_object_id) { auto object_id_wait_state = active_wait_requests_.find(wait_id); + // We never expect to handle a subscription notification for a wait that has + // already completed. RAY_CHECK(object_id_wait_state != active_wait_requests_.end()); - auto &wait_state = active_wait_requests_.find(wait_id)->second; + auto &wait_state = object_id_wait_state->second; RAY_CHECK(wait_state.remaining.erase(subscribe_object_id)); wait_state.found.insert(subscribe_object_id); wait_state.requested_objects.erase(subscribe_object_id); diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index a0697743d0b6..8bb68fcc37ce 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -231,9 +231,20 @@ class ObjectManager : public ObjectManagerInterface { /// A set of active wait requests. std::unordered_map active_wait_requests_; + + /// Creates a wait request and adds it to active_wait_requests_. + ray::Status AddWaitRequest(const UniqueID &wait_id, + const std::vector &object_ids, int64_t timeout_ms, + uint64_t num_required_objects, bool wait_local, + const WaitCallback &callback); + + /// Lookup any remaining objects that are not local. This is invoked after + /// the wait request is created and local objects are identified. + ray::Status LookupRemainingWaitObjects(const UniqueID &wait_id); + /// Invoked when lookup for remaining objects has been invoked. This method subscribes /// to any remaining objects if wait conditions have not yet been satisfied. - void AllWaitLookupsComplete(const UniqueID &wait_id); + void SubscribeRemainingWaitObjects(const UniqueID &wait_id); /// Completion handler for Wait. void WaitComplete(const UniqueID &wait_id); diff --git a/src/ray/object_manager/test/object_manager_test.cc b/src/ray/object_manager/test/object_manager_test.cc index fa57bbb49dc4..0bfc29487919 100644 --- a/src/ray/object_manager/test/object_manager_test.cc +++ b/src/ray/object_manager/test/object_manager_test.cc @@ -71,7 +71,6 @@ class MockServer { } friend class TestObjectManager; - friend class TestObjectManagerCommands; boost::asio::ip::tcp::acceptor object_manager_acceptor_; boost::asio::ip::tcp::socket object_manager_socket_; @@ -79,9 +78,9 @@ class MockServer { ObjectManager object_manager_; }; -class TestObjectManager : public ::testing::Test { +class TestObjectManagerBase : public ::testing::Test { public: - TestObjectManager() {} + TestObjectManagerBase() {} std::string StartStore(const std::string &id) { std::string store_id = "/tmp/store"; @@ -176,10 +175,6 @@ class TestObjectManager : public ::testing::Test { void object_added_handler_2(ObjectID object_id) { v2.push_back(object_id); }; - ObjectDirectoryInterface &get_object_directory(const MockServer &server) { - return *server.object_manager_.object_directory_; - } - protected: std::thread p; boost::asio::io_service main_service; @@ -199,7 +194,7 @@ class TestObjectManager : public ::testing::Test { uint push_timeout_ms; }; -class TestObjectManagerCommands : public TestObjectManager { +class TestObjectManager : public TestObjectManagerBase { public: int current_wait_test = -1; int num_connected_clients = 0; @@ -273,11 +268,11 @@ class TestObjectManagerCommands : public TestObjectManager { uint num_expected_objects1 = 1; uint num_expected_objects2 = 2; if (v1.size() == num_expected_objects1 && v2.size() == num_expected_objects2) { - TestWaitCallbacks1(); + SubscribeObjectThenWait(); } } - void TestWaitCallbacks1() { + void SubscribeObjectThenWait() { int data_size = 100; // Test to ensure Wait works properly during an active subscription to the same // object. @@ -285,23 +280,26 @@ class TestObjectManagerCommands : public TestObjectManager { ObjectID object_2 = WriteDataToClient(client2, data_size); UniqueID sub_id = ray::ObjectID::from_random(); - RAY_CHECK_OK(get_object_directory(*server1).SubscribeObjectLocations( + RAY_CHECK_OK(server1->object_manager_.object_directory_->SubscribeObjectLocations( sub_id, object_1, [this, sub_id, object_1, object_2](const std::vector &, const ray::ObjectID &object_id) { - TestWaitCallbacks2(sub_id, object_1, object_2); + TestWaitWhileSubscribed(sub_id, object_1, object_2); })); } - void TestWaitCallbacks2(UniqueID sub_id, ObjectID object_1, ObjectID object_2) { + void TestWaitWhileSubscribed(UniqueID sub_id, ObjectID object_1, ObjectID object_2) { int num_objects = 2; int required_objects = 1; int timeout_ms = 1000; std::vector object_ids = {object_1, object_2}; boost::posix_time::ptime start_time = boost::posix_time::second_clock::local_time(); - RAY_CHECK_OK(server1->object_manager_.Wait( - object_ids, timeout_ms, required_objects, false, + + UniqueID wait_id = UniqueID::from_random(); + + RAY_CHECK_OK(server1->object_manager_.AddWaitRequest( + wait_id, object_ids, timeout_ms, required_objects, false, [this, sub_id, object_1, object_ids, num_objects, start_time]( const std::vector &found, const std::vector &remaining) { @@ -313,10 +311,14 @@ class TestObjectManagerCommands : public TestObjectManager { RAY_CHECK(found.size() == 1); // There's nothing more to test. A check will fail if unexpected behavior is // triggered. - RAY_CHECK_OK(get_object_directory(*server1).UnsubscribeObjectLocations( - sub_id, object_1)); + RAY_CHECK_OK( + server1->object_manager_.object_directory_->UnsubscribeObjectLocations( + sub_id, object_1)); NextWaitTest(); })); + + // Skip lookups and rely on Subscribe only to test subscribe interaction. + server1->object_manager_.SubscribeRemainingWaitObjects(wait_id); } void NextWaitTest() { @@ -400,7 +402,8 @@ class TestObjectManagerCommands : public TestObjectManager { switch (current_wait_test) { case 0: { - // Ensure timeout_ms = 0 returns expected number of found / remaining objects. + // Ensure timeout_ms = 0 returns expected number of found and remaining + // objects. ASSERT_TRUE(found.size() <= required_objects); ASSERT_TRUE(static_cast(found.size() + remaining.size()) == num_objects); NextWaitTest(); @@ -454,7 +457,7 @@ class TestObjectManagerCommands : public TestObjectManager { } }; -TEST_F(TestObjectManagerCommands, StartTestObjectManagerCommands) { +TEST_F(TestObjectManager, StartTestObjectManager) { auto AsyncStartTests = main_service.wrap([this]() { WaitConnections(); }); AsyncStartTests(); main_service.run(); From 1a99f259933d98c9ede1bb8ee917876ae8bbd952 Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 6 Jun 2018 11:17:26 -0700 Subject: [PATCH 41/42] update documentation. --- src/ray/object_manager/object_directory.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/ray/object_manager/object_directory.h b/src/ray/object_manager/object_directory.h index 4e395c3eb6c1..1cf4323e898e 100644 --- a/src/ray/object_manager/object_directory.h +++ b/src/ray/object_manager/object_directory.h @@ -59,7 +59,9 @@ class ObjectDirectoryInterface { /// Subscribe to be notified of locations (ClientID) of the given object. /// The callback will be invoked whenever locations are obtained for the - /// specified object. + /// specified object. The callback provided to this method may fire immediately, + /// within the call to this method, if any other listener is subscribed to the same + /// object: This occurs when location data for the object has already been obtained. /// /// \param callback_id The id associated with the specified callback. This is /// needed when UnsubscribeObjectLocations is called. From 00eafd71d3ab24016187377ff436d8628435707d Mon Sep 17 00:00:00 2001 From: Melih Elibol Date: Wed, 6 Jun 2018 12:25:42 -0700 Subject: [PATCH 42/42] reference to avoid copy. --- src/ray/object_manager/object_manager.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index d51eeb2bcb6a..5621b15a422a 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -505,7 +505,7 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) { // Order objects according to input order. std::vector found; std::vector remaining; - for (const auto item : wait_state.object_id_order) { + for (const auto &item : wait_state.object_id_order) { if (found.size() < wait_state.num_required_objects && wait_state.found.count(item) > 0) { found.push_back(item);