lidavidm commented on a change in pull request #12442:
URL: https://github.com/apache/arrow/pull/12442#discussion_r839533464



##########
File path: cpp/src/arrow/flight/transport/ucx/ucx_server.cc
##########
@@ -0,0 +1,646 @@
+// 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 "arrow/flight/transport/ucx/ucx_internal.h"
+
+#include <atomic>
+#include <mutex>
+#include <queue>
+#include <thread>
+#include <unordered_map>
+
+#include <arpa/inet.h>
+#include <ucp/api/ucp.h>
+
+#include "arrow/buffer.h"
+#include "arrow/flight/server.h"
+#include "arrow/flight/transport.h"
+#include "arrow/flight/transport/ucx/util_internal.h"
+#include "arrow/flight/transport_server.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/io_util.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+#include "arrow/util/uri.h"
+
+namespace arrow {
+namespace flight {
+namespace transport {
+namespace ucx {
+
+// Send an error to the client and return OK.
+// Statuses returned up to the main server loop trigger a kReset instead.
+#define SERVER_RETURN_NOT_OK(driver, status)                                   
      \
+  do {                                                                         
      \
+    ::arrow::Status s = (status);                                              
      \
+    if (!s.ok()) {                                                             
      \
+      ARROW_ASSIGN_OR_RAISE(auto headers, HeadersFrame::Make(s, {}));          
      \
+      auto payload = std::move(headers).GetBuffer();                           
      \
+      RETURN_NOT_OK(                                                           
      \
+          driver->SendFrame(FrameType::kHeaders, payload->data(), 
payload->size())); \
+      return ::arrow::Status::OK();                                            
      \
+    }                                                                          
      \
+  } while (false)
+
+#define FLIGHT_LOG(LEVEL) (ARROW_LOG(LEVEL) << "[server] ")
+#define FLIGHT_LOG_PEER(LEVEL, PEER) \
+  (ARROW_LOG(LEVEL) << "[server]"    \
+                    << "[peer=" << (PEER) << "] ")
+
+namespace {
+class UcxServerCallContext : public flight::ServerCallContext {
+ public:
+  const std::string& peer_identity() const override { return peer_; }
+  const std::string& peer() const override { return peer_; }
+  ServerMiddleware* GetMiddleware(const std::string& key) const override {
+    return nullptr;
+  }
+  bool is_cancelled() const override { return false; }
+
+ private:
+  std::string peer_;
+};
+
+class UcxServerStream : public internal::ServerDataStream {
+ public:
+  // TODO(lidavidm): backpressure threshold should be dynamic (ideally
+  // auto-adjusted, or at least configurable)
+  constexpr static size_t kBackpressureThreshold = 8;
+
+  explicit UcxServerStream(UcpCallDriver* driver)
+      : peer_(driver->peer()), driver_(driver), writes_done_(false) {}
+
+  Status WritesDone() override {
+    RETURN_NOT_OK(CheckBackpressure(0));
+    writes_done_ = true;
+    return Status::OK();
+  }
+
+ protected:
+  Status CheckBackpressure(size_t limit = kBackpressureThreshold - 1) {
+    while (requests_.size() > limit) {
+      auto& next = requests_.front();
+      while (!next.is_finished()) {
+        driver_->MakeProgress();
+      }
+      RETURN_NOT_OK(next.status());
+      requests_.pop();
+    }
+    return Status::OK();
+  }
+
+  std::string peer_;
+  UcpCallDriver* driver_;
+  bool writes_done_;
+  std::queue<Future<>> requests_;
+};
+
+class GetServerStream : public UcxServerStream {
+ public:
+  using UcxServerStream::UcxServerStream;
+
+  arrow::Result<bool> WriteData(const FlightPayload& payload) override {
+    if (writes_done_) return false;
+    RETURN_NOT_OK(CheckBackpressure());
+    Future<> pending_send = driver_->SendFlightPayload(payload);
+    if (!pending_send.is_finished()) {
+      requests_.push(std::move(pending_send));
+    } else {
+      // Request completed instantly
+      RETURN_NOT_OK(pending_send.status());
+    }
+    return true;
+  }
+};
+
+class PutServerStream : public UcxServerStream {
+ public:
+  explicit PutServerStream(UcpCallDriver* driver)
+      : UcxServerStream(driver), finished_(false) {}
+
+  bool ReadData(internal::FlightData* data) override {
+    if (finished_) return false;
+
+    bool success = true;
+    auto status = ReadImpl(data).Value(&success);
+
+    if (!status.ok() || !success) {
+      finished_ = true;
+      if (!status.ok()) {
+        FLIGHT_LOG_PEER(WARNING, peer_) << "I/O error in DoPut: " << 
status.ToString();
+        return false;
+      }
+    }
+    return success;
+  }
+
+  Status WritePutMetadata(const Buffer& payload) override {
+    if (finished_) return Status::OK();
+    // Send synchronously (we don't control payload lifetime)
+    return driver_->SendFrame(FrameType::kBuffer, payload.data(), 
payload.size());
+  }
+
+ private:
+  ::arrow::Result<bool> ReadImpl(internal::FlightData* data) {
+    ARROW_ASSIGN_OR_RAISE(auto frame, driver_->ReadNextFrame());
+    if (frame->type == FrameType::kHeaders) {
+      // Trailers, client is done writing
+      return false;
+    }
+    RETURN_NOT_OK(driver_->ExpectFrameType(*frame, FrameType::kPayloadHeader));
+    PayloadHeaderFrame payload_header(std::move(frame->buffer));
+    RETURN_NOT_OK(payload_header.ToFlightData(data));
+
+    if (data->metadata) {
+      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;
+  }
+
+  bool finished_;
+};
+
+class ExchangeServerStream : public PutServerStream {
+ public:
+  using PutServerStream::PutServerStream;
+
+  arrow::Result<bool> WriteData(const FlightPayload& payload) override {
+    if (writes_done_) return false;
+    // Don't use backpressure - the application may expect synchronous
+    // behavior (write a message, read the client response)
+    Future<> pending_send = driver_->SendFlightPayload(payload);
+    while (!pending_send.is_finished()) {
+      driver_->MakeProgress();
+    }
+    RETURN_NOT_OK(pending_send.status());
+    return true;
+  }
+  Status WritePutMetadata(const Buffer& payload) override {
+    return Status::NotImplemented("Not supported on this stream");
+  }
+};
+
+class UcxServerImpl : public arrow::flight::internal::ServerTransport {
+ public:
+  using arrow::flight::internal::ServerTransport::ServerTransport;
+
+  virtual ~UcxServerImpl() {
+    if (listening_.load()) {
+      auto st = Shutdown();
+      if (!st.ok()) {
+        ARROW_LOG(WARNING) << "Server did not shut down properly: " << 
st.ToString();
+      }
+    }
+  }
+
+  Status Init(const FlightServerOptions& options,
+              const arrow::internal::Uri& uri) override {
+    const auto num_threads = std::max<uint32_t>(8, 
std::thread::hardware_concurrency());

Review comment:
       It should not, this is a max # of threads (I've renamed it to make that 
clearer) and the pool will only create as many threads as needed. One RPC is 
handled per thread so this will let us handle up to (numcores) concurrent 
requests (the rest will just be queued). 

##########
File path: cpp/src/arrow/flight/transport/ucx/util_internal.cc
##########
@@ -0,0 +1,289 @@
+// 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 "arrow/flight/transport/ucx/util_internal.h"
+
+#include <netdb.h>
+#include <sys/socket.h>
+#include <sys/types.h>
+
+#include <cerrno>
+#include <mutex>
+#include <unordered_map>
+
+#include "arrow/buffer.h"
+#include "arrow/flight/types.h"
+#include "arrow/util/base64.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/io_util.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 {
+
+constexpr char FlightUcxStatusDetail::kTypeId[];
+std::string FlightUcxStatusDetail::ToString() const { return 
ucs_status_string(status_); }
+ucs_status_t FlightUcxStatusDetail::Unwrap(const Status& status) {
+  if (!status.detail() || status.detail()->type_id() != kTypeId) return UCS_OK;
+  return dynamic_cast<const 
FlightUcxStatusDetail*>(status.detail().get())->status_;
+}
+
+arrow::Result<size_t> UriToSockaddr(const arrow::internal::Uri& uri,
+                                    struct sockaddr_storage* addr) {
+  std::string host = uri.host();
+  if (host.empty()) {
+    return Status::Invalid("Must provide a host");
+  } else if (uri.port() < 0) {
+    return Status::Invalid("Must provide a port");
+  }
+
+  std::memset(addr, 0, sizeof(*addr));
+
+  struct addrinfo* info = nullptr;
+  int err = getaddrinfo(host.c_str(), /*service=*/nullptr, /*hints=*/nullptr, 
&info);
+  if (err != 0) {
+    if (err == EAI_SYSTEM) {
+      return arrow::internal::IOErrorFromErrno(errno, "[getaddrinfo] Failure 
resolving ",
+                                               host);
+    } else {
+      return Status::IOError("[getaddrinfo] Failure resolving ", host, ": ",
+                             gai_strerror(err));
+    }
+  }
+
+  struct addrinfo* cur_info = info;
+  while (cur_info) {
+    if (cur_info->ai_family != AF_INET && cur_info->ai_family != AF_INET6) {
+      cur_info = cur_info->ai_next;
+      continue;
+    }
+
+    std::memcpy(addr, cur_info->ai_addr, cur_info->ai_addrlen);
+    if (cur_info->ai_family == AF_INET) {
+      reinterpret_cast<sockaddr_in*>(addr)->sin_port = htons(uri.port());
+    } else if (cur_info->ai_family == AF_INET6) {
+      reinterpret_cast<sockaddr_in6*>(addr)->sin6_port = htons(uri.port());
+    }
+    size_t addrlen = info->ai_addrlen;

Review comment:
       Ah yes - thanks for catching that!




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to