From e9ef82ce568cc88547acec2cc6b0b5834884f703 Mon Sep 17 00:00:00 2001 From: suquark Date: Wed, 20 Feb 2019 11:42:18 +0800 Subject: [PATCH 1/7] Replace native socket operations with asio revert some changes replace event loop with asio update plasma store protocol fix qualifiers update plasma store client protocol Remove all native socket operations. Implement general io support Fix bugs fix all compiling bugs fix bug Fix all tests. Add license header. try to fix cmake try to make asio standalone simplify code add license update url lint lint & fix fix restore entrypoint remove unused unix headers fix Update LICENSE fix rename handle signal move the function to its original place fix doc hide classes stop installing asio headers fix doc reverse changes minor fix tiny fix fix comments minor fix resolve conflicts fix optimize cmake fix update formatter fix according to github comments lint prevent the store from dying fix ARROW_CHECK Fix ARROW-4036: [C++] Pluggable Status message, by exposing an abstract delegate class. This provides less "pluggability" but I think still offers a clean model for extension (subsystems can wrap the constructor for there purposes, and provide external static methods to check for particular types of errors). Author: Micah Kornfield Author: Antoine Pitrou Closes #4484 from emkornfield/status_code_proposal and squashes the following commits: 4d1ab8d1d don't import plasma errors directly into top level pyarrow module a66f999f8 make format 040216d48 fixes for comments outside python 729bba1ff Fix Py2 issues (hopefully) ea56d1e6a Fix PythonErrorDetail to store Python error state (and restore it in check_status()) 21e1b95ac fix compilation 9c905b094 fix lint 74d563cd7 fixes 85786efb1 change messages 3626a9016 try removing message a4e6a1ff2 add logging for debug 4586fd1e2 fix typo 8f011b329 fix status propagation 317ea9c66 fix complie 9f5916070 don't make_shared inline 484b3a232 style fix 14e3467b5 dont rely on rtti cd22df64d format dec458506 not-quite pluggable error codes fix merge fix update update update update fix update fix update update revert some unknown comments rebase CMakeLists rebase eviction_policy.h rebase CMakeLists rebase --- LICENSE.txt | 31 - cpp/apidoc/tutorials/plasma.md | 15 +- cpp/src/arrow/status.h | 171 ++-- cpp/src/plasma/CMakeLists.txt | 73 +- cpp/src/plasma/client.cc | 396 ++++----- cpp/src/plasma/client.h | 15 +- cpp/src/plasma/common.cc | 48 +- cpp/src/plasma/dlmalloc.cc | 2 + cpp/src/plasma/events.cc | 107 --- cpp/src/plasma/events.h | 111 --- cpp/src/plasma/eviction_policy.cc | 1 + cpp/src/plasma/eviction_policy.h | 48 +- cpp/src/plasma/fling.h | 10 +- cpp/src/plasma/io.cc | 241 ------ cpp/src/plasma/io.h | 70 -- cpp/src/plasma/io/basic_connection.cc | 256 ++++++ cpp/src/plasma/io/basic_connection.h | 152 ++++ cpp/src/plasma/io/connection.cc | 343 ++++++++ cpp/src/plasma/io/connection.h | 181 +++++ cpp/src/plasma/malloc.cc | 1 + cpp/src/plasma/malloc.h | 4 +- cpp/src/plasma/plasma.cc | 63 -- cpp/src/plasma/plasma.h | 80 +- cpp/src/plasma/protocol.cc | 426 +++++----- cpp/src/plasma/protocol.h | 204 +++-- cpp/src/plasma/store.cc | 841 ++++++++------------ cpp/src/plasma/store.h | 154 ++-- cpp/src/plasma/test/client_tests.cc | 608 ++++---------- cpp/src/plasma/test/external_store_tests.cc | 16 +- cpp/src/plasma/test/serialization_tests.cc | 188 ++--- cpp/src/plasma/thirdparty/ae/ae.c | 465 ----------- cpp/src/plasma/thirdparty/ae/ae.h | 123 --- cpp/src/plasma/thirdparty/ae/ae_epoll.c | 137 ---- cpp/src/plasma/thirdparty/ae/ae_evport.c | 320 -------- cpp/src/plasma/thirdparty/ae/ae_kqueue.c | 138 ---- cpp/src/plasma/thirdparty/ae/ae_select.c | 106 --- cpp/src/plasma/thirdparty/ae/config.h | 54 -- cpp/src/plasma/thirdparty/ae/zmalloc.h | 45 -- dev/release/rat_exclude_files.txt | 8 - python/pyarrow/_plasma.pyx | 22 +- 40 files changed, 2235 insertions(+), 4039 deletions(-) delete mode 100644 cpp/src/plasma/events.cc delete mode 100644 cpp/src/plasma/events.h delete mode 100644 cpp/src/plasma/io.cc delete mode 100644 cpp/src/plasma/io.h create mode 100644 cpp/src/plasma/io/basic_connection.cc create mode 100644 cpp/src/plasma/io/basic_connection.h create mode 100644 cpp/src/plasma/io/connection.cc create mode 100644 cpp/src/plasma/io/connection.h delete mode 100644 cpp/src/plasma/thirdparty/ae/ae.c delete mode 100644 cpp/src/plasma/thirdparty/ae/ae.h delete mode 100644 cpp/src/plasma/thirdparty/ae/ae_epoll.c delete mode 100644 cpp/src/plasma/thirdparty/ae/ae_evport.c delete mode 100644 cpp/src/plasma/thirdparty/ae/ae_kqueue.c delete mode 100644 cpp/src/plasma/thirdparty/ae/ae_select.c delete mode 100644 cpp/src/plasma/thirdparty/ae/config.h delete mode 100644 cpp/src/plasma/thirdparty/ae/zmalloc.h diff --git a/LICENSE.txt b/LICENSE.txt index af3e97fc2b9..82d37dc2e13 100644 --- a/LICENSE.txt +++ b/LICENSE.txt @@ -221,37 +221,6 @@ limitations under the License. -------------------------------------------------------------------------------- -src/plasma/thirdparty/ae: Modified / 3-Clause BSD - -Copyright (c) 2006-2010, Salvatore Sanfilippo -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - - * Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - * Neither the name of Redis nor the names of its contributors may be used - to endorse or promote products derived from this software without - specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE -ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE -LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR -CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF -SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS -INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN -CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) -ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE -POSSIBILITY OF SUCH DAMAGE. - --------------------------------------------------------------------------------- - src/plasma/thirdparty/dlmalloc.c: CC0 This is a version (aka dlmalloc) of malloc/free/realloc written by diff --git a/cpp/apidoc/tutorials/plasma.md b/cpp/apidoc/tutorials/plasma.md index 40c5a10603e..21ce303f490 100644 --- a/cpp/apidoc/tutorials/plasma.md +++ b/cpp/apidoc/tutorials/plasma.md @@ -384,16 +384,14 @@ sealed in the object store. This may especially be handy when your program is collaborating with other Plasma clients, and needs to know when they make objects available. -First, you can subscribe your current Plasma client to such notifications -by getting a file descriptor: +First, you can subscribe your current Plasma client to such notifications: ```cpp // Start receiving notifications into file_descriptor. -int fd; -ARROW_CHECK_OK(client.Subscribe(&fd)); +ARROW_CHECK_OK(client.Subscribe()); ``` -Once you have the file descriptor, you can have your current Plasma client +Once you have subscribed, you can have your current Plasma client wait to receive the next object notification. Object notifications include information such as Object ID, data size, and metadata size of the next newly available object: @@ -404,7 +402,7 @@ the next newly available object: ObjectID object_id; int64_t data_size; int64_t metadata_size; -ARROW_CHECK_OK(client.GetNotification(fd, &object_id, &data_size, &metadata_size)); +ARROW_CHECK_OK(client.GetNotification(&object_id, &data_size, &metadata_size)); // Get the newly available object. ObjectBuffer object_buffer; @@ -423,14 +421,13 @@ int main(int argc, char** argv) { PlasmaClient client; ARROW_CHECK_OK(client.Connect("/tmp/plasma")); - int fd; - ARROW_CHECK_OK(client.Subscribe(&fd)); + ARROW_CHECK_OK(client.Subscribe()); ObjectID object_id; int64_t data_size; int64_t metadata_size; while (true) { - ARROW_CHECK_OK(client.GetNotification(fd, &object_id, &data_size, &metadata_size)); + ARROW_CHECK_OK(client.GetNotification(&object_id, &data_size, &metadata_size)); std::cout << "Received object notification for object_id = " << object_id.hex() << ", with data_size = " << data_size diff --git a/cpp/src/arrow/status.h b/cpp/src/arrow/status.h index bf1967014ea..984d8b541d2 100644 --- a/cpp/src/arrow/status.h +++ b/cpp/src/arrow/status.h @@ -17,11 +17,13 @@ #include #include -#include #include #include -#include "arrow/util/compare.h" +#ifdef ARROW_EXTRA_ERROR_CONTEXT +#include +#endif + #include "arrow/util/macros.h" #include "arrow/util/string_builder.h" #include "arrow/util/visibility.h" @@ -29,13 +31,14 @@ #ifdef ARROW_EXTRA_ERROR_CONTEXT /// \brief Return with given status if condition is met. -#define ARROW_RETURN_IF_(condition, status, expr) \ - do { \ - if (ARROW_PREDICT_FALSE(condition)) { \ - ::arrow::Status _st = (status); \ - _st.AddContextLine(__FILE__, __LINE__, expr); \ - return _st; \ - } \ +#define ARROW_RETURN_IF_(condition, status, expr) \ + do { \ + if (ARROW_PREDICT_FALSE(condition)) { \ + ::arrow::Status _s = (status); \ + std::stringstream ss; \ + ss << _s.message() << "\n" << __FILE__ << ":" << __LINE__ << " code: " << expr; \ + return ::arrow::Status(_s.code(), ss.str()); \ + } \ } while (0) #else @@ -83,17 +86,21 @@ enum class StatusCode : char { Invalid = 4, IOError = 5, CapacityError = 6, - IndexError = 7, UnknownError = 9, NotImplemented = 10, SerializationError = 11, + PythonError = 12, RError = 13, + ProtocolError = 14, + PlasmaObjectExists = 20, + PlasmaObjectNonexistent = 21, + PlasmaStoreFull = 22, + PlasmaObjectAlreadySealed = 23, + StillExecuting = 24, // Gandiva range of errors CodeGenError = 40, ExpressionValidationError = 41, - ExecutionError = 42, - // Continue generic codes. - AlreadyExists = 45 + ExecutionError = 42 }; #if defined(__clang__) @@ -101,22 +108,6 @@ enum class StatusCode : char { class ARROW_MUST_USE_RESULT ARROW_EXPORT Status; #endif -/// \brief An opaque class that allows subsystems to retain -/// additional information inside the Status. -class ARROW_EXPORT StatusDetail { - public: - virtual ~StatusDetail() = default; - /// \brief Return a unique id for the type of the StatusDetail - /// (effectively a poor man's substitude for RTTI). - virtual const char* type_id() const = 0; - /// \brief Produce a human-readable description of this status. - virtual std::string ToString() const = 0; - - bool operator==(const StatusDetail& other) const noexcept { - return std::string(type_id()) == other.type_id() && ToString() == other.ToString(); - } -}; - /// \brief Status outcome object (success or error) /// /// The Status object is an object holding the outcome of an operation. @@ -125,8 +116,7 @@ class ARROW_EXPORT StatusDetail { /// /// Additionally, if an error occurred, a specific error message is generally /// attached. -class ARROW_EXPORT Status : public util::EqualityComparable, - public util::ToStringOstreamable { +class ARROW_EXPORT Status { public: // Create a success status. Status() noexcept : state_(NULLPTR) {} @@ -139,8 +129,6 @@ class ARROW_EXPORT Status : public util::EqualityComparable, } Status(StatusCode code, const std::string& msg); - /// \brief Pluggable constructor for use by sub-systems. detail cannot be null. - Status(StatusCode code, std::string msg, std::shared_ptr detail); // Copy the specified status. inline Status(const Status& s); @@ -150,8 +138,6 @@ class ARROW_EXPORT Status : public util::EqualityComparable, inline Status(Status&& s) noexcept; inline Status& operator=(Status&& s) noexcept; - inline bool Equals(const Status& s) const; - // AND the statuses. inline Status operator&(const Status& s) const noexcept; inline Status operator&(Status&& s) const noexcept; @@ -161,6 +147,12 @@ class ARROW_EXPORT Status : public util::EqualityComparable, /// Return a success status static Status OK() { return Status(); } + /// Return a success status with a specific message + template + static Status OK(Args&&... args) { + return Status(StatusCode::OK, util::StringBuilder(std::forward(args)...)); + } + /// Return an error status for out-of-memory conditions template static Status OutOfMemory(Args&&... args) { @@ -202,13 +194,6 @@ class ARROW_EXPORT Status : public util::EqualityComparable, return Status(StatusCode::Invalid, util::StringBuilder(std::forward(args)...)); } - /// Return an error status when an index is out of bounds - template - static Status IndexError(Args&&... args) { - return Status(StatusCode::IndexError, - util::StringBuilder(std::forward(args)...)); - } - /// Return an error status when a container's capacity would exceed its limits template static Status CapacityError(Args&&... args) { @@ -234,6 +219,38 @@ class ARROW_EXPORT Status : public util::EqualityComparable, return Status(StatusCode::RError, util::StringBuilder(std::forward(args)...)); } + template + static Status ProtocolError(Args&&... args) { + return Status(StatusCode::ProtocolError, + util::StringBuilder(std::forward(args)...)); + } + + template + static Status PlasmaObjectExists(Args&&... args) { + return Status(StatusCode::PlasmaObjectExists, + util::StringBuilder(std::forward(args)...)); + } + + template + static Status PlasmaObjectNonexistent(Args&&... args) { + return Status(StatusCode::PlasmaObjectNonexistent, + util::StringBuilder(std::forward(args)...)); + } + + template + static Status PlasmaObjectAlreadySealed(Args&&... args) { + return Status(StatusCode::PlasmaObjectAlreadySealed, + util::StringBuilder(std::forward(args)...)); + } + + template + static Status PlasmaStoreFull(Args&&... args) { + return Status(StatusCode::PlasmaStoreFull, + util::StringBuilder(std::forward(args)...)); + } + + static Status StillExecuting() { return Status(StatusCode::StillExecuting, ""); } + template static Status CodeGenError(Args&&... args) { return Status(StatusCode::CodeGenError, @@ -252,12 +269,6 @@ class ARROW_EXPORT Status : public util::EqualityComparable, util::StringBuilder(std::forward(args)...)); } - template - static Status AlreadyExists(Args&&... args) { - return Status(StatusCode::AlreadyExists, - util::StringBuilder(std::forward(args)...)); - } - /// Return true iff the status indicates success. bool ok() const { return (state_ == NULLPTR); } @@ -271,8 +282,6 @@ class ARROW_EXPORT Status : public util::EqualityComparable, bool IsIOError() const { return code() == StatusCode::IOError; } /// Return true iff the status indicates a container reaching capacity limits. bool IsCapacityError() const { return code() == StatusCode::CapacityError; } - /// Return true iff the status indicates an out of bounds index. - bool IsIndexError() const { return code() == StatusCode::IndexError; } /// Return true iff the status indicates a type error. bool IsTypeError() const { return code() == StatusCode::TypeError; } /// Return true iff the status indicates an unknown error. @@ -283,6 +292,22 @@ class ARROW_EXPORT Status : public util::EqualityComparable, bool IsSerializationError() const { return code() == StatusCode::SerializationError; } /// Return true iff the status indicates a R-originated error. bool IsRError() const { return code() == StatusCode::RError; } + /// Return true iff the status indicates a Python-originated error. + bool IsPythonError() const { return code() == StatusCode::PythonError; } + /// Return true iff the status indicates an already existing Plasma object. + bool IsPlasmaObjectExists() const { return code() == StatusCode::PlasmaObjectExists; } + /// Return true iff the status indicates a non-existent Plasma object. + bool IsPlasmaObjectNonexistent() const { + return code() == StatusCode::PlasmaObjectNonexistent; + } + /// Return true iff the status indicates an already sealed Plasma object. + bool IsPlasmaObjectAlreadySealed() const { + return code() == StatusCode::PlasmaObjectAlreadySealed; + } + /// Return true iff the status indicates the Plasma store reached its capacity limit. + bool IsPlasmaStoreFull() const { return code() == StatusCode::PlasmaStoreFull; } + + bool IsStillExecuting() const { return code() == StatusCode::StillExecuting; } bool IsCodeGenError() const { return code() == StatusCode::CodeGenError; } @@ -307,35 +332,10 @@ class ARROW_EXPORT Status : public util::EqualityComparable, /// \brief Return the specific error message attached to this status. std::string message() const { return ok() ? "" : state_->msg; } - /// \brief Return the status detail attached to this message. - std::shared_ptr detail() const { - return state_ == NULLPTR ? NULLPTR : state_->detail; - } - - /// \brief Return a new Status copying the existing status, but - /// updating with the existing detail. - Status WithDetail(std::shared_ptr new_detail) const { - return Status(code(), message(), std::move(new_detail)); - } - - /// \brief Return a new Status with changed message, copying the - /// existing status code and detail. - Status WithMessage(std::string message) const { - return Status(code(), std::move(message), detail()); - } - - [[noreturn]] void Abort() const; - [[noreturn]] void Abort(const std::string& message) const; - -#ifdef ARROW_EXTRA_ERROR_CONTEXT - void AddContextLine(const char* filename, int line, const char* expr); -#endif - private: struct State { StatusCode code; std::string msg; - std::shared_ptr detail; }; // OK status has a `NULL` state_. Otherwise, `state_` points to // a `State` structure containing the error code and message(s) @@ -349,6 +349,11 @@ class ARROW_EXPORT Status : public util::EqualityComparable, inline void MoveFrom(Status& s); }; +static inline std::ostream& operator<<(std::ostream& os, const Status& x) { + os << x.ToString(); + return os; +} + void Status::MoveFrom(Status& s) { delete state_; state_ = s.state_; @@ -374,22 +379,6 @@ Status& Status::operator=(Status&& s) noexcept { return *this; } -bool Status::Equals(const Status& s) const { - if (state_ == s.state_) { - return true; - } - - if (ok() || s.ok()) { - return false; - } - - if (detail() != s.detail() && !(*detail() == *s.detail())) { - return false; - } - - return code() == s.code() && message() == s.message(); -} - /// \cond FALSE // (note: emits warnings on Doxygen < 1.8.15, // see https://github.com/doxygen/doxygen/issues/6295) diff --git a/cpp/src/plasma/CMakeLists.txt b/cpp/src/plasma/CMakeLists.txt index 5f668344ddb..8e605e13a05 100644 --- a/cpp/src/plasma/CMakeLists.txt +++ b/cpp/src/plasma/CMakeLists.txt @@ -22,6 +22,7 @@ add_custom_target(plasma-tests) add_dependencies(plasma-all plasma plasma-tests plasma-benchmarks) # For the moment, Plasma is versioned like Arrow +project(plasma VERSION "${ARROW_BASE_VERSION}") set(PLASMA_VERSION "${ARROW_VERSION}") find_package(Threads) @@ -32,25 +33,61 @@ set(PLASMA_FULL_SO_VERSION "${ARROW_FULL_SO_VERSION}") set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-conversion") +# Compile flatbuffers + +set(PLASMA_FBS_SRC "${CMAKE_CURRENT_LIST_DIR}/format/plasma.fbs" + "${CMAKE_CURRENT_LIST_DIR}/format/common.fbs") +set(OUTPUT_DIR ${ARROW_BINARY_DIR}/src/plasma) + +set(PLASMA_FBS_OUTPUT_FILES "${OUTPUT_DIR}/common_generated.h" + "${OUTPUT_DIR}/plasma_generated.h") + +add_custom_target(gen_plasma_fbs DEPENDS ${PLASMA_FBS_OUTPUT_FILES}) + +add_dependencies(gen_plasma_fbs flatbuffers::flatc) + +add_custom_command( + OUTPUT ${PLASMA_FBS_OUTPUT_FILES} + # The --gen-object-api flag generates a C++ class MessageT for each + # flatbuffers message Message, which can be used to store deserialized + # messages in data structures. This is currently used for ObjectInfo for + # example. + COMMAND flatbuffers::flatc + -c + -o + ${OUTPUT_DIR} + ${PLASMA_FBS_SRC} + --gen-object-api + --scoped-enums + DEPENDS ${PLASMA_FBS_SRC} + COMMENT "Running flatc compiler on ${PLASMA_FBS_SRC}" + VERBATIM) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fPIC") +# Set compiling options for asio headers. +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-conversion -Wno-documentation") + +set(PLASMA_IO_SRCS + fling.cc + io/basic_connection.cc + io/connection.cc + protocol.cc) + set(PLASMA_SRCS client.cc common.cc - fling.cc - io.cc malloc.cc plasma.cc - protocol.cc) + protocol.cc + ${PLASMA_IO_SRCS}) set(PLASMA_STORE_SRCS dlmalloc.cc - events.cc eviction_policy.cc - quota_aware_policy.cc plasma_allocator.cc - store.cc - thirdparty/ae/ae.c) + ${PLASMA_IO_SRCS} + store.cc) set(PLASMA_LINK_LIBS arrow_shared) set(PLASMA_STATIC_LINK_LIBS arrow_static) @@ -74,6 +111,8 @@ add_arrow_lib(plasma ${PLASMA_SRCS} OUTPUTS PLASMA_LIBRARIES + DEPENDENCIES + gen_plasma_fbs SHARED_LINK_FLAGS ${PLASMA_SHARED_LINK_FLAGS} SHARED_LINK_LIBS @@ -106,24 +145,24 @@ endif() list(APPEND PLASMA_EXTERNAL_STORE_SOURCES "external_store.cc" "hash_table_store.cc") -# We use static libraries for the plasma-store-server executable so that it can +# We use static libraries for the plasma_store_server executable so that it can # be copied around and used in different locations. add_executable(plasma-store-server ${PLASMA_EXTERNAL_STORE_SOURCES} ${PLASMA_STORE_SRCS}) if(ARROW_BUILD_STATIC) - target_link_libraries(plasma-store-server plasma_static ${PLASMA_STATIC_LINK_LIBS}) + target_link_libraries(plasma_store_server plasma_static ${PLASMA_STATIC_LINK_LIBS}) else() # Fallback to shared libs in the case that static libraries are not build. - target_link_libraries(plasma-store-server plasma_shared ${PLASMA_LINK_LIBS}) + target_link_libraries(plasma_store_server plasma_shared ${PLASMA_LINK_LIBS}) endif() -add_dependencies(plasma plasma-store-server) +add_dependencies(plasma plasma-store-server) if(ARROW_RPATH_ORIGIN) if(APPLE) set(_lib_install_rpath "@loader_path") else() set(_lib_install_rpath "\$ORIGIN") endif() - set_target_properties(plasma-store-server + set_target_properties(plasma_store_server PROPERTIES INSTALL_RPATH ${_lib_install_rpath}) elseif(APPLE) # With OSX and conda, we need to set the correct RPATH so that dependencies @@ -132,7 +171,7 @@ elseif(APPLE) # $ENV{CONDA_PREFIX}/lib but our test libraries and executables are not # installed there. if(NOT "$ENV{CONDA_PREFIX}" STREQUAL "" AND APPLE) - set_target_properties(plasma-store-server + set_target_properties(plasma_store_server PROPERTIES BUILD_WITH_INSTALL_RPATH TRUE INSTALL_RPATH_USE_LINK_PATH @@ -150,8 +189,8 @@ install(FILES common.h DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/plasma") # Plasma store -set_target_properties(plasma-store-server PROPERTIES INSTALL_RPATH_USE_LINK_PATH TRUE) -install(TARGETS plasma-store-server ${INSTALL_IS_OPTIONAL} DESTINATION +set_target_properties(plasma_store_server PROPERTIES INSTALL_RPATH_USE_LINK_PATH TRUE) +install(TARGETS plasma_store_server ${INSTALL_IS_OPTIONAL} DESTINATION ${CMAKE_INSTALL_BINDIR}) # pkg-config support @@ -236,9 +275,9 @@ add_plasma_test(test/client_tests EXTRA_LINK_LIBS ${PLASMA_TEST_LIBS} EXTRA_DEPENDENCIES - plasma-store-server) + plasma_store_server) add_plasma_test(test/external_store_tests EXTRA_LINK_LIBS ${PLASMA_TEST_LIBS} EXTRA_DEPENDENCIES - plasma-store-server) + plasma_store_server) diff --git a/cpp/src/plasma/client.cc b/cpp/src/plasma/client.cc index 72a92cb971b..8b968f94294 100644 --- a/cpp/src/plasma/client.cc +++ b/cpp/src/plasma/client.cc @@ -23,35 +23,21 @@ #include #endif -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include // PROT_READ, PROT_WRITE, MAP_SHARED, MAP_FAILED #include #include #include -#include #include #include +#include #include #include "arrow/buffer.h" -#include "arrow/util/thread_pool.h" +#include "arrow/util/thread-pool.h" #include "plasma/common.h" -#include "plasma/fling.h" -#include "plasma/io.h" #include "plasma/malloc.h" -#include "plasma/plasma.h" #include "plasma/protocol.h" #ifdef PLASMA_CUDA @@ -65,16 +51,15 @@ using arrow::cuda::CudaDeviceManager; #define XXH_INLINE_ALL 1 #define XXH_NAMESPACE plasma_client_ -#include "arrow/vendored/xxhash.h" +#include "arrow/vendored/xxhash/xxhash.h" #define XXH64_DEFAULT_SEED 0 -namespace fb = plasma::flatbuf; - namespace plasma { -using fb::MessageType; -using fb::PlasmaError; +using flatbuf::MessageType; +using flatbuf::PlasmaError; +using io::ServerConnection; using arrow::MutableBuffer; @@ -88,9 +73,6 @@ constexpr int64_t kBytesInMB = 1 << 20; // GPU support #ifdef PLASMA_CUDA - -namespace { - struct GpuProcessHandle { /// Pointer to CUDA buffer that is backing this GPU object. std::shared_ptr ptr; @@ -101,19 +83,8 @@ struct GpuProcessHandle { // This is necessary as IPC handles can only be mapped once per process. // Thus if multiple clients in the same process get the same gpu object, // they need to access the same mapped CudaBuffer. -std::unordered_map gpu_object_map; -std::mutex gpu_mutex; - -// Return a new CudaBuffer pointing to the same data as the GpuProcessHandle, -// but able to persist after the original IPC-backed buffer is closed -// (ARROW-5924). -std::shared_ptr MakeBufferFromGpuProcessHandle(GpuProcessHandle* handle) { - return std::make_shared(handle->ptr->mutable_data(), handle->ptr->size(), - handle->ptr->context()); -} - -} // namespace - +static std::unordered_map gpu_object_map; +static std::mutex gpu_mutex; #endif // ---------------------------------------------------------------------- @@ -211,6 +182,11 @@ class ClientMmapTableEntry { ARROW_DISALLOW_COPY_AND_ASSIGN(ClientMmapTableEntry); }; +Status PlasmaReceive(const std::shared_ptr& client, + MessageType message_type, std::vector* buffer) { + return client->ReadMessage(static_cast(message_type), buffer); +} + class PlasmaClient::Impl : public std::enable_shared_from_this { public: Impl(); @@ -218,11 +194,7 @@ class PlasmaClient::Impl : public std::enable_shared_from_this* data, int device_num = 0); @@ -230,10 +202,6 @@ class PlasmaClient::Impl : public std::enable_shared_from_this& object_ids, - const std::vector& data, - const std::vector& metadata); - Status Get(const std::vector& object_ids, int64_t timeout_ms, std::vector* object_buffers); @@ -256,18 +224,18 @@ class PlasmaClient::Impl : public std::enable_shared_from_this* object_ids, - std::vector* data_sizes, - std::vector* metadata_sizes); + Status GetNotification(ObjectID* object_id, int64_t* data_size, int64_t* metadata_size); - Status Disconnect(); + inline int GetNativeNotificationHandle() { + return notification_conn_->GetNativeHandle(); + } - std::string DebugString(); + Status Disconnect(); bool IsInUse(const ObjectID& object_id); @@ -279,8 +247,9 @@ class PlasmaClient::Impl : public std::enable_shared_from_this store_conn_; + std::shared_ptr notification_conn_; + /// The name of the socket we are connecting to. + std::string store_socket_name_; /// Table of dlmalloc buffer files that have been memory mapped so far. This /// is a hash table mapping a file descriptor to a struct containing the /// address of the corresponding memory-mapped file. @@ -325,8 +299,6 @@ class PlasmaClient::Impl : public std::enable_shared_from_this deletion_cache_; - /// A queue of notification - std::deque> pending_notification_; /// A mutex which protects this class. std::recursive_mutex client_mutex_; @@ -375,15 +347,14 @@ bool PlasmaClient::Impl::IsInUse(const ObjectID& object_id) { return (elem != objects_in_use_.end()); } -int PlasmaClient::Impl::GetStoreFd(int store_fd) { +Status PlasmaClient::Impl::GetStoreFd(int store_fd, int* fd) { auto entry = mmap_table_.find(store_fd); if (entry == mmap_table_.end()) { - int fd = recv_fd(store_conn_); - ARROW_CHECK(fd >= 0) << "recv not successful"; - return fd; + RETURN_NOT_OK(store_conn_->RecvFd(fd)); } else { - return entry->second->fd(); + *fd = entry->second->fd(); } + return Status::OK(); } void PlasmaClient::Impl::IncrementObjectCount(const ObjectID& object_id, @@ -431,7 +402,8 @@ Status PlasmaClient::Impl::Create(const ObjectID& object_id, int64_t data_size, // If the CreateReply included an error, then the store will not send a file // descriptor. if (device_num == 0) { - int fd = GetStoreFd(store_fd); + int fd; + RETURN_NOT_OK(GetStoreFd(store_fd, &fd)); ARROW_CHECK(object.data_size == data_size); ARROW_CHECK(object.metadata_size == metadata_size); // The metadata should come right after the data. @@ -457,12 +429,12 @@ Status PlasmaClient::Impl::Create(const ObjectID& object_id, int64_t data_size, std::lock_guard lock(gpu_mutex); gpu_object_map[object_id] = handle; } + *data = handle->ptr; if (metadata != NULL) { // Copy the metadata to the buffer. - CudaBufferWriter writer(handle->ptr); + CudaBufferWriter writer(std::dynamic_pointer_cast(*data)); RETURN_NOT_OK(writer.WriteAt(object.data_size, metadata, metadata_size)); } - *data = MakeBufferFromGpuProcessHandle(handle); #else ARROW_LOG(FATAL) << "Arrow GPU library is not enabled."; #endif @@ -504,37 +476,6 @@ Status PlasmaClient::Impl::CreateAndSeal(const ObjectID& object_id, return Status::OK(); } -Status PlasmaClient::Impl::CreateAndSealBatch(const std::vector& object_ids, - const std::vector& data, - const std::vector& metadata) { - std::lock_guard guard(client_mutex_); - - ARROW_LOG(DEBUG) << "called CreateAndSealBatch on conn " << store_conn_; - - int device_num = 0; - std::vector digests; - for (size_t i = 0; i < object_ids.size(); i++) { - // Compute the object hash. - std::string digest; - // CreateAndSeal currently only supports device_num = 0, which corresponds to - // the host. - uint64_t hash = ComputeObjectHash( - reinterpret_cast(data.data()), data.size(), - reinterpret_cast(metadata.data()), metadata.size(), device_num); - digest.assign(reinterpret_cast(&hash), sizeof(hash)); - digests.push_back(digest); - } - - RETURN_NOT_OK( - SendCreateAndSealBatchRequest(store_conn_, object_ids, data, metadata, digests)); - std::vector buffer; - RETURN_NOT_OK( - PlasmaReceive(store_conn_, MessageType::PlasmaCreateAndSealBatchReply, &buffer)); - RETURN_NOT_OK(ReadCreateAndSealBatchReply(buffer.data(), buffer.size())); - - return Status::OK(); -} - Status PlasmaClient::Impl::GetBuffers( const ObjectID* object_ids, int64_t num_objects, int64_t timeout_ms, const std::function( @@ -552,8 +493,10 @@ Status PlasmaClient::Impl::GetBuffers( // This client created the object but hasn't sealed it. If we call Get // with no timeout, we will deadlock, because this client won't be able to // call Seal. - ARROW_CHECK(timeout_ms != -1) - << "Plasma client called get on an unsealed object that it created"; + if (timeout_ms == -1) { + return Status::Invalid( + "Plasma client called get on an unsealed object that it created"); + } ARROW_LOG(WARNING) << "Attempting to get an object that this client created but hasn't sealed."; all_present = false; @@ -571,7 +514,7 @@ Status PlasmaClient::Impl::GetBuffers( auto iter = gpu_object_map.find(object_ids[i]); ARROW_CHECK(iter != gpu_object_map.end()); iter->second->client_count++; - physical_buf = MakeBufferFromGpuProcessHandle(iter->second); + physical_buf = iter->second->ptr; #else ARROW_LOG(FATAL) << "Arrow GPU library is not enabled."; #endif @@ -608,7 +551,8 @@ Status PlasmaClient::Impl::GetBuffers( // in the subsequent loop based on just the store file descriptor and without // having to know the relevant file descriptor received from recv_fd. for (size_t i = 0; i < store_fds.size(); i++) { - int fd = GetStoreFd(store_fds[i]); + int fd; + RETURN_NOT_OK(GetStoreFd(store_fds[i], &fd)); LookupOrMmap(fd, store_fds[i], mmap_sizes[i]); } @@ -634,18 +578,18 @@ Status PlasmaClient::Impl::GetBuffers( } else { #ifdef PLASMA_CUDA std::lock_guard lock(gpu_mutex); - auto iter = gpu_object_map.find(object_ids[i]); - if (iter == gpu_object_map.end()) { + auto handle = gpu_object_map.find(object_ids[i]); + if (handle == gpu_object_map.end()) { std::shared_ptr context; RETURN_NOT_OK(manager_->GetContext(object->device_num - 1, &context)); GpuProcessHandle* obj_handle = new GpuProcessHandle(); obj_handle->client_count = 1; RETURN_NOT_OK(context->OpenIpcBuffer(*object->ipc_handle, &obj_handle->ptr)); gpu_object_map[object_ids[i]] = obj_handle; - physical_buf = MakeBufferFromGpuProcessHandle(obj_handle); + physical_buf = obj_handle->ptr; } else { - iter->second->client_count++; - physical_buf = MakeBufferFromGpuProcessHandle(iter->second); + handle->second->client_count++; + physical_buf = handle->second->ptr; } #else ARROW_LOG(FATAL) << "Arrow GPU library is not enabled."; @@ -710,7 +654,12 @@ Status PlasmaClient::Impl::Release(const ObjectID& object_id) { return Status::OK(); } auto object_entry = objects_in_use_.find(object_id); - ARROW_CHECK(object_entry != objects_in_use_.end()); + if (object_entry == objects_in_use_.end()) { + return Status::Invalid("Trying to release a non-existing object."); + } + auto& entry = *object_entry->second; + entry.count -= 1; + ARROW_CHECK(entry.count >= 0) << "Got negative ref count."; #ifdef PLASMA_CUDA if (object_entry->second->object.device_num != 0) { @@ -724,10 +673,8 @@ Status PlasmaClient::Impl::Release(const ObjectID& object_id) { } #endif - object_entry->second->count -= 1; - ARROW_CHECK(object_entry->second->count >= 0); // Check if the client is no longer using this object. - if (object_entry->second->count == 0) { + if (entry.count == 0) { // Tell the store that the client no longer needs the object. RETURN_NOT_OK(MarkObjectUnused(object_id)); RETURN_NOT_OK(SendReleaseRequest(store_conn_, object_id)); @@ -858,10 +805,9 @@ Status PlasmaClient::Impl::Seal(const ObjectID& object_id) { object_entry->second->is_sealed = true; /// Send the seal request to Plasma. - std::vector digest(kDigestSize); + static unsigned char digest[kDigestSize]; RETURN_NOT_OK(Hash(object_id, &digest[0])); - RETURN_NOT_OK( - SendSealRequest(store_conn_, object_id, std::string(digest.begin(), digest.end()))); + RETURN_NOT_OK(SendSealRequest(store_conn_, object_id, &digest[0])); // We call PlasmaClient::Release to decrement the number of instances of this // object // that are currently being used by this client. The corresponding increment @@ -873,15 +819,21 @@ Status PlasmaClient::Impl::Seal(const ObjectID& object_id) { Status PlasmaClient::Impl::Abort(const ObjectID& object_id) { std::lock_guard guard(client_mutex_); auto object_entry = objects_in_use_.find(object_id); - ARROW_CHECK(object_entry != objects_in_use_.end()) - << "Plasma client called abort on an object without a reference to it"; - ARROW_CHECK(!object_entry->second->is_sealed) - << "Plasma client called abort on a sealed object"; + if (object_entry == objects_in_use_.end()) { + return Status::Invalid( + "Plasma client called abort on an object without a reference to it"); + } + + auto& entry = *object_entry->second; + + if (entry.is_sealed) { + return Status::Invalid("Plasma client called abort on a sealed object"); + } // Make sure that the Plasma client only has one reference to the object. If // it has more, then the client needs to release the buffer before calling // abort. - if (object_entry->second->count > 1) { + if (entry.count > 1) { return Status::Invalid("Plasma client cannot have a reference to the buffer."); } @@ -899,13 +851,12 @@ Status PlasmaClient::Impl::Abort(const ObjectID& object_id) { // Send the abort request. RETURN_NOT_OK(SendAbortRequest(store_conn_, object_id)); // Decrease the reference count to zero, then remove the object. - object_entry->second->count--; + entry.count--; RETURN_NOT_OK(MarkObjectUnused(object_id)); std::vector buffer; ObjectID id; - MessageType type; - RETURN_NOT_OK(ReadMessage(store_conn_, &type, &buffer)); + RETURN_NOT_OK(PlasmaReceive(store_conn_, MessageType::PlasmaAbortReply, &buffer)); return ReadAbortReply(buffer.data(), buffer.size(), &id); } @@ -941,8 +892,7 @@ Status PlasmaClient::Impl::Evict(int64_t num_bytes, int64_t& num_bytes_evicted) RETURN_NOT_OK(SendEvictRequest(store_conn_, num_bytes)); // Wait for a response with the number of bytes actually evicted. std::vector buffer; - MessageType type; - RETURN_NOT_OK(ReadMessage(store_conn_, &type, &buffer)); + RETURN_NOT_OK(PlasmaReceive(store_conn_, MessageType::PlasmaEvictReply, &buffer)); return ReadEvictReply(buffer.data(), buffer.size(), num_bytes_evicted); } @@ -963,94 +913,64 @@ Status PlasmaClient::Impl::Hash(const ObjectID& object_id, uint8_t* digest) { return Status::OK(); } -Status PlasmaClient::Impl::Subscribe(int* fd) { - std::lock_guard guard(client_mutex_); - - int sock[2]; - // Create a non-blocking socket pair. This will only be used to send - // notifications from the Plasma store to the client. - socketpair(AF_UNIX, SOCK_STREAM, 0, sock); - // Make the socket non-blocking. - int flags = fcntl(sock[1], F_GETFL, 0); - ARROW_CHECK(fcntl(sock[1], F_SETFL, flags | O_NONBLOCK) == 0); +Status PlasmaClient::Impl::Subscribe() { + if (store_socket_name_.empty()) { + ARROW_LOG(FATAL) << "Please connect to the store before subscribing messages."; + } + io::PlasmaStream stream(io_context_); + RETURN_NOT_OK(io::CreateLocalStream(store_socket_name_, &stream)); + auto conn = ServerConnection::Create(std::move(stream)); + notification_conn_ = std::move(conn); // Tell the Plasma store about the subscription. - RETURN_NOT_OK(SendSubscribeRequest(store_conn_)); - // Send the file descriptor that the Plasma store should use to push - // notifications about sealed objects to this client. - ARROW_CHECK(send_fd(store_conn_, sock[1]) >= 0); - close(sock[1]); - // Return the file descriptor that the client should use to read notifications - // about sealed objects. - *fd = sock[0]; - return Status::OK(); + return SendSubscribeRequest(notification_conn_); } -Status PlasmaClient::Impl::GetNotification(int fd, ObjectID* object_id, - int64_t* data_size, int64_t* metadata_size) { +// TODO(suquark): Move it to protocol.cc +Status PlasmaClient::Impl::DecodeNotification(const uint8_t* buffer, ObjectID* object_id, + int64_t* data_size, + int64_t* metadata_size) { std::lock_guard guard(client_mutex_); - if (pending_notification_.empty()) { - auto message = ReadMessageAsync(fd); - if (message == NULL) { - return Status::IOError("Failed to read object notification from Plasma socket"); - } - - std::vector object_ids; - std::vector data_sizes; - std::vector metadata_sizes; - RETURN_NOT_OK( - DecodeNotifications(message.get(), &object_ids, &data_sizes, &metadata_sizes)); - for (size_t i = 0; i < object_ids.size(); ++i) { - pending_notification_.emplace_back(object_ids[i], data_sizes[i], metadata_sizes[i]); - } + auto object_info = flatbuffers::GetRoot(buffer); + if (object_info->object_id()->size() != sizeof(ObjectID)) { + return Status::Invalid( + "The size of ObjectID in the message is different from the size " + "of ObjectID in Plasma. The message could have been corrupt."); + } + memcpy(object_id, object_info->object_id()->data(), sizeof(ObjectID)); + if (object_info->is_deletion()) { + *data_size = -1; + *metadata_size = -1; + } else { + *data_size = object_info->data_size(); + *metadata_size = object_info->metadata_size(); } - - auto notification = pending_notification_.front(); - *object_id = std::get<0>(notification); - *data_size = std::get<1>(notification); - *metadata_size = std::get<2>(notification); - - pending_notification_.pop_front(); - return Status::OK(); } -Status PlasmaClient::Impl::DecodeNotifications(const uint8_t* buffer, - std::vector* object_ids, - std::vector* data_sizes, - std::vector* metadata_sizes) { +Status PlasmaClient::Impl::GetNotification(ObjectID* object_id, int64_t* data_size, + int64_t* metadata_size) { std::lock_guard guard(client_mutex_); - auto object_info = flatbuffers::GetRoot(buffer); - - for (size_t i = 0; i < object_info->object_info()->size(); ++i) { - auto info = object_info->object_info()->Get(i); - ObjectID id = ObjectID::from_binary(info->object_id()->str()); - object_ids->push_back(id); - if (info->is_deletion()) { - data_sizes->push_back(-1); - metadata_sizes->push_back(-1); - } else { - data_sizes->push_back(info->data_size()); - metadata_sizes->push_back(info->metadata_size()); - } + std::unique_ptr notification; + if (!notification_conn_) { + ARROW_LOG(ERROR) << "Get notification without subscription."; + return Status::ExecutionError("Get notification without subscription."); } - - return Status::OK(); + auto status = notification_conn_->ReadNotificationMessage(notification); + if (!status.ok()) { + return Status::IOError("Failed to read object notification from Plasma socket"); + } + return DecodeNotification(notification.get(), object_id, data_size, metadata_size); } -Status PlasmaClient::Impl::Connect(const std::string& store_socket_name, - const std::string& manager_socket_name, - int release_delay, int num_retries) { +Status PlasmaClient::Impl::Connect(const std::string& store_socket_name) { std::lock_guard guard(client_mutex_); + store_socket_name_ = store_socket_name; + io::PlasmaStream stream(io_context_); + RETURN_NOT_OK(io::CreateLocalStream(store_socket_name_, &stream)); + auto conn = ServerConnection::Create(std::move(stream)); + store_conn_ = std::move(conn); - RETURN_NOT_OK(ConnectIpcSocketRetry(store_socket_name, num_retries, -1, &store_conn_)); - if (manager_socket_name != "") { - return Status::NotImplemented("plasma manager is no longer supported"); - } - if (release_delay != 0) { - ARROW_LOG(WARNING) << "The release_delay parameter in PlasmaClient::Connect " - << "is deprecated"; - } // Send a ConnectRequest to the store to get its memory capacity. RETURN_NOT_OK(SendConnectRequest(store_conn_)); std::vector buffer; @@ -1059,15 +979,6 @@ Status PlasmaClient::Impl::Connect(const std::string& store_socket_name, return Status::OK(); } -Status PlasmaClient::Impl::SetClientOptions(const std::string& client_name, - int64_t output_memory_quota) { - std::lock_guard guard(client_mutex_); - RETURN_NOT_OK(SendSetOptionsRequest(store_conn_, client_name, output_memory_quota)); - std::vector buffer; - RETURN_NOT_OK(PlasmaReceive(store_conn_, MessageType::PlasmaSetOptionsReply, &buffer)); - return ReadSetOptionsReply(buffer.data(), buffer.size()); -} - Status PlasmaClient::Impl::Disconnect() { std::lock_guard guard(client_mutex_); @@ -1077,25 +988,14 @@ Status PlasmaClient::Impl::Disconnect() { // Close the connections to Plasma. The Plasma store will release the objects // that were in use by us when handling the SIGPIPE. - close(store_conn_); - store_conn_ = -1; - return Status::OK(); -} - -std::string PlasmaClient::Impl::DebugString() { - std::lock_guard guard(client_mutex_); - if (!SendGetDebugStringRequest(store_conn_).ok()) { - return "error sending request"; - } - std::vector buffer; - if (!PlasmaReceive(store_conn_, MessageType::PlasmaGetDebugStringReply, &buffer).ok()) { - return "error receiving reply"; - } - std::string debug_string; - if (!ReadGetDebugStringReply(buffer.data(), buffer.size(), &debug_string).ok()) { - return "error parsing reply"; + if (notification_conn_) { + auto status = notification_conn_->Disconnect(); + if (!status.ok()) { + ARROW_LOG(ERROR) << "Failed to disconnect notification client " + << "(" << status << ")"; + } } - return debug_string; + return store_conn_->Disconnect(); } // ---------------------------------------------------------------------- @@ -1108,13 +1008,19 @@ PlasmaClient::~PlasmaClient() {} Status PlasmaClient::Connect(const std::string& store_socket_name, const std::string& manager_socket_name, int release_delay, int num_retries) { - return impl_->Connect(store_socket_name, manager_socket_name, release_delay, - num_retries); -} - -Status PlasmaClient::SetClientOptions(const std::string& client_name, - int64_t output_memory_quota) { - return impl_->SetClientOptions(client_name, output_memory_quota); + // Keep "manager_socket_name" & "release_delay" for compatibility. + if (manager_socket_name != "") { + return Status::NotImplemented("plasma manager is no longer supported"); + } + if (release_delay != 0) { + ARROW_LOG(WARNING) << "The release_delay parameter in PlasmaClient::Connect " + << "is deprecated"; + } + if (num_retries != -1) { + ARROW_LOG(WARNING) << "The num_retries parameter in PlasmaClient::Connect " + << "is deprecated"; + } + return impl_->Connect(store_socket_name); } Status PlasmaClient::Create(const ObjectID& object_id, int64_t data_size, @@ -1128,12 +1034,6 @@ Status PlasmaClient::CreateAndSeal(const ObjectID& object_id, const std::string& return impl_->CreateAndSeal(object_id, data, metadata); } -Status PlasmaClient::CreateAndSealBatch(const std::vector& object_ids, - const std::vector& data, - const std::vector& metadata) { - return impl_->CreateAndSealBatch(object_ids, data, metadata); -} - Status PlasmaClient::Get(const std::vector& object_ids, int64_t timeout_ms, std::vector* object_buffers) { return impl_->Get(object_ids, timeout_ms, object_buffers); @@ -1174,23 +1074,23 @@ Status PlasmaClient::Hash(const ObjectID& object_id, uint8_t* digest) { return impl_->Hash(object_id, digest); } -Status PlasmaClient::Subscribe(int* fd) { return impl_->Subscribe(fd); } +Status PlasmaClient::Subscribe() { return impl_->Subscribe(); } -Status PlasmaClient::GetNotification(int fd, ObjectID* object_id, int64_t* data_size, +Status PlasmaClient::GetNotification(ObjectID* object_id, int64_t* data_size, int64_t* metadata_size) { - return impl_->GetNotification(fd, object_id, data_size, metadata_size); + return impl_->GetNotification(object_id, data_size, metadata_size); } -Status PlasmaClient::DecodeNotifications(const uint8_t* buffer, - std::vector* object_ids, - std::vector* data_sizes, - std::vector* metadata_sizes) { - return impl_->DecodeNotifications(buffer, object_ids, data_sizes, metadata_sizes); +Status PlasmaClient::DecodeNotification(const uint8_t* buffer, ObjectID* object_id, + int64_t* data_size, int64_t* metadata_size) { + return impl_->DecodeNotification(buffer, object_id, data_size, metadata_size); } -Status PlasmaClient::Disconnect() { return impl_->Disconnect(); } +int PlasmaClient::GetNativeNotificationHandle() { + return impl_->GetNativeNotificationHandle(); +} -std::string PlasmaClient::DebugString() { return impl_->DebugString(); } +Status PlasmaClient::Disconnect() { return impl_->Disconnect(); } bool PlasmaClient::IsInUse(const ObjectID& object_id) { return impl_->IsInUse(object_id); diff --git a/cpp/src/plasma/client.h b/cpp/src/plasma/client.h index 6ee167ae568..f17c66dbb3c 100644 --- a/cpp/src/plasma/client.h +++ b/cpp/src/plasma/client.h @@ -59,7 +59,8 @@ class ARROW_EXPORT PlasmaClient { /// Note that plasma manager is no longer supported, this function /// will return failure if this is not "". /// \param release_delay Deprecated (not used). - /// \param num_retries number of attempts to connect to IPC socket, default 50 + /// \param num_retries number of attempts to connect to IPC socket, + /// default 50. Deprecated (not used). /// \return The return status. Status Connect(const std::string& store_socket_name, const std::string& manager_socket_name = "", int release_delay = 0, @@ -243,21 +244,19 @@ class ARROW_EXPORT PlasmaClient { /// Whenever an object is sealed, a message will be written to the client /// socket that is returned by this method. /// - /// \param fd Out parameter for the file descriptor the client should use to - /// read notifications - /// from the object store about sealed objects. /// \return The return status. - Status Subscribe(int* fd); + Status Subscribe(); + + /// Return the native handle of the notification client. + int GetNativeNotificationHandle(); /// Receive next object notification for this client if Subscribe has been called. /// - /// \param fd The file descriptor we are reading the notification from. /// \param object_id Out parameter, the object_id of the object that was sealed. /// \param data_size Out parameter, the data size of the object that was sealed. /// \param metadata_size Out parameter, the metadata size of the object that was sealed. /// \return The return status. - Status GetNotification(int fd, ObjectID* object_id, int64_t* data_size, - int64_t* metadata_size); + Status GetNotification(ObjectID* object_id, int64_t* data_size, int64_t* metadata_size); Status DecodeNotifications(const uint8_t* buffer, std::vector* object_ids, std::vector* data_sizes, diff --git a/cpp/src/plasma/common.cc b/cpp/src/plasma/common.cc index bbcd2c9c3f1..ce58f89b4fe 100644 --- a/cpp/src/plasma/common.cc +++ b/cpp/src/plasma/common.cc @@ -22,11 +22,53 @@ #include "arrow/util/ubsan.h" -#include "plasma/plasma_generated.h" +namespace plasma { -namespace fb = plasma::flatbuf; +namespace { -namespace plasma { +const char kErrorDetailTypeId[] = "plasma::PlasmaStatusDetail"; + +class PlasmaStatusDetail : public arrow::StatusDetail { + public: + explicit PlasmaStatusDetail(PlasmaErrorCode code) : code_(code) {} + const char* type_id() const override { return kErrorDetailTypeId; } + std::string ToString() const override { + const char* type; + switch (code()) { + case PlasmaErrorCode::PlasmaObjectExists: + type = "Plasma object exists"; + break; + case PlasmaErrorCode::PlasmaObjectNonexistent: + type = "Plasma object is nonexistent"; + break; + case PlasmaErrorCode::PlasmaStoreFull: + type = "Plasma store is full"; + break; + case PlasmaErrorCode::PlasmaObjectAlreadySealed: + type = "Plasma object is already sealed"; + break; + default: + type = "Unknown plasma error"; + break; + } + return std::string(type); + } + PlasmaErrorCode code() const { return code_; } + + private: + PlasmaErrorCode code_; +}; + +bool IsPlasmaStatus(const arrow::Status& status, PlasmaErrorCode code) { + if (status.ok()) { + return false; + } + auto* detail = status.detail().get(); + return detail != nullptr && detail->type_id() == kErrorDetailTypeId && + static_cast(detail)->code() == code; +} + +} // namespace namespace { diff --git a/cpp/src/plasma/dlmalloc.cc b/cpp/src/plasma/dlmalloc.cc index 463e967e036..6acbf46e705 100644 --- a/cpp/src/plasma/dlmalloc.cc +++ b/cpp/src/plasma/dlmalloc.cc @@ -29,6 +29,8 @@ #include #include +#include "arrow/util/logging.h" + #include "plasma/common.h" #include "plasma/plasma.h" diff --git a/cpp/src/plasma/events.cc b/cpp/src/plasma/events.cc deleted file mode 100644 index 28ff1267545..00000000000 --- a/cpp/src/plasma/events.cc +++ /dev/null @@ -1,107 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 "plasma/events.h" - -#include - -#include - -extern "C" { -#include "plasma/thirdparty/ae/ae.h" -} - -namespace plasma { - -// Verify that the constants defined in events.h are defined correctly. -static_assert(kEventLoopTimerDone == AE_NOMORE, "constant defined incorrectly"); -static_assert(kEventLoopOk == AE_OK, "constant defined incorrectly"); -static_assert(kEventLoopRead == AE_READABLE, "constant defined incorrectly"); -static_assert(kEventLoopWrite == AE_WRITABLE, "constant defined incorrectly"); - -void EventLoop::FileEventCallback(aeEventLoop* loop, int fd, void* context, int events) { - FileCallback* callback = reinterpret_cast(context); - (*callback)(events); -} - -int EventLoop::TimerEventCallback(aeEventLoop* loop, TimerID timer_id, void* context) { - TimerCallback* callback = reinterpret_cast(context); - return (*callback)(timer_id); -} - -constexpr int kInitialEventLoopSize = 1024; - -EventLoop::EventLoop() { loop_ = aeCreateEventLoop(kInitialEventLoopSize); } - -bool EventLoop::AddFileEvent(int fd, int events, const FileCallback& callback) { - if (file_callbacks_.find(fd) != file_callbacks_.end()) { - return false; - } - auto data = std::unique_ptr(new FileCallback(callback)); - void* context = reinterpret_cast(data.get()); - // Try to add the file descriptor. - int err = aeCreateFileEvent(loop_, fd, events, EventLoop::FileEventCallback, context); - // If it cannot be added, increase the size of the event loop. - if (err == AE_ERR && errno == ERANGE) { - err = aeResizeSetSize(loop_, 3 * aeGetSetSize(loop_) / 2); - if (err != AE_OK) { - return false; - } - err = aeCreateFileEvent(loop_, fd, events, EventLoop::FileEventCallback, context); - } - // In any case, test if there were errors. - if (err == AE_OK) { - file_callbacks_.emplace(fd, std::move(data)); - return true; - } - return false; -} - -void EventLoop::RemoveFileEvent(int fd) { - aeDeleteFileEvent(loop_, fd, AE_READABLE | AE_WRITABLE); - file_callbacks_.erase(fd); -} - -void EventLoop::Start() { aeMain(loop_); } - -void EventLoop::Stop() { aeStop(loop_); } - -void EventLoop::Shutdown() { - if (loop_ != nullptr) { - aeDeleteEventLoop(loop_); - loop_ = nullptr; - } -} - -EventLoop::~EventLoop() { Shutdown(); } - -int64_t EventLoop::AddTimer(int64_t timeout, const TimerCallback& callback) { - auto data = std::unique_ptr(new TimerCallback(callback)); - void* context = reinterpret_cast(data.get()); - int64_t timer_id = - aeCreateTimeEvent(loop_, timeout, EventLoop::TimerEventCallback, context, NULL); - timer_callbacks_.emplace(timer_id, std::move(data)); - return timer_id; -} - -int EventLoop::RemoveTimer(int64_t timer_id) { - int err = aeDeleteTimeEvent(loop_, timer_id); - timer_callbacks_.erase(timer_id); - return err; -} - -} // namespace plasma diff --git a/cpp/src/plasma/events.h b/cpp/src/plasma/events.h deleted file mode 100644 index 765be9c01fb..00000000000 --- a/cpp/src/plasma/events.h +++ /dev/null @@ -1,111 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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. - -#ifndef PLASMA_EVENTS -#define PLASMA_EVENTS - -#include -#include -#include - -struct aeEventLoop; - -namespace plasma { - -// The constants below are defined using hardcoded values taken from ae.h so -// that ae.h does not need to be included in this file. - -/// Constant specifying that the timer is done and it will be removed. -constexpr int kEventLoopTimerDone = -1; // AE_NOMORE - -/// A successful status. -constexpr int kEventLoopOk = 0; // AE_OK - -/// Read event on the file descriptor. -constexpr int kEventLoopRead = 1; // AE_READABLE - -/// Write event on the file descriptor. -constexpr int kEventLoopWrite = 2; // AE_WRITABLE - -typedef long long TimerID; // NOLINT - -class EventLoop { - public: - // Signature of the handler that will be called when there is a new event - // on the file descriptor that this handler has been registered for. - // - // The arguments are the event flags (read or write). - using FileCallback = std::function; - - // This handler will be called when a timer times out. The timer id is - // passed as an argument. The return is the number of milliseconds the timer - // shall be reset to or kEventLoopTimerDone if the timer shall not be - // triggered again. - using TimerCallback = std::function; - - EventLoop(); - - ~EventLoop(); - - /// Add a new file event handler to the event loop. - /// - /// \param fd The file descriptor we are listening to. - /// \param events The flags for events we are listening to (read or write). - /// \param callback The callback that will be called when the event happens. - /// \return Returns true if the event handler was added successfully. - bool AddFileEvent(int fd, int events, const FileCallback& callback); - - /// Remove a file event handler from the event loop. - /// - /// \param fd The file descriptor of the event handler. - void RemoveFileEvent(int fd); - - /// Register a handler that will be called after a time slice of - /// "timeout" milliseconds. - /// - /// \param timeout The timeout in milliseconds. - /// \param callback The callback for the timeout. - /// \return The ID of the newly created timer. - int64_t AddTimer(int64_t timeout, const TimerCallback& callback); - - /// Remove a timer handler from the event loop. - /// - /// \param timer_id The ID of the timer that is to be removed. - /// \return The ae.c error code. TODO(pcm): needs to be standardized - int RemoveTimer(int64_t timer_id); - - /// \brief Run the event loop. - void Start(); - - /// \brief Stop the event loop - void Stop(); - - void Shutdown(); - - private: - static void FileEventCallback(aeEventLoop* loop, int fd, void* context, int events); - - static int TimerEventCallback(aeEventLoop* loop, TimerID timer_id, void* context); - - aeEventLoop* loop_; - std::unordered_map> file_callbacks_; - std::unordered_map> timer_callbacks_; -}; - -} // namespace plasma - -#endif // PLASMA_EVENTS diff --git a/cpp/src/plasma/eviction_policy.cc b/cpp/src/plasma/eviction_policy.cc index a9cbb89582c..aa4339de19e 100644 --- a/cpp/src/plasma/eviction_policy.cc +++ b/cpp/src/plasma/eviction_policy.cc @@ -16,6 +16,7 @@ // under the License. #include "plasma/eviction_policy.h" +#include "arrow/util/logging.h" #include "plasma/plasma_allocator.h" #include diff --git a/cpp/src/plasma/eviction_policy.h b/cpp/src/plasma/eviction_policy.h index d4fabb9b5f0..0ff1a1b7ca7 100644 --- a/cpp/src/plasma/eviction_policy.h +++ b/cpp/src/plasma/eviction_policy.h @@ -95,9 +95,9 @@ class EvictionPolicy { public: /// Construct an eviction policy. /// - /// @param store_info Information about the Plasma store that is exposed + /// \param store_info Information about the Plasma store that is exposed /// to the eviction policy. - /// @param max_size Max size in bytes total of objects to store. + /// \param max_size Max size in bytes total of objects to store. explicit EvictionPolicy(PlasmaStoreInfo* store_info, int64_t max_size); /// Destroy an eviction policy. @@ -108,38 +108,38 @@ class EvictionPolicy { /// store calls begin_object_access, we can remove the object from the LRU /// cache. /// - /// @param object_id The object ID of the object that was created. - /// @param client The pointer to the client. - /// @param is_create Whether we are creating a new object (vs reading an object). + /// \param object_id The object ID of the object that was created. + /// \param client The pointer to the client. + /// \param is_create Whether we are creating a new object (vs reading an object). virtual void ObjectCreated(const ObjectID& object_id, Client* client, bool is_create); /// Set quota for a client. /// - /// @param client The pointer to the client. - /// @param output_memory_quota Set the quota for this client. This can only be + /// \param client The pointer to the client. + /// \param output_memory_quota Set the quota for this client. This can only be /// called once per client. This is effectively the equivalent of giving /// the client its own LRU cache instance. The memory for this is taken /// out of the capacity of the global LRU cache for the client lifetime. /// - /// @return True if enough space can be reserved for the given client quota. + /// \return True if enough space can be reserved for the given client quota. virtual bool SetClientQuota(Client* client, int64_t output_memory_quota); /// Determine what objects need to be evicted to enforce the given client's quota. /// - /// @param client The pointer to the client creating the object. - /// @param size The size of the object to create. - /// @param is_create Whether we are creating a new object (vs reading an object). - /// @param objects_to_evict The object IDs that were chosen for eviction will + /// \param client The pointer to the client creating the object. + /// \param size The size of the object to create. + /// \param is_create Whether we are creating a new object (vs reading an object). + /// \param objects_to_evict The object IDs that were chosen for eviction will /// be stored into this vector. /// - /// @return True if enough space could be freed and false otherwise. + /// \return True if enough space could be freed and false otherwise. virtual bool EnforcePerClientQuota(Client* client, int64_t size, bool is_create, std::vector* objects_to_evict); /// Called to clean up any resources allocated by this client. This merges any /// per-client LRU queue created by SetClientQuota into the global LRU queue. /// - /// @param client The pointer to the client. + /// \param client The pointer to the client. virtual void ClientDisconnected(Client* client); /// This method will be called when the Plasma store needs more space, perhaps @@ -147,11 +147,11 @@ class EvictionPolicy { /// policy will assume that the objects chosen to be evicted will in fact be /// evicted from the Plasma store by the caller. /// - /// @param size The size in bytes of the new object, including both data and + /// \param size The size in bytes of the new object, including both data and /// metadata. - /// @param objects_to_evict The object IDs that were chosen for eviction will + /// \param objects_to_evict The object IDs that were chosen for eviction will /// be stored into this vector. - /// @return True if enough space can be freed and false otherwise. + /// \return True if enough space can be freed and false otherwise. virtual bool RequireSpace(int64_t size, std::vector* objects_to_evict); /// This method will be called whenever an unused object in the Plasma store @@ -159,7 +159,7 @@ class EvictionPolicy { /// assume that the objects chosen to be evicted will in fact be evicted from /// the Plasma store by the caller. /// - /// @param object_id The ID of the object that is now being used. + /// \param object_id The ID of the object that is now being used. virtual void BeginObjectAccess(const ObjectID& object_id); /// This method will be called whenever an object in the Plasma store that was @@ -167,26 +167,26 @@ class EvictionPolicy { /// eviction policy will assume that the objects chosen to be evicted will in /// fact be evicted from the Plasma store by the caller. /// - /// @param object_id The ID of the object that is no longer being used. + /// \param object_id The ID of the object that is no longer being used. virtual void EndObjectAccess(const ObjectID& object_id); /// Choose some objects to evict from the Plasma store. When this method is /// called, the eviction policy will assume that the objects chosen to be /// evicted will in fact be evicted from the Plasma store by the caller. /// - /// @note This method is not part of the API. It is exposed in the header file + /// \note This method is not part of the API. It is exposed in the header file /// only for testing. /// - /// @param num_bytes_required The number of bytes of space to try to free up. - /// @param objects_to_evict The object IDs that were chosen for eviction will + /// \param num_bytes_required The number of bytes of space to try to free up. + /// \param objects_to_evict The object IDs that were chosen for eviction will /// be stored into this vector. - /// @return The total number of bytes of space chosen to be evicted. + /// \return The total number of bytes of space chosen to be evicted. virtual int64_t ChooseObjectsToEvict(int64_t num_bytes_required, std::vector* objects_to_evict); /// This method will be called when an object is going to be removed /// - /// @param object_id The ID of the object that is now being used. + /// \param object_id The ID of the object that is now being used. virtual void RemoveObject(const ObjectID& object_id); /// Returns debugging information for this eviction policy. diff --git a/cpp/src/plasma/fling.h b/cpp/src/plasma/fling.h index 78ac9d17f26..f05803db690 100644 --- a/cpp/src/plasma/fling.h +++ b/cpp/src/plasma/fling.h @@ -40,13 +40,13 @@ void init_msg(struct msghdr* msg, struct iovec* iov, char* buf, size_t buf_len); // Send a file descriptor over a unix domain socket. // -// @param conn Unix domain socket to send the file descriptor over. -// @param fd File descriptor to send over. -// @return Status code which is < 0 on failure. +// \param conn Unix domain socket to send the file descriptor over. +// \param fd File descriptor to send over. +// \return Status code which is < 0 on failure. int send_fd(int conn, int fd); // Receive a file descriptor over a unix domain socket. // -// @param conn Unix domain socket to receive the file descriptor from. -// @return File descriptor or a value < 0 on failure. +// \param conn Unix domain socket to receive the file descriptor from. +// \return File descriptor or a value < 0 on failure. int recv_fd(int conn); diff --git a/cpp/src/plasma/io.cc b/cpp/src/plasma/io.cc deleted file mode 100644 index ba5f2551919..00000000000 --- a/cpp/src/plasma/io.cc +++ /dev/null @@ -1,241 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 "plasma/io.h" - -#include -#include -#include - -#include "arrow/status.h" -#include "arrow/util/logging.h" - -#include "plasma/common.h" -#include "plasma/plasma_generated.h" - -using arrow::Status; - -/// Number of times we try connecting to a socket. -constexpr int64_t kNumConnectAttempts = 20; -/// Time to wait between connection attempts to a socket. -constexpr int64_t kConnectTimeoutMs = 400; - -namespace plasma { - -using flatbuf::MessageType; - -Status WriteBytes(int fd, uint8_t* cursor, size_t length) { - ssize_t nbytes = 0; - size_t bytesleft = length; - size_t offset = 0; - while (bytesleft > 0) { - // While we haven't written the whole message, write to the file descriptor, - // advance the cursor, and decrease the amount left to write. - nbytes = write(fd, cursor + offset, bytesleft); - if (nbytes < 0) { - if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) { - continue; - } - return Status::IOError(strerror(errno)); - } else if (nbytes == 0) { - return Status::IOError("Encountered unexpected EOF"); - } - ARROW_CHECK(nbytes > 0); - bytesleft -= nbytes; - offset += nbytes; - } - - return Status::OK(); -} - -Status WriteMessage(int fd, MessageType type, int64_t length, uint8_t* bytes) { - int64_t version = kPlasmaProtocolVersion; - RETURN_NOT_OK(WriteBytes(fd, reinterpret_cast(&version), sizeof(version))); - RETURN_NOT_OK(WriteBytes(fd, reinterpret_cast(&type), sizeof(type))); - RETURN_NOT_OK(WriteBytes(fd, reinterpret_cast(&length), sizeof(length))); - return WriteBytes(fd, bytes, length * sizeof(char)); -} - -Status ReadBytes(int fd, uint8_t* cursor, size_t length) { - ssize_t nbytes = 0; - // Termination condition: EOF or read 'length' bytes total. - size_t bytesleft = length; - size_t offset = 0; - while (bytesleft > 0) { - nbytes = read(fd, cursor + offset, bytesleft); - if (nbytes < 0) { - if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) { - continue; - } - return Status::IOError(strerror(errno)); - } else if (0 == nbytes) { - return Status::IOError("Encountered unexpected EOF"); - } - ARROW_CHECK(nbytes > 0); - bytesleft -= nbytes; - offset += nbytes; - } - - return Status::OK(); -} - -Status ReadMessage(int fd, MessageType* type, std::vector* buffer) { - int64_t version; - RETURN_NOT_OK_ELSE(ReadBytes(fd, reinterpret_cast(&version), sizeof(version)), - *type = MessageType::PlasmaDisconnectClient); - ARROW_CHECK(version == kPlasmaProtocolVersion) << "version = " << version; - RETURN_NOT_OK_ELSE(ReadBytes(fd, reinterpret_cast(type), sizeof(*type)), - *type = MessageType::PlasmaDisconnectClient); - int64_t length_temp; - RETURN_NOT_OK_ELSE( - ReadBytes(fd, reinterpret_cast(&length_temp), sizeof(length_temp)), - *type = MessageType::PlasmaDisconnectClient); - // The length must be read as an int64_t, but it should be used as a size_t. - size_t length = static_cast(length_temp); - if (length > buffer->size()) { - buffer->resize(length); - } - RETURN_NOT_OK_ELSE(ReadBytes(fd, buffer->data(), length), - *type = MessageType::PlasmaDisconnectClient); - return Status::OK(); -} - -int BindIpcSock(const std::string& pathname, bool shall_listen) { - struct sockaddr_un socket_address; - int socket_fd = socket(AF_UNIX, SOCK_STREAM, 0); - if (socket_fd < 0) { - ARROW_LOG(ERROR) << "socket() failed for pathname " << pathname; - return -1; - } - // Tell the system to allow the port to be reused. - int on = 1; - if (setsockopt(socket_fd, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast(&on), - sizeof(on)) < 0) { - ARROW_LOG(ERROR) << "setsockopt failed for pathname " << pathname; - close(socket_fd); - return -1; - } - - unlink(pathname.c_str()); - memset(&socket_address, 0, sizeof(socket_address)); - socket_address.sun_family = AF_UNIX; - if (pathname.size() + 1 > sizeof(socket_address.sun_path)) { - ARROW_LOG(ERROR) << "Socket pathname is too long."; - close(socket_fd); - return -1; - } - strncpy(socket_address.sun_path, pathname.c_str(), pathname.size() + 1); - - if (bind(socket_fd, reinterpret_cast(&socket_address), - sizeof(socket_address)) != 0) { - ARROW_LOG(ERROR) << "Bind failed for pathname " << pathname; - close(socket_fd); - return -1; - } - if (shall_listen && listen(socket_fd, 128) == -1) { - ARROW_LOG(ERROR) << "Could not listen to socket " << pathname; - close(socket_fd); - return -1; - } - return socket_fd; -} - -Status ConnectIpcSocketRetry(const std::string& pathname, int num_retries, - int64_t timeout, int* fd) { - // Pick the default values if the user did not specify. - if (num_retries < 0) { - num_retries = kNumConnectAttempts; - } - if (timeout < 0) { - timeout = kConnectTimeoutMs; - } - *fd = ConnectIpcSock(pathname); - while (*fd < 0 && num_retries > 0) { - ARROW_LOG(ERROR) << "Connection to IPC socket failed for pathname " << pathname - << ", retrying " << num_retries << " more times"; - // Sleep for timeout milliseconds. - usleep(static_cast(timeout * 1000)); - *fd = ConnectIpcSock(pathname); - --num_retries; - } - - // If we could not connect to the socket, exit. - if (*fd == -1) { - return Status::IOError("Could not connect to socket ", pathname); - } - - return Status::OK(); -} - -int ConnectIpcSock(const std::string& pathname) { - struct sockaddr_un socket_address; - int socket_fd; - - socket_fd = socket(AF_UNIX, SOCK_STREAM, 0); - if (socket_fd < 0) { - ARROW_LOG(ERROR) << "socket() failed for pathname " << pathname; - return -1; - } - - memset(&socket_address, 0, sizeof(socket_address)); - socket_address.sun_family = AF_UNIX; - if (pathname.size() + 1 > sizeof(socket_address.sun_path)) { - ARROW_LOG(ERROR) << "Socket pathname is too long."; - close(socket_fd); - return -1; - } - strncpy(socket_address.sun_path, pathname.c_str(), pathname.size() + 1); - - if (connect(socket_fd, reinterpret_cast(&socket_address), - sizeof(socket_address)) != 0) { - close(socket_fd); - return -1; - } - - return socket_fd; -} - -int AcceptClient(int socket_fd) { - int client_fd = accept(socket_fd, NULL, NULL); - if (client_fd < 0) { - ARROW_LOG(ERROR) << "Error reading from socket."; - return -1; - } - return client_fd; -} - -std::unique_ptr ReadMessageAsync(int sock) { - int64_t size; - Status s = ReadBytes(sock, reinterpret_cast(&size), sizeof(int64_t)); - if (!s.ok()) { - // The other side has closed the socket. - ARROW_LOG(DEBUG) << "Socket has been closed, or some other error has occurred."; - close(sock); - return NULL; - } - auto message = std::unique_ptr(new uint8_t[size]); - s = ReadBytes(sock, message.get(), size); - if (!s.ok()) { - // The other side has closed the socket. - ARROW_LOG(DEBUG) << "Socket has been closed, or some other error has occurred."; - close(sock); - return NULL; - } - return message; -} - -} // namespace plasma diff --git a/cpp/src/plasma/io.h b/cpp/src/plasma/io.h deleted file mode 100644 index 745518ab227..00000000000 --- a/cpp/src/plasma/io.h +++ /dev/null @@ -1,70 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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. - -#ifndef PLASMA_IO_H -#define PLASMA_IO_H - -#include -#include -#include -#include - -#include -#include -#include - -#include "arrow/status.h" -#include "plasma/compat.h" - -namespace plasma { - -namespace flatbuf { - -// Forward declaration outside the namespace, which is defined in plasma_generated.h. -enum class MessageType : int64_t; - -} // namespace flatbuf - -// TODO(pcm): Replace our own custom message header (message type, -// message length, plasma protocol verion) with one that is serialized -// using flatbuffers. -constexpr int64_t kPlasmaProtocolVersion = 0x0000000000000000; - -using arrow::Status; - -Status WriteBytes(int fd, uint8_t* cursor, size_t length); - -Status WriteMessage(int fd, flatbuf::MessageType type, int64_t length, uint8_t* bytes); - -Status ReadBytes(int fd, uint8_t* cursor, size_t length); - -Status ReadMessage(int fd, flatbuf::MessageType* type, std::vector* buffer); - -int BindIpcSock(const std::string& pathname, bool shall_listen); - -int ConnectIpcSock(const std::string& pathname); - -Status ConnectIpcSocketRetry(const std::string& pathname, int num_retries, - int64_t timeout, int* fd); - -int AcceptClient(int socket_fd); - -std::unique_ptr ReadMessageAsync(int sock); - -} // namespace plasma - -#endif // PLASMA_IO_H diff --git a/cpp/src/plasma/io/basic_connection.cc b/cpp/src/plasma/io/basic_connection.cc new file mode 100644 index 00000000000..7325855ac52 --- /dev/null +++ b/cpp/src/plasma/io/basic_connection.cc @@ -0,0 +1,256 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 "plasma/io/basic_connection.h" +#include "arrow/util/logging.h" + +#include +#include +#include +#include + +namespace plasma { +namespace io { + +/// Connect a Unix local domain socket. +/// +/// \param socket The socket to connect. +/// \param socket_name The name/path of the socket. +/// \return Status. +error_code UnixDomainSocketConnect(asio::local::stream_protocol::socket& socket, + const std::string& socket_name) { + asio::local::stream_protocol::endpoint endpoint(socket_name); + error_code ec; + socket.connect(endpoint, ec); + if (ec) { + // Close the socket if the connect failed. + error_code close_error; + socket.close(close_error); + } + return ec; +} + +Status CreateLocalStream(const std::string& name, PlasmaStream* result) { + // TODO(suquark): May be use "kNumConnectAttempts" and "kConnectTimeoutMs"? + constexpr int num_retries = 50; + constexpr int timeout_ms = 100; + if (name.empty()) { + return Status::Invalid("Cannot connect to empty socket name"); + } +#ifndef _WIN32 + for (int i = 0; i < num_retries; i++) { + error_code ec = UnixDomainSocketConnect(*result, name); + if (!ec) { + break; + } + std::this_thread::sleep_for(std::chrono::milliseconds(timeout_ms)); + if (i > 0) { + ARROW_LOG(ERROR) << "Retrying to connect to socket for pathname " << name + << " (num_attempts = " << i << ", num_retries = " << num_retries + << ")"; + } + } + return Status::OK(); +#else +// For windows: https://stackoverflow.com/questions/1236460/c-using-windows-named-pipes +#error "Windows has not been supported." +#endif +} + +PlasmaAcceptor CreateLocalAcceptor(asio::io_context& io_context, + const std::string& name) { +#ifndef _WIN32 + return PlasmaAcceptor(io_context, asio::local::stream_protocol::endpoint(name)); +#else +// For windows: https://stackoverflow.com/questions/1236460/c-using-windows-named-pipes +#error "Windows has not been supported." +#endif +} + +template +Connection::Connection(T&& stream) + : stream_(std::move(stream)), + async_write_in_flight_(false), + async_write_max_messages_(1), + async_write_queue_() {} + +template +Connection::~Connection() { + // If there are any pending messages, invoke their callbacks with an IOError status. + for (const auto& write_buffer : async_write_queue_) { + write_buffer->Handle( + error_code(static_cast(boost::system::errc::io_error), boost::system::system_category())); + } +} + +template +error_code Connection::ReadBuffer(const asio::mutable_buffer& buffer) { + error_code ec; + // Loop until all bytes are read while handling interrupts. + uint64_t bytes_remaining = asio::buffer_size(buffer); + uint64_t position = 0; + while (bytes_remaining != 0) { + size_t bytes_read = + stream_.read_some(asio::buffer(buffer + position, bytes_remaining), ec); + position += bytes_read; + bytes_remaining -= bytes_read; + if (ec.value() == EINTR) { + continue; + } else if (ec) { + return ec; + } + } + return error_code(); +} + +template +error_code Connection::ReadBuffer( + const std::vector& buffer) { + // Loop until all bytes are read while handling interrupts. + for (const auto& b : buffer) { + auto ec = ReadBuffer(b); + if (ec) return ec; + } + return error_code(); +} + +/// Write a buffer to this connection. +/// +/// \param buffer The buffer. +template +error_code Connection::WriteBuffer(const asio::const_buffer& buffer) { + error_code error; + // Loop until all bytes are written while handling interrupts. + // When profiling with pprof, unhandled interrupts were being sent by the profiler to + // the raylet process, which was causing synchronous reads and writes to fail. + uint64_t bytes_remaining = asio::buffer_size(buffer); + uint64_t position = 0; + while (bytes_remaining != 0) { + size_t bytes_written = + stream_.write_some(asio::buffer(buffer + position, bytes_remaining), error); + position += bytes_written; + bytes_remaining -= bytes_written; + if (error.value() == EINTR) { + continue; + } else if (error) { + return error; + } + } + return error_code(); +} + +template +error_code Connection::WriteBuffer( + const std::vector& buffer) { + error_code error; + // Loop until all bytes are written while handling interrupts. + // When profiling with pprof, unhandled interrupts were being sent by the profiler to + // the raylet process, which was causing synchronous reads and writes to fail. + for (const auto& b : buffer) { + error = WriteBuffer(b); + if (error) { + return error; + } + } + return error_code(); +} + +template +void Connection::WriteBufferAsync(std::unique_ptr write_buffer) { + async_writes_ += 1; + auto size = async_write_queue_.size(); + auto size_is_power_of_two = (size & (size - 1)) == 0; + if (size > 1000 && size_is_power_of_two) { + ARROW_LOG(WARNING) << "Connection has " << size << " buffered async writes"; + } + async_write_queue_.push_back(std::move(write_buffer)); + if (!async_write_in_flight_) { + DoAsyncWrites(); + } +} + +// Shuts down socket for this connection. +template +void Connection::Close() { + error_code ec; + stream_.close(ec); +} + +template +std::string Connection::DebugString() const { + std::stringstream result; + result << "\n- bytes read: " << bytes_read_; + result << "\n- bytes written: " << bytes_written_; + result << "\n- num async writes: " << async_writes_; + result << "\n- num sync writes: " << sync_writes_; + result << "\n- writing: " << async_write_in_flight_; + result << "\n- pending async messages: " << async_write_queue_.size(); + return result.str(); +} + +template +void Connection::DoAsyncWrites() { + // Make sure we were not writing to the socket. + ARROW_CHECK(!async_write_in_flight_); + async_write_in_flight_ = true; + + // Do an async write of everything currently in the queue to the socket. + std::vector message_buffers; + int num_messages = 0; + for (const auto& write_buffer : async_write_queue_) { + write_buffer->ToBuffers(message_buffers); + num_messages++; + if (num_messages >= async_write_max_messages_) { + break; + } + } + + // Ensure lambda holds a reference to this. + auto this_ptr = this->shared_from_this(); + asio::async_write(stream_, message_buffers, + [this, this_ptr, num_messages](const error_code& ec, + size_t bytes_transferred) { + bytes_written_ += bytes_transferred; + bool close_connection = false; + // Call the handlers for the written messages. + for (int i = 0; i < num_messages; i++) { + auto write_buffer = std::move(async_write_queue_.front()); + auto return_code = write_buffer->Handle(ec); + if (return_code != AsyncWriteCallbackCode::OK) { + close_connection = true; + } + async_write_queue_.pop_front(); // release object + } + // We finished writing, so mark that we're no longer doing an + // async write. + async_write_in_flight_ = false; + if (close_connection) { + Close(); + return; + } + // If there is more to write, try to write the rest. + if (!async_write_queue_.empty()) { + DoAsyncWrites(); + } + }); +} + +// We have to fill the template of all possible types. +template class Connection; + +} // namespace io +} // namespace plasma diff --git a/cpp/src/plasma/io/basic_connection.h b/cpp/src/plasma/io/basic_connection.h new file mode 100644 index 00000000000..0bda2eacfc2 --- /dev/null +++ b/cpp/src/plasma/io/basic_connection.h @@ -0,0 +1,152 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#ifndef PLASMA_IO_BASIC_CONNECTION_H +#define PLASMA_IO_BASIC_CONNECTION_H + +#ifndef ASIO_STANDALONE +#define ASIO_STANDALONE +#endif + +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "arrow/status.h" + +namespace asio = boost::asio; + +using error_code = boost::system::error_code; +using arrow::Status; + +namespace plasma { +namespace io { + +enum class AsyncWriteCallbackCode { + OK, + DISCONNECT, + UNKNOWN_ERROR, +}; + +using AsyncWriteCallback = std::function; +// TODO(suquark): Change it according to the platform. +using PlasmaStream = asio::basic_stream_socket; +using PlasmaAcceptor = asio::local::stream_protocol::acceptor; + +/// Create a local acceptor depends on the platform. +PlasmaAcceptor CreateLocalAcceptor(asio::io_context& io_context, const std::string& name); + +/// Create a local stream depends on the platform. +Status CreateLocalStream(const std::string& name, PlasmaStream* result); + +/// A message that is queued for writing asynchronously. +struct AsyncWriteBuffer { + virtual void ToBuffers(std::vector& message_buffers) = 0; + virtual ~AsyncWriteBuffer() {} + inline AsyncWriteCallbackCode Handle(const error_code& ec) { return handler_(ec); } + + protected: + AsyncWriteCallback handler_; +}; + +template +class Connection : public std::enable_shared_from_this> { + public: + explicit Connection(T&& stream); + + ~Connection(); + + /// Read a buffer from this connection. + /// + /// \param buffer The output buffer. + error_code ReadBuffer(const asio::mutable_buffer& buffer); + + /// Read buffers from this connection. + /// + /// \param buffer The output vector of buffers. + error_code ReadBuffer(const std::vector& buffer); + + /// Write a buffer to this connection. + /// + /// \param buffer The buffer. + error_code WriteBuffer(const asio::const_buffer& buffer); + + /// Write buffers to this connection. + /// + /// \param buffer The vector of buffers. + error_code WriteBuffer(const std::vector& buffer); + + /// Write buffers to this connection async. + /// + /// \param write_buffer The buffer to write async. + void WriteBufferAsync(std::unique_ptr write_buffer); + + /// Whether the stream is open. + inline bool IsOpen() { return stream_.is_open(); } + + /// Shuts down the stream for this connection. + void Close(); + + /// Get the native handle from the stream. + inline int GetNativeHandle() { return stream_.native_handle(); } + + /// Get the debug string. + std::string DebugString() const; + + protected: + /// The stream that supports most asio protocols (read, read_some, write, + /// write_some, async_read, async_write, async_read_some, async_write_some). + T stream_; + + /// Whether we are in the middle of an async write. + bool async_write_in_flight_; + + /// Max number of messages to write out at once. + const int async_write_max_messages_; + + /// List of pending messages to write. + std::deque> async_write_queue_; + + /// Count of sync messages sent total. + int64_t sync_writes_ = 0; + + /// Count of async messages sent total. + int64_t async_writes_ = 0; + + /// Count of bytes sent total. + int64_t bytes_written_ = 0; + + /// Count of bytes read total. + int64_t bytes_read_ = 0; + + private: + /// Asynchronously flushes the write queue. While async writes are running, the flag + /// async_write_in_flight_ will be set. This should only be called when no async writes + /// are currently in flight. + void DoAsyncWrites(); +}; + +} // namespace io +} // namespace plasma + +#endif // PLASMA_IO_BASIC_CONNECTION_H diff --git a/cpp/src/plasma/io/connection.cc b/cpp/src/plasma/io/connection.cc new file mode 100644 index 00000000000..97300a7a777 --- /dev/null +++ b/cpp/src/plasma/io/connection.cc @@ -0,0 +1,343 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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 "plasma/io/connection.h" + +#include +#include +#include + +#include "arrow/util/logging.h" +#include "plasma/fling.h" +#include "plasma/plasma_generated.h" +#include "plasma/protocol.h" + +// TODO(pcm): Replace our own custom message header (message type, +// message length, plasma protocol verion) with one that is serialized +// using flatbuffers. +constexpr int64_t kPlasmaProtocolVersion = 0x504C41534D410000; // PLASMA\0\0 + +namespace plasma { +namespace io { + +using flatbuf::MessageType; + +Status asio_to_arrow_status(const error_code& ec) { + if (!ec) { + return Status::OK(); + } + if (ec.value() == EPIPE || ec.value() == EBADF || ec.value() == ECONNRESET) { + ARROW_LOG(WARNING) << "Received SIGPIPE, BAD FILE DESCRIPTOR, or ECONNRESET when " + "processing a message. The client on the other end may " + "have hung up."; + } + return Status::IOError("Error code = ", strerror(ec.value())); +} + +struct AsyncMessageWriteBuffer : public AsyncWriteBuffer { + AsyncMessageWriteBuffer(int64_t version, int64_t type, int64_t length, + const uint8_t* message, AsyncWriteCallback callback) + : write_version(version), write_type(type), write_length(length) { + write_message.resize(length); + write_message.assign(message, message + length); + AsyncWriteBuffer::handler_ = callback; + } + + void ToBuffers(std::vector& message_buffers) override { + message_buffers.push_back(asio::buffer(&write_version, sizeof(write_version))); + message_buffers.push_back(asio::buffer(&write_type, sizeof(write_type))); + message_buffers.push_back(asio::buffer(&write_length, sizeof(write_length))); + message_buffers.push_back(asio::buffer(write_message)); + } + + int64_t write_version; + int64_t write_type; + uint64_t write_length; + std::vector write_message; +}; + +std::shared_ptr ServerConnection::shared_from_this() { + return std::static_pointer_cast(PlasmaConnection::shared_from_this()); +} + +std::shared_ptr ServerConnection::Create(PlasmaStream&& stream) { + std::shared_ptr self(new ServerConnection(std::move(stream))); + return self; +} + +Status ServerConnection::ReadMessage(int64_t type, std::vector* message) { + int64_t read_version, read_type, read_length; + // Wait for a message header from the client. The message header includes the + // protocol version, the message type, and the length of the message. + std::vector header; + header.push_back(asio::buffer(&read_version, sizeof(read_version))); + header.push_back(asio::buffer(&read_type, sizeof(read_type))); + header.push_back(asio::buffer(&read_length, sizeof(read_length))); + + auto ec = PlasmaConnection::ReadBuffer(header); + if (ec) { + return asio_to_arrow_status(ec); + } + // If there was no error, make sure the protocol version matches. + if (read_version != kPlasmaProtocolVersion) { + return Status::IOError( + "Expected Plasma message protocol version: ", kPlasmaProtocolVersion, + ", got protocol version: ", read_version); + } + if (type != read_type) { + if (read_type == static_cast(MessageType::PlasmaDisconnectClient)) { + // Disconnected by client. + return Status::IOError("The other side disconnected."); + } + return Status::IOError("Connection corrupted. Expected message type: ", type, + "; got message type: ", read_type, + ". Check logs or dmesg for previous errors."); + } + // Create read buffer. + message->resize(read_length); + auto buffer = asio::buffer(*message); + // Wait for the message to be read. + return asio_to_arrow_status(PlasmaConnection::ReadBuffer(buffer)); +} + +Status ServerConnection::WriteMessage(int64_t type, int64_t length, + const uint8_t* message) { + PlasmaConnection::sync_writes_ += 1; + PlasmaConnection::bytes_written_ += length; + + std::vector message_buffers; + auto write_version = kPlasmaProtocolVersion; + message_buffers.push_back(asio::buffer(&write_version, sizeof(write_version))); + message_buffers.push_back(asio::buffer(&type, sizeof(type))); + message_buffers.push_back(asio::buffer(&length, sizeof(length))); + message_buffers.push_back(asio::buffer(message, length)); + return asio_to_arrow_status(PlasmaConnection::WriteBuffer(message_buffers)); +} + +void ServerConnection::WriteMessageAsync(int64_t type, int64_t length, + const uint8_t* message, + const AsyncWriteCallback& handler) { + auto write_buffer = std::unique_ptr(new AsyncMessageWriteBuffer( + kPlasmaProtocolVersion, type, length, message, handler)); + PlasmaConnection::WriteBufferAsync(std::move(write_buffer)); +} + +Status ServerConnection::RecvFd(int* fd) { + *fd = recv_fd(GetNativeHandle()); + if (*fd < 0) { + return Status::Invalid("Got an invalid fd."); + } + return Status::OK(); +} + +Status ServerConnection::Disconnect() { + if (!IsOpen()) { + ARROW_LOG(WARNING) << "The client is not connected. 'Disconnect()' is ignored."; + return Status::OK(); + } + // Write the disconnection message. + auto status = + WriteMessage(static_cast(MessageType::PlasmaDisconnectClient), 0, NULLPTR); + Close(); // Close the stream anyway. + return status; +} + +Status ServerConnection::ReadNotificationMessage(std::unique_ptr& message) { + int64_t size; + auto ec = ReadBuffer(asio::mutable_buffer(&size, sizeof(size))); + if (ec) { + // The other side has closed the socket. + ARROW_LOG(DEBUG) << "Socket has been closed, or some other error has occurred."; + return asio_to_arrow_status(ec); + } + message.reset(new uint8_t[size]); + ec = ReadBuffer(asio::mutable_buffer(message.get(), size)); + if (ec) { + // The other side has closed the socket. + ARROW_LOG(DEBUG) << "Socket has been closed, or some other error has occurred."; + return asio_to_arrow_status(ec); + } + return Status::OK(); +} + +ServerConnection::ServerConnection(PlasmaStream&& stream) + : PlasmaConnection(std::move(stream)) {} + +std::shared_ptr ClientConnection::Create( + PlasmaStream&& stream, MessageHandler& message_handler) { + return std::shared_ptr( + new ClientConnection(std::move(stream), message_handler)); +} + +ClientConnection::ClientConnection(PlasmaStream&& stream, MessageHandler& message_handler) + : ServerConnection(std::move(stream)), message_handler_(message_handler) {} + +std::shared_ptr ClientConnection::shared_from_this() { + return std::static_pointer_cast(ServerConnection::shared_from_this()); +} + +void ClientConnection::ProcessMessages() { + // Wait for a message header from the client. The message header includes the + // protocol version, the message type, and the length of the message. + std::vector header{ + asio::buffer(&read_version_, sizeof(read_version_)), + asio::buffer(&read_type_, sizeof(read_type_)), + asio::buffer(&read_length_, sizeof(read_length_))}; + + asio::async_read(ServerConnection::stream_, header, + std::bind(&ClientConnection::ProcessMessageHeader, shared_from_this(), + std::placeholders::_1)); // Ignore byte_transferred +} + +void ClientConnection::ProcessMessageHeader(const error_code& ec) { + auto status = asio_to_arrow_status(ec); + if (!status.ok()) { + // If there was an error, disconnect the client. + ProcessError(status); + return; + } + + // If there was no error, make sure the protocol version matches. + if (read_version_ != kPlasmaProtocolVersion) { + status = Status::IOError( + "Expected Plasma message protocol version: ", kPlasmaProtocolVersion, + ", got protocol version: ", read_version_); + ProcessError(status); + return; + } + // Resize the message buffer to match the received length. + read_message_.resize(read_length_); + ServerConnection::bytes_read_ += read_length_; + // Wait for the message to be read. + asio::async_read(ServerConnection::stream_, asio::buffer(read_message_), + std::bind(&ClientConnection::ProcessMessageBody, shared_from_this(), + std::placeholders::_1)); +} + +void ClientConnection::ProcessMessageBody(const error_code& ec) { + auto status = asio_to_arrow_status(ec); + if (!status.ok()) { + // If there was an error, disconnect the client. + ProcessError(status); + return; + } + + ProcessMessage(read_type_, read_length_, read_message_.data()); +} + +void ClientConnection::ProcessError(const Status& status) { + ARROW_LOG(ERROR) << "Failed when processing message. Disconnecting the client. (" + << status << ")"; + // If there was an error, disconnect the client. + PlasmaConnection::Close(); +} + +void ClientConnection::ProcessMessage(int64_t type, int64_t length, const uint8_t* data) { + message_handler_(shared_from_this(), type, length, data); +} + +struct AsyncObjectNotificationWriteBuffer : public AsyncWriteBuffer { + ~AsyncObjectNotificationWriteBuffer() override {} + + static std::unique_ptr MakeDeletion( + const ObjectID& object_id) { + auto message = new std::vector(); + SerializeObjectDeletionNotification(object_id, message); + return std::unique_ptr( + new AsyncObjectNotificationWriteBuffer(message)); + } + + static std::unique_ptr MakeReady( + const ObjectID& object_id, const ObjectTableEntry& entry) { + auto message = new std::vector(); + SerializeObjectSealedNotification(object_id, entry, message); + return std::unique_ptr( + new AsyncObjectNotificationWriteBuffer(message)); + } + + void ToBuffers(std::vector& message_buffers) override { + message_buffers.push_back(asio::buffer(&size, sizeof(size))); + message_buffers.push_back(asio::buffer(*notification_msg)); + } + + std::unique_ptr> notification_msg; + int64_t size; + + protected: + explicit AsyncObjectNotificationWriteBuffer(std::vector* message) { + // Serialize the object. + notification_msg.reset(message); + size = message->size(); + AsyncWriteBuffer::handler_ = + [](const error_code& status) -> AsyncWriteCallbackCode { + auto errno_ = status.value(); + if (!errno_) { + return AsyncWriteCallbackCode::OK; + } + if (errno_ == EAGAIN || errno_ == EWOULDBLOCK || errno_ == EINTR) { + ARROW_LOG(DEBUG) << "The socket's send buffer is full, so we are caching this " + "notification and will send it later."; + ARROW_LOG(WARNING) << "Blocked unexpectly when sending message async."; + return AsyncWriteCallbackCode::OK; + } else { + ARROW_LOG(WARNING) << "Failed to send notification to client."; + if (errno_ == EPIPE) { + return AsyncWriteCallbackCode::DISCONNECT; + } + return AsyncWriteCallbackCode::UNKNOWN_ERROR; + } + }; + } +}; + +Status ClientConnection::SendFd(int fd) { + // Only send the file descriptor if it hasn't been sent (see analogous + // logic in GetStoreFd in client.cc). + if (used_fds.find(fd) == used_fds.end()) { + auto ec = send_fd(GetNativeHandle(), fd); + if (ec <= 0) { + if (ec == 0) { + return Status::IOError("Encountered unexpected EOF"); + } else { + return Status::IOError("Unknown I/O Error"); + } + } + used_fds.insert(fd); // Succeed, record the fd. + } + return Status::OK(); +} + +void ClientConnection::SendObjectDeletionAsync(const ObjectID& object_id) { + auto raw_ptr = AsyncObjectNotificationWriteBuffer::MakeDeletion(object_id).release(); + auto write_buffer = + std::unique_ptr(static_cast(raw_ptr)); + // Attempt to send a notification about this object ID. + WriteBufferAsync(std::move(write_buffer)); +} + +void ClientConnection::SendObjectReadyAsync(const ObjectID& object_id, + const ObjectTableEntry& entry) { + auto raw_ptr = + AsyncObjectNotificationWriteBuffer::MakeReady(object_id, entry).release(); + auto write_buffer = + std::unique_ptr(static_cast(raw_ptr)); + // Attempt to send a notification about this object ID. + WriteBufferAsync(std::move(write_buffer)); +} + +} // namespace io +} // namespace plasma diff --git a/cpp/src/plasma/io/connection.h b/cpp/src/plasma/io/connection.h new file mode 100644 index 00000000000..1bf232558d0 --- /dev/null +++ b/cpp/src/plasma/io/connection.h @@ -0,0 +1,181 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +#ifndef PLASMA_IO_CONNECTION_H +#define PLASMA_IO_CONNECTION_H + +#include +#include +#include +#include +#include +#include +#include + +#include "arrow/status.h" +#include "plasma/common.h" +#include "plasma/io/basic_connection.h" + +namespace plasma { +namespace io { + +using arrow::Status; + +using PlasmaConnection = Connection; + +Status asio_to_arrow_status(const error_code& ec); + +/// A generic type representing a client connection to a server. This typename +/// can be used to write messages synchronously to the server. +class ServerConnection : public PlasmaConnection { + public: + std::shared_ptr shared_from_this(); + + /// Allocate a new server connection. + /// + /// \param stream A reference to the server stream. + /// \return std::shared_ptr. + static std::shared_ptr Create(PlasmaStream&& stream); + + /// Write a message to the client. + /// + /// \param type The message type (e.g., a flatbuffer enum). + /// \param length The size in bytes of the message. + /// \param message A pointer to the message buffer. + /// \return Status. + Status WriteMessage(int64_t type, int64_t length, const uint8_t* message); + + /// Read a message from the client. + /// + /// \param type The message type (e.g., a flatbuffer enum). + /// \param message A pointer to the message vector. + /// \return Status. + Status ReadMessage(int64_t type, std::vector* message); + + /// Write a message to the client asynchronously. + /// + /// \param type The message type (e.g., a flatbuffer enum). + /// \param length The size in bytes of the message. + /// \param message A pointer to the message buffer. + /// \param handler A callback to run on write completion. + void WriteMessageAsync(int64_t type, int64_t length, const uint8_t* message, + const AsyncWriteCallback& handler); + + Status RecvFd(int* fd); + + Status Disconnect(); + + Status ReadNotificationMessage(std::unique_ptr& message); + + protected: + /// A private constructor for a server connection. + explicit ServerConnection(PlasmaStream&& stream); +}; + +class ClientConnection; +using MessageHandler = std::function, int64_t type, + int64_t length, const uint8_t*)>; + +/// A generic type representing a client connection on a server. In addition to +/// writing messages to the client, like in ServerConnection, this typename can +/// also be used to process messages asynchronously from client. +class ClientConnection : public ServerConnection { + public: + /// Allocate a new node client connection. + /// + /// \param stream The client stream. + /// \param message_handler A reference to the message handler. + /// \return std::shared_ptr. + static std::shared_ptr Create(PlasmaStream&& stream, + MessageHandler& message_handler); + + std::shared_ptr shared_from_this(); + + /// Listen for and process messages from the client connection. Once a + /// message has been fully received, the client manager's + /// ProcessClientMessage handler will be called. + void ProcessMessages(); + + Status SendFd(int fd); + + inline bool ObjectIDExists(const ObjectID& object_id) { + return object_ids.find(object_id) != object_ids.end(); + } + + inline void RemoveObjectID(const ObjectID& object_id) { object_ids.erase(object_id); } + + inline int RemoveObjectIDIfExists(const ObjectID& object_id) { + auto it = object_ids.find(object_id); + if (it != object_ids.end()) { + object_ids.erase(it); + // Return 1 to indicate that the client was removed. + return 1; + } else { + // Return 0 to indicate that the client was not removed. + return 0; + } + } + + /// Send notifications about sealed objects to the subscribers. This is called + /// in SealObject. + void SendObjectReadyAsync(const ObjectID& object_id, const ObjectTableEntry& entry); + + /// Send notifications about evicted objects to the subscribers. + void SendObjectDeletionAsync(const ObjectID& object_id); + + /// Object ids that are used by this client. + std::unordered_set object_ids; + /// File descriptors that are used by this client. + std::unordered_set used_fds; + + private: + /// Process the message header from the client. + /// \param ec The returned error code. + void ProcessMessageHeader(const error_code& ec); + + /// Process the message body from the client. + /// \param ec The returned error code. + void ProcessMessageBody(const error_code& ec); + + /// Process the message from the client. + /// \param type The type of the message. + /// \param length The length of the message. + /// \param data The data buffer of the message. + void ProcessMessage(int64_t type, int64_t length, const uint8_t* data); + + /// Process an error from reading the message from the client. + /// \param status The status code. + void ProcessError(const Status& status); + + /// A private constructor for a node client connection. + ClientConnection(PlasmaStream&& stream, MessageHandler& message_handler); + + /// The handler for a message from the client. + MessageHandler message_handler_; + + int64_t read_version_; + int64_t read_type_; + uint64_t read_length_; + + /// Buffers for the current message being read from the client. + std::vector read_message_; +}; + +} // namespace io +} // namespace plasma + +#endif // PLASMA_IO_CONNECTION_H diff --git a/cpp/src/plasma/malloc.cc b/cpp/src/plasma/malloc.cc index bb027a6cb90..14217894f38 100644 --- a/cpp/src/plasma/malloc.cc +++ b/cpp/src/plasma/malloc.cc @@ -29,6 +29,7 @@ #include #include +#include "arrow/util/logging.h" #include "plasma/common.h" #include "plasma/plasma.h" diff --git a/cpp/src/plasma/malloc.h b/cpp/src/plasma/malloc.h index a081190b3ef..92d6537e948 100644 --- a/cpp/src/plasma/malloc.h +++ b/cpp/src/plasma/malloc.h @@ -35,8 +35,8 @@ void GetMallocMapinfo(void* addr, int* fd, int64_t* map_length, ptrdiff_t* offse /// Get the mmap size corresponding to a specific file descriptor. /// -/// @param fd The file descriptor to look up. -/// @return The size of the corresponding memory-mapped file. +/// \param fd The file descriptor to look up. +/// \return The size of the corresponding memory-mapped file. int64_t GetMmapSize(int fd); struct MmapRecord { diff --git a/cpp/src/plasma/plasma.cc b/cpp/src/plasma/plasma.cc index 0710e34c275..538422d3b49 100644 --- a/cpp/src/plasma/plasma.cc +++ b/cpp/src/plasma/plasma.cc @@ -16,16 +16,7 @@ // under the License. #include "plasma/plasma.h" - -#include -#include -#include - #include "plasma/common.h" -#include "plasma/common_generated.h" -#include "plasma/protocol.h" - -namespace fb = plasma::flatbuf; namespace plasma { @@ -33,60 +24,6 @@ ObjectTableEntry::ObjectTableEntry() : pointer(nullptr), ref_count(0) {} ObjectTableEntry::~ObjectTableEntry() { pointer = nullptr; } -int WarnIfSigpipe(int status, int client_sock) { - if (status >= 0) { - return 0; - } - if (errno == EPIPE || errno == EBADF || errno == ECONNRESET) { - ARROW_LOG(WARNING) << "Received SIGPIPE, BAD FILE DESCRIPTOR, or ECONNRESET when " - "sending a message to client on fd " - << client_sock - << ". The client on the other end may " - "have hung up."; - return errno; - } - ARROW_LOG(FATAL) << "Failed to write message to client on fd " << client_sock << "."; - return -1; // This is never reached. -} - -/** - * This will create a new ObjectInfo buffer. The first sizeof(int64_t) bytes - * of this buffer are the length of the remaining message and the - * remaining message is a serialized version of the object info. - * - * @param object_info The object info to be serialized - * @return The object info buffer. It is the caller's responsibility to free - * this buffer with "delete" after it has been used. - */ -std::unique_ptr CreateObjectInfoBuffer(fb::ObjectInfoT* object_info) { - flatbuffers::FlatBufferBuilder fbb; - auto message = fb::CreateObjectInfo(fbb, object_info); - fbb.Finish(message); - auto notification = - std::unique_ptr(new uint8_t[sizeof(int64_t) + fbb.GetSize()]); - *(reinterpret_cast(notification.get())) = fbb.GetSize(); - memcpy(notification.get() + sizeof(int64_t), fbb.GetBufferPointer(), fbb.GetSize()); - return notification; -} - -std::unique_ptr CreatePlasmaNotificationBuffer( - std::vector& object_info) { - flatbuffers::FlatBufferBuilder fbb; - std::vector> info; - for (size_t i = 0; i < object_info.size(); ++i) { - info.push_back(fb::CreateObjectInfo(fbb, &object_info[i])); - } - - auto info_array = fbb.CreateVector(info); - auto message = fb::CreatePlasmaNotification(fbb, info_array); - fbb.Finish(message); - auto notification = - std::unique_ptr(new uint8_t[sizeof(int64_t) + fbb.GetSize()]); - *(reinterpret_cast(notification.get())) = fbb.GetSize(); - memcpy(notification.get() + sizeof(int64_t), fbb.GetBufferPointer(), fbb.GetSize()); - return notification; -} - ObjectTableEntry* GetObjectTableEntry(PlasmaStoreInfo* store_info, const ObjectID& object_id) { auto it = store_info->objects.find(object_id); diff --git a/cpp/src/plasma/plasma.h b/cpp/src/plasma/plasma.h index 79e33c2f0c3..92386cc83bb 100644 --- a/cpp/src/plasma/plasma.h +++ b/cpp/src/plasma/plasma.h @@ -18,26 +18,14 @@ #ifndef PLASMA_PLASMA_H #define PLASMA_PLASMA_H -#include -#include -#include -#include -#include -#include -#include -#include // pid_t - #include #include #include #include -#include #include "plasma/compat.h" #include "arrow/status.h" -#include "arrow/util/logging.h" -#include "arrow/util/macros.h" #include "plasma/common.h" #ifdef PLASMA_CUDA @@ -46,50 +34,9 @@ using arrow::cuda::CudaIpcMemHandle; namespace plasma { -namespace flatbuf { -struct ObjectInfoT; -} // namespace flatbuf - -#define HANDLE_SIGPIPE(s, fd_) \ - do { \ - Status _s = (s); \ - if (!_s.ok()) { \ - if (errno == EPIPE || errno == EBADF || errno == ECONNRESET) { \ - ARROW_LOG(WARNING) \ - << "Received SIGPIPE, BAD FILE DESCRIPTOR, or ECONNRESET when " \ - "sending a message to client on fd " \ - << fd_ \ - << ". " \ - "The client on the other end may have hung up."; \ - } else { \ - return _s; \ - } \ - } \ - } while (0); - /// Allocation granularity used in plasma for object allocation. constexpr int64_t kBlockSize = 64; -/// Contains all information that is associated with a Plasma store client. -struct Client { - explicit Client(int fd); - - /// The file descriptor used to communicate with the client. - int fd; - - /// Object ids that are used by this client. - std::unordered_set object_ids; - - /// File descriptors that are used by this client. - std::unordered_set used_fds; - - /// The file descriptor used to push notifications to client. This is only valid - /// if client subscribes to plasma store. -1 indicates invalid. - int notification_fd; - - std::string name = "anonymous_client"; -}; - // TODO(pcm): Replace this by the flatbuffers message PlasmaObjectSpec. struct PlasmaObject { #ifdef PLASMA_CUDA @@ -145,34 +92,13 @@ struct PlasmaStoreInfo { /// Get an entry from the object table and return NULL if the object_id /// is not present. /// -/// @param store_info The PlasmaStoreInfo that contains the object table. -/// @param object_id The object_id of the entry we are looking for. -/// @return The entry associated with the object_id or NULL if the object_id +/// \param store_info The PlasmaStoreInfo that contains the object table. +/// \param object_id The object_id of the entry we are looking for. +/// \return The entry associated with the object_id or NULL if the object_id /// is not present. ObjectTableEntry* GetObjectTableEntry(PlasmaStoreInfo* store_info, const ObjectID& object_id); -/// Print a warning if the status is less than zero. This should be used to check -/// the success of messages sent to plasma clients. We print a warning instead of -/// failing because the plasma clients are allowed to die. This is used to handle -/// situations where the store writes to a client file descriptor, and the client -/// may already have disconnected. If we have processed the disconnection and -/// closed the file descriptor, we should get a BAD FILE DESCRIPTOR error. If we -/// have not, then we should get a SIGPIPE. If we write to a TCP socket that -/// isn't connected yet, then we should get an ECONNRESET. -/// -/// @param status The status to check. If it is less less than zero, we will -/// print a warning. -/// @param client_sock The client socket. This is just used to print some extra -/// information. -/// @return The errno set. -int WarnIfSigpipe(int status, int client_sock); - -std::unique_ptr CreateObjectInfoBuffer(flatbuf::ObjectInfoT* object_info); - -std::unique_ptr CreatePlasmaNotificationBuffer( - std::vector& object_info); - } // namespace plasma #endif // PLASMA_PLASMA_H diff --git a/cpp/src/plasma/protocol.cc b/cpp/src/plasma/protocol.cc index b962c35b4c8..9fe340b9e41 100644 --- a/cpp/src/plasma/protocol.cc +++ b/cpp/src/plasma/protocol.cc @@ -19,11 +19,12 @@ #include +#include "arrow/util/logging.h" #include "flatbuffers/flatbuffers.h" +#include "plasma/plasma_generated.h" #include "plasma/common.h" -#include "plasma/io.h" -#include "plasma/plasma_generated.h" +#include "plasma/io/connection.h" #ifdef PLASMA_CUDA #include "arrow/gpu/cuda_api.h" @@ -53,31 +54,6 @@ ToFlatbuffer(flatbuffers::FlatBufferBuilder* fbb, const ObjectID* object_ids, return fbb->CreateVector(arrow::util::MakeNonNull(results.data()), results.size()); } -flatbuffers::Offset>> -ToFlatbuffer(flatbuffers::FlatBufferBuilder* fbb, - const std::vector& strings) { - std::vector> results; - for (size_t i = 0; i < strings.size(); i++) { - results.push_back(fbb->CreateString(strings[i])); - } - - return fbb->CreateVector(arrow::util::MakeNonNull(results.data()), results.size()); -} - -flatbuffers::Offset> ToFlatbuffer( - flatbuffers::FlatBufferBuilder* fbb, const std::vector& data) { - return fbb->CreateVector(arrow::util::MakeNonNull(data.data()), data.size()); -} - -Status PlasmaReceive(int sock, MessageType message_type, std::vector* buffer) { - MessageType type; - RETURN_NOT_OK(ReadMessage(sock, &type, buffer)); - ARROW_CHECK(type == message_type) - << "type = " << static_cast(type) - << ", message_type = " << static_cast(message_type); - return Status::OK(); -} - // Helper function to create a vector of elements from Data (Request/Reply struct). // The Getter function is used to extract one element from Data. template @@ -90,23 +66,6 @@ void ToVector(const Data& request, std::vector* out, const Getter& getter) { } } -template -void ConvertToVector(const FlatbufferVectorPointer fbvector, std::vector* out, - const Converter& converter) { - out->clear(); - out->reserve(fbvector->size()); - for (size_t i = 0; i < fbvector->size(); ++i) { - out->push_back(converter(*fbvector->Get(i))); - } -} - -template -Status PlasmaSend(int sock, MessageType message_type, flatbuffers::FlatBufferBuilder* fbb, - const Message& message) { - fbb->Finish(message); - return WriteMessage(sock, message_type, fbb->GetSize(), fbb->GetBufferPointer()); -} - Status PlasmaErrorStatus(fb::PlasmaError plasma_error) { switch (plasma_error) { case fb::PlasmaError::OK: @@ -126,72 +85,39 @@ Status PlasmaErrorStatus(fb::PlasmaError plasma_error) { return Status::OK(); } -// Set options messages. - -Status SendSetOptionsRequest(int sock, const std::string& client_name, - int64_t output_memory_limit) { - flatbuffers::FlatBufferBuilder fbb; - auto message = fb::CreatePlasmaSetOptionsRequest(fbb, fbb.CreateString(client_name), - output_memory_limit); - return PlasmaSend(sock, MessageType::PlasmaSetOptionsRequest, &fbb, message); -} - -Status ReadSetOptionsRequest(uint8_t* data, size_t size, std::string* client_name, - int64_t* output_memory_quota) { - DCHECK(data); - auto message = flatbuffers::GetRoot(data); - DCHECK(VerifyFlatbuffer(message, data, size)); - *client_name = std::string(message->client_name()->str()); - *output_memory_quota = message->output_memory_quota(); - return Status::OK(); -} - -Status SendSetOptionsReply(int sock, PlasmaError error) { - flatbuffers::FlatBufferBuilder fbb; - auto message = fb::CreatePlasmaSetOptionsReply(fbb, error); - return PlasmaSend(sock, MessageType::PlasmaSetOptionsReply, &fbb, message); -} - -Status ReadSetOptionsReply(uint8_t* data, size_t size) { - DCHECK(data); - auto message = flatbuffers::GetRoot(data); - DCHECK(VerifyFlatbuffer(message, data, size)); - return PlasmaErrorStatus(message->error()); -} - -// Get debug string messages. - -Status SendGetDebugStringRequest(int sock) { - flatbuffers::FlatBufferBuilder fbb; - auto message = fb::CreatePlasmaGetDebugStringRequest(fbb); - return PlasmaSend(sock, MessageType::PlasmaGetDebugStringRequest, &fbb, message); -} - -Status SendGetDebugStringReply(int sock, const std::string& debug_string) { - flatbuffers::FlatBufferBuilder fbb; - auto message = fb::CreatePlasmaGetDebugStringReply(fbb, fbb.CreateString(debug_string)); - return PlasmaSend(sock, MessageType::PlasmaGetDebugStringReply, &fbb, message); +Status PlasmaSend(const std::shared_ptr& client, + MessageType message_type, flatbuffers::FlatBufferBuilder* fbb) { + if (fbb) { + return client->WriteMessage(static_cast(message_type), fbb->GetSize(), + fbb->GetBufferPointer()); + } else { + return client->WriteMessage(static_cast(message_type), 0, NULLPTR); + } } -Status ReadGetDebugStringReply(uint8_t* data, size_t size, std::string* debug_string) { - DCHECK(data); - auto message = flatbuffers::GetRoot(data); - DCHECK(VerifyFlatbuffer(message, data, size)); - *debug_string = message->debug_string()->str(); - return Status::OK(); +Status PlasmaSend(const std::shared_ptr& client, + MessageType message_type, flatbuffers::FlatBufferBuilder* fbb) { + if (fbb) { + return client->WriteMessage(static_cast(message_type), fbb->GetSize(), + fbb->GetBufferPointer()); + } else { + return client->WriteMessage(static_cast(message_type), 0, NULLPTR); + } } // Create messages. -Status SendCreateRequest(int sock, ObjectID object_id, int64_t data_size, - int64_t metadata_size, int device_num) { +Status SendCreateRequest(const std::shared_ptr& client, + ObjectID object_id, int64_t data_size, int64_t metadata_size, + int device_num) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaCreateRequest(fbb, fbb.CreateString(object_id.binary()), data_size, metadata_size, device_num); - return PlasmaSend(sock, MessageType::PlasmaCreateRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaCreateRequest, &fbb); } -Status ReadCreateRequest(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadCreateRequest(const uint8_t* data, size_t size, ObjectID* object_id, int64_t* data_size, int64_t* metadata_size, int* device_num) { DCHECK(data); auto message = flatbuffers::GetRoot(data); @@ -203,8 +129,9 @@ Status ReadCreateRequest(uint8_t* data, size_t size, ObjectID* object_id, return Status::OK(); } -Status SendCreateReply(int sock, ObjectID object_id, PlasmaObject* object, - PlasmaError error_code, int64_t mmap_size) { +Status SendCreateReply(const std::shared_ptr& client, + ObjectID object_id, PlasmaObject* object, PlasmaError error_code, + int64_t mmap_size) { flatbuffers::FlatBufferBuilder fbb; PlasmaObjectSpec plasma_object(object->store_fd, object->data_offset, object->data_size, object->metadata_offset, object->metadata_size, @@ -233,10 +160,11 @@ Status SendCreateReply(int sock, ObjectID object_id, PlasmaObject* object, #endif } auto message = crb.Finish(); - return PlasmaSend(sock, MessageType::PlasmaCreateReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaCreateReply, &fbb); } -Status ReadCreateReply(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadCreateReply(const uint8_t* data, size_t size, ObjectID* object_id, PlasmaObject* object, int* store_fd, int64_t* mmap_size) { DCHECK(data); auto message = flatbuffers::GetRoot(data); @@ -261,20 +189,21 @@ Status ReadCreateReply(uint8_t* data, size_t size, ObjectID* object_id, return PlasmaErrorStatus(message->error()); } -Status SendCreateAndSealRequest(int sock, const ObjectID& object_id, - const std::string& data, const std::string& metadata, - unsigned char* digest) { +Status SendCreateAndSealRequest(const std::shared_ptr& client, + const ObjectID& object_id, const std::string& data, + const std::string& metadata, unsigned char* digest) { flatbuffers::FlatBufferBuilder fbb; auto digest_string = fbb.CreateString(reinterpret_cast(digest), kDigestSize); auto message = fb::CreatePlasmaCreateAndSealRequest( fbb, fbb.CreateString(object_id.binary()), fbb.CreateString(data), fbb.CreateString(metadata), digest_string); - return PlasmaSend(sock, MessageType::PlasmaCreateAndSealRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaCreateAndSealRequest, &fbb); } -Status ReadCreateAndSealRequest(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadCreateAndSealRequest(const uint8_t* data, size_t size, ObjectID* object_id, std::string* object_data, std::string* metadata, - std::string* digest) { + unsigned char* digest) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -283,84 +212,34 @@ Status ReadCreateAndSealRequest(uint8_t* data, size_t size, ObjectID* object_id, *object_data = message->data()->str(); *metadata = message->metadata()->str(); ARROW_CHECK(message->digest()->size() == kDigestSize); - digest->assign(message->digest()->data(), kDigestSize); - return Status::OK(); -} - -Status SendCreateAndSealBatchRequest(int sock, const std::vector& object_ids, - const std::vector& data, - const std::vector& metadata, - const std::vector& digests) { - flatbuffers::FlatBufferBuilder fbb; - - auto message = fb::CreatePlasmaCreateAndSealBatchRequest( - fbb, ToFlatbuffer(&fbb, object_ids.data(), object_ids.size()), - ToFlatbuffer(&fbb, data), ToFlatbuffer(&fbb, metadata), - ToFlatbuffer(&fbb, digests)); - - return PlasmaSend(sock, MessageType::PlasmaCreateAndSealBatchRequest, &fbb, message); -} - -Status ReadCreateAndSealBatchRequest(uint8_t* data, size_t size, - std::vector* object_ids, - std::vector* object_data, - std::vector* metadata, - std::vector* digests) { - DCHECK(data); - auto message = flatbuffers::GetRoot(data); - DCHECK(VerifyFlatbuffer(message, data, size)); - - ConvertToVector(message->object_ids(), object_ids, - [](const flatbuffers::String& element) { - return ObjectID::from_binary(element.str()); - }); - - ConvertToVector(message->data(), object_data, - [](const flatbuffers::String& element) { return element.str(); }); - - ConvertToVector(message->metadata(), metadata, - [](const flatbuffers::String& element) { return element.str(); }); - - ConvertToVector(message->digest(), digests, - [](const flatbuffers::String& element) { return element.str(); }); - + memcpy(digest, message->digest()->data(), kDigestSize); return Status::OK(); } -Status SendCreateAndSealReply(int sock, PlasmaError error) { +Status SendCreateAndSealReply(const std::shared_ptr& client, + PlasmaError error) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaCreateAndSealReply(fbb, static_cast(error)); - return PlasmaSend(sock, MessageType::PlasmaCreateAndSealReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaCreateAndSealReply, &fbb); } -Status ReadCreateAndSealReply(uint8_t* data, size_t size) { +Status ReadCreateAndSealReply(const uint8_t* data, size_t size) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); return PlasmaErrorStatus(message->error()); } -Status SendCreateAndSealBatchReply(int sock, PlasmaError error) { - flatbuffers::FlatBufferBuilder fbb; - auto message = - fb::CreatePlasmaCreateAndSealBatchReply(fbb, static_cast(error)); - return PlasmaSend(sock, MessageType::PlasmaCreateAndSealBatchReply, &fbb, message); -} - -Status ReadCreateAndSealBatchReply(uint8_t* data, size_t size) { - DCHECK(data); - auto message = flatbuffers::GetRoot(data); - DCHECK(VerifyFlatbuffer(message, data, size)); - return PlasmaErrorStatus(message->error()); -} - -Status SendAbortRequest(int sock, ObjectID object_id) { +Status SendAbortRequest(const std::shared_ptr& client, + ObjectID object_id) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaAbortRequest(fbb, fbb.CreateString(object_id.binary())); - return PlasmaSend(sock, MessageType::PlasmaAbortRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaAbortRequest, &fbb); } -Status ReadAbortRequest(uint8_t* data, size_t size, ObjectID* object_id) { +Status ReadAbortRequest(const uint8_t* data, size_t size, ObjectID* object_id) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -368,13 +247,15 @@ Status ReadAbortRequest(uint8_t* data, size_t size, ObjectID* object_id) { return Status::OK(); } -Status SendAbortReply(int sock, ObjectID object_id) { +Status SendAbortReply(const std::shared_ptr& client, + ObjectID object_id) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaAbortReply(fbb, fbb.CreateString(object_id.binary())); - return PlasmaSend(sock, MessageType::PlasmaAbortReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaAbortReply, &fbb); } -Status ReadAbortReply(uint8_t* data, size_t size, ObjectID* object_id) { +Status ReadAbortReply(const uint8_t* data, size_t size, ObjectID* object_id) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -384,32 +265,37 @@ Status ReadAbortReply(uint8_t* data, size_t size, ObjectID* object_id) { // Seal messages. -Status SendSealRequest(int sock, ObjectID object_id, const std::string& digest) { +Status SendSealRequest(const std::shared_ptr& client, + ObjectID object_id, unsigned char* digest) { flatbuffers::FlatBufferBuilder fbb; + auto digest_string = fbb.CreateString(reinterpret_cast(digest), kDigestSize); auto message = fb::CreatePlasmaSealRequest(fbb, fbb.CreateString(object_id.binary()), - fbb.CreateString(digest)); - return PlasmaSend(sock, MessageType::PlasmaSealRequest, &fbb, message); + digest_string); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaSealRequest, &fbb); } -Status ReadSealRequest(uint8_t* data, size_t size, ObjectID* object_id, - std::string* digest) { +Status ReadSealRequest(const uint8_t* data, size_t size, ObjectID* object_id, + unsigned char* digest) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); *object_id = ObjectID::from_binary(message->object_id()->str()); - ARROW_CHECK_EQ(message->digest()->size(), kDigestSize); - digest->assign(message->digest()->data(), kDigestSize); + ARROW_CHECK(message->digest()->size() == kDigestSize); + memcpy(digest, message->digest()->data(), kDigestSize); return Status::OK(); } -Status SendSealReply(int sock, ObjectID object_id, PlasmaError error) { +Status SendSealReply(const std::shared_ptr& client, ObjectID object_id, + PlasmaError error) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaSealReply(fbb, fbb.CreateString(object_id.binary()), error); - return PlasmaSend(sock, MessageType::PlasmaSealReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaSealReply, &fbb); } -Status ReadSealReply(uint8_t* data, size_t size, ObjectID* object_id) { +Status ReadSealReply(const uint8_t* data, size_t size, ObjectID* object_id) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -419,14 +305,16 @@ Status ReadSealReply(uint8_t* data, size_t size, ObjectID* object_id) { // Release messages. -Status SendReleaseRequest(int sock, ObjectID object_id) { +Status SendReleaseRequest(const std::shared_ptr& client, + ObjectID object_id) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaReleaseRequest(fbb, fbb.CreateString(object_id.binary())); - return PlasmaSend(sock, MessageType::PlasmaReleaseRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaReleaseRequest, &fbb); } -Status ReadReleaseRequest(uint8_t* data, size_t size, ObjectID* object_id) { +Status ReadReleaseRequest(const uint8_t* data, size_t size, ObjectID* object_id) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -434,14 +322,16 @@ Status ReadReleaseRequest(uint8_t* data, size_t size, ObjectID* object_id) { return Status::OK(); } -Status SendReleaseReply(int sock, ObjectID object_id, PlasmaError error) { +Status SendReleaseReply(const std::shared_ptr& client, + ObjectID object_id, PlasmaError error) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaReleaseReply(fbb, fbb.CreateString(object_id.binary()), error); - return PlasmaSend(sock, MessageType::PlasmaReleaseReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaReleaseReply, &fbb); } -Status ReadReleaseReply(uint8_t* data, size_t size, ObjectID* object_id) { +Status ReadReleaseReply(const uint8_t* data, size_t size, ObjectID* object_id) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -451,15 +341,18 @@ Status ReadReleaseReply(uint8_t* data, size_t size, ObjectID* object_id) { // Delete objects messages. -Status SendDeleteRequest(int sock, const std::vector& object_ids) { +Status SendDeleteRequest(const std::shared_ptr& client, + const std::vector& object_ids) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaDeleteRequest( fbb, static_cast(object_ids.size()), ToFlatbuffer(&fbb, &object_ids[0], object_ids.size())); - return PlasmaSend(sock, MessageType::PlasmaDeleteRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaDeleteRequest, &fbb); } -Status ReadDeleteRequest(uint8_t* data, size_t size, std::vector* object_ids) { +Status ReadDeleteRequest(const uint8_t* data, size_t size, + std::vector* object_ids) { using fb::PlasmaDeleteRequest; DCHECK(data); @@ -472,7 +365,8 @@ Status ReadDeleteRequest(uint8_t* data, size_t size, std::vector* obje return Status::OK(); } -Status SendDeleteReply(int sock, const std::vector& object_ids, +Status SendDeleteReply(const std::shared_ptr& client, + const std::vector& object_ids, const std::vector& errors) { DCHECK(object_ids.size() == errors.size()); flatbuffers::FlatBufferBuilder fbb; @@ -482,10 +376,12 @@ Status SendDeleteReply(int sock, const std::vector& object_ids, fbb.CreateVector( arrow::util::MakeNonNull(reinterpret_cast(errors.data())), object_ids.size())); - return PlasmaSend(sock, MessageType::PlasmaDeleteReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaDeleteReply, &fbb); } -Status ReadDeleteReply(uint8_t* data, size_t size, std::vector* object_ids, +Status ReadDeleteReply(const uint8_t* data, size_t size, + std::vector* object_ids, std::vector* errors) { using fb::PlasmaDeleteReply; @@ -505,14 +401,16 @@ Status ReadDeleteReply(uint8_t* data, size_t size, std::vector* object // Contains messages. -Status SendContainsRequest(int sock, ObjectID object_id) { +Status SendContainsRequest(const std::shared_ptr& client, + ObjectID object_id) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaContainsRequest(fbb, fbb.CreateString(object_id.binary())); - return PlasmaSend(sock, MessageType::PlasmaContainsRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaContainsRequest, &fbb); } -Status ReadContainsRequest(uint8_t* data, size_t size, ObjectID* object_id) { +Status ReadContainsRequest(const uint8_t* data, size_t size, ObjectID* object_id) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -520,14 +418,16 @@ Status ReadContainsRequest(uint8_t* data, size_t size, ObjectID* object_id) { return Status::OK(); } -Status SendContainsReply(int sock, ObjectID object_id, bool has_object) { +Status SendContainsReply(const std::shared_ptr& client, + ObjectID object_id, bool has_object) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaContainsReply(fbb, fbb.CreateString(object_id.binary()), has_object); - return PlasmaSend(sock, MessageType::PlasmaContainsReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaContainsReply, &fbb); } -Status ReadContainsReply(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadContainsReply(const uint8_t* data, size_t size, ObjectID* object_id, bool* has_object) { DCHECK(data); auto message = flatbuffers::GetRoot(data); @@ -539,15 +439,17 @@ Status ReadContainsReply(uint8_t* data, size_t size, ObjectID* object_id, // List messages. -Status SendListRequest(int sock) { +Status SendListRequest(const std::shared_ptr& client) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaListRequest(fbb); - return PlasmaSend(sock, MessageType::PlasmaListRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaListRequest, &fbb); } -Status ReadListRequest(uint8_t* data, size_t size) { return Status::OK(); } +Status ReadListRequest(const uint8_t* data, size_t size) { return Status::OK(); } -Status SendListReply(int sock, const ObjectTable& objects) { +Status SendListReply(const std::shared_ptr& client, + const ObjectTable& objects) { flatbuffers::FlatBufferBuilder fbb; std::vector> object_infos; for (auto const& entry : objects) { @@ -564,10 +466,11 @@ Status SendListReply(int sock, const ObjectTable& objects) { auto message = fb::CreatePlasmaListReply( fbb, fbb.CreateVector(arrow::util::MakeNonNull(object_infos.data()), object_infos.size())); - return PlasmaSend(sock, MessageType::PlasmaListReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaListReply, &fbb); } -Status ReadListReply(uint8_t* data, size_t size, ObjectTable* objects) { +Status ReadListReply(const uint8_t* data, size_t size, ObjectTable* objects) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -588,21 +491,24 @@ Status ReadListReply(uint8_t* data, size_t size, ObjectTable* objects) { // Connect messages. -Status SendConnectRequest(int sock) { +Status SendConnectRequest(const std::shared_ptr& client) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaConnectRequest(fbb); - return PlasmaSend(sock, MessageType::PlasmaConnectRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaConnectRequest, &fbb); } -Status ReadConnectRequest(uint8_t* data) { return Status::OK(); } +Status ReadConnectRequest(const uint8_t* data) { return Status::OK(); } -Status SendConnectReply(int sock, int64_t memory_capacity) { +Status SendConnectReply(const std::shared_ptr& client, + int64_t memory_capacity) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaConnectReply(fbb, memory_capacity); - return PlasmaSend(sock, MessageType::PlasmaConnectReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaConnectReply, &fbb); } -Status ReadConnectReply(uint8_t* data, size_t size, int64_t* memory_capacity) { +Status ReadConnectReply(const uint8_t* data, size_t size, int64_t* memory_capacity) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -612,13 +518,15 @@ Status ReadConnectReply(uint8_t* data, size_t size, int64_t* memory_capacity) { // Evict messages. -Status SendEvictRequest(int sock, int64_t num_bytes) { +Status SendEvictRequest(const std::shared_ptr& client, + int64_t num_bytes) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaEvictRequest(fbb, num_bytes); - return PlasmaSend(sock, MessageType::PlasmaEvictRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaEvictRequest, &fbb); } -Status ReadEvictRequest(uint8_t* data, size_t size, int64_t* num_bytes) { +Status ReadEvictRequest(const uint8_t* data, size_t size, int64_t* num_bytes) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -626,13 +534,15 @@ Status ReadEvictRequest(uint8_t* data, size_t size, int64_t* num_bytes) { return Status::OK(); } -Status SendEvictReply(int sock, int64_t num_bytes) { +Status SendEvictReply(const std::shared_ptr& client, + int64_t num_bytes) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaEvictReply(fbb, num_bytes); - return PlasmaSend(sock, MessageType::PlasmaEvictReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaEvictReply, &fbb); } -Status ReadEvictReply(uint8_t* data, size_t size, int64_t& num_bytes) { +Status ReadEvictReply(const uint8_t* data, size_t size, int64_t& num_bytes) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -642,15 +552,17 @@ Status ReadEvictReply(uint8_t* data, size_t size, int64_t& num_bytes) { // Get messages. -Status SendGetRequest(int sock, const ObjectID* object_ids, int64_t num_objects, +Status SendGetRequest(const std::shared_ptr& client, + const ObjectID* object_ids, int64_t num_objects, int64_t timeout_ms) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaGetRequest( fbb, ToFlatbuffer(&fbb, object_ids, num_objects), timeout_ms); - return PlasmaSend(sock, MessageType::PlasmaGetRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaGetRequest, &fbb); } -Status ReadGetRequest(uint8_t* data, size_t size, std::vector& object_ids, +Status ReadGetRequest(const uint8_t* data, size_t size, std::vector& object_ids, int64_t* timeout_ms) { DCHECK(data); auto message = flatbuffers::GetRoot(data); @@ -663,7 +575,8 @@ Status ReadGetRequest(uint8_t* data, size_t size, std::vector& object_ return Status::OK(); } -Status SendGetReply(int sock, ObjectID object_ids[], +Status SendGetReply(const std::shared_ptr& client, + ObjectID object_ids[], std::unordered_map& plasma_objects, int64_t num_objects, const std::vector& store_fds, const std::vector& mmap_sizes) { @@ -691,10 +604,11 @@ Status SendGetReply(int sock, ObjectID object_ids[], fbb.CreateVector(arrow::util::MakeNonNull(store_fds.data()), store_fds.size()), fbb.CreateVector(arrow::util::MakeNonNull(mmap_sizes.data()), mmap_sizes.size()), fbb.CreateVector(arrow::util::MakeNonNull(handles.data()), handles.size())); - return PlasmaSend(sock, MessageType::PlasmaGetReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaGetReply, &fbb); } -Status ReadGetReply(uint8_t* data, size_t size, ObjectID object_ids[], +Status ReadGetReply(const uint8_t* data, size_t size, ObjectID object_ids[], PlasmaObject plasma_objects[], int64_t num_objects, std::vector& store_fds, std::vector& mmap_sizes) { DCHECK(data); @@ -731,26 +645,20 @@ Status ReadGetReply(uint8_t* data, size_t size, ObjectID object_ids[], return Status::OK(); } -// Subscribe messages. - -Status SendSubscribeRequest(int sock) { - flatbuffers::FlatBufferBuilder fbb; - auto message = fb::CreatePlasmaSubscribeRequest(fbb); - return PlasmaSend(sock, MessageType::PlasmaSubscribeRequest, &fbb, message); -} - // Data messages. -Status SendDataRequest(int sock, ObjectID object_id, const char* address, int port) { +Status SendDataRequest(const std::shared_ptr& client, + ObjectID object_id, const char* address, int port) { flatbuffers::FlatBufferBuilder fbb; auto addr = fbb.CreateString(address, strlen(address)); auto message = fb::CreatePlasmaDataRequest(fbb, fbb.CreateString(object_id.binary()), addr, port); - return PlasmaSend(sock, MessageType::PlasmaDataRequest, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaDataRequest, &fbb); } -Status ReadDataRequest(uint8_t* data, size_t size, ObjectID* object_id, char** address, - int* port) { +Status ReadDataRequest(const uint8_t* data, size_t size, ObjectID* object_id, + char** address, int* port) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -761,15 +669,16 @@ Status ReadDataRequest(uint8_t* data, size_t size, ObjectID* object_id, char** a return Status::OK(); } -Status SendDataReply(int sock, ObjectID object_id, int64_t object_size, - int64_t metadata_size) { +Status SendDataReply(const std::shared_ptr& client, ObjectID object_id, + int64_t object_size, int64_t metadata_size) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaDataReply(fbb, fbb.CreateString(object_id.binary()), object_size, metadata_size); - return PlasmaSend(sock, MessageType::PlasmaDataReply, &fbb, message); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaDataReply, &fbb); } -Status ReadDataReply(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadDataReply(const uint8_t* data, size_t size, ObjectID* object_id, int64_t* object_size, int64_t* metadata_size) { DCHECK(data); auto message = flatbuffers::GetRoot(data); @@ -780,4 +689,39 @@ Status ReadDataReply(uint8_t* data, size_t size, ObjectID* object_id, return Status::OK(); } +Status SendSubscribeRequest(const std::shared_ptr& client) { + // Subscribe messages. + flatbuffers::FlatBufferBuilder fbb; + auto message = fb::CreatePlasmaSubscribeRequest(fbb); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaSubscribeRequest, &fbb); +} + +void SerializeObjectDeletionNotification(const ObjectID& object_id, + std::vector* serialized) { + flatbuf::ObjectInfoT info; + info.object_id = object_id.binary(); + info.is_deletion = true; + flatbuffers::FlatBufferBuilder fbb; + auto message = fb::CreateObjectInfo(fbb, &info); + fbb.Finish(message); + serialized->resize(fbb.GetSize()); + serialized->assign(fbb.GetBufferPointer(), fbb.GetBufferPointer() + fbb.GetSize()); +} + +void SerializeObjectSealedNotification(const ObjectID& object_id, + const ObjectTableEntry& entry, + std::vector* serialized) { + flatbuf::ObjectInfoT info; + info.object_id = object_id.binary(); + info.data_size = entry.data_size; + info.metadata_size = entry.metadata_size; + info.digest = std::string(reinterpret_cast(&entry.digest[0]), kDigestSize); + flatbuffers::FlatBufferBuilder fbb; + auto message = fb::CreateObjectInfo(fbb, &info); + fbb.Finish(message); + serialized->resize(fbb.GetSize()); + serialized->assign(fbb.GetBufferPointer(), fbb.GetBufferPointer() + fbb.GetSize()); +} + } // namespace plasma diff --git a/cpp/src/plasma/protocol.h b/cpp/src/plasma/protocol.h index 6ba1494fd51..e9abb1935a0 100644 --- a/cpp/src/plasma/protocol.h +++ b/cpp/src/plasma/protocol.h @@ -24,214 +24,192 @@ #include #include "arrow/status.h" +#include "plasma/io/connection.h" #include "plasma/plasma.h" #include "plasma/plasma_generated.h" namespace plasma { using arrow::Status; - -using flatbuf::MessageType; using flatbuf::PlasmaError; +using io::ClientConnection; +using io::ServerConnection; template -bool VerifyFlatbuffer(T* object, uint8_t* data, size_t size) { +bool VerifyFlatbuffer(T* object, const uint8_t* data, size_t size) { flatbuffers::Verifier verifier(data, size); return object->Verify(verifier); } -flatbuffers::Offset>> -ToFlatbuffer(flatbuffers::FlatBufferBuilder* fbb, const ObjectID* object_ids, - int64_t num_objects); - -flatbuffers::Offset>> -ToFlatbuffer(flatbuffers::FlatBufferBuilder* fbb, - const std::vector& strings); - -flatbuffers::Offset> ToFlatbuffer( - flatbuffers::FlatBufferBuilder* fbb, const std::vector& data); - -/* Plasma receive message. */ - -Status PlasmaReceive(int sock, MessageType message_type, std::vector* buffer); - -/* Set options messages. */ - -Status SendSetOptionsRequest(int sock, const std::string& client_name, - int64_t output_memory_limit); - -Status ReadSetOptionsRequest(uint8_t* data, size_t size, std::string* client_name, - int64_t* output_memory_quota); - -Status SendSetOptionsReply(int sock, PlasmaError error); - -Status ReadSetOptionsReply(uint8_t* data, size_t size); - -/* Debug string messages. */ - -Status SendGetDebugStringRequest(int sock); - -Status SendGetDebugStringReply(int sock, const std::string& debug_string); - -Status ReadGetDebugStringReply(uint8_t* data, size_t size, std::string* debug_string); - /* Plasma Create message functions. */ -Status SendCreateRequest(int sock, ObjectID object_id, int64_t data_size, - int64_t metadata_size, int device_num); +Status SendCreateRequest(const std::shared_ptr& client, + ObjectID object_id, int64_t data_size, int64_t metadata_size, + int device_num); -Status ReadCreateRequest(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadCreateRequest(const uint8_t* data, size_t size, ObjectID* object_id, int64_t* data_size, int64_t* metadata_size, int* device_num); -Status SendCreateReply(int sock, ObjectID object_id, PlasmaObject* object, - PlasmaError error, int64_t mmap_size); +Status SendCreateReply(const std::shared_ptr& client, + ObjectID object_id, PlasmaObject* object, PlasmaError error, + int64_t mmap_size); -Status ReadCreateReply(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadCreateReply(const uint8_t* data, size_t size, ObjectID* object_id, PlasmaObject* object, int* store_fd, int64_t* mmap_size); -Status SendCreateAndSealRequest(int sock, const ObjectID& object_id, - const std::string& data, const std::string& metadata, - unsigned char* digest); +Status SendCreateAndSealRequest(const std::shared_ptr& client, + const ObjectID& object_id, const std::string& data, + const std::string& metadata, unsigned char* digest); -Status ReadCreateAndSealRequest(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadCreateAndSealRequest(const uint8_t* data, size_t size, ObjectID* object_id, std::string* object_data, std::string* metadata, - std::string* digest); - -Status SendCreateAndSealBatchRequest(int sock, const std::vector& object_ids, - const std::vector& data, - const std::vector& metadata, - const std::vector& digests); - -Status ReadCreateAndSealBatchRequest(uint8_t* data, size_t size, - std::vector* object_id, - std::vector* object_data, - std::vector* metadata, - std::vector* digests); - -Status SendCreateAndSealReply(int sock, PlasmaError error); - -Status ReadCreateAndSealReply(uint8_t* data, size_t size); + unsigned char* digest); -Status SendCreateAndSealBatchReply(int sock, PlasmaError error); +Status SendCreateAndSealReply(const std::shared_ptr& client, + PlasmaError error); -Status ReadCreateAndSealBatchReply(uint8_t* data, size_t size); +Status ReadCreateAndSealReply(const uint8_t* data, size_t size); -Status SendAbortRequest(int sock, ObjectID object_id); +Status SendAbortRequest(const std::shared_ptr& client, + ObjectID object_id); -Status ReadAbortRequest(uint8_t* data, size_t size, ObjectID* object_id); +Status ReadAbortRequest(const uint8_t* data, size_t size, ObjectID* object_id); -Status SendAbortReply(int sock, ObjectID object_id); +Status SendAbortReply(const std::shared_ptr& client, + ObjectID object_id); -Status ReadAbortReply(uint8_t* data, size_t size, ObjectID* object_id); +Status ReadAbortReply(const uint8_t* data, size_t size, ObjectID* object_id); /* Plasma Seal message functions. */ -Status SendSealRequest(int sock, ObjectID object_id, const std::string& digest); +Status SendSealRequest(const std::shared_ptr& client, + ObjectID object_id, unsigned char* digest); -Status ReadSealRequest(uint8_t* data, size_t size, ObjectID* object_id, - std::string* digest); +Status ReadSealRequest(const uint8_t* data, size_t size, ObjectID* object_id, + unsigned char* digest); -Status SendSealReply(int sock, ObjectID object_id, PlasmaError error); +Status SendSealReply(const std::shared_ptr& client, ObjectID object_id, + PlasmaError error); -Status ReadSealReply(uint8_t* data, size_t size, ObjectID* object_id); +Status ReadSealReply(const uint8_t* data, size_t size, ObjectID* object_id); /* Plasma Get message functions. */ -Status SendGetRequest(int sock, const ObjectID* object_ids, int64_t num_objects, +Status SendGetRequest(const std::shared_ptr& client, + const ObjectID* object_ids, int64_t num_objects, int64_t timeout_ms); -Status ReadGetRequest(uint8_t* data, size_t size, std::vector& object_ids, +Status ReadGetRequest(const uint8_t* data, size_t size, std::vector& object_ids, int64_t* timeout_ms); -Status SendGetReply(int sock, ObjectID object_ids[], +Status SendGetReply(const std::shared_ptr& client, + ObjectID object_ids[], std::unordered_map& plasma_objects, int64_t num_objects, const std::vector& store_fds, const std::vector& mmap_sizes); -Status ReadGetReply(uint8_t* data, size_t size, ObjectID object_ids[], +Status ReadGetReply(const uint8_t* data, size_t size, ObjectID object_ids[], PlasmaObject plasma_objects[], int64_t num_objects, std::vector& store_fds, std::vector& mmap_sizes); /* Plasma Release message functions. */ -Status SendReleaseRequest(int sock, ObjectID object_id); +Status SendReleaseRequest(const std::shared_ptr& client, + ObjectID object_id); -Status ReadReleaseRequest(uint8_t* data, size_t size, ObjectID* object_id); +Status ReadReleaseRequest(const uint8_t* data, size_t size, ObjectID* object_id); -Status SendReleaseReply(int sock, ObjectID object_id, PlasmaError error); +Status SendReleaseReply(const std::shared_ptr& client, + ObjectID object_id, PlasmaError error); -Status ReadReleaseReply(uint8_t* data, size_t size, ObjectID* object_id); +Status ReadReleaseReply(const uint8_t* data, size_t size, ObjectID* object_id); /* Plasma Delete objects message functions. */ -Status SendDeleteRequest(int sock, const std::vector& object_ids); +Status SendDeleteRequest(const std::shared_ptr& client, + const std::vector& object_ids); -Status ReadDeleteRequest(uint8_t* data, size_t size, std::vector* object_ids); +Status ReadDeleteRequest(const uint8_t* data, size_t size, + std::vector* object_ids); -Status SendDeleteReply(int sock, const std::vector& object_ids, +Status SendDeleteReply(const std::shared_ptr& client, + const std::vector& object_ids, const std::vector& errors); -Status ReadDeleteReply(uint8_t* data, size_t size, std::vector* object_ids, +Status ReadDeleteReply(const uint8_t* data, size_t size, + std::vector* object_ids, std::vector* errors); /* Plasma Constains message functions. */ -Status SendContainsRequest(int sock, ObjectID object_id); +Status SendContainsRequest(const std::shared_ptr& client, + ObjectID object_id); -Status ReadContainsRequest(uint8_t* data, size_t size, ObjectID* object_id); +Status ReadContainsRequest(const uint8_t* data, size_t size, ObjectID* object_id); -Status SendContainsReply(int sock, ObjectID object_id, bool has_object); +Status SendContainsReply(const std::shared_ptr& client, + ObjectID object_id, bool has_object); -Status ReadContainsReply(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadContainsReply(const uint8_t* data, size_t size, ObjectID* object_id, bool* has_object); /* Plasma List message functions. */ -Status SendListRequest(int sock); +Status SendListRequest(const std::shared_ptr& client); -Status ReadListRequest(uint8_t* data, size_t size); +Status ReadListRequest(const uint8_t* data, size_t size); -Status SendListReply(int sock, const ObjectTable& objects); +Status SendListReply(const std::shared_ptr& client, + const ObjectTable& objects); -Status ReadListReply(uint8_t* data, size_t size, ObjectTable* objects); +Status ReadListReply(const uint8_t* data, size_t size, ObjectTable* objects); /* Plasma Connect message functions. */ -Status SendConnectRequest(int sock); +Status SendConnectRequest(const std::shared_ptr& client); -Status ReadConnectRequest(uint8_t* data, size_t size); +Status ReadConnectRequest(const uint8_t* data, size_t size); -Status SendConnectReply(int sock, int64_t memory_capacity); +Status SendConnectReply(const std::shared_ptr& client, + int64_t memory_capacity); -Status ReadConnectReply(uint8_t* data, size_t size, int64_t* memory_capacity); +Status ReadConnectReply(const uint8_t* data, size_t size, int64_t* memory_capacity); /* Plasma Evict message functions (no reply so far). */ -Status SendEvictRequest(int sock, int64_t num_bytes); - -Status ReadEvictRequest(uint8_t* data, size_t size, int64_t* num_bytes); - -Status SendEvictReply(int sock, int64_t num_bytes); +Status SendEvictRequest(const std::shared_ptr& client, + int64_t num_bytes); -Status ReadEvictReply(uint8_t* data, size_t size, int64_t& num_bytes); +Status ReadEvictRequest(const uint8_t* data, size_t size, int64_t* num_bytes); -/* Plasma Subscribe message functions. */ +Status SendEvictReply(const std::shared_ptr& client, int64_t num_bytes); -Status SendSubscribeRequest(int sock); +Status ReadEvictReply(const uint8_t* data, size_t size, int64_t& num_bytes); /* Data messages. */ -Status SendDataRequest(int sock, ObjectID object_id, const char* address, int port); +Status SendDataRequest(const std::shared_ptr& client, + ObjectID object_id, const char* address, int port); -Status ReadDataRequest(uint8_t* data, size_t size, ObjectID* object_id, char** address, - int* port); +Status ReadDataRequest(const uint8_t* data, size_t size, ObjectID* object_id, + char** address, int* port); -Status SendDataReply(int sock, ObjectID object_id, int64_t object_size, - int64_t metadata_size); +Status SendDataReply(const std::shared_ptr& client, ObjectID object_id, + int64_t object_size, int64_t metadata_size); -Status ReadDataReply(uint8_t* data, size_t size, ObjectID* object_id, +Status ReadDataReply(const uint8_t* data, size_t size, ObjectID* object_id, int64_t* object_size, int64_t* metadata_size); +/* Plasma notification message functions. */ + +Status SendSubscribeRequest(const std::shared_ptr& client); + +void SerializeObjectDeletionNotification(const ObjectID& object_id, + std::vector* serialized); + +void SerializeObjectSealedNotification(const ObjectID& object_id, + const ObjectTableEntry& entry, + std::vector* serialized); } // namespace plasma #endif /* PLASMA_PROTOCOL */ diff --git a/cpp/src/plasma/store.cc b/cpp/src/plasma/store.cc index 03f7c21738f..7a0a434d622 100644 --- a/cpp/src/plasma/store.cc +++ b/cpp/src/plasma/store.cc @@ -28,23 +28,16 @@ #include "plasma/store.h" -#include -#include #include -#include #include #include #include -#include -#include -#include +#include +#ifdef __linux__ #include -#include -#include -#include - +#endif +#include #include -#include #include #include #include @@ -53,13 +46,11 @@ #include #include "arrow/status.h" - +#include "arrow/util/logging.h" #include "plasma/common.h" -#include "plasma/common_generated.h" -#include "plasma/fling.h" -#include "plasma/io.h" #include "plasma/malloc.h" #include "plasma/plasma_allocator.h" +#include "plasma/plasma_generated.h" #include "plasma/protocol.h" #ifdef PLASMA_CUDA @@ -73,19 +64,68 @@ using arrow::cuda::CudaDeviceManager; using arrow::util::ArrowLog; using arrow::util::ArrowLogLevel; -namespace fb = plasma::flatbuf; - namespace plasma { +using flatbuf::MessageType; + void SetMallocGranularity(int value); struct GetRequest { - GetRequest(Client* client, const std::vector& object_ids); + GetRequest(asio::io_context& io_context, + const std::shared_ptr& client, + const std::vector& object_ids) + : client(client), + object_ids(object_ids.begin(), object_ids.end()), + objects(object_ids.size()), + num_satisfied(0), + timer_(io_context) { + std::unordered_set unique_ids(object_ids.begin(), object_ids.end()); + num_objects_to_wait_for = unique_ids.size(); + } + + void ReturnFromGet() { + // Figure out how many file descriptors we need to send. + std::unordered_set fds_to_send; + std::vector store_fds; + std::vector mmap_sizes; + for (const auto& object_id : object_ids) { + PlasmaObject& object = objects[object_id]; + int fd = object.store_fd; + if (object.data_size != -1 && fds_to_send.count(fd) == 0 && fd != -1) { + fds_to_send.insert(fd); + store_fds.push_back(fd); + mmap_sizes.push_back(GetMmapSize(fd)); + } + } + + // Send the get reply to the client. + Status s = SendGetReply(client, &object_ids[0], objects, object_ids.size(), store_fds, + mmap_sizes); + // If we successfully sent the get reply message to the client, then also send + // the file descriptors. + if (s.ok()) { + // Send all of the file descriptors for the present objects. + for (int store_fd : store_fds) { + auto status = client->SendFd(store_fd); + if (!status.ok()) { + // TODO(suquark): Should we close the client here? + ARROW_LOG(ERROR) << "Failed to send a mmap fd to client"; + } + } + } + } + + void AsyncWait(int64_t timeout_ms, + std::function on_timeout) { + // Set an expiry time relative to now. + timer_.expires_from_now(std::chrono::milliseconds(timeout_ms)); + timer_.async_wait(on_timeout); + } + + void CancelTimer() { timer_.cancel(); } + /// The client that called get. - Client* client; - /// The ID of the timer that will time out and cause this wait to return to - /// the client if it hasn't already returned. - int64_t timer; + std::shared_ptr client; /// The object IDs involved in this request. This is used in the reply. std::vector object_ids; /// The object information for the objects in this request. This is used in @@ -96,31 +136,29 @@ struct GetRequest { /// The number of object requests in this wait request that are already /// satisfied. int64_t num_satisfied; -}; -GetRequest::GetRequest(Client* client, const std::vector& object_ids) - : client(client), - timer(-1), - object_ids(object_ids.begin(), object_ids.end()), - objects(object_ids.size()), - num_satisfied(0) { - std::unordered_set unique_ids(object_ids.begin(), object_ids.end()); - num_objects_to_wait_for = unique_ids.size(); -} - -Client::Client(int fd) : fd(fd), notification_fd(-1) {} + private: + /// The timer that will time out and cause this wait to return to + /// the client if it hasn't already returned. + asio::steady_timer timer_; +}; -PlasmaStore::PlasmaStore(EventLoop* loop, std::string directory, bool hugepages_enabled, - const std::string& socket_name, +PlasmaStore::PlasmaStore(asio::io_context& io_context, std::string directory, + bool hugepages_enabled, const std::string& stream_name, std::shared_ptr external_store) - : loop_(loop), - eviction_policy_(&store_info_, PlasmaAllocator::GetFootprintLimit()), - external_store_(external_store) { + : eviction_policy_(&store_info_), + external_store_(external_store), + io_context_(io_context), + stream_name_(stream_name), + acceptor_(io::CreateLocalAcceptor(io_context, stream_name)), + stream_(io_context) { store_info_.directory = directory; store_info_.hugepages_enabled = hugepages_enabled; #ifdef PLASMA_CUDA DCHECK_OK(CudaDeviceManager::GetInstance(&manager_)); #endif + // Start listening for clients. + DoAccept(); } // TODO(pcm): Get rid of this destructor by using RAII to clean up data. @@ -131,36 +169,19 @@ const PlasmaStoreInfo* PlasmaStore::GetPlasmaStoreInfo() { return &store_info_; // If this client is not already using the object, add the client to the // object's list of clients, otherwise do nothing. void PlasmaStore::AddToClientObjectIds(const ObjectID& object_id, ObjectTableEntry* entry, - Client* client) { + const std::shared_ptr& client) { // Check if this client is already using the object. - if (client->object_ids.find(object_id) != client->object_ids.end()) { + if (client->ObjectIDExists(object_id)) { return; } - // If there are no other clients using this object, notify the eviction policy - // that the object is being used. - if (entry->ref_count == 0) { - // Tell the eviction policy that this object is being used. - eviction_policy_.BeginObjectAccess(object_id); - } - // Increase reference count. - entry->ref_count++; - + IncreaseObjectRefCount(object_id, entry); // Add object id to the list of object ids that this client is using. client->object_ids.insert(object_id); } // Allocate memory uint8_t* PlasmaStore::AllocateMemory(size_t size, int* fd, int64_t* map_size, - ptrdiff_t* offset, Client* client, bool is_create) { - // First free up space from the client's LRU queue if quota enforcement is on. - std::vector client_objects_to_evict; - bool quota_ok = eviction_policy_.EnforcePerClientQuota(client, size, is_create, - &client_objects_to_evict); - if (!quota_ok) { - return nullptr; - } - EvictObjects(client_objects_to_evict); - + ptrdiff_t* offset) { // Try to evict objects until there is enough space. uint8_t* pointer = nullptr; while (true) { @@ -215,7 +236,8 @@ Status PlasmaStore::FreeCudaMemory(int device_num, int64_t size, uint8_t* pointe // Create a new object buffer in the hash table. PlasmaError PlasmaStore::CreateObject(const ObjectID& object_id, int64_t data_size, int64_t metadata_size, int device_num, - Client* client, PlasmaObject* result) { + const std::shared_ptr& client, + PlasmaObject* result) { ARROW_LOG(DEBUG) << "creating object " << object_id.hex(); auto entry = GetObjectTableEntry(&store_info_, object_id); @@ -224,6 +246,10 @@ PlasmaError PlasmaStore::CreateObject(const ObjectID& object_id, int64_t data_si // ignore this requst. return PlasmaError::ObjectExists; } + auto ptr = std::unique_ptr(new ObjectTableEntry()); + entry = store_info_.objects.emplace(object_id, std::move(ptr)).first->second.get(); + entry->data_size = data_size; + entry->metadata_size = metadata_size; int fd = -1; int64_t map_size = 0; @@ -231,35 +257,29 @@ PlasmaError PlasmaStore::CreateObject(const ObjectID& object_id, int64_t data_si uint8_t* pointer = nullptr; auto total_size = data_size + metadata_size; - if (device_num == 0) { - pointer = AllocateMemory(total_size, &fd, &map_size, &offset, client, true); - if (!pointer) { - ARROW_LOG(ERROR) << "Not enough memory to create the object " << object_id.hex() - << ", data_size=" << data_size - << ", metadata_size=" << metadata_size - << ", will send a reply of PlasmaError::OutOfMemory"; - return PlasmaError::OutOfMemory; - } - } else { + if (device_num != 0) { #ifdef PLASMA_CUDA - /// IPC GPU handle to share with clients. - std::shared_ptr<::arrow::cuda::CudaIpcMemHandle> ipc_handle; - auto st = AllocateCudaMemory(device_num, total_size, &pointer, &ipc_handle); + auto st = AllocateCudaMemory(device_num, total_size, &pointer, &entry->ipc_handle); if (!st.ok()) { ARROW_LOG(ERROR) << "Failed to allocate CUDA memory: " << st.ToString(); return PlasmaError::OutOfMemory; } - result->ipc_handle = ipc_handle; + result->ipc_handle = entry->ipc_handle; #else ARROW_LOG(ERROR) << "device_num != 0 but CUDA not enabled"; return PlasmaError::OutOfMemory; #endif + } else { + pointer = AllocateMemory(total_size, &fd, &map_size, &offset); + if (!pointer) { + ARROW_LOG(ERROR) << "Not enough memory to create the object " << object_id.hex() + << ", data_size=" << data_size + << ", metadata_size=" << metadata_size + << ", will send a reply of PlasmaError::OutOfMemory"; + return PlasmaError::OutOfMemory; + } } - auto ptr = std::unique_ptr(new ObjectTableEntry()); - entry = store_info_.objects.emplace(object_id, std::move(ptr)).first->second.get(); - entry->data_size = data_size; - entry->metadata_size = metadata_size; entry->pointer = pointer; // TODO(pcm): Set the other fields. entry->fd = fd; @@ -270,10 +290,6 @@ PlasmaError PlasmaStore::CreateObject(const ObjectID& object_id, int64_t data_si entry->create_time = std::time(nullptr); entry->construct_duration = -1; -#ifdef PLASMA_CUDA - entry->ipc_handle = result->ipc_handle; -#endif - result->store_fd = fd; result->data_offset = offset; result->metadata_offset = offset + data_size; @@ -283,7 +299,7 @@ PlasmaError PlasmaStore::CreateObject(const ObjectID& object_id, int64_t data_si // Notify the eviction policy that this object was created. This must be done // immediately before the call to AddToClientObjectIds so that the // eviction policy does not have an opportunity to evict the object. - eviction_policy_.ObjectCreated(object_id, client, true); + eviction_policy_.ObjectCreated(object_id); // Record that this client is using this object. AddToClientObjectIds(object_id, store_info_.objects[object_id].get(), client); return PlasmaError::OK; @@ -326,13 +342,12 @@ void PlasmaStore::RemoveGetRequest(GetRequest* get_request) { } } // Remove the get request. - if (get_request->timer != -1) { - ARROW_CHECK(loop_->RemoveTimer(get_request->timer) == kEventLoopOk); - } + get_request->CancelTimer(); delete get_request; } -void PlasmaStore::RemoveGetRequestsForClient(Client* client) { +void PlasmaStore::RemoveGetRequestsForClient( + const std::shared_ptr& client) { std::unordered_set get_requests_to_remove; for (auto const& pair : object_get_requests_) { for (GetRequest* get_request : pair.second) { @@ -351,38 +366,7 @@ void PlasmaStore::RemoveGetRequestsForClient(Client* client) { } void PlasmaStore::ReturnFromGet(GetRequest* get_req) { - // Figure out how many file descriptors we need to send. - std::unordered_set fds_to_send; - std::vector store_fds; - std::vector mmap_sizes; - for (const auto& object_id : get_req->object_ids) { - PlasmaObject& object = get_req->objects[object_id]; - int fd = object.store_fd; - if (object.data_size != -1 && fds_to_send.count(fd) == 0 && fd != -1) { - fds_to_send.insert(fd); - store_fds.push_back(fd); - mmap_sizes.push_back(GetMmapSize(fd)); - } - } - - // Send the get reply to the client. - Status s = SendGetReply(get_req->client->fd, &get_req->object_ids[0], get_req->objects, - get_req->object_ids.size(), store_fds, mmap_sizes); - WarnIfSigpipe(s.ok() ? 0 : -1, get_req->client->fd); - // If we successfully sent the get reply message to the client, then also send - // the file descriptors. - if (s.ok()) { - // Send all of the file descriptors for the present objects. - for (int store_fd : store_fds) { - // Only send the file descriptor if it hasn't been sent (see analogous - // logic in GetStoreFd in client.cc). - if (get_req->client->used_fds.find(store_fd) == get_req->client->used_fds.end()) { - WarnIfSigpipe(send_fd(get_req->client->fd, store_fd), get_req->client->fd); - get_req->client->used_fds.insert(store_fd); - } - } - } - + get_req->ReturnFromGet(); // Remove the get request from each of the relevant object_get_requests hash // tables if it is present there. It should only be present there if the get // request timed out. @@ -433,11 +417,11 @@ void PlasmaStore::UpdateObjectGetRequests(const ObjectID& object_id) { } } -void PlasmaStore::ProcessGetRequest(Client* client, - const std::vector& object_ids, - int64_t timeout_ms) { +Status PlasmaStore::ProcessGetRequest(const std::shared_ptr& client, + const std::vector& object_ids, + int64_t timeout_ms) { // Create a get request for this object. - auto get_req = new GetRequest(client, object_ids); + auto get_req = new GetRequest(io_context_, client, object_ids); std::vector evicted_ids; std::vector evicted_entries; for (auto object_id : object_ids) { @@ -456,11 +440,11 @@ void PlasmaStore::ProcessGetRequest(Client* client, ARROW_CHECK(!entry->pointer); entry->pointer = AllocateMemory(entry->data_size + entry->metadata_size, &entry->fd, - &entry->map_size, &entry->offset, client, false); + &entry->map_size, &entry->offset); if (entry->pointer) { entry->state = ObjectState::PLASMA_CREATED; entry->create_time = std::time(nullptr); - eviction_policy_.ObjectCreated(object_id, client, false); + eviction_policy_.ObjectCreated(object_id); AddToClientObjectIds(object_id, store_info_.objects[object_id].get(), client); evicted_ids.push_back(object_id); evicted_entries.push_back(entry); @@ -514,40 +498,14 @@ void PlasmaStore::ProcessGetRequest(Client* client, } else if (timeout_ms != -1) { // Set a timer that will cause the get request to return to the client. Note // that a timeout of -1 is used to indicate that no timer should be set. - get_req->timer = loop_->AddTimer(timeout_ms, [this, get_req](int64_t timer_id) { - ReturnFromGet(get_req); - return kEventLoopTimerDone; - }); - } -} - -int PlasmaStore::RemoveFromClientObjectIds(const ObjectID& object_id, - ObjectTableEntry* entry, Client* client) { - auto it = client->object_ids.find(object_id); - if (it != client->object_ids.end()) { - client->object_ids.erase(it); - // Decrease reference count. - entry->ref_count--; - - // If no more clients are using this object, notify the eviction policy - // that the object is no longer being used. - if (entry->ref_count == 0) { - if (deletion_cache_.count(object_id) == 0) { - // Tell the eviction policy that this object is no longer being used. - eviction_policy_.EndObjectAccess(object_id); - } else { - // Above code does not really delete an object. Instead, it just put an - // object to LRU cache which will be cleaned when the memory is not enough. - deletion_cache_.erase(object_id); - EvictObjects({object_id}); + get_req->AsyncWait(timeout_ms, [this, get_req](const error_code& ec) { + if (ec != asio::error::operation_aborted) { + // Timer was not cancelled, take necessary action. + ReturnFromGet(get_req); } - } - // Return 1 to indicate that the client was removed. - return 1; - } else { - // Return 0 to indicate that the client was not removed. - return 0; + }); } + return Status::OK(); } void PlasmaStore::EraseFromObjectTable(const ObjectID& object_id) { @@ -563,11 +521,13 @@ void PlasmaStore::EraseFromObjectTable(const ObjectID& object_id) { store_info_.objects.erase(object_id); } -void PlasmaStore::ReleaseObject(const ObjectID& object_id, Client* client) { +void PlasmaStore::ReleaseObject(const ObjectID& object_id, + const std::shared_ptr& client) { + // Remove the client from the object's array of clients. + ARROW_CHECK(client->RemoveObjectIDIfExists(object_id)); auto entry = GetObjectTableEntry(&store_info_, object_id); ARROW_CHECK(entry != nullptr); - // Remove the client from the object's array of clients. - ARROW_CHECK(RemoveFromClientObjectIds(object_id, entry, client) == 1); + DecreaseObjectRefCount(object_id, entry); } // Check if an object is present. @@ -579,6 +539,7 @@ ObjectStatus PlasmaStore::ContainsObject(const ObjectID& object_id) { : ObjectStatus::OBJECT_NOT_FOUND; } +// Seal an object that has been created in the hash table. void PlasmaStore::SealObjects(const std::vector& object_ids, const std::vector& digests) { std::vector infos; @@ -610,7 +571,8 @@ void PlasmaStore::SealObjects(const std::vector& object_ids, } } -int PlasmaStore::AbortObject(const ObjectID& object_id, Client* client) { +int PlasmaStore::AbortObject(const ObjectID& object_id, + const std::shared_ptr& client) { auto entry = GetObjectTableEntry(&store_info_, object_id); ARROW_CHECK(entry != nullptr) << "To abort an object it must be in the object table."; ARROW_CHECK(entry->state != ObjectState::PLASMA_SEALED) @@ -655,19 +617,11 @@ PlasmaError PlasmaStore::DeleteObject(ObjectID& object_id) { eviction_policy_.RemoveObject(object_id); EraseFromObjectTable(object_id); // Inform all subscribers that the object has been deleted. - fb::ObjectInfoT notification; - notification.object_id = object_id.binary(); - notification.is_deletion = true; - PushNotification(¬ification); - + PushObjectDeletionNotification(object_id); return PlasmaError::OK; } void PlasmaStore::EvictObjects(const std::vector& object_ids) { - if (object_ids.size() == 0) { - return; - } - std::vector> evicted_object_data; std::vector evicted_entries; for (const auto& object_id : object_ids) { @@ -694,10 +648,7 @@ void PlasmaStore::EvictObjects(const std::vector& object_ids) { // and send a deletion notification. EraseFromObjectTable(object_id); // Inform all subscribers that the object has been deleted. - fb::ObjectInfoT notification; - notification.object_id = object_id.binary(); - notification.is_deletion = true; - PushNotification(¬ification); + PushObjectDeletionNotification(object_id); } } @@ -711,34 +662,107 @@ void PlasmaStore::EvictObjects(const std::vector& object_ids) { } } -void PlasmaStore::ConnectClient(int listener_sock) { - int client_fd = AcceptClient(listener_sock); +void PlasmaStore::IncreaseObjectRefCount(const ObjectID& object_id, + ObjectTableEntry* entry) { + // If there are no other clients using this object, notify the eviction policy + // that the object is being used. + if (entry->ref_count == 0) { + // Tell the eviction policy that this object is being used. + std::vector objects_to_evict; + eviction_policy_.BeginObjectAccess(object_id, &objects_to_evict); + EvictObjects(objects_to_evict); + } + // Increase reference count. + entry->ref_count++; +} - Client* client = new Client(client_fd); - connected_clients_[client_fd] = std::unique_ptr(client); +void PlasmaStore::DecreaseObjectRefCount(const ObjectID& object_id, + ObjectTableEntry* entry) { + // Decrease reference count. + entry->ref_count--; - // Add a callback to handle events on this socket. - // TODO(pcm): Check return value. - loop_->AddFileEvent(client_fd, kEventLoopRead, [this, client](int events) { - Status s = ProcessMessage(client); - if (!s.ok()) { - ARROW_LOG(FATAL) << "Failed to process file event: " << s; + // If no more clients are using this object, notify the eviction policy + // that the object is no longer being used. + if (entry->ref_count == 0) { + if (deletion_cache_.count(object_id) == 0) { + // Tell the eviction policy that this object is no longer being used. + std::vector objects_to_evict; + eviction_policy_.EndObjectAccess(object_id, &objects_to_evict); + EvictObjects(objects_to_evict); + } else { + // Above code does not really delete an object. Instead, it just put an + // object to LRU cache which will be cleaned when the memory is not enough. + deletion_cache_.erase(object_id); + EvictObjects({object_id}); } - }); - ARROW_LOG(DEBUG) << "New connection with fd " << client_fd; + } +} + +void PlasmaStore::PushObjectReadyNotification(const ObjectID& object_id, + const ObjectTableEntry& entry) { + for (const auto& client : notification_clients_) { + client->SendObjectReadyAsync(object_id, entry); + } } -void PlasmaStore::DisconnectClient(int client_fd) { - ARROW_CHECK(client_fd > 0); - auto it = connected_clients_.find(client_fd); - ARROW_CHECK(it != connected_clients_.end()); - loop_->RemoveFileEvent(client_fd); - // Close the socket. - close(client_fd); - ARROW_LOG(INFO) << "Disconnecting client on fd " << client_fd; +void PlasmaStore::PushObjectDeletionNotification(const ObjectID& object_id) { + for (const auto& client : notification_clients_) { + client->SendObjectDeletionAsync(object_id); + } +} + +// Subscribe to notifications about sealed objects. +void PlasmaStore::SubscribeToUpdates(const std::shared_ptr& client) { + ARROW_LOG(DEBUG) << "subscribing to updates on fd " << client->GetNativeHandle(); + if (notification_clients_.count(client) > 0) { + // This client has already subscribed. Return. + return; + } + + // Add this client to the notification set, which is needed for this client to receive + // notifications. + notification_clients_.insert(client); + + // Push notifications to the new subscriber about existing sealed objects. + for (const auto& entry : store_info_.objects) { + if (entry.second->state == ObjectState::PLASMA_SEALED) { + client->SendObjectReadyAsync(entry.first, *entry.second); + } + } +} + +void PlasmaStore::DoAccept() { + // TODO(suquark): Use shared_from_this() here ? + acceptor_.async_accept(stream_, + [this](const error_code& ec) { HandleAccept(ec); }); +} + +void PlasmaStore::HandleAccept(const error_code& error) { + if (!error) { + io::MessageHandler message_handler = [this](std::shared_ptr client, + int64_t message_type, int64_t length, + const uint8_t* message) { + Status s = ProcessClientMessage(client, message_type, length, message); + if (!s.ok()) { + ARROW_LOG(ERROR) << "[PlasmaStore] Failed to process the event" + << "(type=" << message_type << "): " << s << ", " + << "fd = " << client->GetNativeHandle(); + } + }; + // Accept a new local client and dispatch it to the store. + auto new_connection = ClientConnection::Create(std::move(stream_), message_handler); + // Insert the client before processing messages. + connected_clients_.insert(new_connection); + // Process our new connection. + new_connection->ProcessMessages(); + } + // We're ready to accept another client. + DoAccept(); +} + +void PlasmaStore::ReleaseClientResources( + const std::shared_ptr& client) { // Release all the objects that the client was using. - auto client = it->second.get(); - eviction_policy_.ClientDisconnected(client); std::unordered_map sealed_objects; for (const auto& object_id : client->object_ids) { auto it = store_info_.objects.find(object_id); @@ -761,207 +785,86 @@ void PlasmaStore::DisconnectClient(int client_fd) { RemoveGetRequestsForClient(client); for (const auto& entry : sealed_objects) { - RemoveFromClientObjectIds(entry.first, entry.second, client); - } - - if (client->notification_fd > 0) { - // This client has subscribed for notifications. - auto notify_fd = client->notification_fd; - loop_->RemoveFileEvent(notify_fd); - // Close socket. - close(notify_fd); - // Remove notification queue for this fd from global map. - pending_notifications_.erase(notify_fd); - // Reset fd. - client->notification_fd = -1; - } - - connected_clients_.erase(it); -} - -/// Send notifications about sealed objects to the subscribers. This is called -/// in SealObject. If the socket's send buffer is full, the notification will -/// be buffered, and this will be called again when the send buffer has room. -/// Since we call erase on pending_notifications_, all iterators get -/// invalidated, which is why we return a valid iterator to the next client to -/// be used in PushNotification. -/// -/// @param it Iterator that points to the client to send the notification to. -/// @return Iterator pointing to the next client. -PlasmaStore::NotificationMap::iterator PlasmaStore::SendNotifications( - PlasmaStore::NotificationMap::iterator it) { - int client_fd = it->first; - auto& notifications = it->second.object_notifications; - - int num_processed = 0; - bool closed = false; - // Loop over the array of pending notifications and send as many of them as - // possible. - for (size_t i = 0; i < notifications.size(); ++i) { - auto& notification = notifications.at(i); - // Decode the length, which is the first bytes of the message. - int64_t size = *(reinterpret_cast(notification.get())); - - // Attempt to send a notification about this object ID. - ssize_t nbytes = send(client_fd, notification.get(), sizeof(int64_t) + size, 0); - if (nbytes >= 0) { - ARROW_CHECK(nbytes == static_cast(sizeof(int64_t)) + size); - } else if (nbytes == -1 && - (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR)) { - ARROW_LOG(DEBUG) << "The socket's send buffer is full, so we are caching this " - "notification and will send it later."; - // Add a callback to the event loop to send queued notifications whenever - // there is room in the socket's send buffer. Callbacks can be added - // more than once here and will be overwritten. The callback is removed - // at the end of the method. - // TODO(pcm): Introduce status codes and check in case the file descriptor - // is added twice. - loop_->AddFileEvent(client_fd, kEventLoopWrite, [this, client_fd](int events) { - SendNotifications(pending_notifications_.find(client_fd)); - }); - break; - } else { - ARROW_LOG(WARNING) << "Failed to send notification to client on fd " << client_fd; - if (errno == EPIPE) { - closed = true; - break; - } - } - num_processed += 1; - } - // Remove the sent notifications from the array. - notifications.erase(notifications.begin(), notifications.begin() + num_processed); - - // If we have sent all notifications, remove the fd from the event loop. - if (notifications.empty()) { - loop_->RemoveFileEvent(client_fd); - } - - // Stop sending notifications if the pipe was broken. - if (closed) { - close(client_fd); - return pending_notifications_.erase(it); - } else { - return ++it; + // The object ID must exist in client's record. + client->RemoveObjectID(entry.first); + DecreaseObjectRefCount(entry.first, entry.second); } } -void PlasmaStore::PushNotification(fb::ObjectInfoT* object_info) { - auto it = pending_notifications_.begin(); - while (it != pending_notifications_.end()) { - std::vector info; - info.push_back(*object_info); - auto notification = CreatePlasmaNotificationBuffer(info); - it->second.object_notifications.emplace_back(std::move(notification)); - it = SendNotifications(it); - } -} - -void PlasmaStore::PushNotifications(std::vector& object_info) { - auto it = pending_notifications_.begin(); - while (it != pending_notifications_.end()) { - auto notifications = CreatePlasmaNotificationBuffer(object_info); - it->second.object_notifications.emplace_back(std::move(notifications)); - it = SendNotifications(it); - } -} - -void PlasmaStore::PushNotification(fb::ObjectInfoT* object_info, int client_fd) { - auto it = pending_notifications_.find(client_fd); - if (it != pending_notifications_.end()) { - std::vector info; - info.push_back(*object_info); - auto notification = CreatePlasmaNotificationBuffer(info); - it->second.object_notifications.emplace_back(std::move(notification)); - SendNotifications(it); - } -} - -// Subscribe to notifications about sealed objects. -void PlasmaStore::SubscribeToUpdates(Client* client) { - ARROW_LOG(DEBUG) << "subscribing to updates on fd " << client->fd; - if (client->notification_fd > 0) { - // This client has already subscribed. Return. +void PlasmaStore::ProcessDisconnectClient( + const std::shared_ptr& client) { + if (!client->IsOpen()) { + ARROW_LOG(ERROR) << "Received disconnection request from a disconnected client."; return; } - - // TODO(rkn): The store could block here if the client doesn't send a file - // descriptor. - int fd = recv_fd(client->fd); - if (fd < 0) { - // This may mean that the client died before sending the file descriptor. - ARROW_LOG(WARNING) << "Failed to receive file descriptor from client on fd " - << client->fd << "."; + // Close the client. + ARROW_LOG(INFO) << "Disconnecting client on fd " << client->GetNativeHandle(); + client->Close(); + + // Remove the client from the connection set. + auto it = connected_clients_.find(client); + if (it == connected_clients_.end()) { + ARROW_LOG(FATAL) << "[PlasmaStore] (on DisconnectClient) Unexpected error: The " + << "client to disconnect is not in the connected clients list."; return; } - - // Add this fd to global map, which is needed for this client to receive notifications. - pending_notifications_[fd]; - client->notification_fd = fd; - - // Push notifications to the new subscriber about existing sealed objects. - for (const auto& entry : store_info_.objects) { - if (entry.second->state == ObjectState::PLASMA_SEALED) { - ObjectInfoT info; - info.object_id = entry.first.binary(); - info.data_size = entry.second->data_size; - info.metadata_size = entry.second->metadata_size; - info.digest = - std::string(reinterpret_cast(&entry.second->digest[0]), kDigestSize); - PushNotification(&info, fd); - } + connected_clients_.erase(it); + // Remove the client from the notification set. + if (notification_clients_.count(client) > 0) { + notification_clients_.erase(client); } -} -Status PlasmaStore::ProcessMessage(Client* client) { - fb::MessageType type; - Status s = ReadMessage(client->fd, &type, &input_buffer_); - ARROW_CHECK(s.ok() || s.IsIOError()); + // Release resources. + ReleaseClientResources(client); +} - uint8_t* input = input_buffer_.data(); - size_t input_size = input_buffer_.size(); +Status PlasmaStore::ProcessClientMessage(const std::shared_ptr& client, + int64_t message_type, int64_t message_size, + const uint8_t* message_data) { + auto message_type_value = static_cast(message_type); ObjectID object_id; - PlasmaObject object = {}; // Process the different types of requests. - switch (type) { - case fb::MessageType::PlasmaCreateRequest: { + switch (message_type_value) { + case MessageType::PlasmaCreateRequest: { int64_t data_size; int64_t metadata_size; int device_num; - RETURN_NOT_OK(ReadCreateRequest(input, input_size, &object_id, &data_size, + RETURN_NOT_OK(ReadCreateRequest(message_data, message_size, &object_id, &data_size, &metadata_size, &device_num)); + PlasmaObject object = {}; PlasmaError error_code = CreateObject(object_id, data_size, metadata_size, device_num, client, &object); int64_t mmap_size = 0; if (error_code == PlasmaError::OK && device_num == 0) { mmap_size = GetMmapSize(object.store_fd); } - HANDLE_SIGPIPE( - SendCreateReply(client->fd, object_id, &object, error_code, mmap_size), - client->fd); + RETURN_NOT_OK(SendCreateReply(client, object_id, &object, error_code, mmap_size)); // Only send the file descriptor if it hasn't been sent (see analogous // logic in GetStoreFd in client.cc). Similar in ReturnFromGet. - if (error_code == PlasmaError::OK && device_num == 0 && - client->used_fds.find(object.store_fd) == client->used_fds.end()) { - WarnIfSigpipe(send_fd(client->fd, object.store_fd), client->fd); - client->used_fds.insert(object.store_fd); + if (error_code == PlasmaError::OK && device_num == 0) { + auto status = client->SendFd(object.store_fd); + if (!status.ok()) { + // TODO(suquark): Should we close the client here? + ARROW_LOG(ERROR) << "[PlasmaStore] (on CreateRequest) Failed to send a mmap fd" + << " to the client."; + } } } break; - case fb::MessageType::PlasmaCreateAndSealRequest: { + case MessageType::PlasmaCreateAndSealRequest: { std::string data; std::string metadata; - std::string digest; - digest.reserve(kDigestSize); - RETURN_NOT_OK(ReadCreateAndSealRequest(input, input_size, &object_id, &data, - &metadata, &digest)); + unsigned char digest[kDigestSize]; + RETURN_NOT_OK(ReadCreateAndSealRequest(message_data, message_size, &object_id, + &data, &metadata, &digest[0])); + PlasmaObject object = {}; // CreateAndSeal currently only supports device_num = 0, which corresponds // to the host. int device_num = 0; PlasmaError error_code = CreateObject(object_id, data.size(), metadata.size(), device_num, client, &object); // Reply to the client. - HANDLE_SIGPIPE(SendCreateAndSealReply(client->fd, error_code), client->fd); + RETURN_NOT_OK(SendCreateAndSealReply(client, error_code)); // If the object was successfully created, fill out the object data and seal it. if (error_code == PlasmaError::OK) { @@ -970,149 +873,84 @@ Status PlasmaStore::ProcessMessage(Client* client) { // Write the inlined data and metadata into the allocated object. std::memcpy(entry->pointer, data.data(), data.size()); std::memcpy(entry->pointer + data.size(), metadata.data(), metadata.size()); - SealObjects({object_id}, {digest}); + SealObject(object_id, &digest[0]); // Remove the client from the object's array of clients because the // object is not being used by any client. The client was added to the // object's array of clients in CreateObject. This is analogous to the // Release call that happens in the client's Seal method. - ARROW_CHECK(RemoveFromClientObjectIds(object_id, entry, client) == 1); + ARROW_CHECK(client->RemoveObjectIDIfExists(object_id)); + DecreaseObjectRefCount(object_id, entry); } } break; - case fb::MessageType::PlasmaCreateAndSealBatchRequest: { - std::vector object_ids; - std::vector data; - std::vector metadata; - std::vector digests; - - RETURN_NOT_OK(ReadCreateAndSealBatchRequest(input, input_size, &object_ids, &data, - &metadata, &digests)); - - // CreateAndSeal currently only supports device_num = 0, which corresponds - // to the host. - int device_num = 0; - size_t i = 0; - PlasmaError error_code = PlasmaError::OK; - for (i = 0; i < object_ids.size(); i++) { - error_code = CreateObject(object_ids[i], data[i].size(), metadata[i].size(), - device_num, client, &object); - if (error_code != PlasmaError::OK) { - break; - } + case MessageType::PlasmaAbortRequest: { + RETURN_NOT_OK(ReadAbortRequest(message_data, message_size, &object_id)); + if (AbortObject(object_id, client) != 1) { + ARROW_LOG(ERROR) << "[PlasmaStore] (on AbortRequest) To abort an object, the " + << "only client currently using it must be the creator."; } - - HANDLE_SIGPIPE(SendCreateAndSealBatchReply(client->fd, error_code), client->fd); - - // if OK, seal all the objects, - // if error, abort the previous i objects immediately - if (error_code == PlasmaError::OK) { - for (i = 0; i < object_ids.size(); i++) { - auto entry = GetObjectTableEntry(&store_info_, object_ids[i]); - ARROW_CHECK(entry != nullptr); - // Write the inlined data and metadata into the allocated object. - std::memcpy(entry->pointer, data[i].data(), data[i].size()); - std::memcpy(entry->pointer + data[i].size(), metadata[i].data(), - metadata[i].size()); - } - - SealObjects(object_ids, digests); - // Remove the client from the object's array of clients because the - // object is not being used by any client. The client was added to the - // object's array of clients in CreateObject. This is analogous to the - // Release call that happens in the client's Seal method. - for (i = 0; i < object_ids.size(); i++) { - auto entry = GetObjectTableEntry(&store_info_, object_ids[i]); - ARROW_CHECK(RemoveFromClientObjectIds(object_ids[i], entry, client) == 1); - } - } else { - for (size_t j = 0; j < i; j++) { - AbortObject(object_ids[j], client); - } - } - } break; - case fb::MessageType::PlasmaAbortRequest: { - RETURN_NOT_OK(ReadAbortRequest(input, input_size, &object_id)); - ARROW_CHECK(AbortObject(object_id, client) == 1) << "To abort an object, the only " - "client currently using it " - "must be the creator."; - HANDLE_SIGPIPE(SendAbortReply(client->fd, object_id), client->fd); + RETURN_NOT_OK(SendAbortReply(client, object_id)); } break; - case fb::MessageType::PlasmaGetRequest: { - std::vector object_ids_to_get; + case MessageType::PlasmaGetRequest: { + std::vector object_ids; int64_t timeout_ms; - RETURN_NOT_OK(ReadGetRequest(input, input_size, object_ids_to_get, &timeout_ms)); - ProcessGetRequest(client, object_ids_to_get, timeout_ms); + RETURN_NOT_OK(ReadGetRequest(message_data, message_size, object_ids, &timeout_ms)); + RETURN_NOT_OK(ProcessGetRequest(client, object_ids, timeout_ms)); } break; - case fb::MessageType::PlasmaReleaseRequest: { - RETURN_NOT_OK(ReadReleaseRequest(input, input_size, &object_id)); + case MessageType::PlasmaReleaseRequest: { + RETURN_NOT_OK(ReadReleaseRequest(message_data, message_size, &object_id)); ReleaseObject(object_id, client); } break; - case fb::MessageType::PlasmaDeleteRequest: { + case MessageType::PlasmaDeleteRequest: { std::vector object_ids; std::vector error_codes; - RETURN_NOT_OK(ReadDeleteRequest(input, input_size, &object_ids)); + RETURN_NOT_OK(ReadDeleteRequest(message_data, message_size, &object_ids)); error_codes.reserve(object_ids.size()); for (auto& object_id : object_ids) { error_codes.push_back(DeleteObject(object_id)); } - HANDLE_SIGPIPE(SendDeleteReply(client->fd, object_ids, error_codes), client->fd); + RETURN_NOT_OK(SendDeleteReply(client, object_ids, error_codes)); } break; - case fb::MessageType::PlasmaContainsRequest: { - RETURN_NOT_OK(ReadContainsRequest(input, input_size, &object_id)); - if (ContainsObject(object_id) == ObjectStatus::OBJECT_FOUND) { - HANDLE_SIGPIPE(SendContainsReply(client->fd, object_id, 1), client->fd); - } else { - HANDLE_SIGPIPE(SendContainsReply(client->fd, object_id, 0), client->fd); - } + case MessageType::PlasmaContainsRequest: { + RETURN_NOT_OK(ReadContainsRequest(message_data, message_size, &object_id)); + auto has_object = (ContainsObject(object_id) == ObjectStatus::OBJECT_FOUND); + RETURN_NOT_OK(SendContainsReply(client, object_id, has_object)); } break; - case fb::MessageType::PlasmaListRequest: { - RETURN_NOT_OK(ReadListRequest(input, input_size)); - HANDLE_SIGPIPE(SendListReply(client->fd, store_info_.objects), client->fd); + case MessageType::PlasmaListRequest: { + RETURN_NOT_OK(ReadListRequest(message_data, message_size)); + RETURN_NOT_OK(SendListReply(client, store_info_.objects)); } break; - case fb::MessageType::PlasmaSealRequest: { + case MessageType::PlasmaSealRequest: { std::string digest; - RETURN_NOT_OK(ReadSealRequest(input, input_size, &object_id, &digest)); + digest.reserve(kDigestSize); + RETURN_NOT_OK(ReadSealRequest(message_data, message_size, &object_id, &digest[0])); SealObjects({object_id}, {digest}); } break; - case fb::MessageType::PlasmaEvictRequest: { + case MessageType::PlasmaEvictRequest: { // This code path should only be used for testing. int64_t num_bytes; - RETURN_NOT_OK(ReadEvictRequest(input, input_size, &num_bytes)); + RETURN_NOT_OK(ReadEvictRequest(message_data, message_size, &num_bytes)); std::vector objects_to_evict; int64_t num_bytes_evicted = eviction_policy_.ChooseObjectsToEvict(num_bytes, &objects_to_evict); EvictObjects(objects_to_evict); - HANDLE_SIGPIPE(SendEvictReply(client->fd, num_bytes_evicted), client->fd); + RETURN_NOT_OK(SendEvictReply(client, num_bytes_evicted)); } break; - case fb::MessageType::PlasmaSubscribeRequest: + case MessageType::PlasmaSubscribeRequest: SubscribeToUpdates(client); break; - case fb::MessageType::PlasmaConnectRequest: { - HANDLE_SIGPIPE(SendConnectReply(client->fd, PlasmaAllocator::GetFootprintLimit()), - client->fd); - } break; - case fb::MessageType::PlasmaDisconnectClient: - ARROW_LOG(DEBUG) << "Disconnecting client on fd " << client->fd; - DisconnectClient(client->fd); - break; - case fb::MessageType::PlasmaSetOptionsRequest: { - std::string client_name; - int64_t output_memory_quota; - RETURN_NOT_OK( - ReadSetOptionsRequest(input, input_size, &client_name, &output_memory_quota)); - client->name = client_name; - bool success = eviction_policy_.SetClientQuota(client, output_memory_quota); - HANDLE_SIGPIPE(SendSetOptionsReply(client->fd, success ? PlasmaError::OK - : PlasmaError::OutOfMemory), - client->fd); - } break; - case fb::MessageType::PlasmaGetDebugStringRequest: { - HANDLE_SIGPIPE(SendGetDebugStringReply(client->fd, eviction_policy_.DebugString()), - client->fd); + case MessageType::PlasmaConnectRequest: { + RETURN_NOT_OK(SendConnectReply(client, PlasmaAllocator::GetFootprintLimit())); } break; + case MessageType::PlasmaDisconnectClient: + ARROW_LOG(DEBUG) << "Disconnecting client on fd " << client->GetNativeHandle(); + ProcessDisconnectClient(client); + return Status::OK(); // Stop listening for more messages. default: // This code should be unreachable. ARROW_CHECK(0); } + // Listen for more messages. + client->ProcessMessages(); return Status::OK(); } @@ -1120,11 +958,16 @@ class PlasmaStoreRunner { public: PlasmaStoreRunner() {} - void Start(char* socket_name, std::string directory, bool hugepages_enabled, - std::shared_ptr external_store) { + void Start(const std::string& stream_name, std::string directory, + bool hugepages_enabled, std::shared_ptr external_store) { + signal_set_.async_wait([this](std::error_code ec, int signal) { + if (signal == SIGTERM) { + ARROW_LOG(INFO) << "SIGTERM Signal received, closing Plasma Server..."; + Stop(); + } + }); // Create the event loop. - loop_.reset(new EventLoop); - store_.reset(new PlasmaStore(loop_.get(), directory, hugepages_enabled, socket_name, + store_.reset(new PlasmaStore(io_context_, directory, hugepages_enabled, stream_name, external_store)); plasma_config = store_->GetPlasmaStoreInfo(); @@ -1132,7 +975,7 @@ class PlasmaStoreRunner { // large amount of space up front. According to the documentation, // dlmalloc might need up to 128*sizeof(size_t) bytes for internal // bookkeeping. - void* pointer = plasma::PlasmaAllocator::Memalign( + void* pointer = PlasmaAllocator::Memalign( kBlockSize, PlasmaAllocator::GetFootprintLimit() - 256 * sizeof(size_t)); ARROW_CHECK(pointer != nullptr); // This will unmap the file, but the next one created will be as large @@ -1140,57 +983,33 @@ class PlasmaStoreRunner { plasma::PlasmaAllocator::Free( pointer, PlasmaAllocator::GetFootprintLimit() - 256 * sizeof(size_t)); - int socket = BindIpcSock(socket_name, true); - // TODO(pcm): Check return value. - ARROW_CHECK(socket >= 0); - - loop_->AddFileEvent(socket, kEventLoopRead, [this, socket](int events) { - this->store_->ConnectClient(socket); - }); - loop_->Start(); + io_context_.run(); } - void Stop() { loop_->Stop(); } + void Stop() { io_context_.stop(); } void Shutdown() { - loop_->Shutdown(); - loop_ = nullptr; + io_context_.stop(); store_ = nullptr; } private: - std::unique_ptr loop_; + asio::io_context io_context_; + asio::signal_set signal_set_{io_context_, SIGTERM}; + // Ignore SIGPIPE signals. If we don't do this, then when we attempt to write + // to a client that has already died, the store could die. + asio::signal_set signal_ign_{io_context_, SIGPIPE}; std::unique_ptr store_; }; static std::unique_ptr g_runner = nullptr; -void HandleSignal(int signal) { - if (signal == SIGTERM) { - ARROW_LOG(INFO) << "SIGTERM Signal received, closing Plasma Server..."; - if (g_runner != nullptr) { - g_runner->Stop(); - } - } -} - -void StartServer(char* socket_name, std::string plasma_directory, bool hugepages_enabled, - std::shared_ptr external_store) { - // Ignore SIGPIPE signals. If we don't do this, then when we attempt to write - // to a client that has already died, the store could die. - signal(SIGPIPE, SIG_IGN); - - g_runner.reset(new PlasmaStoreRunner()); - signal(SIGTERM, HandleSignal); - g_runner->Start(socket_name, plasma_directory, hugepages_enabled, external_store); -} - } // namespace plasma int main(int argc, char* argv[]) { ArrowLog::StartArrowLog(argv[0], ArrowLogLevel::ARROW_INFO); ArrowLog::InstallFailureSignalHandler(); - char* socket_name = nullptr; + char* stream_name = nullptr; // Directory where plasma memory mapped files are stored. std::string plasma_directory; std::string external_store_endpoint; @@ -1209,7 +1028,7 @@ int main(int argc, char* argv[]) { hugepages_enabled = true; break; case 's': - socket_name = optarg; + stream_name = optarg; break; case 'm': { char extra; @@ -1227,7 +1046,7 @@ int main(int argc, char* argv[]) { } } // Sanity check command line options. - if (!socket_name) { + if (!stream_name) { ARROW_LOG(FATAL) << "please specify socket for incoming connections with -s switch"; } if (system_memory == -1) { @@ -1288,8 +1107,10 @@ int main(int argc, char* argv[]) { ARROW_LOG(DEBUG) << "connecting to external store..."; ARROW_CHECK_OK(external_store->Connect(external_store_endpoint)); } - ARROW_LOG(DEBUG) << "starting server listening on " << socket_name; - plasma::StartServer(socket_name, plasma_directory, hugepages_enabled, external_store); + ARROW_LOG(DEBUG) << "starting server listening on " << stream_name; + plasma::g_runner.reset(new plasma::PlasmaStoreRunner()); + plasma::g_runner->Start(stream_name, plasma_directory, hugepages_enabled, + external_store); plasma::g_runner->Shutdown(); plasma::g_runner = nullptr; diff --git a/cpp/src/plasma/store.h b/cpp/src/plasma/store.h index b69da29992a..206458a9cfe 100644 --- a/cpp/src/plasma/store.h +++ b/cpp/src/plasma/store.h @@ -18,7 +18,6 @@ #ifndef PLASMA_STORE_H #define PLASMA_STORE_H -#include #include #include #include @@ -26,8 +25,8 @@ #include #include "plasma/common.h" -#include "plasma/events.h" #include "plasma/external_store.h" +#include "plasma/io/connection.h" #include "plasma/plasma.h" #include "plasma/protocol.h" #include "plasma/quota_aware_policy.h" @@ -39,12 +38,11 @@ class Status; namespace plasma { namespace flatbuf { -struct ObjectInfoT; enum class PlasmaError; } // namespace flatbuf -using flatbuf::ObjectInfoT; using flatbuf::PlasmaError; +using io::ClientConnection; struct GetRequest; @@ -56,11 +54,8 @@ struct NotificationQueue { class PlasmaStore { public: - using NotificationMap = std::unordered_map; - - // TODO: PascalCase PlasmaStore methods. - PlasmaStore(EventLoop* loop, std::string directory, bool hugepages_enabled, - const std::string& socket_name, + PlasmaStore(asio::io_context& main_context, std::string directory, + bool hugepages_enabled, const std::string& stream_name, std::shared_ptr external_store); ~PlasmaStore(); @@ -71,17 +66,17 @@ class PlasmaStore { /// Create a new object. The client must do a call to release_object to tell /// the store when it is done with the object. /// - /// @param object_id Object ID of the object to be created. - /// @param data_size Size in bytes of the object to be created. - /// @param metadata_size Size in bytes of the object metadata. - /// @param device_num The number of the device where the object is being + /// \param object_id Object ID of the object to be created. + /// \param data_size Size in bytes of the object to be created. + /// \param metadata_size Size in bytes of the object metadata. + /// \param device_num The number of the device where the object is being /// created. /// device_num = 0 corresponds to the host, /// device_num = 1 corresponds to GPU0, /// device_num = 2 corresponds to GPU1, etc. - /// @param client The client that created the object. - /// @param result The object that has been created. - /// @return One of the following error codes: + /// \param client The client that created the object. + /// \param result The object that has been created. + /// \return One of the following error codes: /// - PlasmaError::OK, if the object was created successfully. /// - PlasmaError::ObjectExists, if an object with this ID is already /// present in the store. In this case, the client should not call @@ -90,22 +85,24 @@ class PlasmaStore { /// cannot create the object. In this case, the client should not call /// plasma_release. PlasmaError CreateObject(const ObjectID& object_id, int64_t data_size, - int64_t metadata_size, int device_num, Client* client, + int64_t metadata_size, int device_num, + const std::shared_ptr& client, PlasmaObject* result); /// Abort a created but unsealed object. If the client is not the /// creator, then the abort will fail. /// - /// @param object_id Object ID of the object to be aborted. - /// @param client The client who created the object. If this does not + /// \param object_id Object ID of the object to be aborted. + /// \param client The client who created the object. If this does not /// match the creator of the object, then the abort will fail. - /// @return 1 if the abort succeeds, else 0. - int AbortObject(const ObjectID& object_id, Client* client); + /// \return 1 if the abort succeeds, else 0. + int AbortObject(const ObjectID& object_id, + const std::shared_ptr& client); /// Delete an specific object by object_id that have been created in the hash table. /// - /// @param object_id Object ID of the object to be deleted. - /// @return One of the following error codes: + /// \param object_id Object ID of the object to be deleted. + /// \return One of the following error codes: /// - PlasmaError::OK, if the object was delete successfully. /// - PlasmaError::ObjectNonexistent, if ths object isn't existed. /// - PlasmaError::ObjectInUse, if the object is in use. @@ -113,7 +110,7 @@ class PlasmaStore { /// Evict objects returned by the eviction policy. /// - /// @param object_ids Object IDs of the objects to be evicted. + /// \param object_ids Object IDs of the objects to be evicted. void EvictObjects(const std::vector& object_ids); /// Process a get request from a client. This method assumes that we will @@ -124,14 +121,13 @@ class PlasmaStore { /// For each object, the client must do a call to release_object to tell the /// store when it is done with the object. /// - /// @param client The client making this request. - /// @param object_ids Object IDs of the objects to be gotten. - /// @param timeout_ms The timeout for the get request in milliseconds. - void ProcessGetRequest(Client* client, const std::vector& object_ids, - int64_t timeout_ms); - - /// Seal a vector of objects. The objects are now immutable and can be accessed with - /// get. + /// \param client The client making this request. + /// \param object_ids Object IDs of the objects to be gotten. + /// \param timeout_ms The timeout for the get request in milliseconds. + Status ProcessGetRequest(const std::shared_ptr& client, + const std::vector& object_ids, int64_t timeout_ms); + + /// Seal an object. The object is now immutable and can be accessed with get. /// /// @param object_ids The vector of Object IDs of the objects to be sealed. /// @param digests The vector of digests of the objects. This is used to tell if two @@ -141,67 +137,63 @@ class PlasmaStore { /// Check if the plasma store contains an object: /// - /// @param object_id Object ID that will be checked. - /// @return OBJECT_FOUND if the object is in the store, OBJECT_NOT_FOUND if + /// \param object_id Object ID that will be checked. + /// \return OBJECT_FOUND if the object is in the store, OBJECT_NOT_FOUND if /// not ObjectStatus ContainsObject(const ObjectID& object_id); /// Record the fact that a particular client is no longer using an object. /// - /// @param object_id The object ID of the object that is being released. - /// @param client The client making this request. - void ReleaseObject(const ObjectID& object_id, Client* client); + /// \param object_id The object ID of the object that is being released. + /// \param client The client making this request. + void ReleaseObject(const ObjectID& object_id, + const std::shared_ptr& client); /// Subscribe a file descriptor to updates about new sealed objects. /// - /// @param client The client making this request. - void SubscribeToUpdates(Client* client); - - /// Connect a new client to the PlasmaStore. - /// - /// @param listener_sock The socket that is listening to incoming connections. - void ConnectClient(int listener_sock); - - /// Disconnect a client from the PlasmaStore. - /// - /// @param client_fd The client file descriptor that is disconnected. - void DisconnectClient(int client_fd); - - NotificationMap::iterator SendNotifications(NotificationMap::iterator it); - - arrow::Status ProcessMessage(Client* client); + /// \param client The client making this request. + void SubscribeToUpdates(const std::shared_ptr& client); private: - void PushNotification(ObjectInfoT* object_notification); + // Inform all subscribers that a new object has been sealed. + void PushObjectReadyNotification(const ObjectID& object_id, + const ObjectTableEntry& entry); + // Inform all subscribers that an object has evicted. void PushNotifications(std::vector& object_notifications); void PushNotification(ObjectInfoT* object_notification, int client_fd); void AddToClientObjectIds(const ObjectID& object_id, ObjectTableEntry* entry, - Client* client); + const std::shared_ptr& client); /// Remove a GetRequest and clean up the relevant data structures. /// - /// @param get_request The GetRequest to remove. + /// \param get_request The GetRequest to remove. void RemoveGetRequest(GetRequest* get_request); /// Remove all of the GetRequests for a given client. /// - /// @param client The client whose GetRequests should be removed. - void RemoveGetRequestsForClient(Client* client); + /// \param client The client whose GetRequests should be removed. + void RemoveGetRequestsForClient(const std::shared_ptr& client); + + /// Release all resources used by the client. + /// + /// \param client The client whose resources should be released. + void ReleaseClientResources(const std::shared_ptr& client); void ReturnFromGet(GetRequest* get_req); void UpdateObjectGetRequests(const ObjectID& object_id); - int RemoveFromClientObjectIds(const ObjectID& object_id, ObjectTableEntry* entry, - Client* client); - void EraseFromObjectTable(const ObjectID& object_id); - uint8_t* AllocateMemory(size_t size, int* fd, int64_t* map_size, ptrdiff_t* offset, - Client* client, bool is_create); + uint8_t* AllocateMemory(size_t size, int* fd, int64_t* map_size, ptrdiff_t* offset); + + void IncreaseObjectRefCount(const ObjectID& object_id, ObjectTableEntry* entry); + + void DecreaseObjectRefCount(const ObjectID& object_id, ObjectTableEntry* entry); + #ifdef PLASMA_CUDA Status AllocateCudaMemory(int device_num, int64_t size, uint8_t** out_pointer, std::shared_ptr* out_ipc_handle); @@ -209,27 +201,32 @@ class PlasmaStore { Status FreeCudaMemory(int device_num, int64_t size, uint8_t* out_pointer); #endif - /// Event loop of the plasma store. - EventLoop* loop_; + /// Accept a client connection. + void DoAccept(); + /// Handle an accepted client connection. + void HandleAccept(const error_code& error); + + Status ProcessClientMessage(const std::shared_ptr& client, + int64_t message_type, int64_t message_size, + const uint8_t* message_data); + + /// Disconnect a client from the PlasmaStore. + /// + /// \param client The client that is disconnected. + void ProcessDisconnectClient(const std::shared_ptr& client); + /// The plasma store information, including the object tables, that is exposed /// to the eviction policy. PlasmaStoreInfo store_info_; /// The state that is managed by the eviction policy. QuotaAwarePolicy eviction_policy_; - /// Input buffer. This is allocated only once to avoid mallocs for every - /// call to process_message. - std::vector input_buffer_; /// A hash table mapping object IDs to a vector of the get requests that are /// waiting for the object to arrive. std::unordered_map> object_get_requests_; - /// The pending notifications that have not been sent to subscribers because - /// the socket send buffers were full. This is a hash table from client file - /// descriptor to an array of object_ids to send to that client. - /// TODO(pcm): Consider putting this into the Client data structure and - /// reorganize the code slightly. - NotificationMap pending_notifications_; - std::unordered_map> connected_clients_; + std::unordered_set> notification_clients_; + + std::unordered_set> connected_clients_; std::unordered_set deletion_cache_; @@ -239,6 +236,13 @@ class PlasmaStore { #ifdef PLASMA_CUDA arrow::cuda::CudaDeviceManager* manager_; #endif + asio::io_context& io_context_; + /// The name of the stream this store server listens on. + std::string stream_name_; + /// An acceptor for new clients. + io::PlasmaAcceptor acceptor_; + /// The stream to listen on for new clients. + io::PlasmaStream stream_; }; } // namespace plasma diff --git a/cpp/src/plasma/test/client_tests.cc b/cpp/src/plasma/test/client_tests.cc index af73747e15b..01467f29487 100644 --- a/cpp/src/plasma/test/client_tests.cc +++ b/cpp/src/plasma/test/client_tests.cc @@ -15,26 +15,21 @@ // specific language governing permissions and limitations // under the License. -#include -#include -#include -#include -#include -#include - +#include #include +#include #include #include #include "arrow/testing/gtest_util.h" -#include "arrow/util/io_util.h" +#include "arrow/util/io-util.h" #include "plasma/client.h" #include "plasma/common.h" #include "plasma/plasma.h" #include "plasma/protocol.h" -#include "plasma/test_util.h" +#include "plasma/test-util.h" namespace plasma { @@ -55,22 +50,22 @@ class TestPlasmaStore : public ::testing::Test { // stdout of the object store. Consider changing that. void SetUp() { - ARROW_CHECK_OK(TemporaryDir::Make("cli-test-", &temp_dir_)); + ASSERT_OK(TemporaryDir::Make("cli-test-", &temp_dir_)); store_socket_name_ = temp_dir_->path().ToString() + "store"; std::string plasma_directory = test_executable.substr(0, test_executable.find_last_of("/")); std::string plasma_command = - plasma_directory + "/plasma-store-server -m 10000000 -s " + store_socket_name_ + + plasma_directory + "/plasma_store_server -m 10000000 -s " + store_socket_name_ + " 1> /dev/null 2> /dev/null & " + "echo $! > " + store_socket_name_ + ".pid"; PLASMA_CHECK_SYSTEM(system(plasma_command.c_str())); - ARROW_CHECK_OK(client_.Connect(store_socket_name_, "")); - ARROW_CHECK_OK(client2_.Connect(store_socket_name_, "")); + ASSERT_OK(client_.Connect(store_socket_name_, "")); + ASSERT_OK(client2_.Connect(store_socket_name_, "")); } virtual void TearDown() { - ARROW_CHECK_OK(client_.Disconnect()); - ARROW_CHECK_OK(client2_.Disconnect()); + ASSERT_OK(client_.Disconnect()); + ASSERT_OK(client2_.Disconnect()); // Kill plasma_store process that we started #ifdef COVERAGE_BUILD // Ask plasma_store to exit gracefully and give it time to write out @@ -89,14 +84,14 @@ class TestPlasmaStore : public ::testing::Test { const std::vector& metadata, const std::vector& data, bool release = true) { std::shared_ptr data_buffer; - ARROW_CHECK_OK(client.Create(object_id, data.size(), &metadata[0], metadata.size(), + ASSERT_OK(client.Create(object_id, data.size(), &metadata[0], metadata.size(), &data_buffer)); for (size_t i = 0; i < data.size(); i++) { data_buffer->mutable_data()[i] = data[i]; } - ARROW_CHECK_OK(client.Seal(object_id)); + ASSERT_OK(client.Seal(object_id)); if (release) { - ARROW_CHECK_OK(client.Release(object_id)); + ASSERT_OK(client.Release(object_id)); } } @@ -110,8 +105,8 @@ class TestPlasmaStore : public ::testing::Test { TEST_F(TestPlasmaStore, NewSubscriberTest) { PlasmaClient local_client, local_client2; - ARROW_CHECK_OK(local_client.Connect(store_socket_name_, "")); - ARROW_CHECK_OK(local_client2.Connect(store_socket_name_, "")); + ASSERT_OK(local_client.Connect(store_socket_name_, "")); + ASSERT_OK(local_client2.Connect(store_socket_name_, "")); ObjectID object_id = random_object_id(); @@ -121,74 +116,34 @@ TEST_F(TestPlasmaStore, NewSubscriberTest) { uint8_t metadata[] = {5}; int64_t metadata_size = sizeof(metadata); std::shared_ptr data; - ARROW_CHECK_OK( + ASSERT_OK( local_client.Create(object_id, data_size, metadata, metadata_size, &data)); - ARROW_CHECK_OK(local_client.Seal(object_id)); + ASSERT_OK(local_client.Seal(object_id)); // Test that new subscriber client2 can receive notifications about existing objects. - int fd = -1; - ARROW_CHECK_OK(local_client2.Subscribe(&fd)); - ASSERT_GT(fd, 0); + ASSERT_OK(local_client2.Subscribe()); ObjectID object_id2 = random_object_id(); int64_t data_size2 = 0; int64_t metadata_size2 = 0; - ARROW_CHECK_OK( - local_client2.GetNotification(fd, &object_id2, &data_size2, &metadata_size2)); + ASSERT_OK( + local_client2.GetNotification(&object_id2, &data_size2, &metadata_size2)); ASSERT_EQ(object_id, object_id2); ASSERT_EQ(data_size, data_size2); ASSERT_EQ(metadata_size, metadata_size2); // Delete the object. - ARROW_CHECK_OK(local_client.Release(object_id)); - ARROW_CHECK_OK(local_client.Delete(object_id)); + ASSERT_OK(local_client.Release(object_id)); + ASSERT_OK(local_client.Delete(object_id)); - ARROW_CHECK_OK( - local_client2.GetNotification(fd, &object_id2, &data_size2, &metadata_size2)); + ASSERT_OK( + local_client2.GetNotification(&object_id2, &data_size2, &metadata_size2)); ASSERT_EQ(object_id, object_id2); ASSERT_EQ(-1, data_size2); ASSERT_EQ(-1, metadata_size2); - ARROW_CHECK_OK(local_client2.Disconnect()); - ARROW_CHECK_OK(local_client.Disconnect()); -} - -TEST_F(TestPlasmaStore, BatchNotificationTest) { - PlasmaClient local_client, local_client2; - - ARROW_CHECK_OK(local_client.Connect(store_socket_name_, "")); - ARROW_CHECK_OK(local_client2.Connect(store_socket_name_, "")); - - int fd = -1; - ARROW_CHECK_OK(local_client2.Subscribe(&fd)); - ASSERT_GT(fd, 0); - - ObjectID object_id1 = random_object_id(); - ObjectID object_id2 = random_object_id(); - - std::vector object_ids = {object_id1, object_id2}; - - std::vector data = {"hello", "world!"}; - std::vector metadata = {"1", "23"}; - ARROW_CHECK_OK(local_client.CreateAndSealBatch(object_ids, data, metadata)); - - ObjectID object_id = random_object_id(); - int64_t data_size = 0; - int64_t metadata_size = 0; - ARROW_CHECK_OK( - local_client2.GetNotification(fd, &object_id, &data_size, &metadata_size)); - ASSERT_EQ(object_id, object_id1); - ASSERT_EQ(data_size, 5); - ASSERT_EQ(metadata_size, 1); - - ARROW_CHECK_OK( - local_client2.GetNotification(fd, &object_id, &data_size, &metadata_size)); - ASSERT_EQ(object_id, object_id2); - ASSERT_EQ(data_size, 6); - ASSERT_EQ(metadata_size, 2); - - ARROW_CHECK_OK(local_client2.Disconnect()); - ARROW_CHECK_OK(local_client.Disconnect()); + ASSERT_OK(local_client2.Disconnect()); + ASSERT_OK(local_client.Disconnect()); } TEST_F(TestPlasmaStore, SealErrorsTest) { @@ -204,255 +159,7 @@ TEST_F(TestPlasmaStore, SealErrorsTest) { // Trying to seal it again. result = client_.Seal(object_id); ASSERT_TRUE(IsPlasmaObjectAlreadySealed(result)); - ARROW_CHECK_OK(client_.Release(object_id)); -} - -TEST_F(TestPlasmaStore, SetQuotaBasicTest) { - bool has_object = false; - ObjectID id1 = random_object_id(); - ObjectID id2 = random_object_id(); - - ARROW_CHECK_OK(client_.SetClientOptions("client1", 5 * 1024 * 1024)); - std::vector big_data(3 * 1024 * 1024, 0); - - // First object fits - CreateObject(client_, id1, {42}, big_data, true); - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_TRUE(has_object); - - // Evicts first object - CreateObject(client_, id2, {42}, big_data, true); - ARROW_CHECK_OK(client_.Contains(id2, &has_object)); - ASSERT_TRUE(has_object); - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_FALSE(has_object); - - // Too big to fit in quota at all - std::shared_ptr data_buffer; - ASSERT_FALSE( - client_.Create(random_object_id(), 7 * 1024 * 1024, {}, 0, &data_buffer).ok()); - ASSERT_TRUE( - client_.Create(random_object_id(), 4 * 1024 * 1024, {}, 0, &data_buffer).ok()); -} - -TEST_F(TestPlasmaStore, SetQuotaProvidesIsolationFromOtherClients) { - bool has_object = false; - ObjectID id1 = random_object_id(); - ObjectID id2 = random_object_id(); - - std::vector big_data(3 * 1024 * 1024, 0); - - // First object, created without quota - CreateObject(client_, id1, {42}, big_data, true); - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_TRUE(has_object); - - // Second client creates a bunch of objects - for (int i = 0; i < 10; i++) { - CreateObject(client2_, random_object_id(), {42}, big_data, true); - } - - // First client's object is evicted - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_FALSE(has_object); - - // Try again with quota enabled - ARROW_CHECK_OK(client_.SetClientOptions("client1", 5 * 1024 * 1024)); - CreateObject(client_, id2, {42}, big_data, true); - ARROW_CHECK_OK(client_.Contains(id2, &has_object)); - ASSERT_TRUE(has_object); - - // Second client creates a bunch of objects - for (int i = 0; i < 10; i++) { - CreateObject(client2_, random_object_id(), {42}, big_data, true); - } - - // First client's object is not evicted - ARROW_CHECK_OK(client_.Contains(id2, &has_object)); - ASSERT_TRUE(has_object); -} - -TEST_F(TestPlasmaStore, SetQuotaProtectsOtherClients) { - bool has_object = false; - ObjectID id1 = random_object_id(); - - std::vector big_data(3 * 1024 * 1024, 0); - - // First client has no quota - CreateObject(client_, id1, {42}, big_data, true); - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_TRUE(has_object); - - // Second client creates a bunch of objects under a quota - ARROW_CHECK_OK(client2_.SetClientOptions("client2", 5 * 1024 * 1024)); - for (int i = 0; i < 10; i++) { - CreateObject(client2_, random_object_id(), {42}, big_data, true); - } - - // First client's object is NOT evicted - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_TRUE(has_object); -} - -TEST_F(TestPlasmaStore, SetQuotaCannotExceedSeventyPercentMemory) { - ASSERT_FALSE(client_.SetClientOptions("client1", 8 * 1024 * 1024).ok()); - ASSERT_TRUE(client_.SetClientOptions("client1", 5 * 1024 * 1024).ok()); - // cannot set quota twice - ASSERT_FALSE(client_.SetClientOptions("client1", 5 * 1024 * 1024).ok()); - // cannot exceed 70% summed - ASSERT_FALSE(client2_.SetClientOptions("client2", 3 * 1024 * 1024).ok()); - ASSERT_TRUE(client2_.SetClientOptions("client2", 1 * 1024 * 1024).ok()); -} - -TEST_F(TestPlasmaStore, SetQuotaDemotesPinnedObjectsToGlobalLRU) { - bool has_object = false; - ASSERT_TRUE(client_.SetClientOptions("client1", 5 * 1024 * 1024).ok()); - - ObjectID id1 = random_object_id(); - ObjectID id2 = random_object_id(); - std::vector big_data(3 * 1024 * 1024, 0); - - // Quota is not enough to fit both id1 and id2, but global LRU is - CreateObject(client_, id1, {42}, big_data, false); - CreateObject(client_, id2, {42}, big_data, false); - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_TRUE(has_object); - ARROW_CHECK_OK(client_.Contains(id2, &has_object)); - ASSERT_TRUE(has_object); - - // Release both objects. Now id1 is in global LRU and id2 is in quota - ARROW_CHECK_OK(client_.Release(id1)); - ARROW_CHECK_OK(client_.Release(id2)); - - // This flushes id1 from the object store - for (int i = 0; i < 10; i++) { - CreateObject(client2_, random_object_id(), {42}, big_data, true); - } - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_FALSE(has_object); - ARROW_CHECK_OK(client_.Contains(id2, &has_object)); - ASSERT_TRUE(has_object); -} - -TEST_F(TestPlasmaStore, SetQuotaDemoteDisconnectToGlobalLRU) { - bool has_object = false; - PlasmaClient local_client; - ARROW_CHECK_OK(local_client.Connect(store_socket_name_, "")); - ARROW_CHECK_OK(local_client.SetClientOptions("local", 5 * 1024 * 1024)); - - ObjectID id1 = random_object_id(); - std::vector big_data(3 * 1024 * 1024, 0); - - // First object fits - CreateObject(local_client, id1, {42}, big_data, true); - for (int i = 0; i < 10; i++) { - CreateObject(client_, random_object_id(), {42}, big_data, true); - } - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_TRUE(has_object); - - // Object is still present after disconnect - ARROW_CHECK_OK(local_client.Disconnect()); - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_TRUE(has_object); - - // But is eligible for global LRU - for (int i = 0; i < 10; i++) { - CreateObject(client_, random_object_id(), {42}, big_data, true); - } - ARROW_CHECK_OK(client_.Contains(id1, &has_object)); - ASSERT_FALSE(has_object); -} - -TEST_F(TestPlasmaStore, SetQuotaCleanupObjectMetadata) { - PlasmaClient local_client; - ARROW_CHECK_OK(local_client.Connect(store_socket_name_, "")); - ARROW_CHECK_OK(local_client.SetClientOptions("local", 5 * 1024 * 1024)); - - ObjectID id0 = random_object_id(); - ObjectID id1 = random_object_id(); - ObjectID id2 = random_object_id(); - ObjectID id3 = random_object_id(); - std::vector big_data(3 * 1024 * 1024, 0); - std::vector small_data(1 * 1024 * 1024, 0); - CreateObject(local_client, id0, {42}, small_data, false); - CreateObject(local_client, id1, {42}, big_data, true); - CreateObject(local_client, id2, {42}, big_data, - true); // spills id0 to global, evicts id1 - CreateObject(local_client, id3, {42}, small_data, false); - - ASSERT_TRUE(client_.DebugString().find("num clients with quota: 1") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("quota map size: 2") != std::string::npos); - ASSERT_TRUE(client_.DebugString().find("pinned quota map size: 1") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(global lru) num objects: 0") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(local) num objects: 2") != std::string::npos); - - // release id0 - ARROW_CHECK_OK(local_client.Release(id0)); - ASSERT_TRUE(client_.DebugString().find("(global lru) num objects: 1") != - std::string::npos); - - // delete everything - ARROW_CHECK_OK(local_client.Delete(id0)); - ARROW_CHECK_OK(local_client.Delete(id2)); - ARROW_CHECK_OK(local_client.Delete(id3)); - ARROW_CHECK_OK(local_client.Release(id3)); - ASSERT_TRUE(client_.DebugString().find("quota map size: 0") != std::string::npos); - ASSERT_TRUE(client_.DebugString().find("pinned quota map size: 0") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(global lru) num objects: 0") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(local) num objects: 0") != std::string::npos); - - ARROW_CHECK_OK(local_client.Disconnect()); - int tries = 10; // wait for disconnect to complete - while (tries > 0 && - client_.DebugString().find("num clients with quota: 0") == std::string::npos) { - std::this_thread::sleep_for(std::chrono::milliseconds(200)); - tries -= 1; - } - ASSERT_TRUE(client_.DebugString().find("num clients with quota: 0") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(global lru) capacity: 10000000") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(global lru) used: 0%") != std::string::npos); -} - -TEST_F(TestPlasmaStore, SetQuotaCleanupClientDisconnect) { - PlasmaClient local_client; - ARROW_CHECK_OK(local_client.Connect(store_socket_name_, "")); - ARROW_CHECK_OK(local_client.SetClientOptions("local", 5 * 1024 * 1024)); - - ObjectID id1 = random_object_id(); - ObjectID id2 = random_object_id(); - ObjectID id3 = random_object_id(); - std::vector big_data(3 * 1024 * 1024, 0); - std::vector small_data(1 * 1024 * 1024, 0); - CreateObject(local_client, id1, {42}, big_data, true); - CreateObject(local_client, id2, {42}, big_data, true); - CreateObject(local_client, id3, {42}, small_data, false); - - ARROW_CHECK_OK(local_client.Disconnect()); - int tries = 10; // wait for disconnect to complete - while (tries > 0 && - client_.DebugString().find("num clients with quota: 0") == std::string::npos) { - std::this_thread::sleep_for(std::chrono::milliseconds(200)); - tries -= 1; - } - ASSERT_TRUE(client_.DebugString().find("num clients with quota: 0") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("quota map size: 0") != std::string::npos); - ASSERT_TRUE(client_.DebugString().find("pinned quota map size: 0") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(global lru) num objects: 2") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(global lru) capacity: 10000000") != - std::string::npos); - ASSERT_TRUE(client_.DebugString().find("(global lru) used: 41.9431%") != - std::string::npos); + ASSERT_OK(client_.Release(object_id)); } TEST_F(TestPlasmaStore, DeleteTest) { @@ -460,7 +167,7 @@ TEST_F(TestPlasmaStore, DeleteTest) { // Test for deleting non-existance object. Status result = client_.Delete(object_id); - ARROW_CHECK_OK(result); + ASSERT_OK(result); // Test for the object being in local Plasma store. // First create object. @@ -468,20 +175,20 @@ TEST_F(TestPlasmaStore, DeleteTest) { uint8_t metadata[] = {5}; int64_t metadata_size = sizeof(metadata); std::shared_ptr data; - ARROW_CHECK_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); - ARROW_CHECK_OK(client_.Seal(object_id)); + ASSERT_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Seal(object_id)); result = client_.Delete(object_id); - ARROW_CHECK_OK(result); + ASSERT_OK(result); bool has_object = false; - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_TRUE(has_object); - ARROW_CHECK_OK(client_.Release(object_id)); + ASSERT_OK(client_.Release(object_id)); // object_id is marked as to-be-deleted, when it is not in use, it will be deleted. - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_FALSE(has_object); - ARROW_CHECK_OK(client_.Delete(object_id)); + ASSERT_OK(client_.Delete(object_id)); } TEST_F(TestPlasmaStore, DeleteObjectsTest) { @@ -490,31 +197,31 @@ TEST_F(TestPlasmaStore, DeleteObjectsTest) { // Test for deleting non-existance object. Status result = client_.Delete(std::vector{object_id1, object_id2}); - ARROW_CHECK_OK(result); + ASSERT_OK(result); // Test for the object being in local Plasma store. // First create object. int64_t data_size = 100; uint8_t metadata[] = {5}; int64_t metadata_size = sizeof(metadata); std::shared_ptr data; - ARROW_CHECK_OK(client_.Create(object_id1, data_size, metadata, metadata_size, &data)); - ARROW_CHECK_OK(client_.Seal(object_id1)); - ARROW_CHECK_OK(client_.Create(object_id2, data_size, metadata, metadata_size, &data)); - ARROW_CHECK_OK(client_.Seal(object_id2)); + ASSERT_OK(client_.Create(object_id1, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Seal(object_id1)); + ASSERT_OK(client_.Create(object_id2, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Seal(object_id2)); // Release the ref count of Create function. - ARROW_CHECK_OK(client_.Release(object_id1)); - ARROW_CHECK_OK(client_.Release(object_id2)); + ASSERT_OK(client_.Release(object_id1)); + ASSERT_OK(client_.Release(object_id2)); // Increase the ref count by calling Get using client2_. std::vector object_buffers; - ARROW_CHECK_OK(client2_.Get({object_id1, object_id2}, 0, &object_buffers)); + ASSERT_OK(client2_.Get({object_id1, object_id2}, 0, &object_buffers)); // Objects are still used by client2_. result = client_.Delete(std::vector{object_id1, object_id2}); - ARROW_CHECK_OK(result); + ASSERT_OK(result); // The object is used and it should not be deleted right now. bool has_object = false; - ARROW_CHECK_OK(client_.Contains(object_id1, &has_object)); + ASSERT_OK(client_.Contains(object_id1, &has_object)); ASSERT_TRUE(has_object); - ARROW_CHECK_OK(client_.Contains(object_id2, &has_object)); + ASSERT_OK(client_.Contains(object_id2, &has_object)); ASSERT_TRUE(has_object); // Decrease the ref count by deleting the PlasmaBuffer (in ObjectBuffer). // client2_ won't send the release request immediately because the trigger @@ -522,9 +229,9 @@ TEST_F(TestPlasmaStore, DeleteObjectsTest) { object_buffers.clear(); // Delete the objects. result = client2_.Delete(std::vector{object_id1, object_id2}); - ARROW_CHECK_OK(client_.Contains(object_id1, &has_object)); + ASSERT_OK(client_.Contains(object_id1, &has_object)); ASSERT_FALSE(has_object); - ARROW_CHECK_OK(client_.Contains(object_id2, &has_object)); + ASSERT_OK(client_.Contains(object_id2, &has_object)); ASSERT_FALSE(has_object); } @@ -533,7 +240,7 @@ TEST_F(TestPlasmaStore, ContainsTest) { // Test for object non-existence. bool has_object; - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_FALSE(has_object); // Test for the object being in local Plasma store. @@ -541,8 +248,8 @@ TEST_F(TestPlasmaStore, ContainsTest) { std::vector data(100, 0); CreateObject(client_, object_id, {42}, data); std::vector object_buffers; - ARROW_CHECK_OK(client_.Get({object_id}, -1, &object_buffers)); - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Get({object_id}, -1, &object_buffers)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_TRUE(has_object); } @@ -552,7 +259,7 @@ TEST_F(TestPlasmaStore, GetTest) { ObjectID object_id = random_object_id(); // Test for object non-existence. - ARROW_CHECK_OK(client_.Get({object_id}, 0, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, 0, &object_buffers)); ASSERT_EQ(object_buffers.size(), 1); ASSERT_FALSE(object_buffers[0].metadata); ASSERT_FALSE(object_buffers[0].data); @@ -565,7 +272,7 @@ TEST_F(TestPlasmaStore, GetTest) { EXPECT_FALSE(client_.IsInUse(object_id)); object_buffers.clear(); - ARROW_CHECK_OK(client_.Get({object_id}, -1, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, -1, &object_buffers)); ASSERT_EQ(object_buffers.size(), 1); ASSERT_EQ(object_buffers[0].device_num, 0); AssertObjectBufferEqual(object_buffers[0], {42}, {3, 5, 6, 7, 9}); @@ -588,7 +295,7 @@ TEST_F(TestPlasmaStore, LegacyGetTest) { ObjectBuffer object_buffer; // Test for object non-existence. - ARROW_CHECK_OK(client_.Get(&object_id, 1, 0, &object_buffer)); + ASSERT_OK(client_.Get(&object_id, 1, 0, &object_buffer)); ASSERT_FALSE(object_buffer.metadata); ASSERT_FALSE(object_buffer.data); EXPECT_FALSE(client_.IsInUse(object_id)); @@ -598,12 +305,12 @@ TEST_F(TestPlasmaStore, LegacyGetTest) { CreateObject(client_, object_id, {42}, data); EXPECT_FALSE(client_.IsInUse(object_id)); - ARROW_CHECK_OK(client_.Get(&object_id, 1, -1, &object_buffer)); + ASSERT_OK(client_.Get(&object_id, 1, -1, &object_buffer)); AssertObjectBufferEqual(object_buffer, {42}, {3, 5, 6, 7, 9}); } // Object needs releasing manually EXPECT_TRUE(client_.IsInUse(object_id)); - ARROW_CHECK_OK(client_.Release(object_id)); + ASSERT_OK(client_.Release(object_id)); EXPECT_FALSE(client_.IsInUse(object_id)); } @@ -617,49 +324,25 @@ TEST_F(TestPlasmaStore, MultipleGetTest) { uint8_t metadata[] = {5}; int64_t metadata_size = sizeof(metadata); std::shared_ptr data; - ARROW_CHECK_OK(client_.Create(object_id1, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Create(object_id1, data_size, metadata, metadata_size, &data)); data->mutable_data()[0] = 1; - ARROW_CHECK_OK(client_.Seal(object_id1)); + ASSERT_OK(client_.Seal(object_id1)); - ARROW_CHECK_OK(client_.Create(object_id2, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Create(object_id2, data_size, metadata, metadata_size, &data)); data->mutable_data()[0] = 2; - ARROW_CHECK_OK(client_.Seal(object_id2)); + ASSERT_OK(client_.Seal(object_id2)); - ARROW_CHECK_OK(client_.Get(object_ids, -1, &object_buffers)); + ASSERT_OK(client_.Get(object_ids, -1, &object_buffers)); ASSERT_EQ(object_buffers[0].data->data()[0], 1); ASSERT_EQ(object_buffers[1].data->data()[0], 2); } -TEST_F(TestPlasmaStore, BatchCreateTest) { - ObjectID object_id1 = random_object_id(); - ObjectID object_id2 = random_object_id(); - std::vector object_ids = {object_id1, object_id2}; - - std::vector data = {"hello", "world"}; - std::vector metadata = {"1", "2"}; - - ARROW_CHECK_OK(client_.CreateAndSealBatch(object_ids, data, metadata)); - - std::vector object_buffers; - - ARROW_CHECK_OK(client_.Get(object_ids, -1, &object_buffers)); - - std::string out1, out2; - out1.assign(reinterpret_cast(object_buffers[0].data->data()), - object_buffers[0].data->size()); - out2.assign(reinterpret_cast(object_buffers[1].data->data()), - object_buffers[1].data->size()); - - ASSERT_STREQ(out1.c_str(), "hello"); - ASSERT_STREQ(out2.c_str(), "world"); -} - TEST_F(TestPlasmaStore, AbortTest) { ObjectID object_id = random_object_id(); std::vector object_buffers; // Test for object non-existence. - ARROW_CHECK_OK(client_.Get({object_id}, 0, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, 0, &object_buffers)); ASSERT_FALSE(object_buffers[0].data); // Test object abort. @@ -669,7 +352,7 @@ TEST_F(TestPlasmaStore, AbortTest) { int64_t metadata_size = sizeof(metadata); std::shared_ptr data; uint8_t* data_ptr; - ARROW_CHECK_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); data_ptr = data->mutable_data(); // Write some data. for (int64_t i = 0; i < data_size / 2; i++) { @@ -679,21 +362,21 @@ TEST_F(TestPlasmaStore, AbortTest) { Status status = client_.Abort(object_id); ASSERT_TRUE(status.IsInvalid()); // Release, then abort. - ARROW_CHECK_OK(client_.Release(object_id)); + ASSERT_OK(client_.Release(object_id)); EXPECT_TRUE(client_.IsInUse(object_id)); - ARROW_CHECK_OK(client_.Abort(object_id)); + ASSERT_OK(client_.Abort(object_id)); EXPECT_FALSE(client_.IsInUse(object_id)); // Test for object non-existence after the abort. - ARROW_CHECK_OK(client_.Get({object_id}, 0, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, 0, &object_buffers)); ASSERT_FALSE(object_buffers[0].data); // Create the object successfully this time. CreateObject(client_, object_id, {42, 43}, {1, 2, 3, 4, 5}); // Test that we can get the object. - ARROW_CHECK_OK(client_.Get({object_id}, -1, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, -1, &object_buffers)); AssertObjectBufferEqual(object_buffers[0], {42, 43}, {1, 2, 3, 4, 5}); } @@ -704,7 +387,7 @@ TEST_F(TestPlasmaStore, OneIdCreateRepeatedlyTest) { std::vector object_buffers; // Test for object non-existence. - ARROW_CHECK_OK(client_.Get({object_id}, 0, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, 0, &object_buffers)); ASSERT_FALSE(object_buffers[0].data); int64_t data_size = 20; @@ -714,18 +397,18 @@ TEST_F(TestPlasmaStore, OneIdCreateRepeatedlyTest) { // Test the sequence: create -> release -> abort -> ... for (int64_t i = 0; i < loop_times; i++) { std::shared_ptr data; - ARROW_CHECK_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); - ARROW_CHECK_OK(client_.Release(object_id)); - ARROW_CHECK_OK(client_.Abort(object_id)); + ASSERT_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Release(object_id)); + ASSERT_OK(client_.Abort(object_id)); } // Test the sequence: create -> seal -> release -> delete -> ... for (int64_t i = 0; i < loop_times; i++) { std::shared_ptr data; - ARROW_CHECK_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); - ARROW_CHECK_OK(client_.Seal(object_id)); - ARROW_CHECK_OK(client_.Release(object_id)); - ARROW_CHECK_OK(client_.Delete(object_id)); + ASSERT_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Seal(object_id)); + ASSERT_OK(client_.Release(object_id)); + ASSERT_OK(client_.Delete(object_id)); } } @@ -735,7 +418,7 @@ TEST_F(TestPlasmaStore, MultipleClientTest) { // Test for object non-existence on the first client. bool has_object; - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_FALSE(has_object); // Test for the object being in local Plasma store. @@ -744,25 +427,25 @@ TEST_F(TestPlasmaStore, MultipleClientTest) { uint8_t metadata[] = {5}; int64_t metadata_size = sizeof(metadata); std::shared_ptr data; - ARROW_CHECK_OK(client2_.Create(object_id, data_size, metadata, metadata_size, &data)); - ARROW_CHECK_OK(client2_.Seal(object_id)); + ASSERT_OK(client2_.Create(object_id, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client2_.Seal(object_id)); // Test that the first client can get the object. - ARROW_CHECK_OK(client_.Get({object_id}, -1, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, -1, &object_buffers)); ASSERT_TRUE(object_buffers[0].data); - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_TRUE(has_object); // Test that one client disconnecting does not interfere with the other. // First create object on the second client. object_id = random_object_id(); - ARROW_CHECK_OK(client2_.Create(object_id, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client2_.Create(object_id, data_size, metadata, metadata_size, &data)); // Disconnect the first client. - ARROW_CHECK_OK(client_.Disconnect()); + ASSERT_OK(client_.Disconnect()); // Test that the second client can seal and get the created object. - ARROW_CHECK_OK(client2_.Seal(object_id)); - ARROW_CHECK_OK(client2_.Get({object_id}, -1, &object_buffers)); + ASSERT_OK(client2_.Seal(object_id)); + ASSERT_OK(client2_.Get({object_id}, -1, &object_buffers)); ASSERT_TRUE(object_buffers[0].data); - ARROW_CHECK_OK(client2_.Contains(object_id, &has_object)); + ASSERT_OK(client2_.Contains(object_id, &has_object)); ASSERT_TRUE(has_object); } @@ -776,7 +459,7 @@ TEST_F(TestPlasmaStore, ManyObjectTest) { // Test for object non-existence on the first client. bool has_object; - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_FALSE(has_object); // Test for the object being in local Plasma store. @@ -785,29 +468,29 @@ TEST_F(TestPlasmaStore, ManyObjectTest) { uint8_t metadata[] = {5}; int64_t metadata_size = sizeof(metadata); std::shared_ptr data; - ARROW_CHECK_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); + ASSERT_OK(client_.Create(object_id, data_size, metadata, metadata_size, &data)); if (i % 3 == 0) { // Seal one third of the objects. - ARROW_CHECK_OK(client_.Seal(object_id)); + ASSERT_OK(client_.Seal(object_id)); // Test that the first client can get the object. - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_TRUE(has_object); } else if (i % 3 == 1) { // Abort one third of the objects. - ARROW_CHECK_OK(client_.Release(object_id)); - ARROW_CHECK_OK(client_.Abort(object_id)); + ASSERT_OK(client_.Release(object_id)); + ASSERT_OK(client_.Abort(object_id)); } } // Disconnect the first client. All unsealed objects should be aborted. - ARROW_CHECK_OK(client_.Disconnect()); + ASSERT_OK(client_.Disconnect()); // Check that the second client can query the object store for the first // client's objects. int i = 0; for (auto const& object_id : object_ids) { bool has_object; - ARROW_CHECK_OK(client2_.Contains(object_id, &has_object)); + ASSERT_OK(client2_.Contains(object_id, &has_object)); if (i % 3 == 0) { // The first third should be sealed. ASSERT_TRUE(has_object); @@ -831,13 +514,13 @@ void AssertCudaRead(const std::shared_ptr& buffer, std::shared_ptr gpu_buffer; const size_t data_size = expected_data.size(); - ARROW_CHECK_OK(CudaBuffer::FromBuffer(buffer, &gpu_buffer)); + ASSERT_OK(CudaBuffer::FromBuffer(buffer, &gpu_buffer)); ASSERT_EQ(gpu_buffer->size(), data_size); CudaBufferReader reader(gpu_buffer); std::vector read_data(data_size); int64_t read_data_size; - ARROW_CHECK_OK(reader.Read(data_size, &read_data_size, read_data.data())); + ASSERT_OK(reader.Read(data_size, &read_data_size, read_data.data())); ASSERT_EQ(read_data_size, data_size); for (size_t i = 0; i < data_size; i++) { @@ -852,7 +535,7 @@ TEST_F(TestPlasmaStore, GetGPUTest) { std::vector object_buffers; // Test for object non-existence. - ARROW_CHECK_OK(client_.Get({object_id}, 0, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, 0, &object_buffers)); ASSERT_EQ(object_buffers.size(), 1); ASSERT_FALSE(object_buffers[0].data); @@ -864,15 +547,15 @@ TEST_F(TestPlasmaStore, GetGPUTest) { int64_t metadata_size = sizeof(metadata); std::shared_ptr data_buffer; std::shared_ptr gpu_buffer; - ARROW_CHECK_OK( + ASSERT_OK( client_.Create(object_id, data_size, metadata, metadata_size, &data_buffer, 1)); - ARROW_CHECK_OK(CudaBuffer::FromBuffer(data_buffer, &gpu_buffer)); + ASSERT_OK(CudaBuffer::FromBuffer(data_buffer, &gpu_buffer)); CudaBufferWriter writer(gpu_buffer); - ARROW_CHECK_OK(writer.Write(data, data_size)); - ARROW_CHECK_OK(client_.Seal(object_id)); + ASSERT_OK(writer.Write(data, data_size)); + ASSERT_OK(client_.Seal(object_id)); object_buffers.clear(); - ARROW_CHECK_OK(client_.Get({object_id}, -1, &object_buffers)); + ASSERT_OK(client_.Get({object_id}, -1, &object_buffers)); ASSERT_EQ(object_buffers.size(), 1); ASSERT_EQ(object_buffers[0].device_num, 1); // Check data @@ -887,33 +570,34 @@ TEST_F(TestPlasmaStore, DeleteObjectsGPUTest) { // Test for deleting non-existance object. Status result = client_.Delete(std::vector{object_id1, object_id2}); - ARROW_CHECK_OK(result); + ASSERT_OK(result); // Test for the object being in local Plasma store. // First create object. int64_t data_size = 100; uint8_t metadata[] = {5}; int64_t metadata_size = sizeof(metadata); std::shared_ptr data; - ARROW_CHECK_OK( + ASSERT_OK( client_.Create(object_id1, data_size, metadata, metadata_size, &data, 1)); - ARROW_CHECK_OK(client_.Seal(object_id1)); - ARROW_CHECK_OK( + ASSERT_OK(client_.Seal(object_id1)); + ASSERT_OK( client_.Create(object_id2, data_size, metadata, metadata_size, &data, 1)); - ARROW_CHECK_OK(client_.Seal(object_id2)); + ASSERT_OK(client_.Seal(object_id2)); // Release the ref count of Create function. - ARROW_CHECK_OK(client_.Release(object_id1)); - ARROW_CHECK_OK(client_.Release(object_id2)); + data = nullptr; + ASSERT_OK(client_.Release(object_id1)); + ASSERT_OK(client_.Release(object_id2)); // Increase the ref count by calling Get using client2_. std::vector object_buffers; - ARROW_CHECK_OK(client2_.Get({object_id1, object_id2}, 0, &object_buffers)); + ASSERT_OK(client2_.Get({object_id1, object_id2}, 0, &object_buffers)); // Objects are still used by client2_. result = client_.Delete(std::vector{object_id1, object_id2}); - ARROW_CHECK_OK(result); + ASSERT_OK(result); // The object is used and it should not be deleted right now. bool has_object = false; - ARROW_CHECK_OK(client_.Contains(object_id1, &has_object)); + ASSERT_OK(client_.Contains(object_id1, &has_object)); ASSERT_TRUE(has_object); - ARROW_CHECK_OK(client_.Contains(object_id2, &has_object)); + ASSERT_OK(client_.Contains(object_id2, &has_object)); ASSERT_TRUE(has_object); // Decrease the ref count by deleting the PlasmaBuffer (in ObjectBuffer). // client2_ won't send the release request immediately because the trigger @@ -921,9 +605,9 @@ TEST_F(TestPlasmaStore, DeleteObjectsGPUTest) { object_buffers.clear(); // Delete the objects. result = client2_.Delete(std::vector{object_id1, object_id2}); - ARROW_CHECK_OK(client_.Contains(object_id1, &has_object)); + ASSERT_OK(client_.Contains(object_id1, &has_object)); ASSERT_FALSE(has_object); - ARROW_CHECK_OK(client_.Contains(object_id2, &has_object)); + ASSERT_OK(client_.Contains(object_id2, &has_object)); ASSERT_FALSE(has_object); } @@ -940,49 +624,27 @@ TEST_F(TestPlasmaStore, RepeatlyCreateGPUTest) { ObjectID& object_id = object_ids[i]; std::shared_ptr data; - ARROW_CHECK_OK(client_.Create(object_id, data_size, 0, 0, &data, 1)); - ARROW_CHECK_OK(client_.Seal(object_id)); - ARROW_CHECK_OK(client_.Release(object_id)); + ASSERT_OK(client_.Create(object_id, data_size, 0, 0, &data, 1)); + ASSERT_OK(client_.Seal(object_id)); + ASSERT_OK(client_.Release(object_id)); } // delete and create again for (int64_t i = 0; i < loop_times; i++) { ObjectID& object_id = object_ids[i % object_num]; - ARROW_CHECK_OK(client_.Delete(object_id)); + ASSERT_OK(client_.Delete(object_id)); std::shared_ptr data; - ARROW_CHECK_OK(client_.Create(object_id, data_size, 0, 0, &data, 1)); - ARROW_CHECK_OK(client_.Seal(object_id)); - ARROW_CHECK_OK(client_.Release(object_id)); + ASSERT_OK(client_.Create(object_id, data_size, 0, 0, &data, 1)); + ASSERT_OK(client_.Seal(object_id)); + + data = nullptr; + ASSERT_OK(client_.Release(object_id)); } // delete all - ARROW_CHECK_OK(client_.Delete(object_ids)); -} - -TEST_F(TestPlasmaStore, GPUBufferLifetime) { - // ARROW-5924: GPU buffer is allowed to persist after Release() - ObjectID object_id = random_object_id(); - const int64_t data_size = 40; - - std::shared_ptr create_buff; - ARROW_CHECK_OK(client_.Create(object_id, data_size, nullptr, 0, &create_buff, 1)); - ARROW_CHECK_OK(client_.Seal(object_id)); - ARROW_CHECK_OK(client_.Release(object_id)); - - ObjectBuffer get_buff_1; - ARROW_CHECK_OK(client_.Get(&object_id, 1, -1, &get_buff_1)); - ObjectBuffer get_buff_2; - ARROW_CHECK_OK(client_.Get(&object_id, 1, -1, &get_buff_2)); - ARROW_CHECK_OK(client_.Release(object_id)); - ARROW_CHECK_OK(client_.Release(object_id)); - - ObjectBuffer get_buff_3; - ARROW_CHECK_OK(client_.Get(&object_id, 1, -1, &get_buff_3)); - ARROW_CHECK_OK(client_.Release(object_id)); - - ARROW_CHECK_OK(client_.Delete(object_id)); + ASSERT_OK(client_.Delete(object_ids)); } TEST_F(TestPlasmaStore, MultipleClientGPUTest) { @@ -991,7 +653,7 @@ TEST_F(TestPlasmaStore, MultipleClientGPUTest) { // Test for object non-existence on the first client. bool has_object; - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_FALSE(has_object); // Test for the object being in local Plasma store. @@ -1000,27 +662,27 @@ TEST_F(TestPlasmaStore, MultipleClientGPUTest) { uint8_t metadata[] = {5}; int64_t metadata_size = sizeof(metadata); std::shared_ptr data; - ARROW_CHECK_OK( + ASSERT_OK( client2_.Create(object_id, data_size, metadata, metadata_size, &data, 1)); - ARROW_CHECK_OK(client2_.Seal(object_id)); + ASSERT_OK(client2_.Seal(object_id)); // Test that the first client can get the object. - ARROW_CHECK_OK(client_.Get({object_id}, -1, &object_buffers)); - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Get({object_id}, -1, &object_buffers)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_TRUE(has_object); // Test that one client disconnecting does not interfere with the other. // First create object on the second client. object_id = random_object_id(); - ARROW_CHECK_OK( + ASSERT_OK( client2_.Create(object_id, data_size, metadata, metadata_size, &data, 1)); // Disconnect the first client. - ARROW_CHECK_OK(client_.Disconnect()); + ASSERT_OK(client_.Disconnect()); // Test that the second client can seal and get the created object. - ARROW_CHECK_OK(client2_.Seal(object_id)); + ASSERT_OK(client2_.Seal(object_id)); object_buffers.clear(); - ARROW_CHECK_OK(client2_.Contains(object_id, &has_object)); + ASSERT_OK(client2_.Contains(object_id, &has_object)); ASSERT_TRUE(has_object); - ARROW_CHECK_OK(client2_.Get({object_id}, -1, &object_buffers)); + ASSERT_OK(client2_.Get({object_id}, -1, &object_buffers)); ASSERT_EQ(object_buffers.size(), 1); ASSERT_EQ(object_buffers[0].device_num, 1); AssertCudaRead(object_buffers[0].metadata, {5}); diff --git a/cpp/src/plasma/test/external_store_tests.cc b/cpp/src/plasma/test/external_store_tests.cc index 98b298d8402..0a988516b67 100644 --- a/cpp/src/plasma/test/external_store_tests.cc +++ b/cpp/src/plasma/test/external_store_tests.cc @@ -54,7 +54,7 @@ class TestPlasmaStoreWithExternal : public ::testing::Test { // TODO(pcm): At the moment, stdout of the test gets mixed up with // stdout of the object store. Consider changing that. void SetUp() override { - ARROW_CHECK_OK(TemporaryDir::Make("ext-test-", &temp_dir_)); + ASSERT_OK(TemporaryDir::Make("ext-test-", &temp_dir_)); store_socket_name_ = temp_dir_->path().ToString() + "store"; std::string plasma_directory = @@ -65,11 +65,11 @@ class TestPlasmaStoreWithExternal : public ::testing::Test { " 1> /tmp/log.stdout 2> /tmp/log.stderr & " + "echo $! > " + store_socket_name_ + ".pid"; PLASMA_CHECK_SYSTEM(system(plasma_command.c_str())); - ARROW_CHECK_OK(client_.Connect(store_socket_name_, "")); + ASSERT_OK(client_.Connect(store_socket_name_, "")); } void TearDown() override { - ARROW_CHECK_OK(client_.Disconnect()); + ASSERT_OK(client_.Disconnect()); // Kill plasma_store process that we started #ifdef COVERAGE_BUILD // Ask plasma_store to exit gracefully and give it time to write out @@ -100,14 +100,14 @@ TEST_F(TestPlasmaStoreWithExternal, EvictionTest) { // Test for object non-existence. bool has_object; - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_FALSE(has_object); // Test for the object being in local Plasma store. // Create and seal the object. - ARROW_CHECK_OK(client_.CreateAndSeal(object_id, data, metadata)); + ASSERT_OK(client_.CreateAndSeal(object_id, data, metadata)); // Test that the client can get the object. - ARROW_CHECK_OK(client_.Contains(object_id, &has_object)); + ASSERT_OK(client_.Contains(object_id, &has_object)); ASSERT_TRUE(has_object); } @@ -118,7 +118,7 @@ TEST_F(TestPlasmaStoreWithExternal, EvictionTest) { // external store on failure. This should succeed to fetch the object. // However, it may evict the next few objects. std::vector object_buffers; - ARROW_CHECK_OK(client_.Get({object_ids[i]}, -1, &object_buffers)); + ASSERT_OK(client_.Get({object_ids[i]}, -1, &object_buffers)); ASSERT_EQ(object_buffers.size(), 1); ASSERT_EQ(object_buffers[0].device_num, 0); ASSERT_TRUE(object_buffers[0].data); @@ -127,7 +127,7 @@ TEST_F(TestPlasmaStoreWithExternal, EvictionTest) { // Make sure we still cannot fetch objects that do not exist std::vector object_buffers; - ARROW_CHECK_OK(client_.Get({random_object_id()}, 100, &object_buffers)); + ASSERT_OK(client_.Get({random_object_id()}, 100, &object_buffers)); ASSERT_EQ(object_buffers.size(), 1); ASSERT_EQ(object_buffers[0].device_num, 0); ASSERT_EQ(object_buffers[0].data, nullptr); diff --git a/cpp/src/plasma/test/serialization_tests.cc b/cpp/src/plasma/test/serialization_tests.cc index 7a3c3098a61..58f6020010e 100644 --- a/cpp/src/plasma/test/serialization_tests.cc +++ b/cpp/src/plasma/test/serialization_tests.cc @@ -21,40 +21,56 @@ #include #include +#include #include "arrow/testing/gtest_util.h" -#include "arrow/util/io_util.h" +#include "arrow/util/io-util.h" #include "plasma/common.h" -#include "plasma/io.h" -#include "plasma/plasma.h" +#include "plasma/io/connection.h" #include "plasma/protocol.h" -#include "plasma/test_util.h" - -namespace fb = plasma::flatbuf; +#include "plasma/test-util.h" namespace plasma { -using arrow::internal::TemporaryDir; +using flatbuf::MessageType; +using io::ClientConnection; +using io::ServerConnection; -/** - * Seek to the beginning of a file and read a message from it. - * - * @param fd File descriptor of the file. - * @param message_type Message type that we expect in the file. - * - * @return Pointer to the content of the message. Needs to be freed by the - * caller. - */ -std::vector read_message_from_file(int fd, MessageType message_type) { - /* Go to the beginning of the file. */ - lseek(fd, 0, SEEK_SET); - MessageType type; - std::vector data; - Status s = ReadMessage(fd, &type, &data); - DCHECK_OK(s); - DCHECK_EQ(type, message_type); - return data; +class TestPlasmaSerialization : public ::testing::Test { + public: + void SetUp() override { + using asio::local::stream_protocol; + stream_protocol::socket parentSocket(io_context_); + stream_protocol::socket childSocket(io_context_); + // create socket pair + asio::local::connect_pair(childSocket, parentSocket); + client_ = ServerConnection::Create(std::move(childSocket)); + io::MessageHandler monk_handler = [](std::shared_ptr client, + int64_t type, int64_t length, + const uint8_t* msg) {}; + server_ = ClientConnection::Create(std::move(parentSocket), monk_handler); + } + + void TearDown() override { + client_->Close(); + server_->Close(); + } + + protected: + asio::io_context io_context_; + std::shared_ptr client_; + std::shared_ptr server_; +}; + +Status PlasmaReceive(const std::shared_ptr& client, + MessageType message_type, std::vector* buffer) { + return client->ReadMessage(static_cast(message_type), buffer); +} + +Status PlasmaReceive(const std::shared_ptr& client, + MessageType message_type, std::vector* buffer) { + return client->ReadMessage(static_cast(message_type), buffer); } PlasmaObject random_plasma_object(void) { @@ -70,36 +86,15 @@ PlasmaObject random_plasma_object(void) { return object; } -class TestPlasmaSerialization : public ::testing::Test { - public: - void SetUp() { ARROW_CHECK_OK(TemporaryDir::Make("ser-test-", &temp_dir_)); } - - // Create a temporary file. - // A fd is returned which must be closed manually. The file itself - // is deleted at the end of the test. - int CreateTemporaryFile(void) { - char path[1024]; - - std::stringstream ss; - ss << temp_dir_->path().ToString() << "fileXXXXXX"; - strncpy(path, ss.str().c_str(), sizeof(path)); - ARROW_LOG(INFO) << "file path: '" << path << "'"; - return mkstemp(path); - } - - protected: - std::unique_ptr temp_dir_; -}; - TEST_F(TestPlasmaSerialization, CreateRequest) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); int64_t data_size1 = 42; int64_t metadata_size1 = 11; int device_num1 = 0; - ASSERT_OK(SendCreateRequest(fd, object_id1, data_size1, metadata_size1, device_num1)); - std::vector data = - read_message_from_file(fd, MessageType::PlasmaCreateRequest); + ASSERT_OK( + SendCreateRequest(client_, object_id1, data_size1, metadata_size1, device_num1)); + std::vector data; + ASSERT_OK(PlasmaReceive(server_, MessageType::PlasmaCreateRequest, &data)); ObjectID object_id2; int64_t data_size2; int64_t metadata_size2; @@ -110,16 +105,15 @@ TEST_F(TestPlasmaSerialization, CreateRequest) { ASSERT_EQ(metadata_size1, metadata_size2); ASSERT_EQ(object_id1, object_id2); ASSERT_EQ(device_num1, device_num2); - close(fd); } TEST_F(TestPlasmaSerialization, CreateReply) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); PlasmaObject object1 = random_plasma_object(); int64_t mmap_size1 = 1000000; - ASSERT_OK(SendCreateReply(fd, object_id1, &object1, PlasmaError::OK, mmap_size1)); - std::vector data = read_message_from_file(fd, MessageType::PlasmaCreateReply); + ASSERT_OK(SendCreateReply(server_, object_id1, &object1, PlasmaError::OK, mmap_size1)); + std::vector data; + ASSERT_OK(PlasmaReceive(client_, MessageType::PlasmaCreateReply, &data)); ObjectID object_id2; PlasmaObject object2 = {}; int store_fd; @@ -130,43 +124,40 @@ TEST_F(TestPlasmaSerialization, CreateReply) { ASSERT_EQ(object1.store_fd, store_fd); ASSERT_EQ(mmap_size1, mmap_size2); ASSERT_EQ(memcmp(&object1, &object2, sizeof(object1)), 0); - close(fd); } TEST_F(TestPlasmaSerialization, SealRequest) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); std::string digest1 = std::string(kDigestSize, 7); - ASSERT_OK(SendSealRequest(fd, object_id1, digest1)); - std::vector data = read_message_from_file(fd, MessageType::PlasmaSealRequest); + ASSERT_OK(SendSealRequest(client_, object_id1, digest1)); + std::vector data; + ASSERT_OK(PlasmaReceive(server_, MessageType::PlasmaSealRequest, &data)); ObjectID object_id2; std::string digest2; ASSERT_OK(ReadSealRequest(data.data(), data.size(), &object_id2, &digest2)); ASSERT_EQ(object_id1, object_id2); ASSERT_EQ(memcmp(digest1.data(), digest2.data(), kDigestSize), 0); - close(fd); } TEST_F(TestPlasmaSerialization, SealReply) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); - ASSERT_OK(SendSealReply(fd, object_id1, PlasmaError::ObjectExists)); - std::vector data = read_message_from_file(fd, MessageType::PlasmaSealReply); + ASSERT_OK(SendSealReply(server_, object_id1, PlasmaError::ObjectExists)); + std::vector data; + ASSERT_OK(PlasmaReceive(client_, MessageType::PlasmaSealReply, &data)); ObjectID object_id2; Status s = ReadSealReply(data.data(), data.size(), &object_id2); ASSERT_EQ(object_id1, object_id2); ASSERT_TRUE(IsPlasmaObjectExists(s)); - close(fd); } TEST_F(TestPlasmaSerialization, GetRequest) { - int fd = CreateTemporaryFile(); ObjectID object_ids[2]; object_ids[0] = random_object_id(); object_ids[1] = random_object_id(); int64_t timeout_ms = 1234; - ASSERT_OK(SendGetRequest(fd, object_ids, 2, timeout_ms)); - std::vector data = read_message_from_file(fd, MessageType::PlasmaGetRequest); + ASSERT_OK(SendGetRequest(client_, object_ids, 2, timeout_ms)); + std::vector data; + ASSERT_OK(PlasmaReceive(server_, MessageType::PlasmaGetRequest, &data)); std::vector object_ids_return; int64_t timeout_ms_return; ASSERT_OK( @@ -174,11 +165,9 @@ TEST_F(TestPlasmaSerialization, GetRequest) { ASSERT_EQ(object_ids[0], object_ids_return[0]); ASSERT_EQ(object_ids[1], object_ids_return[1]); ASSERT_EQ(timeout_ms, timeout_ms_return); - close(fd); } TEST_F(TestPlasmaSerialization, GetReply) { - int fd = CreateTemporaryFile(); ObjectID object_ids[2]; object_ids[0] = random_object_id(); object_ids[1] = random_object_id(); @@ -187,9 +176,10 @@ TEST_F(TestPlasmaSerialization, GetReply) { plasma_objects[object_ids[1]] = random_plasma_object(); std::vector store_fds = {1, 2, 3}; std::vector mmap_sizes = {100, 200, 300}; - ASSERT_OK(SendGetReply(fd, object_ids, plasma_objects, 2, store_fds, mmap_sizes)); + ASSERT_OK(SendGetReply(server_, object_ids, plasma_objects, 2, store_fds, mmap_sizes)); - std::vector data = read_message_from_file(fd, MessageType::PlasmaGetReply); + std::vector data; + ASSERT_OK(PlasmaReceive(client_, MessageType::PlasmaGetReply, &data)); ObjectID object_ids_return[2]; PlasmaObject plasma_objects_return[2]; std::vector store_fds_return; @@ -210,53 +200,47 @@ TEST_F(TestPlasmaSerialization, GetReply) { } ASSERT_TRUE(store_fds == store_fds_return); ASSERT_TRUE(mmap_sizes == mmap_sizes_return); - close(fd); } TEST_F(TestPlasmaSerialization, ReleaseRequest) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); - ASSERT_OK(SendReleaseRequest(fd, object_id1)); - std::vector data = - read_message_from_file(fd, MessageType::PlasmaReleaseRequest); + ASSERT_OK(SendReleaseRequest(client_, object_id1)); + std::vector data; + ASSERT_OK(PlasmaReceive(server_, MessageType::PlasmaReleaseRequest, &data)); ObjectID object_id2; ASSERT_OK(ReadReleaseRequest(data.data(), data.size(), &object_id2)); ASSERT_EQ(object_id1, object_id2); - close(fd); } TEST_F(TestPlasmaSerialization, ReleaseReply) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); - ASSERT_OK(SendReleaseReply(fd, object_id1, PlasmaError::ObjectExists)); - std::vector data = read_message_from_file(fd, MessageType::PlasmaReleaseReply); + ASSERT_OK(SendReleaseReply(server_, object_id1, PlasmaError::ObjectExists)); + std::vector data; + ASSERT_OK(PlasmaReceive(client_, MessageType::PlasmaReleaseReply, &data)); ObjectID object_id2; Status s = ReadReleaseReply(data.data(), data.size(), &object_id2); ASSERT_EQ(object_id1, object_id2); ASSERT_TRUE(IsPlasmaObjectExists(s)); - close(fd); } TEST_F(TestPlasmaSerialization, DeleteRequest) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); - ASSERT_OK(SendDeleteRequest(fd, std::vector{object_id1})); - std::vector data = - read_message_from_file(fd, MessageType::PlasmaDeleteRequest); + ASSERT_OK(SendDeleteRequest(client_, std::vector{object_id1})); + std::vector data; + ASSERT_OK(PlasmaReceive(server_, MessageType::PlasmaDeleteRequest, &data)); std::vector object_vec; ASSERT_OK(ReadDeleteRequest(data.data(), data.size(), &object_vec)); ASSERT_EQ(object_vec.size(), 1); ASSERT_EQ(object_id1, object_vec[0]); - close(fd); } TEST_F(TestPlasmaSerialization, DeleteReply) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); PlasmaError error1 = PlasmaError::ObjectExists; - ASSERT_OK(SendDeleteReply(fd, std::vector{object_id1}, + ASSERT_OK(SendDeleteReply(server_, std::vector{object_id1}, std::vector{error1})); - std::vector data = read_message_from_file(fd, MessageType::PlasmaDeleteReply); + std::vector data; + ASSERT_OK(PlasmaReceive(client_, MessageType::PlasmaDeleteReply, &data)); std::vector object_vec; std::vector error_vec; Status s = ReadDeleteReply(data.data(), data.size(), &object_vec, &error_vec); @@ -265,39 +249,36 @@ TEST_F(TestPlasmaSerialization, DeleteReply) { ASSERT_EQ(error_vec.size(), 1); ASSERT_TRUE(error_vec[0] == PlasmaError::ObjectExists); ASSERT_TRUE(s.ok()); - close(fd); } TEST_F(TestPlasmaSerialization, EvictRequest) { - int fd = CreateTemporaryFile(); int64_t num_bytes = 111; - ASSERT_OK(SendEvictRequest(fd, num_bytes)); - std::vector data = read_message_from_file(fd, MessageType::PlasmaEvictRequest); + ASSERT_OK(SendEvictRequest(client_, num_bytes)); + std::vector data; + ASSERT_OK(PlasmaReceive(server_, MessageType::PlasmaEvictRequest, &data)); int64_t num_bytes_received; ASSERT_OK(ReadEvictRequest(data.data(), data.size(), &num_bytes_received)); ASSERT_EQ(num_bytes, num_bytes_received); - close(fd); } TEST_F(TestPlasmaSerialization, EvictReply) { - int fd = CreateTemporaryFile(); int64_t num_bytes = 111; - ASSERT_OK(SendEvictReply(fd, num_bytes)); - std::vector data = read_message_from_file(fd, MessageType::PlasmaEvictReply); + ASSERT_OK(SendEvictReply(server_, num_bytes)); + std::vector data; + ASSERT_OK(PlasmaReceive(client_, MessageType::PlasmaEvictReply, &data)); int64_t num_bytes_received; ASSERT_OK(ReadEvictReply(data.data(), data.size(), num_bytes_received)); ASSERT_EQ(num_bytes, num_bytes_received); - close(fd); } TEST_F(TestPlasmaSerialization, DataRequest) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); const char* address1 = "address1"; int port1 = 12345; - ASSERT_OK(SendDataRequest(fd, object_id1, address1, port1)); + ASSERT_OK(SendDataRequest(client_, object_id1, address1, port1)); /* Reading message back. */ - std::vector data = read_message_from_file(fd, MessageType::PlasmaDataRequest); + std::vector data; + ASSERT_OK(PlasmaReceive(server_, MessageType::PlasmaDataRequest, &data)); ObjectID object_id2; char* address2; int port2; @@ -306,17 +287,16 @@ TEST_F(TestPlasmaSerialization, DataRequest) { ASSERT_EQ(strcmp(address1, address2), 0); ASSERT_EQ(port1, port2); free(address2); - close(fd); } TEST_F(TestPlasmaSerialization, DataReply) { - int fd = CreateTemporaryFile(); ObjectID object_id1 = random_object_id(); int64_t object_size1 = 146; int64_t metadata_size1 = 198; - ASSERT_OK(SendDataReply(fd, object_id1, object_size1, metadata_size1)); + ASSERT_OK(SendDataReply(server_, object_id1, object_size1, metadata_size1)); /* Reading message back. */ - std::vector data = read_message_from_file(fd, MessageType::PlasmaDataReply); + std::vector data; + ASSERT_OK(PlasmaReceive(client_, MessageType::PlasmaDataReply, &data)); ObjectID object_id2; int64_t object_size2; int64_t metadata_size2; diff --git a/cpp/src/plasma/thirdparty/ae/ae.c b/cpp/src/plasma/thirdparty/ae/ae.c deleted file mode 100644 index dfb72244409..00000000000 --- a/cpp/src/plasma/thirdparty/ae/ae.c +++ /dev/null @@ -1,465 +0,0 @@ -/* A simple event-driven programming library. Originally I wrote this code - * for the Jim's event-loop (Jim is a Tcl interpreter) but later translated - * it in form of a library for easy reuse. - * - * Copyright (c) 2006-2010, Salvatore Sanfilippo - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Redis nor the names of its contributors may be used - * to endorse or promote products derived from this software without - * specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "plasma/thirdparty/ae/ae.h" -#include "plasma/thirdparty/ae/zmalloc.h" -#include "plasma/thirdparty/ae/config.h" - -/* Include the best multiplexing layer supported by this system. - * The following should be ordered by performances, descending. */ -#ifdef HAVE_EVPORT -#include "plasma/thirdparty/ae/ae_evport.c" -#else - #ifdef HAVE_EPOLL - #include "plasma/thirdparty/ae/ae_epoll.c" - #else - #ifdef HAVE_KQUEUE - #include "plasma/thirdparty/ae/ae_kqueue.c" - #else - #include "plasma/thirdparty/ae/ae_select.c" - #endif - #endif -#endif - -aeEventLoop *aeCreateEventLoop(int setsize) { - aeEventLoop *eventLoop; - int i; - - if ((eventLoop = zmalloc(sizeof(*eventLoop))) == NULL) goto err; - eventLoop->events = zmalloc(sizeof(aeFileEvent)*setsize); - eventLoop->fired = zmalloc(sizeof(aeFiredEvent)*setsize); - if (eventLoop->events == NULL || eventLoop->fired == NULL) goto err; - eventLoop->setsize = setsize; - eventLoop->lastTime = time(NULL); - eventLoop->timeEventHead = NULL; - eventLoop->timeEventNextId = 0; - eventLoop->stop = 0; - eventLoop->maxfd = -1; - eventLoop->beforesleep = NULL; - if (aeApiCreate(eventLoop) == -1) goto err; - /* Events with mask == AE_NONE are not set. So let's initialize the - * vector with it. */ - for (i = 0; i < setsize; i++) - eventLoop->events[i].mask = AE_NONE; - return eventLoop; - -err: - if (eventLoop) { - zfree(eventLoop->events); - zfree(eventLoop->fired); - zfree(eventLoop); - } - return NULL; -} - -/* Return the current set size. */ -int aeGetSetSize(aeEventLoop *eventLoop) { - return eventLoop->setsize; -} - -/* Resize the maximum set size of the event loop. - * If the requested set size is smaller than the current set size, but - * there is already a file descriptor in use that is >= the requested - * set size minus one, AE_ERR is returned and the operation is not - * performed at all. - * - * Otherwise AE_OK is returned and the operation is successful. */ -int aeResizeSetSize(aeEventLoop *eventLoop, int setsize) { - int i; - - if (setsize == eventLoop->setsize) return AE_OK; - if (eventLoop->maxfd >= setsize) return AE_ERR; - if (aeApiResize(eventLoop,setsize) == -1) return AE_ERR; - - eventLoop->events = zrealloc(eventLoop->events,sizeof(aeFileEvent)*setsize); - eventLoop->fired = zrealloc(eventLoop->fired,sizeof(aeFiredEvent)*setsize); - eventLoop->setsize = setsize; - - /* Make sure that if we created new slots, they are initialized with - * an AE_NONE mask. */ - for (i = eventLoop->maxfd+1; i < setsize; i++) - eventLoop->events[i].mask = AE_NONE; - return AE_OK; -} - -void aeDeleteEventLoop(aeEventLoop *eventLoop) { - aeApiFree(eventLoop); - zfree(eventLoop->events); - zfree(eventLoop->fired); - zfree(eventLoop); -} - -void aeStop(aeEventLoop *eventLoop) { - eventLoop->stop = 1; -} - -int aeCreateFileEvent(aeEventLoop *eventLoop, int fd, int mask, - aeFileProc *proc, void *clientData) -{ - if (fd >= eventLoop->setsize) { - errno = ERANGE; - return AE_ERR; - } - aeFileEvent *fe = &eventLoop->events[fd]; - - if (aeApiAddEvent(eventLoop, fd, mask) == -1) - return AE_ERR; - fe->mask |= mask; - if (mask & AE_READABLE) fe->rfileProc = proc; - if (mask & AE_WRITABLE) fe->wfileProc = proc; - fe->clientData = clientData; - if (fd > eventLoop->maxfd) - eventLoop->maxfd = fd; - return AE_OK; -} - -void aeDeleteFileEvent(aeEventLoop *eventLoop, int fd, int mask) -{ - if (fd >= eventLoop->setsize) return; - aeFileEvent *fe = &eventLoop->events[fd]; - if (fe->mask == AE_NONE) return; - - aeApiDelEvent(eventLoop, fd, mask); - fe->mask = fe->mask & (~mask); - if (fd == eventLoop->maxfd && fe->mask == AE_NONE) { - /* Update the max fd */ - int j; - - for (j = eventLoop->maxfd-1; j >= 0; j--) - if (eventLoop->events[j].mask != AE_NONE) break; - eventLoop->maxfd = j; - } -} - -int aeGetFileEvents(aeEventLoop *eventLoop, int fd) { - if (fd >= eventLoop->setsize) return 0; - aeFileEvent *fe = &eventLoop->events[fd]; - - return fe->mask; -} - -static void aeGetTime(long *seconds, long *milliseconds) -{ - struct timeval tv; - - gettimeofday(&tv, NULL); - *seconds = tv.tv_sec; - *milliseconds = tv.tv_usec/1000; -} - -static void aeAddMillisecondsToNow(long long milliseconds, long *sec, long *ms) { - long cur_sec, cur_ms, when_sec, when_ms; - - aeGetTime(&cur_sec, &cur_ms); - when_sec = cur_sec + milliseconds/1000; - when_ms = cur_ms + milliseconds%1000; - if (when_ms >= 1000) { - when_sec ++; - when_ms -= 1000; - } - *sec = when_sec; - *ms = when_ms; -} - -long long aeCreateTimeEvent(aeEventLoop *eventLoop, long long milliseconds, - aeTimeProc *proc, void *clientData, - aeEventFinalizerProc *finalizerProc) -{ - long long id = eventLoop->timeEventNextId++; - aeTimeEvent *te; - - te = zmalloc(sizeof(*te)); - if (te == NULL) return AE_ERR; - te->id = id; - aeAddMillisecondsToNow(milliseconds,&te->when_sec,&te->when_ms); - te->timeProc = proc; - te->finalizerProc = finalizerProc; - te->clientData = clientData; - te->next = eventLoop->timeEventHead; - eventLoop->timeEventHead = te; - return id; -} - -int aeDeleteTimeEvent(aeEventLoop *eventLoop, long long id) -{ - aeTimeEvent *te = eventLoop->timeEventHead; - while(te) { - if (te->id == id) { - te->id = AE_DELETED_EVENT_ID; - return AE_OK; - } - te = te->next; - } - return AE_ERR; /* NO event with the specified ID found */ -} - -/* Search the first timer to fire. - * This operation is useful to know how many time the select can be - * put in sleep without to delay any event. - * If there are no timers NULL is returned. - * - * Note that's O(N) since time events are unsorted. - * Possible optimizations (not needed by Redis so far, but...): - * 1) Insert the event in order, so that the nearest is just the head. - * Much better but still insertion or deletion of timers is O(N). - * 2) Use a skiplist to have this operation as O(1) and insertion as O(log(N)). - */ -static aeTimeEvent *aeSearchNearestTimer(aeEventLoop *eventLoop) -{ - aeTimeEvent *te = eventLoop->timeEventHead; - aeTimeEvent *nearest = NULL; - - while(te) { - if (!nearest || te->when_sec < nearest->when_sec || - (te->when_sec == nearest->when_sec && - te->when_ms < nearest->when_ms)) - nearest = te; - te = te->next; - } - return nearest; -} - -/* Process time events */ -static int processTimeEvents(aeEventLoop *eventLoop) { - int processed = 0; - aeTimeEvent *te, *prev; - long long maxId; - time_t now = time(NULL); - - /* If the system clock is moved to the future, and then set back to the - * right value, time events may be delayed in a random way. Often this - * means that scheduled operations will not be performed soon enough. - * - * Here we try to detect system clock skews, and force all the time - * events to be processed ASAP when this happens: the idea is that - * processing events earlier is less dangerous than delaying them - * indefinitely, and practice suggests it is. */ - if (now < eventLoop->lastTime) { - te = eventLoop->timeEventHead; - while(te) { - te->when_sec = 0; - te = te->next; - } - } - eventLoop->lastTime = now; - - prev = NULL; - te = eventLoop->timeEventHead; - maxId = eventLoop->timeEventNextId-1; - while(te) { - long now_sec, now_ms; - long long id; - - /* Remove events scheduled for deletion. */ - if (te->id == AE_DELETED_EVENT_ID) { - aeTimeEvent *next = te->next; - if (prev == NULL) - eventLoop->timeEventHead = te->next; - else - prev->next = te->next; - if (te->finalizerProc) - te->finalizerProc(eventLoop, te->clientData); - zfree(te); - te = next; - continue; - } - - /* Make sure we don't process time events created by time events in - * this iteration. Note that this check is currently useless: we always - * add new timers on the head, however if we change the implementation - * detail, this check may be useful again: we keep it here for future - * defense. */ - if (te->id > maxId) { - te = te->next; - continue; - } - aeGetTime(&now_sec, &now_ms); - if (now_sec > te->when_sec || - (now_sec == te->when_sec && now_ms >= te->when_ms)) - { - int retval; - - id = te->id; - retval = te->timeProc(eventLoop, id, te->clientData); - processed++; - if (retval != AE_NOMORE) { - aeAddMillisecondsToNow(retval,&te->when_sec,&te->when_ms); - } else { - te->id = AE_DELETED_EVENT_ID; - } - } - prev = te; - te = te->next; - } - return processed; -} - -/* Process every pending time event, then every pending file event - * (that may be registered by time event callbacks just processed). - * Without special flags the function sleeps until some file event - * fires, or when the next time event occurs (if any). - * - * If flags is 0, the function does nothing and returns. - * if flags has AE_ALL_EVENTS set, all the kind of events are processed. - * if flags has AE_FILE_EVENTS set, file events are processed. - * if flags has AE_TIME_EVENTS set, time events are processed. - * if flags has AE_DONT_WAIT set the function returns ASAP until all - * the events that's possible to process without to wait are processed. - * - * The function returns the number of events processed. */ -int aeProcessEvents(aeEventLoop *eventLoop, int flags) -{ - int processed = 0, numevents; - - /* Nothing to do? return ASAP */ - if (!(flags & AE_TIME_EVENTS) && !(flags & AE_FILE_EVENTS)) return 0; - - /* Note that we want call select() even if there are no - * file events to process as long as we want to process time - * events, in order to sleep until the next time event is ready - * to fire. */ - if (eventLoop->maxfd != -1 || - ((flags & AE_TIME_EVENTS) && !(flags & AE_DONT_WAIT))) { - int j; - aeTimeEvent *shortest = NULL; - struct timeval tv, *tvp; - - if (flags & AE_TIME_EVENTS && !(flags & AE_DONT_WAIT)) - shortest = aeSearchNearestTimer(eventLoop); - if (shortest) { - long now_sec, now_ms; - - aeGetTime(&now_sec, &now_ms); - tvp = &tv; - - /* How many milliseconds we need to wait for the next - * time event to fire? */ - long long ms = - (shortest->when_sec - now_sec)*1000 + - shortest->when_ms - now_ms; - - if (ms > 0) { - tvp->tv_sec = ms/1000; - tvp->tv_usec = (ms % 1000)*1000; - } else { - tvp->tv_sec = 0; - tvp->tv_usec = 0; - } - } else { - /* If we have to check for events but need to return - * ASAP because of AE_DONT_WAIT we need to set the timeout - * to zero */ - if (flags & AE_DONT_WAIT) { - tv.tv_sec = tv.tv_usec = 0; - tvp = &tv; - } else { - /* Otherwise we can block */ - tvp = NULL; /* wait forever */ - } - } - - numevents = aeApiPoll(eventLoop, tvp); - for (j = 0; j < numevents; j++) { - aeFileEvent *fe = &eventLoop->events[eventLoop->fired[j].fd]; - int mask = eventLoop->fired[j].mask; - int fd = eventLoop->fired[j].fd; - int rfired = 0; - - /* note the fe->mask & mask & ... code: maybe an already processed - * event removed an element that fired and we still didn't - * processed, so we check if the event is still valid. */ - if (fe->mask & mask & AE_READABLE) { - rfired = 1; - fe->rfileProc(eventLoop,fd,fe->clientData,mask); - } - if (fe->mask & mask & AE_WRITABLE) { - if (!rfired || fe->wfileProc != fe->rfileProc) - fe->wfileProc(eventLoop,fd,fe->clientData,mask); - } - processed++; - } - } - /* Check time events */ - if (flags & AE_TIME_EVENTS) - processed += processTimeEvents(eventLoop); - - return processed; /* return the number of processed file/time events */ -} - -/* Wait for milliseconds until the given file descriptor becomes - * writable/readable/exception */ -int aeWait(int fd, int mask, long long milliseconds) { - struct pollfd pfd; - int retmask = 0, retval; - - memset(&pfd, 0, sizeof(pfd)); - pfd.fd = fd; - if (mask & AE_READABLE) pfd.events |= POLLIN; - if (mask & AE_WRITABLE) pfd.events |= POLLOUT; - - if ((retval = poll(&pfd, 1, milliseconds))== 1) { - if (pfd.revents & POLLIN) retmask |= AE_READABLE; - if (pfd.revents & POLLOUT) retmask |= AE_WRITABLE; - if (pfd.revents & POLLERR) retmask |= AE_WRITABLE; - if (pfd.revents & POLLHUP) retmask |= AE_WRITABLE; - return retmask; - } else { - return retval; - } -} - -void aeMain(aeEventLoop *eventLoop) { - eventLoop->stop = 0; - while (!eventLoop->stop) { - if (eventLoop->beforesleep != NULL) - eventLoop->beforesleep(eventLoop); - aeProcessEvents(eventLoop, AE_ALL_EVENTS); - } -} - -char *aeGetApiName(void) { - return aeApiName(); -} - -void aeSetBeforeSleepProc(aeEventLoop *eventLoop, aeBeforeSleepProc *beforesleep) { - eventLoop->beforesleep = beforesleep; -} diff --git a/cpp/src/plasma/thirdparty/ae/ae.h b/cpp/src/plasma/thirdparty/ae/ae.h deleted file mode 100644 index 827c4c9e4e5..00000000000 --- a/cpp/src/plasma/thirdparty/ae/ae.h +++ /dev/null @@ -1,123 +0,0 @@ -/* A simple event-driven programming library. Originally I wrote this code - * for the Jim's event-loop (Jim is a Tcl interpreter) but later translated - * it in form of a library for easy reuse. - * - * Copyright (c) 2006-2012, Salvatore Sanfilippo - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Redis nor the names of its contributors may be used - * to endorse or promote products derived from this software without - * specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - -#ifndef __AE_H__ -#define __AE_H__ - -#include - -#define AE_OK 0 -#define AE_ERR -1 - -#define AE_NONE 0 -#define AE_READABLE 1 -#define AE_WRITABLE 2 - -#define AE_FILE_EVENTS 1 -#define AE_TIME_EVENTS 2 -#define AE_ALL_EVENTS (AE_FILE_EVENTS|AE_TIME_EVENTS) -#define AE_DONT_WAIT 4 - -#define AE_NOMORE -1 -#define AE_DELETED_EVENT_ID -1 - -/* Macros */ -#define AE_NOTUSED(V) ((void) V) - -struct aeEventLoop; - -/* Types and data structures */ -typedef void aeFileProc(struct aeEventLoop *eventLoop, int fd, void *clientData, int mask); -typedef int aeTimeProc(struct aeEventLoop *eventLoop, long long id, void *clientData); -typedef void aeEventFinalizerProc(struct aeEventLoop *eventLoop, void *clientData); -typedef void aeBeforeSleepProc(struct aeEventLoop *eventLoop); - -/* File event structure */ -typedef struct aeFileEvent { - int mask; /* one of AE_(READABLE|WRITABLE) */ - aeFileProc *rfileProc; - aeFileProc *wfileProc; - void *clientData; -} aeFileEvent; - -/* Time event structure */ -typedef struct aeTimeEvent { - long long id; /* time event identifier. */ - long when_sec; /* seconds */ - long when_ms; /* milliseconds */ - aeTimeProc *timeProc; - aeEventFinalizerProc *finalizerProc; - void *clientData; - struct aeTimeEvent *next; -} aeTimeEvent; - -/* A fired event */ -typedef struct aeFiredEvent { - int fd; - int mask; -} aeFiredEvent; - -/* State of an event based program */ -typedef struct aeEventLoop { - int maxfd; /* highest file descriptor currently registered */ - int setsize; /* max number of file descriptors tracked */ - long long timeEventNextId; - time_t lastTime; /* Used to detect system clock skew */ - aeFileEvent *events; /* Registered events */ - aeFiredEvent *fired; /* Fired events */ - aeTimeEvent *timeEventHead; - int stop; - void *apidata; /* This is used for polling API specific data */ - aeBeforeSleepProc *beforesleep; -} aeEventLoop; - -/* Prototypes */ -aeEventLoop *aeCreateEventLoop(int setsize); -void aeDeleteEventLoop(aeEventLoop *eventLoop); -void aeStop(aeEventLoop *eventLoop); -int aeCreateFileEvent(aeEventLoop *eventLoop, int fd, int mask, - aeFileProc *proc, void *clientData); -void aeDeleteFileEvent(aeEventLoop *eventLoop, int fd, int mask); -int aeGetFileEvents(aeEventLoop *eventLoop, int fd); -long long aeCreateTimeEvent(aeEventLoop *eventLoop, long long milliseconds, - aeTimeProc *proc, void *clientData, - aeEventFinalizerProc *finalizerProc); -int aeDeleteTimeEvent(aeEventLoop *eventLoop, long long id); -int aeProcessEvents(aeEventLoop *eventLoop, int flags); -int aeWait(int fd, int mask, long long milliseconds); -void aeMain(aeEventLoop *eventLoop); -char *aeGetApiName(void); -void aeSetBeforeSleepProc(aeEventLoop *eventLoop, aeBeforeSleepProc *beforesleep); -int aeGetSetSize(aeEventLoop *eventLoop); -int aeResizeSetSize(aeEventLoop *eventLoop, int setsize); - -#endif diff --git a/cpp/src/plasma/thirdparty/ae/ae_epoll.c b/cpp/src/plasma/thirdparty/ae/ae_epoll.c deleted file mode 100644 index 2f70550a980..00000000000 --- a/cpp/src/plasma/thirdparty/ae/ae_epoll.c +++ /dev/null @@ -1,137 +0,0 @@ -/* Linux epoll(2) based ae.c module - * - * Copyright (c) 2009-2012, Salvatore Sanfilippo - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Redis nor the names of its contributors may be used - * to endorse or promote products derived from this software without - * specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - - -#include - -typedef struct aeApiState { - int epfd; - struct epoll_event *events; -} aeApiState; - -static int aeApiCreate(aeEventLoop *eventLoop) { - aeApiState *state = zmalloc(sizeof(aeApiState)); - - if (!state) return -1; - state->events = zmalloc(sizeof(struct epoll_event)*eventLoop->setsize); - if (!state->events) { - zfree(state); - return -1; - } - state->epfd = epoll_create(1024); /* 1024 is just a hint for the kernel */ - if (state->epfd == -1) { - zfree(state->events); - zfree(state); - return -1; - } - eventLoop->apidata = state; - return 0; -} - -static int aeApiResize(aeEventLoop *eventLoop, int setsize) { - aeApiState *state = eventLoop->apidata; - - state->events = zrealloc(state->events, sizeof(struct epoll_event)*setsize); - return 0; -} - -static void aeApiFree(aeEventLoop *eventLoop) { - aeApiState *state = eventLoop->apidata; - - close(state->epfd); - zfree(state->events); - zfree(state); -} - -static int aeApiAddEvent(aeEventLoop *eventLoop, int fd, int mask) { - aeApiState *state = eventLoop->apidata; - struct epoll_event ee; - memset(&ee, 0, sizeof(struct epoll_event)); // avoid valgrind warning - /* If the fd was already monitored for some event, we need a MOD - * operation. Otherwise we need an ADD operation. */ - int op = eventLoop->events[fd].mask == AE_NONE ? - EPOLL_CTL_ADD : EPOLL_CTL_MOD; - - ee.events = 0; - mask |= eventLoop->events[fd].mask; /* Merge old events */ - if (mask & AE_READABLE) ee.events |= EPOLLIN; - if (mask & AE_WRITABLE) ee.events |= EPOLLOUT; - ee.data.fd = fd; - if (epoll_ctl(state->epfd,op,fd,&ee) == -1) return -1; - return 0; -} - -static void aeApiDelEvent(aeEventLoop *eventLoop, int fd, int delmask) { - aeApiState *state = eventLoop->apidata; - struct epoll_event ee; - memset(&ee, 0, sizeof(struct epoll_event)); // avoid valgrind warning - int mask = eventLoop->events[fd].mask & (~delmask); - - ee.events = 0; - if (mask & AE_READABLE) ee.events |= EPOLLIN; - if (mask & AE_WRITABLE) ee.events |= EPOLLOUT; - ee.data.fd = fd; - if (mask != AE_NONE) { - epoll_ctl(state->epfd,EPOLL_CTL_MOD,fd,&ee); - } else { - /* Note, Kernel < 2.6.9 requires a non null event pointer even for - * EPOLL_CTL_DEL. */ - epoll_ctl(state->epfd,EPOLL_CTL_DEL,fd,&ee); - } -} - -static int aeApiPoll(aeEventLoop *eventLoop, struct timeval *tvp) { - aeApiState *state = eventLoop->apidata; - int retval, numevents = 0; - - retval = epoll_wait(state->epfd,state->events,eventLoop->setsize, - tvp ? (tvp->tv_sec*1000 + tvp->tv_usec/1000) : -1); - if (retval > 0) { - int j; - - numevents = retval; - for (j = 0; j < numevents; j++) { - int mask = 0; - struct epoll_event *e = state->events+j; - - if (e->events & EPOLLIN) mask |= AE_READABLE; - if (e->events & EPOLLOUT) mask |= AE_WRITABLE; - if (e->events & EPOLLERR) mask |= AE_WRITABLE; - if (e->events & EPOLLHUP) mask |= AE_WRITABLE; - eventLoop->fired[j].fd = e->data.fd; - eventLoop->fired[j].mask = mask; - } - } - return numevents; -} - -static char *aeApiName(void) { - return "epoll"; -} diff --git a/cpp/src/plasma/thirdparty/ae/ae_evport.c b/cpp/src/plasma/thirdparty/ae/ae_evport.c deleted file mode 100644 index 5c317becb6f..00000000000 --- a/cpp/src/plasma/thirdparty/ae/ae_evport.c +++ /dev/null @@ -1,320 +0,0 @@ -/* ae.c module for illumos event ports. - * - * Copyright (c) 2012, Joyent, Inc. All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Redis nor the names of its contributors may be used - * to endorse or promote products derived from this software without - * specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - - -#include -#include -#include -#include - -#include -#include - -#include - -static int evport_debug = 0; - -/* - * This file implements the ae API using event ports, present on Solaris-based - * systems since Solaris 10. Using the event port interface, we associate file - * descriptors with the port. Each association also includes the set of poll(2) - * events that the consumer is interested in (e.g., POLLIN and POLLOUT). - * - * There's one tricky piece to this implementation: when we return events via - * aeApiPoll, the corresponding file descriptors become dissociated from the - * port. This is necessary because poll events are level-triggered, so if the - * fd didn't become dissociated, it would immediately fire another event since - * the underlying state hasn't changed yet. We must re-associate the file - * descriptor, but only after we know that our caller has actually read from it. - * The ae API does not tell us exactly when that happens, but we do know that - * it must happen by the time aeApiPoll is called again. Our solution is to - * keep track of the last fds returned by aeApiPoll and re-associate them next - * time aeApiPoll is invoked. - * - * To summarize, in this module, each fd association is EITHER (a) represented - * only via the in-kernel association OR (b) represented by pending_fds and - * pending_masks. (b) is only true for the last fds we returned from aeApiPoll, - * and only until we enter aeApiPoll again (at which point we restore the - * in-kernel association). - */ -#define MAX_EVENT_BATCHSZ 512 - -typedef struct aeApiState { - int portfd; /* event port */ - int npending; /* # of pending fds */ - int pending_fds[MAX_EVENT_BATCHSZ]; /* pending fds */ - int pending_masks[MAX_EVENT_BATCHSZ]; /* pending fds' masks */ -} aeApiState; - -static int aeApiCreate(aeEventLoop *eventLoop) { - int i; - aeApiState *state = zmalloc(sizeof(aeApiState)); - if (!state) return -1; - - state->portfd = port_create(); - if (state->portfd == -1) { - zfree(state); - return -1; - } - - state->npending = 0; - - for (i = 0; i < MAX_EVENT_BATCHSZ; i++) { - state->pending_fds[i] = -1; - state->pending_masks[i] = AE_NONE; - } - - eventLoop->apidata = state; - return 0; -} - -static int aeApiResize(aeEventLoop *eventLoop, int setsize) { - /* Nothing to resize here. */ - return 0; -} - -static void aeApiFree(aeEventLoop *eventLoop) { - aeApiState *state = eventLoop->apidata; - - close(state->portfd); - zfree(state); -} - -static int aeApiLookupPending(aeApiState *state, int fd) { - int i; - - for (i = 0; i < state->npending; i++) { - if (state->pending_fds[i] == fd) - return (i); - } - - return (-1); -} - -/* - * Helper function to invoke port_associate for the given fd and mask. - */ -static int aeApiAssociate(const char *where, int portfd, int fd, int mask) { - int events = 0; - int rv, err; - - if (mask & AE_READABLE) - events |= POLLIN; - if (mask & AE_WRITABLE) - events |= POLLOUT; - - if (evport_debug) - fprintf(stderr, "%s: port_associate(%d, 0x%x) = ", where, fd, events); - - rv = port_associate(portfd, PORT_SOURCE_FD, fd, events, - (void *)(uintptr_t)mask); - err = errno; - - if (evport_debug) - fprintf(stderr, "%d (%s)\n", rv, rv == 0 ? "no error" : strerror(err)); - - if (rv == -1) { - fprintf(stderr, "%s: port_associate: %s\n", where, strerror(err)); - - if (err == EAGAIN) - fprintf(stderr, "aeApiAssociate: event port limit exceeded."); - } - - return rv; -} - -static int aeApiAddEvent(aeEventLoop *eventLoop, int fd, int mask) { - aeApiState *state = eventLoop->apidata; - int fullmask, pfd; - - if (evport_debug) - fprintf(stderr, "aeApiAddEvent: fd %d mask 0x%x\n", fd, mask); - - /* - * Since port_associate's "events" argument replaces any existing events, we - * must be sure to include whatever events are already associated when - * we call port_associate() again. - */ - fullmask = mask | eventLoop->events[fd].mask; - pfd = aeApiLookupPending(state, fd); - - if (pfd != -1) { - /* - * This fd was recently returned from aeApiPoll. It should be safe to - * assume that the consumer has processed that poll event, but we play - * it safer by simply updating pending_mask. The fd will be - * re-associated as usual when aeApiPoll is called again. - */ - if (evport_debug) - fprintf(stderr, "aeApiAddEvent: adding to pending fd %d\n", fd); - state->pending_masks[pfd] |= fullmask; - return 0; - } - - return (aeApiAssociate("aeApiAddEvent", state->portfd, fd, fullmask)); -} - -static void aeApiDelEvent(aeEventLoop *eventLoop, int fd, int mask) { - aeApiState *state = eventLoop->apidata; - int fullmask, pfd; - - if (evport_debug) - fprintf(stderr, "del fd %d mask 0x%x\n", fd, mask); - - pfd = aeApiLookupPending(state, fd); - - if (pfd != -1) { - if (evport_debug) - fprintf(stderr, "deleting event from pending fd %d\n", fd); - - /* - * This fd was just returned from aeApiPoll, so it's not currently - * associated with the port. All we need to do is update - * pending_mask appropriately. - */ - state->pending_masks[pfd] &= ~mask; - - if (state->pending_masks[pfd] == AE_NONE) - state->pending_fds[pfd] = -1; - - return; - } - - /* - * The fd is currently associated with the port. Like with the add case - * above, we must look at the full mask for the file descriptor before - * updating that association. We don't have a good way of knowing what the - * events are without looking into the eventLoop state directly. We rely on - * the fact that our caller has already updated the mask in the eventLoop. - */ - - fullmask = eventLoop->events[fd].mask; - if (fullmask == AE_NONE) { - /* - * We're removing *all* events, so use port_dissociate to remove the - * association completely. Failure here indicates a bug. - */ - if (evport_debug) - fprintf(stderr, "aeApiDelEvent: port_dissociate(%d)\n", fd); - - if (port_dissociate(state->portfd, PORT_SOURCE_FD, fd) != 0) { - perror("aeApiDelEvent: port_dissociate"); - abort(); /* will not return */ - } - } else if (aeApiAssociate("aeApiDelEvent", state->portfd, fd, - fullmask) != 0) { - /* - * ENOMEM is a potentially transient condition, but the kernel won't - * generally return it unless things are really bad. EAGAIN indicates - * we've reached an resource limit, for which it doesn't make sense to - * retry (counter-intuitively). All other errors indicate a bug. In any - * of these cases, the best we can do is to abort. - */ - abort(); /* will not return */ - } -} - -static int aeApiPoll(aeEventLoop *eventLoop, struct timeval *tvp) { - aeApiState *state = eventLoop->apidata; - struct timespec timeout, *tsp; - int mask, i; - uint_t nevents; - port_event_t event[MAX_EVENT_BATCHSZ]; - - /* - * If we've returned fd events before, we must re-associate them with the - * port now, before calling port_get(). See the block comment at the top of - * this file for an explanation of why. - */ - for (i = 0; i < state->npending; i++) { - if (state->pending_fds[i] == -1) - /* This fd has since been deleted. */ - continue; - - if (aeApiAssociate("aeApiPoll", state->portfd, - state->pending_fds[i], state->pending_masks[i]) != 0) { - /* See aeApiDelEvent for why this case is fatal. */ - abort(); - } - - state->pending_masks[i] = AE_NONE; - state->pending_fds[i] = -1; - } - - state->npending = 0; - - if (tvp != NULL) { - timeout.tv_sec = tvp->tv_sec; - timeout.tv_nsec = tvp->tv_usec * 1000; - tsp = &timeout; - } else { - tsp = NULL; - } - - /* - * port_getn can return with errno == ETIME having returned some events (!). - * So if we get ETIME, we check nevents, too. - */ - nevents = 1; - if (port_getn(state->portfd, event, MAX_EVENT_BATCHSZ, &nevents, - tsp) == -1 && (errno != ETIME || nevents == 0)) { - if (errno == ETIME || errno == EINTR) - return 0; - - /* Any other error indicates a bug. */ - perror("aeApiPoll: port_get"); - abort(); - } - - state->npending = nevents; - - for (i = 0; i < nevents; i++) { - mask = 0; - if (event[i].portev_events & POLLIN) - mask |= AE_READABLE; - if (event[i].portev_events & POLLOUT) - mask |= AE_WRITABLE; - - eventLoop->fired[i].fd = event[i].portev_object; - eventLoop->fired[i].mask = mask; - - if (evport_debug) - fprintf(stderr, "aeApiPoll: fd %d mask 0x%x\n", - (int)event[i].portev_object, mask); - - state->pending_fds[i] = event[i].portev_object; - state->pending_masks[i] = (uintptr_t)event[i].portev_user; - } - - return nevents; -} - -static char *aeApiName(void) { - return "evport"; -} diff --git a/cpp/src/plasma/thirdparty/ae/ae_kqueue.c b/cpp/src/plasma/thirdparty/ae/ae_kqueue.c deleted file mode 100644 index 6796f4ceb59..00000000000 --- a/cpp/src/plasma/thirdparty/ae/ae_kqueue.c +++ /dev/null @@ -1,138 +0,0 @@ -/* Kqueue(2)-based ae.c module - * - * Copyright (C) 2009 Harish Mallipeddi - harish.mallipeddi@gmail.com - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Redis nor the names of its contributors may be used - * to endorse or promote products derived from this software without - * specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - - -#include -#include -#include - -typedef struct aeApiState { - int kqfd; - struct kevent *events; -} aeApiState; - -static int aeApiCreate(aeEventLoop *eventLoop) { - aeApiState *state = zmalloc(sizeof(aeApiState)); - - if (!state) return -1; - state->events = zmalloc(sizeof(struct kevent)*eventLoop->setsize); - if (!state->events) { - zfree(state); - return -1; - } - state->kqfd = kqueue(); - if (state->kqfd == -1) { - zfree(state->events); - zfree(state); - return -1; - } - eventLoop->apidata = state; - return 0; -} - -static int aeApiResize(aeEventLoop *eventLoop, int setsize) { - aeApiState *state = eventLoop->apidata; - - state->events = zrealloc(state->events, sizeof(struct kevent)*setsize); - return 0; -} - -static void aeApiFree(aeEventLoop *eventLoop) { - aeApiState *state = eventLoop->apidata; - - close(state->kqfd); - zfree(state->events); - zfree(state); -} - -static int aeApiAddEvent(aeEventLoop *eventLoop, int fd, int mask) { - aeApiState *state = eventLoop->apidata; - struct kevent ke; - - if (mask & AE_READABLE) { - EV_SET(&ke, fd, EVFILT_READ, EV_ADD, 0, 0, NULL); - if (kevent(state->kqfd, &ke, 1, NULL, 0, NULL) == -1) return -1; - } - if (mask & AE_WRITABLE) { - EV_SET(&ke, fd, EVFILT_WRITE, EV_ADD, 0, 0, NULL); - if (kevent(state->kqfd, &ke, 1, NULL, 0, NULL) == -1) return -1; - } - return 0; -} - -static void aeApiDelEvent(aeEventLoop *eventLoop, int fd, int mask) { - aeApiState *state = eventLoop->apidata; - struct kevent ke; - - if (mask & AE_READABLE) { - EV_SET(&ke, fd, EVFILT_READ, EV_DELETE, 0, 0, NULL); - kevent(state->kqfd, &ke, 1, NULL, 0, NULL); - } - if (mask & AE_WRITABLE) { - EV_SET(&ke, fd, EVFILT_WRITE, EV_DELETE, 0, 0, NULL); - kevent(state->kqfd, &ke, 1, NULL, 0, NULL); - } -} - -static int aeApiPoll(aeEventLoop *eventLoop, struct timeval *tvp) { - aeApiState *state = eventLoop->apidata; - int retval, numevents = 0; - - if (tvp != NULL) { - struct timespec timeout; - timeout.tv_sec = tvp->tv_sec; - timeout.tv_nsec = tvp->tv_usec * 1000; - retval = kevent(state->kqfd, NULL, 0, state->events, eventLoop->setsize, - &timeout); - } else { - retval = kevent(state->kqfd, NULL, 0, state->events, eventLoop->setsize, - NULL); - } - - if (retval > 0) { - int j; - - numevents = retval; - for(j = 0; j < numevents; j++) { - int mask = 0; - struct kevent *e = state->events+j; - - if (e->filter == EVFILT_READ) mask |= AE_READABLE; - if (e->filter == EVFILT_WRITE) mask |= AE_WRITABLE; - eventLoop->fired[j].fd = e->ident; - eventLoop->fired[j].mask = mask; - } - } - return numevents; -} - -static char *aeApiName(void) { - return "kqueue"; -} diff --git a/cpp/src/plasma/thirdparty/ae/ae_select.c b/cpp/src/plasma/thirdparty/ae/ae_select.c deleted file mode 100644 index c039a8ea312..00000000000 --- a/cpp/src/plasma/thirdparty/ae/ae_select.c +++ /dev/null @@ -1,106 +0,0 @@ -/* Select()-based ae.c module. - * - * Copyright (c) 2009-2012, Salvatore Sanfilippo - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Redis nor the names of its contributors may be used - * to endorse or promote products derived from this software without - * specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - - -#include -#include - -typedef struct aeApiState { - fd_set rfds, wfds; - /* We need to have a copy of the fd sets as it's not safe to reuse - * FD sets after select(). */ - fd_set _rfds, _wfds; -} aeApiState; - -static int aeApiCreate(aeEventLoop *eventLoop) { - aeApiState *state = zmalloc(sizeof(aeApiState)); - - if (!state) return -1; - FD_ZERO(&state->rfds); - FD_ZERO(&state->wfds); - eventLoop->apidata = state; - return 0; -} - -static int aeApiResize(aeEventLoop *eventLoop, int setsize) { - /* Just ensure we have enough room in the fd_set type. */ - if (setsize >= FD_SETSIZE) return -1; - return 0; -} - -static void aeApiFree(aeEventLoop *eventLoop) { - zfree(eventLoop->apidata); -} - -static int aeApiAddEvent(aeEventLoop *eventLoop, int fd, int mask) { - aeApiState *state = eventLoop->apidata; - - if (mask & AE_READABLE) FD_SET(fd,&state->rfds); - if (mask & AE_WRITABLE) FD_SET(fd,&state->wfds); - return 0; -} - -static void aeApiDelEvent(aeEventLoop *eventLoop, int fd, int mask) { - aeApiState *state = eventLoop->apidata; - - if (mask & AE_READABLE) FD_CLR(fd,&state->rfds); - if (mask & AE_WRITABLE) FD_CLR(fd,&state->wfds); -} - -static int aeApiPoll(aeEventLoop *eventLoop, struct timeval *tvp) { - aeApiState *state = eventLoop->apidata; - int retval, j, numevents = 0; - - memcpy(&state->_rfds,&state->rfds,sizeof(fd_set)); - memcpy(&state->_wfds,&state->wfds,sizeof(fd_set)); - - retval = select(eventLoop->maxfd+1, - &state->_rfds,&state->_wfds,NULL,tvp); - if (retval > 0) { - for (j = 0; j <= eventLoop->maxfd; j++) { - int mask = 0; - aeFileEvent *fe = &eventLoop->events[j]; - - if (fe->mask == AE_NONE) continue; - if (fe->mask & AE_READABLE && FD_ISSET(j,&state->_rfds)) - mask |= AE_READABLE; - if (fe->mask & AE_WRITABLE && FD_ISSET(j,&state->_wfds)) - mask |= AE_WRITABLE; - eventLoop->fired[numevents].fd = j; - eventLoop->fired[numevents].mask = mask; - numevents++; - } - } - return numevents; -} - -static char *aeApiName(void) { - return "select"; -} diff --git a/cpp/src/plasma/thirdparty/ae/config.h b/cpp/src/plasma/thirdparty/ae/config.h deleted file mode 100644 index 4f8e1ea1bc3..00000000000 --- a/cpp/src/plasma/thirdparty/ae/config.h +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright (c) 2009-2012, Salvatore Sanfilippo - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Redis nor the names of its contributors may be used - * to endorse or promote products derived from this software without - * specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - -#ifndef __CONFIG_H -#define __CONFIG_H - -#ifdef __APPLE__ -#include -#endif - -/* Test for polling API */ -#ifdef __linux__ -#define HAVE_EPOLL 1 -#endif - -#if (defined(__APPLE__) && defined(MAC_OS_X_VERSION_10_6)) || defined(__FreeBSD__) || defined(__OpenBSD__) || defined (__NetBSD__) -#define HAVE_KQUEUE 1 -#endif - -#ifdef __sun -#include -#ifdef _DTRACE_VERSION -#define HAVE_EVPORT 1 -#endif -#endif - - -#endif diff --git a/cpp/src/plasma/thirdparty/ae/zmalloc.h b/cpp/src/plasma/thirdparty/ae/zmalloc.h deleted file mode 100644 index 6c27dd4e5c3..00000000000 --- a/cpp/src/plasma/thirdparty/ae/zmalloc.h +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Copyright (c) 2009-2012, Salvatore Sanfilippo - * All rights reserved. - * - * Redistribution and use in source and binary forms, with or without - * modification, are permitted provided that the following conditions are met: - * - * * Redistributions of source code must retain the above copyright notice, - * this list of conditions and the following disclaimer. - * * Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in the - * documentation and/or other materials provided with the distribution. - * * Neither the name of Redis nor the names of its contributors may be used - * to endorse or promote products derived from this software without - * specific prior written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ - -#ifndef _ZMALLOC_H -#define _ZMALLOC_H - -#ifndef zmalloc -#define zmalloc malloc -#endif - -#ifndef zfree -#define zfree free -#endif - -#ifndef zrealloc -#define zrealloc realloc -#endif - -#endif /* _ZMALLOC_H */ diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt index 0d45bc1d952..1f125597633 100644 --- a/dev/release/rat_exclude_files.txt +++ b/dev/release/rat_exclude_files.txt @@ -23,14 +23,6 @@ cpp/cmake_modules/SnappyCMakeLists.txt cpp/cmake_modules/SnappyConfig.h cpp/examples/parquet/parquet-arrow/cmake_modules/FindArrow.cmake cpp/src/parquet/.parquetcppversion -cpp/src/plasma/thirdparty/ae/ae.c -cpp/src/plasma/thirdparty/ae/ae.h -cpp/src/plasma/thirdparty/ae/ae_epoll.c -cpp/src/plasma/thirdparty/ae/ae_evport.c -cpp/src/plasma/thirdparty/ae/ae_kqueue.c -cpp/src/plasma/thirdparty/ae/ae_select.c -cpp/src/plasma/thirdparty/ae/config.h -cpp/src/plasma/thirdparty/ae/zmalloc.h cpp/src/plasma/thirdparty/dlmalloc.c cpp/thirdparty/flatbuffers/include/flatbuffers/base.h cpp/thirdparty/flatbuffers/include/flatbuffers/flatbuffers.h diff --git a/python/pyarrow/_plasma.pyx b/python/pyarrow/_plasma.pyx index 68e889a78bf..93b98bd2731 100644 --- a/python/pyarrow/_plasma.pyx +++ b/python/pyarrow/_plasma.pyx @@ -20,8 +20,6 @@ # cython: embedsignature = True # cython: language_level = 3 -from __future__ import absolute_import - from libcpp cimport bool as c_bool, nullptr from libcpp.memory cimport shared_ptr, unique_ptr, make_shared from libcpp.string cimport string as c_string @@ -31,6 +29,7 @@ from libc.stdint cimport int64_t, uint8_t, uintptr_t from cython.operator cimport dereference as deref, preincrement as inc from cpython.pycapsule cimport * +import collections import random import socket import warnings @@ -125,16 +124,18 @@ cdef extern from "plasma/client.h" nogil: CStatus List(CObjectTable* objects) - CStatus Subscribe(int* fd) + CStatus Subscribe() CStatus DecodeNotifications(const uint8_t* buffer, c_vector[CUniqueID]* object_ids, c_vector[int64_t]* data_sizes, c_vector[int64_t]* metadata_sizes) - CStatus GetNotification(int fd, CUniqueID* object_id, + CStatus GetNotification(CUniqueID* object_id, int64_t* data_size, int64_t* metadata_size) + int GetNativeNotificationHandle() + CStatus Disconnect() CStatus Delete(const c_vector[CUniqueID] object_ids) @@ -302,12 +303,10 @@ cdef class PlasmaClient: cdef: shared_ptr[CPlasmaClient] client - int notification_fd c_string store_socket_name def __cinit__(self): self.client.reset(new CPlasmaClient()) - self.notification_fd = -1 self.store_socket_name = b"" cdef _get_object_buffers(self, object_ids, int64_t timeout_ms, @@ -563,7 +562,7 @@ cdef class PlasmaClient: the object_ids and ObjectNotAvailable if the object was not available. """ - if isinstance(object_ids, compat.Sequence): + if isinstance(object_ids, collections.Sequence): results = [] buffers = self.get_buffers(object_ids, timeout_ms) for i in range(len(object_ids)): @@ -662,14 +661,14 @@ cdef class PlasmaClient: def subscribe(self): """Subscribe to notifications about sealed objects.""" with nogil: - plasma_check_status( - self.client.get().Subscribe(&self.notification_fd)) + plasma_check_status(self.client.get().Subscribe()) def get_notification_socket(self): """ Get the notification socket. """ - return compat.get_socket_from_fd(self.notification_fd, + cdef int fd = self.client.get().GetNativeNotificationHandle() + return compat.get_socket_from_fd(fd, family=socket.AF_UNIX, type=socket.SOCK_STREAM) @@ -717,8 +716,7 @@ cdef class PlasmaClient: cdef int64_t data_size cdef int64_t metadata_size with nogil: - status = self.client.get().GetNotification(self.notification_fd, - &object_id.data, + status = self.client.get().GetNotification(&object_id.data, &data_size, &metadata_size) plasma_check_status(status) From 8d3734d5feadfe38474b502a4ac59ffe265522ce Mon Sep 17 00:00:00 2001 From: Siyuan Date: Sat, 26 Oct 2019 00:00:35 -0700 Subject: [PATCH 2/7] revert some code overwritten by rebase --- cpp/src/arrow/status.h | 171 +++++++++++---------- cpp/src/plasma/CMakeLists.txt | 18 +-- cpp/src/plasma/protocol.cc | 70 +++++++-- cpp/src/plasma/test/serialization_tests.cc | 4 +- python/pyarrow/_plasma.pyx | 2 + 5 files changed, 164 insertions(+), 101 deletions(-) diff --git a/cpp/src/arrow/status.h b/cpp/src/arrow/status.h index 984d8b541d2..bf1967014ea 100644 --- a/cpp/src/arrow/status.h +++ b/cpp/src/arrow/status.h @@ -17,13 +17,11 @@ #include #include +#include #include #include -#ifdef ARROW_EXTRA_ERROR_CONTEXT -#include -#endif - +#include "arrow/util/compare.h" #include "arrow/util/macros.h" #include "arrow/util/string_builder.h" #include "arrow/util/visibility.h" @@ -31,14 +29,13 @@ #ifdef ARROW_EXTRA_ERROR_CONTEXT /// \brief Return with given status if condition is met. -#define ARROW_RETURN_IF_(condition, status, expr) \ - do { \ - if (ARROW_PREDICT_FALSE(condition)) { \ - ::arrow::Status _s = (status); \ - std::stringstream ss; \ - ss << _s.message() << "\n" << __FILE__ << ":" << __LINE__ << " code: " << expr; \ - return ::arrow::Status(_s.code(), ss.str()); \ - } \ +#define ARROW_RETURN_IF_(condition, status, expr) \ + do { \ + if (ARROW_PREDICT_FALSE(condition)) { \ + ::arrow::Status _st = (status); \ + _st.AddContextLine(__FILE__, __LINE__, expr); \ + return _st; \ + } \ } while (0) #else @@ -86,21 +83,17 @@ enum class StatusCode : char { Invalid = 4, IOError = 5, CapacityError = 6, + IndexError = 7, UnknownError = 9, NotImplemented = 10, SerializationError = 11, - PythonError = 12, RError = 13, - ProtocolError = 14, - PlasmaObjectExists = 20, - PlasmaObjectNonexistent = 21, - PlasmaStoreFull = 22, - PlasmaObjectAlreadySealed = 23, - StillExecuting = 24, // Gandiva range of errors CodeGenError = 40, ExpressionValidationError = 41, - ExecutionError = 42 + ExecutionError = 42, + // Continue generic codes. + AlreadyExists = 45 }; #if defined(__clang__) @@ -108,6 +101,22 @@ enum class StatusCode : char { class ARROW_MUST_USE_RESULT ARROW_EXPORT Status; #endif +/// \brief An opaque class that allows subsystems to retain +/// additional information inside the Status. +class ARROW_EXPORT StatusDetail { + public: + virtual ~StatusDetail() = default; + /// \brief Return a unique id for the type of the StatusDetail + /// (effectively a poor man's substitude for RTTI). + virtual const char* type_id() const = 0; + /// \brief Produce a human-readable description of this status. + virtual std::string ToString() const = 0; + + bool operator==(const StatusDetail& other) const noexcept { + return std::string(type_id()) == other.type_id() && ToString() == other.ToString(); + } +}; + /// \brief Status outcome object (success or error) /// /// The Status object is an object holding the outcome of an operation. @@ -116,7 +125,8 @@ class ARROW_MUST_USE_RESULT ARROW_EXPORT Status; /// /// Additionally, if an error occurred, a specific error message is generally /// attached. -class ARROW_EXPORT Status { +class ARROW_EXPORT Status : public util::EqualityComparable, + public util::ToStringOstreamable { public: // Create a success status. Status() noexcept : state_(NULLPTR) {} @@ -129,6 +139,8 @@ class ARROW_EXPORT Status { } Status(StatusCode code, const std::string& msg); + /// \brief Pluggable constructor for use by sub-systems. detail cannot be null. + Status(StatusCode code, std::string msg, std::shared_ptr detail); // Copy the specified status. inline Status(const Status& s); @@ -138,6 +150,8 @@ class ARROW_EXPORT Status { inline Status(Status&& s) noexcept; inline Status& operator=(Status&& s) noexcept; + inline bool Equals(const Status& s) const; + // AND the statuses. inline Status operator&(const Status& s) const noexcept; inline Status operator&(Status&& s) const noexcept; @@ -147,12 +161,6 @@ class ARROW_EXPORT Status { /// Return a success status static Status OK() { return Status(); } - /// Return a success status with a specific message - template - static Status OK(Args&&... args) { - return Status(StatusCode::OK, util::StringBuilder(std::forward(args)...)); - } - /// Return an error status for out-of-memory conditions template static Status OutOfMemory(Args&&... args) { @@ -194,6 +202,13 @@ class ARROW_EXPORT Status { return Status(StatusCode::Invalid, util::StringBuilder(std::forward(args)...)); } + /// Return an error status when an index is out of bounds + template + static Status IndexError(Args&&... args) { + return Status(StatusCode::IndexError, + util::StringBuilder(std::forward(args)...)); + } + /// Return an error status when a container's capacity would exceed its limits template static Status CapacityError(Args&&... args) { @@ -219,38 +234,6 @@ class ARROW_EXPORT Status { return Status(StatusCode::RError, util::StringBuilder(std::forward(args)...)); } - template - static Status ProtocolError(Args&&... args) { - return Status(StatusCode::ProtocolError, - util::StringBuilder(std::forward(args)...)); - } - - template - static Status PlasmaObjectExists(Args&&... args) { - return Status(StatusCode::PlasmaObjectExists, - util::StringBuilder(std::forward(args)...)); - } - - template - static Status PlasmaObjectNonexistent(Args&&... args) { - return Status(StatusCode::PlasmaObjectNonexistent, - util::StringBuilder(std::forward(args)...)); - } - - template - static Status PlasmaObjectAlreadySealed(Args&&... args) { - return Status(StatusCode::PlasmaObjectAlreadySealed, - util::StringBuilder(std::forward(args)...)); - } - - template - static Status PlasmaStoreFull(Args&&... args) { - return Status(StatusCode::PlasmaStoreFull, - util::StringBuilder(std::forward(args)...)); - } - - static Status StillExecuting() { return Status(StatusCode::StillExecuting, ""); } - template static Status CodeGenError(Args&&... args) { return Status(StatusCode::CodeGenError, @@ -269,6 +252,12 @@ class ARROW_EXPORT Status { util::StringBuilder(std::forward(args)...)); } + template + static Status AlreadyExists(Args&&... args) { + return Status(StatusCode::AlreadyExists, + util::StringBuilder(std::forward(args)...)); + } + /// Return true iff the status indicates success. bool ok() const { return (state_ == NULLPTR); } @@ -282,6 +271,8 @@ class ARROW_EXPORT Status { bool IsIOError() const { return code() == StatusCode::IOError; } /// Return true iff the status indicates a container reaching capacity limits. bool IsCapacityError() const { return code() == StatusCode::CapacityError; } + /// Return true iff the status indicates an out of bounds index. + bool IsIndexError() const { return code() == StatusCode::IndexError; } /// Return true iff the status indicates a type error. bool IsTypeError() const { return code() == StatusCode::TypeError; } /// Return true iff the status indicates an unknown error. @@ -292,22 +283,6 @@ class ARROW_EXPORT Status { bool IsSerializationError() const { return code() == StatusCode::SerializationError; } /// Return true iff the status indicates a R-originated error. bool IsRError() const { return code() == StatusCode::RError; } - /// Return true iff the status indicates a Python-originated error. - bool IsPythonError() const { return code() == StatusCode::PythonError; } - /// Return true iff the status indicates an already existing Plasma object. - bool IsPlasmaObjectExists() const { return code() == StatusCode::PlasmaObjectExists; } - /// Return true iff the status indicates a non-existent Plasma object. - bool IsPlasmaObjectNonexistent() const { - return code() == StatusCode::PlasmaObjectNonexistent; - } - /// Return true iff the status indicates an already sealed Plasma object. - bool IsPlasmaObjectAlreadySealed() const { - return code() == StatusCode::PlasmaObjectAlreadySealed; - } - /// Return true iff the status indicates the Plasma store reached its capacity limit. - bool IsPlasmaStoreFull() const { return code() == StatusCode::PlasmaStoreFull; } - - bool IsStillExecuting() const { return code() == StatusCode::StillExecuting; } bool IsCodeGenError() const { return code() == StatusCode::CodeGenError; } @@ -332,10 +307,35 @@ class ARROW_EXPORT Status { /// \brief Return the specific error message attached to this status. std::string message() const { return ok() ? "" : state_->msg; } + /// \brief Return the status detail attached to this message. + std::shared_ptr detail() const { + return state_ == NULLPTR ? NULLPTR : state_->detail; + } + + /// \brief Return a new Status copying the existing status, but + /// updating with the existing detail. + Status WithDetail(std::shared_ptr new_detail) const { + return Status(code(), message(), std::move(new_detail)); + } + + /// \brief Return a new Status with changed message, copying the + /// existing status code and detail. + Status WithMessage(std::string message) const { + return Status(code(), std::move(message), detail()); + } + + [[noreturn]] void Abort() const; + [[noreturn]] void Abort(const std::string& message) const; + +#ifdef ARROW_EXTRA_ERROR_CONTEXT + void AddContextLine(const char* filename, int line, const char* expr); +#endif + private: struct State { StatusCode code; std::string msg; + std::shared_ptr detail; }; // OK status has a `NULL` state_. Otherwise, `state_` points to // a `State` structure containing the error code and message(s) @@ -349,11 +349,6 @@ class ARROW_EXPORT Status { inline void MoveFrom(Status& s); }; -static inline std::ostream& operator<<(std::ostream& os, const Status& x) { - os << x.ToString(); - return os; -} - void Status::MoveFrom(Status& s) { delete state_; state_ = s.state_; @@ -379,6 +374,22 @@ Status& Status::operator=(Status&& s) noexcept { return *this; } +bool Status::Equals(const Status& s) const { + if (state_ == s.state_) { + return true; + } + + if (ok() || s.ok()) { + return false; + } + + if (detail() != s.detail() && !(*detail() == *s.detail())) { + return false; + } + + return code() == s.code() && message() == s.message(); +} + /// \cond FALSE // (note: emits warnings on Doxygen < 1.8.15, // see https://github.com/doxygen/doxygen/issues/6295) diff --git a/cpp/src/plasma/CMakeLists.txt b/cpp/src/plasma/CMakeLists.txt index 8e605e13a05..5c6a354e487 100644 --- a/cpp/src/plasma/CMakeLists.txt +++ b/cpp/src/plasma/CMakeLists.txt @@ -145,14 +145,14 @@ endif() list(APPEND PLASMA_EXTERNAL_STORE_SOURCES "external_store.cc" "hash_table_store.cc") -# We use static libraries for the plasma_store_server executable so that it can +# We use static libraries for the plasma-store-server executable so that it can # be copied around and used in different locations. add_executable(plasma-store-server ${PLASMA_EXTERNAL_STORE_SOURCES} ${PLASMA_STORE_SRCS}) if(ARROW_BUILD_STATIC) - target_link_libraries(plasma_store_server plasma_static ${PLASMA_STATIC_LINK_LIBS}) + target_link_libraries(plasma-store-server plasma_static ${PLASMA_STATIC_LINK_LIBS}) else() # Fallback to shared libs in the case that static libraries are not build. - target_link_libraries(plasma_store_server plasma_shared ${PLASMA_LINK_LIBS}) + target_link_libraries(plasma-store-server plasma_shared ${PLASMA_LINK_LIBS}) endif() add_dependencies(plasma plasma-store-server) @@ -162,7 +162,7 @@ if(ARROW_RPATH_ORIGIN) else() set(_lib_install_rpath "\$ORIGIN") endif() - set_target_properties(plasma_store_server + set_target_properties(plasma-store-server PROPERTIES INSTALL_RPATH ${_lib_install_rpath}) elseif(APPLE) # With OSX and conda, we need to set the correct RPATH so that dependencies @@ -171,7 +171,7 @@ elseif(APPLE) # $ENV{CONDA_PREFIX}/lib but our test libraries and executables are not # installed there. if(NOT "$ENV{CONDA_PREFIX}" STREQUAL "" AND APPLE) - set_target_properties(plasma_store_server + set_target_properties(plasma-store-server PROPERTIES BUILD_WITH_INSTALL_RPATH TRUE INSTALL_RPATH_USE_LINK_PATH @@ -189,8 +189,8 @@ install(FILES common.h DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/plasma") # Plasma store -set_target_properties(plasma_store_server PROPERTIES INSTALL_RPATH_USE_LINK_PATH TRUE) -install(TARGETS plasma_store_server ${INSTALL_IS_OPTIONAL} DESTINATION +set_target_properties(plasma-store-server PROPERTIES INSTALL_RPATH_USE_LINK_PATH TRUE) +install(TARGETS plasma-store-server ${INSTALL_IS_OPTIONAL} DESTINATION ${CMAKE_INSTALL_BINDIR}) # pkg-config support @@ -275,9 +275,9 @@ add_plasma_test(test/client_tests EXTRA_LINK_LIBS ${PLASMA_TEST_LIBS} EXTRA_DEPENDENCIES - plasma_store_server) + plasma-store-server) add_plasma_test(test/external_store_tests EXTRA_LINK_LIBS ${PLASMA_TEST_LIBS} EXTRA_DEPENDENCIES - plasma_store_server) + plasma-store-server) diff --git a/cpp/src/plasma/protocol.cc b/cpp/src/plasma/protocol.cc index 9fe340b9e41..b6f08fab86f 100644 --- a/cpp/src/plasma/protocol.cc +++ b/cpp/src/plasma/protocol.cc @@ -66,6 +66,16 @@ void ToVector(const Data& request, std::vector* out, const Getter& getter) { } } +template +void ConvertToVector(const FlatbufferVectorPointer fbvector, std::vector* out, + const Converter& converter) { + out->clear(); + out->reserve(fbvector->size()); + for (size_t i = 0; i < fbvector->size(); ++i) { + out->push_back(converter(*fbvector->Get(i))); + } +} + Status PlasmaErrorStatus(fb::PlasmaError plasma_error) { switch (plasma_error) { case fb::PlasmaError::OK: @@ -203,7 +213,7 @@ Status SendCreateAndSealRequest(const std::shared_ptr& client, Status ReadCreateAndSealRequest(const uint8_t* data, size_t size, ObjectID* object_id, std::string* object_data, std::string* metadata, - unsigned char* digest) { + std::string* digest) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); @@ -212,7 +222,46 @@ Status ReadCreateAndSealRequest(const uint8_t* data, size_t size, ObjectID* obje *object_data = message->data()->str(); *metadata = message->metadata()->str(); ARROW_CHECK(message->digest()->size() == kDigestSize); - memcpy(digest, message->digest()->data(), kDigestSize); + digest->assign(message->digest()->data(), kDigestSize); + return Status::OK(); +} + +Status SendCreateAndSealBatchRequest(int sock, const std::vector& object_ids, + const std::vector& data, + const std::vector& metadata, + const std::vector& digests) { + flatbuffers::FlatBufferBuilder fbb; + + auto message = fb::CreatePlasmaCreateAndSealBatchRequest( + fbb, ToFlatbuffer(&fbb, object_ids.data(), object_ids.size()), + ToFlatbuffer(&fbb, data), ToFlatbuffer(&fbb, metadata), + ToFlatbuffer(&fbb, digests)); + + return PlasmaSend(sock, MessageType::PlasmaCreateAndSealBatchRequest, &fbb, message); +} + +Status ReadCreateAndSealBatchRequest(uint8_t* data, size_t size, + std::vector* object_ids, + std::vector* object_data, + std::vector* metadata, + std::vector* digests) { + DCHECK(data); + auto message = flatbuffers::GetRoot(data); + DCHECK(VerifyFlatbuffer(message, data, size)); + + ConvertToVector(message->object_ids(), object_ids, + [](const flatbuffers::String& element) { + return ObjectID::from_binary(element.str()); + }); + + ConvertToVector(message->data(), object_data, + [](const flatbuffers::String& element) { return element.str(); }); + + ConvertToVector(message->metadata(), metadata, + [](const flatbuffers::String& element) { return element.str(); }); + + ConvertToVector(message->digest(), digests, + [](const flatbuffers::String& element) { return element.str(); }); return Status::OK(); } @@ -645,6 +694,15 @@ Status ReadGetReply(const uint8_t* data, size_t size, ObjectID object_ids[], return Status::OK(); } +// Subscribe messages. + +Status SendSubscribeRequest(const std::shared_ptr& client) { + flatbuffers::FlatBufferBuilder fbb; + auto message = fb::CreatePlasmaSubscribeRequest(fbb); + fbb.Finish(message); + return PlasmaSend(client, MessageType::PlasmaSubscribeRequest, &fbb); +} + // Data messages. Status SendDataRequest(const std::shared_ptr& client, @@ -689,14 +747,6 @@ Status ReadDataReply(const uint8_t* data, size_t size, ObjectID* object_id, return Status::OK(); } -Status SendSubscribeRequest(const std::shared_ptr& client) { - // Subscribe messages. - flatbuffers::FlatBufferBuilder fbb; - auto message = fb::CreatePlasmaSubscribeRequest(fbb); - fbb.Finish(message); - return PlasmaSend(client, MessageType::PlasmaSubscribeRequest, &fbb); -} - void SerializeObjectDeletionNotification(const ObjectID& object_id, std::vector* serialized) { flatbuf::ObjectInfoT info; diff --git a/cpp/src/plasma/test/serialization_tests.cc b/cpp/src/plasma/test/serialization_tests.cc index 58f6020010e..87489cf2296 100644 --- a/cpp/src/plasma/test/serialization_tests.cc +++ b/cpp/src/plasma/test/serialization_tests.cc @@ -24,12 +24,12 @@ #include #include "arrow/testing/gtest_util.h" -#include "arrow/util/io-util.h" +#include "arrow/util/io_util.h" #include "plasma/common.h" #include "plasma/io/connection.h" #include "plasma/protocol.h" -#include "plasma/test-util.h" +#include "plasma/test_util.h" namespace plasma { diff --git a/python/pyarrow/_plasma.pyx b/python/pyarrow/_plasma.pyx index 93b98bd2731..2bbddc9fad4 100644 --- a/python/pyarrow/_plasma.pyx +++ b/python/pyarrow/_plasma.pyx @@ -20,6 +20,8 @@ # cython: embedsignature = True # cython: language_level = 3 +from __future__ import absolute_import + from libcpp cimport bool as c_bool, nullptr from libcpp.memory cimport shared_ptr, unique_ptr, make_shared from libcpp.string cimport string as c_string From 3deac57f233d6663326688bcf860805a790804a3 Mon Sep 17 00:00:00 2001 From: Siyuan Date: Sat, 26 Oct 2019 00:23:54 -0700 Subject: [PATCH 3/7] revert some code overwritten by rebase --- cpp/src/plasma/protocol.cc | 20 +++++++++++++++++--- cpp/src/plasma/protocol.h | 17 +++++++++++++++++ 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/cpp/src/plasma/protocol.cc b/cpp/src/plasma/protocol.cc index b6f08fab86f..eccf15e0006 100644 --- a/cpp/src/plasma/protocol.cc +++ b/cpp/src/plasma/protocol.cc @@ -280,6 +280,20 @@ Status ReadCreateAndSealReply(const uint8_t* data, size_t size) { return PlasmaErrorStatus(message->error()); } +Status SendCreateAndSealBatchReply(int sock, PlasmaError error) { + flatbuffers::FlatBufferBuilder fbb; + auto message = + fb::CreatePlasmaCreateAndSealBatchReply(fbb, static_cast(error)); + return PlasmaSend(sock, MessageType::PlasmaCreateAndSealBatchReply, &fbb, message); +} + +Status ReadCreateAndSealBatchReply(uint8_t* data, size_t size) { + DCHECK(data); + auto message = flatbuffers::GetRoot(data); + DCHECK(VerifyFlatbuffer(message, data, size)); + return PlasmaErrorStatus(message->error()); +} + Status SendAbortRequest(const std::shared_ptr& client, ObjectID object_id) { flatbuffers::FlatBufferBuilder fbb; @@ -325,13 +339,13 @@ Status SendSealRequest(const std::shared_ptr& client, } Status ReadSealRequest(const uint8_t* data, size_t size, ObjectID* object_id, - unsigned char* digest) { + std::string* digest) { DCHECK(data); auto message = flatbuffers::GetRoot(data); DCHECK(VerifyFlatbuffer(message, data, size)); *object_id = ObjectID::from_binary(message->object_id()->str()); - ARROW_CHECK(message->digest()->size() == kDigestSize); - memcpy(digest, message->digest()->data(), kDigestSize); + ARROW_CHECK_EQ(message->digest()->size(), kDigestSize); + digest->assign(message->digest()->data(), kDigestSize); return Status::OK(); } diff --git a/cpp/src/plasma/protocol.h b/cpp/src/plasma/protocol.h index e9abb1935a0..0249a529173 100644 --- a/cpp/src/plasma/protocol.h +++ b/cpp/src/plasma/protocol.h @@ -65,9 +65,26 @@ Status ReadCreateAndSealRequest(const uint8_t* data, size_t size, ObjectID* obje std::string* object_data, std::string* metadata, unsigned char* digest); +// TODO(suquark): Replace raw socket with client handle. +Status SendCreateAndSealBatchRequest(int sock, const std::vector& object_ids, + const std::vector& data, + const std::vector& metadata, + const std::vector& digests); + +Status ReadCreateAndSealBatchRequest(uint8_t* data, size_t size, + std::vector* object_id, + std::vector* object_data, + std::vector* metadata, + std::vector* digests); + +// TODO(suquark): Replace raw socket with client handle. +Status SendCreateAndSealBatchReply(int sock, PlasmaError error); + Status SendCreateAndSealReply(const std::shared_ptr& client, PlasmaError error); +Status ReadCreateAndSealBatchReply(uint8_t* data, size_t size); + Status ReadCreateAndSealReply(const uint8_t* data, size_t size); Status SendAbortRequest(const std::shared_ptr& client, From 2ff694145507a5508fe5628229a9f6b71b9dfb6f Mon Sep 17 00:00:00 2001 From: Siyuan Date: Sat, 26 Oct 2019 00:29:37 -0700 Subject: [PATCH 4/7] revert some code overwritten by rebase --- python/pyarrow/_plasma.pyx | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/pyarrow/_plasma.pyx b/python/pyarrow/_plasma.pyx index 2bbddc9fad4..e0b8dc82875 100644 --- a/python/pyarrow/_plasma.pyx +++ b/python/pyarrow/_plasma.pyx @@ -31,7 +31,6 @@ from libc.stdint cimport int64_t, uint8_t, uintptr_t from cython.operator cimport dereference as deref, preincrement as inc from cpython.pycapsule cimport * -import collections import random import socket import warnings @@ -564,7 +563,7 @@ cdef class PlasmaClient: the object_ids and ObjectNotAvailable if the object was not available. """ - if isinstance(object_ids, collections.Sequence): + if isinstance(object_ids, compat.Sequence): results = [] buffers = self.get_buffers(object_ids, timeout_ms) for i in range(len(object_ids)): From e20855cc8af520b5892a85a3087d42142958feb1 Mon Sep 17 00:00:00 2001 From: Siyuan Date: Sat, 26 Oct 2019 00:44:02 -0700 Subject: [PATCH 5/7] revert some code overwritten by rebase --- cpp/src/plasma/client.cc | 40 ++++++++++++++++++++++--- cpp/src/plasma/common.cc | 46 ----------------------------- cpp/src/plasma/test/client_tests.cc | 6 ++-- 3 files changed, 39 insertions(+), 53 deletions(-) diff --git a/cpp/src/plasma/client.cc b/cpp/src/plasma/client.cc index 8b968f94294..29bba009657 100644 --- a/cpp/src/plasma/client.cc +++ b/cpp/src/plasma/client.cc @@ -34,7 +34,7 @@ #include #include "arrow/buffer.h" -#include "arrow/util/thread-pool.h" +#include "arrow/util/thread_pool.h" #include "plasma/common.h" #include "plasma/malloc.h" @@ -51,7 +51,7 @@ using arrow::cuda::CudaDeviceManager; #define XXH_INLINE_ALL 1 #define XXH_NAMESPACE plasma_client_ -#include "arrow/vendored/xxhash/xxhash.h" +#include "arrow/vendored/xxhash.h" #define XXH64_DEFAULT_SEED 0 @@ -476,6 +476,37 @@ Status PlasmaClient::Impl::CreateAndSeal(const ObjectID& object_id, return Status::OK(); } +Status PlasmaClient::Impl::CreateAndSealBatch(const std::vector& object_ids, + const std::vector& data, + const std::vector& metadata) { + std::lock_guard guard(client_mutex_); + + ARROW_LOG(DEBUG) << "called CreateAndSealBatch on conn " << store_conn_; + + int device_num = 0; + std::vector digests; + for (size_t i = 0; i < object_ids.size(); i++) { + // Compute the object hash. + std::string digest; + // CreateAndSeal currently only supports device_num = 0, which corresponds to + // the host. + uint64_t hash = ComputeObjectHash( + reinterpret_cast(data.data()), data.size(), + reinterpret_cast(metadata.data()), metadata.size(), device_num); + digest.assign(reinterpret_cast(&hash), sizeof(hash)); + digests.push_back(digest); + } + + RETURN_NOT_OK( + SendCreateAndSealBatchRequest(store_conn_, object_ids, data, metadata, digests)); + std::vector buffer; + RETURN_NOT_OK( + PlasmaReceive(store_conn_, MessageType::PlasmaCreateAndSealBatchReply, &buffer)); + RETURN_NOT_OK(ReadCreateAndSealBatchReply(buffer.data(), buffer.size())); + + return Status::OK(); +} + Status PlasmaClient::Impl::GetBuffers( const ObjectID* object_ids, int64_t num_objects, int64_t timeout_ms, const std::function( @@ -805,9 +836,10 @@ Status PlasmaClient::Impl::Seal(const ObjectID& object_id) { object_entry->second->is_sealed = true; /// Send the seal request to Plasma. - static unsigned char digest[kDigestSize]; + std::vector digest(kDigestSize); RETURN_NOT_OK(Hash(object_id, &digest[0])); - RETURN_NOT_OK(SendSealRequest(store_conn_, object_id, &digest[0])); + RETURN_NOT_OK( + SendSealRequest(store_conn_, object_id, std::string(digest.begin(), digest.end()))); // We call PlasmaClient::Release to decrement the number of instances of this // object // that are currently being used by this client. The corresponding increment diff --git a/cpp/src/plasma/common.cc b/cpp/src/plasma/common.cc index ce58f89b4fe..35b645adca2 100644 --- a/cpp/src/plasma/common.cc +++ b/cpp/src/plasma/common.cc @@ -70,52 +70,6 @@ bool IsPlasmaStatus(const arrow::Status& status, PlasmaErrorCode code) { } // namespace -namespace { - -const char kErrorDetailTypeId[] = "plasma::PlasmaStatusDetail"; - -class PlasmaStatusDetail : public arrow::StatusDetail { - public: - explicit PlasmaStatusDetail(PlasmaErrorCode code) : code_(code) {} - const char* type_id() const override { return kErrorDetailTypeId; } - std::string ToString() const override { - const char* type; - switch (code()) { - case PlasmaErrorCode::PlasmaObjectExists: - type = "Plasma object exists"; - break; - case PlasmaErrorCode::PlasmaObjectNonexistent: - type = "Plasma object is nonexistent"; - break; - case PlasmaErrorCode::PlasmaStoreFull: - type = "Plasma store is full"; - break; - case PlasmaErrorCode::PlasmaObjectAlreadySealed: - type = "Plasma object is already sealed"; - break; - default: - type = "Unknown plasma error"; - break; - } - return std::string(type); - } - PlasmaErrorCode code() const { return code_; } - - private: - PlasmaErrorCode code_; -}; - -bool IsPlasmaStatus(const arrow::Status& status, PlasmaErrorCode code) { - if (status.ok()) { - return false; - } - auto* detail = status.detail().get(); - return detail != nullptr && detail->type_id() == kErrorDetailTypeId && - static_cast(detail)->code() == code; -} - -} // namespace - using arrow::Status; arrow::Status MakePlasmaError(PlasmaErrorCode code, std::string message) { diff --git a/cpp/src/plasma/test/client_tests.cc b/cpp/src/plasma/test/client_tests.cc index 01467f29487..ffc9bcea8f8 100644 --- a/cpp/src/plasma/test/client_tests.cc +++ b/cpp/src/plasma/test/client_tests.cc @@ -23,13 +23,13 @@ #include #include "arrow/testing/gtest_util.h" -#include "arrow/util/io-util.h" +#include "arrow/util/io_util.h" #include "plasma/client.h" #include "plasma/common.h" #include "plasma/plasma.h" #include "plasma/protocol.h" -#include "plasma/test-util.h" +#include "plasma/test_util.h" namespace plasma { @@ -56,7 +56,7 @@ class TestPlasmaStore : public ::testing::Test { std::string plasma_directory = test_executable.substr(0, test_executable.find_last_of("/")); std::string plasma_command = - plasma_directory + "/plasma_store_server -m 10000000 -s " + store_socket_name_ + + plasma_directory + "/plasma-store-server -m 10000000 -s " + store_socket_name_ + " 1> /dev/null 2> /dev/null & " + "echo $! > " + store_socket_name_ + ".pid"; PLASMA_CHECK_SYSTEM(system(plasma_command.c_str())); ASSERT_OK(client_.Connect(store_socket_name_, "")); From 00d7d3b4e09922dc9a311f130c320ec9c7fabff8 Mon Sep 17 00:00:00 2001 From: Siyuan Date: Sun, 27 Oct 2019 00:43:11 -0700 Subject: [PATCH 6/7] revert some code overwritten by rebase --- cpp/src/plasma/client.cc | 56 +++++++++++++++++++++++++++++++--------- 1 file changed, 44 insertions(+), 12 deletions(-) diff --git a/cpp/src/plasma/client.cc b/cpp/src/plasma/client.cc index 29bba009657..326ddf84e96 100644 --- a/cpp/src/plasma/client.cc +++ b/cpp/src/plasma/client.cc @@ -73,6 +73,9 @@ constexpr int64_t kBytesInMB = 1 << 20; // GPU support #ifdef PLASMA_CUDA + +namespace { + struct GpuProcessHandle { /// Pointer to CUDA buffer that is backing this GPU object. std::shared_ptr ptr; @@ -83,8 +86,18 @@ struct GpuProcessHandle { // This is necessary as IPC handles can only be mapped once per process. // Thus if multiple clients in the same process get the same gpu object, // they need to access the same mapped CudaBuffer. -static std::unordered_map gpu_object_map; -static std::mutex gpu_mutex; +std::unordered_map gpu_object_map; +std::mutex gpu_mutex; + +// Return a new CudaBuffer pointing to the same data as the GpuProcessHandle, +// but able to persist after the original IPC-backed buffer is closed +// (ARROW-5924). +std::shared_ptr MakeBufferFromGpuProcessHandle(GpuProcessHandle* handle) { + return std::make_shared(handle->ptr->mutable_data(), handle->ptr->size(), + handle->ptr->context()); +} + +} // namespace #endif // ---------------------------------------------------------------------- @@ -196,12 +209,18 @@ class PlasmaClient::Impl : public std::enable_shared_from_this* data, int device_num = 0); Status CreateAndSeal(const ObjectID& object_id, const std::string& data, const std::string& metadata); + Status CreateAndSealBatch(const std::vector& object_ids, + const std::vector& data, + const std::vector& metadata); + Status Get(const std::vector& object_ids, int64_t timeout_ms, std::vector* object_buffers); @@ -609,18 +628,18 @@ Status PlasmaClient::Impl::GetBuffers( } else { #ifdef PLASMA_CUDA std::lock_guard lock(gpu_mutex); - auto handle = gpu_object_map.find(object_ids[i]); - if (handle == gpu_object_map.end()) { + auto iter = gpu_object_map.find(object_ids[i]); + if (iter == gpu_object_map.end()) { std::shared_ptr context; RETURN_NOT_OK(manager_->GetContext(object->device_num - 1, &context)); GpuProcessHandle* obj_handle = new GpuProcessHandle(); obj_handle->client_count = 1; RETURN_NOT_OK(context->OpenIpcBuffer(*object->ipc_handle, &obj_handle->ptr)); gpu_object_map[object_ids[i]] = obj_handle; - physical_buf = obj_handle->ptr; + physical_buf = MakeBufferFromGpuProcessHandle(obj_handle); } else { - handle->second->client_count++; - physical_buf = handle->second->ptr; + iter->second->client_count++; + physical_buf = MakeBufferFromGpuProcessHandle(iter->second); } #else ARROW_LOG(FATAL) << "Arrow GPU library is not enabled."; @@ -689,11 +708,8 @@ Status PlasmaClient::Impl::Release(const ObjectID& object_id) { return Status::Invalid("Trying to release a non-existing object."); } auto& entry = *object_entry->second; - entry.count -= 1; - ARROW_CHECK(entry.count >= 0) << "Got negative ref count."; - #ifdef PLASMA_CUDA - if (object_entry->second->object.device_num != 0) { + if (entry.object.device_num != 0) { std::lock_guard lock(gpu_mutex); auto iter = gpu_object_map.find(object_id); ARROW_CHECK(iter != gpu_object_map.end()); @@ -703,7 +719,8 @@ Status PlasmaClient::Impl::Release(const ObjectID& object_id) { } } #endif - + entry.count -= 1; + ARROW_CHECK(entry.count >= 0) << "Got negative ref count."; // Check if the client is no longer using this object. if (entry.count == 0) { // Tell the store that the client no longer needs the object. @@ -1011,6 +1028,15 @@ Status PlasmaClient::Impl::Connect(const std::string& store_socket_name) { return Status::OK(); } +Status PlasmaClient::Impl::SetClientOptions(const std::string& client_name, + int64_t output_memory_quota) { + std::lock_guard guard(client_mutex_); + RETURN_NOT_OK(SendSetOptionsRequest(store_conn_, client_name, output_memory_quota)); + std::vector buffer; + RETURN_NOT_OK(PlasmaReceive(store_conn_, MessageType::PlasmaSetOptionsReply, &buffer)); + return ReadSetOptionsReply(buffer.data(), buffer.size()); +} + Status PlasmaClient::Impl::Disconnect() { std::lock_guard guard(client_mutex_); @@ -1066,6 +1092,12 @@ Status PlasmaClient::CreateAndSeal(const ObjectID& object_id, const std::string& return impl_->CreateAndSeal(object_id, data, metadata); } +Status PlasmaClient::CreateAndSealBatch(const std::vector& object_ids, + const std::vector& data, + const std::vector& metadata) { + return impl_->CreateAndSealBatch(object_ids, data, metadata); +} + Status PlasmaClient::Get(const std::vector& object_ids, int64_t timeout_ms, std::vector* object_buffers) { return impl_->Get(object_ids, timeout_ms, object_buffers); From 6b200a737b31ca24f24e28ec2224e1664405df1b Mon Sep 17 00:00:00 2001 From: Siyuan Date: Sun, 27 Oct 2019 00:53:42 -0700 Subject: [PATCH 7/7] revert some code overwritten by rebase --- cpp/src/plasma/protocol.cc | 50 ++++++++++++++++++++++++++++++++++++++ cpp/src/plasma/protocol.h | 24 ++++++++++++++++++ 2 files changed, 74 insertions(+) diff --git a/cpp/src/plasma/protocol.cc b/cpp/src/plasma/protocol.cc index eccf15e0006..542c9298fea 100644 --- a/cpp/src/plasma/protocol.cc +++ b/cpp/src/plasma/protocol.cc @@ -115,6 +115,56 @@ Status PlasmaSend(const std::shared_ptr& client, } } +// Set options messages. + +// TODO(suquark): Get rid of raw socket handle +Status SendSetOptionsRequest(int sock, const std::string& client_name, + int64_t output_memory_limit) { + flatbuffers::FlatBufferBuilder fbb; + auto message = fb::CreatePlasmaSetOptionsRequest(fbb, fbb.CreateString(client_name), + output_memory_limit); + return PlasmaSend(sock, MessageType::PlasmaSetOptionsRequest, &fbb, message); +} + +Status ReadSetOptionsRequest(uint8_t* data, size_t size, std::string* client_name, + int64_t* output_memory_quota) { + DCHECK(data); + auto message = flatbuffers::GetRoot(data); + DCHECK(VerifyFlatbuffer(message, data, size)); + *client_name = std::string(message->client_name()->str()); + *output_memory_quota = message->output_memory_quota(); + return Status::OK(); +} + +// TODO(suquark): Get rid of raw socket handle +Status SendSetOptionsReply(int sock, PlasmaError error) { + flatbuffers::FlatBufferBuilder fbb; + auto message = fb::CreatePlasmaSetOptionsReply(fbb, error); + return PlasmaSend(sock, MessageType::PlasmaSetOptionsReply, &fbb, message); +} + +Status ReadSetOptionsReply(uint8_t* data, size_t size) { + DCHECK(data); + auto message = flatbuffers::GetRoot(data); + DCHECK(VerifyFlatbuffer(message, data, size)); + return PlasmaErrorStatus(message->error()); +} + +// Get debug string messages. + +// TODO(suquark): Get rid of raw socket handle +Status SendGetDebugStringRequest(int sock) { + flatbuffers::FlatBufferBuilder fbb; + auto message = fb::CreatePlasmaGetDebugStringRequest(fbb); + return PlasmaSend(sock, MessageType::PlasmaGetDebugStringRequest, &fbb, message); +} + +// TODO(suquark): Get rid of raw socket handle +Status SendGetDebugStringReply(int sock, const std::string& debug_string) { + flatbuffers::FlatBufferBuilder fbb; + auto message = fb::CreatePlasmaGetDebugStringReply(fbb, fbb.CreateString(debug_string)); + return PlasmaSend(sock, MessageType::PlasmaGetDebugStringReply, &fbb, message); + // Create messages. Status SendCreateRequest(const std::shared_ptr& client, diff --git a/cpp/src/plasma/protocol.h b/cpp/src/plasma/protocol.h index 0249a529173..fa40ae36083 100644 --- a/cpp/src/plasma/protocol.h +++ b/cpp/src/plasma/protocol.h @@ -41,6 +41,30 @@ bool VerifyFlatbuffer(T* object, const uint8_t* data, size_t size) { return object->Verify(verifier); } +/* Set options messages. */ + +// TODO(suquark): Get rid of raw socket handle +Status SendSetOptionsRequest(int sock, const std::string& client_name, + int64_t output_memory_limit); + +Status ReadSetOptionsRequest(uint8_t* data, size_t size, std::string* client_name, + int64_t* output_memory_quota); + +// TODO(suquark): Get rid of raw socket handle +Status SendSetOptionsReply(int sock, PlasmaError error); + +Status ReadSetOptionsReply(uint8_t* data, size_t size); + +/* Debug string messages. */ + +// TODO(suquark): Get rid of raw socket handle +Status SendGetDebugStringRequest(int sock); + +// TODO(suquark): Get rid of raw socket handle +Status SendGetDebugStringReply(int sock, const std::string& debug_string); + +Status ReadGetDebugStringReply(uint8_t* data, size_t size, std::string* debug_string); + /* Plasma Create message functions. */ Status SendCreateRequest(const std::shared_ptr& client,