From 86ab5c71dd67c1959164b8c6fb8ee5290484f76b Mon Sep 17 00:00:00 2001 From: scv119 Date: Wed, 1 Jun 2022 12:36:56 -0700 Subject: [PATCH 01/48] add --- src/ray/util/filesystem.cc | 25 +++++++++++++++++++++++++ src/ray/util/filesystem.h | 13 +++++++++++++ 2 files changed, 38 insertions(+) diff --git a/src/ray/util/filesystem.cc b/src/ray/util/filesystem.cc index 4e1c119e54ce..a69bcaaacc33 100644 --- a/src/ray/util/filesystem.cc +++ b/src/ray/util/filesystem.cc @@ -60,4 +60,29 @@ std::string GetUserTempDir() { return result; } +FileSystemMonitor::FileSystemMonitor(const std::string &path, double capacity_threshold) + : ray_file_path_(path), capacity_threshold_(capacity_threshold) {} + +std::optional FileSystemMonitor::Space() const { + std::error_code ec; + const std::filesystem::space_info si = std::filesystem::space(ray_file_path_, ec); + if (ec) { + RAY_LOG_EVERY_MS(WARNING, 60 * 1000) << "Failed to get capacity of " << ray_file_path_ + << " with error: " << ec.message(); + return std::nullopt; + } + return si; +} + +bool FileSystemMonitor::OverCapacity() const { + auto space_info = Space(); + if (!space_info.has_value()) { + return false; + } + if (space_info->capacity == 0) { + RAY_LOG_EVERY_MS(WARNING, 60 * 1000) << ray_file_path_ << " has no capacity."; + return true; + } + return (1 - 1.0f * space_info->available / space_info->capacity) > capacity_threshold_; +} } // namespace ray diff --git a/src/ray/util/filesystem.h b/src/ray/util/filesystem.h index 995bf5586c45..1cfa55487043 100644 --- a/src/ray/util/filesystem.h +++ b/src/ray/util/filesystem.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include @@ -58,4 +59,16 @@ std::string JoinPaths(std::string base, const Paths &...components) { (join(base, std::string_view(components)), ...); return base; } + +/// Monitor the filesystem capacity ray is using. +class FileSystemMonitor { + public: + FileSystemMonitor(const std::string &path, double capacity_threshold); + std::optional Space() const; + bool OverCapacity() const; + + private: + const std::string ray_file_path_; + const double capacity_threshold_; +}; } // namespace ray From 0827fe67a007ee977b262f864d54118fb2445d8b Mon Sep 17 00:00:00 2001 From: scv119 Date: Wed, 1 Jun 2022 13:07:38 -0700 Subject: [PATCH 02/48] add --- src/ray/common/ray_config_def.h | 3 +++ src/ray/object_manager/plasma/store.cc | 6 ++++++ src/ray/object_manager/plasma/store.h | 5 +++++ src/ray/object_manager/plasma/store_runner.cc | 3 +++ src/ray/object_manager/plasma/store_runner.h | 2 ++ src/ray/util/filesystem.cc | 12 ++++++++++-- 6 files changed, 29 insertions(+), 2 deletions(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 803bdf1aa7af..6d55ada8345b 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -483,6 +483,9 @@ RAY_CONFIG(int64_t, oom_grace_period_s, 2) /// This is configured based on object_spilling_config. RAY_CONFIG(bool, is_external_storage_type_fs, true) +/// TODO +RAY_CONFIG(float, local_fs_capacity_threshold, 0.9); + /* Configuration parameters for locality-aware scheduling. */ /// Whether to enable locality-aware leasing. If enabled, then Ray will consider task /// dependency locality when choosing a worker for leasing. diff --git a/src/ray/object_manager/plasma/store.cc b/src/ray/object_manager/plasma/store.cc index c8588ee03f9b..fb443921646f 100644 --- a/src/ray/object_manager/plasma/store.cc +++ b/src/ray/object_manager/plasma/store.cc @@ -71,6 +71,7 @@ ray::ObjectID GetCreateRequestObjectId(const std::vector &message) { PlasmaStore::PlasmaStore(instrumented_io_context &main_service, IAllocator &allocator, + ray::FileSystemMonitor &fs_monitor, const std::string &socket_name, uint32_t delay_on_oom_ms, float object_spilling_threshold, @@ -83,6 +84,7 @@ PlasmaStore::PlasmaStore(instrumented_io_context &main_service, acceptor_(main_service, ParseUrlEndpoint(socket_name)), socket_(main_service), allocator_(allocator), + fs_monitor_(fs_monitor), add_object_callback_(add_object_callback), delete_object_callback_(delete_object_callback), object_lifecycle_mgr_(allocator_, delete_object_callback_), @@ -158,6 +160,10 @@ PlasmaError PlasmaStore::HandleCreateObjectRequest(const std::shared_ptr return PlasmaError::OutOfMemory; } + if (fs_monitor_.OverCapacity()) { + RAY_LOG(ERROR) << "device_num != 0 but CUDA not enabled"; + return PlasmaError::OutOfMemory; + } auto error = CreateObject(object_info, source, client, fallback_allocator, object); if (error == PlasmaError::OutOfMemory) { RAY_LOG(DEBUG) << "Not enough memory to create the object " << object_info.object_id diff --git a/src/ray/object_manager/plasma/store.h b/src/ray/object_manager/plasma/store.h index 758900305104..fba175ab051e 100644 --- a/src/ray/object_manager/plasma/store.h +++ b/src/ray/object_manager/plasma/store.h @@ -40,6 +40,7 @@ #include "ray/object_manager/plasma/plasma.h" #include "ray/object_manager/plasma/plasma_allocator.h" #include "ray/object_manager/plasma/protocol.h" +#include "ray/util/filesystem.h" namespace plasma { @@ -55,6 +56,7 @@ class PlasmaStore { public: PlasmaStore(instrumented_io_context &main_service, IAllocator &allocator, + ray::FileSystemMonitor &fs_monitor, const std::string &socket_name, uint32_t delay_on_oom_ms, float object_spilling_threshold, @@ -249,6 +251,9 @@ class PlasmaStore { /// The allocator that allocates mmaped memory. IAllocator &allocator_ GUARDED_BY(mutex_); + /// Monitor the disk utilization. + ray::FileSystemMonitor &fs_monitor_; + /// A callback to asynchronously notify that an object is sealed. /// NOTE: This function should guarantee the thread-safety because the callback is /// shared with the main raylet thread. diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index b5727d0d8865..dedf792afe69 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -90,8 +90,11 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, absl::MutexLock lock(&store_runner_mutex_); allocator_ = std::make_unique( plasma_directory_, fallback_directory_, hugepages_enabled_, system_memory_); + fs_monitor_ = std::make_unique( + fallback_directory_, RayConfig::instance().local_fs_capacity_threshold()); store_.reset(new PlasmaStore(main_service_, *allocator_, + *fs_monitor_, socket_name_, RayConfig::instance().object_store_full_delay_ms(), RayConfig::instance().object_spilling_threshold(), diff --git a/src/ray/object_manager/plasma/store_runner.h b/src/ray/object_manager/plasma/store_runner.h index e2962be4d31e..272d3be15c0f 100644 --- a/src/ray/object_manager/plasma/store_runner.h +++ b/src/ray/object_manager/plasma/store_runner.h @@ -7,6 +7,7 @@ #include "ray/common/asio/instrumented_io_context.h" #include "ray/object_manager/plasma/plasma_allocator.h" #include "ray/object_manager/plasma/store.h" +#include "ray/util/filesystem.h" namespace plasma { @@ -43,6 +44,7 @@ class PlasmaStoreRunner { std::string fallback_directory_; mutable instrumented_io_context main_service_; std::unique_ptr allocator_; + std::unique_ptr fs_monitor_; std::unique_ptr store_; }; diff --git a/src/ray/util/filesystem.cc b/src/ray/util/filesystem.cc index a69bcaaacc33..d26a5321a682 100644 --- a/src/ray/util/filesystem.cc +++ b/src/ray/util/filesystem.cc @@ -80,9 +80,17 @@ bool FileSystemMonitor::OverCapacity() const { return false; } if (space_info->capacity == 0) { - RAY_LOG_EVERY_MS(WARNING, 60 * 1000) << ray_file_path_ << " has no capacity."; + RAY_LOG_EVERY_MS(ERROR, 60 * 1000) << ray_file_path_ << " has no capacity."; return true; } - return (1 - 1.0f * space_info->available / space_info->capacity) > capacity_threshold_; + + if ((1 - 1.0f * space_info->available / space_info->capacity) < capacity_threshold_) { + return false; + } + + RAY_LOG_EVERY_MS(ERROR, 10 * 1000) + << ray_file_path_ << " is over capacity, available: " << space_info->available + << ", capacity: " << space_info->capacity << ", threshold: " << capacity_threshold_; + return true; } } // namespace ray From 2869ae1b33fb8d3c8ebf356e82d324ebef5c51fb Mon Sep 17 00:00:00 2001 From: scv119 Date: Wed, 1 Jun 2022 13:53:23 -0700 Subject: [PATCH 03/48] add --- python/ray/tests/test_out_of_disk_space.py | 54 +++++++++++++++++++ .../store_provider/plasma_store_provider.cc | 3 +- 2 files changed, 56 insertions(+), 1 deletion(-) create mode 100644 python/ray/tests/test_out_of_disk_space.py diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py new file mode 100644 index 000000000000..fe952f77d3a2 --- /dev/null +++ b/python/ray/tests/test_out_of_disk_space.py @@ -0,0 +1,54 @@ +import sys +import numpy as np + +import pytest +import ray + + +def test_put_out_of_disk(shutdown_only): + ray.init( + num_cpus=1, + _system_config={ + "local_fs_capacity_threshold": 0, + }, + ) + arr = np.random.rand(10 * 1024 * 1024) # 80 MB data + with pytest.raises(ray.exceptions.ObjectStoreFullError): + ray.put(arr) + + +def test_task_of_disk(shutdown_only): + ray.init( + num_cpus=1, + _system_config={ + "local_fs_capacity_threshold": 0, + }, + ) + + @ray.remote + def foo(): + return np.random.rand(10 * 1024 * 1024) # 80 MB data + + with pytest.raises(ray.exceptions.RayTaskError): + ray.get(foo.remote()) + + +def test_task_of_disk_1(shutdown_only): + ray.init( + num_cpus=1, + _system_config={ + "local_fs_capacity_threshold": 0, + }, + ) + + @ray.remote + def foo(): + ref = ray.put(np.random.rand(10 * 1024 * 1024)) # 80 MB data + return ref + + with pytest.raises(ray.exceptions.RayTaskError): + ray.get(foo.remote()) + + +if __name__ == "__main__": + sys.exit(pytest.main(["-sv", __file__])) diff --git a/src/ray/core_worker/store_provider/plasma_store_provider.cc b/src/ray/core_worker/store_provider/plasma_store_provider.cc index 02a46f983160..0060a9a3decb 100644 --- a/src/ray/core_worker/store_provider/plasma_store_provider.cc +++ b/src/ray/core_worker/store_provider/plasma_store_provider.cc @@ -470,7 +470,8 @@ Status CoreWorkerPlasmaStoreProvider::WarmupStore() { ObjectID object_id = ObjectID::FromRandom(); std::shared_ptr data; RAY_RETURN_NOT_OK( - Create(nullptr, 8, object_id, rpc::Address(), &data, /*created_by_worker=*/true)); + Create(nullptr, 8, object_id, rpc::Address(), &data, + /*created_by_worker=*/true)); RAY_RETURN_NOT_OK(Seal(object_id)); RAY_RETURN_NOT_OK(Release(object_id)); RAY_RETURN_NOT_OK(Delete({object_id}, true)); From 85eec4009716ba89d98043398a37c9a6a5d15df9 Mon Sep 17 00:00:00 2001 From: scv119 Date: Wed, 1 Jun 2022 14:00:03 -0700 Subject: [PATCH 04/48] add --- src/ray/common/ray_config_def.h | 3 ++- src/ray/util/filesystem.cc | 1 + src/ray/util/filesystem.h | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 6d55ada8345b..19e15dadb4d1 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -483,7 +483,8 @@ RAY_CONFIG(int64_t, oom_grace_period_s, 2) /// This is configured based on object_spilling_config. RAY_CONFIG(bool, is_external_storage_type_fs, true) -/// TODO +/// Control the capacity threshold for ray local file system (for object store). +/// Once we are over the capacity, all subsequent object creation will fail. RAY_CONFIG(float, local_fs_capacity_threshold, 0.9); /* Configuration parameters for locality-aware scheduling. */ diff --git a/src/ray/util/filesystem.cc b/src/ray/util/filesystem.cc index d26a5321a682..4738bc338b87 100644 --- a/src/ray/util/filesystem.cc +++ b/src/ray/util/filesystem.cc @@ -65,6 +65,7 @@ FileSystemMonitor::FileSystemMonitor(const std::string &path, double capacity_th std::optional FileSystemMonitor::Space() const { std::error_code ec; + // TODO: add cache to improve performance. const std::filesystem::space_info si = std::filesystem::space(ray_file_path_, ec); if (ec) { RAY_LOG_EVERY_MS(WARNING, 60 * 1000) << "Failed to get capacity of " << ray_file_path_ diff --git a/src/ray/util/filesystem.h b/src/ray/util/filesystem.h index 1cfa55487043..8ceeca6733a3 100644 --- a/src/ray/util/filesystem.h +++ b/src/ray/util/filesystem.h @@ -61,6 +61,7 @@ std::string JoinPaths(std::string base, const Paths &...components) { } /// Monitor the filesystem capacity ray is using. +/// This class is not thread safe. class FileSystemMonitor { public: FileSystemMonitor(const std::string &path, double capacity_threshold); From a9449bca579f1b005a60379fdb37eb701d355380 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 11:11:24 -0700 Subject: [PATCH 05/48] add --- .../store_provider/plasma_store_provider.cc | 9 ++++++--- .../object_manager/plasma/create_request_queue.cc | 14 ++++++++++++++ .../object_manager/plasma/create_request_queue.h | 12 ++++++++++-- src/ray/object_manager/plasma/plasma.fbs | 4 +++- src/ray/object_manager/plasma/store.cc | 5 +---- 5 files changed, 34 insertions(+), 10 deletions(-) diff --git a/src/ray/core_worker/store_provider/plasma_store_provider.cc b/src/ray/core_worker/store_provider/plasma_store_provider.cc index 0060a9a3decb..827678edc0f8 100644 --- a/src/ray/core_worker/store_provider/plasma_store_provider.cc +++ b/src/ray/core_worker/store_provider/plasma_store_provider.cc @@ -469,9 +469,12 @@ void CoreWorkerPlasmaStoreProvider::WarnIfFetchHanging( Status CoreWorkerPlasmaStoreProvider::WarmupStore() { ObjectID object_id = ObjectID::FromRandom(); std::shared_ptr data; - RAY_RETURN_NOT_OK( - Create(nullptr, 8, object_id, rpc::Address(), &data, - /*created_by_worker=*/true)); + RAY_RETURN_NOT_OK(Create(nullptr, + 8, + object_id, + rpc::Address(), + &data, + /*created_by_worker=*/true)); RAY_RETURN_NOT_OK(Seal(object_id)); RAY_RETURN_NOT_OK(Release(object_id)); RAY_RETURN_NOT_OK(Delete({object_id}, true)); diff --git a/src/ray/object_manager/plasma/create_request_queue.cc b/src/ray/object_manager/plasma/create_request_queue.cc index 12d3d92c640e..e9ca0f414f22 100644 --- a/src/ray/object_manager/plasma/create_request_queue.cc +++ b/src/ray/object_manager/plasma/create_request_queue.cc @@ -85,6 +85,14 @@ Status CreateRequestQueue::ProcessRequest(bool fallback_allocator, } } +bool CreateRequestQueue::MayHandleOutOfDisk(std::unique_ptr &request) { + if (request->error == PlasmaError::OutOfMemory && fs_monitor_.OverCapacity()) { + request->error = PlasmaError::OutOfDisk; + return true; + } + return false; +} + Status CreateRequestQueue::ProcessRequests() { // Suppress OOM dump to once per grace period. bool logged_oom = false; @@ -93,6 +101,12 @@ Status CreateRequestQueue::ProcessRequests() { bool spilling_required = false; auto status = ProcessRequest(/*fallback_allocator=*/false, *request_it, &spilling_required); + + if (MayHandleOutOfDisk(*request_it)) { + FinishRequest(request_it); + continue; + } + if (spilling_required) { spill_objects_callback_(); } diff --git a/src/ray/object_manager/plasma/create_request_queue.h b/src/ray/object_manager/plasma/create_request_queue.h index 740ef7eed035..526c1054597a 100644 --- a/src/ray/object_manager/plasma/create_request_queue.h +++ b/src/ray/object_manager/plasma/create_request_queue.h @@ -25,6 +25,7 @@ #include "ray/object_manager/plasma/connection.h" #include "ray/object_manager/plasma/plasma.h" #include "ray/object_manager/plasma/protocol.h" +#include "ray/util/filesystem.h" namespace plasma { @@ -33,12 +34,14 @@ class CreateRequestQueue { using CreateObjectCallback = std::function; - CreateRequestQueue(int64_t oom_grace_period_s, + CreateRequestQueue(ray::FileSystemMonitor &fs_monitor, + int64_t oom_grace_period_s, ray::SpillObjectsCallback spill_objects_callback, std::function trigger_global_gc, std::function get_time, std::function dump_debug_info_callback = nullptr) - : oom_grace_period_ns_(oom_grace_period_s * 1e9), + : fs_monitor_(fs_monitor), + oom_grace_period_ns_(oom_grace_period_s * 1e9), spill_objects_callback_(spill_objects_callback), trigger_global_gc_(trigger_global_gc), get_time_(get_time), @@ -156,9 +159,14 @@ class CreateRequestQueue { std::unique_ptr &request, bool *spilling_required); + bool MayHandleOutOfDisk(std::unique_ptr &request); + /// Finish a queued request and remove it from the queue. void FinishRequest(std::list>::iterator request_it); + /// Monitor the disk utilization. + ray::FileSystemMonitor &fs_monitor_; + /// The next request ID to assign, so that the caller can get the results of /// a request by retrying. Start at 1 because 0 means "do not retry". uint64_t next_req_id_ = 1; diff --git a/src/ray/object_manager/plasma/plasma.fbs b/src/ray/object_manager/plasma/plasma.fbs index f8ef0ceab48f..4540935cf858 100644 --- a/src/ray/object_manager/plasma/plasma.fbs +++ b/src/ray/object_manager/plasma/plasma.fbs @@ -72,7 +72,7 @@ enum PlasmaError:int { ObjectExists, // Trying to access an object that doesn't exist. ObjectNonexistent, - // Trying to create an object but there isn't enough space in the store. + // Trying to create an object but there isn't enough memory in the store. OutOfMemory, // Trying to delete an object but it's not sealed. ObjectNotSealed, @@ -84,6 +84,8 @@ enum PlasmaError:int { UnexpectedError, // Trying to abort an object but it's not sealed. ObjectSealed, + // Trying to create an object but there isn't enough disk in the store. + OutOfDisk, } // Plasma store messages diff --git a/src/ray/object_manager/plasma/store.cc b/src/ray/object_manager/plasma/store.cc index fb443921646f..932da27b0029 100644 --- a/src/ray/object_manager/plasma/store.cc +++ b/src/ray/object_manager/plasma/store.cc @@ -91,6 +91,7 @@ PlasmaStore::PlasmaStore(instrumented_io_context &main_service, delay_on_oom_ms_(delay_on_oom_ms), object_spilling_threshold_(object_spilling_threshold), create_request_queue_( + fs_monitor_, /*oom_grace_period_s=*/RayConfig::instance().oom_grace_period_s(), spill_objects_callback, object_store_full_callback, @@ -160,10 +161,6 @@ PlasmaError PlasmaStore::HandleCreateObjectRequest(const std::shared_ptr return PlasmaError::OutOfMemory; } - if (fs_monitor_.OverCapacity()) { - RAY_LOG(ERROR) << "device_num != 0 but CUDA not enabled"; - return PlasmaError::OutOfMemory; - } auto error = CreateObject(object_info, source, client, fallback_allocator, object); if (error == PlasmaError::OutOfMemory) { RAY_LOG(DEBUG) << "Not enough memory to create the object " << object_info.object_id From 7c89d20243023a1a3357cb080b5f6efae20e53eb Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 11:58:00 -0700 Subject: [PATCH 06/48] add --- python/ray/tests/test_out_of_disk_space.py | 11 +++++++---- src/ray/common/status.h | 6 ++++++ src/ray/object_manager/plasma/protocol.cc | 2 ++ 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index fe952f77d3a2..12baa6d8ac15 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -8,18 +8,20 @@ def test_put_out_of_disk(shutdown_only): ray.init( num_cpus=1, + object_store_memory=80 * 1024 * 1024, _system_config={ "local_fs_capacity_threshold": 0, }, ) - arr = np.random.rand(10 * 1024 * 1024) # 80 MB data - with pytest.raises(ray.exceptions.ObjectStoreFullError): + arr = np.random.rand(20 * 1024 * 1024) # 80 MB data + with pytest.raises(ray.exceptions.RaySystemError): ray.put(arr) def test_task_of_disk(shutdown_only): ray.init( num_cpus=1, + object_store_memory=80 * 1024 * 1024, _system_config={ "local_fs_capacity_threshold": 0, }, @@ -27,7 +29,7 @@ def test_task_of_disk(shutdown_only): @ray.remote def foo(): - return np.random.rand(10 * 1024 * 1024) # 80 MB data + return np.random.rand(20 * 1024 * 1024) # 80 MB data with pytest.raises(ray.exceptions.RayTaskError): ray.get(foo.remote()) @@ -36,6 +38,7 @@ def foo(): def test_task_of_disk_1(shutdown_only): ray.init( num_cpus=1, + object_store_memory=80 * 1024 * 1024, _system_config={ "local_fs_capacity_threshold": 0, }, @@ -43,7 +46,7 @@ def test_task_of_disk_1(shutdown_only): @ray.remote def foo(): - ref = ray.put(np.random.rand(10 * 1024 * 1024)) # 80 MB data + ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 80 MB data return ref with pytest.raises(ray.exceptions.RayTaskError): diff --git a/src/ray/common/status.h b/src/ray/common/status.h index c536738c5045..14995f48bfc7 100644 --- a/src/ray/common/status.h +++ b/src/ray/common/status.h @@ -108,6 +108,7 @@ enum class StatusCode : char { // This represents all other status codes // returned by grpc that are not defined above. GrpcUnknown = 27, + OutOfDisk = 28, }; #if defined(__clang__) @@ -211,6 +212,10 @@ class RAY_EXPORT Status { return Status(StatusCode::TransientObjectStoreFull, msg); } + static Status OutOfDisk(const std::string &msg) { + return Status(StatusCode::OutOfDisk, msg); + } + static Status GrpcUnavailable(const std::string &msg) { return Status(StatusCode::GrpcUnavailable, msg); } @@ -225,6 +230,7 @@ class RAY_EXPORT Status { bool ok() const { return (state_ == NULL); } bool IsOutOfMemory() const { return code() == StatusCode::OutOfMemory; } + bool IsOutOfDisk() const { return code() == StatusCode::OutOfDisk; } bool IsKeyError() const { return code() == StatusCode::KeyError; } bool IsInvalid() const { return code() == StatusCode::Invalid; } bool IsIOError() const { return code() == StatusCode::IOError; } diff --git a/src/ray/object_manager/plasma/protocol.cc b/src/ray/object_manager/plasma/protocol.cc index 181b263d44b3..4333c3d70ad3 100644 --- a/src/ray/object_manager/plasma/protocol.cc +++ b/src/ray/object_manager/plasma/protocol.cc @@ -152,6 +152,8 @@ Status PlasmaErrorStatus(fb::PlasmaError plasma_error) { return Status::ObjectNotFound("object does not exist in the plasma store"); case fb::PlasmaError::OutOfMemory: return Status::ObjectStoreFull("object does not fit in the plasma store"); + case fb::PlasmaError::OutOfDisk: + return Status::OutOfDisk("Local disk is full"); case fb::PlasmaError::UnexpectedError: return Status::UnknownError( "an unexpected error occurred, likely due to a bug in the system or caller"); From af82bcd7d3c84a1a3f6d54a2e06260032c4852bb Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 12:52:25 -0700 Subject: [PATCH 07/48] add --- python/ray/tests/test_out_of_disk_space.py | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index 12baa6d8ac15..c0c1c72303f3 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -18,7 +18,7 @@ def test_put_out_of_disk(shutdown_only): ray.put(arr) -def test_task_of_disk(shutdown_only): +def test_task_returns(shutdown_only): ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, @@ -35,7 +35,24 @@ def foo(): ray.get(foo.remote()) -def test_task_of_disk_1(shutdown_only): +def test_task_put(shutdown_only): + ray.init( + num_cpus=1, + object_store_memory=80 * 1024 * 1024, + _system_config={ + "local_fs_capacity_threshold": 0, + }, + ) + + @ray.remote + def foo(): + ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 80 MB data + return ref + + with pytest.raises(ray.exceptions.RayTaskError): + ray.get(foo.remote()) + +def test_task_args(shutdown_only): ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, From 11fb149be3df12edba2ba3fb143150a8a32d07d1 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 13:08:47 -0700 Subject: [PATCH 08/48] ad --- python/ray/_raylet.pyx | 3 +++ python/ray/exceptions.py | 16 ++++++++++++++++ python/ray/includes/common.pxd | 1 + python/ray/tests/test_out_of_disk_space.py | 3 ++- 4 files changed, 22 insertions(+), 1 deletion(-) diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 4b505467c489..4ed9def543f4 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -120,6 +120,7 @@ from ray.exceptions import ( RaySystemError, RayTaskError, ObjectStoreFullError, + OutOfDiskError, GetTimeoutError, TaskCancelledError, AsyncioActorExit, @@ -166,6 +167,8 @@ cdef int check_status(const CRayStatus& status) nogil except -1: if status.IsObjectStoreFull(): raise ObjectStoreFullError(message) + elif status.IsOutOfDisk(): + raise OutOfDiskError(message) elif status.IsInterrupted(): raise KeyboardInterrupt() elif status.IsTimedOut(): diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index e7d6bc4a85f2..f6a5d1301848 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -326,6 +326,22 @@ def __str__(self): ) +class OutOfDiskError(RayError): + """Indicates that the local disk is full. + + This is raised if the attempt to store the object fails + because both the object store and disk are full. + """ + + def __str__(self): + return super(OutOfDiskError, self).__str__() + ( + "\n" + "The local object store is full of objects that are still in " + "scope and cannot be evicted. Tip: Use the `ray memory` command " + "to list active objects in the cluster." + ) + + class ObjectLostError(RayError): """Indicates that the object is lost from distributed memory, due to node failure or system error. diff --git a/python/ray/includes/common.pxd b/python/ray/includes/common.pxd index a23c2034aa33..96c30f725fbd 100644 --- a/python/ray/includes/common.pxd +++ b/python/ray/includes/common.pxd @@ -108,6 +108,7 @@ cdef extern from "ray/common/status.h" namespace "ray" nogil: c_bool IsUnknownError() c_bool IsNotImplemented() c_bool IsObjectStoreFull() + c_bool IsOutOfDisk() c_bool IsRedisError() c_bool IsTimedOut() c_bool IsInterrupted() diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index c0c1c72303f3..a6ab15d102a8 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -14,7 +14,7 @@ def test_put_out_of_disk(shutdown_only): }, ) arr = np.random.rand(20 * 1024 * 1024) # 80 MB data - with pytest.raises(ray.exceptions.RaySystemError): + with pytest.raises(ray.exceptions.OutOfDiskError): ray.put(arr) @@ -52,6 +52,7 @@ def foo(): with pytest.raises(ray.exceptions.RayTaskError): ray.get(foo.remote()) + def test_task_args(shutdown_only): ray.init( num_cpus=1, From b771d84b6796e7d78c49314a2d358747d653b453 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 13:14:04 -0700 Subject: [PATCH 09/48] add --- python/ray/exceptions.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index f6a5d1301848..7c9afafbe300 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -336,8 +336,8 @@ class OutOfDiskError(RayError): def __str__(self): return super(OutOfDiskError, self).__str__() + ( "\n" - "The local object store is full of objects that are still in " - "scope and cannot be evicted. Tip: Use the `ray memory` command " + "The local object store is full and local disk is also full." + "Tip: Use the `ray memory` command " "to list active objects in the cluster." ) From 328952ec999c76e528384fe9dfbaedfcb4249a99 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 13:48:01 -0700 Subject: [PATCH 10/48] add --- python/ray/tests/test_out_of_disk_space.py | 32 +++++++++++++++++----- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index a6ab15d102a8..c2c5f929b073 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -1,64 +1,82 @@ +import json import sys +import shutil import numpy as np +import platform import pytest import ray +def calculate_capacity_threshold(disk_capacity_in_bytes): + usage = shutil.disk_usage("/tmp") + threshold = min(1, 1.0 * (usage.used + disk_capacity_in_bytes) / usage.total) + assert threshold > 0 and threshold < 1 + return threshold + + +@pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_put_out_of_disk(shutdown_only): + local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, _system_config={ - "local_fs_capacity_threshold": 0, + "local_fs_capacity_threshold": local_fs_capacity_threshold, }, ) - arr = np.random.rand(20 * 1024 * 1024) # 80 MB data + arr = np.random.rand(20 * 1024 * 1024) # 160 MB data with pytest.raises(ray.exceptions.OutOfDiskError): ray.put(arr) +@pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_task_returns(shutdown_only): + local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, _system_config={ - "local_fs_capacity_threshold": 0, + "local_fs_capacity_threshold": local_fs_capacity_threshold, }, ) @ray.remote def foo(): - return np.random.rand(20 * 1024 * 1024) # 80 MB data + return np.random.rand(20 * 1024 * 1024) # 160 MB data with pytest.raises(ray.exceptions.RayTaskError): ray.get(foo.remote()) +@pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_task_put(shutdown_only): + local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, _system_config={ - "local_fs_capacity_threshold": 0, + "local_fs_capacity_threshold": local_fs_capacity_threshold, }, ) @ray.remote def foo(): - ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 80 MB data + ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 160 MB data return ref with pytest.raises(ray.exceptions.RayTaskError): ray.get(foo.remote()) +@pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_task_args(shutdown_only): + local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, _system_config={ - "local_fs_capacity_threshold": 0, + "local_fs_capacity_threshold": local_fs_capacity_threshold, }, ) From ae001aa8c8c11855f5b714689486d2f27c773c59 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 13:53:23 -0700 Subject: [PATCH 11/48] update --- src/ray/util/filesystem.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/util/filesystem.cc b/src/ray/util/filesystem.cc index 4738bc338b87..db11879244c0 100644 --- a/src/ray/util/filesystem.cc +++ b/src/ray/util/filesystem.cc @@ -80,7 +80,7 @@ bool FileSystemMonitor::OverCapacity() const { if (!space_info.has_value()) { return false; } - if (space_info->capacity == 0) { + if (space_info->capacity <= 0) { RAY_LOG_EVERY_MS(ERROR, 60 * 1000) << ray_file_path_ << " has no capacity."; return true; } From f3ff35a1bfccd49fbe121435c20ce691ed670b49 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 16:24:04 -0700 Subject: [PATCH 12/48] fix-tests --- .../object_manager/test/create_request_queue_test.cc | 12 ++++++++++++ src/ray/util/filesystem.cc | 3 +++ 2 files changed, 15 insertions(+) diff --git a/src/ray/object_manager/test/create_request_queue_test.cc b/src/ray/object_manager/test/create_request_queue_test.cc index b81579e1ba00..b4863e631e8a 100644 --- a/src/ray/object_manager/test/create_request_queue_test.cc +++ b/src/ray/object_manager/test/create_request_queue_test.cc @@ -17,6 +17,7 @@ #include "gmock/gmock.h" #include "gtest/gtest.h" #include "ray/common/status.h" +#include "ray/util/filesystem.h" namespace plasma { @@ -52,7 +53,9 @@ class CreateRequestQueueTest : public ::testing::Test { CreateRequestQueueTest() : oom_grace_period_s_(1), current_time_ns_(0), + monitor_("/", 1), queue_( + monitor_, /*oom_grace_period_s=*/oom_grace_period_s_, /*spill_object_callback=*/[&]() { return false; }, /*on_global_gc=*/[&]() { num_global_gc_++; }, @@ -68,6 +71,7 @@ class CreateRequestQueueTest : public ::testing::Test { int64_t oom_grace_period_s_; int64_t current_time_ns_; + ray::FileSystemMonitor monitor_; CreateRequestQueue queue_; int num_global_gc_ = 0; }; @@ -180,7 +184,9 @@ TEST_F(CreateRequestQueueTest, TestFallbackAllocator) { TEST(CreateRequestQueueParameterTest, TestOomInfiniteRetry) { int num_global_gc_ = 0; int64_t current_time_ns; + ray::FileSystemMonitor monitor{"/", 1}; CreateRequestQueue queue( + monitor, /*oom_grace_period_s=*/100, // Spilling is failing. /*spill_object_callback=*/[&]() { return false; }, @@ -212,7 +218,9 @@ TEST(CreateRequestQueueParameterTest, TestOomInfiniteRetry) { } TEST_F(CreateRequestQueueTest, TestTransientOom) { + ray::FileSystemMonitor monitor{"/", 1}; CreateRequestQueue queue( + monitor, /*oom_grace_period_s=*/oom_grace_period_s_, /*spill_object_callback=*/[&]() { return true; }, /*on_global_gc=*/[&]() { num_global_gc_++; }, @@ -258,6 +266,7 @@ TEST_F(CreateRequestQueueTest, TestTransientOom) { TEST_F(CreateRequestQueueTest, TestOomTimerWithSpilling) { int spill_object_callback_ret = true; CreateRequestQueue queue( + monitor_, /*oom_grace_period_s=*/oom_grace_period_s_, /*spill_object_callback=*/ [&]() { return spill_object_callback_ret; }, @@ -316,6 +325,7 @@ TEST_F(CreateRequestQueueTest, TestOomTimerWithSpilling) { TEST_F(CreateRequestQueueTest, TestTransientOomThenOom) { bool is_spilling_possible = true; CreateRequestQueue queue( + monitor_, /*oom_grace_period_s=*/oom_grace_period_s_, /*spill_object_callback=*/[&]() { return is_spilling_possible; }, /*on_global_gc=*/[&]() { num_global_gc_++; }, @@ -368,7 +378,9 @@ TEST_F(CreateRequestQueueTest, TestTransientOomThenOom) { TEST(CreateRequestQueueParameterTest, TestNoEvictIfFull) { int64_t current_time_ns = 0; + ray::FileSystemMonitor monitor{"/", 1}; CreateRequestQueue queue( + monitor, /*oom_grace_period_s=*/1, /*spill_object_callback=*/[&]() { return false; }, /*on_global_gc=*/[&]() {}, diff --git a/src/ray/util/filesystem.cc b/src/ray/util/filesystem.cc index db11879244c0..f97e1b864e50 100644 --- a/src/ray/util/filesystem.cc +++ b/src/ray/util/filesystem.cc @@ -76,6 +76,9 @@ std::optional FileSystemMonitor::Space() const { } bool FileSystemMonitor::OverCapacity() const { + if (capacity_threshold_ == 1) { + return false; + } auto space_info = Space(); if (!space_info.has_value()) { return false; From 9efd76615acb3c6a68e651d934dcae21c508ac60 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 18:22:53 -0700 Subject: [PATCH 13/48] add tests --- src/ray/util/filesystem.cc | 6 +++++- src/ray/util/filesystem.h | 7 +++++++ src/ray/util/filesystem_test.cc | 33 +++++++++++++++++++++++++++++++++ 3 files changed, 45 insertions(+), 1 deletion(-) diff --git a/src/ray/util/filesystem.cc b/src/ray/util/filesystem.cc index f97e1b864e50..1619efdb56e5 100644 --- a/src/ray/util/filesystem.cc +++ b/src/ray/util/filesystem.cc @@ -79,7 +79,11 @@ bool FileSystemMonitor::OverCapacity() const { if (capacity_threshold_ == 1) { return false; } - auto space_info = Space(); + return OverCapacityImpl(Space()); +} + +bool FileSystemMonitor::OverCapacityImpl( + const std::optional &space_info) const { if (!space_info.has_value()) { return false; } diff --git a/src/ray/util/filesystem.h b/src/ray/util/filesystem.h index 8ceeca6733a3..e7bdce121715 100644 --- a/src/ray/util/filesystem.h +++ b/src/ray/util/filesystem.h @@ -14,6 +14,8 @@ #pragma once +#include + #include #include #include @@ -69,6 +71,11 @@ class FileSystemMonitor { bool OverCapacity() const; private: + // For testing purpose. + bool OverCapacityImpl(const std::optional &info) const; + + private: + FRIEND_TEST(FileSystemTest, TestOverCapacity); const std::string ray_file_path_; const double capacity_threshold_; }; diff --git a/src/ray/util/filesystem_test.cc b/src/ray/util/filesystem_test.cc index 179839af9a85..1e31f73866c3 100644 --- a/src/ray/util/filesystem_test.cc +++ b/src/ray/util/filesystem_test.cc @@ -14,6 +14,8 @@ #include "ray/util/filesystem.h" +#include + #include "gtest/gtest.h" namespace ray { @@ -63,6 +65,37 @@ TEST(FileSystemTest, JoinPathTest) { #endif } +TEST(FileSystemTest, TestFileSystemMonitor) { + std::string tmp_path = std::filesystem::temp_directory_path().string(); + { + FileSystemMonitor monitor(tmp_path, 1); + ASSERT_FALSE(monitor.OverCapacity()); + } + + { + FileSystemMonitor monitor(tmp_path, 0); + ASSERT_TRUE(monitor.OverCapacity()); + } + + { + FileSystemMonitor monitor(tmp_path, 0); + auto result = monitor.Space(); + ASSERT_TRUE(result.has_value()); + ASSERT_TRUE(result->available > 0); + ASSERT_TRUE(result->capacity > 0); + } +} + +TEST(FileSystemTest, TestOverCapacity) { + std::string tmp_path = std::filesystem::temp_directory_path().string(); + FileSystemMonitor monitor(tmp_path, 0.1); + ASSERT_FALSE(monitor.OverCapacityImpl(std::nullopt)); + ASSERT_FALSE(monitor.OverCapacityImpl({std::filesystem::space_info{ + /* capacity */ 11, /* free */ 10, /* available */ 1}})); + ASSERT_TRUE(monitor.OverCapacityImpl( + {std::filesystem::space_info{/* capacity */ 11, /* free */ 9, /* available */ 2}})); +} + } // namespace ray int main(int argc, char **argv) { From cd6b0dda1beb41ac1a048007718a827ffecbb244 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 18:27:00 -0700 Subject: [PATCH 14/48] add more tests --- src/ray/util/filesystem_test.cc | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/ray/util/filesystem_test.cc b/src/ray/util/filesystem_test.cc index 1e31f73866c3..95ece311b538 100644 --- a/src/ray/util/filesystem_test.cc +++ b/src/ray/util/filesystem_test.cc @@ -91,9 +91,11 @@ TEST(FileSystemTest, TestOverCapacity) { FileSystemMonitor monitor(tmp_path, 0.1); ASSERT_FALSE(monitor.OverCapacityImpl(std::nullopt)); ASSERT_FALSE(monitor.OverCapacityImpl({std::filesystem::space_info{ - /* capacity */ 11, /* free */ 10, /* available */ 1}})); + /* capacity */ 11, /* free */ 10, /* available */ 10}})); ASSERT_TRUE(monitor.OverCapacityImpl( - {std::filesystem::space_info{/* capacity */ 11, /* free */ 9, /* available */ 2}})); + {std::filesystem::space_info{/* capacity */ 11, /* free */ 9, /* available */ 9}})); + ASSERT_TRUE(monitor.OverCapacityImpl( + {std::filesystem::space_info{/* capacity */ 0, /* free */ 0, /* available */ 0}})); } } // namespace ray From ad03dba8355b905c93830b724562c9b7f72b0bc4 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 23:06:45 -0700 Subject: [PATCH 15/48] add --- src/ray/common/ray_config_def.h | 3 +++ src/ray/util/filesystem.cc | 44 ++++++++++++++++++++++++++++++--- src/ray/util/filesystem.h | 18 +++++++++++--- src/ray/util/filesystem_test.cc | 8 +++--- 4 files changed, 62 insertions(+), 11 deletions(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 19e15dadb4d1..723da3efc687 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -487,6 +487,9 @@ RAY_CONFIG(bool, is_external_storage_type_fs, true) /// Once we are over the capacity, all subsequent object creation will fail. RAY_CONFIG(float, local_fs_capacity_threshold, 0.9); +/// Control the frequency of checking the disk usage. +RAY_CONFIG(int64_t, local_fs_monitor_interval_ms, 1000); + /* Configuration parameters for locality-aware scheduling. */ /// Whether to enable locality-aware leasing. If enabled, then Ray will consider task /// dependency locality when choosing a worker for leasing. diff --git a/src/ray/util/filesystem.cc b/src/ray/util/filesystem.cc index 1619efdb56e5..e5ce88ef3819 100644 --- a/src/ray/util/filesystem.cc +++ b/src/ray/util/filesystem.cc @@ -60,10 +60,46 @@ std::string GetUserTempDir() { return result; } -FileSystemMonitor::FileSystemMonitor(const std::string &path, double capacity_threshold) - : ray_file_path_(path), capacity_threshold_(capacity_threshold) {} +FileSystemMonitor::FileSystemMonitor(const std::string &path, + double capacity_threshold, + int64_t monitor_interval_ms) + : ray_file_path_(path), + capacity_threshold_(capacity_threshold), + monitor_interval_ms_(monitor_interval_ms), + mutex_(), + last_check_result_(), + last_check_time_() { + last_check_time_ = std::chrono::steady_clock::now(); + last_check_result_ = SpaceImpl(); +} + +std::optional FileSystemMonitor::Space() { + if (monitor_interval_ms_ == 0) { + return SpaceImpl(); + } + { + absl::ReaderMutexLock lock(&mutex_); + if (std::chrono::duration_cast( + std::chrono::steady_clock::now() - last_check_time_) + .count() < monitor_interval_ms_) { + return last_check_result_; + } + } + + { + absl::WriterMutexLock lock(&mutex_); + if (std::chrono::duration_cast( + std::chrono::steady_clock::now() - last_check_time_) + .count() < monitor_interval_ms_) { + return last_check_result_; + } + last_check_time_ = std::chrono::steady_clock::now(); + last_check_result_ = SpaceImpl(); + return last_check_result_; + } +} -std::optional FileSystemMonitor::Space() const { +std::optional FileSystemMonitor::SpaceImpl() const { std::error_code ec; // TODO: add cache to improve performance. const std::filesystem::space_info si = std::filesystem::space(ray_file_path_, ec); @@ -75,7 +111,7 @@ std::optional FileSystemMonitor::Space() const { return si; } -bool FileSystemMonitor::OverCapacity() const { +bool FileSystemMonitor::OverCapacity() { if (capacity_threshold_ == 1) { return false; } diff --git a/src/ray/util/filesystem.h b/src/ray/util/filesystem.h index e7bdce121715..b9836535c597 100644 --- a/src/ray/util/filesystem.h +++ b/src/ray/util/filesystem.h @@ -21,6 +21,9 @@ #include #include +#include "absl/base/thread_annotations.h" +#include "absl/synchronization/mutex.h" + // Filesystem and path manipulation APIs. // (NTFS stream & attribute paths are not supported.) @@ -66,17 +69,26 @@ std::string JoinPaths(std::string base, const Paths &...components) { /// This class is not thread safe. class FileSystemMonitor { public: - FileSystemMonitor(const std::string &path, double capacity_threshold); - std::optional Space() const; - bool OverCapacity() const; + FileSystemMonitor(const std::string &path, + double capacity_threshold, + int64_t monitor_interval_ms); + std::optional Space(); + bool OverCapacity(); private: // For testing purpose. bool OverCapacityImpl(const std::optional &info) const; + std::optional SpaceImpl() const; + private: FRIEND_TEST(FileSystemTest, TestOverCapacity); const std::string ray_file_path_; const double capacity_threshold_; + const int64_t monitor_interval_ms_; + + mutable absl::Mutex mutex_; + std::optional last_check_result_ GUARDED_BY(mutex_); + std::chrono::time_point last_check_time_ GUARDED_BY(mutex_); }; } // namespace ray diff --git a/src/ray/util/filesystem_test.cc b/src/ray/util/filesystem_test.cc index 95ece311b538..ca3472288925 100644 --- a/src/ray/util/filesystem_test.cc +++ b/src/ray/util/filesystem_test.cc @@ -68,17 +68,17 @@ TEST(FileSystemTest, JoinPathTest) { TEST(FileSystemTest, TestFileSystemMonitor) { std::string tmp_path = std::filesystem::temp_directory_path().string(); { - FileSystemMonitor monitor(tmp_path, 1); + FileSystemMonitor monitor(tmp_path, 1, 0); ASSERT_FALSE(monitor.OverCapacity()); } { - FileSystemMonitor monitor(tmp_path, 0); + FileSystemMonitor monitor(tmp_path, 0, 0); ASSERT_TRUE(monitor.OverCapacity()); } { - FileSystemMonitor monitor(tmp_path, 0); + FileSystemMonitor monitor(tmp_path, 0, 0); auto result = monitor.Space(); ASSERT_TRUE(result.has_value()); ASSERT_TRUE(result->available > 0); @@ -88,7 +88,7 @@ TEST(FileSystemTest, TestFileSystemMonitor) { TEST(FileSystemTest, TestOverCapacity) { std::string tmp_path = std::filesystem::temp_directory_path().string(); - FileSystemMonitor monitor(tmp_path, 0.1); + FileSystemMonitor monitor(tmp_path, 0.1, 0); ASSERT_FALSE(monitor.OverCapacityImpl(std::nullopt)); ASSERT_FALSE(monitor.OverCapacityImpl({std::filesystem::space_info{ /* capacity */ 11, /* free */ 10, /* available */ 10}})); From f47b9771ac1b74a076921895b1b8960117362798 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 23:19:39 -0700 Subject: [PATCH 16/48] add --- src/ray/object_manager/plasma/store_runner.cc | 4 +++- src/ray/util/filesystem.h | 13 +++++++++++-- src/ray/util/filesystem_test.cc | 8 ++++---- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index dedf792afe69..5d7da2bd87e7 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -91,7 +91,9 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, allocator_ = std::make_unique( plasma_directory_, fallback_directory_, hugepages_enabled_, system_memory_); fs_monitor_ = std::make_unique( - fallback_directory_, RayConfig::instance().local_fs_capacity_threshold()); + fallback_directory_, + RayConfig::instance().local_fs_capacity_threshold(), + RayConfig::instance().local_fs_monitor_interval_ms()); store_.reset(new PlasmaStore(main_service_, *allocator_, *fs_monitor_, diff --git a/src/ray/util/filesystem.h b/src/ray/util/filesystem.h index b9836535c597..94ec6c25156b 100644 --- a/src/ray/util/filesystem.h +++ b/src/ray/util/filesystem.h @@ -66,13 +66,22 @@ std::string JoinPaths(std::string base, const Paths &...components) { } /// Monitor the filesystem capacity ray is using. -/// This class is not thread safe. +/// This class is thread safe. class FileSystemMonitor { public: + /// Constructor. + /// + /// \param path path of the file system to monitor the usage. + /// \param capacity_threshold a value between 0-1 indicates the capacity limit. + /// \param monitor_interval_ms control the frequency to check the disk usage. FileSystemMonitor(const std::string &path, double capacity_threshold, - int64_t monitor_interval_ms); + int64_t monitor_interval_ms = 0); + + /// return the disk usage. std::optional Space(); + + /// returns true if the disk usage is over the capacity threshold. bool OverCapacity(); private: diff --git a/src/ray/util/filesystem_test.cc b/src/ray/util/filesystem_test.cc index ca3472288925..95ece311b538 100644 --- a/src/ray/util/filesystem_test.cc +++ b/src/ray/util/filesystem_test.cc @@ -68,17 +68,17 @@ TEST(FileSystemTest, JoinPathTest) { TEST(FileSystemTest, TestFileSystemMonitor) { std::string tmp_path = std::filesystem::temp_directory_path().string(); { - FileSystemMonitor monitor(tmp_path, 1, 0); + FileSystemMonitor monitor(tmp_path, 1); ASSERT_FALSE(monitor.OverCapacity()); } { - FileSystemMonitor monitor(tmp_path, 0, 0); + FileSystemMonitor monitor(tmp_path, 0); ASSERT_TRUE(monitor.OverCapacity()); } { - FileSystemMonitor monitor(tmp_path, 0, 0); + FileSystemMonitor monitor(tmp_path, 0); auto result = monitor.Space(); ASSERT_TRUE(result.has_value()); ASSERT_TRUE(result->available > 0); @@ -88,7 +88,7 @@ TEST(FileSystemTest, TestFileSystemMonitor) { TEST(FileSystemTest, TestOverCapacity) { std::string tmp_path = std::filesystem::temp_directory_path().string(); - FileSystemMonitor monitor(tmp_path, 0.1, 0); + FileSystemMonitor monitor(tmp_path, 0.1); ASSERT_FALSE(monitor.OverCapacityImpl(std::nullopt)); ASSERT_FALSE(monitor.OverCapacityImpl({std::filesystem::space_info{ /* capacity */ 11, /* free */ 10, /* available */ 10}})); From 35332e2fe7060811145af55c588354b9307693d6 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 23:23:29 -0700 Subject: [PATCH 17/48] add --- python/ray/tests/test_out_of_disk_space.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index c2c5f929b073..51cb5ea85418 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -15,6 +15,20 @@ def calculate_capacity_threshold(disk_capacity_in_bytes): return threshold +@pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") +def test_put_fits_in_memory(shutdown_only): + local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) + ray.init( + num_cpus=1, + object_store_memory=80 * 1024 * 1024, + _system_config={ + "local_fs_capacity_threshold": local_fs_capacity_threshold, + }, + ) + arr = np.random.rand(9 * 1024 * 1024) # 160 MB data + ray.put(arr) + + @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_put_out_of_disk(shutdown_only): local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) From f14b8abf08b2808b4f797a5006c714b6be48a788 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 2 Jun 2022 23:44:13 -0700 Subject: [PATCH 18/48] add --- python/ray/tests/test_out_of_disk_space.py | 28 ++++++++++++++++------ 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index 51cb5ea85418..fdfa07f07846 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -1,4 +1,3 @@ -import json import sys import shutil import numpy as np @@ -7,6 +6,8 @@ import pytest import ray +from ray.cluster_utils import Cluster + def calculate_capacity_threshold(disk_capacity_in_bytes): usage = shutil.disk_usage("/tmp") @@ -85,22 +86,35 @@ def foo(): @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_task_args(shutdown_only): - local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) - ray.init( + cluster = Cluster() + cluster.add_node( num_cpus=1, object_store_memory=80 * 1024 * 1024, _system_config={ - "local_fs_capacity_threshold": local_fs_capacity_threshold, + "local_fs_capacity_threshold": 0, }, + resources={"out_of_memory": 1}, + ) + cluster.add_node( + num_cpus=1, + object_store_memory=100 * 1024 * 1024, + resources={"sufficient_memory": 1}, ) + cluster.wait_for_nodes() + ray.init(address=cluster.address) @ray.remote def foo(): - ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 80 MB data + ref = ray.put(np.random.rand(11 * 1024 * 1024)) # 88 MB data return ref - with pytest.raises(ray.exceptions.RayTaskError): - ray.get(foo.remote()) + @ray.remote + def bar(obj): + print(obj) + + ref = foo.options(resources={"sufficient_memory": 1}).remote() + # with pytest.raises(ray.exceptions.RayTaskError): + ray.get(bar.options(resources={"out_of_memory": 1}).remote(ref)) if __name__ == "__main__": From 16bcadb6b4dd323370c5bb6ad338220d263f2798 Mon Sep 17 00:00:00 2001 From: scv119 Date: Fri, 3 Jun 2022 10:04:51 -0700 Subject: [PATCH 19/48] update --- python/ray/tests/test_out_of_disk_space.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index fdfa07f07846..e691a5432258 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -97,7 +97,7 @@ def test_task_args(shutdown_only): ) cluster.add_node( num_cpus=1, - object_store_memory=100 * 1024 * 1024, + object_store_memory=200 * 1024 * 1024, resources={"sufficient_memory": 1}, ) cluster.wait_for_nodes() @@ -105,8 +105,7 @@ def test_task_args(shutdown_only): @ray.remote def foo(): - ref = ray.put(np.random.rand(11 * 1024 * 1024)) # 88 MB data - return ref + return np.random.rand(20 * 1024 * 1024) # 160 MB data @ray.remote def bar(obj): From 8eae0c12e3f36128e04a9e3f1c876ebc6fd0d72c Mon Sep 17 00:00:00 2001 From: scv119 Date: Sat, 4 Jun 2022 23:44:13 -0700 Subject: [PATCH 20/48] add --- python/ray/serialization.py | 5 +++++ python/ray/tests/test_out_of_disk_space.py | 4 ++-- src/ray/object_manager/object_buffer_pool.cc | 4 ++++ src/ray/object_manager/object_manager.cc | 5 ++++- src/ray/object_manager/object_manager.h | 2 +- src/ray/object_manager/pull_manager.cc | 9 +++++++-- src/ray/object_manager/pull_manager.h | 6 ++++-- src/ray/protobuf/common.proto | 2 ++ src/ray/raylet/node_manager.cc | 5 ++--- 9 files changed, 31 insertions(+), 11 deletions(-) diff --git a/python/ray/serialization.py b/python/ray/serialization.py index cce212da93c1..8d50217606e7 100644 --- a/python/ray/serialization.py +++ b/python/ray/serialization.py @@ -21,6 +21,7 @@ ObjectReconstructionFailedError, ObjectReconstructionFailedMaxAttemptsExceededError, ObjectReconstructionFailedLineageEvictedError, + OutOfDiskError, RaySystemError, RuntimeEnvSetupError, TaskPlacementGroupRemoved, @@ -274,6 +275,10 @@ def _deserialize_object(self, data, metadata, object_ref): return ObjectFetchTimedOutError( object_ref.hex(), object_ref.owner_address(), object_ref.call_site() ) + elif error_type == ErrorType.Value("OBJECT_FETCH_OUT_OF_DISK"): + return OutOfDiskError( + object_ref.hex(), object_ref.owner_address(), object_ref.call_site() + ) elif error_type == ErrorType.Value("OBJECT_DELETED"): return ReferenceCountingAssertionError( object_ref.hex(), object_ref.owner_address(), object_ref.call_site() diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index e691a5432258..f5ba1bcf7e2d 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -112,8 +112,8 @@ def bar(obj): print(obj) ref = foo.options(resources={"sufficient_memory": 1}).remote() - # with pytest.raises(ray.exceptions.RayTaskError): - ray.get(bar.options(resources={"out_of_memory": 1}).remote(ref)) + with pytest.raises(ray.exceptions.RayTaskError): + ray.get(bar.options(resources={"out_of_memory": 1}).remote(ref)) if __name__ == "__main__": diff --git a/src/ray/object_manager/object_buffer_pool.cc b/src/ray/object_manager/object_buffer_pool.cc index 69bf4fb0e92c..8004fb588811 100644 --- a/src/ray/object_manager/object_buffer_pool.cc +++ b/src/ray/object_manager/object_buffer_pool.cc @@ -246,6 +246,7 @@ ray::Status ObjectBufferPool::EnsureBufferExists(const ObjectID &object_id, static_cast(metadata_size), &data, plasma::flatbuf::ObjectSource::ReceivedFromRemoteRaylet); + pool_mutex_.Lock(); // No other thread could have created the buffer. @@ -261,6 +262,9 @@ ray::Status ObjectBufferPool::EnsureBufferExists(const ObjectID &object_id, } if (!s.ok()) { + if (s.IsOutOfDisk()) { + return s; + } // Create failed. Buffer creation will be tried by another chunk. // And this chunk will eventually make it here via retried pull requests. return ray::Status::IOError(s.message()); diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 6a23a125a2a6..3b679b705c00 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -65,7 +65,7 @@ ObjectManager::ObjectManager( AddObjectCallback add_object_callback, DeleteObjectCallback delete_object_callback, std::function(const ObjectID &object_id)> pin_object, - const std::function fail_pull_request) + const std::function fail_pull_request) : main_service_(&main_service), self_node_id_(self_node_id), config_(config), @@ -622,6 +622,9 @@ bool ObjectManager::ReceiveObjectChunk(const NodeID &node_id, } else { num_chunks_received_failed_due_to_plasma_++; RAY_LOG(INFO) << "Error receiving chunk:" << chunk_status.message(); + if (chunk_status.IsOutOfDisk()) { + pull_manager_->SetOutOfDisk(object_id); + } return false; } } diff --git a/src/ray/object_manager/object_manager.h b/src/ray/object_manager/object_manager.h index f3e184b1013f..71ea80ee6e44 100644 --- a/src/ray/object_manager/object_manager.h +++ b/src/ray/object_manager/object_manager.h @@ -173,7 +173,7 @@ class ObjectManager : public ObjectManagerInterface, AddObjectCallback add_object_callback, DeleteObjectCallback delete_object_callback, std::function(const ObjectID &object_id)> pin_object, - const std::function fail_pull_request); + const std::function fail_pull_request); ~ObjectManager(); diff --git a/src/ray/object_manager/pull_manager.cc b/src/ray/object_manager/pull_manager.cc index e252afbedc50..13d581789533 100644 --- a/src/ray/object_manager/pull_manager.cc +++ b/src/ray/object_manager/pull_manager.cc @@ -25,7 +25,7 @@ PullManager::PullManager( const std::function object_is_local, const std::function send_pull_request, const std::function cancel_pull_request, - const std::function fail_pull_request, + const std::function fail_pull_request, const RestoreSpilledObjectCallback restore_spilled_object, const std::function get_time_seconds, int pull_timeout_ms, @@ -495,7 +495,7 @@ void PullManager::TryToMakeObjectLocal(const ObjectID &object_id) { RayConfig::instance().fetch_fail_timeout_milliseconds() / 1e3; } else if (get_time_seconds_() > request.expiration_time_seconds) { // Object has no locations and is not being reconstructed by its owner. - fail_pull_request_(object_id); + fail_pull_request_(object_id, rpc::ErrorType::OBJECT_FETCH_TIMED_OUT); request.expiration_time_seconds = 0; } } @@ -782,4 +782,9 @@ std::string PullManager::DebugString() const { return result.str(); } +void PullManager::SetOutOfDisk(const ObjectID &object_id) { + RAY_LOG(DEBUG) << "Pull of object failed due to out of disk: " << object_id; + fail_pull_request_(object_id, rpc::ErrorType::OBJECT_FETCH_OUT_OF_DISK); +} + } // namespace ray diff --git a/src/ray/object_manager/pull_manager.h b/src/ray/object_manager/pull_manager.h index fec96bf6a92a..9f5e4b0368f5 100644 --- a/src/ray/object_manager/pull_manager.h +++ b/src/ray/object_manager/pull_manager.h @@ -64,7 +64,7 @@ class PullManager { const std::function object_is_local, const std::function send_pull_request, const std::function cancel_pull_request, - const std::function fail_pull_request, + const std::function fail_pull_request, const RestoreSpilledObjectCallback restore_spilled_object, const std::function get_time_seconds, int pull_timeout_ms, @@ -165,6 +165,8 @@ class PullManager { /// we are OverQuota(). Visible for testing. int64_t RemainingQuota(); + void SetOutOfDisk(const ObjectID &object_id); + private: /// A helper structure for tracking information about each ongoing object pull. struct ObjectPullRequest { @@ -467,7 +469,7 @@ class PullManager { std::function get_locally_spilled_object_url_; // A callback to fail a hung pull request. - std::function fail_pull_request_; + std::function fail_pull_request_; /// Internally maintained random number generator. std::mt19937_64 gen_; diff --git a/src/ray/protobuf/common.proto b/src/ray/protobuf/common.proto index 5e9d5a8b3604..2d3e62b461da 100644 --- a/src/ray/protobuf/common.proto +++ b/src/ray/protobuf/common.proto @@ -193,6 +193,8 @@ enum ErrorType { TASK_UNSCHEDULABLE_ERROR = 18; // The actor is unschedulable. ACTOR_UNSCHEDULABLE_ERROR = 19; + // We use this error for object fetches that failed due to out of disk. + OBJECT_FETCH_OUT_OF_DISK = 20; } /// The information per ray error type. diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index e151839fb6d5..3e7e46ebfd0f 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -274,11 +274,10 @@ NodeManager::NodeManager(instrumented_io_context &io_service, return result; }, /*fail_pull_request=*/ - [this](const ObjectID &object_id) { + [this](const ObjectID &object_id, rpc::ErrorType error_type) { rpc::ObjectReference ref; ref.set_object_id(object_id.Binary()); - MarkObjectsAsFailed( - rpc::ErrorType::OBJECT_FETCH_TIMED_OUT, {ref}, JobID::Nil()); + MarkObjectsAsFailed(error_type, {ref}, JobID::Nil()); }), periodical_runner_(io_service), report_resources_period_ms_(config.report_resources_period_ms), From e44f51941acce9cea2274ae6c5dc95889d3aae85 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 00:43:53 -0700 Subject: [PATCH 21/48] add --- BUILD.bazel | 1 + src/ray/util/filesystem.cc | 77 --------------------------------- src/ray/util/filesystem.h | 40 ----------------- src/ray/util/filesystem_test.cc | 21 +++++---- 4 files changed, 14 insertions(+), 125 deletions(-) diff --git a/BUILD.bazel b/BUILD.bazel index c1440c1f35b7..d03fc2843ecb 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -1432,6 +1432,7 @@ cc_test( tags = ["team:core"], deps = [ ":ray_util", + ":ray_common", "@com_google_googletest//:gtest_main", ], ) diff --git a/src/ray/util/filesystem.cc b/src/ray/util/filesystem.cc index e5ce88ef3819..4e1c119e54ce 100644 --- a/src/ray/util/filesystem.cc +++ b/src/ray/util/filesystem.cc @@ -60,81 +60,4 @@ std::string GetUserTempDir() { return result; } -FileSystemMonitor::FileSystemMonitor(const std::string &path, - double capacity_threshold, - int64_t monitor_interval_ms) - : ray_file_path_(path), - capacity_threshold_(capacity_threshold), - monitor_interval_ms_(monitor_interval_ms), - mutex_(), - last_check_result_(), - last_check_time_() { - last_check_time_ = std::chrono::steady_clock::now(); - last_check_result_ = SpaceImpl(); -} - -std::optional FileSystemMonitor::Space() { - if (monitor_interval_ms_ == 0) { - return SpaceImpl(); - } - { - absl::ReaderMutexLock lock(&mutex_); - if (std::chrono::duration_cast( - std::chrono::steady_clock::now() - last_check_time_) - .count() < monitor_interval_ms_) { - return last_check_result_; - } - } - - { - absl::WriterMutexLock lock(&mutex_); - if (std::chrono::duration_cast( - std::chrono::steady_clock::now() - last_check_time_) - .count() < monitor_interval_ms_) { - return last_check_result_; - } - last_check_time_ = std::chrono::steady_clock::now(); - last_check_result_ = SpaceImpl(); - return last_check_result_; - } -} - -std::optional FileSystemMonitor::SpaceImpl() const { - std::error_code ec; - // TODO: add cache to improve performance. - const std::filesystem::space_info si = std::filesystem::space(ray_file_path_, ec); - if (ec) { - RAY_LOG_EVERY_MS(WARNING, 60 * 1000) << "Failed to get capacity of " << ray_file_path_ - << " with error: " << ec.message(); - return std::nullopt; - } - return si; -} - -bool FileSystemMonitor::OverCapacity() { - if (capacity_threshold_ == 1) { - return false; - } - return OverCapacityImpl(Space()); -} - -bool FileSystemMonitor::OverCapacityImpl( - const std::optional &space_info) const { - if (!space_info.has_value()) { - return false; - } - if (space_info->capacity <= 0) { - RAY_LOG_EVERY_MS(ERROR, 60 * 1000) << ray_file_path_ << " has no capacity."; - return true; - } - - if ((1 - 1.0f * space_info->available / space_info->capacity) < capacity_threshold_) { - return false; - } - - RAY_LOG_EVERY_MS(ERROR, 10 * 1000) - << ray_file_path_ << " is over capacity, available: " << space_info->available - << ", capacity: " << space_info->capacity << ", threshold: " << capacity_threshold_; - return true; -} } // namespace ray diff --git a/src/ray/util/filesystem.h b/src/ray/util/filesystem.h index 94ec6c25156b..bb36ef5b6cbc 100644 --- a/src/ray/util/filesystem.h +++ b/src/ray/util/filesystem.h @@ -14,16 +14,11 @@ #pragma once -#include - #include #include #include #include -#include "absl/base/thread_annotations.h" -#include "absl/synchronization/mutex.h" - // Filesystem and path manipulation APIs. // (NTFS stream & attribute paths are not supported.) @@ -65,39 +60,4 @@ std::string JoinPaths(std::string base, const Paths &...components) { return base; } -/// Monitor the filesystem capacity ray is using. -/// This class is thread safe. -class FileSystemMonitor { - public: - /// Constructor. - /// - /// \param path path of the file system to monitor the usage. - /// \param capacity_threshold a value between 0-1 indicates the capacity limit. - /// \param monitor_interval_ms control the frequency to check the disk usage. - FileSystemMonitor(const std::string &path, - double capacity_threshold, - int64_t monitor_interval_ms = 0); - - /// return the disk usage. - std::optional Space(); - - /// returns true if the disk usage is over the capacity threshold. - bool OverCapacity(); - - private: - // For testing purpose. - bool OverCapacityImpl(const std::optional &info) const; - - std::optional SpaceImpl() const; - - private: - FRIEND_TEST(FileSystemTest, TestOverCapacity); - const std::string ray_file_path_; - const double capacity_threshold_; - const int64_t monitor_interval_ms_; - - mutable absl::Mutex mutex_; - std::optional last_check_result_ GUARDED_BY(mutex_); - std::chrono::time_point last_check_time_ GUARDED_BY(mutex_); -}; } // namespace ray diff --git a/src/ray/util/filesystem_test.cc b/src/ray/util/filesystem_test.cc index 95ece311b538..29ac1bf74881 100644 --- a/src/ray/util/filesystem_test.cc +++ b/src/ray/util/filesystem_test.cc @@ -17,6 +17,7 @@ #include #include "gtest/gtest.h" +#include "ray/common/file_system_monitor.h" namespace ray { @@ -68,18 +69,18 @@ TEST(FileSystemTest, JoinPathTest) { TEST(FileSystemTest, TestFileSystemMonitor) { std::string tmp_path = std::filesystem::temp_directory_path().string(); { - FileSystemMonitor monitor(tmp_path, 1); + ray::FileSystemMonitor monitor({tmp_path}, 1); ASSERT_FALSE(monitor.OverCapacity()); } { - FileSystemMonitor monitor(tmp_path, 0); + FileSystemMonitor monitor({tmp_path}, 0); ASSERT_TRUE(monitor.OverCapacity()); } { - FileSystemMonitor monitor(tmp_path, 0); - auto result = monitor.Space(); + FileSystemMonitor monitor({tmp_path}, 0); + auto result = monitor.Space(tmp_path); ASSERT_TRUE(result.has_value()); ASSERT_TRUE(result->available > 0); ASSERT_TRUE(result->capacity > 0); @@ -88,13 +89,17 @@ TEST(FileSystemTest, TestFileSystemMonitor) { TEST(FileSystemTest, TestOverCapacity) { std::string tmp_path = std::filesystem::temp_directory_path().string(); - FileSystemMonitor monitor(tmp_path, 0.1); - ASSERT_FALSE(monitor.OverCapacityImpl(std::nullopt)); - ASSERT_FALSE(monitor.OverCapacityImpl({std::filesystem::space_info{ - /* capacity */ 11, /* free */ 10, /* available */ 10}})); + FileSystemMonitor monitor({tmp_path}, 0.1); + ASSERT_FALSE(monitor.OverCapacityImpl(tmp_path, std::nullopt)); + ASSERT_FALSE(monitor.OverCapacityImpl( + tmp_path, + {std::filesystem::space_info{ + /* capacity */ 11, /* free */ 10, /* available */ 10}})); ASSERT_TRUE(monitor.OverCapacityImpl( + tmp_path, {std::filesystem::space_info{/* capacity */ 11, /* free */ 9, /* available */ 9}})); ASSERT_TRUE(monitor.OverCapacityImpl( + tmp_path, {std::filesystem::space_info{/* capacity */ 0, /* free */ 0, /* available */ 0}})); } From 7fcc50c841000f7859ad8ef6a68f1509bc7e8021 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 00:48:08 -0700 Subject: [PATCH 22/48] update --- src/ray/common/ray_config_def.h | 2 +- src/ray/object_manager/plasma/create_request_queue.h | 2 +- src/ray/object_manager/plasma/store.h | 2 +- src/ray/object_manager/plasma/store_runner.cc | 2 +- src/ray/object_manager/plasma/store_runner.h | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 723da3efc687..79ef5e2b64bb 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -488,7 +488,7 @@ RAY_CONFIG(bool, is_external_storage_type_fs, true) RAY_CONFIG(float, local_fs_capacity_threshold, 0.9); /// Control the frequency of checking the disk usage. -RAY_CONFIG(int64_t, local_fs_monitor_interval_ms, 1000); +RAY_CONFIG(uint64_t, local_fs_monitor_interval_ms, 1000); /* Configuration parameters for locality-aware scheduling. */ /// Whether to enable locality-aware leasing. If enabled, then Ray will consider task diff --git a/src/ray/object_manager/plasma/create_request_queue.h b/src/ray/object_manager/plasma/create_request_queue.h index 526c1054597a..d692db82710f 100644 --- a/src/ray/object_manager/plasma/create_request_queue.h +++ b/src/ray/object_manager/plasma/create_request_queue.h @@ -19,13 +19,13 @@ #include #include "absl/container/flat_hash_map.h" +#include "ray/common/file_system_monitor.h" #include "ray/common/status.h" #include "ray/object_manager/common.h" #include "ray/object_manager/plasma/common.h" #include "ray/object_manager/plasma/connection.h" #include "ray/object_manager/plasma/plasma.h" #include "ray/object_manager/plasma/protocol.h" -#include "ray/util/filesystem.h" namespace plasma { diff --git a/src/ray/object_manager/plasma/store.h b/src/ray/object_manager/plasma/store.h index fba175ab051e..34ca1e50d245 100644 --- a/src/ray/object_manager/plasma/store.h +++ b/src/ray/object_manager/plasma/store.h @@ -27,6 +27,7 @@ #include "absl/base/thread_annotations.h" #include "absl/synchronization/mutex.h" #include "ray/common/asio/instrumented_io_context.h" +#include "ray/common/file_system_monitor.h" #include "ray/common/ray_config.h" #include "ray/common/status.h" #include "ray/object_manager/common.h" @@ -40,7 +41,6 @@ #include "ray/object_manager/plasma/plasma.h" #include "ray/object_manager/plasma/plasma_allocator.h" #include "ray/object_manager/plasma/protocol.h" -#include "ray/util/filesystem.h" namespace plasma { diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index 5d7da2bd87e7..2d091a5dc09a 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -91,7 +91,7 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, allocator_ = std::make_unique( plasma_directory_, fallback_directory_, hugepages_enabled_, system_memory_); fs_monitor_ = std::make_unique( - fallback_directory_, + {fallback_directory_}, RayConfig::instance().local_fs_capacity_threshold(), RayConfig::instance().local_fs_monitor_interval_ms()); store_.reset(new PlasmaStore(main_service_, diff --git a/src/ray/object_manager/plasma/store_runner.h b/src/ray/object_manager/plasma/store_runner.h index 272d3be15c0f..22e31b014684 100644 --- a/src/ray/object_manager/plasma/store_runner.h +++ b/src/ray/object_manager/plasma/store_runner.h @@ -5,9 +5,9 @@ #include "absl/synchronization/mutex.h" #include "ray/common/asio/instrumented_io_context.h" +#include "ray/common/file_system_monitor.h" #include "ray/object_manager/plasma/plasma_allocator.h" #include "ray/object_manager/plasma/store.h" -#include "ray/util/filesystem.h" namespace plasma { From 9068b57685d690ad821d743b3ec3bc59b7f836aa Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 01:07:04 -0700 Subject: [PATCH 23/48] add --- python/ray/tests/BUILD | 1 + python/ray/tests/test_out_of_disk_space.py | 52 +++++++++++++++++++ src/ray/object_manager/plasma/store_runner.cc | 2 +- 3 files changed, 54 insertions(+), 1 deletion(-) diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 1a8286330f84..7c6db1064656 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -251,6 +251,7 @@ py_test_module_list( files = [ "test_basic_3.py", "test_output.py", + "test_out_of_disk_space.py", "test_failure_4.py", "test_object_spilling.py", "test_object_spilling_no_asan.py", diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index f5ba1bcf7e2d..98c508776c10 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -116,5 +116,57 @@ def bar(obj): ray.get(bar.options(resources={"out_of_memory": 1}).remote(ref)) +@pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") +def test_actor(shutdown_only): + cluster = Cluster() + cluster.add_node( + num_cpus=1, + object_store_memory=80 * 1024 * 1024, + _system_config={ + "local_fs_capacity_threshold": 0, + }, + resources={"out_of_memory": 1}, + ) + cluster.add_node( + num_cpus=1, + object_store_memory=200 * 1024 * 1024, + resources={"sufficient_memory": 1}, + ) + cluster.wait_for_nodes() + ray.init(address=cluster.address) + + @ray.remote + def foo(): + return np.random.rand(20 * 1024 * 1024) # 160 MB data + + @ray.remote + class Actor: + def __init__(self, obj): + self._obj = obj + + def foo(self): + print(self._obj) + + def args_ood(self, obj): + print(obj) + + def return_ood(self): + return np.random.rand(20 * 1024 * 1024) + + ref = foo.options(resources={"sufficient_memory": 1}).remote() + with pytest.raises(ray.exceptions.RayActorError): + a = Actor.options(resources={"out_of_memory": 0.001}).remote(ref) + ray.get(a.foo.remote()) + + a = Actor.options(resources={"out_of_memory": 1}).remote(1) + ray.get(a.foo.remote()) + with pytest.raises(ray.exceptions.RayTaskError): + ray.get(a.args_ood.remote(ref)) + + ray.get(a.foo.remote()) + with pytest.raises(ray.exceptions.RayTaskError): + ray.get(a.return_ood.remote()) + + if __name__ == "__main__": sys.exit(pytest.main(["-sv", __file__])) diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index 2d091a5dc09a..7cd6faa4eeb0 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -91,7 +91,7 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, allocator_ = std::make_unique( plasma_directory_, fallback_directory_, hugepages_enabled_, system_memory_); fs_monitor_ = std::make_unique( - {fallback_directory_}, + std::vector{fallback_directory_}, RayConfig::instance().local_fs_capacity_threshold(), RayConfig::instance().local_fs_monitor_interval_ms()); store_.reset(new PlasmaStore(main_service_, From 79073d9f02d748147209dd6928c81f42bed3bb40 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 01:51:46 -0700 Subject: [PATCH 24/48] support spilled path --- python/ray/node.py | 18 ++++++++++++++++++ src/ray/common/ray_config_def.h | 4 ++++ src/ray/object_manager/plasma/store_runner.cc | 10 +++++++++- 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/python/ray/node.py b/python/ray/node.py index 3d2477736a0a..5dcadc45f623 100644 --- a/python/ray/node.py +++ b/python/ray/node.py @@ -1489,6 +1489,24 @@ def validate_external_storage(self): ] = is_external_storage_type_fs self._config["is_external_storage_type_fs"] = is_external_storage_type_fs + if is_external_storage_type_fs: + directory_paths = [] + try: + directory_path = deserialized_config["params"]["directory_path"] + if type(directory_path) == list: + directory_paths = directory_path + else: + directory_paths.append(directory_path) + except KeyError: + logger.warning( + f"Failed to read spilling path from {deserialized_config}" + ) + local_spilling_paths = ",".join(directory_path) + self._ray_params._system_config[ + "local_spilling_paths" + ] = local_spilling_paths + self._config["local_spilling_paths"] = local_spilling_paths + # Validate external storage usage. from ray import external_storage diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 79ef5e2b64bb..20126e7ef586 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -483,6 +483,10 @@ RAY_CONFIG(int64_t, oom_grace_period_s, 2) /// This is configured based on object_spilling_config. RAY_CONFIG(bool, is_external_storage_type_fs, true) +/// Directories used for local file system spilling, separated by comma. +/// This is configured based on object_spilling_config. +RAY_CONFIG(std::string, local_spilling_paths, "") + /// Control the capacity threshold for ray local file system (for object store). /// Once we are over the capacity, all subsequent object creation will fail. RAY_CONFIG(float, local_fs_capacity_threshold, 0.9); diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index 7cd6faa4eeb0..cfc3b438a726 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -1,5 +1,6 @@ #include "ray/object_manager/plasma/store_runner.h" +#include #ifndef _WIN32 #include #include @@ -90,8 +91,15 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, absl::MutexLock lock(&store_runner_mutex_); allocator_ = std::make_unique( plasma_directory_, fallback_directory_, hugepages_enabled_, system_memory_); + std::vector local_spilling_paths; + if (!RayConfig::instance().local_spilling_paths().empty()) { + boost::split(local_spilling_paths, + RayConfig::instance().local_spilling_paths(), + boost::is_any_of(",")); + } + local_spilling_paths.push_back(fallback_directory_); fs_monitor_ = std::make_unique( - std::vector{fallback_directory_}, + local_spilling_paths, RayConfig::instance().local_fs_capacity_threshold(), RayConfig::instance().local_fs_monitor_interval_ms()); store_.reset(new PlasmaStore(main_service_, From f53ff89433721251ac13e5a7ff2b1a2f2c95687c Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 02:41:19 -0700 Subject: [PATCH 25/48] add --- python/ray/node.py | 18 ----- src/ray/common/ray_config_def.h | 4 -- src/ray/object_manager/plasma/store_runner.cc | 8 +-- src/ray/util/filesystem.h | 1 + src/ray/util/filesystem_test.cc | 69 +++++++++++++++++++ 5 files changed, 73 insertions(+), 27 deletions(-) diff --git a/python/ray/node.py b/python/ray/node.py index 5dcadc45f623..3d2477736a0a 100644 --- a/python/ray/node.py +++ b/python/ray/node.py @@ -1489,24 +1489,6 @@ def validate_external_storage(self): ] = is_external_storage_type_fs self._config["is_external_storage_type_fs"] = is_external_storage_type_fs - if is_external_storage_type_fs: - directory_paths = [] - try: - directory_path = deserialized_config["params"]["directory_path"] - if type(directory_path) == list: - directory_paths = directory_path - else: - directory_paths.append(directory_path) - except KeyError: - logger.warning( - f"Failed to read spilling path from {deserialized_config}" - ) - local_spilling_paths = ",".join(directory_path) - self._ray_params._system_config[ - "local_spilling_paths" - ] = local_spilling_paths - self._config["local_spilling_paths"] = local_spilling_paths - # Validate external storage usage. from ray import external_storage diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 20126e7ef586..79ef5e2b64bb 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -483,10 +483,6 @@ RAY_CONFIG(int64_t, oom_grace_period_s, 2) /// This is configured based on object_spilling_config. RAY_CONFIG(bool, is_external_storage_type_fs, true) -/// Directories used for local file system spilling, separated by comma. -/// This is configured based on object_spilling_config. -RAY_CONFIG(std::string, local_spilling_paths, "") - /// Control the capacity threshold for ray local file system (for object store). /// Once we are over the capacity, all subsequent object creation will fail. RAY_CONFIG(float, local_fs_capacity_threshold, 0.9); diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index cfc3b438a726..c65411a6d0a6 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -1,6 +1,5 @@ #include "ray/object_manager/plasma/store_runner.h" -#include #ifndef _WIN32 #include #include @@ -92,10 +91,9 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, allocator_ = std::make_unique( plasma_directory_, fallback_directory_, hugepages_enabled_, system_memory_); std::vector local_spilling_paths; - if (!RayConfig::instance().local_spilling_paths().empty()) { - boost::split(local_spilling_paths, - RayConfig::instance().local_spilling_paths(), - boost::is_any_of(",")); + if (RayConfig::instance().is_external_storage_type_fs()) { + local_spilling_paths = + ray::ParseSpillingPaths(RayConfig::instance().object_spilling_config()); } local_spilling_paths.push_back(fallback_directory_); fs_monitor_ = std::make_unique( diff --git a/src/ray/util/filesystem.h b/src/ray/util/filesystem.h index bb36ef5b6cbc..e4dcdd5f71c8 100644 --- a/src/ray/util/filesystem.h +++ b/src/ray/util/filesystem.h @@ -60,4 +60,5 @@ std::string JoinPaths(std::string base, const Paths &...components) { return base; } +std::vector GetLocalSpillingPaths(); } // namespace ray diff --git a/src/ray/util/filesystem_test.cc b/src/ray/util/filesystem_test.cc index 29ac1bf74881..e7d1480a52b0 100644 --- a/src/ray/util/filesystem_test.cc +++ b/src/ray/util/filesystem_test.cc @@ -103,6 +103,75 @@ TEST(FileSystemTest, TestOverCapacity) { {std::filesystem::space_info{/* capacity */ 0, /* free */ 0, /* available */ 0}})); } +TEST(FileSystemTest, ParseLocalSpillingPaths) { + { + std::vector expected{"/tmp/spill", "/tmp/spill_1"}; + auto parsed = ParseSpillingPaths( + "{" + " \"type\": \"filesystem\"," + " \"params\": {" + " \"directory_path\": [" + " \"/tmp/spill\"," + " \"/tmp/spill_1\"" + " ]" + " }" + "}"); + ASSERT_EQ(expected, parsed); + } + + { + std::vector expected{"/tmp/spill"}; + auto parsed = ParseSpillingPaths( + "{" + " \"type\": \"filesystem\"," + " \"params\": {" + " \"directory_path\": \"/tmp/spill\"" + " }" + "}"); + ASSERT_EQ(expected, parsed); + } + + { + std::vector expected{}; + auto parsed = ParseSpillingPaths( + "{" + " \"type\": \"filesystem\"," + " \"params\": {" + " \"directory_1path\": \"/tmp/spill\"" + " }" + "}"); + ASSERT_EQ(expected, parsed); + } + + { + std::vector expected{}; + auto parsed = ParseSpillingPaths( + "{" + " \"type\": \"filesystem\"," + " \"params\": {" + " \"directory_path\": 3" + " }" + "}"); + ASSERT_EQ(expected, parsed); + } + + { + std::vector expected{"/tmp/spill", "/tmp/spill_1"}; + auto parsed = ParseSpillingPaths( + "{" + " \"type\": \"filesystem\"," + " \"params\": {" + " \"directory_path\": [" + " \"/tmp/spill\"," + " 2," + " \"/tmp/spill_1\"" + " ]" + " }" + "}"); + ASSERT_EQ(expected, parsed); + } +} + } // namespace ray int main(int argc, char **argv) { From 08a47c2575925748be2f28cdb36142e4c0fc32bf Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 21:20:28 -0700 Subject: [PATCH 26/48] fix-build --- src/ray/common/file_system_monitor.cc | 132 ++++++++++++++++++++++++++ src/ray/common/file_system_monitor.h | 68 +++++++++++++ 2 files changed, 200 insertions(+) create mode 100644 src/ray/common/file_system_monitor.cc create mode 100644 src/ray/common/file_system_monitor.h diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc new file mode 100644 index 000000000000..6be2c9f26c0a --- /dev/null +++ b/src/ray/common/file_system_monitor.cc @@ -0,0 +1,132 @@ +// Copyright 2020 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/common/file_system_monitor.h" +#include "ray/util/logging.h" + +#include "nlohmann/json.hpp" + +using json = nlohmann::json; + + +namespace ray { + +FileSystemMonitor::FileSystemMonitor(std::vector paths, + double capacity_threshold, + uint64_t monitor_interval_ms) + : paths_(std::move(paths)), + capacity_threshold_(capacity_threshold), + over_capacity_(CheckIfAnyPathOverCapacity()), + io_context_(), + monitor_thread_([this] { + /// The asio work to keep io_contex_ alive. + boost::asio::io_service::work io_service_work_(io_context_); + io_context_.run(); + }), + runner_(io_context_) { + runner_.RunFnPeriodically([this] { over_capacity_ = CheckIfAnyPathOverCapacity(); }, + monitor_interval_ms, + "FileSystemMonitor.CheckIfAnyPathOverCapacity"); +} + + +FileSystemMonitor::~FileSystemMonitor() { + io_context_.stop(); + if (monitor_thread_.joinable()) { + monitor_thread_.join(); + } +} + +std::optional FileSystemMonitor::Space( + const std::string &path) const { + std::error_code ec; + const std::filesystem::space_info si = std::filesystem::space(path, ec); + if (ec) { + RAY_LOG_EVERY_MS(WARNING, 60 * 1000) + << "Failed to get capacity of " << path << " with error: " << ec.message(); + return std::nullopt; + } + return si; +} + +bool FileSystemMonitor::OverCapacity() const { return over_capacity_.load(); } + +bool FileSystemMonitor::CheckIfAnyPathOverCapacity() const { + if (paths_.empty()) { + return false; + } + + if (capacity_threshold_ == 0) { + return true; + } + + if (capacity_threshold_ >= 1) { + return false; + } + + for (auto &path : paths_) { + if(OverCapacityImpl(path, Space(path))) { + return true; + } + } + return false; +} + +bool FileSystemMonitor::OverCapacityImpl( + const std::string& path, const std::optional &space_info) const { + if (!space_info.has_value()) { + return false; + } + if (space_info->capacity <= 0) { + RAY_LOG_EVERY_MS(ERROR, 60 * 1000) << path << " has no capacity."; + return true; + } + + if ((1 - 1.0f * space_info->available / space_info->capacity) < capacity_threshold_) { + return false; + } + + RAY_LOG_EVERY_MS(ERROR, 10 * 1000) + << path << " is over capacity, available: " << space_info->available + << ", capacity: " << space_info->capacity << ", threshold: " << capacity_threshold_; + return true; +} + +std::vector ParseSpillingPaths(const std::string& spilling_config) { + std::vector spilling_paths; + + try { + json spill_config = json::parse(spilling_config); + auto &directory_path = spill_config.at("params").at("directory_path"); + if (directory_path.is_string()) { + spilling_paths.push_back(directory_path); + } else if (directory_path.is_array()) { + for (auto &entry : directory_path) { + if (entry.is_string()) { + spilling_paths.push_back(entry); + } else { + RAY_LOG(ERROR) << "Failed to parse spilling path: " << entry; + } + } + } else { + RAY_LOG(ERROR) << "Failed to parse spilling path: " << directory_path; + } + } catch (json::exception &ex) { + RAY_LOG(ERROR) << "Failed to load spilling config: " << ex.what() + << " The config string is: " + << spilling_config; + } + return spilling_paths; +} +} // namespace ray diff --git a/src/ray/common/file_system_monitor.h b/src/ray/common/file_system_monitor.h new file mode 100644 index 000000000000..0c77c04e206b --- /dev/null +++ b/src/ray/common/file_system_monitor.h @@ -0,0 +1,68 @@ +// Copyright 2020 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +#include +#include +#include +#include +#include + +#include "ray/common/asio/instrumented_io_context.h" +#include "ray/common/asio/periodical_runner.h" + + +namespace ray { +/// Monitor the filesystem capacity ray is using. +/// This class is thread safe. +class FileSystemMonitor { + public: + /// Constructor. + /// + /// \param paths paths of the file system to monitor the usage. + /// \param capacity_threshold a value between 0-1 indicates the capacity limit. + /// \param monitor_interval_ms control the frequency to check the disk usage. + FileSystemMonitor(std::vector paths, + double capacity_threshold, + uint64_t monitor_interval_ms = 1000); + + ~FileSystemMonitor(); + + /// return the disk usage. + std::optional Space(const std::string &path) const; + + /// returns true if ANY path's disk usage is over the capacity threshold. + bool OverCapacity() const; + + private: + bool CheckIfAnyPathOverCapacity() const; + // For testing purpose. + bool OverCapacityImpl(const std::string& path, const std::optional &info) const; + + private: + FRIEND_TEST(FileSystemTest, TestOverCapacity); + const std::vector paths_; + const double capacity_threshold_; + std::atomic over_capacity_; + instrumented_io_context io_context_; + std::thread monitor_thread_; + PeriodicalRunner runner_; +}; + + +std::vector ParseSpillingPaths(const std::string& spilling_config); +} // namespace ray From 08eb12a05a7f7b37ca3f7484d05c5f7cfd41d6f6 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 21:23:21 -0700 Subject: [PATCH 27/48] linter --- src/ray/common/file_system_monitor.cc | 16 +++++++--------- src/ray/common/file_system_monitor.h | 9 ++++----- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc index 6be2c9f26c0a..8892b5241f71 100644 --- a/src/ray/common/file_system_monitor.cc +++ b/src/ray/common/file_system_monitor.cc @@ -13,13 +13,12 @@ // limitations under the License. #include "ray/common/file_system_monitor.h" -#include "ray/util/logging.h" #include "nlohmann/json.hpp" +#include "ray/util/logging.h" using json = nlohmann::json; - namespace ray { FileSystemMonitor::FileSystemMonitor(std::vector paths, @@ -40,7 +39,6 @@ FileSystemMonitor::FileSystemMonitor(std::vector paths, "FileSystemMonitor.CheckIfAnyPathOverCapacity"); } - FileSystemMonitor::~FileSystemMonitor() { io_context_.stop(); if (monitor_thread_.joinable()) { @@ -70,13 +68,13 @@ bool FileSystemMonitor::CheckIfAnyPathOverCapacity() const { if (capacity_threshold_ == 0) { return true; } - + if (capacity_threshold_ >= 1) { return false; } for (auto &path : paths_) { - if(OverCapacityImpl(path, Space(path))) { + if (OverCapacityImpl(path, Space(path))) { return true; } } @@ -84,7 +82,8 @@ bool FileSystemMonitor::CheckIfAnyPathOverCapacity() const { } bool FileSystemMonitor::OverCapacityImpl( - const std::string& path, const std::optional &space_info) const { + const std::string &path, + const std::optional &space_info) const { if (!space_info.has_value()) { return false; } @@ -103,7 +102,7 @@ bool FileSystemMonitor::OverCapacityImpl( return true; } -std::vector ParseSpillingPaths(const std::string& spilling_config) { +std::vector ParseSpillingPaths(const std::string &spilling_config) { std::vector spilling_paths; try { @@ -124,8 +123,7 @@ std::vector ParseSpillingPaths(const std::string& spilling_config) } } catch (json::exception &ex) { RAY_LOG(ERROR) << "Failed to load spilling config: " << ex.what() - << " The config string is: " - << spilling_config; + << " The config string is: " << spilling_config; } return spilling_paths; } diff --git a/src/ray/common/file_system_monitor.h b/src/ray/common/file_system_monitor.h index 0c77c04e206b..e5d8df59e72a 100644 --- a/src/ray/common/file_system_monitor.h +++ b/src/ray/common/file_system_monitor.h @@ -19,13 +19,12 @@ #include #include #include -#include #include +#include #include "ray/common/asio/instrumented_io_context.h" #include "ray/common/asio/periodical_runner.h" - namespace ray { /// Monitor the filesystem capacity ray is using. /// This class is thread safe. @@ -51,7 +50,8 @@ class FileSystemMonitor { private: bool CheckIfAnyPathOverCapacity() const; // For testing purpose. - bool OverCapacityImpl(const std::string& path, const std::optional &info) const; + bool OverCapacityImpl(const std::string &path, + const std::optional &info) const; private: FRIEND_TEST(FileSystemTest, TestOverCapacity); @@ -63,6 +63,5 @@ class FileSystemMonitor { PeriodicalRunner runner_; }; - -std::vector ParseSpillingPaths(const std::string& spilling_config); +std::vector ParseSpillingPaths(const std::string &spilling_config); } // namespace ray From 911fe2e28ab9d253d1593574e906bdc96335f5b0 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 21:58:23 -0700 Subject: [PATCH 28/48] update --- src/ray/util/filesystem.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/ray/util/filesystem.h b/src/ray/util/filesystem.h index e4dcdd5f71c8..5f15d055368e 100644 --- a/src/ray/util/filesystem.h +++ b/src/ray/util/filesystem.h @@ -59,6 +59,4 @@ std::string JoinPaths(std::string base, const Paths &...components) { (join(base, std::string_view(components)), ...); return base; } - -std::vector GetLocalSpillingPaths(); } // namespace ray From cf986073edf82f306f4c88c091da2c86443e0aa4 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 5 Jun 2022 23:44:35 -0700 Subject: [PATCH 29/48] fix-ci --- src/ray/object_manager/test/create_request_queue_test.cc | 8 ++++---- src/ray/object_manager/test/pull_manager_test.cc | 4 +++- src/ray/protobuf/common.proto | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/ray/object_manager/test/create_request_queue_test.cc b/src/ray/object_manager/test/create_request_queue_test.cc index b4863e631e8a..d2ea80a39203 100644 --- a/src/ray/object_manager/test/create_request_queue_test.cc +++ b/src/ray/object_manager/test/create_request_queue_test.cc @@ -53,7 +53,7 @@ class CreateRequestQueueTest : public ::testing::Test { CreateRequestQueueTest() : oom_grace_period_s_(1), current_time_ns_(0), - monitor_("/", 1), + monitor_({"/"}, 1), queue_( monitor_, /*oom_grace_period_s=*/oom_grace_period_s_, @@ -184,7 +184,7 @@ TEST_F(CreateRequestQueueTest, TestFallbackAllocator) { TEST(CreateRequestQueueParameterTest, TestOomInfiniteRetry) { int num_global_gc_ = 0; int64_t current_time_ns; - ray::FileSystemMonitor monitor{"/", 1}; + ray::FileSystemMonitor monitor{{"/"}, 1}; CreateRequestQueue queue( monitor, /*oom_grace_period_s=*/100, @@ -218,7 +218,7 @@ TEST(CreateRequestQueueParameterTest, TestOomInfiniteRetry) { } TEST_F(CreateRequestQueueTest, TestTransientOom) { - ray::FileSystemMonitor monitor{"/", 1}; + ray::FileSystemMonitor monitor{{"/"}, 1}; CreateRequestQueue queue( monitor, /*oom_grace_period_s=*/oom_grace_period_s_, @@ -378,7 +378,7 @@ TEST_F(CreateRequestQueueTest, TestTransientOomThenOom) { TEST(CreateRequestQueueParameterTest, TestNoEvictIfFull) { int64_t current_time_ns = 0; - ray::FileSystemMonitor monitor{"/", 1}; + ray::FileSystemMonitor monitor{{"/"}, 1}; CreateRequestQueue queue( monitor, /*oom_grace_period_s=*/1, diff --git a/src/ray/object_manager/test/pull_manager_test.cc b/src/ray/object_manager/test/pull_manager_test.cc index 8fa7fe07d2aa..60b0c3df9054 100644 --- a/src/ray/object_manager/test/pull_manager_test.cc +++ b/src/ray/object_manager/test/pull_manager_test.cc @@ -38,7 +38,9 @@ class PullManagerTestWithCapacity { num_send_pull_request_calls_++; }, [this](const ObjectID &object_id) { num_abort_calls_[object_id]++; }, - [this](const ObjectID &object_id) { timed_out_objects_.insert(object_id); }, + [this](const ObjectID &object_id, rpc::ErrorType) { + timed_out_objects_.insert(object_id); + }, [this](const ObjectID &, int64_t size, const std::string &, diff --git a/src/ray/protobuf/common.proto b/src/ray/protobuf/common.proto index 2d3e62b461da..1756db0bedee 100644 --- a/src/ray/protobuf/common.proto +++ b/src/ray/protobuf/common.proto @@ -193,7 +193,7 @@ enum ErrorType { TASK_UNSCHEDULABLE_ERROR = 18; // The actor is unschedulable. ACTOR_UNSCHEDULABLE_ERROR = 19; - // We use this error for object fetches that failed due to out of disk. + // We use this error for object fetches that failed due to out of disk. OBJECT_FETCH_OUT_OF_DISK = 20; } From b526d792127968e54ba2dddcbbd8c2a066df637e Mon Sep 17 00:00:00 2001 From: scv119 Date: Tue, 7 Jun 2022 09:46:45 -0700 Subject: [PATCH 30/48] update --- python/ray/tests/test_out_of_disk_space.py | 24 ++++--------------- .../plasma/create_request_queue.cc | 13 ++++------ .../plasma/create_request_queue.h | 2 -- 3 files changed, 9 insertions(+), 30 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index 98c508776c10..41993d388cf9 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -16,20 +16,6 @@ def calculate_capacity_threshold(disk_capacity_in_bytes): return threshold -@pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") -def test_put_fits_in_memory(shutdown_only): - local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) - ray.init( - num_cpus=1, - object_store_memory=80 * 1024 * 1024, - _system_config={ - "local_fs_capacity_threshold": local_fs_capacity_threshold, - }, - ) - arr = np.random.rand(9 * 1024 * 1024) # 160 MB data - ray.put(arr) - - @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_put_out_of_disk(shutdown_only): local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) @@ -40,7 +26,7 @@ def test_put_out_of_disk(shutdown_only): "local_fs_capacity_threshold": local_fs_capacity_threshold, }, ) - arr = np.random.rand(20 * 1024 * 1024) # 160 MB data + arr = np.random.rand(20 * 1024 * 1024) # 800 MB data with pytest.raises(ray.exceptions.OutOfDiskError): ray.put(arr) @@ -58,7 +44,7 @@ def test_task_returns(shutdown_only): @ray.remote def foo(): - return np.random.rand(20 * 1024 * 1024) # 160 MB data + return np.random.rand(20 * 1024 * 1024) # 800 MB data with pytest.raises(ray.exceptions.RayTaskError): ray.get(foo.remote()) @@ -77,7 +63,7 @@ def test_task_put(shutdown_only): @ray.remote def foo(): - ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 160 MB data + ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 800 MB data return ref with pytest.raises(ray.exceptions.RayTaskError): @@ -105,7 +91,7 @@ def test_task_args(shutdown_only): @ray.remote def foo(): - return np.random.rand(20 * 1024 * 1024) # 160 MB data + return np.random.rand(20 * 1024 * 1024) # 800 MB data @ray.remote def bar(obj): @@ -137,7 +123,7 @@ def test_actor(shutdown_only): @ray.remote def foo(): - return np.random.rand(20 * 1024 * 1024) # 160 MB data + return np.random.rand(20 * 1024 * 1024) # 800 MB data @ray.remote class Actor: diff --git a/src/ray/object_manager/plasma/create_request_queue.cc b/src/ray/object_manager/plasma/create_request_queue.cc index e9ca0f414f22..2f50424cb6b5 100644 --- a/src/ray/object_manager/plasma/create_request_queue.cc +++ b/src/ray/object_manager/plasma/create_request_queue.cc @@ -85,14 +85,6 @@ Status CreateRequestQueue::ProcessRequest(bool fallback_allocator, } } -bool CreateRequestQueue::MayHandleOutOfDisk(std::unique_ptr &request) { - if (request->error == PlasmaError::OutOfMemory && fs_monitor_.OverCapacity()) { - request->error = PlasmaError::OutOfDisk; - return true; - } - return false; -} - Status CreateRequestQueue::ProcessRequests() { // Suppress OOM dump to once per grace period. bool logged_oom = false; @@ -102,7 +94,10 @@ Status CreateRequestQueue::ProcessRequests() { auto status = ProcessRequest(/*fallback_allocator=*/false, *request_it, &spilling_required); - if (MayHandleOutOfDisk(*request_it)) { + // if allocation failed due to OOM, and fs_monitor_ indicates the local disk is full, + // we should failed the request with out of disk error + if ((*request_it)->error == PlasmaError::OutOfMemory && fs_monitor_.OverCapacity()) { + (*request_it)->error = PlasmaError::OutOfDisk; FinishRequest(request_it); continue; } diff --git a/src/ray/object_manager/plasma/create_request_queue.h b/src/ray/object_manager/plasma/create_request_queue.h index d692db82710f..be313636b58a 100644 --- a/src/ray/object_manager/plasma/create_request_queue.h +++ b/src/ray/object_manager/plasma/create_request_queue.h @@ -159,8 +159,6 @@ class CreateRequestQueue { std::unique_ptr &request, bool *spilling_required); - bool MayHandleOutOfDisk(std::unique_ptr &request); - /// Finish a queued request and remove it from the queue. void FinishRequest(std::list>::iterator request_it); From 6586d40af2a22c44a8400eb4e1dd7ec9bf87695c Mon Sep 17 00:00:00 2001 From: scv119 Date: Tue, 7 Jun 2022 09:48:09 -0700 Subject: [PATCH 31/48] add --- python/ray/tests/test_out_of_disk_space.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index 41993d388cf9..c7c2b66167ea 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -18,7 +18,7 @@ def calculate_capacity_threshold(disk_capacity_in_bytes): @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_put_out_of_disk(shutdown_only): - local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) + local_fs_capacity_threshold = calculate_capacity_threshold(1 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, @@ -26,14 +26,14 @@ def test_put_out_of_disk(shutdown_only): "local_fs_capacity_threshold": local_fs_capacity_threshold, }, ) - arr = np.random.rand(20 * 1024 * 1024) # 800 MB data + arr = np.random.rand(20 * 1024 * 1024) # 160 MB data with pytest.raises(ray.exceptions.OutOfDiskError): ray.put(arr) @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_task_returns(shutdown_only): - local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) + local_fs_capacity_threshold = calculate_capacity_threshold(1 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, @@ -44,7 +44,7 @@ def test_task_returns(shutdown_only): @ray.remote def foo(): - return np.random.rand(20 * 1024 * 1024) # 800 MB data + return np.random.rand(20 * 1024 * 1024) # 160 MB data with pytest.raises(ray.exceptions.RayTaskError): ray.get(foo.remote()) @@ -52,7 +52,7 @@ def foo(): @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_task_put(shutdown_only): - local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) + local_fs_capacity_threshold = calculate_capacity_threshold(1 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, @@ -63,7 +63,7 @@ def test_task_put(shutdown_only): @ray.remote def foo(): - ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 800 MB data + ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 160 MB data return ref with pytest.raises(ray.exceptions.RayTaskError): @@ -91,7 +91,7 @@ def test_task_args(shutdown_only): @ray.remote def foo(): - return np.random.rand(20 * 1024 * 1024) # 800 MB data + return np.random.rand(20 * 1024 * 1024) # 160 MB data @ray.remote def bar(obj): @@ -123,7 +123,7 @@ def test_actor(shutdown_only): @ray.remote def foo(): - return np.random.rand(20 * 1024 * 1024) # 800 MB data + return np.random.rand(20 * 1024 * 1024) # 160 MB data @ray.remote class Actor: From fba24e3228bc8521e003af00a7f179a6e5ec6b7a Mon Sep 17 00:00:00 2001 From: scv119 Date: Wed, 8 Jun 2022 22:26:02 -0700 Subject: [PATCH 32/48] address comments --- python/ray/serialization.py | 2 +- src/ray/common/file_system_monitor.cc | 2 +- src/ray/object_manager/pull_manager.cc | 11 +++++++++-- src/ray/protobuf/common.proto | 2 +- 4 files changed, 12 insertions(+), 5 deletions(-) diff --git a/python/ray/serialization.py b/python/ray/serialization.py index 8d50217606e7..5150283ca04a 100644 --- a/python/ray/serialization.py +++ b/python/ray/serialization.py @@ -275,7 +275,7 @@ def _deserialize_object(self, data, metadata, object_ref): return ObjectFetchTimedOutError( object_ref.hex(), object_ref.owner_address(), object_ref.call_site() ) - elif error_type == ErrorType.Value("OBJECT_FETCH_OUT_OF_DISK"): + elif error_type == ErrorType.Value("OUT_OF_DISK_ERROR"): return OutOfDiskError( object_ref.hex(), object_ref.owner_address(), object_ref.call_site() ) diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc index 8892b5241f71..69e005980f84 100644 --- a/src/ray/common/file_system_monitor.cc +++ b/src/ray/common/file_system_monitor.cc @@ -107,7 +107,7 @@ std::vector ParseSpillingPaths(const std::string &spilling_config) try { json spill_config = json::parse(spilling_config); - auto &directory_path = spill_config.at("params").at("directory_path"); + const auto &directory_path = spill_config.at("params").at("directory_path"); if (directory_path.is_string()) { spilling_paths.push_back(directory_path); } else if (directory_path.is_array()) { diff --git a/src/ray/object_manager/pull_manager.cc b/src/ray/object_manager/pull_manager.cc index 13d581789533..f14b7375405b 100644 --- a/src/ray/object_manager/pull_manager.cc +++ b/src/ray/object_manager/pull_manager.cc @@ -783,8 +783,15 @@ std::string PullManager::DebugString() const { } void PullManager::SetOutOfDisk(const ObjectID &object_id) { - RAY_LOG(DEBUG) << "Pull of object failed due to out of disk: " << object_id; - fail_pull_request_(object_id, rpc::ErrorType::OBJECT_FETCH_OUT_OF_DISK); + bool is_actively_pulled = false; + { + absl::MutexLock lock(&active_objects_mu_); + is_actively_pulled = active_object_pull_requests_.count(object_id) > 0; + } + if (is_actively_pulled) { + RAY_LOG(DEBUG) << "Pull of object failed due to out of disk: " << object_id; + fail_pull_request_(object_id, rpc::ErrorType::OUT_OF_DISK_ERROR); + } } } // namespace ray diff --git a/src/ray/protobuf/common.proto b/src/ray/protobuf/common.proto index 1756db0bedee..ba789720fe5e 100644 --- a/src/ray/protobuf/common.proto +++ b/src/ray/protobuf/common.proto @@ -194,7 +194,7 @@ enum ErrorType { // The actor is unschedulable. ACTOR_UNSCHEDULABLE_ERROR = 19; // We use this error for object fetches that failed due to out of disk. - OBJECT_FETCH_OUT_OF_DISK = 20; + OUT_OF_DISK_ERROR = 20; } /// The information per ray error type. From 4dc64db9f6c6aef5ae9442fc0273c6539ec29a42 Mon Sep 17 00:00:00 2001 From: scv119 Date: Wed, 8 Jun 2022 22:33:21 -0700 Subject: [PATCH 33/48] add --- src/ray/common/file_system_monitor.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/ray/common/file_system_monitor.h b/src/ray/common/file_system_monitor.h index e5d8df59e72a..abe67405238e 100644 --- a/src/ray/common/file_system_monitor.h +++ b/src/ray/common/file_system_monitor.h @@ -41,7 +41,12 @@ class FileSystemMonitor { ~FileSystemMonitor(); - /// return the disk usage. + /// return the disk usage of a given path. + /// + /// \param path path of the file system to query the disk usage. + /// \return std::filesystem::space_info if query succeeds; or return empty optional + /// if error happens. Refer to https://en.cppreference.com/w/cpp/filesystem/space_info + /// for struct details. std::optional Space(const std::string &path) const; /// returns true if ANY path's disk usage is over the capacity threshold. From ffd987c1e1a481e1e2f41b6075cf38b56049a83b Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 9 Jun 2022 00:18:10 -0700 Subject: [PATCH 34/48] working on tests --- python/ray/tests/test_out_of_disk_space.py | 28 ++++++++++++++++++---- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index c7c2b66167ea..85fb5d381aa8 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -1,5 +1,6 @@ import sys import shutil +import time import numpy as np import platform @@ -11,24 +12,41 @@ def calculate_capacity_threshold(disk_capacity_in_bytes): usage = shutil.disk_usage("/tmp") - threshold = min(1, 1.0 * (usage.used + disk_capacity_in_bytes) / usage.total) - assert threshold > 0 and threshold < 1 + threshold = min(1, 1.0 - 1.0 * (usage.free - disk_capacity_in_bytes) / usage.total) return threshold +def get_current_usage(): + usage = shutil.disk_usage("/tmp") + print(f"free: {usage.free} ") + print(f"current usage: {1.0 - 1.0 * usage.free / usage.total}") + return 1.0 - 1.0 * usage.free / usage.total + + @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_put_out_of_disk(shutdown_only): - local_fs_capacity_threshold = calculate_capacity_threshold(1 * 1024 * 1024) + local_fs_capacity_threshold = calculate_capacity_threshold(100 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, _system_config={ "local_fs_capacity_threshold": local_fs_capacity_threshold, + "local_fs_monitor_interval_ms": 10, }, ) - arr = np.random.rand(20 * 1024 * 1024) # 160 MB data + print(get_current_usage()) + assert get_current_usage() < local_fs_capacity_threshold + ref = ray.put(np.random.rand(20 * 1024 * 1024)) + ray.wait([ref]) + print(get_current_usage()) + assert get_current_usage() > local_fs_capacity_threshold + time.sleep(1) with pytest.raises(ray.exceptions.OutOfDiskError): - ray.put(arr) + ray.put(np.random.rand(20 * 1024 * 1024)) + del ref + assert get_current_usage() < local_fs_capacity_threshold + time.sleep(1) + ray.put(np.random.rand(20 * 1024 * 1024)) @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") From 613a95c4de69017a9300e5a154619b97874ab57c Mon Sep 17 00:00:00 2001 From: scv119 Date: Mon, 13 Jun 2022 12:05:22 -0700 Subject: [PATCH 35/48] fix test --- python/ray/tests/test_out_of_disk_space.py | 44 +++++++++++++++++++--- 1 file changed, 38 insertions(+), 6 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index 85fb5d381aa8..d4c29c277798 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -1,6 +1,8 @@ +import os import sys import shutil import time +import tempfile import numpy as np import platform @@ -23,9 +25,17 @@ def get_current_usage(): return 1.0 - 1.0 * usage.free / usage.total +def create_tmp_file(bytes): + tmp_dir = tempfile.mkdtemp(dir="/tmp") + tmp_path = os.path.join(tmp_dir, "test.txt") + with open(tmp_path, "wb") as f: + f.write(os.urandom(bytes)) + return tmp_path + + @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_put_out_of_disk(shutdown_only): - local_fs_capacity_threshold = calculate_capacity_threshold(100 * 1024 * 1024) + local_fs_capacity_threshold = calculate_capacity_threshold(200 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, @@ -34,16 +44,19 @@ def test_put_out_of_disk(shutdown_only): "local_fs_monitor_interval_ms": 10, }, ) - print(get_current_usage()) assert get_current_usage() < local_fs_capacity_threshold ref = ray.put(np.random.rand(20 * 1024 * 1024)) - ray.wait([ref]) - print(get_current_usage()) + del ref + # create a temp file so that the disk size is over the threshold. + # ray.put doesn't work is that fallback allocation uses mmaped file + # that doesn't neccssary allocate disk spaces. + tmp_file = create_tmp_file(250 * 1024 * 1024) assert get_current_usage() > local_fs_capacity_threshold time.sleep(1) with pytest.raises(ray.exceptions.OutOfDiskError): ray.put(np.random.rand(20 * 1024 * 1024)) - del ref + # delete tmp file to reclaim space back. + os.remove(tmp_file) assert get_current_usage() < local_fs_capacity_threshold time.sleep(1) ray.put(np.random.rand(20 * 1024 * 1024)) @@ -51,22 +64,33 @@ def test_put_out_of_disk(shutdown_only): @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_task_returns(shutdown_only): - local_fs_capacity_threshold = calculate_capacity_threshold(1 * 1024 * 1024) + local_fs_capacity_threshold = calculate_capacity_threshold(10 * 1024 * 1024) ray.init( num_cpus=1, object_store_memory=80 * 1024 * 1024, _system_config={ "local_fs_capacity_threshold": local_fs_capacity_threshold, + "local_fs_monitor_interval_ms": 10, }, ) + # create a temp file so that the disk size is over the threshold. + # ray.put doesn't work is that fallback allocation uses mmaped file + # that doesn't neccssary allocate disk spaces. + tmp_file = create_tmp_file(50 * 1024 * 1024) + assert get_current_usage() > local_fs_capacity_threshold + time.sleep(1) + @ray.remote def foo(): + time.sleep(1) return np.random.rand(20 * 1024 * 1024) # 160 MB data with pytest.raises(ray.exceptions.RayTaskError): ray.get(foo.remote()) + os.remove(tmp_file) + @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") def test_task_put(shutdown_only): @@ -76,9 +100,17 @@ def test_task_put(shutdown_only): object_store_memory=80 * 1024 * 1024, _system_config={ "local_fs_capacity_threshold": local_fs_capacity_threshold, + "local_fs_monitor_interval_ms": 10, }, ) + # create a temp file so that the disk size is over the threshold. + # ray.put doesn't work is that fallback allocation uses mmaped file + # that doesn't neccssary allocate disk spaces. + tmp_file = create_tmp_file(50 * 1024 * 1024) + assert get_current_usage() > local_fs_capacity_threshold + time.sleep(1) + @ray.remote def foo(): ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 160 MB data From cb89f977eb270b4ccecfba6723f7ff9f2883a614 Mon Sep 17 00:00:00 2001 From: scv119 Date: Mon, 13 Jun 2022 12:26:30 -0700 Subject: [PATCH 36/48] update --- python/ray/exceptions.py | 10 +++- python/ray/tests/test_out_of_disk_space.py | 57 ++++++++++++---------- 2 files changed, 38 insertions(+), 29 deletions(-) diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index 7c9afafbe300..db46325d38e0 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -334,11 +334,17 @@ class OutOfDiskError(RayError): """ def __str__(self): + # TODO(scv119): expose more disk usage information and link to a doc. return super(OutOfDiskError, self).__str__() + ( "\n" "The local object store is full and local disk is also full." - "Tip: Use the `ray memory` command " - "to list active objects in the cluster." + "Tip: Run `df` command to check the disk usage; " + "Use the `ray memory` command to check the plasma memory usage." + "You can adjust plasma memory capacity by calling " + "`ray.init(object_store_memory=...)`;" + "You can also try to spread out of disk usage across multiple " + "physical devices, following" + " https://docs.ray.io/en/master/ray-core/objects/object-spilling.html" ) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index d4c29c277798..29389472728d 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -1,3 +1,4 @@ +from contextlib import contextmanager import os import sys import shutil @@ -25,12 +26,16 @@ def get_current_usage(): return 1.0 - 1.0 * usage.free / usage.total +@contextmanager def create_tmp_file(bytes): tmp_dir = tempfile.mkdtemp(dir="/tmp") tmp_path = os.path.join(tmp_dir, "test.txt") with open(tmp_path, "wb") as f: f.write(os.urandom(bytes)) - return tmp_path + try: + yield tmp_path + finally: + os.remove(tmp_path) @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") @@ -50,13 +55,13 @@ def test_put_out_of_disk(shutdown_only): # create a temp file so that the disk size is over the threshold. # ray.put doesn't work is that fallback allocation uses mmaped file # that doesn't neccssary allocate disk spaces. - tmp_file = create_tmp_file(250 * 1024 * 1024) - assert get_current_usage() > local_fs_capacity_threshold - time.sleep(1) - with pytest.raises(ray.exceptions.OutOfDiskError): - ray.put(np.random.rand(20 * 1024 * 1024)) - # delete tmp file to reclaim space back. - os.remove(tmp_file) + with create_tmp_file(250 * 1024 * 1024): + assert get_current_usage() > local_fs_capacity_threshold + time.sleep(1) + with pytest.raises(ray.exceptions.OutOfDiskError): + ray.put(np.random.rand(20 * 1024 * 1024)) + # delete tmp file to reclaim space back. + assert get_current_usage() < local_fs_capacity_threshold time.sleep(1) ray.put(np.random.rand(20 * 1024 * 1024)) @@ -77,19 +82,17 @@ def test_task_returns(shutdown_only): # create a temp file so that the disk size is over the threshold. # ray.put doesn't work is that fallback allocation uses mmaped file # that doesn't neccssary allocate disk spaces. - tmp_file = create_tmp_file(50 * 1024 * 1024) - assert get_current_usage() > local_fs_capacity_threshold - time.sleep(1) - - @ray.remote - def foo(): + with create_tmp_file(250 * 1024 * 1024): + assert get_current_usage() > local_fs_capacity_threshold time.sleep(1) - return np.random.rand(20 * 1024 * 1024) # 160 MB data - with pytest.raises(ray.exceptions.RayTaskError): - ray.get(foo.remote()) + @ray.remote + def foo(): + time.sleep(1) + return np.random.rand(20 * 1024 * 1024) # 160 MB data - os.remove(tmp_file) + with pytest.raises(ray.exceptions.RayTaskError): + ray.get(foo.remote()) @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") @@ -107,17 +110,17 @@ def test_task_put(shutdown_only): # create a temp file so that the disk size is over the threshold. # ray.put doesn't work is that fallback allocation uses mmaped file # that doesn't neccssary allocate disk spaces. - tmp_file = create_tmp_file(50 * 1024 * 1024) - assert get_current_usage() > local_fs_capacity_threshold - time.sleep(1) + with create_tmp_file(250 * 1024 * 1024): + assert get_current_usage() > local_fs_capacity_threshold + time.sleep(1) - @ray.remote - def foo(): - ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 160 MB data - return ref + @ray.remote + def foo(): + ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 160 MB data + return ref - with pytest.raises(ray.exceptions.RayTaskError): - ray.get(foo.remote()) + with pytest.raises(ray.exceptions.RayTaskError): + ray.get(foo.remote()) @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") From 9990a346ad7ba424c2a930a7444335a29dd7ddc3 Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 16 Jun 2022 23:34:32 -0700 Subject: [PATCH 37/48] address comments --- python/ray/exceptions.py | 25 +++++------ python/ray/serialization.py | 49 +++++++++++----------- python/ray/tests/test_out_of_disk_space.py | 12 +++--- src/ray/common/file_system_monitor.cc | 15 ++++--- src/ray/common/file_system_monitor.h | 4 +- src/ray/common/status.h | 2 + src/ray/object_manager/plasma/plasma.fbs | 2 +- 7 files changed, 57 insertions(+), 52 deletions(-) diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index db46325d38e0..207318dfd020 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -1,14 +1,20 @@ import os from traceback import format_exception +from typing import Optional, Union -from typing import Union, Optional +import colorama import ray.cloudpickle as pickle -from ray.core.generated.common_pb2 import RayException, Language, PYTHON -from ray.core.generated.common_pb2 import Address, ActorDiedErrorContext import ray.ray_constants as ray_constants -from ray._raylet import WorkerID, ActorID, TaskID -import colorama +from ray._raylet import ActorID, TaskID, WorkerID +from ray.core.generated.common_pb2 import ( + PYTHON, + ActorDiedErrorContext, + Address, + Language, + RayException, +) + import setproctitle @@ -338,13 +344,8 @@ def __str__(self): return super(OutOfDiskError, self).__str__() + ( "\n" "The local object store is full and local disk is also full." - "Tip: Run `df` command to check the disk usage; " - "Use the `ray memory` command to check the plasma memory usage." - "You can adjust plasma memory capacity by calling " - "`ray.init(object_store_memory=...)`;" - "You can also try to spread out of disk usage across multiple " - "physical devices, following" - " https://docs.ray.io/en/master/ray-core/objects/object-spilling.html" + "Tip: Use `df` on this node to check disk usage and " + "`ray memory` to check object store memory usage." ) diff --git a/python/ray/serialization.py b/python/ray/serialization.py index 5150283ca04a..a6f8b61febc3 100644 --- a/python/ray/serialization.py +++ b/python/ray/serialization.py @@ -2,44 +2,43 @@ import threading import traceback -import ray.cloudpickle as pickle -from ray import ray_constants import ray._private.utils +import ray.cloudpickle as pickle +from ray import ray_constants, serialization_addons from ray._private.gcs_utils import ErrorType +from ray._raylet import ( + MessagePackSerializedObject, + MessagePackSerializer, + Pickle5SerializedObject, + Pickle5Writer, + RawSerializedObject, + split_buffer, + unpack_pickle5_buffers, +) from ray.core.generated.common_pb2 import RayErrorInfo from ray.exceptions import ( - RayError, - PlasmaObjectNotAvailable, - RayTaskError, - RayActorError, - TaskCancelledError, - WorkerCrashedError, - ObjectLostError, + ActorPlacementGroupRemoved, + ActorUnschedulableError, + LocalRayletDiedError, ObjectFetchTimedOutError, - ReferenceCountingAssertionError, - OwnerDiedError, + ObjectLostError, ObjectReconstructionFailedError, - ObjectReconstructionFailedMaxAttemptsExceededError, ObjectReconstructionFailedLineageEvictedError, + ObjectReconstructionFailedMaxAttemptsExceededError, OutOfDiskError, + OwnerDiedError, + PlasmaObjectNotAvailable, + RayActorError, + RayError, RaySystemError, + RayTaskError, + ReferenceCountingAssertionError, RuntimeEnvSetupError, + TaskCancelledError, TaskPlacementGroupRemoved, - ActorPlacementGroupRemoved, - LocalRayletDiedError, TaskUnschedulableError, - ActorUnschedulableError, -) -from ray._raylet import ( - split_buffer, - unpack_pickle5_buffers, - Pickle5Writer, - Pickle5SerializedObject, - MessagePackSerializer, - MessagePackSerializedObject, - RawSerializedObject, + WorkerCrashedError, ) -from ray import serialization_addons logger = logging.getLogger(__name__) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index 29389472728d..d0ec369823a6 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -1,15 +1,15 @@ -from contextlib import contextmanager import os -import sys +import platform import shutil -import time +import sys import tempfile -import numpy as np +import time +from contextlib import contextmanager -import platform +import numpy as np import pytest -import ray +import ray from ray.cluster_utils import Cluster diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc index 69e005980f84..50c6e6ec2322 100644 --- a/src/ray/common/file_system_monitor.cc +++ b/src/ray/common/file_system_monitor.cc @@ -88,7 +88,8 @@ bool FileSystemMonitor::OverCapacityImpl( return false; } if (space_info->capacity <= 0) { - RAY_LOG_EVERY_MS(ERROR, 60 * 1000) << path << " has no capacity."; + RAY_LOG_EVERY_MS(ERROR, 60 * 1000) + << path << " has no capacity, object creation will fail if spilling is required."; return true; } @@ -97,9 +98,9 @@ bool FileSystemMonitor::OverCapacityImpl( } RAY_LOG_EVERY_MS(ERROR, 10 * 1000) - << path << " is over capacity, available: " << space_info->available - << ", capacity: " << space_info->capacity << ", threshold: " << capacity_threshold_; - return true; + << path << " is over " << capacity_threshold_ + << "\% full, available space: " << space_info->available + << ". Object creation will fail if spilling is required." return true; } std::vector ParseSpillingPaths(const std::string &spilling_config) { @@ -115,11 +116,13 @@ std::vector ParseSpillingPaths(const std::string &spilling_config) if (entry.is_string()) { spilling_paths.push_back(entry); } else { - RAY_LOG(ERROR) << "Failed to parse spilling path: " << entry; + RAY_LOG(ERROR) << "Failed to parse spilling path: " << entry + << ", expecting a string literal."; } } } else { - RAY_LOG(ERROR) << "Failed to parse spilling path: " << directory_path; + RAY_LOG(ERROR) << "Failed to parse spilling path: " << directory_path + << ", expecting string or array."; } } catch (json::exception &ex) { RAY_LOG(ERROR) << "Failed to load spilling config: " << ex.what() diff --git a/src/ray/common/file_system_monitor.h b/src/ray/common/file_system_monitor.h index abe67405238e..18f10b03eaf9 100644 --- a/src/ray/common/file_system_monitor.h +++ b/src/ray/common/file_system_monitor.h @@ -41,7 +41,7 @@ class FileSystemMonitor { ~FileSystemMonitor(); - /// return the disk usage of a given path. + /// Returns the disk usage of a given path. /// /// \param path path of the file system to query the disk usage. /// \return std::filesystem::space_info if query succeeds; or return empty optional @@ -49,7 +49,7 @@ class FileSystemMonitor { /// for struct details. std::optional Space(const std::string &path) const; - /// returns true if ANY path's disk usage is over the capacity threshold. + /// Returns true if ANY path's disk usage is over the capacity threshold. bool OverCapacity() const; private: diff --git a/src/ray/common/status.h b/src/ray/common/status.h index 14995f48bfc7..4e9a7a6da90f 100644 --- a/src/ray/common/status.h +++ b/src/ray/common/status.h @@ -108,6 +108,8 @@ enum class StatusCode : char { // This represents all other status codes // returned by grpc that are not defined above. GrpcUnknown = 27, + // Object store is both out of memory and + // out of disk. OutOfDisk = 28, }; diff --git a/src/ray/object_manager/plasma/plasma.fbs b/src/ray/object_manager/plasma/plasma.fbs index 4540935cf858..abbf000e4a01 100644 --- a/src/ray/object_manager/plasma/plasma.fbs +++ b/src/ray/object_manager/plasma/plasma.fbs @@ -84,7 +84,7 @@ enum PlasmaError:int { UnexpectedError, // Trying to abort an object but it's not sealed. ObjectSealed, - // Trying to create an object but there isn't enough disk in the store. + // Trying to create an object but there isn't enough disk space. OutOfDisk, } From f44ffd13e46c7c9e9b855a1eaac0e20537ca032c Mon Sep 17 00:00:00 2001 From: scv119 Date: Thu, 16 Jun 2022 23:39:19 -0700 Subject: [PATCH 38/48] add --- src/ray/common/file_system_monitor.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc index 50c6e6ec2322..450d3402a3d4 100644 --- a/src/ray/common/file_system_monitor.cc +++ b/src/ray/common/file_system_monitor.cc @@ -125,8 +125,8 @@ std::vector ParseSpillingPaths(const std::string &spilling_config) << ", expecting string or array."; } } catch (json::exception &ex) { - RAY_LOG(ERROR) << "Failed to load spilling config: " << ex.what() - << " The config string is: " << spilling_config; + RAY_LOG(ERROR) << "Failed to parse spilling config, error message: " << ex.what() + << "The config string is probably invalid json: " << spilling_config; } return spilling_paths; } From 9a54a66666a04dedde7e953446fd57f30ce26672 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 19 Jun 2022 21:03:25 -0700 Subject: [PATCH 39/48] add --- python/ray/exceptions.py | 3 ++- src/ray/common/file_system_monitor.cc | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index 207318dfd020..c9ca59ec03d8 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -343,7 +343,8 @@ def __str__(self): # TODO(scv119): expose more disk usage information and link to a doc. return super(OutOfDiskError, self).__str__() + ( "\n" - "The local object store is full and local disk is also full." + "The object cannot be created because the local object store" + " is full and at least 90% of the local disk is in use." "Tip: Use `df` on this node to check disk usage and " "`ray memory` to check object store memory usage." ) diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc index 450d3402a3d4..8834ecd1a9b2 100644 --- a/src/ray/common/file_system_monitor.cc +++ b/src/ray/common/file_system_monitor.cc @@ -100,7 +100,8 @@ bool FileSystemMonitor::OverCapacityImpl( RAY_LOG_EVERY_MS(ERROR, 10 * 1000) << path << " is over " << capacity_threshold_ << "\% full, available space: " << space_info->available - << ". Object creation will fail if spilling is required." return true; + << ". Object creation will fail if spilling is required."; + return true; } std::vector ParseSpillingPaths(const std::string &spilling_config) { From cc3b06c7e3b86885f8bd4e38102de08bac917ae3 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 19 Jun 2022 21:22:16 -0700 Subject: [PATCH 40/48] add --- python/ray/tests/test_out_of_disk_space.py | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/python/ray/tests/test_out_of_disk_space.py b/python/ray/tests/test_out_of_disk_space.py index d0ec369823a6..d651b0c8ddb6 100644 --- a/python/ray/tests/test_out_of_disk_space.py +++ b/python/ray/tests/test_out_of_disk_space.py @@ -91,8 +91,10 @@ def foo(): time.sleep(1) return np.random.rand(20 * 1024 * 1024) # 160 MB data - with pytest.raises(ray.exceptions.RayTaskError): + try: ray.get(foo.remote()) + except ray.exceptions.RayTaskError as e: + assert isinstance(e.cause, ray.exceptions.OutOfDiskError) @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") @@ -119,8 +121,10 @@ def foo(): ref = ray.put(np.random.rand(20 * 1024 * 1024)) # 160 MB data return ref - with pytest.raises(ray.exceptions.RayTaskError): + try: ray.get(foo.remote()) + except ray.exceptions.RayTaskError as e: + assert isinstance(e.cause, ray.exceptions.OutOfDiskError) @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") @@ -151,8 +155,10 @@ def bar(obj): print(obj) ref = foo.options(resources={"sufficient_memory": 1}).remote() - with pytest.raises(ray.exceptions.RayTaskError): + try: ray.get(bar.options(resources={"out_of_memory": 1}).remote(ref)) + except ray.exceptions.RayTaskError as e: + assert isinstance(e.cause, ray.exceptions.OutOfDiskError) @pytest.mark.skipif(platform.system() == "Windows", reason="Not targeting Windows") @@ -199,12 +205,16 @@ def return_ood(self): a = Actor.options(resources={"out_of_memory": 1}).remote(1) ray.get(a.foo.remote()) - with pytest.raises(ray.exceptions.RayTaskError): + try: ray.get(a.args_ood.remote(ref)) + except ray.exceptions.RayTaskError as e: + assert isinstance(e.cause, ray.exceptions.OutOfDiskError) ray.get(a.foo.remote()) - with pytest.raises(ray.exceptions.RayTaskError): + try: ray.get(a.return_ood.remote()) + except ray.exceptions.RayTaskError as e: + assert isinstance(e.cause, ray.exceptions.OutOfDiskError) if __name__ == "__main__": From fc0dbc1268c407db9d2429aeaffeb99a9ac2c70d Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 19 Jun 2022 21:44:57 -0700 Subject: [PATCH 41/48] add test --- .../plasma/create_request_queue.cc | 5 +++- .../test/create_request_queue_test.cc | 27 +++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/src/ray/object_manager/plasma/create_request_queue.cc b/src/ray/object_manager/plasma/create_request_queue.cc index 2f50424cb6b5..3bc482901f57 100644 --- a/src/ray/object_manager/plasma/create_request_queue.cc +++ b/src/ray/object_manager/plasma/create_request_queue.cc @@ -98,8 +98,10 @@ Status CreateRequestQueue::ProcessRequests() { // we should failed the request with out of disk error if ((*request_it)->error == PlasmaError::OutOfMemory && fs_monitor_.OverCapacity()) { (*request_it)->error = PlasmaError::OutOfDisk; + RAY_LOG(INFO) << "Out-of-disk: Failed to create object " << (*request_it)->object_id + << " of size " << (*request_it)->object_size / 1024 / 1024 << "MB\n"; FinishRequest(request_it); - continue; + return Status::OutOfDisk("System running out of disk."); } if (spilling_required) { @@ -150,6 +152,7 @@ Status CreateRequestQueue::ProcessRequests() { } } } + return Status::OK(); } diff --git a/src/ray/object_manager/test/create_request_queue_test.cc b/src/ray/object_manager/test/create_request_queue_test.cc index d2ea80a39203..dc7d279906f9 100644 --- a/src/ray/object_manager/test/create_request_queue_test.cc +++ b/src/ray/object_manager/test/create_request_queue_test.cc @@ -463,6 +463,33 @@ TEST_F(CreateRequestQueueTest, TestTryRequestImmediately) { AssertNoLeaks(); } +TEST_F(CreateRequestQueueTest, TestOOMAndOOD) { + ray::FileSystemMonitor out_of_disk_monitor{{"/"}, /*capacity_threshold*/ 0}; + bool is_spilling_possible = true; + CreateRequestQueue queue( + out_of_disk_monitor, + /*oom_grace_period_s=*/oom_grace_period_s_, + /*spill_object_callback=*/[&]() { return is_spilling_possible; }, + /*on_global_gc=*/[&]() { num_global_gc_++; }, + /*get_time=*/[&]() { return current_time_ns_; }); + + auto return_status = PlasmaError::OutOfMemory; + auto oom_request = [&](bool fallback, PlasmaObject *result, bool *spill_requested) { + if (return_status == PlasmaError::OK) { + result->data_size = 1234; + } + return return_status; + }; + + auto client = std::make_shared(); + auto req_id1 = queue.AddRequest(ObjectID::Nil(), client, oom_request, 1234); + + // Transient OOM should not use up any until grace period is done. + ASSERT_TRUE(queue.ProcessRequests().IsOutOfDisk()); + ASSERT_REQUEST_FINISHED(queue, req_id1, PlasmaError::OutOfDisk); + AssertNoLeaks(); +} + } // namespace plasma int main(int argc, char **argv) { From af15ae98ac989a0fd9cfab8c69a1a8ee06ec233a Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 19 Jun 2022 22:02:51 -0700 Subject: [PATCH 42/48] add --- src/ray/common/ray_config_def.h | 4 ++-- src/ray/object_manager/test/create_request_queue_test.cc | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 79ef5e2b64bb..7896eae44baa 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -485,10 +485,10 @@ RAY_CONFIG(bool, is_external_storage_type_fs, true) /// Control the capacity threshold for ray local file system (for object store). /// Once we are over the capacity, all subsequent object creation will fail. -RAY_CONFIG(float, local_fs_capacity_threshold, 0.9); +RAY_CONFIG(float, local_fs_capacity_threshold, 0.95); /// Control the frequency of checking the disk usage. -RAY_CONFIG(uint64_t, local_fs_monitor_interval_ms, 1000); +RAY_CONFIG(uint64_t, local_fs_monitor_interval_ms, 100); /* Configuration parameters for locality-aware scheduling. */ /// Whether to enable locality-aware leasing. If enabled, then Ray will consider task diff --git a/src/ray/object_manager/test/create_request_queue_test.cc b/src/ray/object_manager/test/create_request_queue_test.cc index dc7d279906f9..7af5548fcf7f 100644 --- a/src/ray/object_manager/test/create_request_queue_test.cc +++ b/src/ray/object_manager/test/create_request_queue_test.cc @@ -484,7 +484,7 @@ TEST_F(CreateRequestQueueTest, TestOOMAndOOD) { auto client = std::make_shared(); auto req_id1 = queue.AddRequest(ObjectID::Nil(), client, oom_request, 1234); - // Transient OOM should not use up any until grace period is done. + // Should fail with out of disk. ASSERT_TRUE(queue.ProcessRequests().IsOutOfDisk()); ASSERT_REQUEST_FINISHED(queue, req_id1, PlasmaError::OutOfDisk); AssertNoLeaks(); From 127fafc95bd18c4fbaeaed7d4498689f691840be Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 19 Jun 2022 22:06:07 -0700 Subject: [PATCH 43/48] add --- python/ray/exceptions.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index c9ca59ec03d8..94c1c6e04342 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -344,7 +344,8 @@ def __str__(self): return super(OutOfDiskError, self).__str__() + ( "\n" "The object cannot be created because the local object store" - " is full and at least 90% of the local disk is in use." + " is full and the local disk's utilization is over capacity" + " (95% by default)." "Tip: Use `df` on this node to check disk usage and " "`ray memory` to check object store memory usage." ) From f9fde2c827f3db2cd01f83a3a466e8d3bfadb4c5 Mon Sep 17 00:00:00 2001 From: scv119 Date: Sun, 19 Jun 2022 22:24:22 -0700 Subject: [PATCH 44/48] add --- src/ray/common/file_system_monitor.cc | 6 ++++++ src/ray/common/file_system_monitor.h | 3 +++ src/ray/object_manager/plasma/store_runner.cc | 6 ++++++ src/ray/util/filesystem_test.cc | 3 +++ 4 files changed, 18 insertions(+) diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc index 8834ecd1a9b2..0842062eb4e1 100644 --- a/src/ray/common/file_system_monitor.cc +++ b/src/ray/common/file_system_monitor.cc @@ -39,6 +39,12 @@ FileSystemMonitor::FileSystemMonitor(std::vector paths, "FileSystemMonitor.CheckIfAnyPathOverCapacity"); } +FileSystemMonitor FileSystemMonitor::NoopFileSystemMonitor() { + return FileSystemMonitor(/*paths*/ {}, + /*capacity_threshold*/ 1, + /*monitor_interval_ms*/ 365 * 24 * 60 * 60 * 1000); +} + FileSystemMonitor::~FileSystemMonitor() { io_context_.stop(); if (monitor_thread_.joinable()) { diff --git a/src/ray/common/file_system_monitor.h b/src/ray/common/file_system_monitor.h index 18f10b03eaf9..d42e2c783c79 100644 --- a/src/ray/common/file_system_monitor.h +++ b/src/ray/common/file_system_monitor.h @@ -52,6 +52,9 @@ class FileSystemMonitor { /// Returns true if ANY path's disk usage is over the capacity threshold. bool OverCapacity() const; + /// Creates a Noop monitor that never reports out of space. + static FileSystemMonitor NoopFileSystemMonitor(); + private: bool CheckIfAnyPathOverCapacity() const; // For testing purpose. diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index c65411a6d0a6..c7c088ba207e 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -90,6 +90,7 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, absl::MutexLock lock(&store_runner_mutex_); allocator_ = std::make_unique( plasma_directory_, fallback_directory_, hugepages_enabled_, system_memory_); +#ifndef _WIN32 std::vector local_spilling_paths; if (RayConfig::instance().is_external_storage_type_fs()) { local_spilling_paths = @@ -100,6 +101,11 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, local_spilling_paths, RayConfig::instance().local_fs_capacity_threshold(), RayConfig::instance().local_fs_monitor_interval_ms()); +#else + // Skip monitor for Windows. + fs_monitor_ = std::make_unique( + ray::FileSystemMonitor::NoopFileSystemMonitor()); +#endif store_.reset(new PlasmaStore(main_service_, *allocator_, *fs_monitor_, diff --git a/src/ray/util/filesystem_test.cc b/src/ray/util/filesystem_test.cc index e7d1480a52b0..ee17cc4f9b59 100644 --- a/src/ray/util/filesystem_test.cc +++ b/src/ray/util/filesystem_test.cc @@ -85,6 +85,9 @@ TEST(FileSystemTest, TestFileSystemMonitor) { ASSERT_TRUE(result->available > 0); ASSERT_TRUE(result->capacity > 0); } + + auto noop = ray::FileSystemMonitor::NoopFileSystemMonitor(); + ASSERT_FALSE(noop.OverCapacity()); } TEST(FileSystemTest, TestOverCapacity) { From 9cff82be2cdbb42d8141df8d01353344eda77a2e Mon Sep 17 00:00:00 2001 From: scv119 Date: Mon, 20 Jun 2022 00:05:23 -0700 Subject: [PATCH 45/48] fix-windows --- src/ray/common/file_system_monitor.cc | 9 ++++----- src/ray/common/file_system_monitor.h | 6 +++--- src/ray/object_manager/plasma/store_runner.cc | 5 ++--- src/ray/util/filesystem_test.cc | 4 ++-- 4 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc index 0842062eb4e1..411138d30b1f 100644 --- a/src/ray/common/file_system_monitor.cc +++ b/src/ray/common/file_system_monitor.cc @@ -39,11 +39,10 @@ FileSystemMonitor::FileSystemMonitor(std::vector paths, "FileSystemMonitor.CheckIfAnyPathOverCapacity"); } -FileSystemMonitor FileSystemMonitor::NoopFileSystemMonitor() { - return FileSystemMonitor(/*paths*/ {}, - /*capacity_threshold*/ 1, - /*monitor_interval_ms*/ 365 * 24 * 60 * 60 * 1000); -} +FileSystemMonitor::FileSystemMonitor() + : FileSystemMonitor(/*paths*/ {}, + /*capacity_threshold*/ 1, + /*monitor_interval_ms*/ 365 * 24 * 60 * 60 * 1000) {} FileSystemMonitor::~FileSystemMonitor() { io_context_.stop(); diff --git a/src/ray/common/file_system_monitor.h b/src/ray/common/file_system_monitor.h index d42e2c783c79..ef87156b48ff 100644 --- a/src/ray/common/file_system_monitor.h +++ b/src/ray/common/file_system_monitor.h @@ -39,6 +39,9 @@ class FileSystemMonitor { double capacity_threshold, uint64_t monitor_interval_ms = 1000); + /// Creates a Noop monitor that never reports out of space. + FileSystemMonitor(); + ~FileSystemMonitor(); /// Returns the disk usage of a given path. @@ -52,9 +55,6 @@ class FileSystemMonitor { /// Returns true if ANY path's disk usage is over the capacity threshold. bool OverCapacity() const; - /// Creates a Noop monitor that never reports out of space. - static FileSystemMonitor NoopFileSystemMonitor(); - private: bool CheckIfAnyPathOverCapacity() const; // For testing purpose. diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index c7c088ba207e..ca1f8a71e455 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -102,9 +102,8 @@ void PlasmaStoreRunner::Start(ray::SpillObjectsCallback spill_objects_callback, RayConfig::instance().local_fs_capacity_threshold(), RayConfig::instance().local_fs_monitor_interval_ms()); #else - // Skip monitor for Windows. - fs_monitor_ = std::make_unique( - ray::FileSystemMonitor::NoopFileSystemMonitor()); + // Create noop monitor for Windows. + fs_monitor_ = std::make_unique(); #endif store_.reset(new PlasmaStore(main_service_, *allocator_, diff --git a/src/ray/util/filesystem_test.cc b/src/ray/util/filesystem_test.cc index ee17cc4f9b59..53e5e5ce8dcf 100644 --- a/src/ray/util/filesystem_test.cc +++ b/src/ray/util/filesystem_test.cc @@ -86,8 +86,8 @@ TEST(FileSystemTest, TestFileSystemMonitor) { ASSERT_TRUE(result->capacity > 0); } - auto noop = ray::FileSystemMonitor::NoopFileSystemMonitor(); - ASSERT_FALSE(noop.OverCapacity()); + auto noop_monitor = std::make_unique(); + ASSERT_FALSE(noop_monitor->OverCapacity()); } TEST(FileSystemTest, TestOverCapacity) { From f3bc923c39a8212848bf4ee56a82785c403d01b2 Mon Sep 17 00:00:00 2001 From: scv119 Date: Mon, 20 Jun 2022 00:09:23 -0700 Subject: [PATCH 46/48] add --- src/ray/common/file_system_monitor.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/common/file_system_monitor.cc b/src/ray/common/file_system_monitor.cc index 411138d30b1f..7c01e3f4bd87 100644 --- a/src/ray/common/file_system_monitor.cc +++ b/src/ray/common/file_system_monitor.cc @@ -42,7 +42,7 @@ FileSystemMonitor::FileSystemMonitor(std::vector paths, FileSystemMonitor::FileSystemMonitor() : FileSystemMonitor(/*paths*/ {}, /*capacity_threshold*/ 1, - /*monitor_interval_ms*/ 365 * 24 * 60 * 60 * 1000) {} + /*monitor_interval_ms*/ 365ULL * 24 * 60 * 60 * 1000) {} FileSystemMonitor::~FileSystemMonitor() { io_context_.stop(); From dc333af7ce39af8271fa0528bef0a5e07986ee66 Mon Sep 17 00:00:00 2001 From: scv119 Date: Mon, 20 Jun 2022 18:04:42 -0700 Subject: [PATCH 47/48] fix ci --- BUILD.bazel | 2 +- python/ray/tests/test_plasma_unlimited.py | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/BUILD.bazel b/BUILD.bazel index 96703a0c26ef..a99d03affb6f 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -1431,8 +1431,8 @@ cc_test( copts = COPTS, tags = ["team:core"], deps = [ - ":ray_util", ":ray_common", + ":ray_util", "@com_google_googletest//:gtest_main", ], ) diff --git a/python/ray/tests/test_plasma_unlimited.py b/python/ray/tests/test_plasma_unlimited.py index 19d0e14e97fd..2b6497700049 100644 --- a/python/ray/tests/test_plasma_unlimited.py +++ b/python/ray/tests/test_plasma_unlimited.py @@ -193,6 +193,8 @@ def test_fallback_allocation_failure(shutdown_only): _temp_dir="/dev/shm", _system_config={ "object_spilling_config": json.dumps(file_system_config), + # set local fs capacity to 100% so it never errors with out of disk. + "local_fs_capacity_threshold": 1, }, ) shm_size = shutil.disk_usage("/dev/shm").total From cb32c24f509616eaa40c40883a3f8e6b7978dd29 Mon Sep 17 00:00:00 2001 From: scv119 Date: Tue, 21 Jun 2022 15:25:28 -0700 Subject: [PATCH 48/48] fix merge failure --- python/ray/exceptions.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index 830e7267e2e1..07c54ba38195 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -341,7 +341,7 @@ def __str__(self): ) -<<<<<<< HEAD +@PublicAPI class OutOfDiskError(RayError): """Indicates that the local disk is full. @@ -361,9 +361,7 @@ def __str__(self): ) -======= @PublicAPI ->>>>>>> upstream/master class ObjectLostError(RayError): """Indicates that the object is lost from distributed memory, due to node failure or system error.