From 25e1ffaba03798a0886a2b1b2b0058d3b31e46ee Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Wed, 25 Jan 2023 22:24:01 -0800 Subject: [PATCH 001/267] [docs] missing parenthesis (#31950) Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../cluster/vms/user-guides/large-cluster-best-practices.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/doc/source/cluster/vms/user-guides/large-cluster-best-practices.rst b/doc/source/cluster/vms/user-guides/large-cluster-best-practices.rst index 49a031bef4df..64287b25254f 100644 --- a/doc/source/cluster/vms/user-guides/large-cluster-best-practices.rst +++ b/doc/source/cluster/vms/user-guides/large-cluster-best-practices.rst @@ -97,8 +97,8 @@ General recommendations with AWS instance types: * The latest gen GPU is almost always the best bang for your buck (p3 > p2, g4 > g3), for most well designed applications the performance outweighs the - price (the instance price may be higher, but you’ll use the instance for less - time. + price. (The instance price may be higher, but you use the instance for less + time.) * You may want to consider using older instances if you’re doing dev work and won’t actually fully utilize the GPUs though. * If you’re doing training (ML or RL), you should use a P instance. If you’re From 1f3226ef866d1625c1d6d43ac553798dfbd941f5 Mon Sep 17 00:00:00 2001 From: Yi Cheng <74173148+iycheng@users.noreply.github.com> Date: Wed, 25 Jan 2023 22:56:49 -0800 Subject: [PATCH 002/267] [core][scalability] Change ray syncer from unary call to streaming call (#30460) To handle the failure of resource broadcasting, it's hard to do fault tolerance since the status need to be maintained. This PR updated the communication protocol to streaming. There are several things changed for the protocol: Once we received the message, it'll be pushed immediately. But it'll be buffered (512kb), so the cost is not big. If there is no more message or it exceeded the buffer, it'll flush. The PR has been tested with 2k nodes (2 cpus per node) and 14k actors. --- BUILD.bazel | 1 + .../many_nodes_tests/actor_test.py | 15 +- src/mock/ray/common/ray_syncer/ray_syncer.h | 20 +- src/ray/common/id.cc | 3 + src/ray/common/id.h | 4 + src/ray/common/ray_config_def.h | 4 + src/ray/common/ray_syncer/ray_syncer-inl.h | 313 +++++++++--- src/ray/common/ray_syncer/ray_syncer.cc | 478 +++++++----------- src/ray/common/ray_syncer/ray_syncer.h | 87 +--- src/ray/common/test/ray_syncer_test.cc | 388 ++++++++++---- .../common/test/syncer_service_e2e_test.cc | 2 +- src/ray/gcs/gcs_server/gcs_actor_scheduler.cc | 1 - src/ray/gcs/gcs_server/gcs_server.cc | 38 +- src/ray/gcs/gcs_server/gcs_server.h | 4 +- src/ray/protobuf/ray_syncer.proto | 33 +- src/ray/raylet/node_manager.cc | 2 + src/ray/rpc/grpc_client.h | 3 +- src/ray/rpc/grpc_server.cc | 3 +- 18 files changed, 811 insertions(+), 588 deletions(-) diff --git a/BUILD.bazel b/BUILD.bazel index 88088be4788d..82e9a74be901 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -2631,6 +2631,7 @@ cc_test( copts = COPTS, tags = ["team:core"], deps = [ + ":grpc_common_lib", ":ray_common", ":ray_mock", "@com_google_googletest//:gtest", diff --git a/release/nightly_tests/many_nodes_tests/actor_test.py b/release/nightly_tests/many_nodes_tests/actor_test.py index 620db9cefbd8..884647d1c0f3 100644 --- a/release/nightly_tests/many_nodes_tests/actor_test.py +++ b/release/nightly_tests/many_nodes_tests/actor_test.py @@ -16,11 +16,6 @@ def foo(self): return actors -def test_actor_ready(actors): - remaining = [actor.foo.remote() for actor in actors] - ray.get(remaining) - - def parse_script_args(): parser = argparse.ArgumentParser() parser.add_argument("--cpus-per-actor", type=float, default=0.2) @@ -43,7 +38,15 @@ def main(): sleep(10) return actor_ready_start = perf_counter() - test_actor_ready(actors) + total_actors = len(actors) + objs = [actor.foo.remote() for actor in actors] + + while len(objs) != 0: + objs_ready, objs = ray.wait(objs, timeout=10) + print( + f"Status: {total_actors - len(objs)}/{total_actors}, " + f"{perf_counter() - actor_ready_start}" + ) actor_ready_end = perf_counter() actor_ready_time = actor_ready_end - actor_ready_start diff --git a/src/mock/ray/common/ray_syncer/ray_syncer.h b/src/mock/ray/common/ray_syncer/ray_syncer.h index 0f768dab64c3..2ef430420697 100644 --- a/src/mock/ray/common/ray_syncer/ray_syncer.h +++ b/src/mock/ray/common/ray_syncer/ray_syncer.h @@ -43,10 +43,24 @@ class MockReceiverInterface : public ReceiverInterface { namespace ray { namespace syncer { -class MockNodeSyncConnection : public NodeSyncConnection { +class MockRaySyncerBidiReactor : public RaySyncerBidiReactor { public: - using NodeSyncConnection::NodeSyncConnection; - MOCK_METHOD(void, DoSend, (), (override)); + using RaySyncerBidiReactor::RaySyncerBidiReactor; + + MOCK_METHOD(void, Disconnect, (), (override)); + + MOCK_METHOD(bool, + PushToSendingQueue, + (std::shared_ptr), + (override)); +}; + +template +class MockRaySyncerBidiReactorBase : public RaySyncerBidiReactorBase { + public: + using RaySyncerBidiReactorBase::RaySyncerBidiReactorBase; + + MOCK_METHOD(void, Disconnect, (), (override)); }; } // namespace syncer diff --git a/src/ray/common/id.cc b/src/ray/common/id.cc index a7c51ccce55a..444770769568 100644 --- a/src/ray/common/id.cc +++ b/src/ray/common/id.cc @@ -337,4 +337,7 @@ ID_OSTREAM_OPERATOR(ActorID); ID_OSTREAM_OPERATOR(TaskID); ID_OSTREAM_OPERATOR(ObjectID); ID_OSTREAM_OPERATOR(PlacementGroupID); + +const NodeID kGCSNodeID = NodeID::FromBinary(std::string(kUniqueIDSize, 0)); + } // namespace ray diff --git a/src/ray/common/id.h b/src/ray/common/id.h index efe6c8ed9774..a6c753a1de35 100644 --- a/src/ray/common/id.h +++ b/src/ray/common/id.h @@ -561,3 +561,7 @@ DEFINE_UNIQUE_ID(PlacementGroupID); #undef DEFINE_UNIQUE_ID } // namespace std + +namespace ray { +extern const NodeID kGCSNodeID; +} diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index fd01f2bb0471..eb93e48f520d 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -686,6 +686,10 @@ RAY_CONFIG(int64_t, grpc_client_keepalive_time_ms, 300000) /// grpc keepalive timeout for client. RAY_CONFIG(int64_t, grpc_client_keepalive_timeout_ms, 120000) +/// grpc streaming buffer size +/// Set it to 512kb +RAY_CONFIG(int64_t, grpc_stream_buffer_size, 512 * 1024); + /// Whether to use log reporter in event framework RAY_CONFIG(bool, event_log_reporter_enabled, false) diff --git a/src/ray/common/ray_syncer/ray_syncer-inl.h b/src/ray/common/ray_syncer/ray_syncer-inl.h index 506ab14b6d9f..7726bc3da654 100644 --- a/src/ray/common/ray_syncer/ray_syncer-inl.h +++ b/src/ray/common/ray_syncer/ray_syncer-inl.h @@ -79,12 +79,48 @@ class NodeState { cluster_view_; }; -class NodeSyncConnection { +/// This is the base class for the bidi-streaming call and defined the method +/// needed. A bidi-stream for ray syncer needs to support pushing message and +/// disconnect from the remote node. +/// For the implementation, in the constructor, it needs to connect to the remote +/// node and it needs to implement the communication between the two nodes. +/// +/// Please refer to https://github.com/grpc/proposal/blob/master/L67-cpp-callback-api.md +/// for the callback API +/// +// clang-format off +/// For the server side: +/// grpc end (error or request) +/// +---------------------------------------------------------------+ +/// | v +/// +------------+ +-------------+ canceled by client +----------+ +--------+ +--------+ +/// | StartRead | <--> | OnReadDone | -----------------------------> | OnCancel | --> | Finish | --> | OnDone | +/// +------------+ +-------------+ +----------+ +--------+ +--------+ +/// canceled by client ^ ^ +/// +----------------------------------------------+ | +/// | | +/// +------------+ +-------------+ grpc end (error or request) | +/// | StartWrite | <--> | OnWriteDone | --------------------------------------------------+ +/// +------------+ +-------------+ +/// +/// +/// For the client side: +/// +------------+ +-------------+ +------------+ gRPC error or disconnected +--------+ +/// | StartCall | ---> | StartRead | <---> | OnReadDone | ----------------------------> | OnDone | +/// +------------+ +-------------+ +------------+ +--------+ +/// | ^ +/// | | +/// v | +/// +------------+ +-------------+ gRPC error or disconnected | +/// | StartWrite | <--> | OnWriteDone | -------------------------------------------------------+ +/// +------------+ +-------------+ +// clang-format on +class RaySyncerBidiReactor { public: - NodeSyncConnection( - instrumented_io_context &io_context, - std::string remote_node_id, - std::function)> message_processor); + RaySyncerBidiReactor(const std::string &remote_node_id) + : remote_node_id_(remote_node_id) {} + + virtual ~RaySyncerBidiReactor(){}; /// Push a message to the sending queue to be sent later. Some message /// might be dropped if the module think the target node has already got the @@ -94,38 +130,192 @@ class NodeSyncConnection { /// \param message The message to be sent. /// /// \return true if push to queue successfully. - bool PushToSendingQueue(std::shared_ptr message); - - /// Send the message queued. - virtual void DoSend() = 0; - - virtual ~NodeSyncConnection() {} + virtual bool PushToSendingQueue(std::shared_ptr message) = 0; /// Return the remote node id of this connection. const std::string &GetRemoteNodeID() const { return remote_node_id_; } + /// Disconnect will terminate the communication between local and remote node. + /// It also needs to do proper cleanup. + virtual void Disconnect() = 0; + + private: + std::string remote_node_id_; +}; + +/// This class implements the communication between two nodes except the initialization +/// and cleanup. +/// It keeps track of the message received and sent between two nodes and uses that to +/// deduplicate the messages. It also supports the batching for performance purposes. +template +class RaySyncerBidiReactorBase : public RaySyncerBidiReactor, public T { + public: + /// Constructor of RaySyncerBidiReactor. + /// + /// \param io_context The io context for the callback. + /// \param remote_node_id The node id connects to. + /// \param message_processor The callback for the message received. + /// \param cleanup_cb When the connection terminates, it'll be called to cleanup + /// the environment. + RaySyncerBidiReactorBase( + instrumented_io_context &io_context, + const std::string &remote_node_id, + std::function)> message_processor) + : RaySyncerBidiReactor(remote_node_id), + io_context_(io_context), + message_processor_(std::move(message_processor)) {} + + bool PushToSendingQueue(std::shared_ptr message) override { + // Try to filter out the messages the target node already has. + // Usually it'll be the case when the message is generated from the + // target node or it's sent from the target node. + // No need to resend the message sent from a node back. + if (message->node_id() == GetRemoteNodeID()) { + // Skip the message when it's about the node of this connection. + return false; + } + + auto &node_versions = GetNodeComponentVersions(message->node_id()); + if (node_versions[message->message_type()] < message->version()) { + node_versions[message->message_type()] = message->version(); + sending_buffer_[std::make_pair(message->node_id(), message->message_type())] = + std::move(message); + StartSend(); + return true; + } + return false; + } + + virtual ~RaySyncerBidiReactorBase() {} + + void StartPull() { + receiving_message_ = std::make_shared(); + RAY_LOG(DEBUG) << "Start reading: " << NodeID::FromBinary(GetRemoteNodeID()); + StartRead(receiving_message_.get()); + } + + protected: + /// The io context + instrumented_io_context &io_context_; + + private: /// Handle the udpates sent from the remote node. /// /// \param messages The message received. - void ReceiveUpdate(RaySyncMessages messages); + void ReceiveUpdate(std::shared_ptr message) { + auto &node_versions = GetNodeComponentVersions(message->node_id()); + RAY_LOG(DEBUG) << "Receive update: " + << " message_type=" << message->message_type() + << ", message_version=" << message->version() + << ", local_message_version=" + << node_versions[message->message_type()]; + if (node_versions[message->message_type()] < message->version()) { + node_versions[message->message_type()] = message->version(); + message_processor_(message); + } else { + RAY_LOG_EVERY_N(WARNING, 100) + << "Drop message received from " << NodeID::FromBinary(message->node_id()) + << " because the message version " << message->version() + << " is older than the local version " << node_versions[message->message_type()] + << ". Message type: " << message->message_type(); + } + } + + void SendNext() { + sending_ = false; + StartSend(); + } + + void StartSend() { + if (sending_) { + return; + } + + if (sending_buffer_.size() != 0) { + auto iter = sending_buffer_.begin(); + auto msg = std::move(iter->second); + sending_buffer_.erase(iter); + Send(std::move(msg), sending_buffer_.empty()); + sending_ = true; + } + } + + /// Sending a message to the remote node + /// + /// \param message The message to be sent + /// \param flush Whether to flush the sending queue in gRPC. + void Send(std::shared_ptr message, bool flush) { + sending_message_ = std::move(message); + grpc::WriteOptions opts; + if (flush) { + opts.clear_buffer_hint(); + } else { + opts.set_buffer_hint(); + } + RAY_LOG(DEBUG) << "[BidiReactor] Sending message to " + << NodeID::FromBinary(GetRemoteNodeID()) << " about node " + << NodeID::FromBinary(sending_message_->node_id()); + StartWrite(sending_message_.get(), opts); + } + + // Please refer to grpc callback api for the following four methods: + // https://github.com/grpc/proposal/blob/master/L67-cpp-callback-api.md + using T::StartRead; + using T::StartWrite; + + void OnWriteDone(bool ok) override { + if (ok) { + io_context_.dispatch([this]() { SendNext(); }, ""); + } else { + // No need to resent the message since if ok=false, it's the end + // of gRPC call and client will reconnect in case of a failure. + // In gRPC, OnDone will be called after. + RAY_LOG_EVERY_N(ERROR, 100) + << "Failed to send the message to: " << NodeID::FromBinary(GetRemoteNodeID()); + } + } + + void OnReadDone(bool ok) override { + if (ok) { + io_context_.dispatch( + [this, msg = std::move(receiving_message_)]() mutable { + RAY_CHECK(!msg->node_id().empty()); + ReceiveUpdate(std::move(msg)); + StartPull(); + }, + ""); + } else { + // No need to resent the message since if ok=false, it's the end + // of gRPC call and client will reconnect in case of a failure. + // In gRPC, OnDone will be called after. + RAY_LOG_EVERY_N(ERROR, 100) + << "Failed to read the message from: " << NodeID::FromBinary(GetRemoteNodeID()); + } + } + + /// grpc requests for sending and receiving + std::shared_ptr sending_message_; + std::shared_ptr receiving_message_; - protected: // For testing - FRIEND_TEST(RaySyncerTest, NodeSyncConnection); + FRIEND_TEST(RaySyncerTest, RaySyncerBidiReactorBase); friend struct SyncerServerTest; std::array &GetNodeComponentVersions( - const std::string &node_id); - - /// The io context - instrumented_io_context &io_context_; - - /// The remote node id. - std::string remote_node_id_; + const std::string &node_id) { + auto iter = node_versions_.find(node_id); + if (iter == node_versions_.end()) { + iter = node_versions_.emplace(node_id, std::array()) + .first; + iter->second.fill(-1); + } + return iter->second; + } /// Handler of a message update. - std::function)> message_processor_; + const std::function)> message_processor_; + private: /// Buffering all the updates. Sending will be done in an async way. absl::flat_hash_map, std::shared_ptr> @@ -136,58 +326,63 @@ class NodeSyncConnection { /// We'll filter the received or sent messages when the message is stale. absl::flat_hash_map> node_versions_; + + bool sending_ = false; }; -/// SyncConnection for gRPC server side. It has customized logic for sending. -class ServerSyncConnection : public NodeSyncConnection { +/// Reactor for gRPC server side. It defines the server's specific behavior for a +/// streaming call. +class RayServerBidiReactor : public RaySyncerBidiReactorBase { public: - ServerSyncConnection( + RayServerBidiReactor( + grpc::CallbackServerContext *server_context, instrumented_io_context &io_context, - const std::string &remote_node_id, - std::function)> message_processor); + const std::string &local_node_id, + std::function)> message_processor, + std::function cleanup_cb); - ~ServerSyncConnection() override; + ~RayServerBidiReactor() override = default; - void HandleLongPollingRequest(grpc::ServerUnaryReactor *reactor, - RaySyncMessages *response); + void Disconnect() override; - protected: - /// Send the message from the pending queue to the target node. - /// It'll send nothing unless there is a long-polling request. - /// TODO (iycheng): Unify the sending algorithm when we migrate to gRPC streaming - void DoSend() override; - - /// These two fields are RPC related. When the server got long-polling requests, - /// these two fields will be set so that it can be used to send message. - /// After the message being sent, these two fields will be set to be empty again. - /// When the periodical timer wake up, it'll check whether these two fields are set - /// and it'll only send data when these are set. - std::vector responses_; - std::vector unary_reactors_; + private: + void OnCancel() override; + void OnDone() override; + + /// Cleanup callback when the call ends. + const std::function cleanup_cb_; + + /// grpc callback context + grpc::CallbackServerContext *server_context_; }; -/// SyncConnection for gRPC client side. It has customized logic for sending. -class ClientSyncConnection : public NodeSyncConnection { +/// Reactor for gRPC client side. It defines the client's specific behavior for a +/// streaming call. +class RayClientBidiReactor : public RaySyncerBidiReactorBase { public: - ClientSyncConnection( + RayClientBidiReactor( + const std::string &remote_node_id, + const std::string &local_node_id, instrumented_io_context &io_context, - const std::string &node_id, - std::function)> message_processor, - std::shared_ptr channel); + std::function)> message_processor, + std::function cleanup_cb, + std::unique_ptr stub); - protected: - /// Send the message from the pending queue to the target node. - /// It'll use gRPC to send the message directly. - void DoSend() override; + ~RayClientBidiReactor() override = default; - /// Start to send long-polling request to remote nodes. - void StartLongPolling(); + void Disconnect() override; - /// Stub for this connection. - std::unique_ptr stub_; + private: + /// Callback from gRPC + void OnDone(const grpc::Status &status) override; - /// Dummy request for long-polling. - DummyRequest dummy_; + /// Cleanup callback when the call ends. + const std::function cleanup_cb_; + + /// grpc callback context + grpc::ClientContext client_context_; + + std::unique_ptr stub_; }; } // namespace syncer diff --git a/src/ray/common/ray_syncer/ray_syncer.cc b/src/ray/common/ray_syncer/ray_syncer.cc index 86afc1625146..7dd8420e78a9 100644 --- a/src/ray/common/ray_syncer/ray_syncer.cc +++ b/src/ray/common/ray_syncer/ray_syncer.cc @@ -55,9 +55,10 @@ std::optional NodeState::CreateSyncMessage(MessageType message_t bool NodeState::ConsumeSyncMessage(std::shared_ptr message) { auto ¤t = cluster_view_[message->node_id()][message->message_type()]; - RAY_LOG(DEBUG) << "ConsumeSyncMessage: " << (current ? current->version() : -1) - << " message_version: " << message->version() - << ", message_from: " << NodeID::FromBinary(message->node_id()); + RAY_LOG(DEBUG) << "ConsumeSyncMessage: local_version=" + << (current ? current->version() : -1) + << " message_version=" << message->version() + << ", message_from=" << NodeID::FromBinary(message->node_id()); // Check whether newer version of this message has been received. if (current && current->version() >= message->version()) { return false; @@ -66,180 +67,85 @@ bool NodeState::ConsumeSyncMessage(std::shared_ptr message current = message; auto receiver = receivers_[message->message_type()]; if (receiver != nullptr) { + RAY_LOG(DEBUG) << "Consume message from: " << NodeID::FromBinary(message->node_id()); receiver->ConsumeSyncMessage(message); } return true; } -NodeSyncConnection::NodeSyncConnection( - instrumented_io_context &io_context, - std::string remote_node_id, - std::function)> message_processor) - : io_context_(io_context), - remote_node_id_(std::move(remote_node_id)), - message_processor_(std::move(message_processor)) {} - -void NodeSyncConnection::ReceiveUpdate(RaySyncMessages messages) { - for (auto &message : *messages.mutable_sync_messages()) { - auto &node_versions = GetNodeComponentVersions(message.node_id()); - RAY_LOG(DEBUG) << "Receive update: " - << " message_type=" << message.message_type() - << ", message_version=" << message.version() - << ", local_message_version=" << node_versions[message.message_type()]; - if (node_versions[message.message_type()] < message.version()) { - node_versions[message.message_type()] = message.version(); - message_processor_(std::make_shared(std::move(message))); - } - } -} +namespace { -bool NodeSyncConnection::PushToSendingQueue( - std::shared_ptr message) { - // Try to filter out the messages the target node already has. - // Usually it'll be the case when the message is generated from the - // target node or it's sent from the target node. - if (message->node_id() == GetRemoteNodeID()) { - // Skip the message when it's about the node of this connection. - return false; - } - - auto &node_versions = GetNodeComponentVersions(message->node_id()); - if (node_versions[message->message_type()] < message->version()) { - node_versions[message->message_type()] = message->version(); - sending_buffer_[std::make_pair(message->node_id(), message->message_type())] = - message; - return true; - } - return false; +std::string GetNodeIDFromServerContext(grpc::CallbackServerContext *server_context) { + const auto &metadata = server_context->client_metadata(); + auto iter = metadata.find("node_id"); + RAY_CHECK(iter != metadata.end()); + return NodeID::FromHex(std::string(iter->second.begin(), iter->second.end())).Binary(); } -std::array &NodeSyncConnection::GetNodeComponentVersions( - const std::string &node_id) { - auto iter = node_versions_.find(node_id); - if (iter == node_versions_.end()) { - iter = - node_versions_.emplace(node_id, std::array()).first; - iter->second.fill(-1); - } - return iter->second; -} +} // namespace -ClientSyncConnection::ClientSyncConnection( +RayServerBidiReactor::RayServerBidiReactor( + grpc::CallbackServerContext *server_context, instrumented_io_context &io_context, - const std::string &node_id, - std::function)> message_processor, - std::shared_ptr channel) - : NodeSyncConnection(io_context, node_id, std::move(message_processor)), - stub_(ray::rpc::syncer::RaySyncer::NewStub(channel)) { - for (int64_t i = 0; i < RayConfig::instance().ray_syncer_polling_buffer(); ++i) { - StartLongPolling(); - } + const std::string &local_node_id, + std::function)> message_processor, + std::function cleanup_cb) + : RaySyncerBidiReactorBase( + io_context, + GetNodeIDFromServerContext(server_context), + std::move(message_processor)), + cleanup_cb_(std::move(cleanup_cb)), + server_context_(server_context) { + // Send the local node id to the remote + server_context_->AddInitialMetadata("node_id", NodeID::FromBinary(local_node_id).Hex()); + StartSendInitialMetadata(); + + // Start pulling from remote + StartPull(); } -void ClientSyncConnection::StartLongPolling() { - // This will be a long-polling request. The node will only reply if - // 1. there is a new version of message - // 2. and it has passed X ms since last update. - auto client_context = std::make_shared(); - auto in_message = std::make_shared(); - stub_->async()->LongPolling( - client_context.get(), - &dummy_, - in_message.get(), - [this, client_context, in_message](grpc::Status status) mutable { - if (status.ok()) { - io_context_.dispatch( - [this, messages = std::move(*in_message)]() mutable { - ReceiveUpdate(std::move(messages)); - }, - "LongPollingCallback"); - // Start the next polling. - StartLongPolling(); - } - }); +void RayServerBidiReactor::Disconnect() { + io_context_.dispatch([this]() { Finish(grpc::Status::OK); }, ""); } -void ClientSyncConnection::DoSend() { - if (sending_buffer_.empty()) { - return; - } +void RayServerBidiReactor::OnCancel() { Disconnect(); } - auto client_context = std::make_shared(); - auto arena = std::make_shared(); - auto request = google::protobuf::Arena::CreateMessage(arena.get()); - auto response = google::protobuf::Arena::CreateMessage(arena.get()); - - std::vector> holder; - - size_t message_bytes = 0; - auto iter = sending_buffer_.begin(); - while (message_bytes < RayConfig::instance().max_sync_message_batch_bytes() && - iter != sending_buffer_.end()) { - message_bytes += iter->second->sync_message().size(); - // TODO (iycheng): Use arena allocator for optimization - request->mutable_sync_messages()->UnsafeArenaAddAllocated( - const_cast(iter->second.get())); - holder.push_back(iter->second); - sending_buffer_.erase(iter++); - } - if (request->sync_messages_size() != 0) { - stub_->async()->Update( - client_context.get(), - request, - response, - [arena, client_context, holder = std::move(holder)](grpc::Status status) { - if (!status.ok()) { - RAY_LOG(ERROR) << "Sending request failed because of " - << status.error_message(); - } - }); - } +void RayServerBidiReactor::OnDone() { + io_context_.dispatch( + [this]() { + cleanup_cb_(GetRemoteNodeID(), false); + delete this; + }, + ""); } -ServerSyncConnection::ServerSyncConnection( - instrumented_io_context &io_context, +RayClientBidiReactor::RayClientBidiReactor( const std::string &remote_node_id, - std::function)> message_processor) - : NodeSyncConnection(io_context, remote_node_id, std::move(message_processor)) {} - -ServerSyncConnection::~ServerSyncConnection() { - // If there is a pending request, we need to cancel it. Otherwise, rpc will - // hang there forever. - while (!unary_reactors_.empty()) { - unary_reactors_.back()->Finish(grpc::Status::CANCELLED); - unary_reactors_.pop_back(); - } + const std::string &local_node_id, + instrumented_io_context &io_context, + std::function)> message_processor, + std::function cleanup_cb, + std::unique_ptr stub) + : RaySyncerBidiReactorBase( + io_context, remote_node_id, std::move(message_processor)), + cleanup_cb_(std::move(cleanup_cb)), + stub_(std::move(stub)) { + client_context_.AddMetadata("node_id", NodeID::FromBinary(local_node_id).Hex()); + stub_->async()->StartSync(&client_context_, this); + StartPull(); } -void ServerSyncConnection::HandleLongPollingRequest(grpc::ServerUnaryReactor *reactor, - RaySyncMessages *response) { - unary_reactors_.push_back(reactor); - responses_.push_back(response); +void RayClientBidiReactor::OnDone(const grpc::Status &status) { + io_context_.dispatch( + [this, status]() { + cleanup_cb_(GetRemoteNodeID(), !status.ok()); + delete this; + }, + ""); } -void ServerSyncConnection::DoSend() { - // There is no receive request - if (unary_reactors_.empty() || sending_buffer_.empty()) { - return; - } - - RAY_CHECK(!responses_.empty()); - - size_t message_bytes = 0; - auto iter = sending_buffer_.begin(); - while (message_bytes < RayConfig::instance().max_sync_message_batch_bytes() && - iter != sending_buffer_.end()) { - message_bytes += iter->second->sync_message().size(); - // TODO (iycheng): Use arena allocator for optimization - responses_.back()->add_sync_messages()->CopyFrom(*iter->second); - sending_buffer_.erase(iter++); - } - - if (responses_.back()->sync_messages_size() != 0) { - unary_reactors_.back()->Finish(grpc::Status::OK); - responses_.pop_back(); - unary_reactors_.pop_back(); - } +void RayClientBidiReactor::Disconnect() { + io_context_.dispatch([this]() { StartWritesDone(); }, ""); } RaySyncer::RaySyncer(instrumented_io_context &io_context, @@ -249,109 +155,133 @@ RaySyncer::RaySyncer(instrumented_io_context &io_context, node_state_(std::make_unique()), timer_(io_context) { stopped_ = std::make_shared(false); - timer_.RunFnPeriodically( - [this]() { - for (auto &[_, sync_connection] : sync_connections_) { - sync_connection->DoSend(); - } - }, - RayConfig::instance().raylet_report_resources_period_milliseconds()); } RaySyncer::~RaySyncer() { *stopped_ = true; - for (auto &call : inflight_requests_) { - auto f = call->promise.get_future(); - if (!f.valid()) { - call->context.TryCancel(); - } - f.get(); - } + io_context_.dispatch( + [reactors = sync_reactors_]() { + for (auto [_, reactor] : reactors) { + reactor->Disconnect(); + } + }, + ""); } -void RaySyncer::Connect(std::shared_ptr channel) { - auto call = std::make_unique(); - - auto stub = ray::rpc::syncer::RaySyncer::NewStub(channel); - call->request.set_node_id(local_node_id_); - - stub->async()->StartSync( - &call->context, - &call->request, - &call->response, - [this, channel, call = call.get(), stopped = this->stopped_](grpc::Status status) { - call->promise.set_value(); - if (*stopped) { - return; - } - if (status.ok()) { - io_context_.dispatch( - [this, channel, node_id = call->response.node_id()]() { - auto connection = std::make_unique( - io_context_, - node_id, - [this](auto msg) { BroadcastMessage(msg); }, - channel); - Connect(std::move(connection)); - }, - "StartSyncCallback"); +std::vector RaySyncer::GetAllConnectedNodeIDs() const { + std::promise> promise; + io_context_.dispatch( + [&]() { + std::vector nodes; + for (auto [node_id, _] : sync_reactors_) { + nodes.push_back(node_id); } - }); - inflight_requests_.emplace(std::move(call)); + promise.set_value(std::move(nodes)); + }, + ""); + return promise.get_future().get(); +} + +void RaySyncer::Connect(const std::string &node_id, + std::shared_ptr channel) { + io_context_.dispatch( + [=]() { + auto stub = ray::rpc::syncer::RaySyncer::NewStub(channel); + auto reactor = new RayClientBidiReactor( + /* remote_node_id */ node_id, + /* local_node_id */ GetLocalNodeID(), + /* io_context */ io_context_, + /* message_processor */ [this](auto msg) { BroadcastRaySyncMessage(msg); }, + /* cleanup_cb */ + [this, channel](const std::string &node_id, bool restart) { + sync_reactors_.erase(node_id); + if (restart) { + RAY_LOG(INFO) << "Connection is broken. Reconnect to node: " + << NodeID::FromBinary(node_id); + Connect(node_id, channel); + } + }, + /* stub */ std::move(stub)); + Connect(reactor); + reactor->StartCall(); + }, + ""); } -void RaySyncer::Connect(std::unique_ptr connection) { - // Somehow connection=std::move(connection) won't be compiled here. - // Potentially it might have a leak here if the function is not executed. +void RaySyncer::Connect(RaySyncerBidiReactor *reactor) { io_context_.dispatch( - [this, connection = connection.release()]() mutable { - RAY_CHECK(connection != nullptr); - RAY_CHECK(sync_connections_[connection->GetRemoteNodeID()] == nullptr); - auto &conn = *connection; - sync_connections_[connection->GetRemoteNodeID()].reset(connection); + [this, reactor]() { + RAY_CHECK(sync_reactors_.find(reactor->GetRemoteNodeID()) == + sync_reactors_.end()); + sync_reactors_[reactor->GetRemoteNodeID()] = reactor; + // Send the view for new connections. for (const auto &[_, messages] : node_state_->GetClusterView()) { - for (auto &message : messages) { + for (const auto &message : messages) { if (!message) { continue; } - conn.PushToSendingQueue(message); + RAY_LOG(DEBUG) << "Push init view from: " + << NodeID::FromBinary(GetLocalNodeID()) << " to " + << NodeID::FromBinary(reactor->GetRemoteNodeID()) << " about " + << NodeID::FromBinary(message->node_id()); + reactor->PushToSendingQueue(message); } } }, - "RaySyncer::Connect"); + "RaySyncerConnect"); } void RaySyncer::Disconnect(const std::string &node_id) { + std::promise promise; io_context_.dispatch( - [this, node_id]() { - auto iter = sync_connections_.find(node_id); - if (iter != sync_connections_.end()) { - sync_connections_.erase(iter); + [&]() { + auto iter = sync_reactors_.find(node_id); + if (iter == sync_reactors_.end()) { + promise.set_value(nullptr); + return; + } + + auto reactor = iter->second; + if (iter != sync_reactors_.end()) { + sync_reactors_.erase(iter); } + promise.set_value(reactor); }, "RaySyncerDisconnect"); + auto reactor = promise.get_future().get(); + if (reactor != nullptr) { + reactor->Disconnect(); + } } -bool RaySyncer::Register(MessageType message_type, +void RaySyncer::Register(MessageType message_type, const ReporterInterface *reporter, ReceiverInterface *receiver, int64_t pull_from_reporter_interval_ms) { - if (!node_state_->SetComponent(message_type, reporter, receiver)) { - return false; - } + io_context_.dispatch( + [this, message_type, reporter, receiver, pull_from_reporter_interval_ms]() mutable { + if (!node_state_->SetComponent(message_type, reporter, receiver)) { + return; + } - // Set job to pull from reporter periodically - if (reporter != nullptr && pull_from_reporter_interval_ms > 0) { - timer_.RunFnPeriodically( - [this, message_type]() { OnDemandBroadcasting(message_type); }, - pull_from_reporter_interval_ms); - } + // Set job to pull from reporter periodically + if (reporter != nullptr && pull_from_reporter_interval_ms > 0) { + timer_.RunFnPeriodically( + [this, stopped = stopped_, message_type]() { + if (*stopped) { + return; + } + OnDemandBroadcasting(message_type); + }, + pull_from_reporter_interval_ms); + } - RAY_LOG(DEBUG) << "Registered components: " - << "message_type:" << message_type << ", reporter:" << reporter - << ", receiver:" << receiver - << ", pull_from_reporter_interval_ms:" << pull_from_reporter_interval_ms; - return true; + RAY_LOG(DEBUG) << "Registered components: " + << "message_type:" << message_type << ", reporter:" << reporter + << ", receiver:" << receiver << ", pull_from_reporter_interval_ms:" + << pull_from_reporter_interval_ms; + }, + "RaySyncerRegister"); } bool RaySyncer::OnDemandBroadcasting(MessageType message_type) { @@ -372,90 +302,38 @@ void RaySyncer::BroadcastMessage(std::shared_ptr message) io_context_.dispatch( [this, message] { // The message is stale. Just skip this one. + RAY_LOG(DEBUG) << "Receive message from: " + << NodeID::FromBinary(message->node_id()) << " to " + << NodeID::FromBinary(GetLocalNodeID()); if (!node_state_->ConsumeSyncMessage(message)) { return; } - for (auto &connection : sync_connections_) { - connection.second->PushToSendingQueue(message); + for (auto &reactor : sync_reactors_) { + reactor.second->PushToSendingQueue(message); } }, "RaySyncer.BroadcastMessage"); } -grpc::ServerUnaryReactor *RaySyncerService::StartSync( - grpc::CallbackServerContext *context, - const StartSyncRequest *request, - StartSyncResponse *response) { - auto *reactor = context->DefaultReactor(); - // Make sure server only have one client - if (!remote_node_id_.empty()) { - RAY_LOG(WARNING) << "Get a new sync request from " - << NodeID::FromBinary(request->node_id()) << ". " - << "Now disconnect from " << NodeID::FromBinary(remote_node_id_); - syncer_.Disconnect(remote_node_id_); - } - remote_node_id_ = request->node_id(); - RAY_LOG(DEBUG) << "Get connect from: " << NodeID::FromBinary(remote_node_id_); - syncer_.GetIOContext().dispatch( - [this, response, reactor, context]() { - if (context->IsCancelled()) { - reactor->Finish(grpc::Status::CANCELLED); - return; - } - - syncer_.Connect(std::make_unique( - syncer_.GetIOContext(), remote_node_id_, [this](auto msg) { - syncer_.BroadcastMessage(msg); - })); - response->set_node_id(syncer_.GetLocalNodeID()); - reactor->Finish(grpc::Status::OK); - }, - "RaySyncer::StartSync"); - return reactor; -} - -grpc::ServerUnaryReactor *RaySyncerService::Update(grpc::CallbackServerContext *context, - const RaySyncMessages *request, - DummyResponse *) { - auto *reactor = context->DefaultReactor(); - // Make sure request is allocated from heap so that it can be moved safely. - RAY_CHECK(request->GetArena() == nullptr); - syncer_.GetIOContext().dispatch( - [this, request = std::move(*const_cast(request))]() mutable { - auto *sync_connection = dynamic_cast( - syncer_.GetSyncConnection(remote_node_id_)); - if (sync_connection != nullptr) { - sync_connection->ReceiveUpdate(std::move(request)); - } else { - RAY_LOG(FATAL) << "Fail to get the sync context"; - } - }, - "SyncerUpdate"); - reactor->Finish(grpc::Status::OK); - return reactor; -} - -grpc::ServerUnaryReactor *RaySyncerService::LongPolling( - grpc::CallbackServerContext *context, - const DummyRequest *, - RaySyncMessages *response) { - auto *reactor = context->DefaultReactor(); - syncer_.GetIOContext().dispatch( - [this, reactor, response]() mutable { - auto *sync_connection = dynamic_cast( - syncer_.GetSyncConnection(remote_node_id_)); - if (sync_connection != nullptr) { - sync_connection->HandleLongPollingRequest(reactor, response); - } else { - RAY_LOG(ERROR) << "Fail to setup long-polling"; - reactor->Finish(grpc::Status::CANCELLED); - } - }, - "SyncLongPolling"); +ServerBidiReactor *RaySyncerService::StartSync(grpc::CallbackServerContext *context) { + auto reactor = new RayServerBidiReactor( + context, + syncer_.GetIOContext(), + syncer_.GetLocalNodeID(), + [this](auto msg) mutable { syncer_.BroadcastMessage(msg); }, + [this](const std::string &node_id, bool reconnect) mutable { + // No need to reconnect for server side. + RAY_CHECK(!reconnect); + syncer_.sync_reactors_.erase(node_id); + }); + RAY_LOG(DEBUG) << "Get connection from " + << NodeID::FromBinary(reactor->GetRemoteNodeID()) << " to " + << NodeID::FromBinary(syncer_.GetLocalNodeID()); + syncer_.Connect(reactor); return reactor; } -RaySyncerService::~RaySyncerService() { syncer_.Disconnect(remote_node_id_); } +RaySyncerService::~RaySyncerService() {} } // namespace syncer } // namespace ray diff --git a/src/ray/common/ray_syncer/ray_syncer.h b/src/ray/common/ray_syncer/ray_syncer.h index 07c636b6cc29..abe835c60e0a 100644 --- a/src/ray/common/ray_syncer/ray_syncer.h +++ b/src/ray/common/ray_syncer/ray_syncer.h @@ -27,13 +27,11 @@ namespace ray { namespace syncer { -using ray::rpc::syncer::DummyRequest; -using ray::rpc::syncer::DummyResponse; using ray::rpc::syncer::MessageType; using ray::rpc::syncer::RaySyncMessage; -using ray::rpc::syncer::RaySyncMessages; -using ray::rpc::syncer::StartSyncRequest; -using ray::rpc::syncer::StartSyncResponse; + +using ServerBidiReactor = grpc::ServerBidiReactor; +using ClientBidiReactor = grpc::ClientBidiReactor; static constexpr size_t kComponentArraySize = static_cast(ray::rpc::syncer::MessageType_ARRAYSIZE); @@ -73,7 +71,7 @@ struct ReceiverInterface { // Forward declaration of internal structures class NodeState; -class NodeSyncConnection; +class RaySyncerBidiReactor; /// RaySyncer is an embedding service for component synchronization. /// All operations in this class needs to be finished GetIOContext() @@ -81,9 +79,9 @@ class NodeSyncConnection; /// RaySyncer is the control plane to make sure all connections eventually /// have the latest view of the cluster components registered. /// RaySyncer has two components: -/// 1. NodeSyncConnection: keeps track of the sending and receiving information +/// 1. RaySyncerBidiReactor: keeps track of the sending and receiving information /// and make sure not sending the information the remote node knows. -/// 2. NodeState: keeps track of the local status, similar to NodeSyncConnection, +/// 2. NodeState: keeps track of the local status, similar to RaySyncerBidiReactor, // but it's for local node. class RaySyncer { public: @@ -98,15 +96,9 @@ class RaySyncer { /// TODO (iycheng): Introduce grpc channel pool and use node_id /// for the connection. /// - /// \param connection The connection to the remote node. - void Connect(std::unique_ptr connection); - - /// Connect to a node. - /// TODO (iycheng): Introduce grpc channel pool and use node_id - /// for the connection. - /// - /// \param connection The connection to the remote node. - void Connect(std::shared_ptr channel); + /// \param node_id The id of the node connect to. + /// \param channel The gRPC channel. + void Connect(const std::string &node_id, std::shared_ptr channel); void Disconnect(const std::string &node_id); @@ -121,7 +113,7 @@ class RaySyncer { /// \param pull_from_reporter_interval_ms The frequence to pull a message. 0 means /// never pull a message in syncer. /// from reporter and push it to sending queue. - bool Register(MessageType message_type, + void Register(MessageType message_type, const ReporterInterface *reporter, ReceiverInterface *receiver, int64_t pull_from_reporter_interval_ms = 100); @@ -143,24 +135,16 @@ class RaySyncer { /// \param message The message to be broadcasted. void BroadcastRaySyncMessage(std::shared_ptr message); + std::vector GetAllConnectedNodeIDs() const; + private: + void Connect(RaySyncerBidiReactor *connection); + + std::shared_ptr stopped_; + /// Get the io_context used by RaySyncer. instrumented_io_context &GetIOContext() { return io_context_; } - /// Get the SyncConnection of a node. - /// - /// \param node_id The node id to lookup. - /// - /// \return nullptr if it doesn't exist, otherwise, the connection associated with the - /// node. - NodeSyncConnection *GetSyncConnection(const std::string &node_id) const { - auto iter = sync_connections_.find(node_id); - if (iter == sync_connections_.end()) { - return nullptr; - } - return iter->second.get(); - } - /// Function to broadcast the messages to other nodes. /// A message will be sent to a node if that node doesn't have this message. /// The message can be generated by local reporter or received by the other node. @@ -175,29 +159,14 @@ class RaySyncer { const std::string local_node_id_; /// Manage connections. Here the key is the NodeID in binary form. - absl::flat_hash_map> sync_connections_; - - /// Upward connections. These are connections initialized not by the local node. - absl::flat_hash_set upward_connections_; + absl::flat_hash_map sync_reactors_; /// The local node state std::unique_ptr node_state_; - /// Context of a rpc call. - struct StartSyncCall { - StartSyncRequest request; - StartSyncResponse response; - grpc::ClientContext context; - std::promise promise; - }; - - absl::flat_hash_set> inflight_requests_; - /// Timer is used to do broadcasting. ray::PeriodicalRunner timer_; - std::shared_ptr stopped_; - friend class RaySyncerService; /// Test purpose friend struct SyncerServerTest; @@ -209,9 +178,6 @@ class RaySyncer { FRIEND_TEST(SyncerTest, Reconnect); }; -class ClientSyncConnection; -class ServerSyncConnection; - /// RaySyncerService is a service to take care of resource synchronization /// related operations. /// Right now only raylet needs to setup this service. But in the future, @@ -223,25 +189,10 @@ class RaySyncerService : public ray::rpc::syncer::RaySyncer::CallbackService { ~RaySyncerService(); - grpc::ServerUnaryReactor *StartSync(grpc::CallbackServerContext *context, - const StartSyncRequest *request, - StartSyncResponse *response) override; - - grpc::ServerUnaryReactor *Update(grpc::CallbackServerContext *context, - const RaySyncMessages *request, - DummyResponse *) override; - - grpc::ServerUnaryReactor *LongPolling(grpc::CallbackServerContext *context, - const DummyRequest *, - RaySyncMessages *response) override; + grpc::ServerBidiReactor *StartSync( + grpc::CallbackServerContext *context) override; private: - // This will be created after connection is established. - // Ideally this should be owned by RaySyncer, but since we are doing - // long-polling right now, we have to put it here so that when - // long-polling request comes, we can set it up. - std::string remote_node_id_; - // The ray syncer this RPC wrappers of. RaySyncer &syncer_; }; diff --git a/src/ray/common/test/ray_syncer_test.cc b/src/ray/common/test/ray_syncer_test.cc index 8ca793737778..ff0652d22be3 100644 --- a/src/ray/common/test/ray_syncer_test.cc +++ b/src/ray/common/test/ray_syncer_test.cc @@ -27,6 +27,7 @@ #include #include "ray/common/ray_syncer/ray_syncer.h" +#include "ray/rpc/grpc_server.h" #include "mock/ray/common/ray_syncer/ray_syncer.h" // clang-format on @@ -44,6 +45,9 @@ namespace syncer { constexpr size_t kTestComponents = 1; +using work_guard_type = + boost::asio::executor_work_guard; + RaySyncMessage MakeMessage(MessageType cid, int64_t version, const NodeID &id) { auto msg = RaySyncMessage(); msg.set_version(version); @@ -55,6 +59,7 @@ RaySyncMessage MakeMessage(MessageType cid, int64_t version, const NodeID &id) { class RaySyncerTest : public ::testing::Test { protected: void SetUp() override { + work_guard_ = std::make_unique(io_context_.get_executor()); local_versions_.fill(0); for (size_t cid = 0; cid < reporters_.size(); ++cid) { receivers_[cid] = std::make_unique(); @@ -74,10 +79,7 @@ class RaySyncerTest : public ::testing::Test { ON_CALL(*reporter, CreateSyncMessage(_, _)) .WillByDefault(WithArg<0>(Invoke(take_snapshot))); } - thread_ = std::make_unique([this]() { - boost::asio::io_context::work work(io_context_); - io_context_.run(); - }); + thread_ = std::make_unique([this]() { io_context_.run(); }); local_id_ = NodeID::FromRandom(); syncer_ = std::make_unique(io_context_, local_id_.Binary()); } @@ -95,7 +97,7 @@ class RaySyncerTest : public ::testing::Test { } void TearDown() override { - io_context_.stop(); + work_guard_->reset(); thread_->join(); } @@ -106,6 +108,7 @@ class RaySyncerTest : public ::testing::Test { nullptr}; instrumented_io_context io_context_; + std::unique_ptr work_guard_; std::unique_ptr thread_; std::unique_ptr syncer_; @@ -145,40 +148,58 @@ TEST_F(RaySyncerTest, NodeStateConsume) { ASSERT_FALSE(node_status->ConsumeSyncMessage(std::make_shared(msg))); } -TEST_F(RaySyncerTest, NodeSyncConnection) { +struct MockReactor { + void StartRead(RaySyncMessage *) { ++read_cnt; } + + void StartWrite(const RaySyncMessage *, + grpc::WriteOptions opts = grpc::WriteOptions()) { + ++write_cnt; + } + + virtual void OnWriteDone(bool ok) {} + virtual void OnReadDone(bool ok) {} + + size_t read_cnt = 0; + size_t write_cnt = 0; +}; + +TEST_F(RaySyncerTest, RaySyncerBidiReactorBase) { auto node_id = NodeID::FromRandom(); - MockNodeSyncConnection sync_connection( + MockRaySyncerBidiReactorBase sync_reactor( io_context_, node_id.Binary(), - [](std::shared_ptr) {}); + [](std::shared_ptr) {}); auto from_node_id = NodeID::FromRandom(); auto msg = MakeMessage(MessageType::RESOURCE_VIEW, 0, from_node_id); + auto msg_ptr1 = std::make_shared(msg); + msg.set_version(2); + auto msg_ptr2 = std::make_shared(msg); + msg.set_version(3); + auto msg_ptr3 = std::make_shared(msg); // First push will succeed and the second one will be deduplicated. - ASSERT_TRUE(sync_connection.PushToSendingQueue(std::make_shared(msg))); - ASSERT_FALSE(sync_connection.PushToSendingQueue(std::make_shared(msg))); - ASSERT_EQ(1, sync_connection.sending_buffer_.size()); - ASSERT_EQ(0, sync_connection.sending_buffer_.begin()->second->version()); - ASSERT_EQ(1, sync_connection.node_versions_.size()); + ASSERT_TRUE(sync_reactor.PushToSendingQueue(msg_ptr1)); + ASSERT_FALSE(sync_reactor.PushToSendingQueue(msg_ptr1)); + ASSERT_EQ(0, sync_reactor.sending_buffer_.size()); + + ASSERT_TRUE(sync_reactor.PushToSendingQueue(msg_ptr2)); + ASSERT_EQ(1, sync_reactor.sending_buffer_.size()); + ASSERT_EQ(1, sync_reactor.node_versions_.size()); + ASSERT_EQ(2, sync_reactor.sending_buffer_.begin()->second->version()); ASSERT_EQ( - 0, - sync_connection.node_versions_[from_node_id.Binary()][MessageType::RESOURCE_VIEW]); + 2, sync_reactor.node_versions_[from_node_id.Binary()][MessageType::RESOURCE_VIEW]); - msg.set_version(2); - ASSERT_TRUE(sync_connection.PushToSendingQueue(std::make_shared(msg))); - ASSERT_FALSE(sync_connection.PushToSendingQueue(std::make_shared(msg))); - // The previous message is deleted. - ASSERT_EQ(1, sync_connection.sending_buffer_.size()); - ASSERT_EQ(1, sync_connection.node_versions_.size()); - ASSERT_EQ(2, sync_connection.sending_buffer_.begin()->second->version()); + ASSERT_TRUE(sync_reactor.PushToSendingQueue(msg_ptr3)); + ASSERT_EQ(1, sync_reactor.sending_buffer_.size()); + ASSERT_EQ(1, sync_reactor.node_versions_.size()); + ASSERT_EQ(3, sync_reactor.sending_buffer_.begin()->second->version()); ASSERT_EQ( - 2, - sync_connection.node_versions_[from_node_id.Binary()][MessageType::RESOURCE_VIEW]); + 3, sync_reactor.node_versions_[from_node_id.Binary()][MessageType::RESOURCE_VIEW]); } struct SyncerServerTest { - SyncerServerTest(std::string port) { + SyncerServerTest(std::string port) : work_guard(io_context.get_executor()) { this->server_port = port; // Setup io context auto node_id = NodeID::FromRandom(); @@ -187,6 +208,7 @@ struct SyncerServerTest { } // Setup syncer and grpc server syncer = std::make_unique(io_context, node_id.Binary()); + thread = std::make_unique([this] { io_context.run(); }); auto server_address = std::string("0.0.0.0:") + port; grpc::ServerBuilder builder; @@ -196,7 +218,10 @@ struct SyncerServerTest { server = builder.BuildAndStart(); for (size_t cid = 0; cid < reporters.size(); ++cid) { - auto snapshot_received = [this](std::shared_ptr message) { + auto snapshot_received = [this, + node_id](std::shared_ptr message) { + RAY_LOG(DEBUG) << "Message received: from " + << NodeID::FromBinary(message->node_id()) << " to " << node_id; auto iter = received_versions.find(message->node_id()); if (iter == received_versions.end()) { for (auto &v : received_versions[message->node_id()]) { @@ -208,6 +233,9 @@ struct SyncerServerTest { received_versions[message->node_id()][message->message_type()] = message->version(); message_consumed[message->node_id()]++; + RAY_LOG(DEBUG) << "Message consumed from " + << NodeID::FromBinary(message->node_id()) + << ", local_id=" << node_id; }; receivers[cid] = std::make_unique(); EXPECT_CALL(*receivers[cid], ConsumeSyncMessage(_)) @@ -232,10 +260,6 @@ struct SyncerServerTest { syncer->Register( static_cast(cid), reporter.get(), receivers[cid].get()); } - thread = std::make_unique([this] { - boost::asio::io_context::work work(io_context); - io_context.run(); - }); } void WaitSendingFlush() { @@ -244,13 +268,22 @@ struct SyncerServerTest { auto f = p.get_future(); io_context.post( [&p, this]() mutable { - for (const auto &[node_id, conn] : syncer->sync_connections_) { - if (!conn->sending_buffer_.empty()) { + for (const auto &[node_id, conn] : syncer->sync_reactors_) { + auto ptr = dynamic_cast(conn); + size_t remainings = 0; + if (ptr == nullptr) { + remainings = + dynamic_cast(conn)->sending_buffer_.size(); + } else { + remainings = ptr->sending_buffer_.size(); + } + + if (remainings != 0) { p.set_value(false); RAY_LOG(INFO) << NodeID::FromBinary(syncer->GetLocalNodeID()) << ": " << "Waiting for message on " << NodeID::FromBinary(node_id) << " to be sent." - << " Remainings " << conn->sending_buffer_.size(); + << " Remainings " << remainings; return; } } @@ -281,11 +314,19 @@ struct SyncerServerTest { return false; } - ~SyncerServerTest() { - service.reset(); - server.reset(); + void Stop() { + for (auto node_id : syncer->GetAllConnectedNodeIDs()) { + syncer->Disconnect(node_id); + } + + server->Shutdown(); + io_context.stop(); thread->join(); + + server.reset(); + service.reset(); + syncer.reset(); } @@ -314,7 +355,9 @@ struct SyncerServerTest { std::unique_ptr syncer; std::unique_ptr server; std::unique_ptr thread; + instrumented_io_context io_context; + work_guard_type work_guard; std::string server_port; std::array, kTestComponents> local_versions; std::array, kTestComponents> reporters = { @@ -367,38 +410,57 @@ using TClusterView = absl::flat_hash_map< std::string, std::array, kComponentArraySize>>; -TEST(SyncerTest, Test1To1) { - auto s1 = SyncerServerTest("19990"); +class SyncerTest : public ::testing::Test { + public: + SyncerServerTest &MakeServer(std::string port) { + servers.emplace_back(std::make_unique(port)); + return *servers.back(); + } + + protected: + void TearDown() override { + // Drain all grpc requests. + for (auto &s : servers) { + s->Stop(); + } + + std::this_thread::sleep_for(1s); + } + std::vector> servers; +}; + +TEST_F(SyncerTest, Test1To1) { + auto &s1 = MakeServer("19990"); - auto s2 = SyncerServerTest("19991"); + auto &s2 = MakeServer("19991"); // Make sure the setup is correct ASSERT_NE(nullptr, s1.receivers[MessageType::RESOURCE_VIEW]); ASSERT_NE(nullptr, s2.receivers[MessageType::RESOURCE_VIEW]); ASSERT_NE(nullptr, s1.reporters[MessageType::RESOURCE_VIEW]); ASSERT_NE(nullptr, s2.reporters[MessageType::RESOURCE_VIEW]); + RAY_LOG(DEBUG) << "s1: " << NodeID::FromBinary(s1.syncer->GetLocalNodeID()); + RAY_LOG(DEBUG) << "s2: " << NodeID::FromBinary(s2.syncer->GetLocalNodeID()); auto channel_to_s2 = MakeChannel("19991"); - s1.syncer->Connect(channel_to_s2); + s1.syncer->Connect(s2.syncer->GetLocalNodeID(), channel_to_s2); // Make sure s2 adds s1 ASSERT_TRUE(s2.WaitUntil( - [&s2]() { - return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 1; - }, + [&s2]() { return s2.syncer->sync_reactors_.size() == 1 && s2.snapshot_taken == 1; }, 5)); // Make sure s1 adds s2 ASSERT_TRUE(s1.WaitUntil( - [&s1]() { - return s1.syncer->sync_connections_.size() == 1 && s1.snapshot_taken == 1; - }, + [&s1]() { return s1.syncer->sync_reactors_.size() == 1 && s1.snapshot_taken == 1; }, 5)); // s1 will only send 1 message to s2 because it only has one reporter ASSERT_TRUE(s2.WaitUntil( [&s2, node_id = s1.syncer->GetLocalNodeID()]() { + RAY_LOG(DEBUG) << NodeID::FromBinary(node_id) << " - " + << s2.GetNumConsumedMessages(node_id); return s2.GetNumConsumedMessages(node_id) == 1; }, 5)); @@ -406,6 +468,9 @@ TEST(SyncerTest, Test1To1) { // s2 will send 2 messages to s1 because it has two reporters. ASSERT_TRUE(s1.WaitUntil( [&s1, node_id = s2.syncer->GetLocalNodeID()]() { + RAY_LOG(DEBUG) << "Num of messages from " << NodeID::FromBinary(node_id) << " to " + << NodeID::FromBinary(s1.syncer->GetLocalNodeID()) << " is " + << s1.GetNumConsumedMessages(node_id); return s1.GetNumConsumedMessages(node_id) == 1; }, 5)); @@ -475,7 +540,7 @@ TEST(SyncerTest, Test1To1) { ASSERT_LE(s2.GetNumConsumedMessages(s1.syncer->GetLocalNodeID()), max_sends + 3); } -TEST(SyncerTest, Reconnect) { +TEST_F(SyncerTest, Reconnect) { // This test is to check reconnect works. // Firstly // s1 -> s3 @@ -483,59 +548,47 @@ TEST(SyncerTest, Reconnect) { // s2 -> s3 // And we need to ensure s3 is connecting to s2 - auto s1 = SyncerServerTest("19990"); - auto s2 = SyncerServerTest("19991"); - auto s3 = SyncerServerTest("19992"); + auto &s1 = MakeServer("19990"); + auto &s2 = MakeServer("19991"); + auto &s3 = MakeServer("19992"); - s1.syncer->Connect(MakeChannel("19992")); + s1.syncer->Connect(s3.syncer->GetLocalNodeID(), MakeChannel("19992")); // Make sure the setup is correct ASSERT_TRUE(s1.WaitUntil( - [&s1]() { - return s1.syncer->sync_connections_.size() == 1 && s1.snapshot_taken == 1; - }, + [&s1]() { return s1.syncer->sync_reactors_.size() == 1 && s1.snapshot_taken == 1; }, 5)); ASSERT_TRUE(s1.WaitUntil( - [&s3]() { - return s3.syncer->sync_connections_.size() == 1 && s3.snapshot_taken == 1; - }, + [&s3]() { return s3.syncer->sync_reactors_.size() == 1 && s3.snapshot_taken == 1; }, 5)); - s2.syncer->Connect(MakeChannel("19992")); + s2.syncer->Connect(s3.syncer->GetLocalNodeID(), MakeChannel("19992")); ASSERT_TRUE(s1.WaitUntil( - [&s2]() { - return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 1; - }, + [&s2]() { return s2.syncer->sync_reactors_.size() == 1 && s2.snapshot_taken == 1; }, 5)); } -TEST(SyncerTest, Broadcast) { +TEST_F(SyncerTest, Broadcast) { // This test covers the broadcast feature of ray syncer. - auto s1 = SyncerServerTest("19990"); - auto s2 = SyncerServerTest("19991"); - auto s3 = SyncerServerTest("19992"); + auto &s1 = MakeServer("19990"); + auto &s2 = MakeServer("19991"); + auto &s3 = MakeServer("19992"); // We need to make sure s1 is sending data to s3 for s2 - s1.syncer->Connect(MakeChannel("19991")); - s1.syncer->Connect(MakeChannel("19992")); + s1.syncer->Connect(s2.syncer->GetLocalNodeID(), MakeChannel("19991")); + s1.syncer->Connect(s3.syncer->GetLocalNodeID(), MakeChannel("19992")); // Make sure the setup is correct ASSERT_TRUE(s1.WaitUntil( - [&s1]() { - return s1.syncer->sync_connections_.size() == 2 && s1.snapshot_taken == 1; - }, + [&s1]() { return s1.syncer->sync_reactors_.size() == 2 && s1.snapshot_taken == 1; }, 5)); ASSERT_TRUE(s1.WaitUntil( - [&s2]() { - return s2.syncer->sync_connections_.size() == 1 && s2.snapshot_taken == 1; - }, + [&s2]() { return s2.syncer->sync_reactors_.size() == 1 && s2.snapshot_taken == 1; }, 5)); ASSERT_TRUE(s1.WaitUntil( - [&s3]() { - return s3.syncer->sync_connections_.size() == 1 && s3.snapshot_taken == 1; - }, + [&s3]() { return s3.syncer->sync_reactors_.size() == 1 && s3.snapshot_taken == 1; }, 5)); // Change the resource in s2 and make sure s1 && s3 are correct @@ -554,7 +607,7 @@ TEST(SyncerTest, Broadcast) { 5)); } -bool CompareViews(const std::vector> &servers, +bool CompareViews(const std::vector &servers, const std::vector &views, const std::vector> &g) { // Check broadcasting is working @@ -595,7 +648,7 @@ bool CompareViews(const std::vector> &servers, } bool TestCorrectness(std::function get_cluster_view, - std::vector> &servers, + std::vector &servers, const std::vector> &g) { auto check = [&servers, get_cluster_view, &g]() { std::vector views; @@ -656,15 +709,16 @@ bool TestCorrectness(std::function get_cluster_ return check(); } -TEST(SyncerTest, Test1ToN) { +TEST_F(SyncerTest, Test1ToN) { size_t base_port = 18990; - std::vector> servers; + std::vector servers; for (int i = 0; i < 20; ++i) { - servers.push_back(std::make_unique(std::to_string(i + base_port))); + servers.push_back(&MakeServer(std::to_string(i + base_port))); } std::vector> g(servers.size()); for (size_t i = 1; i < servers.size(); ++i) { - servers[0]->syncer->Connect(MakeChannel(servers[i]->server_port)); + servers[0]->syncer->Connect(servers[i]->syncer->GetLocalNodeID(), + MakeChannel(servers[i]->server_port)); g[0].insert(i); } @@ -680,11 +734,11 @@ TEST(SyncerTest, Test1ToN) { ASSERT_TRUE(TestCorrectness(get_cluster_view, servers, g)); } -TEST(SyncerTest, TestMToN) { +TEST_F(SyncerTest, TestMToN) { size_t base_port = 18990; - std::vector> servers; + std::vector servers; for (int i = 0; i < 20; ++i) { - servers.push_back(std::make_unique(std::to_string(i + base_port))); + servers.push_back(&MakeServer(std::to_string(i + base_port))); } std::vector> g(servers.size()); // Try to construct a tree based structure @@ -693,7 +747,8 @@ TEST(SyncerTest, TestMToN) { while (i < servers.size()) { // try to connect to 2 servers per node. for (int k = 0; k < 2 && i < servers.size(); ++k, ++i) { - servers[curr]->syncer->Connect(MakeChannel(servers[i]->server_port)); + servers[curr]->syncer->Connect(servers[i]->syncer->GetLocalNodeID(), + MakeChannel(servers[i]->server_port)); g[curr].insert(i); } ++curr; @@ -710,5 +765,166 @@ TEST(SyncerTest, TestMToN) { ASSERT_TRUE(TestCorrectness(get_cluster_view, servers, g)); } +struct MockRaySyncerService : public ray::rpc::syncer::RaySyncer::CallbackService { + MockRaySyncerService( + instrumented_io_context &_io_context, + std::function)> _message_processor, + std::function _cleanup_cb) + : message_processor(_message_processor), + cleanup_cb(_cleanup_cb), + node_id(NodeID::FromRandom()), + io_context(_io_context) {} + grpc::ServerBidiReactor *StartSync( + grpc::CallbackServerContext *context) override { + reactor = new RayServerBidiReactor( + context, io_context, node_id.Binary(), message_processor, cleanup_cb); + return reactor; + } + + std::function)> message_processor; + std::function cleanup_cb; + NodeID node_id; + instrumented_io_context &io_context; + RayServerBidiReactor *reactor = nullptr; +}; + +class SyncerReactorTest : public ::testing::Test { + protected: + void SetUp() override { + rpc_service_ = std::make_unique( + io_context_, + [this](auto msg) { server_received_message.set_value(msg); }, + [this](auto &node, bool restart) { + server_cleanup.set_value(std::make_pair(node, restart)); + }); + grpc::ServerBuilder builder; + builder.AddListeningPort("0.0.0.0:18990", grpc::InsecureServerCredentials()); + builder.RegisterService(rpc_service_.get()); + server = builder.BuildAndStart(); + + client_node_id = NodeID::FromRandom(); + cli_channel = MakeChannel("18990"); + auto cli_stub = ray::rpc::syncer::RaySyncer::NewStub(cli_channel); + cli_reactor = std::make_unique( + rpc_service_->node_id.Binary(), + client_node_id.Binary(), + io_context_, + [this](auto msg) { client_received_message.set_value(msg); }, + [this](const std::string &n, bool r) { + client_cleanup.set_value(std::make_pair(n, r)); + }, + std::move(cli_stub)) + .release(); + cli_reactor->StartCall(); + + work_guard_ = std::make_unique(io_context_.get_executor()); + thread_ = std::make_unique([this]() { io_context_.run(); }); + + auto start = steady_clock::now(); + while (duration_cast(steady_clock::now() - start).count() <= 5) { + RAY_LOG(INFO) << "Waiting: " + << duration_cast(steady_clock::now() - start).count(); + if (rpc_service_->reactor != nullptr) { + break; + }; + std::this_thread::sleep_for(1s); + } + } + + void TearDown() override { + io_context_.stop(); + thread_->join(); + } + + std::pair GetReactors() { + return std::make_pair(rpc_service_->reactor, cli_reactor); + } + + std::pair GetNodeID() { + return std::make_pair(rpc_service_->node_id.Binary(), client_node_id.Binary()); + } + + void ResetPromise() { + server_received_message = std::promise>(); + client_received_message = std::promise>(); + server_cleanup = std::promise>(); + client_cleanup = std::promise>(); + } + + instrumented_io_context io_context_; + std::unique_ptr work_guard_; + std::unique_ptr thread_; + std::unique_ptr rpc_service_; + std::unique_ptr server; + std::promise> server_received_message; + std::promise> client_received_message; + std::promise> server_cleanup; + std::promise> client_cleanup; + + grpc::ClientContext cli_context; + RayClientBidiReactor *cli_reactor; + std::shared_ptr cli_channel; + NodeID client_node_id; +}; + +TEST_F(SyncerReactorTest, TestReactor) { + auto [s, c] = GetReactors(); + auto [node_s, node_c] = GetNodeID(); + ASSERT_TRUE(s != nullptr); + ASSERT_TRUE(c != nullptr); + + auto msg_s = std::make_shared(); + msg_s->set_version(1); + msg_s->set_node_id(node_s); + + s->PushToSendingQueue(msg_s); + + auto msg_c = std::make_shared(); + msg_c->set_version(2); + msg_c->set_node_id(node_c); + + c->PushToSendingQueue(msg_c); + // Make sure sending is working + auto server_received = server_received_message.get_future().get(); + auto client_received = client_received_message.get_future().get(); + ResetPromise(); + ASSERT_EQ(server_received->version(), 2); + ASSERT_EQ(server_received->node_id(), node_c); + ASSERT_EQ(client_received->version(), 1); + ASSERT_EQ(client_received->node_id(), node_s); + + s->Disconnect(); + auto c_cleanup = client_cleanup.get_future().get(); + ASSERT_EQ(node_s, c_cleanup.first); + ASSERT_EQ(false, c_cleanup.second); +} + +TEST_F(SyncerReactorTest, TestReactorFailure) { + auto [s, c] = GetReactors(); + auto [node_s, node_c] = GetNodeID(); + ASSERT_TRUE(s != nullptr); + ASSERT_TRUE(c != nullptr); + s->Finish(grpc::Status::CANCELLED); + auto c_cleanup = client_cleanup.get_future().get(); + ASSERT_EQ(node_s, c_cleanup.first); + ASSERT_EQ(true, c_cleanup.second); +} + } // namespace syncer } // namespace ray + +int main(int argc, char **argv) { + InitShutdownRAII ray_log_shutdown_raii(ray::RayLog::StartRayLog, + ray::RayLog::ShutDownRayLog, + argv[0], + ray::RayLogLevel::INFO, + /*log_dir=*/""); + ray::RayLog::InstallFailureSignalHandler(argv[0]); + ray::RayLog::InstallTerminateHandler(); + + ::testing::InitGoogleTest(&argc, argv); + auto ret = RUN_ALL_TESTS(); + // Sleep for gRPC to gracefully shutdown. + std::this_thread::sleep_for(2s); + return ret; +} diff --git a/src/ray/common/test/syncer_service_e2e_test.cc b/src/ray/common/test/syncer_service_e2e_test.cc index f8e3999439b7..0e8c37d249c3 100644 --- a/src/ray/common/test/syncer_service_e2e_test.cc +++ b/src/ray/common/test/syncer_service_e2e_test.cc @@ -123,7 +123,7 @@ int main(int argc, char *argv[]) { channel = grpc::CreateCustomChannel( "localhost:" + leader_port, grpc::InsecureChannelCredentials(), argument); - syncer.Connect(channel); + syncer.Connect(ray::NodeID::FromRandom().Binary(), channel); } boost::asio::io_context::work work(io_context); diff --git a/src/ray/gcs/gcs_server/gcs_actor_scheduler.cc b/src/ray/gcs/gcs_server/gcs_actor_scheduler.cc index 0d08b337e8d5..7fa71a8791cf 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_scheduler.cc +++ b/src/ray/gcs/gcs_server/gcs_actor_scheduler.cc @@ -444,7 +444,6 @@ void GcsActorScheduler::CreateActorOnWorker(std::shared_ptr actor, RAY_LOG(INFO) << "Start creating actor " << actor->GetActorID() << " on worker " << worker->GetWorkerID() << " at node " << actor->GetNodeID() << ", job id = " << actor->GetActorID().JobId(); - std::unique_ptr request(new rpc::PushTaskRequest()); request->set_intended_worker_id(worker->GetWorkerID().Binary()); request->mutable_task_spec()->CopyFrom( diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 706a483c3760..5d9f98db3b83 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -50,7 +50,6 @@ GcsServer::GcsServer(const ray::gcs::GcsServerConfig &config, RayConfig::instance().gcs_server_rpc_client_thread_num()), raylet_client_pool_( std::make_shared(client_call_manager_)), - local_node_id_(NodeID::FromRandom()), pubsub_periodical_runner_(pubsub_io_service_), periodical_runner_(main_service), is_started_(false), @@ -273,7 +272,7 @@ void GcsServer::InitGcsResourceManager(const GcsInitData &gcs_init_data) { gcs_resource_manager_ = std::make_shared( main_service_, cluster_resource_scheduler_->GetClusterResourceManager(), - local_node_id_, + kGCSNodeID, cluster_task_manager_); // Initialize by gcs tables data. @@ -320,7 +319,7 @@ void GcsServer::InitGcsResourceManager(const GcsInitData &gcs_init_data) { void GcsServer::InitClusterResourceScheduler() { cluster_resource_scheduler_ = std::make_shared( - scheduling::NodeID(local_node_id_.Binary()), + scheduling::NodeID(kGCSNodeID.Binary()), NodeResources(), /*is_node_available_fn=*/ [](auto) { return true; }, @@ -330,7 +329,7 @@ void GcsServer::InitClusterResourceScheduler() { void GcsServer::InitClusterTaskManager() { RAY_CHECK(cluster_resource_scheduler_); cluster_task_manager_ = std::make_shared( - local_node_id_, + kGCSNodeID, cluster_resource_scheduler_, /*get_node_info=*/ [this](const NodeID &node_id) { @@ -472,8 +471,8 @@ std::string GcsServer::StorageType() const { void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { if (RayConfig::instance().use_ray_syncer()) { - ray_syncer_ = std::make_unique(ray_syncer_io_context_, - local_node_id_.Binary()); + ray_syncer_ = + std::make_unique(ray_syncer_io_context_, kGCSNodeID.Binary()); ray_syncer_->Register( syncer::MessageType::RESOURCE_VIEW, nullptr, gcs_resource_manager_.get()); ray_syncer_->Register( @@ -482,19 +481,8 @@ void GcsServer::InitRaySyncer(const GcsInitData &gcs_init_data) { boost::asio::io_service::work work(ray_syncer_io_context_); ray_syncer_io_context_.run(); }); - - for (const auto &pair : gcs_init_data.Nodes()) { - if (pair.second.state() == - rpc::GcsNodeInfo_GcsNodeState::GcsNodeInfo_GcsNodeState_ALIVE) { - rpc::Address address; - address.set_raylet_id(pair.second.node_id()); - address.set_ip_address(pair.second.node_manager_address()); - address.set_port(pair.second.node_manager_port()); - - auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(address); - ray_syncer_->Connect(raylet_client->GetChannel()); - } - } + ray_syncer_service_ = std::make_unique(*ray_syncer_); + rpc_server_.RegisterService(*ray_syncer_service_); } else { /* The current synchronization flow is: @@ -622,9 +610,7 @@ void GcsServer::InstallEventListeners() { } cluster_task_manager_->ScheduleAndDispatchTasks(); - if (RayConfig::instance().use_ray_syncer()) { - ray_syncer_->Connect(raylet_client->GetChannel()); - } else { + if (!RayConfig::instance().use_ray_syncer()) { gcs_ray_syncer_->AddNode(*node); } }); @@ -640,9 +626,7 @@ void GcsServer::InstallEventListeners() { raylet_client_pool_->Disconnect(node_id); gcs_healthcheck_manager_->RemoveNode(node_id); - if (RayConfig::instance().use_ray_syncer()) { - ray_syncer_->Disconnect(node_id.Binary()); - } else { + if (!RayConfig::instance().use_ray_syncer()) { gcs_ray_syncer_->RemoveNode(*node); } }); @@ -776,14 +760,14 @@ void GcsServer::TryGlobalGC() { if (RayConfig::instance().use_ray_syncer()) { auto msg = std::make_shared(); msg->set_version(absl::GetCurrentTimeNanos()); - msg->set_node_id(local_node_id_.Binary()); + msg->set_node_id(kGCSNodeID.Binary()); msg->set_message_type(syncer::MessageType::COMMANDS); std::string serialized_msg; RAY_CHECK(resources_data.SerializeToString(&serialized_msg)); msg->set_sync_message(std::move(serialized_msg)); ray_syncer_->BroadcastRaySyncMessage(std::move(msg)); } else { - resources_data.set_node_id(local_node_id_.Binary()); + resources_data.set_node_id(kGCSNodeID.Binary()); gcs_ray_syncer_->Update(resources_data); } diff --git a/src/ray/gcs/gcs_server/gcs_server.h b/src/ray/gcs/gcs_server/gcs_server.h index f326d954bfea..f6397452aceb 100644 --- a/src/ray/gcs/gcs_server/gcs_server.h +++ b/src/ray/gcs/gcs_server/gcs_server.h @@ -188,9 +188,6 @@ class GcsServer { std::shared_ptr raylet_client_pool_; /// The gcs resource manager. std::shared_ptr gcs_resource_manager_; - /// The gcs server's node id, for the creation of `cluster_resource_scheduler_` and - /// `cluster_task_manager_`. - NodeID local_node_id_; /// The cluster resource scheduler. std::shared_ptr cluster_resource_scheduler_; /// The cluster task manager. @@ -226,6 +223,7 @@ class GcsServer { /// Ray Syncer realted fields. std::unique_ptr ray_syncer_; + std::unique_ptr ray_syncer_service_; std::unique_ptr ray_syncer_thread_; instrumented_io_context ray_syncer_io_context_; diff --git a/src/ray/protobuf/ray_syncer.proto b/src/ray/protobuf/ray_syncer.proto index f171665ef6f0..b24a0ecb323b 100644 --- a/src/ray/protobuf/ray_syncer.proto +++ b/src/ray/protobuf/ray_syncer.proto @@ -32,37 +32,6 @@ message RaySyncMessage { bytes node_id = 4; } -message RaySyncMessages { - // The bached messages. - repeated RaySyncMessage sync_messages = 1; -} - -message StartSyncRequest { - bytes node_id = 1; -} - -message StartSyncResponse { - bytes node_id = 1; -} - -message DummyRequest {} -message DummyResponse {} - service RaySyncer { - // Ideally these should be a streaming API like this - // rpc StartSync(stream RaySyncMessages) returns (stream RaySyncMessages); - // But to make sure it's the same as the current protocol, we still use - // unary rpc. - // TODO (iycheng): Using grpc streaming for the protocol. - - // This is the first message that should be sent. It will initialize - // some structure between nodes. - rpc StartSync(StartSyncRequest) returns (StartSyncResponse); - - // These two RPCs are for messages reporting and broadcasting. - // Update is used by the client to send update request to the server. - rpc Update(RaySyncMessages) returns (DummyResponse); - - // LongPolling is used by the server to send request to the client. - rpc LongPolling(DummyRequest) returns (RaySyncMessages); + rpc StartSync(stream RaySyncMessage) returns (stream RaySyncMessage); } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index ad019b5dc233..31f722e8bf6a 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -527,6 +527,8 @@ ray::Status NodeManager::RegisterGcs() { /* receiver */ this, /* pull_from_reporter_interval_ms */ 0); + auto gcs_channel = gcs_client_->GetGcsRpcClient().GetChannel(); + ray_syncer_.Connect(kGCSNodeID.Binary(), gcs_channel); periodical_runner_.RunFnPeriodically( [this] { auto triggered_by_global_gc = TryLocalGC(); diff --git a/src/ray/rpc/grpc_client.h b/src/ray/rpc/grpc_client.h index e5c5065030aa..8c67353e2cb1 100644 --- a/src/ray/rpc/grpc_client.h +++ b/src/ray/rpc/grpc_client.h @@ -57,7 +57,8 @@ inline std::shared_ptr BuildChannel( ::RayConfig::instance().grpc_enable_http_proxy() ? 1 : 0); arguments->SetMaxSendMessageSize(::RayConfig::instance().max_grpc_message_size()); arguments->SetMaxReceiveMessageSize(::RayConfig::instance().max_grpc_message_size()); - + arguments->SetInt(GRPC_ARG_HTTP2_WRITE_BUFFER_SIZE, + ::RayConfig::instance().grpc_stream_buffer_size()); std::shared_ptr channel; if (::RayConfig::instance().USE_TLS()) { std::string server_cert_file = std::string(::RayConfig::instance().TLS_SERVER_CERT()); diff --git a/src/ray/rpc/grpc_server.cc b/src/ray/rpc/grpc_server.cc index 962017aca9f7..d38e02d57151 100644 --- a/src/ray/rpc/grpc_server.cc +++ b/src/ray/rpc/grpc_server.cc @@ -83,7 +83,8 @@ void GrpcServer::Run() { builder.AddChannelArgument(GRPC_ARG_KEEPALIVE_TIMEOUT_MS, RayConfig::instance().grpc_keepalive_timeout_ms()); builder.AddChannelArgument(GRPC_ARG_KEEPALIVE_PERMIT_WITHOUT_CALLS, 0); - + builder.AddChannelArgument(GRPC_ARG_HTTP2_WRITE_BUFFER_SIZE, + RayConfig::instance().grpc_stream_buffer_size()); // NOTE(rickyyx): This argument changes how frequent the gRPC server expects a keepalive // ping from the client. See https://github.com/grpc/grpc/blob/HEAD/doc/keepalive.md#faq // We set this to 1min because GCS gRPC client currently sends keepalive every 1min: From 38ee064fa87fe9f2e6fa9889c49aa6de3914a001 Mon Sep 17 00:00:00 2001 From: shrekris-anyscale <92341594+shrekris-anyscale@users.noreply.github.com> Date: Thu, 26 Jan 2023 07:38:27 -0800 Subject: [PATCH 003/267] [Serve] Avoid killing replicas that are actively being deployed in `long_running_serve_failure` test (#31945) The `long_running_serve_failure` release test is unstable and passes about 50% of the time. It seems to fail after a `.deploy()` call errors with the message: ``` Traceback (most recent call last): File "workloads/serve_failure.py", line 159, in tester.run() File "workloads/serve_failure.py", line 130, in run action_chosen() File "workloads/serve_failure.py", line 107, in create_deployment handler.deploy() File "/home/ray/anaconda3/lib/python3.7/site-packages/ray/serve/deployment.py", line 217, in deploy self._deploy(*init_args, _blocking=_blocking, **init_kwargs) File "/home/ray/anaconda3/lib/python3.7/site-packages/ray/serve/deployment.py", line 244, in _deploy _blocking=_blocking, File "/home/ray/anaconda3/lib/python3.7/site-packages/ray/serve/_private/client.py", line 37, in check return f(self, *args, **kwargs) File "/home/ray/anaconda3/lib/python3.7/site-packages/ray/serve/_private/client.py", line 238, in deploy self._wait_for_deployment_healthy(name) File "/home/ray/anaconda3/lib/python3.7/site-packages/ray/serve/_private/client.py", line 174, in _wait_for_deployment_healthy f"Deployment {name} is UNHEALTHY: " f"{status.message}" RuntimeError: Deployment InWRYtjoUK is UNHEALTHY: ``` The deployment that's marked `UNHEALTHY` is always killed by the `RandomKiller` before this message appears. The issue is likely that: 1. The `RandomKiller` kills the deployment while it's still deploying 2. This causes the deployment to be [marked `UNHEALTHY`](https://github.com/ray-project/ray/blob/455100b54f0a82ce44dfa2257d40297c275424e0/python/ray/serve/_private/deployment_state.py#L1517-L1523) 3. This in turn causes [the `deploy()` call](https://github.com/ray-project/ray/blob/3735ba410ddfa63cc8516f69fa8b3997259a3f11/release/long_running_tests/workloads/serve_failure.py#L107) (and the test) [to raise an error and fail](https://github.com/ray-project/ray/blob/455100b54f0a82ce44dfa2257d40297c275424e0/python/ray/serve/_private/client.py#L172-L175). This change guards against this by preventing the `RandomKiller` from kill the deployment while it's deploying. --- python/ray/serve/_private/deployment_state.py | 6 ++- python/ray/serve/controller.py | 6 ++- .../workloads/serve_failure.py | 49 +++++++++++++------ 3 files changed, 43 insertions(+), 18 deletions(-) diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index 45b865241395..ba7cae52964c 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -1519,7 +1519,11 @@ def _check_and_update_replicas(self) -> bool: # recovered or a new deploy happens. if replica.version == self._target_state.version: self._curr_status_info: DeploymentStatusInfo = DeploymentStatusInfo( - self._name, DeploymentStatus.UNHEALTHY + name=self._name, + status=DeploymentStatus.UNHEALTHY, + message="A replica's health check failed. This " + "deployment will be UNHEALTHY until the replica " + "recovers or a new deploy happens.", ) slow_start_replicas = [] diff --git a/python/ray/serve/controller.py b/python/ray/serve/controller.py index 9dd231c84166..a05a47429e2c 100644 --- a/python/ray/serve/controller.py +++ b/python/ray/serve/controller.py @@ -312,7 +312,11 @@ def _recover_config_from_checkpoint(self): self.deploy_app(ServeApplicationSchema.parse_obj(config), update_time=False) def _all_running_replicas(self) -> Dict[str, List[RunningReplicaInfo]]: - """Used for testing.""" + """Used for testing. + + Returned dictionary maps deployment names to replica infos. + """ + return self.deployment_state_manager.get_running_replica_infos() def get_http_config(self): diff --git a/release/long_running_tests/workloads/serve_failure.py b/release/long_running_tests/workloads/serve_failure.py index fa94bfeef31c..319551b7c94f 100644 --- a/release/long_running_tests/workloads/serve_failure.py +++ b/release/long_running_tests/workloads/serve_failure.py @@ -2,11 +2,13 @@ import random import string import time +import asyncio import requests import ray from ray import serve +from ray.serve.context import get_global_client from ray.cluster_utils import Cluster from ray._private.test_utils import safe_write_to_results_json @@ -62,25 +64,35 @@ def update_progress(result): class RandomKiller: def __init__(self, kill_period_s=1): self.kill_period_s = kill_period_s + self.sanctuary = set() - def _get_all_serve_actors(self): - controller = serve.context.get_global_client()._controller + async def run(self): + while True: + chosen = random.choice(self._get_serve_actors()) + print(f"Killing {chosen}") + ray.kill(chosen, no_restart=False) + await asyncio.sleep(self.kill_period_s) + + async def spare(self, deployment_name: str): + print(f'Sparing deployment "{deployment_name}" replicas.') + self.sanctuary.add(deployment_name) + + async def stop_spare(self, deployment_name: str): + print(f'No longer sparing deployment "{deployment_name}" replicas.') + self.sanctuary.discard(deployment_name) + + def _get_serve_actors(self): + controller = get_global_client()._controller routers = list(ray.get(controller.get_http_proxies.remote()).values()) all_handles = routers + [controller] - worker_handle_dict = ray.get(controller._all_running_replicas.remote()) - for _, replica_info_list in worker_handle_dict.items(): - for replica_info in replica_info_list: - all_handles.append(replica_info.actor_handle) + replica_dict = ray.get(controller._all_running_replicas.remote()) + for deployment_name, replica_info_list in replica_dict.items(): + if deployment_name not in self.sanctuary: + for replica_info in replica_info_list: + all_handles.append(replica_info.actor_handle) return all_handles - def run(self): - while True: - chosen = random.choice(self._get_all_serve_actors()) - print(f"Killing {chosen}") - ray.kill(chosen, no_restart=False) - time.sleep(self.kill_period_s) - class RandomTest: def __init__(self, max_deployments=1): @@ -90,8 +102,11 @@ def __init__(self, max_deployments=1): (self.verify_deployment, 4), ] self.deployments = [] + self.random_killer = RandomKiller.remote() + for _ in range(max_deployments): self.create_deployment() + self.random_killer.run.remote() def create_deployment(self): if len(self.deployments) == self.max_deployments: @@ -104,10 +119,14 @@ def create_deployment(self): def handler(self, *args): return new_name - handler.deploy() + ray.get(self.random_killer.spare.remote(new_name)) + + handler.deploy(_blocking=True) self.deployments.append(new_name) + ray.get(self.random_killer.stop_spare.remote(new_name)) + def verify_deployment(self): deployment = random.choice(self.deployments) for _ in range(100): @@ -154,6 +173,4 @@ def run(self): tester = RandomTest(max_deployments=NUM_NODES * CPUS_PER_NODE) -random_killer = RandomKiller.remote() -random_killer.run.remote() tester.run() From 0c68ba46d770cbd56a7edabbf332d8ab1beffa1d Mon Sep 17 00:00:00 2001 From: Sihan Wang Date: Thu, 26 Jan 2023 07:44:25 -0800 Subject: [PATCH 004/267] [Serve] Serve python API to support multi application (#31589) - Python API for multi applications. serve.run() & serve.delete() --- .../ray/serve/api/ServeControllerClient.java | 20 +- python/ray/serve/BUILD | 10 +- python/ray/serve/__init__.py | 2 + .../ray/serve/_private/application_state.py | 224 ++++++++++++++++++ python/ray/serve/_private/client.py | 51 +++- python/ray/serve/_private/common.py | 9 +- .../serve/_private/deployment_graph_build.py | 15 +- python/ray/serve/_private/deployment_state.py | 12 +- python/ray/serve/api.py | 41 +++- python/ray/serve/controller.py | 194 ++++++++------- python/ray/serve/schema.py | 16 ++ python/ray/serve/tests/test_api.py | 108 +++++++++ .../ray/serve/tests/test_application_state.py | 136 +++++++++++ python/ray/serve/tests/test_schema.py | 27 ++- src/ray/protobuf/serve.proto | 3 + 15 files changed, 739 insertions(+), 129 deletions(-) create mode 100644 python/ray/serve/_private/application_state.py create mode 100644 python/ray/serve/tests/test_application_state.py diff --git a/java/serve/src/main/java/io/ray/serve/api/ServeControllerClient.java b/java/serve/src/main/java/io/ray/serve/api/ServeControllerClient.java index f1d8a4e51729..39c366f1ba66 100644 --- a/java/serve/src/main/java/io/ray/serve/api/ServeControllerClient.java +++ b/java/serve/src/main/java/io/ray/serve/api/ServeControllerClient.java @@ -369,19 +369,13 @@ private StatusOverview getServeStatus() { } private DeploymentStatusInfo getDeploymentStatus(String name) { - StatusOverview statusOverview = getServeStatus(); - if (statusOverview == null - || statusOverview.getDeploymentStatuses() == null - || statusOverview.getDeploymentStatuses().getDeploymentStatusInfosList() == null) { - return null; - } - for (DeploymentStatusInfo deploymentStatusInfo : - statusOverview.getDeploymentStatuses().getDeploymentStatusInfosList()) { - if (StringUtils.equals(name, deploymentStatusInfo.getName())) { - return deploymentStatusInfo; - } - } - return null; + return ServeProtoUtil.bytesToProto( + (byte[]) + ((PyActorHandle) controller) + .task(PyActorMethod.of("get_deployment_status"), name) + .remote() + .get(), + DeploymentStatusInfo::parseFrom); } public BaseActorHandle getController() { diff --git a/python/ray/serve/BUILD b/python/ray/serve/BUILD index 68935517a11b..c01a7f3a6b3d 100644 --- a/python/ray/serve/BUILD +++ b/python/ray/serve/BUILD @@ -19,7 +19,7 @@ filegroup( py_test( name = "test_api", - size = "medium", + size = "large", srcs = serve_tests_srcs, tags = ["exclusive", "team:serve"], deps = [":serve_lib"], @@ -33,6 +33,14 @@ py_test( deps = [":serve_lib"], ) +py_test( + name = "test_application_state", + size = "small", + srcs = serve_tests_srcs, + tags = ["exclusive", "team:serve"], + deps = [":serve_lib"], +) + py_test( name = "test_deploy", size = "large", diff --git a/python/ray/serve/__init__.py b/python/ray/serve/__init__.py index 5e997ee09907..23c0eac007c4 100644 --- a/python/ray/serve/__init__.py +++ b/python/ray/serve/__init__.py @@ -10,6 +10,7 @@ run, shutdown, start, + delete, ) from ray.serve.air_integrations import PredictorDeployment from ray.serve.batching import batch @@ -38,4 +39,5 @@ "list_deployments", "run", "PredictorDeployment", + "delete", ] diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py new file mode 100644 index 000000000000..636b7bba9f51 --- /dev/null +++ b/python/ray/serve/_private/application_state.py @@ -0,0 +1,224 @@ +import ray +import traceback +from typing import Dict, List +from ray.serve._private.common import ApplicationStatus +from ray.serve._private.deployment_state import DeploymentStateManager +from ray.serve._private.common import ( + DeploymentStatus, + DeploymentStatusInfo, + ApplicationStatusInfo, +) +import time +from ray.exceptions import RayTaskError, RuntimeEnvSetupError +import logging +from ray.serve._private.constants import SERVE_LOGGER_NAME +from ray.types import ObjectRef + +logger = logging.getLogger(SERVE_LOGGER_NAME) + + +class ApplicationState: + """Manage single application states with all operations""" + + def __init__( + self, + name: str, + deployment_state_manager: DeploymentStateManager, + deployment_params: List[Dict] = None, + deploy_obj_ref: ObjectRef = None, + deployment_time: float = 0, + ): + """ + Args: + name: application name + deployment_state_manager: deployment state manager which is used for + fetching deployment information + deployment_params: all deployment parameters to deploy. + deploy_obj_ref: Task ObjRef of deploying application. + deployment_time: Deployment timestamp + """ + if deploy_obj_ref: + self.status: ApplicationStatus = ApplicationStatus.DEPLOYING + else: + self.status: ApplicationStatus = ApplicationStatus.NOT_STARTED + self.name = name + self.deployment_params = deployment_params + self.to_be_deleted = False + self.deployment_state_manager = deployment_state_manager + if deployment_time: + self.deployment_timestamp = deployment_time + else: + self.deployment_timestamp = time.time() + self.deploy_obj_ref = deploy_obj_ref + self.app_msg = "" + + def delete(self): + """Delete the application""" + self.status = ApplicationStatus.DELETING + + def deploy(self): + """Deploy the application""" + self.status = ApplicationStatus.DEPLOYING + + def update(self): + """Update the application status, maintain the ApplicationStatus. + This method should be idempotent. + + Status: + DEPLOYING -> RUNNING: All deployments are healthy. + DEPLOYING -> DEPLOY_FAILED: Not all deployments are healthy. + DELETING: Mark to_be_deleted as True when all deployments are gone. + """ + + if self.to_be_deleted: + return + + if self.status == ApplicationStatus.DELETING: + mark_delete = True + for name in self.get_all_deployments(): + if self.deployment_state_manager.get_deployment(name): + mark_delete = False + break + self.to_be_deleted = mark_delete + return + + if self.status == ApplicationStatus.DEPLOYING: + if self.deploy_obj_ref: + finished, pending = ray.wait([self.deploy_obj_ref], timeout=0) + if pending: + return + try: + ray.get(finished[0]) + except RayTaskError: + self.status = ApplicationStatus.DEPLOY_FAILED + self.app_msg = f"Deployment failed:\n{traceback.format_exc()}" + self.deploy_obj_ref = None + return + except RuntimeEnvSetupError: + self.status = ApplicationStatus.DEPLOY_FAILED + self.app_msg = ( + f"Runtime env setup failed:\n{traceback.format_exc()}" + ) + self.deploy_obj_ref = None + return + deployments_statuses = ( + self.deployment_state_manager.get_deployment_statuses( + self.get_all_deployments() + ) + ) + num_health_deployments = 0 + for deployment_status in deployments_statuses: + if deployment_status.status == DeploymentStatus.UNHEALTHY: + self.status = ApplicationStatus.DEPLOY_FAILED + return + if deployment_status.status == DeploymentStatus.HEALTHY: + num_health_deployments += 1 + if num_health_deployments == len(deployments_statuses): + self.status = ApplicationStatus.RUNNING + + def get_all_deployments(self) -> List[str]: + """Return all deployments name from the application""" + if self.deployment_params is None: + return [] + return [params["name"] for params in self.deployment_params] + + def get_deployments_statuses(self) -> List[DeploymentStatusInfo]: + """Return all deployment status information""" + return self.deployment_state_manager.get_deployment_statuses( + self.get_all_deployments() + ) + + def get_application_status_info(self) -> ApplicationStatusInfo: + """Return the application status information""" + return ApplicationStatusInfo( + self.status, + message=self.app_msg, + deployment_timestamp=self.deployment_timestamp, + ) + + +class ApplicationStateManager: + def __init__(self, deployment_state_manager): + self.deployment_state_manager = deployment_state_manager + self._application_states: Dict[str, ApplicationState] = {} + + def delete_application(self, name: str): + """Delete application by name""" + if name not in self._application_states: + return + self._application_states[name].delete() + + def deploy_application(self, name: str, deployment_args: List[Dict]): + """Deploy single application + + Args: + name: application name + deployment_args: deployment args + """ + if name in self._application_states: + self._application_states[name].deployment_params = deployment_args + else: + self._application_states[name] = ApplicationState( + name, + self.deployment_state_manager, + deployment_args, + ) + self._application_states[name].deploy() + + def get_deployments(self, app_name: str) -> List[str]: + """Return all deployment names by app name""" + if app_name not in self._application_states: + return [] + return self._application_states[app_name].get_all_deployments() + + def get_deployments_statuses(self, app_name: str) -> List[DeploymentStatusInfo]: + """Return all deployment statuses by app name""" + if app_name not in self._application_states: + return [] + return self._application_states[app_name].get_deployments_statuses() + + def get_app_status(self, name: str) -> ApplicationStatusInfo: + if name not in self._application_states: + return ApplicationStatusInfo( + ApplicationStatus.NOT_STARTED, + message=f"Application {name} doesn't exist", + deployment_timestamp=0, + ) + return self._application_states[name].get_application_status_info() + + def create_application_state( + self, name: str, deploy_obj_ref: ObjectRef, deployment_time: float = 0 + ): + """Create application state + This is used for holding the deploy_obj_ref which is created by run_graph method + """ + if ( + name in self._application_states + and self._application_states[name].deploy_obj_ref + ): + logger.info( + f"Received new config deployment for {name} request. Cancelling " + "previous request." + ) + ray.cancel(self._application_states[name].deploy_obj_ref) + self._application_states[name] = ApplicationState( + name, + self.deployment_state_manager, + deploy_obj_ref=deploy_obj_ref, + deployment_time=deployment_time, + ) + + def get_deployment_timestamp(self, name: str) -> float: + if name not in self._application_states: + return -1 + return self._application_states[name].deployment_timestamp + + def update(self): + """Update each application state""" + apps_to_be_deleted = [] + for name, app in self._application_states.items(): + app.update() + if app.to_be_deleted: + apps_to_be_deleted.append(name) + for app_name in apps_to_be_deleted: + del self._application_states[app_name] diff --git a/python/ray/serve/_private/client.py b/python/ray/serve/_private/client.py index 36cfad1e45a5..653bbf5c254a 100644 --- a/python/ray/serve/_private/client.py +++ b/python/ray/serve/_private/client.py @@ -12,6 +12,7 @@ DeploymentStatus, StatusOverview, ApplicationStatus, + DeploymentStatusInfo, ) from ray.serve.config import DeploymentConfig, HTTPOptions, ReplicaConfig from ray.serve._private.constants import ( @@ -23,6 +24,9 @@ from ray.serve.exceptions import RayServeException from ray.serve.generated.serve_pb2 import DeploymentRoute, DeploymentRouteList from ray.serve.generated.serve_pb2 import StatusOverview as StatusOverviewProto +from ray.serve.generated.serve_pb2 import ( + DeploymentStatusInfo as DeploymentStatusInfoProto, +) from ray.serve.handle import RayServeHandle, RayServeSyncHandle from ray.serve.schema import ServeApplicationSchema @@ -159,14 +163,18 @@ def _wait_for_deployment_healthy(self, name: str, timeout_s: int = -1): start = time.time() while time.time() - start < timeout_s or timeout_s < 0: - status = self.get_serve_status().get_deployment_status(name) + status_bytes = ray.get(self._controller.get_deployment_status.remote(name)) - if status is None: + if status_bytes is None: raise RuntimeError( f"Waiting for deployment {name} to be HEALTHY, " "but deployment doesn't exist." ) + status = DeploymentStatusInfo.from_proto( + DeploymentStatusInfoProto.FromString(status_bytes) + ) + if status.status == DeploymentStatus.HEALTHY: break elif status.status == DeploymentStatus.UNHEALTHY: @@ -194,9 +202,14 @@ def _wait_for_deployment_deleted(self, name: str, timeout_s: int = 60): """ start = time.time() while time.time() - start < timeout_s: - curr_status = self.get_serve_status().get_deployment_status(name) - if curr_status is None: + curr_status_bytes = ray.get( + self._controller.get_deployment_status.remote(name) + ) + if curr_status_bytes is None: break + curr_status = DeploymentStatusInfo.from_proto( + DeploymentStatusInfoProto.FromString(curr_status_bytes) + ) logger.debug( f"Waiting for {name} to be deleted, current status: {curr_status}." ) @@ -241,6 +254,7 @@ def deploy( @_ensure_connected def deploy_group( self, + name, deployments: List[Dict], _blocking: bool = True, remove_past_deployments: bool = True, @@ -262,7 +276,7 @@ def deploy_group( ) updating_list = ray.get( - self._controller.deploy_group.remote(deployment_args_list) + self._controller.deploy_group.remote(name, deployment_args_list) ) tags = [] @@ -312,6 +326,29 @@ def deploy_app( f"Serve application isn't running after {timeout_s}s." ) + @_ensure_connected + def delete_apps(self, names: List[str], blocking: bool = True): + logger.info(f"Deleting app {names}") + self._controller.delete_apps.remote(names) + if blocking: + start = time.time() + while time.time() - start < 60: + curr_statuses_bytes = ray.get( + self._controller.get_serve_statuses.remote(names) + ) + all_deleted = True + for cur_status_bytes in curr_statuses_bytes: + cur_status = StatusOverview.from_proto( + StatusOverviewProto.FromString(cur_status_bytes) + ) + if cur_status.app_status.status != ApplicationStatus.NOT_STARTED: + all_deleted = False + if all_deleted: + return + time.sleep(CLIENT_POLLING_INTERVAL_S) + else: + raise TimeoutError(f"Deployment {names} wasn't deleted after 60s.") + @_ensure_connected def delete_deployments(self, names: Iterable[str], blocking: bool = True) -> None: ray.get(self._controller.delete_deployments.remote(names)) @@ -348,9 +385,9 @@ def get_app_config(self) -> Dict: return ray.get(self._controller.get_app_config.remote()) @_ensure_connected - def get_serve_status(self) -> StatusOverview: + def get_serve_status(self, name: str = "") -> StatusOverview: proto = StatusOverviewProto.FromString( - ray.get(self._controller.get_serve_status.remote()) + ray.get(self._controller.get_serve_status.remote(name)) ) return StatusOverview.from_proto(proto) diff --git a/python/ray/serve/_private/common.py b/python/ray/serve/_private/common.py index 439bec125495..faa7c422cdbb 100644 --- a/python/ray/serve/_private/common.py +++ b/python/ray/serve/_private/common.py @@ -33,6 +33,7 @@ class ApplicationStatus(str, Enum): DEPLOYING = "DEPLOYING" RUNNING = "RUNNING" DEPLOY_FAILED = "DEPLOY_FAILED" + DELETING = "DELETING" @dataclass(eq=True) @@ -92,6 +93,7 @@ def from_proto(cls, proto: DeploymentStatusInfoProto): @dataclass(eq=True) class StatusOverview: app_status: ApplicationStatusInfo + name: str = "" deployment_statuses: List[DeploymentStatusInfo] = field(default_factory=list) def debug_string(self): @@ -131,6 +133,7 @@ def to_proto(self): # Return protobuf encapsulating application and deployment protos return StatusOverviewProto( + name=self.name, app_status=app_status_proto, deployment_statuses=deployment_status_proto_list, ) @@ -147,7 +150,11 @@ def from_proto(cls, proto: StatusOverviewProto) -> "StatusOverview": deployment_statuses.append(DeploymentStatusInfo.from_proto(info_proto)) # Recreate StatusInfo - return cls(app_status=app_status, deployment_statuses=deployment_statuses) + return cls( + app_status=app_status, + deployment_statuses=deployment_statuses, + name=proto.name, + ) HEALTH_CHECK_CONCURRENCY_GROUP = "health_check" diff --git a/python/ray/serve/_private/deployment_graph_build.py b/python/ray/serve/_private/deployment_graph_build.py index 4a812bd39a3b..a75852f3e75d 100644 --- a/python/ray/serve/_private/deployment_graph_build.py +++ b/python/ray/serve/_private/deployment_graph_build.py @@ -32,7 +32,7 @@ from ray.experimental.gradio_utils import type_to_string -def build(ray_dag_root_node: DAGNode) -> List[Deployment]: +def build(ray_dag_root_node: DAGNode, name: str = None) -> List[Deployment]: """Do all the DAG transformation, extraction and generation needed to produce a runnable and deployable serve pipeline application from a valid DAG authored with Ray DAG API. @@ -64,6 +64,8 @@ def build(ray_dag_root_node: DAGNode) -> List[Deployment]: ray_dag_root_node: DAGNode acting as root of a Ray authored DAG. It should be executable via `ray_dag_root_node.execute(user_input)` and should have `InputNode` in it. + name: Application name,. If provided, formatting all the deployment name to + {name}_{deployment_name} Returns: deployments: All deployments needed for an e2e runnable serve pipeline, @@ -86,7 +88,7 @@ def build(ray_dag_root_node: DAGNode) -> List[Deployment]: """ with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag_root_node.apply_recursive( - lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) + lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator, name) ) deployments = extract_deployments_from_serve_dag(serve_root_dag) @@ -133,11 +135,12 @@ def get_and_validate_ingress_deployment( def transform_ray_dag_to_serve_dag( - dag_node: DAGNode, node_name_generator: _DAGNodeNameGenerator + dag_node: DAGNode, node_name_generator: _DAGNodeNameGenerator, name: str = None ): """ Transform a Ray DAG to a Serve DAG. Map ClassNode to DeploymentNode with ray decorated body passed in, and ClassMethodNode to DeploymentMethodNode. + When provided name, all Deployment name will {name}_{deployment_name} """ if isinstance(dag_node, ClassNode): deployment_name = node_name_generator.get_node_name(dag_node) @@ -195,6 +198,9 @@ def replace_with_handle(node): ): deployment_name = deployment_shell.name + if name: + deployment_name = f"{name}_{deployment_name}" + # Set the route prefix, prefer the one user supplied, # otherwise set it to /deployment_name if ( @@ -257,6 +263,9 @@ def replace_with_handle(node): dag_node._body.__annotations__["return"] ) + if name: + deployment_name = f"{name}_{deployment_name}" + return DeploymentFunctionNode( dag_node._body, deployment_name, diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index ba7cae52964c..9db5d6256de9 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -1979,10 +1979,14 @@ def get_deployment( else: return None - def get_deployment_statuses(self) -> List[DeploymentStatusInfo]: - return list( - map(lambda state: state.curr_status_info, self._deployment_states.values()) - ) + def get_deployment_statuses( + self, names: List[str] = None + ) -> List[DeploymentStatusInfo]: + statuses = [] + for name, state in self._deployment_states.items(): + if not names or name in names: + statuses.append(state.curr_status_info) + return statuses def deploy(self, deployment_name: str, deployment_info: DeploymentInfo) -> bool: """Deploy the deployment. diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 7ae5999b4e38..714611c0e83e 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -457,6 +457,8 @@ def run( _blocking: bool = True, host: str = DEFAULT_HTTP_HOST, port: int = DEFAULT_HTTP_PORT, + name: str = "", + route_prefix: str = "/", ) -> Optional[RayServeHandle]: """Run a Serve application and return a ServeHandle to the ingress. @@ -473,6 +475,11 @@ def run( "127.0.0.1". To expose Serve publicly, you probably want to set this to "0.0.0.0". port: Port for HTTP server. Defaults to 8000. + name: Application name. If not provided, this will be the only + application running on the cluster (it will delete all others). + route_prefix: Route prefix for HTTP requests. If not provided, it will use + route_prefix of the ingress deployment. By default, the ingress route + prefix is '/'. Returns: RayServeHandle: A regular ray serve handle that can be called by user @@ -488,14 +495,17 @@ def run( if isinstance(target, Application): deployments = list(target.deployments.values()) + if name: + for deployment in deployments: + deployment._name = f"{name}_{deployment._name}" ingress = target.ingress # Each DAG should always provide a valid Driver ClassNode elif isinstance(target, ClassNode): - deployments = pipeline_build(target) + deployments = pipeline_build(target, name) ingress = get_and_validate_ingress_deployment(deployments) # Special case where user is doing single function serve.run(func.bind()) elif isinstance(target, FunctionNode): - deployments = pipeline_build(target) + deployments = pipeline_build(target, name) ingress = get_and_validate_ingress_deployment(deployments) if len(deployments) != 1: raise ValueError( @@ -517,9 +527,18 @@ def run( f"Got unexpected type {type(target)} instead." ) + # when name provided, keep all existing applications + # otherwise, delete all of them. + remove_past_deployments = True + if name: + remove_past_deployments = False + parameter_group = [] for deployment in deployments: + # Overwrite route prefix + if route_prefix != "/" and deployment._route_prefix: + deployment._route_prefix = route_prefix deployment_parameters = { "name": deployment._name, "func_or_class": deployment._func_or_class, @@ -534,7 +553,10 @@ def run( } parameter_group.append(deployment_parameters) client.deploy_group( - parameter_group, _blocking=_blocking, remove_past_deployments=True + name, + parameter_group, + _blocking=_blocking, + remove_past_deployments=remove_past_deployments, ) if ingress is not None: @@ -574,3 +596,16 @@ def build(target: Union[ClassNode, FunctionNode]) -> Application: # TODO(edoakes): this should accept host and port, but we don't # currently support them in the REST API. return Application(pipeline_build(target)) + + +@PublicAPI(stability="alpha") +def delete(name: str, _blocking: bool = True): + """Delete an app by its name + + Deletes the app with all corresponding deployments. + + Args: + name: the name of app to delete. + """ + client = get_global_client() + client.delete_apps([name], blocking=_blocking) diff --git a/python/ray/serve/controller.py b/python/ray/serve/controller.py index a05a47429e2c..0d4d13734497 100644 --- a/python/ray/serve/controller.py +++ b/python/ray/serve/controller.py @@ -4,7 +4,6 @@ import os import pickle import time -import traceback from collections import defaultdict from typing import Any, Dict, Iterable, List, Optional, Tuple, Union @@ -15,12 +14,9 @@ ) from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy from ray.actor import ActorHandle -from ray.exceptions import RayTaskError, RuntimeEnvSetupError from ray._private.gcs_utils import GcsClient from ray.serve._private.autoscaling_policy import BasicAutoscalingPolicy from ray.serve._private.common import ( - ApplicationStatus, - ApplicationStatusInfo, DeploymentInfo, EndpointInfo, EndpointTag, @@ -48,7 +44,7 @@ override_runtime_envs_except_env_vars, get_random_letters, ) -from ray.types import ObjectRef +from ray.serve._private.application_state import ApplicationStateManager logger = logging.getLogger(SERVE_LOGGER_NAME) @@ -146,11 +142,10 @@ async def __init__( all_serve_actor_names, ) - # Reference to Ray task executing most recent deployment request - self.config_deployment_request_ref: ObjectRef = None - - # Unix timestamp of latest config deployment request. Defaults to 0. - self.deployment_timestamp = 0 + # Manage all applications' state + self.application_state_manager = ApplicationStateManager( + self.deployment_state_manager + ) run_background_task(self.run_control_loop()) @@ -255,6 +250,10 @@ async def run_control_loop(self) -> None: self.deployment_state_manager.update() except Exception: logger.exception("Exception updating deployment state.") + try: + self.application_state_manager.update() + except Exception: + logger.exception("Exception updating application state.") try: self._put_serve_snapshot() @@ -308,8 +307,12 @@ def _put_serve_snapshot(self) -> None: def _recover_config_from_checkpoint(self): checkpoint = self.kv_store.get(CONFIG_CHECKPOINT_KEY) if checkpoint is not None: - self.deployment_timestamp, config, _ = pickle.loads(checkpoint) - self.deploy_app(ServeApplicationSchema.parse_obj(config), update_time=False) + config_checkpoints_dict = pickle.loads(checkpoint) + for name in config_checkpoints_dict: + deployment_time, config, _ = config_checkpoints_dict[name] + self.deploy_app( + ServeApplicationSchema.parse_obj(config), deployment_time + ) def _all_running_replicas(self) -> Dict[str, List[RunningReplicaInfo]]: """Used for testing. @@ -415,7 +418,7 @@ def deploy( return updating - def deploy_group(self, deployment_args_list: List[Dict]) -> List[bool]: + def deploy_group(self, name: str, deployment_args_list: List[Dict]) -> List[bool]: """ Takes in a list of dictionaries that contain keyword arguments for the controller's deploy() function. Calls deploy on all the argument @@ -423,66 +426,77 @@ def deploy_group(self, deployment_args_list: List[Dict]) -> List[bool]: group of deployments. """ - return [self.deploy(**args) for args in deployment_args_list] + deployments_success = [self.deploy(**args) for args in deployment_args_list] + self.application_state_manager.deploy_application(name, deployment_args_list) + return deployments_success def deploy_app( - self, config: ServeApplicationSchema, update_time: bool = True + self, + config: ServeApplicationSchema, + deployment_time: float = 0, ) -> None: """Kicks off a task that deploys a Serve application. - Cancels any previous in-progress task that is deploying a Serve - application. + Cancels in-progress task that is deploying a Serve + application with same name. Args: config: Contains the following: + name: Application name. If not provided, it is empty string. import_path: Serve deployment graph's import path runtime_env: runtime_env to run the deployment graph in deployment_override_options: Dictionaries that contain argument-value options that can be passed directly into a set_options() call. Overrides deployment options set in the graph's code itself. - update_time: Whether to update the deployment_timestamp. + deployment_time: set deployment_timestamp. If not provided, time.time() is + used to indicate the deployment time. """ - - if update_time: - self.deployment_timestamp = time.time() - config_dict = config.dict(exclude_unset=True) # Compare new config options with old ones and set versions of new deployments - config_checkpoint = self.kv_store.get(CONFIG_CHECKPOINT_KEY) - - if config_checkpoint is not None: - _, last_config_dict, last_version_dict = pickle.loads(config_checkpoint) + config_checkpoints = self.kv_store.get(CONFIG_CHECKPOINT_KEY) + if config_checkpoints is None: + config_checkpoints_dict = {} + else: + config_checkpoints_dict = pickle.loads(config_checkpoints) + if config.name in config_checkpoints_dict: + _, last_config_dict, last_version_dict = config_checkpoints_dict[ + config.name + ] updated_version_dict = _generate_deployment_config_versions( config_dict, last_config_dict, last_version_dict ) else: updated_version_dict = _generate_deployment_config_versions(config_dict) + deployment_override_options = config_dict.get("deployments", []) + + if not deployment_time: + deployment_time = time.time() + + config_checkpoints_dict[config.name] = ( + deployment_time, + config_dict, + updated_version_dict, + ) self.kv_store.put( CONFIG_CHECKPOINT_KEY, - pickle.dumps( - (self.deployment_timestamp, config_dict, updated_version_dict) - ), + pickle.dumps(config_checkpoints_dict), ) - deployment_override_options = config_dict.get("deployments", []) - - if self.config_deployment_request_ref is not None: - ray.cancel(self.config_deployment_request_ref) - logger.info( - "Received new config deployment request. Cancelling " - "previous request." - ) - - self.config_deployment_request_ref = run_graph.options( - runtime_env=config.runtime_env - ).remote( + deploy_obj_ref = run_graph.options(runtime_env=config.runtime_env).remote( config.import_path, config.runtime_env, deployment_override_options, updated_version_dict, + config.name, + config_dict.get("route_prefix", "/"), + ) + self.application_state_manager.create_application_state( + config.name, + deploy_obj_ref=deploy_obj_ref, + deployment_time=deployment_time, ) def delete_deployment(self, name: str): @@ -571,53 +585,62 @@ def list_deployments(self, include_deleted: Optional[bool] = False) -> bytes: ) return deployment_route_list.SerializeToString() - async def get_serve_status(self) -> bytes: - - serve_app_status = ApplicationStatus.RUNNING - serve_app_message = "" - deployment_timestamp = self.deployment_timestamp - - if self.config_deployment_request_ref: - finished, pending = ray.wait( - [self.config_deployment_request_ref], timeout=0 - ) + def get_serve_status(self, name: str = "") -> bytes: + """Return application status + Args: + name: application name. If application name doesn't exist, app_status + is NOT_STARTED. + """ - if pending: - serve_app_status = ApplicationStatus.DEPLOYING - else: - try: - await finished[0] - except Exception as e: - serve_app_status = ApplicationStatus.DEPLOY_FAILED - tb = traceback.format_exc() - - if isinstance(e, RayTaskError): - serve_app_message = f"Deployment failed:\n{tb}" - elif isinstance(e, RuntimeEnvSetupError): - serve_app_message = f"Runtime env setup failed:\n{tb}" - else: - serve_app_message = f"Unknown error occurred:\n{tb}" - - app_status = ApplicationStatusInfo( - serve_app_status, serve_app_message, deployment_timestamp + app_status = self.application_state_manager.get_app_status(name) + deployment_statuses = self.application_state_manager.get_deployments_statuses( + name ) - deployment_statuses = self.deployment_state_manager.get_deployment_statuses() - status_info = StatusOverview( + name=name, app_status=app_status, deployment_statuses=deployment_statuses, ) - return status_info.to_proto().SerializeToString() - def get_app_config(self) -> Dict: + def get_serve_statuses(self, names: List[str]) -> List[bytes]: + statuses = [] + for name in names: + statuses.append(self.get_serve_status(name)) + return statuses + + def get_app_config(self, name: str = "") -> Dict: checkpoint = self.kv_store.get(CONFIG_CHECKPOINT_KEY) if checkpoint is None: return ServeApplicationSchema.get_empty_schema_dict() else: - _, config, _ = pickle.loads(checkpoint) + config_checkpoints_dict = pickle.loads(checkpoint) + if name not in config_checkpoints_dict: + return ServeApplicationSchema.get_empty_schema_dict() + _, config, _ = config_checkpoints_dict[name] + return config + def get_deployment_status(self, name: str) -> Union[None, bytes]: + """Get deployment status by deployment name""" + status = self.deployment_state_manager.get_deployment_statuses([name]) + if not status: + return None + return status[0].to_proto().SerializeToString() + + def delete_apps(self, names: Iterable[str]): + """Delete applications based on names + + During deletion, the application status is DELETING + """ + deployments_to_delete = [] + for name in names: + deployments_to_delete.extend( + self.application_state_manager.get_deployments(name) + ) + self.application_state_manager.delete_application(name) + self.delete_deployments(deployments_to_delete) + def _generate_deployment_config_versions( new_config: Dict, @@ -710,9 +733,11 @@ def run_graph( graph_env: Dict, deployment_override_options: List[Dict], deployment_versions: Dict, + name: str = "", + route_prefix: str = "/", ): """ - Deploys a Serve application to the controller's Ray cluster. + Build application object from user config Args: import_path: Serve deployment graph's import path @@ -722,6 +747,9 @@ def run_graph( deployment_versions: Versions of each deployment, each of which is the same as the last deployment if it is a config update or a new randomly generated version if it is a code update + name: application name. If specified, application will be deployed + without removing existing applications. + route_prefix: route_prefix. Define the route path for the application. """ try: from ray import serve @@ -733,7 +761,7 @@ def run_graph( # Override options for each deployment for options in deployment_override_options: - name = options["name"] + deployment_name = options["name"] # Merge graph-level and deployment-level runtime_envs if "ray_actor_options" in options: @@ -742,23 +770,21 @@ def run_graph( else: # Otherwise, get options from graph code (and default to {} if code # sets options to None) - ray_actor_options = app.deployments[name].ray_actor_options or {} - + ray_actor_options = ( + app.deployments[deployment_name].ray_actor_options or {} + ) deployment_env = ray_actor_options.get("runtime_env", {}) merged_env = override_runtime_envs_except_env_vars( graph_env, deployment_env ) - ray_actor_options.update({"runtime_env": merged_env}) options["ray_actor_options"] = ray_actor_options - - options["version"] = deployment_versions[name] - + options["version"] = deployment_versions[deployment_name] # Update the deployment's options - app.deployments[name].set_options(**options, _internal=True) + app.deployments[deployment_name].set_options(**options, _internal=True) # Run the graph locally on the cluster - serve.run(app) + serve.run(app, name=name, route_prefix=route_prefix) except KeyboardInterrupt: # Error is raised when this task is canceled with ray.cancel(), which # happens when deploy_app() is called. diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 2a7e5f86c2ca..79f1e347bc5e 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -242,6 +242,20 @@ def get_user_configured_option_names(self) -> Set[str]: @PublicAPI(stability="beta") class ServeApplicationSchema(BaseModel, extra=Extra.forbid): + name: str = Field( + default="", + description=( + "Application name, the name should be unique within the serve instance" + ), + ) + route_prefix: str = Field( + default="/", + description=( + "Route prefix for HTTP requests. If not provided, it will use" + "route_prefix of the ingress deployment. By default, the ingress route" + "prefix is '/'." + ), + ) import_path: str = Field( default=None, description=( @@ -398,6 +412,7 @@ def kubernetes_dict(self, **kwargs) -> Dict: @PublicAPI(stability="beta") class ServeStatusSchema(BaseModel, extra=Extra.forbid): + name: str = Field(description="Application name", default="") app_status: ApplicationStatusInfo = Field( ..., description=( @@ -438,6 +453,7 @@ def get_empty_schema_dict() -> Dict: def serve_status_to_schema(serve_status: StatusOverview) -> ServeStatusSchema: return ServeStatusSchema( + name=serve_status.name, app_status=serve_status.app_status, deployment_statuses=serve_status.deployment_statuses, ) diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index ebb009293d3e..9acbc2b2c2ad 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -5,11 +5,13 @@ import requests import pytest import starlette.responses +from fastapi import FastAPI import ray from ray import serve from ray._private.test_utils import SignalActor, wait_for_condition from ray.serve.application import Application +from ray.serve.drivers import DAGDriver @serve.deployment() @@ -476,6 +478,112 @@ def f(): f.set_options(max_concurrent_queries=-4) +def test_deploy_application(serve_instance): + """Test deploy multiple applications""" + + @serve.deployment + def f(): + return "got f" + + @serve.deployment + def g(): + return "got g" + + @serve.deployment(route_prefix="/my_prefix") + def h(): + return "got h" + + @serve.deployment + class Model1: + def __call__(self): + return "got model1" + + app = FastAPI() + + @serve.deployment(route_prefix="/hello") + @serve.ingress(app) + class MyFastAPIDeployment: + @app.get("/") + def root(self): + return "Hello, world!" + + # Test function deployment with app name + f_handle = serve.run(f.bind(), name="app_f") + assert ray.get(f_handle.remote()) == "got f" + assert requests.get("http://127.0.0.1:8000/").text == "got f" + + # Test function deployment with app name and route_prefix + g_handle = serve.run(g.bind(), name="app_g", route_prefix="/app_g") + assert ray.get(g_handle.remote()) == "got g" + assert requests.get("http://127.0.0.1:8000/app_g").text == "got g" + + # Test function deployment with app name and route_prefix set in deployment + # decorator + h_handle = serve.run(h.bind(), name="app_h") + assert ray.get(h_handle.remote()) == "got h" + assert requests.get("http://127.0.0.1:8000/my_prefix").text == "got h" + + # Test deployment graph + graph_handle = serve.run( + DAGDriver.bind(Model1.bind()), name="graph", route_prefix="/my_graph" + ) + assert ray.get(graph_handle.predict.remote()) == "got model1" + assert requests.get("http://127.0.0.1:8000/my_graph").text == '"got model1"' + + # Test FastAPI + serve.run(MyFastAPIDeployment.bind(), name="FastAPI") + assert requests.get("http://127.0.0.1:8000/hello").text == '"Hello, world!"' + + +def test_delete_application(serve_instance): + """Test delete single application""" + + @serve.deployment + def f(): + return "got f" + + @serve.deployment + def g(): + return "got g" + + f_handle = serve.run(f.bind(), name="app_f") + g_handle = serve.run(g.bind(), name="app_g", route_prefix="/app_g") + assert ray.get(f_handle.remote()) == "got f" + assert requests.get("http://127.0.0.1:8000/").text == "got f" + + serve.delete("app_f") + assert "Path '/' not found" in requests.get("http://127.0.0.1:8000/").text + + # delete again, no exception & crash expected. + serve.delete("app_f") + + # make sure no affect to app_g + assert ray.get(g_handle.remote()) == "got g" + assert requests.get("http://127.0.0.1:8000/app_g").text == "got g" + + +def test_deployment_name_with_app_name(): + """Test replica name with app name as prefix""" + + controller = serve.context._global_client._controller + + @serve.deployment + def g(): + return "got g" + + serve.run(g.bind()) + deployment_info = ray.get(controller._all_running_replicas.remote()) + assert "g" in deployment_info + + @serve.deployment + def f(): + return "got f" + + serve.run(f.bind(), name="app1") + deployment_info = ray.get(controller._all_running_replicas.remote()) + assert "app1_f" in deployment_info + + if __name__ == "__main__": import sys diff --git a/python/ray/serve/tests/test_application_state.py b/python/ray/serve/tests/test_application_state.py new file mode 100644 index 000000000000..82b5a4e16038 --- /dev/null +++ b/python/ray/serve/tests/test_application_state.py @@ -0,0 +1,136 @@ +import sys +import pytest +from typing import List +import time + +import ray +from ray._private.test_utils import SignalActor +from ray.serve._private.application_state import ApplicationStateManager +from ray.serve._private.common import ApplicationStatus +from ray.serve._private.common import DeploymentStatus, DeploymentStatusInfo + + +class MockDeploymentStateManager: + def __init__(self): + self.deployment_statuses = [ + DeploymentStatusInfo("d1", DeploymentStatus.UPDATING), + DeploymentStatusInfo("d2", DeploymentStatus.UPDATING), + ] + + def set_deployment_statuses_unhealthy(self, index: int = 0): + self.deployment_statuses[index].status = DeploymentStatus.UNHEALTHY + + def set_deployment_statuses_healthy(self, index: int = 0): + self.deployment_statuses[index].status = DeploymentStatus.HEALTHY + + def get_deployment_statuses(self, deployment_names: List[str]): + return self.deployment_statuses + + def get_all_deployments(self): + return ["d1", "d2"] + + +def test_deploy_app(): + """Test DEPLOYING status""" + app_state_manager = ApplicationStateManager(MockDeploymentStateManager()) + app_state_manager.deploy_application("test_app", {}) + + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOYING + assert app_status.deployment_timestamp > 0 + + +def test_delete_app(): + """Test DELETING status""" + app_state_manager = ApplicationStateManager(MockDeploymentStateManager()) + app_state_manager.deploy_application("test_app", {}) + app_state_manager.delete_application("test_app") + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DELETING + + +def test_create_app(): + """Test object ref based deploy and set DEPLOYING""" + app_state_manager = ApplicationStateManager(MockDeploymentStateManager()) + app_state_manager.create_application_state("test_app", ray.ObjectRef.nil()) + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOYING + + +def test_update_app_running(): + """Test DEPLOYING -> RUNNING""" + app_state_manager = ApplicationStateManager(MockDeploymentStateManager()) + app_state_manager.deploy_application("test_app", {}) + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOYING + app_state_manager.deployment_state_manager.set_deployment_statuses_healthy(0) + app_state_manager.update() + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOYING + app_state_manager.deployment_state_manager.set_deployment_statuses_healthy(1) + app_state_manager.update() + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.RUNNING + + # rerun update, application status should not make difference + app_state_manager.update() + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.RUNNING + + +def test_update_app_deploy_failed(): + """Test DEPLOYING -> DEPLOY_FAILED""" + app_state_manager = ApplicationStateManager(MockDeploymentStateManager()) + app_state_manager.deploy_application("test_app", {}) + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOYING + app_state_manager.deployment_state_manager.set_deployment_statuses_unhealthy(0) + app_state_manager.update() + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOY_FAILED + # rerun update, application status should not make difference + app_state_manager.update() + assert app_status.status == ApplicationStatus.DEPLOY_FAILED + + +@pytest.mark.skipif(sys.platform == "win32", reason="Failing on Windows.") +@pytest.mark.parametrize("fail_deploy", [False, True]) +def test_config_deploy_app(fail_deploy): + """Test config based deploy + DEPLOYING -> RUNNING + DEPLOYING -> DEPLOY_FAILED + """ + signal = SignalActor.remote() + + @ray.remote + def task(): + ray.get(signal.wait.remote()) + if fail_deploy: + raise Exception("fail!") + + object_ref = task.remote() + app_state_manager = ApplicationStateManager(MockDeploymentStateManager()) + app_state_manager.create_application_state("test_app", object_ref) + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOYING + + app_state_manager.update() + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOYING + + signal.send.remote() + time.sleep(2) + if fail_deploy: + app_state_manager.update() + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.DEPLOY_FAILED + else: + app_state_manager.deployment_state_manager.set_deployment_statuses_healthy(0) + app_state_manager.deployment_state_manager.set_deployment_statuses_healthy(1) + app_state_manager.update() + app_status = app_state_manager.get_app_status("test_app") + assert app_status.status == ApplicationStatus.RUNNING + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", "-s", __file__])) diff --git a/python/ray/serve/tests/test_schema.py b/python/ray/serve/tests/test_schema.py index 0e289b0eabc2..c0e94936db72 100644 --- a/python/ray/serve/tests/test_schema.py +++ b/python/ray/serve/tests/test_schema.py @@ -664,22 +664,23 @@ def f1(): def f2(): pass - f1._func_or_class = "ray.serve.tests.test_schema.global_f" - f2._func_or_class = "ray.serve.tests.test_schema.global_f" - client = serve.start() - - f1.deploy() - f2.deploy() + serve.run(f1.bind(), name="app1") + serve.run(f2.bind(), name="app2") # Check statuses - statuses = serve_status_to_schema(client.get_serve_status()).deployment_statuses - deployment_names = {"f1", "f2"} - for deployment_status in statuses: - assert deployment_status.status in {"UPDATING", "HEALTHY"} - assert deployment_status.name in deployment_names - deployment_names.remove(deployment_status.name) - assert len(deployment_names) == 0 + f1_statuses = serve_status_to_schema( + client.get_serve_status("app1") + ).deployment_statuses + f2_statuses = serve_status_to_schema( + client.get_serve_status("app2") + ).deployment_statuses + assert len(f1_statuses) == 1 + assert f1_statuses[0].status in {"UPDATING", "HEALTHY"} + assert f1_statuses[0].name == "app1_f1" + assert len(f2_statuses) == 1 + assert f2_statuses[0].status in {"UPDATING", "HEALTHY"} + assert f2_statuses[0].name == "app2_f2" serve.shutdown() diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index f75beaa9ecdc..d328a7f4d8fe 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -203,6 +203,8 @@ enum ApplicationStatus { DEPLOYING = 0; RUNNING = 1; DEPLOY_FAILED = 2; + DELETING = 3; + NOT_STARTED = 5; } message ApplicationStatusInfo { @@ -214,6 +216,7 @@ message ApplicationStatusInfo { message StatusOverview { ApplicationStatusInfo app_status = 1; DeploymentStatusInfoList deployment_statuses = 2; + string name = 3; } // RPC Schema From 0cb80704509fd789c6da13ad70dfaa5d218fbd7d Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Thu, 26 Jan 2023 17:54:42 +0100 Subject: [PATCH 005/267] [RLlib] AlgorithmConfig objects supported by all (internally used) `Algorithm.default_resource_request()` methods. (#31958) --- rllib/algorithms/algorithm.py | 9 ---- rllib/algorithms/alpha_star/alpha_star.py | 42 +++++++++------- rllib/algorithms/apex_dqn/apex_dqn.py | 48 +++++++++---------- rllib/algorithms/impala/impala.py | 44 ++++++++--------- rllib/algorithms/mock.py | 2 +- ...e_envs_with_inference_done_on_main_node.py | 21 +++++--- 6 files changed, 86 insertions(+), 80 deletions(-) diff --git a/rllib/algorithms/algorithm.py b/rllib/algorithms/algorithm.py index 81859387d005..61b75e8e4fbf 100644 --- a/rllib/algorithms/algorithm.py +++ b/rllib/algorithms/algorithm.py @@ -2725,15 +2725,6 @@ def _run_one_evaluation( Returns: The results dict from the evaluation call. """ - - eval_results = { - "evaluation": { - "episode_reward_max": np.nan, - "episode_reward_min": np.nan, - "episode_reward_mean": np.nan, - } - } - eval_func_to_use = ( self._evaluate_async if self.config.enable_async_evaluation diff --git a/rllib/algorithms/alpha_star/alpha_star.py b/rllib/algorithms/alpha_star/alpha_star.py index a388895b31cf..e829bde1900a 100644 --- a/rllib/algorithms/alpha_star/alpha_star.py +++ b/rllib/algorithms/alpha_star/alpha_star.py @@ -2,7 +2,7 @@ A multi-agent, distributed multi-GPU, league-capable asynch. PPO ================================================================ """ -from typing import Any, Dict, Optional, Type +from typing import Any, Dict, Optional, Type, Union import gymnasium as gym import tree @@ -257,25 +257,31 @@ class AlphaStar(appo.APPO): @classmethod @override(Algorithm) - def default_resource_request(cls, config): - cf = dict(cls.get_default_config(), **config) + def default_resource_request( + cls, + config: Union[AlgorithmConfig, PartialAlgorithmConfigDict], + ): + if isinstance(config, AlgorithmConfig): + cf: AlphaStarConfig = config + else: + cf: AlphaStarConfig = cls.get_default_config().update_from_dict(config) # Construct a dummy LeagueBuilder, such that it gets the opportunity to # adjust the multiagent config, according to its setup, and we can then # properly infer the resources to allocate. - from_config(cf["league_builder_config"], algo=None, algo_config=cf) + from_config(cf.league_builder_config, algo=None, algo_config=cf) - max_num_policies_to_train = cf["max_num_policies_to_train"] or len( - cf["multiagent"].get("policies_to_train") or cf["multiagent"]["policies"] + max_num_policies_to_train = cf.max_num_policies_to_train or len( + cf.policies_to_train or cf.policies ) num_learner_shards = min( - cf["num_gpus"] or max_num_policies_to_train, max_num_policies_to_train + cf.num_gpus or max_num_policies_to_train, max_num_policies_to_train ) - num_gpus_per_shard = cf["num_gpus"] / num_learner_shards + num_gpus_per_shard = cf.num_gpus / num_learner_shards num_policies_per_shard = max_num_policies_to_train / num_learner_shards - fake_gpus = cf["_fake_gpus"] + fake_gpus = cf._fake_gpus - eval_config = cf["evaluation_config"] + eval_config = cf.get_evaluation_config_object() # Return PlacementGroupFactory containing all needed resources # (already properly defined as device bundles). @@ -283,15 +289,15 @@ def default_resource_request(cls, config): bundles=[ { # Driver (no GPUs). - "CPU": cf["num_cpus_for_driver"], + "CPU": cf.num_cpus_for_local_worker, } ] + [ { # RolloutWorkers (no GPUs). - "CPU": cf["num_cpus_per_worker"], + "CPU": cf.num_cpus_per_worker, } - for _ in range(cf["num_workers"]) + for _ in range(cf.num_workers) ] + [ { @@ -310,20 +316,20 @@ def default_resource_request(cls, config): # Note: The local eval worker is located on the driver # CPU or not even created iff >0 eval workers. "CPU": eval_config.get( - "num_cpus_per_worker", cf["num_cpus_per_worker"] + "num_cpus_per_worker", cf.num_cpus_per_worker ), } - for _ in range(cf["evaluation_num_workers"]) + for _ in range(cf.evaluation_num_workers) ] - if cf["evaluation_interval"] + if cf.evaluation_interval else [] ), - strategy=config.get("placement_strategy", "PACK"), + strategy=cf.placement_strategy, ) @classmethod @override(appo.APPO) - def get_default_config(cls) -> AlgorithmConfig: + def get_default_config(cls) -> AlphaStarConfig: return AlphaStarConfig() @override(appo.APPO) diff --git a/rllib/algorithms/apex_dqn/apex_dqn.py b/rllib/algorithms/apex_dqn/apex_dqn.py index 4d46c0690039..5eea96eccc8e 100644 --- a/rllib/algorithms/apex_dqn/apex_dqn.py +++ b/rllib/algorithms/apex_dqn/apex_dqn.py @@ -15,7 +15,7 @@ import platform import random from collections import defaultdict -from typing import Any, Dict, List, Optional, Tuple +from typing import Any, Dict, List, Optional, Tuple, Union import ray from ray._private.dict import merge_dicts @@ -42,6 +42,7 @@ TARGET_NET_UPDATE_TIMER, ) from ray.rllib.utils.typing import ( + PartialAlgorithmConfigDict, ResultDict, SampleBatchType, ) @@ -700,10 +701,16 @@ def _compile_iteration_results(self, *args, **kwargs): @classmethod @override(Algorithm) - def default_resource_request(cls, config): - cf = dict(cls.get_default_config(), **config) + def default_resource_request( + cls, + config: Union[AlgorithmConfig, PartialAlgorithmConfigDict], + ): + if isinstance(config, AlgorithmConfig): + cf: ApexDQNConfig = config + else: + cf: ApexDQNConfig = cls.get_default_config().update_from_dict(config) - eval_config = cf["evaluation_config"] + eval_config = cf.get_evaluation_config_object() # Return PlacementGroupFactory containing all needed resources # (already properly defined as device bundles). @@ -715,19 +722,19 @@ def default_resource_request(cls, config): # data bandwidth between buffers and the learner (driver). # Replay buffer actors each contain one shard of the total # replay buffer and use 1 CPU each. - "CPU": cf["num_cpus_for_driver"] - + cf["optimizer"]["num_replay_buffer_shards"], - "GPU": 0 if cf["_fake_gpus"] else cf["num_gpus"], + "CPU": cf.num_cpus_for_local_worker + + cf.optimizer["num_replay_buffer_shards"], + "GPU": 0 if cf._fake_gpus else cf.num_gpus, } ] + [ { # RolloutWorkers. - "CPU": cf["num_cpus_per_worker"], - "GPU": cf["num_gpus_per_worker"], - **cf["custom_resources_per_worker"], + "CPU": cf.num_cpus_per_worker, + "GPU": cf.num_gpus_per_worker, + **cf.custom_resources_per_worker, } - for _ in range(cf["num_workers"]) + for _ in range(cf.num_rollout_workers) ] + ( [ @@ -735,23 +742,16 @@ def default_resource_request(cls, config): # Evaluation workers. # Note: The local eval worker is located on the driver # CPU. - "CPU": eval_config.get( - "num_cpus_per_worker", cf["num_cpus_per_worker"] - ), - "GPU": eval_config.get( - "num_gpus_per_worker", cf["num_gpus_per_worker"] - ), - **eval_config.get( - "custom_resources_per_worker", - cf["custom_resources_per_worker"], - ), + "CPU": eval_config.num_cpus_per_worker, + "GPU": eval_config.num_gpus_per_worker, + **eval_config.custom_resources_per_worker, } - for _ in range(cf["evaluation_num_workers"]) + for _ in range(cf.evaluation_num_workers) ] - if cf["evaluation_interval"] + if cf.evaluation_interval else [] ), - strategy=config.get("placement_strategy", "PACK"), + strategy=cf.placement_strategy, ) diff --git a/rllib/algorithms/impala/impala.py b/rllib/algorithms/impala/impala.py index 301589e095b7..bea342df72c2 100644 --- a/rllib/algorithms/impala/impala.py +++ b/rllib/algorithms/impala/impala.py @@ -42,6 +42,7 @@ from ray.rllib.utils.metrics.learner_info import LearnerInfoBuilder from ray.rllib.utils.typing import ( + PartialAlgorithmConfigDict, PolicyID, ResultDict, SampleBatchType, @@ -593,10 +594,16 @@ def training_step(self) -> ResultDict: @classmethod @override(Algorithm) - def default_resource_request(cls, config): - cf = dict(cls.get_default_config(), **config) + def default_resource_request( + cls, + config: Union[AlgorithmConfig, PartialAlgorithmConfigDict], + ): + if isinstance(config, AlgorithmConfig): + cf: ImpalaConfig = config + else: + cf: ImpalaConfig = cls.get_default_config().update_from_dict(config) - eval_config = cf["evaluation_config"] + eval_config = cf.get_evaluation_config_object() # Return PlacementGroupFactory containing all needed resources # (already properly defined as device bundles). @@ -610,18 +617,18 @@ def default_resource_request(cls, config): # from RolloutWorkers (n rollout workers map to m # aggregation workers, where m < n) and always use 1 CPU # each. - "CPU": cf["num_cpus_for_driver"] + cf["num_aggregation_workers"], - "GPU": 0 if cf["_fake_gpus"] else cf["num_gpus"], + "CPU": cf.num_cpus_for_local_worker + cf.num_aggregation_workers, + "GPU": 0 if cf._fake_gpus else cf.num_gpus, } ] + [ { # RolloutWorkers. - "CPU": cf["num_cpus_per_worker"], - "GPU": cf["num_gpus_per_worker"], - **cf["custom_resources_per_worker"], + "CPU": cf.num_cpus_per_worker, + "GPU": cf.num_gpus_per_worker, + **cf.custom_resources_per_worker, } - for _ in range(cf["num_workers"]) + for _ in range(cf.num_rollout_workers) ] + ( [ @@ -629,23 +636,16 @@ def default_resource_request(cls, config): # Evaluation (remote) workers. # Note: The local eval worker is located on the driver # CPU or not even created iff >0 eval workers. - "CPU": eval_config.get( - "num_cpus_per_worker", cf["num_cpus_per_worker"] - ), - "GPU": eval_config.get( - "num_gpus_per_worker", cf["num_gpus_per_worker"] - ), - **eval_config.get( - "custom_resources_per_worker", - cf["custom_resources_per_worker"], - ), + "CPU": eval_config.num_cpus_per_worker, + "GPU": eval_config.num_gpus_per_worker, + **eval_config.custom_resources_per_worker, } - for _ in range(cf["evaluation_num_workers"]) + for _ in range(cf.evaluation_num_workers) ] - if cf["evaluation_interval"] + if cf.evaluation_interval else [] ), - strategy=config.get("placement_strategy", "PACK"), + strategy=cf.placement_strategy, ) def concatenate_batches_and_pre_queue(self, batches: List[SampleBatch]): diff --git a/rllib/algorithms/mock.py b/rllib/algorithms/mock.py index b00b72a7c546..f89955812b1e 100644 --- a/rllib/algorithms/mock.py +++ b/rllib/algorithms/mock.py @@ -27,7 +27,7 @@ def get_default_config(cls) -> AlgorithmConfig: ) @classmethod - def default_resource_request(cls, config): + def default_resource_request(cls, config: AlgorithmConfig): return None @override(Algorithm) diff --git a/rllib/examples/remote_envs_with_inference_done_on_main_node.py b/rllib/examples/remote_envs_with_inference_done_on_main_node.py index ed9957f98cfe..bbefcc0439e1 100644 --- a/rllib/examples/remote_envs_with_inference_done_on_main_node.py +++ b/rllib/examples/remote_envs_with_inference_done_on_main_node.py @@ -11,13 +11,16 @@ """ import argparse import os +from typing import Union import ray +from ray import air, tune from ray.rllib.algorithms.ppo import PPO, PPOConfig from ray.rllib.algorithms.algorithm import Algorithm +from ray.rllib.algorithms.algorithm_config import AlgorithmConfig from ray.rllib.utils.annotations import override from ray.rllib.utils.test_utils import check_learning_achieved -from ray import air, tune +from ray.rllib.utils.typing import PartialAlgorithmConfigDict from ray.tune import PlacementGroupFactory from ray.tune.logger import pretty_print @@ -82,8 +85,14 @@ def get_cli_args(): class PPORemoteInference(PPO): @classmethod @override(Algorithm) - def default_resource_request(cls, config): - cf = dict(cls.get_default_config(), **config) + def default_resource_request( + cls, + config: Union[AlgorithmConfig, PartialAlgorithmConfigDict], + ): + if isinstance(config, AlgorithmConfig): + cf = config + else: + cf = cls.get_default_config().update_from_dict(config) # Return PlacementGroupFactory containing all needed resources # (already properly defined as device bundles). @@ -94,15 +103,15 @@ def default_resource_request(cls, config): # main model in this example (num_workers=0). "CPU": 1, # Possibly add n GPUs to this. - "GPU": cf["num_gpus"], + "GPU": cf.num_gpus, }, { # Different bundle (meaning: possibly different node) # for your n "remote" envs (set remote_worker_envs=True). - "CPU": cf["num_envs_per_worker"], + "CPU": cf.num_envs_per_worker, }, ], - strategy=config.get("placement_strategy", "PACK"), + strategy=cf.placement_strategy, ) From e72a389c82a2f535072f3510c30312bf1b87e599 Mon Sep 17 00:00:00 2001 From: Amog Kamsetty Date: Thu, 26 Jan 2023 10:03:22 -0800 Subject: [PATCH 006/267] [Data] Add Logical Operator for `filter` and `flat_map` (#31954) Signed-off-by: amogkam amogkamsetty@yahoo.com Adds logical operators for filter() and flat_map() Closes #31936 --- .../logical/operators/map_operator.py | 62 ++++++++++++++++ python/ray/data/dataset.py | 29 +++++++- .../data/tests/test_execution_optimizer.py | 70 ++++++++++++++++--- 3 files changed, 151 insertions(+), 10 deletions(-) diff --git a/python/ray/data/_internal/logical/operators/map_operator.py b/python/ray/data/_internal/logical/operators/map_operator.py index 5ecddcd9d32a..261e198cc3b6 100644 --- a/python/ray/data/_internal/logical/operators/map_operator.py +++ b/python/ray/data/_internal/logical/operators/map_operator.py @@ -26,8 +26,28 @@ class AbstractMap(LogicalOperator): """Abstract class for logical operators should be converted to physical MapOperator. + + Args: + name: Name for this operator. This is the name that will appear when inspecting + the logical plan of a Dataset. + input_op: The operator preceding this operator in the plan DAG. The outputs of + `input_op` will be the inputs to this operator. + block_fn: The transform function to apply to each input block to produce output + blocks. + target_block_size: The target size for blocks outputted by this operator. + fn: User provided UDF to be called in `block_fn`. + fn_args: Arguments to `fn`. + fn_kwargs: Keyword arguments to `fn`. + fn_constructor_args: Arguments to provide to the initializor of `fn` if `fn` is + a callable class. + fn_constructor_kwargs: Keyword Arguments to provide to the initializor of `fn` + if `fn` is a callable class. + ray_remote_args: Args to provide to ray.remote. """ + # TODO: Replace `fn`, `fn_args`, `fn_kwargs`, `fn_constructor_args`, and + # `fn_constructor_kwargs` from this API, in favor of `block_fn_args` and + # `block_fn_kwargs`. Operators should only be concerned with `block_fn`. def __init__( self, name: str, @@ -112,6 +132,48 @@ def __init__( ) +class Filter(AbstractMap): + """Logical operator for filter.""" + + def __init__( + self, + input_op: LogicalOperator, + block_fn: BlockTransform, + fn: RowUDF, + compute: Optional[Union[str, ComputeStrategy]] = None, + ray_remote_args: Optional[Dict[str, Any]] = None, + ): + super().__init__( + "Filter", + input_op, + block_fn, + compute=compute, + fn=fn, + ray_remote_args=ray_remote_args, + ) + + +class FlatMap(AbstractMap): + """Logical operator for flat_map.""" + + def __init__( + self, + input_op: LogicalOperator, + block_fn: BlockTransform, + fn: RowUDF, + compute: Optional[Union[str, ComputeStrategy]] = None, + ray_remote_args: Optional[Dict[str, Any]] = None, + ): + super().__init__( + "FlatMap", + input_op, + block_fn, + compute=compute, + fn=fn, + ray_remote_args=ray_remote_args, + ) + + def plan_map_op(op: AbstractMap, input_physical_dag: PhysicalOperator) -> MapOperator: """Get the corresponding physical operators DAG for AbstractMap operators.""" compute = get_compute(op._compute) diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index d2f146bf10aa..7ab51669992b 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -32,6 +32,8 @@ from ray.air.util.data_batch_conversion import BlockFormat from ray.data._internal.logical.optimizers import LogicalPlan from ray.data._internal.logical.operators.map_operator import ( + Filter, + FlatMap, MapRows, MapBatches, ) @@ -902,7 +904,18 @@ def transform(block: Block, fn: RowUDF[T, U]) -> Iterable[Block]: plan = self._plan.with_stage( OneToOneStage("flat_map", transform, compute, ray_remote_args, fn=fn) ) - return Dataset(plan, self._epoch, self._lazy) + + logical_plan = self._logical_plan + if logical_plan is not None: + op = FlatMap( + input_op=logical_plan.dag, + block_fn=transform, + fn=fn, + compute=compute, + ray_remote_args=ray_remote_args, + ) + logical_plan = LogicalPlan(op) + return Dataset(plan, self._epoch, self._lazy, logical_plan) def filter( self, @@ -967,7 +980,19 @@ def transform(block: Block, fn: RowUDF[T, U]) -> Iterable[Block]: plan = self._plan.with_stage( OneToOneStage("filter", transform, compute, ray_remote_args, fn=fn) ) - return Dataset(plan, self._epoch, self._lazy) + + logical_plan = self._logical_plan + if logical_plan is not None: + op = Filter( + input_op=logical_plan.dag, + block_fn=transform, + fn=fn, + compute=compute, + ray_remote_args=ray_remote_args, + ) + logical_plan = LogicalPlan(op) + + return Dataset(plan, self._epoch, self._lazy, logical_plan) def repartition(self, num_blocks: int, *, shuffle: bool = False) -> "Dataset[T]": """Repartition the dataset into exactly this number of blocks. diff --git a/python/ray/data/tests/test_execution_optimizer.py b/python/ray/data/tests/test_execution_optimizer.py index 34596fc82ca4..8901826ee616 100644 --- a/python/ray/data/tests/test_execution_optimizer.py +++ b/python/ray/data/tests/test_execution_optimizer.py @@ -4,20 +4,18 @@ from ray.data._internal.execution.operators.map_operator import MapOperator from ray.data._internal.execution.operators.input_data_buffer import InputDataBuffer from ray.data._internal.logical.operators.read_operator import Read -from ray.data._internal.logical.operators.map_operator import MapRows, MapBatches +from ray.data._internal.logical.operators.map_operator import ( + MapRows, + MapBatches, + Filter, + FlatMap, +) from ray.data._internal.logical.planner import Planner from ray.data.datasource.parquet_datasource import ParquetDatasource from ray.tests.conftest import * # noqa -def test_e2e_optimizer_sanity(ray_start_cluster_enabled, enable_optimizer): - ds = ray.data.range(5) - ds = ds.map_batches(lambda x: x) - ds = ds.map(lambda x: x + 1) - assert ds.take_all() == [1, 2, 3, 4, 5], ds - - def test_read_operator(ray_start_cluster_enabled, enable_optimizer): planner = Planner() op = Read(ParquetDatasource()) @@ -45,6 +43,12 @@ def test_map_batches_operator(ray_start_cluster_enabled, enable_optimizer): assert isinstance(physical_op.input_dependencies[0], MapOperator) +def test_map_batches_e2e(ray_start_cluster_enabled, enable_optimizer): + ds = ray.data.range(5) + ds = ds.map_batches(lambda x: x) + assert ds.take_all() == list(range(5)), ds + + def test_map_rows_operator(ray_start_cluster_enabled, enable_optimizer): planner = Planner() read_op = Read(ParquetDatasource()) @@ -61,6 +65,56 @@ def test_map_rows_operator(ray_start_cluster_enabled, enable_optimizer): assert isinstance(physical_op.input_dependencies[0], MapOperator) +def test_map_rows_e2e(ray_start_cluster_enabled, enable_optimizer): + ds = ray.data.range(5) + ds = ds.map(lambda x: x + 1) + assert ds.take_all() == [1, 2, 3, 4, 5], ds + + +def test_filter_operator(ray_start_cluster_enabled, enable_optimizer): + planner = Planner() + read_op = Read(ParquetDatasource()) + op = Filter( + read_op, + lambda it: (x for x in it), + lambda x: x, + ) + physical_op = planner.plan(op) + + assert op.name == "Filter" + assert isinstance(physical_op, MapOperator) + assert len(physical_op.input_dependencies) == 1 + assert isinstance(physical_op.input_dependencies[0], MapOperator) + + +def test_filter_e2e(ray_start_cluster_enabled, enable_optimizer): + ds = ray.data.range(5) + ds = ds.filter(fn=lambda x: x % 2 == 0) + assert ds.take_all() == [0, 2, 4], ds + + +def test_flat_map(ray_start_cluster_enabled, enable_optimizer): + planner = Planner() + read_op = Read(ParquetDatasource()) + op = FlatMap( + read_op, + lambda it: ([x, x] for x in it), + lambda x: x, + ) + physical_op = planner.plan(op) + + assert op.name == "FlatMap" + assert isinstance(physical_op, MapOperator) + assert len(physical_op.input_dependencies) == 1 + assert isinstance(physical_op.input_dependencies[0], MapOperator) + + +def test_flat_map_e2e(ray_start_cluster_enabled, enable_optimizer): + ds = ray.data.range(2) + ds = ds.flat_map(fn=lambda x: [x, x]) + assert ds.take_all() == [0, 0, 1, 1], ds + + if __name__ == "__main__": import sys From 994c3d63440665c664766f64e0eb48a60a4cf4e4 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 26 Jan 2023 11:18:57 -0800 Subject: [PATCH 007/267] [docker/arm64] Build docker images (with suffix) for arm64 (#31861) This PR adds Docker image builds for arm64 (aarch64). The images will be tagged with a suffix `-aarch64`. In a future retagging step, we can combine the existing x86_64 images with the aarch64 images for true multiplatform images. Changes needed to make this work: - Move libgcc to libgcc-ng because the former does not exist on conda for aarch64 - Upgrade Python 3.7.7 to 3.7.16 because the former does not exist on conda for aarch74 Please also note that aarch64 base images only exist for CUDA >= 11.2. Lastly, we are not building ray-ml docker images. This is because not all current dependencies are compatible with aarch64 - specifically, tensorflow requires at least 1.10.1 and torch 1.13.0 does not seem to be released for linux aarch64. Signed-off-by: Kai Fricke --- .buildkite/pipeline.arm64.yml | 80 +++++++++++ build-docker.sh | 2 +- ci/build/build-docker-images.py | 181 +++++++++++++++++++------ docker/base-deps/Dockerfile | 8 +- docker/base-deps/README.md | 14 +- docker/ray-deps/README.md | 13 +- docker/ray-ml/README.md | 2 + docker/ray-worker-container/Dockerfile | 6 +- docker/ray/README.md | 28 +++- 9 files changed, 283 insertions(+), 51 deletions(-) diff --git a/.buildkite/pipeline.arm64.yml b/.buildkite/pipeline.arm64.yml index 99388bf18671..e49fff07fb41 100644 --- a/.buildkite/pipeline.arm64.yml +++ b/.buildkite/pipeline.arm64.yml @@ -54,3 +54,83 @@ # - python .buildkite/copy_files.py --destination branch_wheels --path ./.whl # # Upload to latest directory. # - if [ "$BUILDKITE_BRANCH" == "master" ]; then python .buildkite/copy_files.py --destination wheels --path ./.whl; fi + + +- label: ":mechanical_arm: :docker: Build Images: py37 [aarch64] (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + instance_size: arm64-medium + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py37 --device-types cpu cu112 --build-type BUILDKITE --build-base --suffix aarch64 + +- label: ":mechanical_arm: :docker: Build Images: py37 [aarch64] (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + instance_size: arm64-medium + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py37 --device-types cu113 cu116 --build-type BUILDKITE --build-base --suffix aarch64 + +- label: ":mechanical_arm: :docker: Build Images: py38 [aarch64] (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + instance_size: arm64-medium + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py38 --device-types cpu cu112 --build-type BUILDKITE --build-base --suffix aarch64 + +- label: ":mechanical_arm: :docker: Build Images: py38 [aarch64] (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + instance_size: arm64-medium + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py38 --device-types cu113 cu116 --build-type BUILDKITE --build-base --suffix aarch64 + +- label: ":mechanical_arm: :docker: Build Images: py39 [aarch64] (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + instance_size: arm64-medium + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py39 --device-types cpu cu112 --build-type BUILDKITE --build-base --suffix aarch64 + +- label: ":mechanical_arm: :docker: Build Images: py39 [aarch64] (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + instance_size: arm64-medium + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py39 --device-types cu113 cu116 --build-type BUILDKITE --build-base --suffix aarch64 + +- label: ":mechanical_arm: :docker: Build Images: py310 [aarch64] (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + instance_size: arm64-medium + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py310 --device-types cpu cu112 --build-type BUILDKITE --build-base --suffix aarch64 + +- label: ":mechanical_arm: :docker: Build Images: py310 [aarch64] (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + instance_size: arm64-medium + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py310 --device-types cu113 cu116 --build-type BUILDKITE --build-base --suffix aarch64 diff --git a/build-docker.sh b/build-docker.sh index 0b640c040637..8d8f5fbe250e 100755 --- a/build-docker.sh +++ b/build-docker.sh @@ -8,7 +8,7 @@ set -x GPU="" BASE_IMAGE="ubuntu:focal" WHEEL_URL="https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-3.0.0.dev0-cp37-cp37m-manylinux2014_x86_64.whl" -PYTHON_VERSION="3.7.7" +PYTHON_VERSION="3.7.16" while [[ $# -gt 0 ]] diff --git a/ci/build/build-docker-images.py b/ci/build/build-docker-images.py index fd8745af00e1..e3a3fec464c6 100644 --- a/ci/build/build-docker-images.py +++ b/ci/build/build-docker-images.py @@ -5,12 +5,13 @@ import glob import itertools import os +import platform import re import shutil import subprocess import sys from collections import defaultdict -from typing import List, Tuple +from typing import List, Optional, Tuple import docker @@ -18,6 +19,7 @@ DOCKER_USERNAME = "raytravisbot" DOCKER_CLIENT = None PYTHON_WHL_VERSION = "cp3" +ADDITIONAL_PLATFORMS = ["aarch64"] DOCKER_HUB_DESCRIPTION = { "base-deps": ( @@ -70,6 +72,12 @@ IMAGE_NAMES = list(DOCKER_HUB_DESCRIPTION.keys()) +def _with_suffix(tag: str, suffix: Optional[str] = None): + if suffix: + return tag + "-" + suffix + return tag + + def _get_branch(): branch = os.environ.get("TRAVIS_BRANCH") or os.environ.get("BUILDKITE_BRANCH") if not branch: @@ -159,7 +167,11 @@ def _check_if_docker_files_modified(): def _build_docker_image( - image_name: str, py_version: str, image_type: str, no_cache=True + image_name: str, + py_version: str, + image_type: str, + suffix: Optional[str] = None, + no_cache=True, ): """Builds Docker image with the provided info. @@ -169,6 +181,7 @@ def _build_docker_image( Must be one of PY_MATRIX.keys() image_type: The image type to build. Must be one of BASE_IMAGES.keys() + suffix: Suffix to add to the tags (e.g. "aarch64" for "ray:sha256-aarch64") no_cache: If True, don't use caching when building the image. """ @@ -197,6 +210,9 @@ def _build_docker_image( # I.e. "py310"[3:] == 10 build_args["PYTHON_MINOR_VERSION"] = py_version[3:] + if platform.processor() in ADDITIONAL_PLATFORMS: + build_args["HOSTTYPE"] = platform.processor() + device_tag = f"{image_type}" if image_name == "base-deps": @@ -204,6 +220,8 @@ def _build_docker_image( else: base_image = f"-{py_version}-{device_tag}" + base_image = _with_suffix(base_image, suffix=suffix) + if image_name != "ray-worker-container": build_args["BASE_IMAGE"] = base_image @@ -216,6 +234,8 @@ def _build_docker_image( tagged_name = f"rayproject/{image_name}:nightly-{py_version}-{device_tag}" + tagged_name = _with_suffix(tagged_name, suffix=suffix) + for i in range(2): cleanup = DOCKER_CLIENT.containers.prune().get("SpaceReclaimed") if cleanup is not None: @@ -305,22 +325,33 @@ def check_staleness(repository, tag): return is_stale -def build_for_all_versions(image_name, py_versions, image_types, **kwargs): +def build_for_all_versions(image_name, py_versions, image_types, suffix, **kwargs): """Builds the given Docker image for all Python & CUDA versions""" for py_version in py_versions: for image_type in image_types: _build_docker_image( - image_name, py_version=py_version, image_type=image_type, **kwargs + image_name, + py_version=py_version, + image_type=image_type, + suffix=suffix, + **kwargs, ) -def build_base_images(py_versions, image_types): - build_for_all_versions("base-deps", py_versions, image_types, no_cache=False) - build_for_all_versions("ray-deps", py_versions, image_types, no_cache=False) +def build_base_images(py_versions, image_types, suffix): + build_for_all_versions( + "base-deps", py_versions, image_types, suffix=suffix, no_cache=False + ) + build_for_all_versions( + "ray-deps", py_versions, image_types, suffix=suffix, no_cache=False + ) def build_or_pull_base_images( - py_versions: List[str], image_types: List[str], rebuild_base_images: bool = True + py_versions: List[str], + image_types: List[str], + rebuild_base_images: bool = True, + suffix: Optional[str] = None, ) -> bool: """Returns images to tag and build.""" repositories = ["rayproject/base-deps", "rayproject/ray-deps"] @@ -342,7 +373,7 @@ def build_or_pull_base_images( is_stale = True if rebuild_base_images or _release_build() or is_stale: - build_base_images(py_versions, image_types) + build_base_images(py_versions, image_types, suffix=suffix) return True else: print("Just pulling images!") @@ -441,8 +472,9 @@ def _create_new_tags(all_tags, old_str, new_str): def push_and_tag_images( py_versions: List[str], image_types: List[str], - push_base_images: bool, merge_build: bool = False, + image_list: Optional[List[str]] = None, + suffix: Optional[str] = None, ): date_tag = datetime.datetime.now().strftime("%Y-%m-%d") @@ -452,10 +484,6 @@ def push_and_tag_images( date_tag = release_name sha_tag = release_name - image_list = ["ray", "ray-ml"] - if push_base_images: - image_list.extend(["base-deps", "ray-deps"]) - for image_name in image_list: full_image_name = f"rayproject/{image_name}" @@ -476,7 +504,8 @@ def push_and_tag_images( ) continue - tag = f"nightly-{py_name}-{image_type}" + tag = _with_suffix(f"nightly-{py_name}-{image_type}", suffix=suffix) + tag_mapping[tag].append(tag) # If no device is specified, it should map to CPU image. @@ -519,41 +548,83 @@ def push_and_tag_images( ) tag_mapping[old_tag].extend(new_tags) + print(f"These tags will be created for {image_name}: ", tag_mapping) + # Sanity checking. for old_tag in tag_mapping.keys(): if DEFAULT_PYTHON_VERSION in old_tag: if "-cpu" in old_tag: - assert "nightly-cpu" in tag_mapping[old_tag] + assert ( + _with_suffix("nightly-cpu", suffix=suffix) + in tag_mapping[old_tag] + ) if "-deps" in image_name: - assert "nightly" in tag_mapping[old_tag] - assert f"{date_tag}-cpu" in tag_mapping[old_tag] - assert f"{date_tag}" in tag_mapping[old_tag] + assert ( + _with_suffix("nightly", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{date_tag}-cpu", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{date_tag}", suffix=suffix) + in tag_mapping[old_tag] + ) elif image_name == "ray": - assert "nightly" in tag_mapping[old_tag] - assert f"{sha_tag}-cpu" in tag_mapping[old_tag] - assert f"{sha_tag}" in tag_mapping[old_tag] + assert ( + _with_suffix("nightly", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{sha_tag}-cpu", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{sha_tag}", suffix=suffix) + in tag_mapping[old_tag] + ) # For ray-ml, nightly should refer to the GPU image. elif image_name == "ray-ml": - assert f"{sha_tag}-cpu" in tag_mapping[old_tag] + assert ( + _with_suffix(f"{sha_tag}-cpu", suffix=suffix) + in tag_mapping[old_tag] + ) else: raise RuntimeError(f"Invalid image name: {image_name}") elif ML_CUDA_VERSION in old_tag: - assert "nightly-gpu" in tag_mapping[old_tag] + assert ( + _with_suffix("nightly-gpu", suffix=suffix) + in tag_mapping[old_tag] + ) if "-deps" in image_name: - assert f"{date_tag}-gpu" in tag_mapping[old_tag] + assert ( + _with_suffix(f"{date_tag}-gpu", suffix=suffix) + in tag_mapping[old_tag] + ) elif image_name == "ray": - assert f"{sha_tag}-gpu" in tag_mapping[old_tag] + assert ( + _with_suffix(f"{sha_tag}-gpu", suffix=suffix) + in tag_mapping[old_tag] + ) # For ray-ml, nightly should refer to the GPU image. elif image_name == "ray-ml": - assert "nightly" in tag_mapping[old_tag] - assert f"{sha_tag}" in tag_mapping[old_tag] - assert f"{sha_tag}-gpu" in tag_mapping[old_tag] + assert ( + _with_suffix("nightly", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{sha_tag}", suffix=suffix) + in tag_mapping[old_tag] + ) + assert ( + _with_suffix(f"{sha_tag}-gpu", suffix=suffix) + in tag_mapping[old_tag] + ) else: raise RuntimeError(f"Invalid image name: {image_name}") - print(f"These tags will be created for {image_name}: ", tag_mapping) - # Tag and push all images. for old_tag in tag_mapping.keys(): for new_tag in tag_mapping[old_tag]: @@ -636,6 +707,12 @@ def push_readmes(merge_build: bool): required=True, help="Whether to bypass checking if docker is affected", ) + parser.add_argument( + "--suffix", + required=False, + choices=ADDITIONAL_PLATFORMS, + help="Suffix to append to the build tags", + ) parser.add_argument( "--build-base", dest="base", @@ -718,28 +795,47 @@ def push_readmes(merge_build: bool): DOCKER_CLIENT.api.login(username=username, password=password) copy_wheels(build_type == HUMAN) is_base_images_built = build_or_pull_base_images( - py_versions, image_types, args.base + py_versions, image_types, args.base, suffix=args.suffix ) if args.only_build_worker_container: - build_for_all_versions("ray-worker-container", py_versions, image_types) + build_for_all_versions( + "ray-worker-container", py_versions, image_types, suffix=args.suffix + ) # TODO Currently don't push ray_worker_container else: # Build Ray Docker images. - build_for_all_versions("ray", py_versions, image_types) + build_for_all_versions("ray", py_versions, image_types, suffix=args.suffix) + + # List of images to tag and push to docker hub + images_to_tag_and_push = [] + + if is_base_images_built: + images_to_tag_and_push += ["base-deps", "ray-deps"] + + # Always tag/push ray + images_to_tag_and_push += ["ray"] # Only build ML Docker images for ML_CUDA_VERSION or cpu. - ml_image_types = [ - image_type - for image_type in image_types - if image_type in [ML_CUDA_VERSION, "cpu"] - ] + if platform.processor() not in ADDITIONAL_PLATFORMS: + ml_image_types = [ + image_type + for image_type in image_types + if image_type in [ML_CUDA_VERSION, "cpu"] + ] + else: + # Do not build ray-ml e.g. for arm64 + ml_image_types = [] if len(ml_image_types) > 0: prep_ray_ml() build_for_all_versions( - "ray-ml", py_versions, image_types=ml_image_types + "ray-ml", + py_versions, + image_types=ml_image_types, + suffix=args.suffix, ) + images_to_tag_and_push += ["ray-ml"] if build_type in {MERGE, PR}: valid_branch = _valid_branch() @@ -748,8 +844,9 @@ def push_readmes(merge_build: bool): push_and_tag_images( py_versions, image_types, - is_base_images_built, - valid_branch and is_merge, + merge_build=valid_branch and is_merge, + image_list=images_to_tag_and_push, + suffix=args.suffix, ) # TODO(ilr) Re-Enable Push READMEs by using a normal password diff --git a/docker/base-deps/Dockerfile b/docker/base-deps/Dockerfile index 6334c1359305..0b60772e8415 100644 --- a/docker/base-deps/Dockerfile +++ b/docker/base-deps/Dockerfile @@ -11,7 +11,9 @@ ENV TZ=America/Los_Angeles # TODO(ilr) $HOME seems to point to result in "" instead of "/home/ray" ENV PATH "/home/ray/anaconda3/bin:$PATH" ARG DEBIAN_FRONTEND=noninteractive -ARG PYTHON_VERSION=3.7.7 +ARG PYTHON_VERSION=3.7.16 + +ARG HOSTTYPE=${HOSTTYPE:-x86_64} ARG RAY_UID=1000 ARG RAY_GID=100 @@ -42,14 +44,14 @@ RUN sudo apt-get update -y && sudo apt-get upgrade -y \ openssh-client \ gnupg; fi) \ && wget \ - --quiet "https://repo.anaconda.com/miniconda/Miniconda3-latest-Linux-x86_64.sh" \ + --quiet "https://repo.anaconda.com/miniconda/Miniconda3-latest-Linux-${HOSTTYPE}.sh" \ -O /tmp/miniconda.sh \ && /bin/bash /tmp/miniconda.sh -b -u -p $HOME/anaconda3 \ && $HOME/anaconda3/bin/conda init \ && echo 'export PATH=$HOME/anaconda3/bin:$PATH' >> /home/ray/.bashrc \ && rm /tmp/miniconda.sh \ && $HOME/anaconda3/bin/conda install -y \ - libgcc python=$PYTHON_VERSION \ + libgcc-ng python=$PYTHON_VERSION \ && $HOME/anaconda3/bin/conda clean -y --all \ && $HOME/anaconda3/bin/pip install --no-cache-dir \ flatbuffers \ diff --git a/docker/base-deps/README.md b/docker/base-deps/README.md index f7d92140a597..048020779b71 100644 --- a/docker/base-deps/README.md +++ b/docker/base-deps/README.md @@ -6,7 +6,7 @@ This image has the system-level dependencies for `Ray` and the `Ray Autoscaler` ## Tags -Images are `tagged` with the format `{Ray version}[-{Python version}][-{Platform}]`. `Ray version` tag can be one of the following: +Images are `tagged` with the format `{Ray version}[-{Python version}][-{Platform}][-{Architecture}]`. `Ray version` tag can be one of the following: | Ray version tag | Description | | --------------- | ----------- | @@ -26,6 +26,18 @@ The optional `Platform` tag specifies the platform where the image is intended f | `-gpu` | Aliases to a specific `-cuXX` tagged image. | | no tag | Aliases to `-cpu` tagged images for `ray`, and aliases to ``-gpu`` tagged images for `ray-ml`. | +The optional `Architecture` tag can be used to specify images for different CPU architectures. +Currently, we support the `x86_64` (`amd64`) and `aarch64` (`arm64`) architectures. + +Please note that suffixes are only used to specify `aarch64` images. No suffix means +`x86_64`/`amd64`-compatible images. + +| Platform tag | Description | +|--------------|-------------------------| +| `-aarch64` | arm64-compatible images | +| no tag | Defaults to `amd64` | + + ## Other Images * [`rayproject/ray`](https://hub.docker.com/repository/docker/rayproject/ray) - Ray and all of its dependencies. * [`rayproject/ray-ml`](https://hub.docker.com/repository/docker/rayproject/ray-ml) - This image with common ML libraries to make development & deployment more smooth! diff --git a/docker/ray-deps/README.md b/docker/ray-deps/README.md index 9be0ee4795e5..d510fdb16861 100644 --- a/docker/ray-deps/README.md +++ b/docker/ray-deps/README.md @@ -5,7 +5,7 @@ This has the python-level dependencies for `Ray` and the `Ray Autoscaler`. The ` ## Tags -Images are `tagged` with the format `{Ray version}[-{Python version}][-{Platform}]`. `Ray version` tag can be one of the following: +Images are `tagged` with the format `{Ray version}[-{Python version}][-{Platform}][-{Architecture}]`. `Ray version` tag can be one of the following: | Ray version tag | Description | | --------------- | ----------- | @@ -25,6 +25,17 @@ The optional `Platform` tag specifies the platform where the image is intended f | `-gpu` | Aliases to a specific `-cuXX` tagged image. | | no tag | Aliases to `-cpu` tagged images for `ray`, and aliases to ``-gpu`` tagged images for `ray-ml`. | +The optional `Architecture` tag can be used to specify images for different CPU architectures. +Currently, we support the `x86_64` (`amd64`) and `aarch64` (`arm64`) architectures. + +Please note that suffixes are only used to specify `aarch64` images. No suffix means +`x86_64`/`amd64`-compatible images. + +| Platform tag | Description | +|--------------|-------------------------| +| `-aarch64` | arm64-compatible images | +| no tag | Defaults to `amd64` | + ## Other Images * [`rayproject/ray`](https://hub.docker.com/repository/docker/rayproject/ray) - Ray and all of its dependencies. * [`rayproject/ray-ml`](https://hub.docker.com/repository/docker/rayproject/ray-ml) - This image with common ML libraries to make development & deployment more smooth! diff --git a/docker/ray-ml/README.md b/docker/ray-ml/README.md index 56d32e9a029b..22a6cb13ee5f 100644 --- a/docker/ray-ml/README.md +++ b/docker/ray-ml/README.md @@ -29,5 +29,7 @@ Examples tags: - `nightly-py38-cpu` - `806c18-py38-cu112` +The `ray-ml` images are not built for the `arm64` (`aarch64`) architecture. + ## Other Images * [`rayproject/ray`](https://hub.docker.com/repository/docker/rayproject/ray) - Ray and all of its dependencies. diff --git a/docker/ray-worker-container/Dockerfile b/docker/ray-worker-container/Dockerfile index e340b6af0b62..fc6f2c6a404f 100644 --- a/docker/ray-worker-container/Dockerfile +++ b/docker/ray-worker-container/Dockerfile @@ -6,9 +6,11 @@ ARG BASE_IMAGE ARG AUTOSCALER="autoscaler" ENV TZ=America/Los_Angeles +ARG HOSTTYPE=${HOSTTYPE:-x86_64} + ENV PATH "/root/anaconda3/bin:$PATH" ARG DEBIAN_FRONTEND=noninteractive -ARG PYTHON_VERSION=3.7.7 +ARG PYTHON_VERSION=3.7.16 RUN apt-get update -y \ && apt-get install -y sudo tzdata \ @@ -30,7 +32,7 @@ RUN apt-get update -y && sudo apt-get upgrade -y \ gnupg; fi) \ unzip \ && wget \ - --quiet "https://repo.anaconda.com/miniconda/Miniconda3-latest-Linux-x86_64.sh" \ + --quiet "https://repo.anaconda.com/miniconda/Miniconda3-latest-Linux-${HOSTTYPE}.sh" \ -O /tmp/miniconda.sh \ && /bin/bash /tmp/miniconda.sh -b -u -p $HOME/anaconda3 \ && $HOME/anaconda3/bin/conda init \ diff --git a/docker/ray/README.md b/docker/ray/README.md index 4a0958d2d25d..0a908187881a 100644 --- a/docker/ray/README.md +++ b/docker/ray/README.md @@ -4,7 +4,7 @@ everything needed to get started with running Ray! They work for both local deve ## Tags -Images are `tagged` with the format `{Ray version}[-{Python version}][-{Platform}]`. `Ray version` tag can be one of the following: +Images are `tagged` with the format `{Ray version}[-{Python version}][-{Platform}][-{Architecture}]`. `Ray version` tag can be one of the following: | Ray version tag | Description | | --------------- | ----------- | @@ -24,11 +24,37 @@ The optional `Platform` tag specifies the platform where the image is intended f | `-gpu` | Aliases to a specific `-cuXX` tagged image. | | no tag | Aliases to `-cpu` tagged images for `ray`, and aliases to ``-gpu`` tagged images for `ray-ml`. | +The optional `Architecture` tag can be used to specify images for different CPU architectures. +Currently, we support the `x86_64` (`amd64`) and `aarch64` (`arm64`) architectures. + +Please note that suffixes are only used to specify `aarch64` images. No suffix means +`x86_64`/`amd64`-compatible images. + +| Platform tag | Description | +|--------------|-------------------------| +| `-aarch64` | arm64-compatible images | +| no tag | Defaults to `amd64` | + Examples tags: - none: equivalent to `latest` - `latest`: equivalent to `latest-py37-cpu`, i.e. image for the most recent Ray release - `nightly-py38-cpu` - `806c18-py38-cu112` +- `806c18-py38-cu116-aarch64` + +## Roadmap + +Ray 2.3 will be the first release for which arm64 images are released. These images will have the `-aarch64` suffix. + +There won't be a `:latest-aarch64` image, instead `:2.3.0-aarch64` should be used explicitly. This is because +we may remove suffixes in the next release. + +For Ray 2.4, we aim to have support for multiplatform images. This means that specifying the suffix +will not be needed anymore - docker will automatically choose a compatible image. + +We may stop publishing architecture suffixes completely when we have support for multiplatform images. + +There is an open RFC issue on GitHub to discuss this roadmap: [Link to issue](https://github.com/ray-project/ray/issues/31966) ## Other Images * [`rayproject/ray-ml`](https://hub.docker.com/repository/docker/rayproject/ray-ml) - This image with common ML libraries to make development & deployment more smooth! From 6b081e9e000f181cb05036c91e3369e58c2baba3 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 26 Jan 2023 11:35:56 -0800 Subject: [PATCH 008/267] [data] [streaming] [11/n]--- Run streaming executor on a helper thread (#31951) We currently run the streaming control loop on the user thread, which means it stalls between generator yields. This moves it in a separate thread to avoid unnecessary stalls. We still provide backpressure via the queue between threads (size set to 1). This improves ingest throughput by 20% on a benchmark I tried (1.0->1.2GB/s). The queue size did not seem to matter, just having it >=1 was sufficient to provide the performance improvement. --- .../_internal/execution/streaming_executor.py | 66 ++++++++++++++----- 1 file changed, 48 insertions(+), 18 deletions(-) diff --git a/python/ray/data/_internal/execution/streaming_executor.py b/python/ray/data/_internal/execution/streaming_executor.py index 1d3543100b0c..b2f169869b59 100644 --- a/python/ray/data/_internal/execution/streaming_executor.py +++ b/python/ray/data/_internal/execution/streaming_executor.py @@ -1,4 +1,6 @@ import logging +import queue +import threading import os from typing import Iterator, Optional @@ -27,7 +29,7 @@ DEBUG_TRACE_SCHEDULING = "RAY_DATASET_TRACE_SCHEDULING" in os.environ -class StreamingExecutor(Executor): +class StreamingExecutor(Executor, threading.Thread): """A streaming Dataset executor. This implementation executes Dataset DAGs in a fully streamed way. It runs @@ -41,9 +43,16 @@ def __init__(self, options: ExecutionOptions): # data updates the stats object in legacy code). self._stats: Optional[DatasetStats] = None self._global_info: Optional[ProgressBar] = None - if options.locality_with_output: - raise NotImplementedError("locality with output") - super().__init__(options) + + # Internal execution state shared across thread boundaries. We run the control + # loop on a separate thread so that it doesn't become stalled between + # generator `yield`s. + self._runner_thread_out = queue.Queue(maxsize=1) + self._topology: Optional[Topology] = None + self._output_node: Optional[OpState] = None + + Executor.__init__(self, options) + threading.Thread.__init__(self) def execute( self, dag: PhysicalOperator, initial_stats: Optional[DatasetStats] = None @@ -57,27 +66,48 @@ def execute( logger.info("Executing DAG %s", dag) self._global_info = ProgressBar("Resource usage vs limits", 1, 0) - # Setup the streaming DAG topology. - topology, self._stats = build_streaming_topology(dag, self._options) - output_node: OpState = topology[dag] + # Setup the streaming DAG topology and start the runner thread. + self._topology, self._stats = build_streaming_topology(dag, self._options) + _validate_topology(self._topology, self._get_or_refresh_resource_limits()) + self._output_node: OpState = self._topology[dag] + self.start() + + # Drain items from the runner thread until completion. try: - _validate_topology(topology, self._get_or_refresh_resource_limits()) - output_node: OpState = topology[dag] + item = self._runner_thread_out.get() + while item is not None: + if isinstance(item, Exception): + raise item + else: + yield item + item = self._runner_thread_out.get() + finally: + for op in self._topology: + op.shutdown() + if self._global_info: + self._global_info.close() + def run(self): + """Run the control loop in a helper thread. + + Results are returned via the `self._runner_thread_out` queue. + """ + try: # Run scheduling loop until complete. - while self._scheduling_loop_step(topology): - while output_node.outqueue: - yield output_node.outqueue.pop(0) + while self._scheduling_loop_step(self._topology): + while self._output_node.outqueue: + self._runner_thread_out.put(self._output_node.outqueue.pop(0)) # Handle any leftover outputs. - while output_node.outqueue: - yield output_node.outqueue.pop(0) + while self._output_node.outqueue: + self._runner_thread_out.put(self._output_node.outqueue.pop(0)) + except Exception as e: + # Propagate it to the result iterator. + self._runner_thread_out.put(e) finally: - for op in topology: - op.shutdown() - if self._global_info: - self._global_info.close() + # Signal end of results. + self._runner_thread_out.put(None) def get_stats(self): """Return the stats object for the streaming execution. From 39a901eb40aa1d55639ddd90bcd4094182e999a9 Mon Sep 17 00:00:00 2001 From: Yi Cheng <74173148+iycheng@users.noreply.github.com> Date: Thu, 26 Jan 2023 14:21:39 -0800 Subject: [PATCH 009/267] [nightly] Delete deprecated flags in nightly test. (#31973) ## Why are these changes needed? These flags are no longer useful because the migration has been finished. Delete them. ## Related issue number ## Checks - [ ] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --- release/ray_release/cluster_manager/cluster_manager.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/release/ray_release/cluster_manager/cluster_manager.py b/release/ray_release/cluster_manager/cluster_manager.py index f4e44579af90..5455360c2107 100644 --- a/release/ray_release/cluster_manager/cluster_manager.py +++ b/release/ray_release/cluster_manager/cluster_manager.py @@ -54,8 +54,6 @@ def set_cluster_env(self, cluster_env: Dict[str, Any]): # Add flags for redisless Ray self.cluster_env.setdefault("env_vars", {}) self.cluster_env["env_vars"]["MATCH_AUTOSCALER_AND_RAY_IMAGES"] = "1" - self.cluster_env["env_vars"]["RAY_gcs_storage"] = "memory" - self.cluster_env["env_vars"]["RAY_bootstrap_with_gcs"] = "1" self.cluster_env["env_vars"]["RAY_USAGE_STATS_ENABLED"] = "1" self.cluster_env["env_vars"]["RAY_USAGE_STATS_SOURCE"] = "nightly-tests" self.cluster_env["env_vars"][ From a32b9b1bf21bb3cd971ce48aa9517810693d8f1f Mon Sep 17 00:00:00 2001 From: Yi Cheng <74173148+iycheng@users.noreply.github.com> Date: Thu, 26 Jan 2023 15:09:40 -0800 Subject: [PATCH 010/267] [core] Fix gcs healthch manager crash when node is removed by node manager. (#31917) The root cause is because the data structure is deleted, but call backs is not canceled and got executed. This PR simplify the life model and make it the way gRPC works. We only delete the structure after gRPC OnDone is called. In the shortcut, according to the doc https://github.com/grpc/proposal/blob/master/L67-cpp-callback-api.md#unary-rpc-shortcuts , OnDone will call the callback function. A better model is needed here. The code will be changed once we update the threading model in gRPC. --- .bazelrc | 1 + BUILD.bazel | 2 +- .../gcs_server/gcs_health_check_manager.cc | 48 ++++++------ .../gcs/gcs_server/gcs_health_check_manager.h | 23 ++---- .../test/gcs_health_check_manager_test.cc | 75 +++++++++++++++---- 5 files changed, 90 insertions(+), 59 deletions(-) diff --git a/.bazelrc b/.bazelrc index 1c4b9f296b8f..1b98b8f72a3f 100644 --- a/.bazelrc +++ b/.bazelrc @@ -97,6 +97,7 @@ build:tsan --copt -g build:tsan --copt -fno-omit-frame-pointer build:tsan --copt -Wno-uninitialized build:tsan --linkopt -fsanitize=thread +build:tsan --cxxopt="-D_RAY_TSAN_BUILD" # This config is only for running TSAN with LLVM toolchain on Linux. build:tsan-clang --config=tsan build:tsan-clang --config=llvm diff --git a/BUILD.bazel b/BUILD.bazel index 82e9a74be901..9b6b22d43240 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -1856,7 +1856,7 @@ cc_library( cc_test( name = "gcs_health_check_manager_test", - size = "small", + size = "medium", srcs = [ "src/ray/gcs/gcs_server/test/gcs_health_check_manager_test.cc", ], diff --git a/src/ray/gcs/gcs_server/gcs_health_check_manager.cc b/src/ray/gcs/gcs_server/gcs_health_check_manager.cc index 831f345ef81a..7a3d99e74e05 100644 --- a/src/ray/gcs/gcs_server/gcs_health_check_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_health_check_manager.cc @@ -53,6 +53,7 @@ void GcsHealthCheckManager::RemoveNode(const NodeID &node_id) { if (iter == health_check_contexts_.end()) { return; } + iter->second->Stop(); health_check_contexts_.erase(iter); }, "GcsHealthCheckManager::RemoveNode"); @@ -60,8 +61,11 @@ void GcsHealthCheckManager::RemoveNode(const NodeID &node_id) { void GcsHealthCheckManager::FailNode(const NodeID &node_id) { RAY_LOG(WARNING) << "Node " << node_id << " is dead because the health check failed."; - on_node_death_callback_(node_id); - health_check_contexts_.erase(node_id); + auto iter = health_check_contexts_.find(node_id); + if (iter != health_check_contexts_.end()) { + on_node_death_callback_(node_id); + health_check_contexts_.erase(iter); + } } std::vector GcsHealthCheckManager::GetAllNodes() const { @@ -75,27 +79,23 @@ std::vector GcsHealthCheckManager::GetAllNodes() const { void GcsHealthCheckManager::HealthCheckContext::StartHealthCheck() { using ::grpc::health::v1::HealthCheckResponse; - context_ = std::make_shared(); + // Reset the context/request/response for the next request. + context_.~ClientContext(); + new (&context_) grpc::ClientContext(); + response_.Clear(); auto deadline = std::chrono::system_clock::now() + std::chrono::milliseconds(manager_->timeout_ms_); - context_->set_deadline(deadline); + context_.set_deadline(deadline); stub_->async()->Check( - context_.get(), - &request_, - &response_, - [this, stopped = this->stopped_, context = this->context_, now = absl::Now()]( - ::grpc::Status status) { + &context_, &request_, &response_, [this, now = absl::Now()](::grpc::Status status) { // This callback is done in gRPC's thread pool. STATS_health_check_rpc_latency_ms.Record( absl::ToInt64Milliseconds(absl::Now() - now)); - if (status.error_code() == ::grpc::StatusCode::CANCELLED) { - return; - } manager_->io_service_.post( - [this, stopped, status]() { - // Stopped has to be read in the same thread where it's updated. - if (*stopped) { + [this, status]() { + if (stopped_) { + delete this; return; } RAY_LOG(DEBUG) << "Health check status: " << int(response_.status()); @@ -110,32 +110,28 @@ void GcsHealthCheckManager::HealthCheckContext::StartHealthCheck() { } if (health_check_remaining_ == 0) { - manager_->io_service_.post([this]() { manager_->FailNode(node_id_); }, - ""); + manager_->FailNode(node_id_); + delete this; } else { // Do another health check. timer_.expires_from_now( boost::posix_time::milliseconds(manager_->period_ms_)); - timer_.async_wait([this, stopped](auto ec) { - // We need to check stopped here as well since cancel - // won't impact the queued tasks. - if (ec != boost::asio::error::operation_aborted && !*stopped) { - StartHealthCheck(); - } - }); + timer_.async_wait([this](auto) { StartHealthCheck(); }); } }, "HealthCheck"); }); } +void GcsHealthCheckManager::HealthCheckContext::Stop() { stopped_ = true; } + void GcsHealthCheckManager::AddNode(const NodeID &node_id, std::shared_ptr channel) { io_service_.dispatch( [this, channel, node_id]() { RAY_CHECK(health_check_contexts_.count(node_id) == 0); - auto context = std::make_unique(this, channel, node_id); - health_check_contexts_.emplace(std::make_pair(node_id, std::move(context))); + auto context = new HealthCheckContext(this, channel, node_id); + health_check_contexts_.emplace(std::make_pair(node_id, context)); }, "GcsHealthCheckManager::AddNode"); } diff --git a/src/ray/gcs/gcs_server/gcs_health_check_manager.h b/src/ray/gcs/gcs_server/gcs_health_check_manager.h index e0daea5edfcf..d877a217d803 100644 --- a/src/ray/gcs/gcs_server/gcs_health_check_manager.h +++ b/src/ray/gcs/gcs_server/gcs_health_check_manager.h @@ -91,27 +91,16 @@ class GcsHealthCheckManager { NodeID node_id) : manager_(manager), node_id_(node_id), - stopped_(std::make_shared(false)), timer_(manager->io_service_), health_check_remaining_(manager->failure_threshold_) { request_.set_service(node_id.Hex()); stub_ = grpc::health::v1::Health::NewStub(channel); timer_.expires_from_now( boost::posix_time::milliseconds(manager_->initial_delay_ms_)); - timer_.async_wait([this](auto ec) { - if (ec != boost::asio::error::operation_aborted) { - StartHealthCheck(); - } - }); + timer_.async_wait([this](auto) { StartHealthCheck(); }); } - ~HealthCheckContext() { - timer_.cancel(); - if (context_ != nullptr) { - context_->TryCancel(); - } - *stopped_ = true; - } + void Stop(); private: void StartHealthCheck(); @@ -121,14 +110,12 @@ class GcsHealthCheckManager { NodeID node_id_; // Whether the health check has stopped. - std::shared_ptr stopped_; + bool stopped_ = false; /// gRPC related fields std::unique_ptr<::grpc::health::v1::Health::Stub> stub_; - // The context is used in the gRPC callback which is in another - // thread, so we need it to be a shared_ptr. - std::shared_ptr context_; + grpc::ClientContext context_; ::grpc::health::v1::HealthCheckRequest request_; ::grpc::health::v1::HealthCheckResponse response_; @@ -146,7 +133,7 @@ class GcsHealthCheckManager { std::function on_node_death_callback_; /// The context of the health check for each nodes. - absl::flat_hash_map> health_check_contexts_; + absl::flat_hash_map health_check_contexts_; /// The delay for the first health check request. const int64_t initial_delay_ms_; diff --git a/src/ray/gcs/gcs_server/test/gcs_health_check_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_health_check_manager_test.cc index f4f4bf8cefb2..31be1bbd5d6d 100644 --- a/src/ray/gcs/gcs_server/test/gcs_health_check_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_health_check_manager_test.cc @@ -19,9 +19,13 @@ #include #include #include +#include #include using namespace boost; +using namespace boost::asio; +using namespace boost::asio::ip; + #include #include @@ -30,6 +34,20 @@ using namespace boost; #include "gtest/gtest.h" #include "ray/gcs/gcs_server/gcs_health_check_manager.h" +int GetFreePort() { + io_service io_service; + tcp::acceptor acceptor(io_service); + tcp::endpoint endpoint; + + // try to bind to port 0 to find a free port + acceptor.open(tcp::v4()); + acceptor.bind(tcp::endpoint(tcp::v4(), 0)); + endpoint = acceptor.local_endpoint(); + auto port = endpoint.port(); + acceptor.close(); + return port; +} + using namespace ray; using namespace std::literals::chrono_literals; @@ -46,7 +64,6 @@ class GcsHealthCheckManagerTest : public ::testing::Test { timeout_ms, period_ms, failure_threshold); - port = 10000; } void TearDown() override { @@ -65,7 +82,8 @@ class GcsHealthCheckManagerTest : public ::testing::Test { NodeID AddServer(bool alive = true) { std::promise port_promise; auto node_id = NodeID::FromRandom(); - + auto port = GetFreePort(); + RAY_LOG(INFO) << "Get port " << port; auto server = std::make_shared(node_id.Hex(), port, true); auto channel = grpc::CreateChannel("localhost:" + std::to_string(port), @@ -76,7 +94,6 @@ class GcsHealthCheckManagerTest : public ::testing::Test { } servers.emplace(node_id, server); health_check->AddNode(node_id, channel); - ++port; return node_id; } @@ -115,14 +132,13 @@ class GcsHealthCheckManagerTest : public ::testing::Test { } } - int port; instrumented_io_context io_service; std::unique_ptr health_check; std::unordered_map> servers; std::unordered_set dead_nodes; - const int64_t initial_delay_ms = 1000; - const int64_t timeout_ms = 1000; - const int64_t period_ms = 1000; + const int64_t initial_delay_ms = 100; + const int64_t timeout_ms = 10; + const int64_t period_ms = 10; const int64_t failure_threshold = 5; }; @@ -143,8 +159,6 @@ TEST_F(GcsHealthCheckManagerTest, TestBasic) { Run(2); // One for starting RPC and one for the RPC callback. } - Run(); // For failure callback. - ASSERT_EQ(1, dead_nodes.size()); ASSERT_TRUE(dead_nodes.count(node_id)); } @@ -169,8 +183,6 @@ TEST_F(GcsHealthCheckManagerTest, StoppedAndResume) { } } - Run(); // For failure callback. - ASSERT_EQ(0, dead_nodes.size()); } @@ -196,8 +208,6 @@ TEST_F(GcsHealthCheckManagerTest, Crashed) { Run(2); // One for starting RPC and one for the RPC callback. } - Run(); // For failure callback. - ASSERT_EQ(1, dead_nodes.size()); ASSERT_TRUE(dead_nodes.count(node_id)); } @@ -230,12 +240,49 @@ TEST_F(GcsHealthCheckManagerTest, NoRegister) { Run(2); // One for starting RPC and one for the RPC callback. } - Run(2); + Run(1); ASSERT_EQ(1, dead_nodes.size()); ASSERT_TRUE(dead_nodes.count(node_id)); } +TEST_F(GcsHealthCheckManagerTest, StressTest) { +#ifdef _RAY_TSAN_BUILD + GTEST_SKIP() << "Disabled in tsan because of performance"; +#endif + boost::asio::io_service::work work(io_service); + std::srand(std::time(nullptr)); + auto t = std::make_unique([this]() { this->io_service.run(); }); + + std::vector alive_nodes; + + for (int i = 0; i < 200; ++i) { + alive_nodes.emplace_back(AddServer(true)); + std::this_thread::sleep_for(10ms); + } + + for (size_t i = 0; i < 20000UL; ++i) { + RAY_LOG(INFO) << "Progress: " << i << "/20000"; + auto iter = alive_nodes.begin() + std::rand() % alive_nodes.size(); + health_check->RemoveNode(*iter); + DeleteServer(*iter); + alive_nodes.erase(iter); + alive_nodes.emplace_back(AddServer(true)); + } + RAY_LOG(INFO) << "Finished!"; + io_service.stop(); + t->join(); +} + int main(int argc, char **argv) { + InitShutdownRAII ray_log_shutdown_raii(ray::RayLog::StartRayLog, + ray::RayLog::ShutDownRayLog, + argv[0], + ray::RayLogLevel::INFO, + /*log_dir=*/""); + + ray::RayLog::InstallFailureSignalHandler(argv[0]); + ray::RayLog::InstallTerminateHandler(); + ::testing::InitGoogleTest(&argc, argv); return RUN_ALL_TESTS(); } From e753b03c23aa630d59c7f032cd6f17877788f950 Mon Sep 17 00:00:00 2001 From: Alex Wu Date: Thu, 26 Jan 2023 15:27:20 -0800 Subject: [PATCH 011/267] [1/n] Stabilize GCS/Autoscaler interface: Introduce monitor server (#31827) This is the first PR towards stabilizing the GCS autoscaler interface by introducing a new grpc service definition which we will provide backwards compatibility guarantees. This PR mostly just introduces scaffolding and a trivial GetRayVersion endpoint. By the end of this series of PRs, monitor.py will only communicate with the rest of the ray cluster via this service definition. --- BUILD.bazel | 36 +++++++++++++++ python/ray/tests/BUILD | 8 ++++ python/ray/tests/test_monitor_service.py | 18 ++++++++ src/ray/gcs/gcs_server/gcs_monitor_server.cc | 32 +++++++++++++ src/ray/gcs/gcs_server/gcs_monitor_server.h | 33 +++++++++++++ src/ray/gcs/gcs_server/gcs_server.cc | 10 ++++ src/ray/gcs/gcs_server/gcs_server.h | 8 ++++ .../test/gcs_monitor_server_test.cc | 46 +++++++++++++++++++ src/ray/protobuf/BUILD | 15 ++++++ src/ray/protobuf/monitor.proto | 30 ++++++++++++ src/ray/rpc/gcs_server/gcs_rpc_server.h | 42 +++++++++++++++++ 11 files changed, 278 insertions(+) create mode 100644 python/ray/tests/test_monitor_service.py create mode 100644 src/ray/gcs/gcs_server/gcs_monitor_server.cc create mode 100644 src/ray/gcs/gcs_server/gcs_monitor_server.h create mode 100644 src/ray/gcs/gcs_server/test/gcs_monitor_server_test.cc create mode 100644 src/ray/protobuf/monitor.proto diff --git a/BUILD.bazel b/BUILD.bazel index 9b6b22d43240..39c9ddd285da 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -272,6 +272,23 @@ cc_library( ], ) +# monitor/autoscaler service +cc_grpc_library( + name = "monitor_cc_grpc", + srcs = ["//src/ray/protobuf:monitor_proto"], + grpc_only = True, + deps = ["//src/ray/protobuf:monitor_cc_proto"], +) + +cc_library( + name = "monitor_rpc", + copts = COPTS, + visibility = ["//visibility:public"], + deps = [ + ":monitor_cc_grpc", + ], +) + # === End of rpc definitions === # === Begin of plasma definitions === @@ -541,6 +558,7 @@ cc_library( ":gcs_service_cc_grpc", ":gcs_service_rpc", ":gcs_table_storage_lib", + ":monitor_rpc", ":node_manager_rpc", ":observable_store_client", ":pubsub_lib", @@ -2020,6 +2038,23 @@ cc_test( ], ) +cc_test( + name = "gcs_monitor_server_test", + size = "small", + srcs = [ + "src/ray/gcs/gcs_server/test/gcs_monitor_server_test.cc", + ], + copts = COPTS, + tags = ["team:serverless"], + deps = [ + ":gcs_server_lib", + ":gcs_server_test_util", + ":gcs_test_util_lib", + ":ray_mock", + "@com_google_googletest//:gtest_main", + ], +) + cc_library( name = "gcs_table_storage_lib", srcs = glob( @@ -2828,6 +2863,7 @@ filegroup( "//src/ray/protobuf:gcs_py_proto", "//src/ray/protobuf:gcs_service_py_proto", "//src/ray/protobuf:job_agent_py_proto", + "//src/ray/protobuf:monitor_py_proto", "//src/ray/protobuf:node_manager_py_proto", "//src/ray/protobuf:ray_client_py_proto", "//src/ray/protobuf:reporter_py_proto", diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index d158b9372a8b..56241ea1343b 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -203,6 +203,14 @@ py_test_module_list( deps = ["//:ray_lib", ":conftest"], ) +py_test_module_list( + files = [ + "test_monitor_service.py", + ], + size = "medium", + tags = ["exclusive", "medium_size_python_tests_k_to_z", "team:serverless"], + deps = ["//:ray_lib", ":conftest"], +) py_test_module_list( files = [ diff --git a/python/ray/tests/test_monitor_service.py b/python/ray/tests/test_monitor_service.py new file mode 100644 index 000000000000..71c7a3e2af47 --- /dev/null +++ b/python/ray/tests/test_monitor_service.py @@ -0,0 +1,18 @@ +import pytest + +import ray +import grpc +from ray.core.generated import monitor_pb2, monitor_pb2_grpc + + +@pytest.fixture +def monitor_stub(ray_start_regular_shared): + channel = grpc.insecure_channel(ray_start_regular_shared["gcs_address"]) + + return monitor_pb2_grpc.MonitorGcsServiceStub(channel) + + +def test_ray_version(monitor_stub): + request = monitor_pb2.GetRayVersionRequest() + response = monitor_stub.GetRayVersion(request) + assert response.version == ray.__version__ diff --git a/src/ray/gcs/gcs_server/gcs_monitor_server.cc b/src/ray/gcs/gcs_server/gcs_monitor_server.cc new file mode 100644 index 000000000000..af0d36c93134 --- /dev/null +++ b/src/ray/gcs/gcs_server/gcs_monitor_server.cc @@ -0,0 +1,32 @@ +// Copyright 2017 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "ray/gcs/gcs_server/gcs_monitor_server.h" + +#include "ray/common/constants.h" + +namespace ray { +namespace gcs { + +GcsMonitorServer::GcsMonitorServer() {} + +void GcsMonitorServer::HandleGetRayVersion(rpc::GetRayVersionRequest request, + rpc::GetRayVersionReply *reply, + rpc::SendReplyCallback send_reply_callback) { + reply->set_version(kRayVersion); + send_reply_callback(Status::OK(), nullptr, nullptr); +} + +} // namespace gcs +} // namespace ray diff --git a/src/ray/gcs/gcs_server/gcs_monitor_server.h b/src/ray/gcs/gcs_server/gcs_monitor_server.h new file mode 100644 index 000000000000..4fa85bb3b5f7 --- /dev/null +++ b/src/ray/gcs/gcs_server/gcs_monitor_server.h @@ -0,0 +1,33 @@ +// Copyright 2023 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include "ray/rpc/gcs_server/gcs_rpc_server.h" + +namespace ray { +namespace gcs { + +/// GcsMonitorServer is a shim responsible for providing a compatible interface between +/// GCS and `monitor.py` +class GcsMonitorServer : public rpc::MonitorServiceHandler { + public: + explicit GcsMonitorServer(); + + void HandleGetRayVersion(rpc::GetRayVersionRequest request, + rpc::GetRayVersionReply *reply, + rpc::SendReplyCallback send_reply_callback) override; +}; +} // namespace gcs +} // namespace ray diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 5d9f98db3b83..1b013bd79f22 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -166,6 +166,9 @@ void GcsServer::DoStart(const GcsInitData &gcs_init_data) { // Init GCS task manager. InitGcsTaskManager(); + // Init Monitor service. + InitMonitorServer(); + // Install event listeners. InstallEventListeners(); @@ -586,6 +589,13 @@ void GcsServer::InitGcsTaskManager() { rpc_server_.RegisterService(*task_info_service_); } +void GcsServer::InitMonitorServer() { + monitor_server_ = std::make_unique(); + monitor_grpc_service_.reset( + new rpc::MonitorGrpcService(main_service_, *monitor_server_)); + rpc_server_.RegisterService(*monitor_grpc_service_); +} + void GcsServer::InstallEventListeners() { // Install node event listeners. gcs_node_manager_->AddNodeAddedListener([this](std::shared_ptr node) { diff --git a/src/ray/gcs/gcs_server/gcs_server.h b/src/ray/gcs/gcs_server/gcs_server.h index f6397452aceb..dadc1f81e6e7 100644 --- a/src/ray/gcs/gcs_server/gcs_server.h +++ b/src/ray/gcs/gcs_server/gcs_server.h @@ -22,6 +22,7 @@ #include "ray/gcs/gcs_server/gcs_health_check_manager.h" #include "ray/gcs/gcs_server/gcs_init_data.h" #include "ray/gcs/gcs_server/gcs_kv_manager.h" +#include "ray/gcs/gcs_server/gcs_monitor_server.h" #include "ray/gcs/gcs_server/gcs_redis_failure_detector.h" #include "ray/gcs/gcs_server/gcs_table_storage.h" #include "ray/gcs/gcs_server/gcs_task_manager.h" @@ -151,6 +152,9 @@ class GcsServer { /// Install event listeners. void InstallEventListeners(); + /// Initialize monitor service. + void InitMonitorServer(); + private: /// Gets the type of KV storage to use from config. std::string StorageType() const; @@ -215,6 +219,10 @@ class GcsServer { std::unique_ptr function_manager_; /// Node resource info handler and service. std::unique_ptr node_resource_info_service_; + /// Monitor server supports monitor.py + std::unique_ptr monitor_server_; + /// Monitor service for monitor server + std::unique_ptr monitor_grpc_service_; /// Synchronization service for ray. /// TODO(iycheng): Deprecate this gcs_ray_syncer_ one once we roll out diff --git a/src/ray/gcs/gcs_server/test/gcs_monitor_server_test.cc b/src/ray/gcs/gcs_server/test/gcs_monitor_server_test.cc new file mode 100644 index 000000000000..cba584afc6a3 --- /dev/null +++ b/src/ray/gcs/gcs_server/test/gcs_monitor_server_test.cc @@ -0,0 +1,46 @@ + +// Copyright 2017 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include + +// clang-format off +#include "gtest/gtest.h" +#include "ray/gcs/gcs_server/test/gcs_server_test_util.h" +#include "ray/gcs/test/gcs_test_util.h" +#include "ray/gcs/gcs_server/gcs_monitor_server.h" +#include "mock/ray/pubsub/publisher.h" +// clang-format on + +namespace ray { +class GcsMonitorServerTest : public ::testing::Test { + public: + GcsMonitorServerTest() : monitor_server_() {} + + protected: + gcs::GcsMonitorServer monitor_server_; +}; + +TEST_F(GcsMonitorServerTest, TestRayVersion) { + rpc::GetRayVersionRequest request; + rpc::GetRayVersionReply reply; + auto send_reply_callback = + [](ray::Status status, std::function f1, std::function f2) {}; + + monitor_server_.HandleGetRayVersion(request, &reply, send_reply_callback); + + ASSERT_EQ(reply.version(), kRayVersion); +} + +} // namespace ray diff --git a/src/ray/protobuf/BUILD b/src/ray/protobuf/BUILD index 499e0445be9c..22054c994261 100644 --- a/src/ray/protobuf/BUILD +++ b/src/ray/protobuf/BUILD @@ -4,6 +4,21 @@ load("@rules_proto//proto:defs.bzl", "proto_library") load("@rules_cc//cc:defs.bzl", "cc_binary", "cc_library", "cc_proto_library", "cc_test") load("@rules_proto_grpc//python:defs.bzl", "python_grpc_compile") +proto_library( + name = "monitor_proto", + srcs = ["monitor.proto"], +) + +cc_proto_library( + name = "monitor_cc_proto", + deps = [":monitor_proto"], +) + +python_grpc_compile( + name = "monitor_py_proto", + deps = [":monitor_proto"], +) + proto_library( name = "common_proto", srcs = ["common.proto"], diff --git a/src/ray/protobuf/monitor.proto b/src/ray/protobuf/monitor.proto new file mode 100644 index 000000000000..b8190fd9d1e1 --- /dev/null +++ b/src/ray/protobuf/monitor.proto @@ -0,0 +1,30 @@ +// Copyright 2017 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; +option cc_enable_arenas = true; +package ray.rpc; + +message GetRayVersionRequest {} + +message GetRayVersionReply { + string version = 1; +} + +// This service provides a stable interface for a monitor/autoscaler process to interact +// with Ray. +service MonitorGcsService { + // Get the ray version of the service. + rpc GetRayVersion(GetRayVersionRequest) returns (GetRayVersionReply); +} diff --git a/src/ray/rpc/gcs_server/gcs_rpc_server.h b/src/ray/rpc/gcs_server/gcs_rpc_server.h index c2453c88a1df..bcfca5052d9b 100644 --- a/src/ray/rpc/gcs_server/gcs_rpc_server.h +++ b/src/ray/rpc/gcs_server/gcs_rpc_server.h @@ -19,6 +19,7 @@ #include "ray/rpc/grpc_server.h" #include "ray/rpc/server_call.h" #include "src/ray/protobuf/gcs_service.grpc.pb.h" +#include "src/ray/protobuf/monitor.grpc.pb.h" namespace ray { namespace rpc { @@ -31,6 +32,11 @@ namespace rpc { #define ACTOR_INFO_SERVICE_RPC_HANDLER(HANDLER, MAX_ACTIVE_RPCS) \ RPC_SERVICE_HANDLER(ActorInfoGcsService, HANDLER, MAX_ACTIVE_RPCS) +#define MONITOR_SERVICE_RPC_HANDLER(HANDLER) \ + RPC_SERVICE_HANDLER(MonitorGcsService, \ + HANDLER, \ + RayConfig::instance().gcs_max_active_rpcs_per_handler()) + #define NODE_INFO_SERVICE_RPC_HANDLER(HANDLER) \ RPC_SERVICE_HANDLER(NodeInfoGcsService, \ HANDLER, \ @@ -209,6 +215,41 @@ class ActorInfoGrpcService : public GrpcService { ActorInfoGcsServiceHandler &service_handler_; }; +class MonitorGcsServiceHandler { + public: + virtual ~MonitorGcsServiceHandler() = default; + + virtual void HandleGetRayVersion(GetRayVersionRequest request, + GetRayVersionReply *reply, + SendReplyCallback send_reply_callback) = 0; +}; + +/// The `GrpcService` for `MonitorServer`. +class MonitorGrpcService : public GrpcService { + public: + /// Constructor. + /// + /// \param[in] handler The service handler that actually handle the requests. + explicit MonitorGrpcService(instrumented_io_context &io_service, + MonitorGcsServiceHandler &handler) + : GrpcService(io_service), service_handler_(handler){}; + + protected: + grpc::Service &GetGrpcService() override { return service_; } + + void InitServerCallFactories( + const std::unique_ptr &cq, + std::vector> *server_call_factories) override { + MONITOR_SERVICE_RPC_HANDLER(GetRayVersion); + } + + private: + /// The grpc async service object. + MonitorGcsService::AsyncService service_; + /// The service handler that actually handle the requests. + MonitorGcsServiceHandler &service_handler_; +}; + class NodeInfoGcsServiceHandler { public: virtual ~NodeInfoGcsServiceHandler() = default; @@ -581,6 +622,7 @@ class InternalPubSubGrpcService : public GrpcService { using JobInfoHandler = JobInfoGcsServiceHandler; using ActorInfoHandler = ActorInfoGcsServiceHandler; +using MonitorServiceHandler = MonitorGcsServiceHandler; using NodeInfoHandler = NodeInfoGcsServiceHandler; using NodeResourceInfoHandler = NodeResourceInfoGcsServiceHandler; using WorkerInfoHandler = WorkerInfoGcsServiceHandler; From f59f1e6b53a1f385619f8927295a90e7fa066508 Mon Sep 17 00:00:00 2001 From: Balaji Veeramani Date: Thu, 26 Jan 2023 16:08:11 -0800 Subject: [PATCH 012/267] [docs] Improve visibility of API annotations (#31971) Closes https://github.com/ray-project/ray/issues/30396 --- python/ray/util/annotations.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/util/annotations.py b/python/ray/util/annotations.py index 9996b092fcab..f7b93746f910 100644 --- a/python/ray/util/annotations.py +++ b/python/ray/util/annotations.py @@ -49,7 +49,7 @@ def PublicAPI(*args, **kwargs): def wrap(obj): if stability in ["alpha", "beta"]: message = ( - f"PublicAPI ({stability}): This API is in {stability} " + f"**PublicAPI ({stability}):** This API is in {stability} " "and may change before becoming stable." ) else: @@ -80,7 +80,8 @@ def DeveloperAPI(*args, **kwargs): def wrap(obj): _append_doc( - obj, message="DeveloperAPI: This API may change across minor Ray releases." + obj, + message="**DeveloperAPI:** This API may change across minor Ray releases.", ) _mark_annotated(obj) return obj From d770668a1402acb1406b48fa50fc34f1baba3d73 Mon Sep 17 00:00:00 2001 From: Ricky Xu Date: Thu, 26 Jan 2023 19:15:13 -0500 Subject: [PATCH 013/267] [ci][core][data] Have tests from data to be owned by data team on flaky dashboard. (#31972) --- python/ray/data/BUILD | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/data/BUILD b/python/ray/data/BUILD index 2dd31bbbf138..0217c480ab40 100644 --- a/python/ray/data/BUILD +++ b/python/ray/data/BUILD @@ -23,7 +23,7 @@ py_test( size = "large", srcs = ["tests/test_dataset_formats.py"], data = glob(["tests/image-folder/**/*"]), - tags = ["team:core", "exclusive"], + tags = ["team:data", "exclusive"], deps = ["//:ray_lib", ":conftest"], ) @@ -31,7 +31,7 @@ py_test( name = "test_mongo_dataset", size = "large", srcs = ["tests/test_mongo_dataset.py"], - tags = ["team:core", "exclusive", "dataset_integration"], + tags = ["team:data", "exclusive", "dataset_integration"], deps = ["//:ray_lib", ":conftest"], ) @@ -45,6 +45,6 @@ py_test_module_list( ], ), size = "large", - tags = ["team:core", "exclusive"], + tags = ["team:data", "exclusive"], deps = ["//:ray_lib", ":conftest"], ) From 5b0f7a2ddb8eb5dc761cf81fa16087df9be72c5c Mon Sep 17 00:00:00 2001 From: Ricky Xu Date: Thu, 26 Jan 2023 19:19:28 -0500 Subject: [PATCH 014/267] [core][state] Task backend return most recently added tasks when getting all tasks (#31837) Signed-off-by: rickyyx This PR changes the implicit ordering of task events returned by GcsTaskManager when querying all task events with limit: it will now always returns the most recently added task events. Before this PR, there is no implicit ordering and usually the least recently added task events are returned. This allows users to get recent data when data truncation happens. --- python/ray/tests/test_state_api.py | 4 +- src/ray/gcs/gcs_server/gcs_task_manager.cc | 25 +++++++++++- src/ray/gcs/gcs_server/gcs_task_manager.h | 5 ++- .../gcs_server/test/gcs_task_manager_test.cc | 40 +++++++++++++++++++ 4 files changed, 69 insertions(+), 5 deletions(-) diff --git a/python/ray/tests/test_state_api.py b/python/ray/tests/test_state_api.py index 1be1619a7ecc..3859eb11f43a 100644 --- a/python/ray/tests/test_state_api.py +++ b/python/ray/tests/test_state_api.py @@ -2361,10 +2361,10 @@ def f(): def verify(task_attempts): assert len(task_attempts) == 3 - for task_attempt in task_attempts[:-1]: + for task_attempt in task_attempts[1:]: assert task_attempt["state"] == "FAILED" - task_attempts[-1]["state"] == "FINISHED" + task_attempts[0]["state"] == "FINISHED" assert {task_attempt["attempt_number"] for task_attempt in task_attempts} == { 0, diff --git a/src/ray/gcs/gcs_server/gcs_task_manager.cc b/src/ray/gcs/gcs_server/gcs_task_manager.cc index 68dc813758cc..68f75cd918a3 100644 --- a/src/ray/gcs/gcs_server/gcs_task_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_task_manager.cc @@ -30,7 +30,26 @@ void GcsTaskManager::Stop() { std::vector GcsTaskManager::GcsTaskManagerStorage::GetTaskEvents() const { - return task_events_; + std::vector ret; + // NOTE(rickyx): This could be done better if we expose an iterator - which we + // probably have to do if we are supporting pagination in the future. + // As for now, this will make sure data is returned w.r.t insertion order, so we could + // return the more recent entries when limit applies. + RAY_CHECK(next_idx_to_overwrite_ == 0 || next_idx_to_overwrite_ < task_events_.size()) + << "next_idx_to_overwrite=" << next_idx_to_overwrite_ + << " should be in bound. (size=" << task_events_.size() << ")"; + // Copy from the least recently generated data, where `next_idx_to_overwrite_` points to + // the least recently added data. + std::copy(task_events_.begin() + next_idx_to_overwrite_, + task_events_.end(), + std::back_inserter(ret)); + // Copy the wrapped around if any + if (next_idx_to_overwrite_ > 0) { + std::copy(task_events_.begin(), + task_events_.begin() + next_idx_to_overwrite_, + std::back_inserter(ret)); + } + return ret; } std::vector GcsTaskManager::GcsTaskManagerStorage::GetTaskEvents( @@ -345,7 +364,9 @@ void GcsTaskManager::HandleGetTaskEvents(rpc::GetTaskEventsRequest request, auto count = 0; int32_t num_profile_event_limit = 0; int32_t num_status_event_limit = 0; - for (auto &task_event : task_events) { + + for (auto itr = task_events.rbegin(); itr != task_events.rend(); ++itr) { + auto &task_event = *itr; if (request.exclude_driver_task() && !task_event.has_state_updates()) { // Driver related profile events will generate TaskEvent w/o any task state updates. continue; diff --git a/src/ray/gcs/gcs_server/gcs_task_manager.h b/src/ray/gcs/gcs_server/gcs_task_manager.h index afd49d9604eb..f8e779ef7c1a 100644 --- a/src/ray/gcs/gcs_server/gcs_task_manager.h +++ b/src/ray/gcs/gcs_server/gcs_task_manager.h @@ -137,7 +137,10 @@ class GcsTaskManager : public rpc::TaskInfoHandler { /// Get all task events. /// - /// \return all task events stored. + /// This retrieves copies of all task events ordered from the least recently inserted + /// to the most recently inserted task events. + /// + /// \return all task events stored sorted with insertion order. std::vector GetTaskEvents() const; /// Get task events from tasks corresponding to `task_ids`. diff --git a/src/ray/gcs/gcs_server/test/gcs_task_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_task_manager_test.cc index d106f25c7c1d..bf192b2ec609 100644 --- a/src/ray/gcs/gcs_server/test/gcs_task_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_task_manager_test.cc @@ -732,5 +732,45 @@ TEST_F(GcsTaskManagerMemoryLimitedTest, TestLimitTaskEvents) { } } +TEST_F(GcsTaskManagerMemoryLimitedTest, TestLimitReturnRecentTasksWhenGetAll) { + // Keep adding tasks and make sure even with eviction, the returned tasks are + // the mo + size_t num_to_insert = 200; + size_t num_query = 10; + size_t inserted = 0; + + auto task_ids = GenTaskIDs(num_to_insert); + + for (size_t i = 0; i < num_to_insert; ++i) { + // Add a task event + { + inserted++; + auto events = GenTaskEvents({task_ids[i]}, + /* attempt_number */ 0, + /* job_id */ 0, + /* profile event */ absl::nullopt, + GenStateUpdate({{rpc::TaskStatus::RUNNING, 1}})); + auto events_data = Mocker::GenTaskEventsData(events); + SyncAddTaskEventData(events_data); + } + + if (inserted < num_query || inserted % num_query != 0) { + continue; + } + + // Expect returned tasks with limit are the most recently added ones. + { + absl::flat_hash_set query_ids(task_ids.begin() + (inserted - num_query), + task_ids.begin() + inserted); + auto reply = SyncGetTaskEvents( + /* task_ids */ {}, /* job_id */ absl::nullopt, /* limit */ num_query); + for (const auto &task_event : reply.events_by_task()) { + EXPECT_EQ(query_ids.count(TaskID::FromBinary(task_event.task_id())), 1) + << TaskID::FromBinary(task_event.task_id()).Hex() << "not there, at " << i; + } + } + } +} + } // namespace gcs } // namespace ray From 6bcaa9c55f7ad4ea14366ac7762458d5bf17ffd5 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 26 Jan 2023 17:02:29 -0800 Subject: [PATCH 015/267] [tune] Only keep cached actors if search has not ended (#31974) We currently keep one actor cached if no other trials have been staged to prevent us removing actors when we may need them in one of the next iterations. However, when the search algorithm won't produce new trials anymore, we keep this actor needlessly. This can keep resources occupied needlessly and prevent downscaling, as caught in #31883. This PR passes another flag to the trial executor cleanup method indicating if new trials are to be expected. If not, we can cleanup unneeded actors even if no further trials are staged. This behavior is tested in the `cluster_tune_scale_up_down` release test. We won't add a unit test for this as this would effectively mimic the release test. Instead, we can add proper actor reuse testing once we made more progress with the execution refactor. Signed-off-by: Kai Fricke --- .../ray/tune/execution/ray_trial_executor.py | 14 +++++++---- python/ray/tune/execution/trial_runner.py | 2 +- python/ray/tune/tests/test_trial_runner_3.py | 2 +- .../workloads/tune_scale_up_down.py | 24 +++++++++++++++++++ 4 files changed, 35 insertions(+), 7 deletions(-) diff --git a/python/ray/tune/execution/ray_trial_executor.py b/python/ray/tune/execution/ray_trial_executor.py index a3e8f3f56af8..035bb55defd2 100644 --- a/python/ray/tune/execution/ray_trial_executor.py +++ b/python/ray/tune/execution/ray_trial_executor.py @@ -843,8 +843,8 @@ def on_step_begin(self) -> None: """Before step() is called, update the available resources.""" self._resource_updater.update_avail_resources() - def on_step_end(self) -> None: - self._cleanup_cached_actors() + def on_step_end(self, search_ended: bool = False) -> None: + self._cleanup_cached_actors(search_ended=search_ended) self._do_force_trial_cleanup() def _count_staged_resources(self): @@ -854,7 +854,9 @@ def _count_staged_resources(self): counter[resource_request] += 1 return counter - def _cleanup_cached_actors(self, force_all: bool = False): + def _cleanup_cached_actors( + self, search_ended: bool = False, force_all: bool = False + ): """Clean up unneeded cached actors. Ray Tune caches actors for re-use to avoid initialization overhead. This is @@ -884,8 +886,10 @@ def _cleanup_cached_actors(self, force_all: bool = False): resources for all cached actors. If we cached more actors than we need, we terminate the excess actors and free the resources. """ - if not self._staged_trials and not force_all: - # If we don't have any staged trials, keep cached actors + if not self._staged_trials and not force_all and not search_ended: + # If we don't have any staged trials, keep cached actors, + # unless cleanup is forced or no new trials are going to be generated + # (if the search ended). return staged_resources = self._count_staged_resources() diff --git a/python/ray/tune/execution/trial_runner.py b/python/ray/tune/execution/trial_runner.py index 0479dd6eeb4d..f25e63828220 100644 --- a/python/ray/tune/execution/trial_runner.py +++ b/python/ray/tune/execution/trial_runner.py @@ -967,7 +967,7 @@ def step(self): self._reconcile_live_trials() with warn_if_slow("on_step_end"): - self.trial_executor.on_step_end() + self.trial_executor.on_step_end(search_ended=self._search_alg.is_finished()) with warn_if_slow("callbacks.on_step_end"): self._callbacks.on_step_end(iteration=self._iteration, trials=self._trials) diff --git a/python/ray/tune/tests/test_trial_runner_3.py b/python/ray/tune/tests/test_trial_runner_3.py index a50b714553de..79c130377f42 100644 --- a/python/ray/tune/tests/test_trial_runner_3.py +++ b/python/ray/tune/tests/test_trial_runner_3.py @@ -75,7 +75,7 @@ def on_step_begin(self): cnt = self.pre_step if hasattr(self, "pre_step") else 0 self.pre_step = cnt + 1 - def on_step_end(self): + def on_step_end(self, search_ended: bool = False): cnt = self.pre_step if hasattr(self, "post_step") else 0 self.post_step = 1 + cnt diff --git a/release/cluster_tests/workloads/tune_scale_up_down.py b/release/cluster_tests/workloads/tune_scale_up_down.py index f99ebcfc496d..7909ce8aa0c7 100644 --- a/release/cluster_tests/workloads/tune_scale_up_down.py +++ b/release/cluster_tests/workloads/tune_scale_up_down.py @@ -1,3 +1,27 @@ +"""Test cluster up/down scaling behavior. + +This test should run on a cluster with autoscaling enabled. It assumes 1-3 nodes +with 4 CPUs each. + +We start a Ray Tune run with 3 trials. Each trial uses 4 CPUs, so fills up a node +completely. This means we will trigger autoscaling after starting up. + +The trial on the head node will run for 30 minutes. This is to make sure that +we have enough time that the nodes for the other two trials come up, complete +training, and come down before the first trial finishes. + +The other two trials will run once their nodes are up, and take 3 minutes each +to finish. The three minutes have been chosen to make sure that both trials +run in parallel for some time, i.e. to avoid that both additional trials run on +only one node. + +We keep track of the number of nodes we observe at any point during the run. + +Test owner: krfricke + +Acceptance criteria: Should have scaled to 3 nodes at some point during the run. +Should have scaled down to 1 node at the end. +""" from collections import Counter import time From 581cd4e21e74f7e7b85386fbd1e80d40c3251623 Mon Sep 17 00:00:00 2001 From: Yi Cheng <74173148+iycheng@users.noreply.github.com> Date: Thu, 26 Jan 2023 20:03:13 -0800 Subject: [PATCH 016/267] [nightly] Update the many nodes actor workload to 1k nodes. (#31935) This PR update the test to run with 1k nodes 20k actors. Right now the infra can't scale to 1k nodes when starting because of rate limiting. This PR scale up inside the testing script slowly to avoid the rate limiting. --- .../many_nodes_tests/actor_test.py | 125 ++++++++++++++++++ .../many_nodes_tests/app_config.yaml | 0 .../many_nodes_tests/compute_config.yaml | 6 +- .../many_nodes_tests/multi_master_test.py | 0 .../many_nodes_tests/actor_test.py | 73 ---------- release/release_tests.yaml | 19 +-- 6 files changed, 138 insertions(+), 85 deletions(-) create mode 100644 release/benchmarks/distributed/many_nodes_tests/actor_test.py rename release/{nightly_tests => benchmarks/distributed}/many_nodes_tests/app_config.yaml (100%) rename release/{nightly_tests => benchmarks/distributed}/many_nodes_tests/compute_config.yaml (71%) rename release/{nightly_tests => benchmarks/distributed}/many_nodes_tests/multi_master_test.py (100%) delete mode 100644 release/nightly_tests/many_nodes_tests/actor_test.py diff --git a/release/benchmarks/distributed/many_nodes_tests/actor_test.py b/release/benchmarks/distributed/many_nodes_tests/actor_test.py new file mode 100644 index 000000000000..495570a74c6d --- /dev/null +++ b/release/benchmarks/distributed/many_nodes_tests/actor_test.py @@ -0,0 +1,125 @@ +import argparse +import os +import math +from time import sleep, perf_counter +import json +import ray +import psutil + + +def test_max_actors_launch(cpus_per_actor, total_actors): + @ray.remote(num_cpus=cpus_per_actor) + class Actor: + def foo(self): + pass + + print("Start launch actors") + actors = [Actor.options(max_restarts=-1).remote() for _ in range(total_actors)] + return actors + + +def parse_script_args(): + parser = argparse.ArgumentParser() + parser.add_argument("--cpus-per-actor", type=float, default=0.2) + parser.add_argument("--total-actors", nargs="+", type=int, required=True) + parser.add_argument("--no-report", default=False, action="store_true") + parser.add_argument("--no-wait", default=False, action="store_true") + return parser.parse_known_args() + + +def scale_cluster_up(num_cpus): + print(f"Start to scale up to {num_cpus} cpus") + + def get_curr_cpus(): + return int(sum([r.get("Resources", {}).get("CPU", 0) for r in ray.nodes()])) + + step = 1000 + curr_cpus = get_curr_cpus() + target_cpus = curr_cpus + + while curr_cpus < num_cpus: + curr_cpus = get_curr_cpus() + new_target_cpus = min(curr_cpus + step, num_cpus) + if new_target_cpus != target_cpus: + target_cpus = new_target_cpus + ray.autoscaler.sdk.request_resources(num_cpus=target_cpus) + print(f"Waiting for cluster to be up: {curr_cpus}->{target_cpus}->{num_cpus}") + sleep(10) + + +def run_one(total_actors, cpus_per_actor, no_wait): + total_cpus = cpus_per_actor * total_actors + psutil.cpu_count() + total_cpus = int(math.ceil(total_cpus)) + scale_cluster_up(total_cpus) + + actor_launch_start = perf_counter() + actors = test_max_actors_launch(cpus_per_actor, total_actors) + actor_launch_end = perf_counter() + actor_launch_time = actor_launch_end - actor_launch_start + actor_ready_start = perf_counter() + total_actors = len(actors) + objs = [actor.foo.remote() for actor in actors] + + while len(objs) != 0: + timeout = None if no_wait else 30 + objs_ready, objs = ray.wait(objs, num_returns=len(objs), timeout=timeout) + print( + f"Status: {total_actors - len(objs)}/{total_actors}, " + f"{perf_counter() - actor_ready_start}" + ) + actor_ready_end = perf_counter() + actor_ready_time = actor_ready_end - actor_ready_start + + throughput = total_actors / (actor_ready_time + actor_launch_time) + print(f"Actor launch time: {actor_launch_time} ({total_actors} actors)") + print(f"Actor ready time: {actor_ready_time} ({total_actors} actors)") + print( + f"Total time: {actor_launch_time + actor_ready_time}" + f" ({total_actors} actors)" + ) + print(f"Through put: {throughput}") + + return { + "actor_launch_time": actor_launch_time, + "actor_ready_time": actor_ready_time, + "total_time": actor_launch_time + actor_ready_time, + "num_actors": total_actors, + "success": "1", + "throughput": throughput, + } + + +def main(): + args, unknown = parse_script_args() + args.total_actors.sort() + + ray.init(address="auto") + + dashboard_test = None + # Enable it once v2 support prometheus + # dashboard_test = DashboardTestAtScale(addr) + result = {} + for i in args.total_actors: + result[f"many_nodes_actor_tests_{i}"] = run_one( + i, args.cpus_per_actor, args.no_wait + ) + + if "TEST_OUTPUT_JSON" in os.environ and not args.no_report: + out_file = open(os.environ["TEST_OUTPUT_JSON"], "w") + if dashboard_test is not None: + perf = [ + { + "perf_metric_name": name, + "perf_metric_value": r["throughput"], + "perf_metric_type": "THROUGHPUT", + } + for (name, r) in result.items() + ] + result["perf_metrics"] = perf + dashboard_test.update_release_test_result(result) + + json.dump(result, out_file) + + +if __name__ == "__main__": + main() diff --git a/release/nightly_tests/many_nodes_tests/app_config.yaml b/release/benchmarks/distributed/many_nodes_tests/app_config.yaml similarity index 100% rename from release/nightly_tests/many_nodes_tests/app_config.yaml rename to release/benchmarks/distributed/many_nodes_tests/app_config.yaml diff --git a/release/nightly_tests/many_nodes_tests/compute_config.yaml b/release/benchmarks/distributed/many_nodes_tests/compute_config.yaml similarity index 71% rename from release/nightly_tests/many_nodes_tests/compute_config.yaml rename to release/benchmarks/distributed/many_nodes_tests/compute_config.yaml index 8a0e63ebb518..882fca85b154 100644 --- a/release/nightly_tests/many_nodes_tests/compute_config.yaml +++ b/release/benchmarks/distributed/many_nodes_tests/compute_config.yaml @@ -8,7 +8,7 @@ head_node_type: worker_node_types: - name: worker_node - instance_type: m5.xlarge - min_workers: 250 - max_workers: 250 + instance_type: m5.large + min_workers: 500 + max_workers: 2000 use_spot: false diff --git a/release/nightly_tests/many_nodes_tests/multi_master_test.py b/release/benchmarks/distributed/many_nodes_tests/multi_master_test.py similarity index 100% rename from release/nightly_tests/many_nodes_tests/multi_master_test.py rename to release/benchmarks/distributed/many_nodes_tests/multi_master_test.py diff --git a/release/nightly_tests/many_nodes_tests/actor_test.py b/release/nightly_tests/many_nodes_tests/actor_test.py deleted file mode 100644 index 884647d1c0f3..000000000000 --- a/release/nightly_tests/many_nodes_tests/actor_test.py +++ /dev/null @@ -1,73 +0,0 @@ -import argparse -import os -from time import sleep, perf_counter -import json -import ray - - -def test_max_actors_launch(cpus_per_actor, total_actors): - @ray.remote(num_cpus=cpus_per_actor) - class Actor: - def foo(self): - pass - - print("Start launch actors") - actors = [Actor.options(max_restarts=-1).remote() for _ in range(total_actors)] - return actors - - -def parse_script_args(): - parser = argparse.ArgumentParser() - parser.add_argument("--cpus-per-actor", type=float, default=0.2) - parser.add_argument("--total-actors", type=int, default=5000) - parser.add_argument("--no-report", default=False, action="store_true") - parser.add_argument("--fail", default=False, action="store_true") - return parser.parse_known_args() - - -def main(): - args, unknown = parse_script_args() - - ray.init(address="auto") - actor_launch_start = perf_counter() - actors = test_max_actors_launch(args.cpus_per_actor, args.total_actors) - actor_launch_end = perf_counter() - actor_launch_time = actor_launch_end - actor_launch_start - - if args.fail: - sleep(10) - return - actor_ready_start = perf_counter() - total_actors = len(actors) - objs = [actor.foo.remote() for actor in actors] - - while len(objs) != 0: - objs_ready, objs = ray.wait(objs, timeout=10) - print( - f"Status: {total_actors - len(objs)}/{total_actors}, " - f"{perf_counter() - actor_ready_start}" - ) - actor_ready_end = perf_counter() - actor_ready_time = actor_ready_end - actor_ready_start - - print(f"Actor launch time: {actor_launch_time} ({args.total_actors} actors)") - print(f"Actor ready time: {actor_ready_time} ({args.total_actors} actors)") - print( - f"Total time: {actor_launch_time + actor_ready_time}" - f" ({args.total_actors} actors)" - ) - - if "TEST_OUTPUT_JSON" in os.environ and not args.no_report: - out_file = open(os.environ["TEST_OUTPUT_JSON"], "w") - results = { - "actor_launch_time": actor_launch_time, - "actor_ready_time": actor_ready_time, - "total_time": actor_launch_time + actor_ready_time, - "num_actors": args.total_actors, - "success": "1", - } - json.dump(results, out_file) - - -if __name__ == "__main__": - main() diff --git a/release/release_tests.yaml b/release/release_tests.yaml index bb830a6db379..1c6e51a078dc 100644 --- a/release/release_tests.yaml +++ b/release/release_tests.yaml @@ -3872,27 +3872,28 @@ num_nodes: 32 -- name: many_nodes_actor_test +- name: many_nodes_actor_test_on_v2 group: core-daily-test - working_dir: nightly_tests + working_dir: benchmarks legacy: test_name: many_nodes_actor_test - test_suite: nightly_tests + test_suite: benchmarks frequency: nightly-3x team: core env: staging cluster: - cluster_env: many_nodes_tests/app_config.yaml - cluster_compute: many_nodes_tests/compute_config.yaml + cluster_env: distributed/many_nodes_tests/app_config.yaml + cluster_compute: distributed/many_nodes_tests/compute_config.yaml run: timeout: 7200 - # 4cpus per node x 250 nodes / 0.2 cpus per actor = 5k - script: python many_nodes_tests/actor_test.py --cpus-per-actor=0.2 --total-actors=5000 - wait_for_nodes: - num_nodes: 251 + # 2cpus per node x 1000 nodes / 0.2 cpus per actor = 10k + # 2cpus per node x 2000 nodes / 0.2 cpus per actor = 20k + script: python distributed/many_nodes_tests/actor_test.py --no-wait --cpus-per-actor=0.2 --total-actors 1000 2000 type: job + wait_for_nodes: + num_nodes: 500 #- name: many_nodes_multi_master_test # group: core-daily-test From b60f887597f1d5cd29039bd6fe584dfb986ca177 Mon Sep 17 00:00:00 2001 From: Yi Cheng <74173148+iycheng@users.noreply.github.com> Date: Thu, 26 Jan 2023 22:06:04 -0800 Subject: [PATCH 017/267] [nightly] Increase the number of actors started for the test (#31990) Signed-off-by: Yi Cheng <74173148+iycheng@users.noreply.github.com> In the previously PR, the number failed to be updated. This PR updated it. ``` many_nodes_actor_tests_10000 = {'actor_launch_time': 14.038594387999979, 'actor_ready_time': 44.27844531799997, 'total_time': 58.317039705999946, 'num_actors': 10000, 'success': '1', 'throughput': 17 1.47646812002273} many_nodes_actor_tests_20000 = {'actor_launch_time': 17.670945039000117, 'actor_ready_time': 230.851894916, 'total_time': 248.5228399550001, 'num_actors': 20000, 'success': '1', 'throughput': 80.4755 0077739892} ``` --- release/release_tests.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/release/release_tests.yaml b/release/release_tests.yaml index 1c6e51a078dc..e1db50e84d51 100644 --- a/release/release_tests.yaml +++ b/release/release_tests.yaml @@ -3890,7 +3890,7 @@ timeout: 7200 # 2cpus per node x 1000 nodes / 0.2 cpus per actor = 10k # 2cpus per node x 2000 nodes / 0.2 cpus per actor = 20k - script: python distributed/many_nodes_tests/actor_test.py --no-wait --cpus-per-actor=0.2 --total-actors 1000 2000 + script: python distributed/many_nodes_tests/actor_test.py --no-wait --cpus-per-actor=0.2 --total-actors 10000 20000 type: job wait_for_nodes: num_nodes: 500 From 3343c76332ce0db9d0f8c98d6ac153804e4afe18 Mon Sep 17 00:00:00 2001 From: Peyton Murray Date: Thu, 26 Jan 2023 23:18:49 -0800 Subject: [PATCH 018/267] Add informative progress bar names to map_batches (#31526) Signed-off-by: pdmurray Signed-off-by: pdmurray --- .bazeliskrc | 1 + .gitignore | 2 + python/ray/air/tests/test_dataset_config.py | 6 +- python/ray/data/_internal/plan.py | 26 +++++++- python/ray/data/dataset.py | 10 ++- python/ray/data/tests/test_dataset.py | 10 +-- python/ray/data/tests/test_optimize.py | 64 +++++++++++-------- python/ray/data/tests/test_stats.py | 45 +++++++------ .../ray/train/tests/test_batch_predictor.py | 16 ++--- 9 files changed, 115 insertions(+), 65 deletions(-) create mode 100644 .bazeliskrc diff --git a/.bazeliskrc b/.bazeliskrc new file mode 100644 index 000000000000..9adf2699b0d0 --- /dev/null +++ b/.bazeliskrc @@ -0,0 +1 @@ +USE_BAZEL_VERSION=5.x diff --git a/.gitignore b/.gitignore index 297a7b9d0608..9268fa948c4e 100644 --- a/.gitignore +++ b/.gitignore @@ -224,3 +224,5 @@ workflow_data/ # Auto-generated tag mapping tag-mapping.json + +.bazeliskrc diff --git a/python/ray/air/tests/test_dataset_config.py b/python/ray/air/tests/test_dataset_config.py index 6616395f93bd..902835e3f4b9 100644 --- a/python/ray/air/tests/test_dataset_config.py +++ b/python/ray/air/tests/test_dataset_config.py @@ -251,7 +251,7 @@ def checker(shard, results): # applying the preprocessor on each epoch. assert results[0] == results[1], results stats = shard.stats() - assert "Stage 1 read->map_batches: 1/1 blocks executed " in stats, stats + assert "Stage 1 read->BatchMapper: 1/1 blocks executed " in stats, stats def rand(x): x["value"] = [random.random() for _ in range(len(x))] @@ -284,8 +284,8 @@ def checker(shard, results): assert results[0] != results[1], results stats = shard.stats() assert ( - "Stage 1 read->randomize_block_order->map_batches: 1/1 blocks executed " - in stats + "Stage 1 read->randomize_block_order->" + "BatchMapper: 1/1 blocks executed " in stats ), stats test = TestStream( diff --git a/python/ray/data/_internal/plan.py b/python/ray/data/_internal/plan.py index 3bc492e944e9..47097bf2821b 100644 --- a/python/ray/data/_internal/plan.py +++ b/python/ray/data/_internal/plan.py @@ -45,6 +45,30 @@ logger = DatasetLogger(__name__) +def capfirst(s: str): + """Capitalize the first letter of a string + + Args: + s: String to capitalize + + Returns: + Capitalized string + """ + return s[0].upper() + s[1:] + + +def capitalize(s: str): + """Capitalize a string, removing '_' and keeping camelcase. + + Args: + s: String to capitalize + + Returns: + Capitalized string with no underscores. + """ + return "".join(capfirst(x) for x in s.split("_")) + + class Stage: """Represents a Dataset transform stage (e.g., map or shuffle).""" @@ -157,7 +181,7 @@ def get_plan_as_string(self) -> str: # Get string representation of each stage in reverse order. for stage in self._stages_after_snapshot[::-1]: # Get name of each stage in camel case. - stage_name = stage.name.title().replace("_", "") + stage_name = capitalize(stage.name) if num_stages == 0: plan_str += f"{stage_name}\n" else: diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index 7ab51669992b..f2f9db25753d 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -671,8 +671,16 @@ def process_next_batch(batch: DataBatch) -> Iterator[Block]: if output_buffer.has_next(): yield output_buffer.next() + # breakpoint() + if hasattr(fn, "__self__") and isinstance( + fn.__self__, ray.data.preprocessor.Preprocessor + ): + stage_name = fn.__self__.__class__.__name__ + else: + stage_name = f'MapBatches({getattr(fn, "__name__", type(fn))})' + stage = OneToOneStage( - "map_batches", + stage_name, transform, compute, ray_remote_args, diff --git a/python/ray/data/tests/test_dataset.py b/python/ray/data/tests/test_dataset.py index e593c5ca60ce..4cd1d39aa5a8 100644 --- a/python/ray/data/tests/test_dataset.py +++ b/python/ray/data/tests/test_dataset.py @@ -1517,26 +1517,28 @@ def test_dataset_repr(ray_start_regular_shared): assert repr(ds) == "Dataset(num_blocks=10, num_rows=10, schema=)" ds = ds.map_batches(lambda x: x) assert repr(ds) == ( - "MapBatches\n" "+- Dataset(num_blocks=10, num_rows=10, schema=)" + "MapBatches()\n" + "+- Dataset(num_blocks=10, num_rows=10, schema=)" ) ds = ds.filter(lambda x: x > 0) assert repr(ds) == ( "Filter\n" - "+- MapBatches\n" + "+- MapBatches()\n" " +- Dataset(num_blocks=10, num_rows=10, schema=)" ) ds = ds.random_shuffle() assert repr(ds) == ( "RandomShuffle\n" "+- Filter\n" - " +- MapBatches\n" + " +- MapBatches()\n" " +- Dataset(num_blocks=10, num_rows=10, schema=)" ) ds.fully_executed() assert repr(ds) == "Dataset(num_blocks=10, num_rows=9, schema=)" ds = ds.map_batches(lambda x: x) assert repr(ds) == ( - "MapBatches\n" "+- Dataset(num_blocks=10, num_rows=9, schema=)" + "MapBatches()\n" + "+- Dataset(num_blocks=10, num_rows=9, schema=)" ) ds1, ds2 = ds.split(2) assert ( diff --git a/python/ray/data/tests/test_optimize.py b/python/ray/data/tests/test_optimize.py index e8d0893aa3a4..97b62885e1b2 100644 --- a/python/ray/data/tests/test_optimize.py +++ b/python/ray/data/tests/test_optimize.py @@ -59,6 +59,11 @@ def expect_stages(pipe, num_stages_expected, stage_names): ), pipe._optimized_stages +def dummy_map(x): + """Dummy function used in calls to map_batches in these tests.""" + return x + + def test_memory_sanity(shutdown_only): info = ray.init(num_cpus=1, object_store_memory=500e6) ds = ray.data.range(10) @@ -312,23 +317,23 @@ def test_optimize_reorder(ray_start_regular_shared): context.optimize_fuse_read_stages = True context.optimize_reorder_stages = True - ds = ray.data.range(10).randomize_block_order().map_batches(lambda x: x) + ds = ray.data.range(10).randomize_block_order().map_batches(dummy_map) expect_stages( ds, 2, - ["read->map_batches", "randomize_block_order"], + ["read->MapBatches(dummy_map)", "randomize_block_order"], ) ds2 = ( ray.data.range(10) .randomize_block_order() .repartition(10) - .map_batches(lambda x: x) + .map_batches(dummy_map) ) expect_stages( ds2, 3, - ["read->randomize_block_order", "repartition", "map_batches"], + ["read->randomize_block_order", "repartition", "MapBatches(dummy_map)"], ) @@ -338,10 +343,10 @@ def test_window_randomize_fusion(ray_start_regular_shared): context.optimize_fuse_read_stages = True context.optimize_reorder_stages = True - pipe = ray.data.range(100).randomize_block_order().window().map_batches(lambda x: x) + pipe = ray.data.range(100).randomize_block_order().window().map_batches(dummy_map) pipe.take() stats = pipe.stats() - assert "read->randomize_block_order->map_batches" in stats, stats + assert "read->randomize_block_order->MapBatches(dummy_map)" in stats, stats def test_optimize_fuse(ray_start_regular_shared): @@ -349,8 +354,8 @@ def test_optimize_fuse(ray_start_regular_shared): def build_pipe(): pipe = ray.data.range(3).window(blocks_per_window=1).repeat(2) - pipe = pipe.map_batches(lambda x: x) - pipe = pipe.map_batches(lambda x: x) + pipe = pipe.map_batches(dummy_map) + pipe = pipe.map_batches(dummy_map) pipe = pipe.random_shuffle_each_window() results = [sorted(p.take()) for p in pipe.iter_epochs()] assert results == [[0, 1, 2], [0, 1, 2]], results @@ -362,7 +367,10 @@ def build_pipe(): expect_stages( build_pipe(), 1, - ["read->map_batches->map_batches->random_shuffle_map", "random_shuffle_reduce"], + [ + "read->MapBatches(dummy_map)->MapBatches(dummy_map)->random_shuffle_map", + "random_shuffle_reduce", + ], ) context.optimize_fuse_stages = True @@ -373,7 +381,7 @@ def build_pipe(): 1, [ "read", - "map_batches->map_batches->random_shuffle_map", + "MapBatches(dummy_map)->MapBatches(dummy_map)->random_shuffle_map", "random_shuffle_reduce", ], ) @@ -386,7 +394,7 @@ def build_pipe(): 2, [ "read", - "map_batches->map_batches", + "MapBatches(dummy_map)->MapBatches(dummy_map)", "random_shuffle_map", "random_shuffle_reduce", ], @@ -400,8 +408,8 @@ def build_pipe(): 3, [ "read", - "map_batches", - "map_batches", + "MapBatches(dummy_map)", + "MapBatches(dummy_map)", "random_shuffle_map", "random_shuffle_reduce", ], @@ -428,14 +436,14 @@ def test_optimize_equivalent_remote_args(ray_start_regular_shared): for kwb in equivalent_kwargs: print("CHECKING", kwa, kwb) pipe = ray.data.range(3).repeat(2) - pipe = pipe.map_batches(lambda x: x, compute="tasks", **kwa) - pipe = pipe.map_batches(lambda x: x, compute="tasks", **kwb) + pipe = pipe.map_batches(dummy_map, compute="tasks", **kwa) + pipe = pipe.map_batches(dummy_map, compute="tasks", **kwb) pipe.take() expect_stages( pipe, 1, [ - "read->map_batches->map_batches", + "read->MapBatches(dummy_map)->MapBatches(dummy_map)", ], ) @@ -443,14 +451,14 @@ def test_optimize_equivalent_remote_args(ray_start_regular_shared): for kwb in equivalent_kwargs: print("CHECKING", kwa, kwb) pipe = ray.data.range(3).repeat(2) - pipe = pipe.map_batches(lambda x: x, compute="tasks", **kwa) + pipe = pipe.map_batches(dummy_map, compute="tasks", **kwa) pipe = pipe.random_shuffle_each_window(**kwb) pipe.take() expect_stages( pipe, 1, [ - "read->map_batches->random_shuffle_map", + "read->MapBatches(dummy_map)->random_shuffle_map", "random_shuffle_reduce", ], ) @@ -464,32 +472,32 @@ def test_optimize_incompatible_stages(ray_start_regular_shared): pipe = ray.data.range(3).repeat(2) # Should get fused as long as their resource types are compatible. - pipe = pipe.map_batches(lambda x: x, compute="actors") + pipe = pipe.map_batches(dummy_map, compute="actors") # Cannot fuse actors->tasks. - pipe = pipe.map_batches(lambda x: x, compute="tasks") + pipe = pipe.map_batches(dummy_map, compute="tasks") pipe = pipe.random_shuffle_each_window() pipe.take() expect_stages( pipe, 2, [ - "read->map_batches", - "map_batches->random_shuffle_map", + "read->MapBatches(dummy_map)", + "MapBatches(dummy_map)->random_shuffle_map", "random_shuffle_reduce", ], ) pipe = ray.data.range(3).repeat(2) - pipe = pipe.map_batches(lambda x: x, compute="tasks") - pipe = pipe.map_batches(lambda x: x, num_cpus=0.75) + pipe = pipe.map_batches(dummy_map, compute="tasks") + pipe = pipe.map_batches(dummy_map, num_cpus=0.75) pipe = pipe.random_shuffle_each_window() pipe.take() expect_stages( pipe, 3, [ - "read->map_batches", - "map_batches", + "read->MapBatches(dummy_map)", + "MapBatches(dummy_map)", "random_shuffle_map", "random_shuffle_reduce", ], @@ -556,7 +564,7 @@ def __call__(self, x): pipe, 1, [ - "read->map_batches->map_batches", + "read->MapBatches(CallableFn)->MapBatches(CallableFn)", ], ) @@ -592,7 +600,7 @@ def __call__(self, x): pipe, 1, [ - "read->map_batches->map_batches", + "read->MapBatches()->MapBatches(CallableFn)", ], ) diff --git a/python/ray/data/tests/test_stats.py b/python/ray/data/tests/test_stats.py index c5c85c27ddda..454750723cfd 100644 --- a/python/ray/data/tests/test_stats.py +++ b/python/ray/data/tests/test_stats.py @@ -25,6 +25,11 @@ def canonicalize(stats: str) -> str: return s4 +def dummy_map_batches(x): + """Dummy function used in calls to map_batches below.""" + return x + + def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): context = DatasetContext.get_current() context.optimize_fuse_stages = True @@ -39,7 +44,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): ) with patch.object(logger, "info") as mock_logger: ds = ray.data.range(1000, parallelism=10) - ds = ds.map_batches(lambda x: x).fully_executed() + ds = ds.map_batches(dummy_map_batches).fully_executed() if enable_auto_log_stats: logger_args, logger_kwargs = mock_logger.call_args @@ -47,7 +52,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): if context.new_execution_backend: assert ( canonicalize(logger_args[0]) - == """Stage N read->map_batches: N/N blocks executed in T + == """Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -61,7 +66,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): else: assert ( canonicalize(logger_args[0]) - == """Stage N read->map_batches: N/N blocks executed in T + == """Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -71,7 +76,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): """ ) - ds = ds.map(lambda x: x).fully_executed() + ds = ds.map(dummy_map_batches).fully_executed() if enable_auto_log_stats: logger_args, logger_kwargs = mock_logger.call_args @@ -108,7 +113,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): if context.new_execution_backend: assert ( stats - == """Stage N read->map_batches: N/N blocks executed in T + == """Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -140,7 +145,7 @@ def test_dataset_stats_basic(ray_start_regular_shared, enable_auto_log_stats): else: assert ( stats - == """Stage N read->map_batches: N/N blocks executed in T + == """Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -369,7 +374,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ with patch.object(logger, "info") as mock_logger: ds = ray.data.range(1000, parallelism=10) - ds = ds.map_batches(lambda x: x).fully_executed() + ds = ds.map_batches(dummy_map_batches).fully_executed() if enable_auto_log_stats: logger_args, logger_kwargs = mock_logger.call_args @@ -377,7 +382,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ if context.new_execution_backend: assert ( canonicalize(logger_args[0]) - == """Stage N read->map_batches: N/N blocks executed in T + == """Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -391,7 +396,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ else: assert ( canonicalize(logger_args[0]) - == """Stage N read->map_batches: N/N blocks executed in T + == """Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -402,14 +407,14 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ ) pipe = ds.repeat(5) - pipe = pipe.map(lambda x: x) + pipe = pipe.map(dummy_map_batches) if enable_auto_log_stats: # Stats only include first stage, and not for pipelined map logger_args, logger_kwargs = mock_logger.call_args if context.new_execution_backend: assert ( canonicalize(logger_args[0]) - == """Stage N read->map_batches: N/N blocks executed in T + == """Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -423,7 +428,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ else: assert ( canonicalize(logger_args[0]) - == """Stage N read->map_batches: N/N blocks executed in T + == """Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -473,7 +478,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ assert ( stats == """== Pipeline Window N == -Stage N read->map_batches: N/N blocks executed in T +Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -494,7 +499,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ 'obj_store_mem_peak': N} == Pipeline Window N == -Stage N read->map_batches: [execution cached] +Stage N read->MapBatches(dummy_map_batches): [execution cached] * Extra metrics: {'obj_store_mem_alloc': N, 'obj_store_mem_freed': N, \ 'obj_store_mem_peak': N} @@ -509,7 +514,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ 'obj_store_mem_peak': N} == Pipeline Window N == -Stage N read->map_batches: [execution cached] +Stage N read->MapBatches(dummy_map_batches): [execution cached] * Extra metrics: {'obj_store_mem_alloc': N, 'obj_store_mem_freed': N, \ 'obj_store_mem_peak': N} @@ -540,7 +545,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ assert ( stats == """== Pipeline Window N == -Stage N read->map_batches: N/N blocks executed in T +Stage N read->MapBatches(dummy_map_batches): N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total * Remote cpu time: T min, T max, T mean, T total * Peak heap memory usage (MiB): N min, N max, N mean @@ -557,7 +562,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ * Tasks per node: N min, N max, N mean; N nodes used == Pipeline Window N == -Stage N read->map_batches: [execution cached] +Stage N read->MapBatches(dummy_map_batches): [execution cached] Stage N map: N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total @@ -568,7 +573,7 @@ def test_dataset_pipeline_stats_basic(ray_start_regular_shared, enable_auto_log_ * Tasks per node: N min, N max, N mean; N nodes used == Pipeline Window N == -Stage N read->map_batches: [execution cached] +Stage N read->MapBatches(dummy_map_batches): [execution cached] Stage N map: N/N blocks executed in T * Remote wall time: T min, T max, T mean, T total @@ -607,11 +612,11 @@ def test_dataset_pipeline_cache_cases(ray_start_regular_shared): assert "[execution cached]" in stats # CACHED (eager map stage). - ds = ray.data.range(10).map_batches(lambda x: x).repeat(2) + ds = ray.data.range(10).map_batches(dummy_map_batches).repeat(2) ds.take(999) stats = ds.stats() assert "[execution cached]" in stats - assert "read->map_batches" in stats + assert "read->MapBatches(dummy_map_batches)" in stats def test_dataset_pipeline_split_stats_basic(ray_start_regular_shared): diff --git a/python/ray/train/tests/test_batch_predictor.py b/python/ray/train/tests/test_batch_predictor.py index 5b04009318ed..5c51a502408e 100644 --- a/python/ray/train/tests/test_batch_predictor.py +++ b/python/ray/train/tests/test_batch_predictor.py @@ -116,8 +116,8 @@ def test_separate_gpu_stage(shutdown_only): allow_gpu=True, ) stats = ds.stats() - assert "Stage 1 read->map_batches:" in stats, stats - assert "Stage 2 map_batches:" in stats, stats + assert "Stage 1 read->DummyPreprocessor:" in stats, stats + assert "Stage 2 MapBatches(ScoringWrapper):" in stats, stats assert ds.max("value") == 36.0, ds ds = batch_predictor.predict( @@ -128,7 +128,7 @@ def test_separate_gpu_stage(shutdown_only): ) stats = ds.stats() assert "Stage 1 read:" in stats, stats - assert "Stage 2 map_batches:" in stats, stats + assert "Stage 2 MapBatches(ScoringWrapper):" in stats, stats assert ds.max("value") == 36.0, ds @@ -160,7 +160,7 @@ def test_batch_prediction(): test_dataset = ray.data.range_table(4) ds = batch_predictor.predict(test_dataset) # Check fusion occurred. - assert "read->map_batches" in ds.stats(), ds.stats() + assert "read->DummyPreprocessor" in ds.stats(), ds.stats() assert ds.to_pandas().to_numpy().squeeze().tolist() == [ 0.0, 4.0, @@ -278,7 +278,7 @@ def test_batch_prediction_various_combination(): ds = batch_predictor.predict(input_dataset) print(ds.stats()) # Check no fusion needed since we're not doing a dataset read. - assert "Stage 1 map_batches" in ds.stats(), ds.stats() + assert f"Stage 1 {preprocessor.__class__.__name__}" in ds.stats(), ds.stats() assert ds.to_pandas().to_numpy().squeeze().tolist() == [ 4.0, 8.0, @@ -544,8 +544,8 @@ def test_separate_gpu_stage_pipelined(shutdown_only): ) out = [x["value"] for x in ds.iter_rows()] stats = ds.stats() - assert "Stage 1 read->map_batches:" in stats, stats - assert "Stage 2 map_batches:" in stats, stats + assert "Stage 1 read->DummyPreprocessor:" in stats, stats + assert "Stage 2 MapBatches(ScoringWrapper):" in stats, stats assert max(out) == 16.0, out ds = batch_predictor.predict_pipelined( @@ -558,7 +558,7 @@ def test_separate_gpu_stage_pipelined(shutdown_only): out = [x["value"] for x in ds.iter_rows()] stats = ds.stats() assert "Stage 1 read:" in stats, stats - assert "Stage 2 map_batches:" in stats, stats + assert "Stage 2 MapBatches(ScoringWrapper):" in stats, stats assert max(out) == 16.0, out From 7b2299be8e69f4abf96b5cbb076c035b0a285792 Mon Sep 17 00:00:00 2001 From: andreapiso <14676006+andreapiso@users.noreply.github.com> Date: Fri, 27 Jan 2023 22:53:31 +0800 Subject: [PATCH 019/267] Enable Log Rotation on Serve (#31844) This PR adds log rotation for Ray Serve, letting it inherit rotation parameters (max_bytes, backup_count) from Ray Core, bringing a more consistent logging experience to Ray (as opposed to having the serve/ folder grow forever while the other logs rotate. --- doc/source/ray-observability/ray-logging.rst | 3 ++ .../serve/production-guide/monitoring.md | 2 ++ python/ray/serve/_private/logging_utils.py | 12 ++++++- python/ray/serve/tests/test_logging.py | 32 +++++++++++++++++++ 4 files changed, 48 insertions(+), 1 deletion(-) diff --git a/doc/source/ray-observability/ray-logging.rst b/doc/source/ray-observability/ray-logging.rst index b4220fef0bc5..692fbeb479e0 100644 --- a/doc/source/ray-observability/ray-logging.rst +++ b/doc/source/ray-observability/ray-logging.rst @@ -173,8 +173,11 @@ Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/std - ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. - ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver.- +.. _ray-log-rotation: + Log rotation ------------ + Ray supports log rotation of log files. Note that not all components are currently supporting log rotation. (Raylet and Python/Java worker logs are not rotating). By default, logs are rotating when it reaches to 512MB (maxBytes), and there could be up to 5 backup files (backupCount). Indexes are appended to all backup files (e.g., `raylet.out.1`) diff --git a/doc/source/serve/production-guide/monitoring.md b/doc/source/serve/production-guide/monitoring.md index 47b96604be4f..d9d913a9fd18 100644 --- a/doc/source/serve/production-guide/monitoring.md +++ b/doc/source/serve/production-guide/monitoring.md @@ -99,6 +99,8 @@ This causes the HTTP proxy and deployment replica to print log statements to the A copy of these logs are stored at `/tmp/ray/session_latest/logs/serve/`. You can parse these stored logs with a logging stack such as ELK or [Loki](serve-logging-loki) to search them by deployment or replica. +Serve supports [Log Rotation](ray-log-rotation) of these logs through setting the environment variables `RAY_ROTATION_MAX_BYTES` and `RAY_ROTATION_BACKUP_COUNT`. + To silence the replica-level logs or otherwise configure logging, configure the `"ray.serve"` logger **inside the deployment constructor**: ```python diff --git a/python/ray/serve/_private/logging_utils.py b/python/ray/serve/_private/logging_utils.py index 1d76be1057f5..11604f8e3de8 100644 --- a/python/ray/serve/_private/logging_utils.py +++ b/python/ray/serve/_private/logging_utils.py @@ -22,6 +22,8 @@ def configure_component_logger( log_level: int = logging.INFO, log_to_stream: bool = True, log_to_file: bool = True, + max_bytes: Optional[int] = None, + backup_count: Optional[int] = None, ): """Returns a logger to be used by a Serve component. @@ -51,12 +53,20 @@ def configure_component_logger( ray._private.worker._global_node.get_logs_dir_path(), "serve" ) os.makedirs(logs_dir, exist_ok=True) + if max_bytes is None: + max_bytes = ray._private.worker._global_node.max_bytes + if backup_count is None: + backup_count = ray._private.worker._global_node.backup_count if component_type is not None: component_name = f"{component_type}_{component_name}" log_file_name = LOG_FILE_FMT.format( component_name=component_name, component_id=component_id ) - file_handler = logging.FileHandler(os.path.join(logs_dir, log_file_name)) + file_handler = logging.handlers.RotatingFileHandler( + os.path.join(logs_dir, log_file_name), + maxBytes=max_bytes, + backupCount=backup_count, + ) file_handler.setFormatter(formatter) logger.addHandler(file_handler) diff --git a/python/ray/serve/tests/test_logging.py b/python/ray/serve/tests/test_logging.py index 47ef85a622b1..6b5b2c37c21b 100644 --- a/python/ray/serve/tests/test_logging.py +++ b/python/ray/serve/tests/test_logging.py @@ -12,6 +12,38 @@ from ray._private.test_utils import wait_for_condition +def set_logging_config(monkeypatch, max_bytes, backup_count): + monkeypatch.setenv("RAY_ROTATION_MAX_BYTES", str(max_bytes)) + monkeypatch.setenv("RAY_ROTATION_BACKUP_COUNT", str(backup_count)) + + +def test_log_rotation_config(monkeypatch, ray_shutdown): + # This test should be executed before any test that uses + # the shared serve_instance, as environment variables + # for log rotation need to be set before ray.init + logger = logging.getLogger("ray.serve") + max_bytes = 100 + backup_count = 3 + set_logging_config(monkeypatch, max_bytes, backup_count) + ray.init(num_cpus=1) + + @serve.deployment + class Handle: + def __call__(self): + handlers = logger.handlers + res = {} + for handler in handlers: + if isinstance(handler, logging.handlers.RotatingFileHandler): + res["max_bytes"] = handler.maxBytes + res["backup_count"] = handler.backupCount + return res + + handle = serve.run(Handle.bind()) + rotation_config = ray.get(handle.remote()) + assert rotation_config["max_bytes"] == max_bytes + assert rotation_config["backup_count"] == backup_count + + def test_handle_access_log(serve_instance): name = "handler" From ed72ca8f10a09c9934a51fc3579d11c64fda58b4 Mon Sep 17 00:00:00 2001 From: Ricky Xu Date: Fri, 27 Jan 2023 09:55:43 -0500 Subject: [PATCH 020/267] [core][state] Handle driver tasks (#31832) This PR adds additional information to the driver task event, namely, driver task type, and it's running/finished timestamps. This allows users (i.e. the dashboard) to inspect driver task more easily. This PR also exposes the exclude_driver flag to state API, allowing requests through https and ListAPiOptions to get driver tasks, while the default behaviour from state API will still be excluding it. This PR also filters out any tasks w/o task_info to prevent missing data issue. --- dashboard/modules/state/state_head.py | 9 ++- dashboard/state_aggregator.py | 8 +- python/ray/experimental/state/api.py | 2 +- python/ray/experimental/state/common.py | 3 + .../ray/experimental/state/state_manager.py | 8 +- python/ray/tests/test_state_api.py | 4 +- python/ray/tests/test_task_events.py | 81 ++++++++++++++++--- src/ray/core_worker/core_worker.cc | 31 ++++++- src/ray/core_worker/task_manager.cc | 2 + src/ray/gcs/gcs_server/gcs_task_manager.cc | 9 ++- .../gcs_server/test/gcs_task_manager_test.cc | 51 +++++++++++- src/ray/protobuf/gcs_service.proto | 2 +- 12 files changed, 185 insertions(+), 25 deletions(-) diff --git a/dashboard/modules/state/state_head.py b/dashboard/modules/state/state_head.py index 3358f03da337..b2e755b35e0d 100644 --- a/dashboard/modules/state/state_head.py +++ b/dashboard/modules/state/state_head.py @@ -192,9 +192,16 @@ def _options_from_req(self, req: aiohttp.web.Request) -> ListApiOptions: timeout = int(req.query.get("timeout", 30)) filters = self._get_filters_from_req(req) detail = convert_string_to_type(req.query.get("detail", False), bool) + exclude_driver = convert_string_to_type( + req.query.get("exclude_driver", True), bool + ) return ListApiOptions( - limit=limit, timeout=timeout, filters=filters, detail=detail + limit=limit, + timeout=timeout, + filters=filters, + detail=detail, + exclude_driver=exclude_driver, ) def _summary_options_from_req(self, req: aiohttp.web.Request) -> SummaryApiOptions: diff --git a/dashboard/state_aggregator.py b/dashboard/state_aggregator.py index ebec09d7b917..c75b5d5385db 100644 --- a/dashboard/state_aggregator.py +++ b/dashboard/state_aggregator.py @@ -373,13 +373,15 @@ async def list_tasks(self, *, option: ListApiOptions) -> ListApiResponse: """ job_id = None for filter in option.filters: - if filter[0] == "job_id": + if filter[0] == "job_id" and filter[1] == "=": + # Filtering by job_id == xxxx, pass it to source side filtering. # tuple consists of (job_id, predicate, value) job_id = filter[2] - try: reply = await self._client.get_all_task_info( - timeout=option.timeout, job_id=job_id + timeout=option.timeout, + job_id=job_id, + exclude_driver=option.exclude_driver, ) except DataSourceUnavailable: raise DataSourceUnavailable(GCS_QUERY_FAILURE_WARNING) diff --git a/python/ray/experimental/state/api.py b/python/ray/experimental/state/api.py index 4512e8bda780..daefed8b423b 100644 --- a/python/ray/experimental/state/api.py +++ b/python/ray/experimental/state/api.py @@ -174,7 +174,7 @@ def _make_param(cls, options: Union[ListApiOptions, GetApiOptions]) -> Dict: continue option_val = getattr(options, field.name) - if option_val: + if option_val is not None: options_dict[field.name] = option_val return options_dict diff --git a/python/ray/experimental/state/common.py b/python/ray/experimental/state/common.py index 664041240aca..6e997820857c 100644 --- a/python/ray/experimental/state/common.py +++ b/python/ray/experimental/state/common.py @@ -88,6 +88,8 @@ class ListApiOptions: filters: Optional[List[Tuple[str, PredicateType, SupportedFilterType]]] = field( default_factory=list ) + # [only tasks] If driver tasks should be excluded. + exclude_driver: bool = True # When the request is processed on the server side, # we should apply multiplier so that server side can finish # processing a request within timeout. Otherwise, @@ -99,6 +101,7 @@ def __post_init__(self): assert isinstance(self.limit, int) assert isinstance(self.timeout, int) assert isinstance(self.detail, bool) + assert isinstance(self.exclude_driver, bool) assert isinstance(self.filters, list) or self.filters is None, ( "filters must be a list type. Given filters: " f"{self.filters} type: {type(self.filters)}. " diff --git a/python/ray/experimental/state/state_manager.py b/python/ray/experimental/state/state_manager.py index c82c0211f311..70d22b8fa070 100644 --- a/python/ray/experimental/state/state_manager.py +++ b/python/ray/experimental/state/state_manager.py @@ -234,14 +234,18 @@ async def get_all_actor_info( @handle_grpc_network_errors async def get_all_task_info( - self, timeout: int = None, limit: int = None, job_id: Optional[str] = None + self, + timeout: int = None, + limit: int = None, + job_id: Optional[str] = None, + exclude_driver: bool = True, ) -> Optional[GetTaskEventsReply]: if not limit: limit = RAY_MAX_LIMIT_FROM_DATA_SOURCE if job_id: job_id = JobID(hex_to_binary(job_id)).binary() request = GetTaskEventsRequest( - limit=limit, exclude_driver_task=True, job_id=job_id + limit=limit, exclude_driver=exclude_driver, job_id=job_id ) reply = await self._gcs_task_info_stub.GetTaskEvents(request, timeout=timeout) return reply diff --git a/python/ray/tests/test_state_api.py b/python/ray/tests/test_state_api.py index 3859eb11f43a..c37f4b0fc00c 100644 --- a/python/ray/tests/test_state_api.py +++ b/python/ray/tests/test_state_api.py @@ -283,6 +283,7 @@ def create_api_options( limit: int = DEFAULT_LIMIT, filters: List[Tuple[str, SupportedFilterType]] = None, detail: bool = False, + exclude_driver: bool = True, ): if not filters: filters = [] @@ -292,6 +293,7 @@ def create_api_options( filters=filters, _server_timeout_multiplier=1.0, detail=detail, + exclude_driver=exclude_driver, ) @@ -804,7 +806,7 @@ async def test_api_manager_list_tasks(state_api_manager): ] result = await state_api_manager.list_tasks(option=create_api_options()) data_source_client.get_all_task_info.assert_any_await( - timeout=DEFAULT_RPC_TIMEOUT, job_id=None + timeout=DEFAULT_RPC_TIMEOUT, job_id=None, exclude_driver=True ) data = result.result data = data diff --git a/python/ray/tests/test_task_events.py b/python/ray/tests/test_task_events.py index 87d16772a363..425f1f5931cc 100644 --- a/python/ray/tests/test_task_events.py +++ b/python/ray/tests/test_task_events.py @@ -4,12 +4,13 @@ import time import ray +from ray.experimental.state.common import ListApiOptions, StateResource from ray._private.test_utils import ( raw_metrics, run_string_as_driver_nonblocking, wait_for_condition, ) -from ray.experimental.state.api import list_tasks +from ray.experimental.state.api import StateApiClient, list_tasks from ray._private.worker import RayContext @@ -68,7 +69,7 @@ def verify(): "At least 10 tasks events should be reported. " "Could be more than 10 with multiple flush." ) - assert metric["STORED"] == 10, "10 task's events should be stored." + assert metric["STORED"] == 11, "10 task + 1 driver's events should be stored." return True @@ -123,6 +124,66 @@ def verify(): ) +def test_handle_driver_tasks(shutdown_only): + ray.init(_system_config=_SYSTEM_CONFIG) + + job_id = ray.get_runtime_context().get_job_id() + script = """ +import ray +import time +ray.init("auto") + +@ray.remote +def f(): + time.sleep(3) + + +ray.get(f.remote()) +""" + run_string_as_driver_nonblocking(script) + + client = StateApiClient() + + def list_tasks(exclude_driver): + return client.list( + StateResource.TASKS, + # Filter out this driver + options=ListApiOptions( + exclude_driver=exclude_driver, filters=[("job_id", "!=", job_id)] + ), + raise_on_missing_output=True, + ) + + # Check driver running + def verify(): + tasks_with_driver = list_tasks(exclude_driver=False) + assert len(tasks_with_driver) == 2, tasks_with_driver + task_types = {task["type"] for task in tasks_with_driver} + assert task_types == {"NORMAL_TASK", "DRIVER_TASK"} + + for task in tasks_with_driver: + if task["type"] == "DRIVER_TASK": + assert task["state"] == "RUNNING", task + + return True + + wait_for_condition(verify, timeout=15, retry_interval_ms=1000) + + # Check driver finishes + def verify(): + tasks_with_driver = list_tasks(exclude_driver=False) + assert len(tasks_with_driver) == 2, tasks_with_driver + for task in tasks_with_driver: + if task["type"] == "DRIVER_TASK": + assert task["state"] == "FINISHED", task + + tasks_no_driver = list_tasks(exclude_driver=True) + assert len(tasks_no_driver) == 1, tasks_no_driver + return True + + wait_for_condition(verify) + + def test_fault_tolerance_job_failed(shutdown_only): ray.init(num_cpus=8, _system_config=_SYSTEM_CONFIG) script = """ @@ -184,11 +245,11 @@ def verify(): for task in tasks: if "finished" in task["func_or_class_name"]: assert ( - task["scheduling_state"] == "FINISHED" + task["state"] == "FINISHED" ), f"task {task['func_or_class_name']} has wrong state" else: assert ( - task["scheduling_state"] == "FAILED" + task["state"] == "FAILED" ), f"task {task['func_or_class_name']} has wrong state" return True @@ -247,9 +308,9 @@ def verify(): ), "1 creation task + 1 actor tasks + 2 normal tasks run by the actor tasks" for task in tasks: if "finish" in task["name"] or "__init__" in task["name"]: - assert task["scheduling_state"] == "FINISHED", task + assert task["state"] == "FINISHED", task else: - assert task["scheduling_state"] == "FAILED", task + assert task["state"] == "FAILED", task return True @@ -276,9 +337,9 @@ def verify(): ) for task in tasks: if "finish" in task["name"] or "__init__" in task["name"]: - assert task["scheduling_state"] == "FINISHED", task + assert task["state"] == "FINISHED", task else: - assert task["scheduling_state"] == "FAILED", task + assert task["state"] == "FAILED", task return True @@ -405,9 +466,9 @@ def add_death_tasks_recur(task, execution_graph, dead_tasks): for task in target_tasks: if task["name"] in dead_tasks: - assert task["scheduling_state"] == "FAILED", task["name"] + assert task["state"] == "FAILED", task["name"] else: - assert task["scheduling_state"] == "RUNNING", task["name"] + assert task["state"] == "RUNNING", task["name"] return True diff --git a/src/ray/core_worker/core_worker.cc b/src/ray/core_worker/core_worker.cc index 61a10cc5f847..c3ea072b52db 100644 --- a/src/ray/core_worker/core_worker.cc +++ b/src/ray/core_worker/core_worker.cc @@ -360,11 +360,27 @@ CoreWorker::CoreWorker(const CoreWorkerOptions &options, const WorkerID &worker_ builder.SetDriverTaskSpec(task_id, options_.language, worker_context_.GetCurrentJobID(), - TaskID::ComputeDriverTaskId(worker_context_.GetWorkerID()), + // Driver has no parent task + /* parent_task_id */ TaskID::Nil(), GetCallerId(), rpc_address_); // Drivers are never re-executed. SetCurrentTaskId(task_id, /*attempt_number=*/0, "driver"); + + // Add the driver task info. + if (task_event_buffer_->Enabled()) { + rpc::TaskEvents task_event; + const auto spec = builder.Build(); + auto task_info = task_manager_->MakeTaskInfoEntry(spec); + task_event.set_task_id(task_id.Binary()); + task_event.set_job_id(spec.JobId().Binary()); + task_event.set_attempt_number(0); + task_event.mutable_task_info()->Swap(&task_info); + gcs::FillTaskStatusUpdateTime(rpc::TaskStatus::RUNNING, + absl::GetCurrentTimeNanos(), + task_event.mutable_state_updates()); + task_event_buffer_->AddTaskEvent(std::move(task_event)); + } } auto raylet_client_factory = [this](const std::string ip_address, int port) { @@ -643,6 +659,19 @@ void CoreWorker::Disconnect( // Force stats export before exiting the worker. RecordMetrics(); + // Driver exiting. + if (options_.worker_type == WorkerType::DRIVER && task_event_buffer_->Enabled()) { + // Mark Driver as finished. + rpc::TaskEvents task_event; + task_event.set_task_id(worker_context_.GetCurrentTaskID().Binary()); + task_event.set_job_id(worker_context_.GetCurrentJobID().Binary()); + task_event.set_attempt_number(0); + gcs::FillTaskStatusUpdateTime(rpc::TaskStatus::FINISHED, + absl::GetCurrentTimeNanos(), + task_event.mutable_state_updates()); + task_event_buffer_->AddTaskEvent(std::move(task_event)); + } + // Force task state events push before exiting the worker. task_event_buffer_->FlushEvents(/* forced */ true); diff --git a/src/ray/core_worker/task_manager.cc b/src/ray/core_worker/task_manager.cc index a3839592c9f5..3cf71e383d19 100644 --- a/src/ray/core_worker/task_manager.cc +++ b/src/ray/core_worker/task_manager.cc @@ -842,6 +842,8 @@ rpc::TaskInfoEntry TaskManager::MakeTaskInfoEntry( rpc::TaskType type; if (task_spec.IsNormalTask()) { type = rpc::TaskType::NORMAL_TASK; + } else if (task_spec.IsDriverTask()) { + type = rpc::TaskType::DRIVER_TASK; } else if (task_spec.IsActorCreationTask()) { type = rpc::TaskType::ACTOR_CREATION_TASK; task_info.set_actor_id(task_spec.ActorCreationId().Binary()); diff --git a/src/ray/gcs/gcs_server/gcs_task_manager.cc b/src/ray/gcs/gcs_server/gcs_task_manager.cc index 68f75cd918a3..7810f2ff0b50 100644 --- a/src/ray/gcs/gcs_server/gcs_task_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_task_manager.cc @@ -367,8 +367,13 @@ void GcsTaskManager::HandleGetTaskEvents(rpc::GetTaskEventsRequest request, for (auto itr = task_events.rbegin(); itr != task_events.rend(); ++itr) { auto &task_event = *itr; - if (request.exclude_driver_task() && !task_event.has_state_updates()) { - // Driver related profile events will generate TaskEvent w/o any task state updates. + if (!task_event.has_task_info()) { + // Skip task events w/o task info. + continue; + } + + if (request.exclude_driver() && + task_event.task_info().type() == rpc::TaskType::DRIVER_TASK) { continue; } diff --git a/src/ray/gcs/gcs_server/test/gcs_task_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_task_manager_test.cc index bf192b2ec609..4932794d521b 100644 --- a/src/ray/gcs/gcs_server/test/gcs_task_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_task_manager_test.cc @@ -99,7 +99,8 @@ class GcsTaskManagerTest : public ::testing::Test { rpc::GetTaskEventsReply SyncGetTaskEvents(absl::flat_hash_set task_ids, absl::optional job_id = absl::nullopt, - int64_t limit = -1) { + int64_t limit = -1, + bool exclude_driver = true) { rpc::GetTaskEventsRequest request; rpc::GetTaskEventsReply reply; std::promise promise; @@ -118,6 +119,8 @@ class GcsTaskManagerTest : public ::testing::Test { request.set_limit(limit); } + request.set_exclude_driver(exclude_driver); + task_manager->HandleGetTaskEvents( request, &reply, @@ -131,11 +134,14 @@ class GcsTaskManagerTest : public ::testing::Test { return reply; } - static rpc::TaskInfoEntry GenTaskInfo(JobID job_id, - TaskID parent_task_id = TaskID::Nil()) { + static rpc::TaskInfoEntry GenTaskInfo( + JobID job_id, + TaskID parent_task_id = TaskID::Nil(), + rpc::TaskType task_type = rpc::TaskType::NORMAL_TASK) { rpc::TaskInfoEntry task_info; task_info.set_job_id(job_id.Binary()); task_info.set_parent_task_id(parent_task_id.Binary()); + task_info.set_type(task_type); return task_info; } @@ -188,6 +194,8 @@ class GcsTaskManagerTest : public ::testing::Test { if (task_info.has_value()) { events.mutable_task_info()->CopyFrom(*task_info); + } else { + events.mutable_task_info()->CopyFrom(GenTaskInfo(JobID::FromInt(job_id))); } result.push_back(events); @@ -732,6 +740,43 @@ TEST_F(GcsTaskManagerMemoryLimitedTest, TestLimitTaskEvents) { } } +TEST_F(GcsTaskManagerTest, TestGetTaskEventsWithDriver) { + // Add task events + auto task_ids = GenTaskIDs(1); + auto driver_task = task_ids[0]; + + // Add Driver + { + auto events = GenTaskEvents( + {driver_task}, + /* attempt_number */ 0, + /* job_id */ 0, + /* profile event */ absl::nullopt, + /* status_update*/ absl::nullopt, + GenTaskInfo( + /* job_id */ JobID::FromInt(0), TaskID::Nil(), rpc::TaskType::DRIVER_TASK)); + auto events_data = Mocker::GenTaskEventsData(events); + SyncAddTaskEventData(events_data); + } + + // Should get the event when including driver + { + auto reply = SyncGetTaskEvents(/* task_ids */ {}, + /* job_id */ absl::nullopt, + /* limit */ -1, + /* exclude_driver*/ false); + EXPECT_EQ(reply.events_by_task_size(), 1); + } + + // Default exclude driver + { + auto reply = SyncGetTaskEvents(/* task_ids */ {}, + /* job_id */ absl::nullopt, + /* limit */ -1); + EXPECT_EQ(reply.events_by_task_size(), 0); + } +} + TEST_F(GcsTaskManagerMemoryLimitedTest, TestLimitReturnRecentTasksWhenGetAll) { // Keep adding tasks and make sure even with eviction, the returned tasks are // the mo diff --git a/src/ray/protobuf/gcs_service.proto b/src/ray/protobuf/gcs_service.proto index 14bf644fed9d..2d7e214c6e60 100644 --- a/src/ray/protobuf/gcs_service.proto +++ b/src/ray/protobuf/gcs_service.proto @@ -611,7 +611,7 @@ message GetTaskEventsRequest { // guarantee. optional int64 limit = 3; // True if task events from driver (only profiling events) should be excluded. - bool exclude_driver_task = 4; + bool exclude_driver = 4; } message GetTaskEventsReply { From 3f1a880f7a4b248366ca00ea31483341d2eee1cf Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Fri, 27 Jan 2023 09:18:46 -0800 Subject: [PATCH 021/267] [serve] Add exponential backoff when retrying replicas (#31436) If deployment is repeatedly failing, perform exponential backoff so as to not repeatedly try to restart the replica at a very fast rate. Related issue number Closes #31121 --- python/ray/serve/_private/deployment_state.py | 81 ++++++++++++++----- .../ray/serve/tests/test_deployment_state.py | 51 ++++++++++++ 2 files changed, 111 insertions(+), 21 deletions(-) diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index 9db5d6256de9..002a8fa3ea46 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -114,6 +114,9 @@ def from_deployment_info( os.environ.get("SERVE_SLOW_STARTUP_WARNING_PERIOD_S", 30) ) +EXPONENTIAL_BACKOFF_FACTOR = float(os.environ.get("EXPONENTIAL_BACKOFF_FACTOR", 2.0)) +MAX_BACKOFF_TIME_S = int(os.environ.get("SERVE_MAX_BACKOFF_TIME_S", 64)) + ALL_REPLICA_STATES = list(ReplicaState) _SCALING_LOG_ENABLED = os.environ.get("SERVE_ENABLE_SCALING_LOG", "0") != "0" @@ -978,6 +981,9 @@ def __init__( # DeploymentInfo and bring current deployment to meet new status. self._target_state: DeploymentTargetState = DeploymentTargetState.default() self._prev_startup_warning: float = time.time() + # Exponential backoff when retrying a consistently failing deployment + self._last_retry: float = 0.0 + self._backoff_time_s: int = 1 self._replica_constructor_retry_counter: int = 0 self._replicas: ReplicaStateContainer = ReplicaStateContainer() self._curr_status_info: DeploymentStatusInfo = DeploymentStatusInfo( @@ -1104,6 +1110,7 @@ def _set_target_state(self, target_info: DeploymentInfo) -> None: self._name, DeploymentStatus.UPDATING ) self._replica_constructor_retry_counter = 0 + self._backoff_time_s = 1 logger.debug(f"Deploying new version of {self._name}: {target_state.version}.") @@ -1308,28 +1315,43 @@ def _scale_deployment_replicas(self) -> bool: ) to_add = max(delta_replicas - stopping_replicas, 0) if to_add > 0: + # Exponential backoff + failed_to_start_threshold = min( + MAX_DEPLOYMENT_CONSTRUCTOR_RETRY_COUNT, + self._target_state.num_replicas * 3, + ) + if self._replica_constructor_retry_counter >= failed_to_start_threshold: + # Wait 1, 2, 4, ... seconds before consecutive retries, with random + # offset added to avoid synchronization + if ( + time.time() - self._last_retry + < self._backoff_time_s + random.uniform(0, 3) + ): + return replicas_stopped + + self._last_retry = time.time() logger.info( f"Adding {to_add} replica{'s' if to_add > 1 else ''} " f"to deployment '{self._name}'." ) - for _ in range(to_add): - replica_name = ReplicaName(self._name, get_random_letters()) - new_deployment_replica = DeploymentReplica( - self._controller_name, - self._detached, - replica_name.replica_tag, - replica_name.deployment_tag, - self._target_state.version, - ) - new_deployment_replica.start( - self._target_state.info, self._target_state.version - ) + for _ in range(to_add): + replica_name = ReplicaName(self._name, get_random_letters()) + new_deployment_replica = DeploymentReplica( + self._controller_name, + self._detached, + replica_name.replica_tag, + replica_name.deployment_tag, + self._target_state.version, + ) + new_deployment_replica.start( + self._target_state.info, self._target_state.version + ) - self._replicas.add(ReplicaState.STARTING, new_deployment_replica) - logger.debug( - "Adding STARTING to replica_tag: " - f"{replica_name}, deployment: {self._name}" - ) + self._replicas.add(ReplicaState.STARTING, new_deployment_replica) + logger.debug( + "Adding STARTING to replica_tag: " + f"{replica_name}, deployment: {self._name}" + ) elif delta_replicas < 0: replicas_stopped = True @@ -1407,10 +1429,10 @@ def _check_curr_status(self) -> bool: name=self._name, status=DeploymentStatus.UNHEALTHY, message=( - f"The Deployment failed to start {failed_to_start_count} " - "times in a row. This may be due to a problem with the " - "deployment constructor or the initial health check failing. " - "See logs for details." + f"The Deployment failed to start {failed_to_start_count} times " + "in a row. This may be due to a problem with the deployment " + "constructor or the initial health check failing. See logs for " + f"details. Retrying after {self._backoff_time_s} seconds." ), ) return False @@ -1453,6 +1475,7 @@ def _check_startup_replicas( """ slow_replicas = [] transitioned_to_running = False + replicas_failed = False for replica in self._replicas.pop(states=[original_state]): start_status = replica.check_started() if start_status == ReplicaStartupStatus.SUCCEEDED: @@ -1466,6 +1489,7 @@ def _check_startup_replicas( # Increase startup failure counter if we're tracking it self._replica_constructor_retry_counter += 1 + replicas_failed = True replica.stop(graceful=False) self._replicas.add(ReplicaState.STOPPING, replica) elif start_status in [ @@ -1485,6 +1509,21 @@ def _check_startup_replicas( else: self._replicas.add(original_state, replica) + # If replicas have failed enough times, execute exponential backoff + # Wait 1, 2, 4, ... seconds before consecutive retries (or use a custom + # backoff factor by setting EXPONENTIAL_BACKOFF_FACTOR) + failed_to_start_threshold = min( + MAX_DEPLOYMENT_CONSTRUCTOR_RETRY_COUNT, + self._target_state.num_replicas * 3, + ) + if ( + replicas_failed + and self._replica_constructor_retry_counter > failed_to_start_threshold + ): + self._backoff_time_s = min( + EXPONENTIAL_BACKOFF_FACTOR * self._backoff_time_s, MAX_BACKOFF_TIME_S + ) + return slow_replicas, transitioned_to_running def _check_and_update_replicas(self) -> bool: diff --git a/python/ray/serve/tests/test_deployment_state.py b/python/ray/serve/tests/test_deployment_state.py index 1ce865997d96..5113a0761b7c 100644 --- a/python/ray/serve/tests/test_deployment_state.py +++ b/python/ray/serve/tests/test_deployment_state.py @@ -1991,6 +1991,57 @@ def test_deploy_with_transient_constructor_failure( assert deployment_state.curr_status_info.status == DeploymentStatus.HEALTHY +@pytest.mark.parametrize("mock_deployment_state", [False], indirect=True) +@patch.object(DriverDeploymentState, "_get_all_node_ids") +def test_exponential_backoff(mock_get_all_node_ids, mock_deployment_state): + """Test exponential backoff.""" + deployment_state, timer = mock_deployment_state + mock_get_all_node_ids.return_value = [(str(i), str(i)) for i in range(2)] + + b_info_1, b_version_1 = deployment_info(num_replicas=2) + updating = deployment_state.deploy(b_info_1) + assert updating + assert deployment_state.curr_status_info.status == DeploymentStatus.UPDATING + + _constructor_failure_loop_two_replica(deployment_state, 3) + assert deployment_state._replica_constructor_retry_counter == 6 + last_retry = timer.time() + + for i in range(7): + while timer.time() - last_retry < 2**i: + deployment_state.update() + assert deployment_state._replica_constructor_retry_counter == 6 + 2 * i + # Check that during backoff time, no replicas are created + check_counts(deployment_state, total=0) + timer.advance(0.1) # simulate time passing between each call to udpate + + # Skip past random additional backoff time used to avoid synchronization + timer.advance(5) + + # Set new replicas to fail consecutively + check_counts(deployment_state, total=0) # No replicas + deployment_state.update() + last_retry = timer.time() # This should be time at which replicas were retried + check_counts(deployment_state, total=2) # Two new replicas + replica_1 = deployment_state._replicas.get()[0] + replica_2 = deployment_state._replicas.get()[1] + replica_1._actor.set_failed_to_start() + replica_2._actor.set_failed_to_start() + timer.advance(0.1) # simulate time passing between each call to udpate + + # Now the replica should be marked STOPPING after failure. + deployment_state.update() + check_counts(deployment_state, total=2, by_state=[(ReplicaState.STOPPING, 2)]) + timer.advance(0.1) # simulate time passing between each call to udpate + + # Once it's done stopping, replica should be removed. + replica_1._actor.set_done_stopping() + replica_2._actor.set_done_stopping() + deployment_state.update() + check_counts(deployment_state, total=0) + timer.advance(0.1) # simulate time passing between each call to udpate + + @pytest.fixture def mock_deployment_state_manager(request) -> Tuple[DeploymentStateManager, Mock]: ray.init() From 76d746786bf335a7f093bbab71e7e48ee16fccd3 Mon Sep 17 00:00:00 2001 From: kourosh hakhamaneshi <31483498+kouroshHakha@users.noreply.github.com> Date: Fri, 27 Jan 2023 09:24:16 -0800 Subject: [PATCH 022/267] [RLlib] Fixed the autorom dependency issue (#31933) Co-authored-by: Cade Daniel Closes https://github.com/ray-project/ray/issues/31880 --- release/rllib_tests/app_config.yaml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/release/rllib_tests/app_config.yaml b/release/rllib_tests/app_config.yaml index b66417df9249..2dae5b40912c 100755 --- a/release/rllib_tests/app_config.yaml +++ b/release/rllib_tests/app_config.yaml @@ -19,7 +19,10 @@ python: - ale-py==0.8.0 - gym==0.26.2 - mujoco-py<2.2,>=2.1 - - autorom[accept-rom-license] + # AutoROM downloads ROMs via torrent when they are built. The torrent is unreliable, + # so we built it for py3 and use that instead. This wheel was tested for python 3.7, 3.8, + # and 3.9. + - https://ray-ci-deps-wheels.s3.us-west-2.amazonaws.com/AutoROM.accept_rom_license-0.5.4-py3-none-any.whl conda_packages: [] post_build_cmds: From 15af4857c30eaaf4233ffd5934af9dbd30a4b720 Mon Sep 17 00:00:00 2001 From: Alan Guo Date: Fri, 27 Jan 2023 09:57:53 -0800 Subject: [PATCH 023/267] Polish the Dashboard new IA part 2 (#31946) Adds back the metrics page Adds button to visit new dashboard and to go back Adds buttons for leaving feedback and viewing docs Add color to status badges of tasks and placement groups table Add alert when grafana is not running Fix copy button icon Separate metrics page into sections (both new IA and old IA) --- dashboard/client/src/App.tsx | 12 +- .../client/src/components/EventTable.tsx | 7 +- .../MetadataSection/MetadataSection.tsx | 4 +- .../src/components/PlacementGroupTable.tsx | 2 +- .../client/src/components/StatusChip.tsx | 41 ++- dashboard/client/src/components/TaskTable.tsx | 2 +- dashboard/client/src/pages/actor/index.tsx | 8 + dashboard/client/src/pages/job/JobDetail.tsx | 55 +++- dashboard/client/src/pages/job/JobRow.tsx | 49 +-- .../client/src/pages/layout/MainNavLayout.tsx | 75 ++++- dashboard/client/src/pages/layout/index.tsx | 17 + .../pages/metrics/Metrics.component.test.tsx | 70 +++++ .../client/src/pages/metrics/Metrics.tsx | 296 ++++++++++++------ dashboard/client/src/pages/metrics/utils.ts | 36 ++- .../overview/OverviewPage.component.test.tsx | 1 - .../src/pages/overview/OverviewPage.tsx | 11 - .../overview/cards/ClusterUtilizationCard.tsx | 45 ++- .../pages/overview/cards/NodeCountCard.tsx | 35 ++- ...{placementGroup.d.ts => placementGroup.ts} | 0 dashboard/modules/metrics/metrics_head.py | 2 +- 20 files changed, 571 insertions(+), 197 deletions(-) create mode 100644 dashboard/client/src/pages/metrics/Metrics.component.test.tsx rename dashboard/client/src/type/{placementGroup.d.ts => placementGroup.ts} (100%) diff --git a/dashboard/client/src/App.tsx b/dashboard/client/src/App.tsx index 302a9afe3985..b45d1f0db50f 100644 --- a/dashboard/client/src/App.tsx +++ b/dashboard/client/src/App.tsx @@ -52,6 +52,10 @@ type GlobalContextType = { * running as detected by the grafana healthcheck endpoint. */ grafanaHost: string | undefined; + /** + * Whether prometheus is runing or not + */ + prometheusHealth: boolean | undefined; /** * The name of the currently running ray session. */ @@ -63,6 +67,7 @@ export const GlobalContext = React.createContext({ ipLogMap: {}, namespaceMap: {}, grafanaHost: undefined, + prometheusHealth: undefined, sessionName: undefined, }); @@ -79,6 +84,7 @@ const App = () => { ipLogMap: {}, namespaceMap: {}, grafanaHost: undefined, + prometheusHealth: undefined, sessionName: undefined, }); const getTheme = (name: string) => { @@ -119,11 +125,13 @@ const App = () => { // Detect if grafana is running useEffect(() => { const doEffect = async () => { - const { grafanaHost, sessionName } = await getMetricsInfo(); + const { grafanaHost, sessionName, prometheusHealth } = + await getMetricsInfo(); setContext((existingContext) => ({ ...existingContext, grafanaHost, sessionName, + prometheusHealth, })); }; doEffect(); @@ -221,6 +229,8 @@ const App = () => { /> + } path="actors" /> + } path="metrics" /> } path="logs"> {/* TODO(aguo): Refactor Logs component to use optional query params since react-router 6 doesn't support optional path params... */} diff --git a/dashboard/client/src/components/EventTable.tsx b/dashboard/client/src/components/EventTable.tsx index c4d9cfec7a17..174a94e9f5a9 100644 --- a/dashboard/client/src/components/EventTable.tsx +++ b/dashboard/client/src/components/EventTable.tsx @@ -35,11 +35,16 @@ const useStyles = makeStyles((theme) => ({ padding: theme.spacing(2), marginTop: theme.spacing(2), }, + filterContainer: { + display: "flex", + alignItems: "center", + }, search: { margin: theme.spacing(1), display: "inline-block", fontSize: 12, lineHeight: "46px", + height: 56, }, infokv: { margin: theme.spacing(1), @@ -166,7 +171,7 @@ const EventTable = (props: EventTableProps) => { return (
-
+
- + )} diff --git a/dashboard/client/src/components/PlacementGroupTable.tsx b/dashboard/client/src/components/PlacementGroupTable.tsx index f3768034b477..dd2ac7084e98 100644 --- a/dashboard/client/src/components/PlacementGroupTable.tsx +++ b/dashboard/client/src/components/PlacementGroupTable.tsx @@ -148,7 +148,7 @@ const PlacementGroupTable = ({ {name ? name : "-"} {creator_job_id} - + {stats ? stats.scheduling_state : "-"} diff --git a/dashboard/client/src/components/StatusChip.tsx b/dashboard/client/src/components/StatusChip.tsx index 7c57c0d724b1..c945c3a6cbd9 100644 --- a/dashboard/client/src/components/StatusChip.tsx +++ b/dashboard/client/src/components/StatusChip.tsx @@ -11,6 +11,8 @@ import { import { CSSProperties } from "@material-ui/core/styles/withStyles"; import React, { ReactNode } from "react"; import { ActorEnum } from "../type/actor"; +import { PlacementGroupState } from "../type/placementGroup"; +import { TypeTaskStatus } from "../type/task"; const colorMap = { node: { @@ -26,18 +28,37 @@ const colorMap = { [ActorEnum.PENDING]: blue, [ActorEnum.RECONSTRUCTING]: lightBlue, }, + task: { + [TypeTaskStatus.FAILED]: red, + [TypeTaskStatus.FINISHED]: green, + [TypeTaskStatus.RUNNING]: blue, + [TypeTaskStatus.RUNNING_IN_RAY_GET]: blue, + [TypeTaskStatus.RUNNING_IN_RAY_WAIT]: blue, + [TypeTaskStatus.SUBMITTED_TO_WORKER]: "#cfcf08", + [TypeTaskStatus.PENDING_ARGS_FETCH]: blue, + [TypeTaskStatus.PENDING_OBJ_STORE_MEM_AVAIL]: blue, + [TypeTaskStatus.PENDING_NODE_ASSIGNMENT]: "#cfcf08", + [TypeTaskStatus.PENDING_ARGS_AVAIL]: "#f79e02", + }, job: { INIT: grey, - SUBMITTED: blue, + SUBMITTED: "#cfcf08", DISPATCHED: lightBlue, - RUNNING: green, - COMPLETED: cyan, - FINISHED: cyan, + RUNNING: blue, + COMPLETED: green, + SUCCEEDED: green, + FINISHED: green, FAILED: red, }, + placementGroup: { + [PlacementGroupState.PENDING]: "#f79e02", + [PlacementGroupState.CREATED]: blue, + [PlacementGroupState.REMOVED]: red, + [PlacementGroupState.RESCHEDULING]: "#cfcf08", + }, } as { [key: string]: { - [key: string]: Color; + [key: string]: Color | string; }; }; @@ -66,7 +87,7 @@ export const StatusChip = ({ margin: 2, } as CSSProperties; - let color = blueGrey as Color; + let color: Color | string = blueGrey; if (typeMap[type]) { color = typeMap[type]; @@ -78,10 +99,12 @@ export const StatusChip = ({ color = colorMap[type][status]; } - style.color = color[500]; - style.borderColor = color[500]; + const colorValue = typeof color === "string" ? color : color[500]; + + style.color = colorValue; + style.borderColor = colorValue; if (color !== blueGrey) { - style.backgroundColor = `${color[500]}20`; + style.backgroundColor = `${colorValue}20`; } return ( diff --git a/dashboard/client/src/components/TaskTable.tsx b/dashboard/client/src/components/TaskTable.tsx index c833cb1a2ae0..c4f59ac3c66a 100644 --- a/dashboard/client/src/components/TaskTable.tsx +++ b/dashboard/client/src/components/TaskTable.tsx @@ -170,7 +170,7 @@ const TaskTable = ({ {name ? name : "-"} {job_id} - + {start_time_ms && start_time_ms > 0 ? ( diff --git a/dashboard/client/src/pages/actor/index.tsx b/dashboard/client/src/pages/actor/index.tsx index 4cbe9c2afef3..7661539b518a 100644 --- a/dashboard/client/src/pages/actor/index.tsx +++ b/dashboard/client/src/pages/actor/index.tsx @@ -1,6 +1,7 @@ import { makeStyles } from "@material-ui/core"; import React from "react"; import TitleCard from "../../components/TitleCard"; +import { MainNavPageInfo } from "../layout/mainNavContext"; import ActorList from "./ActorList"; const useStyles = makeStyles((theme) => ({ @@ -18,6 +19,13 @@ const Actors = () => { return (
+ diff --git a/dashboard/client/src/pages/job/JobDetail.tsx b/dashboard/client/src/pages/job/JobDetail.tsx index 8f0b6cdf1c21..dbc408ff8f80 100644 --- a/dashboard/client/src/pages/job/JobDetail.tsx +++ b/dashboard/client/src/pages/job/JobDetail.tsx @@ -1,12 +1,15 @@ import { makeStyles } from "@material-ui/core"; import { Alert } from "@material-ui/lab"; import dayjs from "dayjs"; -import React from "react"; +import React, { useContext } from "react"; +import { Link } from "react-router-dom"; +import { GlobalContext } from "../../App"; import { DurationText } from "../../common/DurationText"; import Loading from "../../components/Loading"; import { MetadataSection } from "../../components/MetadataSection"; import { StatusChip } from "../../components/StatusChip"; import TitleCard from "../../components/TitleCard"; +import { UnifiedJob } from "../../type/job"; import ActorList from "../actor/ActorList"; import PlacementGroupList from "../state/PlacementGroup"; import TaskList from "../state/task"; @@ -163,6 +166,10 @@ export const JobDetailChartsPage = ({ : "-", }, }, + { + label: "Logs", + content: , + }, ]} /> @@ -177,3 +184,49 @@ export const JobDetailChartsPage = ({
); }; + +type JobLogsLinkProps = { + job: Pick< + UnifiedJob, + | "driver_agent_http_address" + | "driver_info" + | "job_id" + | "submission_id" + | "type" + >; + newIA?: boolean; +}; + +export const JobLogsLink = ({ + job: { driver_agent_http_address, driver_info, job_id, submission_id, type }, + newIA = false, +}: JobLogsLinkProps) => { + const { ipLogMap } = useContext(GlobalContext); + + let link: string | undefined; + + const baseLink = newIA ? "/new/logs" : "/log"; + + if (driver_agent_http_address) { + link = `${baseLink}/${encodeURIComponent( + `${driver_agent_http_address}/logs`, + )}`; + } else if (driver_info && ipLogMap[driver_info.node_ip_address]) { + link = `${baseLink}/${encodeURIComponent( + ipLogMap[driver_info.node_ip_address], + )}`; + } + + if (link) { + link += `?fileName=${ + type === "DRIVER" ? job_id : `driver-${submission_id}` + }`; + return ( + + Log + + ); + } + + return -; +}; diff --git a/dashboard/client/src/pages/job/JobRow.tsx b/dashboard/client/src/pages/job/JobRow.tsx index 0be459efa1df..010754eb4976 100644 --- a/dashboard/client/src/pages/job/JobRow.tsx +++ b/dashboard/client/src/pages/job/JobRow.tsx @@ -1,18 +1,20 @@ import { TableCell, TableRow, Tooltip } from "@material-ui/core"; import { makeStyles } from "@material-ui/core/styles"; import dayjs from "dayjs"; -import React, { useContext } from "react"; +import React from "react"; import { Link } from "react-router-dom"; -import { GlobalContext } from "../../App"; import { DurationText } from "../../common/DurationText"; +import { StatusChip } from "../../components/StatusChip"; import { UnifiedJob } from "../../type/job"; import { useJobProgress } from "./hook/useJobProgress"; +import { JobLogsLink } from "./JobDetail"; import { MiniTaskProgressBar } from "./TaskProgressBar"; const useStyles = makeStyles((theme) => ({ overflowCell: { display: "block", - width: "150px", + margin: "auto", + maxWidth: 360, textOverflow: "ellipsis", overflow: "hidden", whiteSpace: "nowrap", @@ -24,21 +26,16 @@ type JobRowProps = { newIA?: boolean; }; -export const JobRow = ({ - job: { +export const JobRow = ({ job, newIA = false }: JobRowProps) => { + const { job_id, submission_id, driver_info, - type, status, start_time, end_time, entrypoint, - driver_agent_http_address, - }, - newIA = false, -}: JobRowProps) => { - const { ipLogMap } = useContext(GlobalContext); + } = job; const { progress, error, driverExists } = useJobProgress(job_id ?? undefined); const classes = useStyles(); @@ -57,30 +54,6 @@ export const JobRow = ({ } })(); - const logsLink = (() => { - let link: string | undefined; - if (driver_agent_http_address) { - link = `/log/${encodeURIComponent(`${driver_agent_http_address}/logs`)}`; - } else if (driver_info && ipLogMap[driver_info.node_ip_address]) { - link = `/log/${encodeURIComponent( - ipLogMap[driver_info.node_ip_address], - )}`; - } - - if (link) { - link += `?fileName=${ - type === "DRIVER" ? job_id : `driver-${submission_id}` - }`; - return ( - - Log - - ); - } - - return "-"; - })(); - return ( @@ -101,7 +74,9 @@ export const JobRow = ({
{entrypoint}
- {status} + + + {start_time && start_time > 0 ? ( @@ -113,7 +88,7 @@ export const JobRow = ({ {/* TODO(aguo): Also show logs for the job id instead of just the submission's logs */} - {logsLink} + {dayjs(Number(start_time)).format("YYYY/MM/DD HH:mm:ss")} diff --git a/dashboard/client/src/pages/layout/MainNavLayout.tsx b/dashboard/client/src/pages/layout/MainNavLayout.tsx index 85dabc6d97a3..fb2fcd6f968c 100644 --- a/dashboard/client/src/pages/layout/MainNavLayout.tsx +++ b/dashboard/client/src/pages/layout/MainNavLayout.tsx @@ -1,12 +1,19 @@ -import { createStyles, makeStyles, Typography } from "@material-ui/core"; +import { + createStyles, + IconButton, + makeStyles, + Tooltip, + Typography, +} from "@material-ui/core"; import classNames from "classnames"; import React, { useContext } from "react"; +import { RiBookMarkLine, RiFeedbackLine } from "react-icons/ri/"; import { Link, Outlet } from "react-router-dom"; import Logo from "../../logo.svg"; import { MainNavContext, useMainNavState } from "./mainNavContext"; -const MAIN_NAV_HEIGHT = 56; -const BREADCRUMBS_HEIGHT = 36; +export const MAIN_NAV_HEIGHT = 56; +export const BREADCRUMBS_HEIGHT = 36; const useStyles = makeStyles((theme) => createStyles({ @@ -103,6 +110,23 @@ const useMainNavBarStyles = makeStyles((theme) => navItemHighlighted: { color: "#036DCF", }, + flexSpacer: { + flexGrow: 1, + }, + actionItemsContainer: { + marginRight: theme.spacing(2), + }, + backToOld: { + marginRight: theme.spacing(1.5), + textDecoration: "none", + }, + backToOldText: { + letterSpacing: 0.25, + fontWeight: 500, + }, + actionItem: { + color: "#5F6469", + }, }), ); @@ -122,6 +146,16 @@ const NAV_ITEMS = [ path: "/new/cluster", id: "cluster", }, + { + title: "Actors", + path: "/new/actors", + id: "actors", + }, + { + title: "Metrics", + path: "/new/metrics", + id: "metrics", + }, { title: "Logs", path: "/new/logs", @@ -152,6 +186,41 @@ const MainNavBar = () => { ))} +
+
+ + + Back to old UI + + + + + + + + + + + + +
); }; diff --git a/dashboard/client/src/pages/layout/index.tsx b/dashboard/client/src/pages/layout/index.tsx index ee11cd765377..cdc6e07a3c76 100644 --- a/dashboard/client/src/pages/layout/index.tsx +++ b/dashboard/client/src/pages/layout/index.tsx @@ -61,6 +61,14 @@ const useStyles = makeStyles((theme) => ({ child: { flex: 1, }, + newUI: { + fontWeight: 500, + color: "#036DCF", + backgroundColor: "#036DCF20", + padding: theme.spacing(0.5), + margin: theme.spacing(-0.5), + borderRadius: 4, + }, })); const BasicLayout = ({ @@ -151,6 +159,15 @@ const BasicLayout = ({ METRICS )} + navigate("/new")} + > + + TRY THE NEW UI + + ) => { + return ( + + {children} + + ); +}; + +const MetricsDisabledWrapper = ({ children }: PropsWithChildren<{}>) => { + return ( + + {children} + + ); +}; + +describe("Metrics", () => { + it("renders", async () => { + expect.assertions(6); + + render(, { wrapper: Wrapper }); + await screen.findByText(/View in Grafana/); + expect(screen.getByText(/30 minutes/)).toBeVisible(); + expect(screen.getByText(/Tasks/)).toBeVisible(); + expect(screen.getByText(/Actors/)).toBeVisible(); + expect(screen.getByText(/Scheduler and autoscaler/)).toBeVisible(); + expect(screen.getByText(/Node metrics/)).toBeVisible(); + expect( + screen.queryByText(/Grafana or prometheus server not detected./), + ).toBeNull(); + }); + + it("renders warning when ", async () => { + expect.assertions(6); + + render(, { wrapper: MetricsDisabledWrapper }); + await screen.findByText(/Grafana or prometheus server not detected./); + expect(screen.queryByText(/View in Grafana/)).toBeNull(); + expect(screen.queryByText(/30 minutes/)).toBeNull(); + expect(screen.queryByText(/Tasks/)).toBeNull(); + expect(screen.queryByText(/Actors/)).toBeNull(); + expect(screen.queryByText(/Scheduler and autoscaler/)).toBeNull(); + expect(screen.queryByText(/Node metrics/)).toBeNull(); + }); +}); diff --git a/dashboard/client/src/pages/metrics/Metrics.tsx b/dashboard/client/src/pages/metrics/Metrics.tsx index 8610f6961622..db51011e4fb6 100644 --- a/dashboard/client/src/pages/metrics/Metrics.tsx +++ b/dashboard/client/src/pages/metrics/Metrics.tsx @@ -7,29 +7,36 @@ import { TextField, } from "@material-ui/core"; import { Alert } from "@material-ui/lab"; +import classNames from "classnames"; import React, { useContext, useEffect, useState } from "react"; +import { RiExternalLinkLine } from "react-icons/ri"; import { GlobalContext } from "../../App"; +import { CollapsibleSection } from "../../common/CollapsibleSection"; +import { ClassNameProps } from "../../common/props"; +import { MainNavPageInfo } from "../layout/mainNavContext"; +import { MAIN_NAV_HEIGHT } from "../layout/MainNavLayout"; const useStyles = makeStyles((theme) => createStyles({ - root: {}, + metricsRoot: { margin: theme.spacing(1) }, + metricsSection: { + marginTop: theme.spacing(3), + }, grafanaEmbedsContainer: { - marginTop: theme.spacing(1), - marginLeft: theme.spacing(1), display: "flex", flexDirection: "row", flexWrap: "wrap", gap: theme.spacing(3), + marginTop: theme.spacing(2), }, chart: { - flex: "1 0 448px", - maxWidth: "100%", + width: "100%", height: 300, overflow: "hidden", [theme.breakpoints.up("md")]: { // Calculate max width based on 1/3 of the total width minus padding between cards - maxWidth: `calc((100% - ${theme.spacing(3)}px * 2) / 3)`, + width: `calc((100% - ${theme.spacing(3)}px * 2) / 3)`, }, }, grafanaEmbed: { @@ -38,16 +45,26 @@ const useStyles = makeStyles((theme) => }, topBar: { position: "sticky", + top: 0, width: "100%", display: "flex", flexDirection: "row", alignItems: "center", justifyContent: "flex-end", padding: theme.spacing(1), + boxShadow: "0px 1px 0px #D2DCE6", + zIndex: 1, + height: 36, + }, + topBarNewIA: { + top: MAIN_NAV_HEIGHT, }, timeRangeButton: { marginLeft: theme.spacing(2), }, + alert: { + marginTop: 30, + }, }), ); @@ -75,85 +92,121 @@ const TIME_RANGE_TO_FROM_VALUE: Record = { [TimeRangeOptions.SEVEN_DAYS]: "now-7d", }; +type MetricConfig = { + title: string; + path: string; +}; + +type MetricsSectionConfig = { + title: string; + contents: MetricConfig[]; +}; + // NOTE: please keep the titles here in sync with grafana_dashboard_factory.py -const METRICS_CONFIG = [ - { - title: "Scheduler Task State", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=26", - }, - { - title: "Active Tasks by Name", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=35", - }, - { - title: "Scheduler Actor State", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=33", - }, - { - title: "Active Actors by Name", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=36", - }, +const METRICS_CONFIG: MetricsSectionConfig[] = [ { - title: "Scheduler CPUs (logical slots)", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=27", - }, - { - title: "Object Store Memory", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=29", - }, - { - title: "Scheduler GPUs (logical slots)", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=28", - }, - { - title: "Scheduler Placement Groups", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=40", - }, - { - title: "Node CPU (hardware utilization)", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=2", - }, - { - title: "Node GPU (hardware utilization)", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=8", - }, - { - title: "Node Disk", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=6", - }, - { - title: "Node Disk IO Speed", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=32", - }, - { - title: "Node Memory (heap + object store)", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=4", - }, - { - title: "Node Memory by Component", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=34", - }, - { - title: "Node CPU by Component", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=37", + title: "Tasks", + contents: [ + { + title: "Scheduler Task State", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=26", + }, + { + title: "Active Tasks by Name", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=35", + }, + ], }, { - title: "Node GPU Memory (GRAM)", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=18", + title: "Actors", + contents: [ + { + title: "Scheduler Actor State", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=33", + }, + { + title: "Active Actors by Name", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=36", + }, + ], }, { - title: "Node Network", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=20", + title: "Scheduler and autoscaler", + contents: [ + { + title: "Node Count", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=24", + }, + { + title: "Scheduler CPUs (logical slots)", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=27", + }, + { + title: "Scheduler GPUs (logical slots)", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=28", + }, + { + title: "Scheduler Placement Groups", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=40", + }, + ], }, { - title: "Node Count", - path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=24", + title: "Node metrics", + contents: [ + { + title: "Node CPU (hardware utilization)", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=2", + }, + { + title: "Node Memory (heap + object store)", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=4", + }, + { + title: "Node GPU (hardware utilization)", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=8", + }, + { + title: "Node GPU Memory (GRAM)", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=18", + }, + { + title: "Node Disk", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=6", + }, + { + title: "Node Disk IO Speed", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=32", + }, + { + title: "Node Network", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=20", + }, + { + title: "Node CPU by Component", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=37", + }, + + { + title: "Node Memory by Component", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=34", + }, + { + title: "Object Store Memory", + path: "/d-solo/rayDefaultDashboard/default-dashboard?orgId=1&theme=light&panelId=29", + }, + ], }, ]; -export const Metrics = () => { +type MetricsProps = { + newIA?: boolean; +}; + +export const Metrics = ({ newIA = false }: MetricsProps) => { const classes = useStyles(); - const { grafanaHost, sessionName } = useContext(GlobalContext); + const { grafanaHost, sessionName, prometheusHealth } = + useContext(GlobalContext); const [timeRangeOption, setTimeRangeOption] = useState( TimeRangeOptions.THIRTY_MINS, @@ -172,31 +225,28 @@ export const Metrics = () => { const timeRangeParams = `${fromParam}${toParam}`; return ( -
- {grafanaHost === undefined ? ( - - Grafana server not detected. Please make sure the grafana server is - running and refresh this page. See:{" "} - - https://docs.ray.io/en/latest/ray-observability/ray-metrics.html - - . -
- If you are hosting grafana on a separate machine or using a - non-default port, please set the RAY_GRAFANA_HOST env var to point to - your grafana server when launching ray. -
+
+ + {grafanaHost === undefined || !prometheusHealth ? ( + ) : (
- + @@ -204,7 +254,6 @@ export const Metrics = () => { className={classes.timeRangeButton} select size="small" - variant="outlined" style={{ width: 120 }} value={timeRangeOption} onChange={({ target: { value } }) => { @@ -223,17 +272,34 @@ export const Metrics = () => { time-series graph. You can use control/cmd + click to filter out a line in the time-series graph. -
- {METRICS_CONFIG.map(({ title, path }) => ( - -