You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/04 09:12:52 UTC

[GitHub] [arrow] cyb70289 commented on a diff in pull request #12442: ARROW-15706: [C++][FlightRPC] Implement a UCX transport

cyb70289 commented on code in PR #12442:
URL: https://github.com/apache/arrow/pull/12442#discussion_r841525350


##########
cpp/src/arrow/flight/transport/ucx/ucx_client.cc:
##########
@@ -0,0 +1,730 @@
+// 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.
+
+/// The client-side implementation of a UCX-based transport for
+/// Flight.
+///
+/// Each UCX driver is used to support one call at a time. This gives
+/// the greatest throughput for data plane methods, but is relatively
+/// expensive in terms of other resources, both for the server and the
+/// client. (UCX drivers have multiple threading modes: single-thread
+/// access, serialized access, and multi-thread access. Testing found
+/// that multi-thread access incurred high synchronization costs.)
+/// Hence, for concurrent calls in a single client, we must maintain
+/// multiple drivers, and so unlike gRPC, there is no real difference
+/// between using one client concurrently and using multiple
+/// independent clients.
+
+#include "arrow/flight/transport/ucx/ucx_internal.h"
+
+#include <condition_variable>
+#include <deque>
+#include <mutex>
+#include <thread>
+
+#include <arpa/inet.h>
+#include <ucp/api/ucp.h>
+
+#include "arrow/buffer.h"
+#include "arrow/flight/client.h"
+#include "arrow/flight/transport.h"
+#include "arrow/flight/transport/ucx/util_internal.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/uri.h"
+
+namespace arrow {
+namespace flight {
+namespace transport {
+namespace ucx {
+
+namespace {
+class UcxClientImpl;
+
+Status MergeStatuses(Status server_status, Status transport_status) {
+  if (server_status.ok()) {
+    if (transport_status.ok()) return server_status;
+    return transport_status;
+  } else if (transport_status.ok()) {
+    return server_status;
+  }
+  return Status::FromDetailAndArgs(server_status.code(), server_status.detail(),
+                                   server_status.message(),
+                                   ". Transport context: ", transport_status.ToString());
+}
+
+/// \brief An individual connection to the server.
+class ClientConnection {
+ public:
+  ClientConnection() = default;
+  ARROW_DISALLOW_COPY_AND_ASSIGN(ClientConnection);
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(ClientConnection);
+  ~ClientConnection() { DCHECK(!driver_) << "Connection was not closed!"; }
+
+  Status Init(std::shared_ptr<UcpContext> ucp_context, const arrow::internal::Uri& uri) {
+    auto status = InitImpl(std::move(ucp_context), uri);
+    // Clean up after-the-fact if we fail to initialize
+    if (!status.ok()) {
+      if (driver_) {
+        status = MergeStatuses(std::move(status), driver_->Close());
+        driver_.reset();
+        remote_endpoint_ = nullptr;
+      }
+      if (ucp_worker_) ucp_worker_.reset();
+    }
+    return status;
+  }
+
+  Status InitImpl(std::shared_ptr<UcpContext> ucp_context,
+                  const arrow::internal::Uri& uri) {
+    {
+      ucs_status_t status;
+      ucp_worker_params_t worker_params;
+      std::memset(&worker_params, 0, sizeof(worker_params));
+      worker_params.field_mask = UCP_WORKER_PARAM_FIELD_THREAD_MODE;
+      worker_params.thread_mode = UCS_THREAD_MODE_SERIALIZED;
+
+      ucp_worker_h ucp_worker;
+      status = ucp_worker_create(ucp_context->get(), &worker_params, &ucp_worker);
+      RETURN_NOT_OK(FromUcsStatus("ucp_worker_create", status));
+      ucp_worker_.reset(new UcpWorker(std::move(ucp_context), ucp_worker));
+    }
+    {
+      // Create endpoint for remote worker
+      struct sockaddr_storage connect_addr;
+      ARROW_ASSIGN_OR_RAISE(auto addrlen, UriToSockaddr(uri, &connect_addr));
+      std::string peer;
+      ARROW_UNUSED(SockaddrToString(connect_addr).Value(&peer));
+      ARROW_LOG(DEBUG) << "Connecting to " << peer;
+
+      ucp_ep_params_t params;
+      params.field_mask = UCP_EP_PARAM_FIELD_FLAGS | UCP_EP_PARAM_FIELD_NAME |
+                          UCP_EP_PARAM_FIELD_SOCK_ADDR;
+      params.flags = UCP_EP_PARAMS_FLAGS_CLIENT_SERVER;
+      params.name = "UcxClientImpl";
+      params.sockaddr.addr = reinterpret_cast<const sockaddr*>(&connect_addr);
+      params.sockaddr.addrlen = addrlen;
+
+      auto status = ucp_ep_create(ucp_worker_->get(), &params, &remote_endpoint_);
+      RETURN_NOT_OK(FromUcsStatus("ucp_ep_create", status));
+    }
+
+    driver_.reset(new UcpCallDriver(ucp_worker_, remote_endpoint_));
+    ARROW_LOG(DEBUG) << "Connected to " << driver_->peer();
+
+    {
+      // Set up Active Message (AM) handler
+      ucp_am_handler_param_t handler_params;
+      handler_params.field_mask = UCP_AM_HANDLER_PARAM_FIELD_ID |
+                                  UCP_AM_HANDLER_PARAM_FIELD_CB |
+                                  UCP_AM_HANDLER_PARAM_FIELD_ARG;
+      handler_params.id = kUcpAmHandlerId;
+      handler_params.cb = HandleIncomingActiveMessage;
+      handler_params.arg = driver_.get();
+      ucs_status_t status =
+          ucp_worker_set_am_recv_handler(ucp_worker_->get(), &handler_params);
+      RETURN_NOT_OK(FromUcsStatus("ucp_worker_set_am_recv_handler", status));
+    }
+
+    return Status::OK();
+  }
+
+  Status Close() {
+    if (!driver_) return Status::OK();
+
+    auto status = driver_->SendFrame(FrameType::kDisconnect, nullptr, 0);
+    const auto ucs_status = FlightUcxStatusDetail::Unwrap(status);
+    if (IsIgnorableDisconnectError(ucs_status)) {
+      status = Status::OK();
+    }
+    status = MergeStatuses(std::move(status), driver_->Close());
+
+    driver_.reset();
+    remote_endpoint_ = nullptr;
+    ucp_worker_.reset();
+    return status;
+  }
+
+  UcpCallDriver* driver() {
+    DCHECK(driver_);
+    return driver_.get();
+  }
+
+ private:
+  static ucs_status_t HandleIncomingActiveMessage(void* self, const void* header,
+                                                  size_t header_length, void* data,
+                                                  size_t data_length,
+                                                  const ucp_am_recv_param_t* param) {
+    auto* driver = reinterpret_cast<UcpCallDriver*>(self);
+    return driver->RecvActiveMessage(header, header_length, data, data_length, param);
+  }
+
+  std::shared_ptr<UcpWorker> ucp_worker_;
+  ucp_ep_h remote_endpoint_;
+  std::unique_ptr<UcpCallDriver> driver_;
+};
+
+class UcxClientStream : public internal::ClientDataStream {
+ public:
+  UcxClientStream(UcxClientImpl* impl, ClientConnection conn)
+      : impl_(impl),
+        conn_(std::move(conn)),
+        driver_(conn_.driver()),
+        writes_done_(false),
+        finished_(false) {}
+
+ protected:
+  Status DoFinish() override;
+
+  UcxClientImpl* impl_;
+  ClientConnection conn_;
+  UcpCallDriver* driver_;
+  bool writes_done_;
+  bool finished_;
+  Status io_status_;
+  Status server_status_;
+};
+
+class GetClientStream : public UcxClientStream {
+ public:
+  GetClientStream(UcxClientImpl* impl, ClientConnection conn)
+      : UcxClientStream(impl, std::move(conn)) {
+    writes_done_ = true;
+  }
+
+  bool ReadData(internal::FlightData* data) override {
+    if (finished_) return false;
+
+    bool success = true;
+    io_status_ = ReadImpl(data).Value(&success);
+
+    if (!io_status_.ok() || !success) {
+      finished_ = true;
+    }
+    return success;
+  }
+
+ private:
+  ::arrow::Result<bool> ReadImpl(internal::FlightData* data) {
+    ARROW_ASSIGN_OR_RAISE(auto frame, driver_->ReadNextFrame());
+
+    if (frame->type == FrameType::kHeaders) {
+      // Trailers, stream is over
+      ARROW_ASSIGN_OR_RAISE(auto headers, HeadersFrame::Parse(std::move(frame->buffer)));
+      RETURN_NOT_OK(headers.GetStatus(&server_status_));
+      return false;
+    }
+
+    RETURN_NOT_OK(driver_->ExpectFrameType(*frame, FrameType::kPayloadHeader));
+    PayloadHeaderFrame payload_header(std::move(frame->buffer));
+    RETURN_NOT_OK(payload_header.ToFlightData(data));
+
+    // DoGet does not support metadata-only messages, so we can always
+    // assume we have an IPC payload
+    ARROW_ASSIGN_OR_RAISE(auto message, ipc::Message::Open(data->metadata, nullptr));
+
+    if (ipc::Message::HasBody(message->type())) {
+      ARROW_ASSIGN_OR_RAISE(frame, driver_->ReadNextFrame());
+      RETURN_NOT_OK(driver_->ExpectFrameType(*frame, FrameType::kPayloadBody));
+      data->body = std::move(frame->buffer);
+    }
+    return true;
+  }
+};
+
+class WriteClientStream : public UcxClientStream {
+ public:
+  WriteClientStream(UcxClientImpl* impl, ClientConnection conn)
+      : UcxClientStream(impl, std::move(conn)) {
+    std::thread t(&WriteClientStream::DriveWorker, this);
+    driver_thread_.swap(t);
+  }
+  arrow::Result<bool> WriteData(const FlightPayload& payload) override {
+    std::unique_lock<std::mutex> guard(driver_mutex_);
+    if (finished_ || writes_done_) return Status::Invalid("Already done writing");
+    outgoing_ = driver_->SendFlightPayload(payload);
+    working_cv_.notify_all();
+    received_cv_.wait(guard, [this] { return outgoing_.is_finished(); });
+
+    auto status = outgoing_.status();
+    outgoing_ = Future<>();
+    RETURN_NOT_OK(status);
+    return true;
+  }
+  Status WritesDone() override {
+    std::unique_lock<std::mutex> guard(driver_mutex_);
+    if (!writes_done_) {
+      ARROW_ASSIGN_OR_RAISE(auto headers, HeadersFrame::Make({}));
+      outgoing_ =
+          driver_->SendFrameAsync(FrameType::kHeaders, std::move(headers).GetBuffer());
+      working_cv_.notify_all();
+      received_cv_.wait(guard, [this] { return outgoing_.is_finished(); });
+
+      writes_done_ = true;
+      auto status = outgoing_.status();
+      outgoing_ = Future<>();
+      RETURN_NOT_OK(status);
+    }
+    return Status::OK();
+  }
+
+ protected:
+  void JoinThread() {
+    try {
+      driver_thread_.join();
+    } catch (const std::system_error&) {
+      // Ignore
+    }
+  }
+  void DriveWorker() {

Review Comment:
   Are the ucx callbacks (`AmSendCallback`, `HandleIncomingActiveMessage`) invoked inside `ucp_wroker_progress()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org