cyb70289 commented on a change in pull request #12442: URL: https://github.com/apache/arrow/pull/12442#discussion_r837193426
########## File path: cpp/src/arrow/flight/transport/ucx/flight_transport_ucx_test.cc ########## @@ -0,0 +1,399 @@ +// 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 <gmock/gmock.h> +#include <gtest/gtest.h> + +#include "arrow/array/array_base.h" +#include "arrow/flight/test_definitions.h" +#include "arrow/flight/test_util.h" +#include "arrow/flight/transport/ucx/ucx.h" +#include "arrow/table.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/util/config.h" + +#ifdef UCP_API_VERSION +#error "UCX headers should not be in public API" +#endif + +#include "arrow/flight/transport/ucx/ucx_internal.h" + +#ifdef ARROW_CUDA +#include "arrow/gpu/cuda_api.h" +#endif + +namespace arrow { +namespace flight { + +class UcxEnvironment : public ::testing::Environment { + public: + void SetUp() override { transport::ucx::InitializeFlightUcx(); } +}; + +testing::Environment* const kUcxEnvironment = + testing::AddGlobalTestEnvironment(new UcxEnvironment()); + +//------------------------------------------------------------ +// Common transport tests + +class UcxConnectivityTest : public ConnectivityTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_CONNECTIVITY(UcxConnectivityTest); + +class UcxDataTest : public DataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_DATA(UcxDataTest); + +class UcxDoPutTest : public DoPutTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_DO_PUT(UcxDoPutTest); + +class UcxAppMetadataTest : public AppMetadataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_APP_METADATA(UcxAppMetadataTest); + +class UcxIpcOptionsTest : public IpcOptionsTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_IPC_OPTIONS(UcxIpcOptionsTest); + +class UcxCudaDataTest : public CudaDataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_CUDA_DATA(UcxCudaDataTest); + +//------------------------------------------------------------ +// UCX internals tests + +constexpr std::initializer_list<StatusCode> kStatusCodes = { + StatusCode::OK, + StatusCode::OutOfMemory, + StatusCode::KeyError, + StatusCode::TypeError, + StatusCode::Invalid, + StatusCode::IOError, + StatusCode::CapacityError, + StatusCode::IndexError, + StatusCode::Cancelled, + StatusCode::UnknownError, + StatusCode::NotImplemented, + StatusCode::SerializationError, + StatusCode::RError, + StatusCode::CodeGenError, + StatusCode::ExpressionValidationError, + StatusCode::ExecutionError, + StatusCode::AlreadyExists, +}; + +constexpr std::initializer_list<FlightStatusCode> kFlightStatusCodes = { + FlightStatusCode::Internal, FlightStatusCode::TimedOut, + FlightStatusCode::Cancelled, FlightStatusCode::Unauthenticated, + FlightStatusCode::Unauthorized, FlightStatusCode::Unavailable, + FlightStatusCode::Failed, +}; + +class TestStatusDetail : public StatusDetail { + public: + const char* type_id() const override { return "test-status-detail"; } + std::string ToString() const override { return "Custom status detail"; } +}; + +namespace transport { +namespace ucx { + +static constexpr std::initializer_list<FrameType> kFrameTypes = { + FrameType::kHeaders, FrameType::kBuffer, FrameType::kPayloadHeader, + FrameType::kPayloadBody, FrameType::kDisconnect, +}; + +TEST(FrameHeader, Basics) { + for (const auto frame_type : kFrameTypes) { + FrameHeader header; + ASSERT_OK(header.Set(frame_type, /*counter=*/42, /*body_size=*/65535)); + if (frame_type == FrameType::kDisconnect) { + ASSERT_RAISES(Cancelled, Frame::ParseHeader(header.data(), header.size())); + } else { + ASSERT_OK_AND_ASSIGN(auto frame, Frame::ParseHeader(header.data(), header.size())); + ASSERT_EQ(frame->type, frame_type); + ASSERT_EQ(frame->counter, 42); + ASSERT_EQ(frame->size, 65535); + } + } +} + +TEST(FrameHeader, FrameType) { + for (const auto frame_type : kFrameTypes) { + ASSERT_LE(static_cast<int>(frame_type), static_cast<int>(FrameType::kMaxFrameType)); + } +} + +TEST(HeadersFrame, Parse) { + const char* data = + ("\x00\x00\x00\x02\x00\x00\x00\x05\x00\x00\x00\x03x-foobar" + "\x00\x00\x00\x05\x00\x00\x00\x01x-bin\x01"); + constexpr int64_t size = 34; + + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), size)); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Parse(std::move(buffer))); + ASSERT_OK_AND_ASSIGN(auto foo, headers.Get("x-foo")); + ASSERT_EQ(foo, "bar"); + ASSERT_OK_AND_ASSIGN(auto bin, headers.Get("x-bin")); + ASSERT_EQ(bin, "\x01"); + } + { + std::unique_ptr<Buffer> buffer(new Buffer(reinterpret_cast<const uint8_t*>(data), 3)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected number of headers"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer(new Buffer(reinterpret_cast<const uint8_t*>(data), 7)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected length of key 1"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 10)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected length of value 1"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 12)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::HasSubstr("expected key 1 to have length 5, but only 0 bytes remain"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 17)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::HasSubstr( + "expected value 1 to have length 3, but only 0 bytes remain"), + HeadersFrame::Parse(std::move(buffer))); + } +} + +TEST(HeadersFrame, RoundTripStatus) { + for (const auto code : kStatusCodes) { + { + Status expected = code == StatusCode::OK ? Status() : Status(code, "foo"); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + + if (code == StatusCode::OK) continue; + + // Attach a generic status detail + { + auto detail = std::make_shared<TestStatusDetail>(); + Status original(code, "foo", detail); + Status expected(code, "foo", + std::make_shared<FlightStatusDetail>(FlightStatusCode::Internal, + detail->ToString())); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + + // Attach a Flight status detail + for (const auto flight_code : kFlightStatusCodes) { + Status expected(code, "foo", + std::make_shared<FlightStatusDetail>(flight_code, "extra")); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + } +} +} // namespace ucx +} // namespace transport + +//------------------------------------------------------------ +// Ad-hoc UCX-specific tests + +class SimpleTestServer : public FlightServerBase { + public: + Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, + std::unique_ptr<FlightInfo>* info) override { + if (request.path.size() > 0 && request.path[0] == "error") { + return status_; + } + auto examples = ExampleFlightInfo(); + *info = std::unique_ptr<FlightInfo>(new FlightInfo(examples[0])); Review comment: `info->reset(new ...)`? ########## File path: cpp/src/arrow/flight/transport/ucx/flight_transport_ucx_test.cc ########## @@ -0,0 +1,399 @@ +// 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 <gmock/gmock.h> +#include <gtest/gtest.h> + +#include "arrow/array/array_base.h" +#include "arrow/flight/test_definitions.h" +#include "arrow/flight/test_util.h" +#include "arrow/flight/transport/ucx/ucx.h" +#include "arrow/table.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/util/config.h" + +#ifdef UCP_API_VERSION +#error "UCX headers should not be in public API" +#endif + +#include "arrow/flight/transport/ucx/ucx_internal.h" + +#ifdef ARROW_CUDA +#include "arrow/gpu/cuda_api.h" +#endif + +namespace arrow { +namespace flight { + +class UcxEnvironment : public ::testing::Environment { + public: + void SetUp() override { transport::ucx::InitializeFlightUcx(); } +}; + +testing::Environment* const kUcxEnvironment = + testing::AddGlobalTestEnvironment(new UcxEnvironment()); + +//------------------------------------------------------------ +// Common transport tests + +class UcxConnectivityTest : public ConnectivityTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_CONNECTIVITY(UcxConnectivityTest); + +class UcxDataTest : public DataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_DATA(UcxDataTest); + +class UcxDoPutTest : public DoPutTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_DO_PUT(UcxDoPutTest); + +class UcxAppMetadataTest : public AppMetadataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_APP_METADATA(UcxAppMetadataTest); + +class UcxIpcOptionsTest : public IpcOptionsTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_IPC_OPTIONS(UcxIpcOptionsTest); + +class UcxCudaDataTest : public CudaDataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_CUDA_DATA(UcxCudaDataTest); + +//------------------------------------------------------------ +// UCX internals tests + +constexpr std::initializer_list<StatusCode> kStatusCodes = { + StatusCode::OK, + StatusCode::OutOfMemory, + StatusCode::KeyError, + StatusCode::TypeError, + StatusCode::Invalid, + StatusCode::IOError, + StatusCode::CapacityError, + StatusCode::IndexError, + StatusCode::Cancelled, + StatusCode::UnknownError, + StatusCode::NotImplemented, + StatusCode::SerializationError, + StatusCode::RError, + StatusCode::CodeGenError, + StatusCode::ExpressionValidationError, + StatusCode::ExecutionError, + StatusCode::AlreadyExists, +}; + +constexpr std::initializer_list<FlightStatusCode> kFlightStatusCodes = { + FlightStatusCode::Internal, FlightStatusCode::TimedOut, + FlightStatusCode::Cancelled, FlightStatusCode::Unauthenticated, + FlightStatusCode::Unauthorized, FlightStatusCode::Unavailable, + FlightStatusCode::Failed, +}; + +class TestStatusDetail : public StatusDetail { + public: + const char* type_id() const override { return "test-status-detail"; } + std::string ToString() const override { return "Custom status detail"; } +}; + +namespace transport { +namespace ucx { + +static constexpr std::initializer_list<FrameType> kFrameTypes = { + FrameType::kHeaders, FrameType::kBuffer, FrameType::kPayloadHeader, + FrameType::kPayloadBody, FrameType::kDisconnect, +}; + +TEST(FrameHeader, Basics) { + for (const auto frame_type : kFrameTypes) { + FrameHeader header; + ASSERT_OK(header.Set(frame_type, /*counter=*/42, /*body_size=*/65535)); + if (frame_type == FrameType::kDisconnect) { + ASSERT_RAISES(Cancelled, Frame::ParseHeader(header.data(), header.size())); + } else { + ASSERT_OK_AND_ASSIGN(auto frame, Frame::ParseHeader(header.data(), header.size())); + ASSERT_EQ(frame->type, frame_type); + ASSERT_EQ(frame->counter, 42); + ASSERT_EQ(frame->size, 65535); + } + } +} + +TEST(FrameHeader, FrameType) { + for (const auto frame_type : kFrameTypes) { + ASSERT_LE(static_cast<int>(frame_type), static_cast<int>(FrameType::kMaxFrameType)); + } +} + +TEST(HeadersFrame, Parse) { + const char* data = + ("\x00\x00\x00\x02\x00\x00\x00\x05\x00\x00\x00\x03x-foobar" + "\x00\x00\x00\x05\x00\x00\x00\x01x-bin\x01"); + constexpr int64_t size = 34; + + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), size)); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Parse(std::move(buffer))); + ASSERT_OK_AND_ASSIGN(auto foo, headers.Get("x-foo")); + ASSERT_EQ(foo, "bar"); + ASSERT_OK_AND_ASSIGN(auto bin, headers.Get("x-bin")); + ASSERT_EQ(bin, "\x01"); + } + { + std::unique_ptr<Buffer> buffer(new Buffer(reinterpret_cast<const uint8_t*>(data), 3)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected number of headers"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer(new Buffer(reinterpret_cast<const uint8_t*>(data), 7)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected length of key 1"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 10)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected length of value 1"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 12)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::HasSubstr("expected key 1 to have length 5, but only 0 bytes remain"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 17)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::HasSubstr( + "expected value 1 to have length 3, but only 0 bytes remain"), + HeadersFrame::Parse(std::move(buffer))); + } +} + +TEST(HeadersFrame, RoundTripStatus) { + for (const auto code : kStatusCodes) { + { + Status expected = code == StatusCode::OK ? Status() : Status(code, "foo"); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + + if (code == StatusCode::OK) continue; + + // Attach a generic status detail + { + auto detail = std::make_shared<TestStatusDetail>(); + Status original(code, "foo", detail); + Status expected(code, "foo", + std::make_shared<FlightStatusDetail>(FlightStatusCode::Internal, + detail->ToString())); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + + // Attach a Flight status detail + for (const auto flight_code : kFlightStatusCodes) { + Status expected(code, "foo", + std::make_shared<FlightStatusDetail>(flight_code, "extra")); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + } +} +} // namespace ucx +} // namespace transport + +//------------------------------------------------------------ +// Ad-hoc UCX-specific tests + +class SimpleTestServer : public FlightServerBase { + public: + Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, + std::unique_ptr<FlightInfo>* info) override { + if (request.path.size() > 0 && request.path[0] == "error") { + return status_; + } + auto examples = ExampleFlightInfo(); + *info = std::unique_ptr<FlightInfo>(new FlightInfo(examples[0])); + return Status::OK(); + } + + Status DoGet(const ServerCallContext& context, const Ticket& request, + std::unique_ptr<FlightDataStream>* data_stream) override { + RecordBatchVector batches; + RETURN_NOT_OK(ExampleIntBatches(&batches)); + auto batch_reader = std::make_shared<BatchIterator>(batches[0]->schema(), batches); + *data_stream = std::unique_ptr<FlightDataStream>(new RecordBatchStream(batch_reader)); + return Status::OK(); + } + + void set_error_status(Status st) { status_ = std::move(st); } + + private: + Status status_; +}; + +class TestUcx : public ::testing::Test { + public: + void SetUp() { + ASSERT_OK_AND_ASSIGN(auto location, Location::ForScheme("ucx", "localhost", 0)); + ASSERT_OK(MakeServer<SimpleTestServer>( + location, &server_, &client_, + [](FlightServerOptions* options) { return Status::OK(); }, + [](FlightClientOptions* options) { return Status::OK(); })); + } + + void TearDown() { + ASSERT_OK(client_->Close()); + ASSERT_OK(server_->Shutdown()); + } + + protected: + std::unique_ptr<FlightClient> client_; + std::unique_ptr<FlightServerBase> server_; +}; + +TEST_F(TestUcx, GetFlightInfo) { + auto descriptor = FlightDescriptor::Path({"foo", "bar"}); + std::unique_ptr<FlightInfo> info; + ASSERT_OK(client_->GetFlightInfo(descriptor, &info)); + // Test that we can reuse the connection + ASSERT_OK(client_->GetFlightInfo(descriptor, &info)); +} + +TEST_F(TestUcx, SequentialClients) { + std::unique_ptr<FlightClient> client2; + ASSERT_OK(FlightClient::Connect(server_->location(), FlightClientOptions::Defaults(), + &client2)); + + Ticket ticket{"a"}; + + std::unique_ptr<FlightStreamReader> stream1, stream2; + std::shared_ptr<Table> table1, table2; + + ASSERT_OK(client_->DoGet(ticket, &stream1)); + ASSERT_OK(stream1->ReadAll(&table1)); + + ASSERT_OK(client_->DoGet(ticket, &stream2)); Review comment: Should it be `client2`? ########## File path: cpp/src/arrow/flight/transport/ucx/ucx.cc ########## @@ -0,0 +1,43 @@ +// 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.h" + +#include <mutex> + +#include "arrow/flight/transport.h" +#include "arrow/flight/transport/ucx/ucx_internal.h" +#include "arrow/flight/transport_server.h" +#include "arrow/util/logging.h" + +namespace arrow { +namespace flight { +namespace transport { +namespace ucx { + +std::once_flag kInitializeOnce; Review comment: Put it inside anonymous namespace? ########## File path: cpp/src/arrow/flight/transport/ucx/ucx_server.cc ########## @@ -0,0 +1,647 @@ +// 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_NOT_OK(driver_->SendFrame(FrameType::kBuffer, payload.data(), payload.size())); + return Status::OK(); + } + + 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"); + } +}; +} // namespace + +class ARROW_FLIGHT_EXPORT 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) { + // TODO: this pool should be resized to match CPU cores + ARROW_ASSIGN_OR_RAISE(rpc_pool_, arrow::internal::ThreadPool::Make(8)); + + struct sockaddr_storage listen_addr; + ARROW_ASSIGN_OR_RAISE(auto addrlen, UriToSockaddr(uri, &listen_addr)); + + // Init UCX + { + ucp_config_t* ucp_config; + ucp_params_t ucp_params; + ucs_status_t status; + + status = ucp_config_read(nullptr, nullptr, &ucp_config); + RETURN_NOT_OK(FromUcsStatus("ucp_config_read", status)); + + // If location is IPv6, must adjust UCX config + if (listen_addr.ss_family == AF_INET6) { + status = ucp_config_modify(ucp_config, "AF_PRIO", "inet6"); + RETURN_NOT_OK(FromUcsStatus("ucp_config_modify", status)); + } + + // Allow application to override UCP config + if (options.builder_hook) options.builder_hook(ucp_config); + + std::memset(&ucp_params, 0, sizeof(ucp_params)); + ucp_params.field_mask = + UCP_PARAM_FIELD_FEATURES | UCP_PARAM_FIELD_MT_WORKERS_SHARED; + ucp_params.features = UCP_FEATURE_AM | UCP_FEATURE_WAKEUP; + ucp_params.mt_workers_shared = UCS_THREAD_MODE_MULTI; + + ucp_context_h ucp_context; + status = ucp_init(&ucp_params, ucp_config, &ucp_context); + ucp_config_release(ucp_config); + RETURN_NOT_OK(FromUcsStatus("ucp_init", status)); + ucp_context_.reset(new UcpContext(ucp_context)); + } + + { + // Create one worker to listen for incoming connections. + ucp_worker_params_t worker_params; + ucs_status_t status; + + 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_MULTI; + ucp_worker_h worker; + status = ucp_worker_create(ucp_context_->get(), &worker_params, &worker); + RETURN_NOT_OK(FromUcsStatus("ucp_worker_create", status)); + worker_conn_.reset(new UcpWorker(ucp_context_, worker)); + } + + // Start listening for connections. + { + ucp_listener_params_t params; + ucs_status_t status; + + params.field_mask = + UCP_LISTENER_PARAM_FIELD_SOCK_ADDR | UCP_LISTENER_PARAM_FIELD_CONN_HANDLER; + params.sockaddr.addr = reinterpret_cast<const sockaddr*>(&listen_addr); + params.sockaddr.addrlen = addrlen; + params.conn_handler.cb = HandleIncomingConnection; + params.conn_handler.arg = this; + + status = ucp_listener_create(worker_conn_->get(), ¶ms, &listener_); + RETURN_NOT_OK(FromUcsStatus("ucp_listener_create", status)); + + // Get the real address/port + ucp_listener_attr_t attr; + attr.field_mask = UCP_LISTENER_ATTR_FIELD_SOCKADDR; + status = ucp_listener_query(listener_, &attr); + RETURN_NOT_OK(FromUcsStatus("ucp_listener_query", status)); + + std::string raw_uri = "ucx://"; + if (uri.host().find(':') != std::string::npos) { + // IPv6 host + raw_uri += '['; + raw_uri += uri.host(); + raw_uri += ']'; + } else { + raw_uri += uri.host(); + } + raw_uri += ":"; + raw_uri += std::to_string( + ntohs(reinterpret_cast<const sockaddr_in*>(&attr.sockaddr)->sin_port)); + std::string listen_str; + ARROW_UNUSED(SockaddrToString(attr.sockaddr).Value(&listen_str)); + FLIGHT_LOG(DEBUG) << "Listening on " << listen_str; + RETURN_NOT_OK(Location::Parse(raw_uri, &location_)); + } + + { + listening_.store(true); + std::thread listener_thread(&UcxServerImpl::DriveConnections, this); + listener_thread_.swap(listener_thread); + } + + return Status::OK(); + } + + Status Shutdown() override { + if (!listening_.load()) return Status::OK(); + Status status; + + // Wait for current RPCs to finish + listening_.store(false); + RETURN_NOT_OK( + FromUcsStatus("ucp_worker_signal", ucp_worker_signal(worker_conn_->get()))); + status &= Wait(); + + { + // Reject all pending connections + std::unique_lock<std::mutex> guard(pending_connections_mutex_); + while (!pending_connections_.empty()) { + status &= + FromUcsStatus("ucp_listener_reject", + ucp_listener_reject(listener_, pending_connections_.front())); + pending_connections_.pop(); + } + ucp_listener_destroy(listener_); + worker_conn_.reset(); + } + + status &= rpc_pool_->Shutdown(); + rpc_pool_.reset(); + + ucp_context_.reset(); + return status; + } + + Status Shutdown(const std::chrono::system_clock::time_point& deadline) override { + // TODO(lidavidm): implement shutdown with deadline + return Shutdown(); + } + + Status Wait() override { + std::unique_lock<std::mutex> guard(join_mutex_); Review comment: `lock_guard` looks more suitable, with least functions. ########## File path: cpp/src/arrow/flight/transport/ucx/flight_transport_ucx_test.cc ########## @@ -0,0 +1,399 @@ +// 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 <gmock/gmock.h> +#include <gtest/gtest.h> + +#include "arrow/array/array_base.h" +#include "arrow/flight/test_definitions.h" +#include "arrow/flight/test_util.h" +#include "arrow/flight/transport/ucx/ucx.h" +#include "arrow/table.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/util/config.h" + +#ifdef UCP_API_VERSION +#error "UCX headers should not be in public API" +#endif + +#include "arrow/flight/transport/ucx/ucx_internal.h" + +#ifdef ARROW_CUDA +#include "arrow/gpu/cuda_api.h" +#endif + +namespace arrow { +namespace flight { + +class UcxEnvironment : public ::testing::Environment { + public: + void SetUp() override { transport::ucx::InitializeFlightUcx(); } +}; + +testing::Environment* const kUcxEnvironment = + testing::AddGlobalTestEnvironment(new UcxEnvironment()); + +//------------------------------------------------------------ +// Common transport tests + +class UcxConnectivityTest : public ConnectivityTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_CONNECTIVITY(UcxConnectivityTest); + +class UcxDataTest : public DataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_DATA(UcxDataTest); + +class UcxDoPutTest : public DoPutTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_DO_PUT(UcxDoPutTest); + +class UcxAppMetadataTest : public AppMetadataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_APP_METADATA(UcxAppMetadataTest); + +class UcxIpcOptionsTest : public IpcOptionsTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_IPC_OPTIONS(UcxIpcOptionsTest); + +class UcxCudaDataTest : public CudaDataTest { + protected: + std::string transport() const override { return "ucx"; } +}; +ARROW_FLIGHT_TEST_CUDA_DATA(UcxCudaDataTest); + +//------------------------------------------------------------ +// UCX internals tests + +constexpr std::initializer_list<StatusCode> kStatusCodes = { + StatusCode::OK, + StatusCode::OutOfMemory, + StatusCode::KeyError, + StatusCode::TypeError, + StatusCode::Invalid, + StatusCode::IOError, + StatusCode::CapacityError, + StatusCode::IndexError, + StatusCode::Cancelled, + StatusCode::UnknownError, + StatusCode::NotImplemented, + StatusCode::SerializationError, + StatusCode::RError, + StatusCode::CodeGenError, + StatusCode::ExpressionValidationError, + StatusCode::ExecutionError, + StatusCode::AlreadyExists, +}; + +constexpr std::initializer_list<FlightStatusCode> kFlightStatusCodes = { + FlightStatusCode::Internal, FlightStatusCode::TimedOut, + FlightStatusCode::Cancelled, FlightStatusCode::Unauthenticated, + FlightStatusCode::Unauthorized, FlightStatusCode::Unavailable, + FlightStatusCode::Failed, +}; + +class TestStatusDetail : public StatusDetail { + public: + const char* type_id() const override { return "test-status-detail"; } + std::string ToString() const override { return "Custom status detail"; } +}; + +namespace transport { +namespace ucx { + +static constexpr std::initializer_list<FrameType> kFrameTypes = { + FrameType::kHeaders, FrameType::kBuffer, FrameType::kPayloadHeader, + FrameType::kPayloadBody, FrameType::kDisconnect, +}; + +TEST(FrameHeader, Basics) { + for (const auto frame_type : kFrameTypes) { + FrameHeader header; + ASSERT_OK(header.Set(frame_type, /*counter=*/42, /*body_size=*/65535)); + if (frame_type == FrameType::kDisconnect) { + ASSERT_RAISES(Cancelled, Frame::ParseHeader(header.data(), header.size())); + } else { + ASSERT_OK_AND_ASSIGN(auto frame, Frame::ParseHeader(header.data(), header.size())); + ASSERT_EQ(frame->type, frame_type); + ASSERT_EQ(frame->counter, 42); + ASSERT_EQ(frame->size, 65535); + } + } +} + +TEST(FrameHeader, FrameType) { + for (const auto frame_type : kFrameTypes) { + ASSERT_LE(static_cast<int>(frame_type), static_cast<int>(FrameType::kMaxFrameType)); + } +} + +TEST(HeadersFrame, Parse) { + const char* data = + ("\x00\x00\x00\x02\x00\x00\x00\x05\x00\x00\x00\x03x-foobar" + "\x00\x00\x00\x05\x00\x00\x00\x01x-bin\x01"); + constexpr int64_t size = 34; + + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), size)); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Parse(std::move(buffer))); + ASSERT_OK_AND_ASSIGN(auto foo, headers.Get("x-foo")); + ASSERT_EQ(foo, "bar"); + ASSERT_OK_AND_ASSIGN(auto bin, headers.Get("x-bin")); + ASSERT_EQ(bin, "\x01"); + } + { + std::unique_ptr<Buffer> buffer(new Buffer(reinterpret_cast<const uint8_t*>(data), 3)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected number of headers"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer(new Buffer(reinterpret_cast<const uint8_t*>(data), 7)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected length of key 1"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 10)); + EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, + ::testing::HasSubstr("expected length of value 1"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 12)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::HasSubstr("expected key 1 to have length 5, but only 0 bytes remain"), + HeadersFrame::Parse(std::move(buffer))); + } + { + std::unique_ptr<Buffer> buffer( + new Buffer(reinterpret_cast<const uint8_t*>(data), 17)); + EXPECT_RAISES_WITH_MESSAGE_THAT( + Invalid, + ::testing::HasSubstr( + "expected value 1 to have length 3, but only 0 bytes remain"), + HeadersFrame::Parse(std::move(buffer))); + } +} + +TEST(HeadersFrame, RoundTripStatus) { + for (const auto code : kStatusCodes) { + { + Status expected = code == StatusCode::OK ? Status() : Status(code, "foo"); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + + if (code == StatusCode::OK) continue; + + // Attach a generic status detail + { + auto detail = std::make_shared<TestStatusDetail>(); + Status original(code, "foo", detail); + Status expected(code, "foo", + std::make_shared<FlightStatusDetail>(FlightStatusCode::Internal, + detail->ToString())); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + + // Attach a Flight status detail + for (const auto flight_code : kFlightStatusCodes) { + Status expected(code, "foo", + std::make_shared<FlightStatusDetail>(flight_code, "extra")); + ASSERT_OK_AND_ASSIGN(auto headers, HeadersFrame::Make(expected, {})); + Status status; + ASSERT_OK(headers.GetStatus(&status)); + ASSERT_EQ(status, expected); + } + } +} +} // namespace ucx +} // namespace transport + +//------------------------------------------------------------ +// Ad-hoc UCX-specific tests + +class SimpleTestServer : public FlightServerBase { + public: + Status GetFlightInfo(const ServerCallContext& context, const FlightDescriptor& request, + std::unique_ptr<FlightInfo>* info) override { + if (request.path.size() > 0 && request.path[0] == "error") { + return status_; + } + auto examples = ExampleFlightInfo(); + *info = std::unique_ptr<FlightInfo>(new FlightInfo(examples[0])); + return Status::OK(); + } + + Status DoGet(const ServerCallContext& context, const Ticket& request, + std::unique_ptr<FlightDataStream>* data_stream) override { + RecordBatchVector batches; + RETURN_NOT_OK(ExampleIntBatches(&batches)); + auto batch_reader = std::make_shared<BatchIterator>(batches[0]->schema(), batches); + *data_stream = std::unique_ptr<FlightDataStream>(new RecordBatchStream(batch_reader)); + return Status::OK(); + } + + void set_error_status(Status st) { status_ = std::move(st); } + + private: + Status status_; +}; + +class TestUcx : public ::testing::Test { + public: + void SetUp() { + ASSERT_OK_AND_ASSIGN(auto location, Location::ForScheme("ucx", "127.0.0.1", 0)); + ASSERT_OK(MakeServer<SimpleTestServer>( + location, &server_, &client_, + [](FlightServerOptions* options) { return Status::OK(); }, + [](FlightClientOptions* options) { return Status::OK(); })); + } + + void TearDown() { + ASSERT_OK(client_->Close()); + ASSERT_OK(server_->Shutdown()); + } + + protected: + std::unique_ptr<FlightClient> client_; + std::unique_ptr<FlightServerBase> server_; +}; + +TEST_F(TestUcx, GetFlightInfo) { + auto descriptor = FlightDescriptor::Path({"foo", "bar"}); + std::unique_ptr<FlightInfo> info; + ASSERT_OK(client_->GetFlightInfo(descriptor, &info)); + // Test that we can reuse the connection + ASSERT_OK(client_->GetFlightInfo(descriptor, &info)); +} + +TEST_F(TestUcx, SequentialClients) { + std::unique_ptr<FlightClient> client2; + ASSERT_OK(FlightClient::Connect(server_->location(), FlightClientOptions::Defaults(), + &client2)); + + Ticket ticket{"a"}; + + std::unique_ptr<FlightStreamReader> stream1, stream2; + std::shared_ptr<Table> table1, table2; + + ASSERT_OK(client_->DoGet(ticket, &stream1)); + ASSERT_OK(stream1->ReadAll(&table1)); Review comment: Many deprecated warning about `ReadAll` ########## File path: cpp/src/arrow/flight/transport/ucx/ucx_server.cc ########## @@ -0,0 +1,647 @@ +// 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_NOT_OK(driver_->SendFrame(FrameType::kBuffer, payload.data(), payload.size())); + return Status::OK(); + } + + 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"); + } +}; +} // namespace + +class ARROW_FLIGHT_EXPORT 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) { Review comment: clang-13 complains this function should be marked `override` ########## File path: cpp/src/arrow/flight/transport/ucx/ucx_server.cc ########## @@ -0,0 +1,647 @@ +// 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_NOT_OK(driver_->SendFrame(FrameType::kBuffer, payload.data(), payload.size())); + return Status::OK(); + } + + 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"); + } +}; +} // namespace + +class ARROW_FLIGHT_EXPORT 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) { + // TODO: this pool should be resized to match CPU cores + ARROW_ASSIGN_OR_RAISE(rpc_pool_, arrow::internal::ThreadPool::Make(8)); + + struct sockaddr_storage listen_addr; + ARROW_ASSIGN_OR_RAISE(auto addrlen, UriToSockaddr(uri, &listen_addr)); + + // Init UCX + { + ucp_config_t* ucp_config; + ucp_params_t ucp_params; + ucs_status_t status; + + status = ucp_config_read(nullptr, nullptr, &ucp_config); + RETURN_NOT_OK(FromUcsStatus("ucp_config_read", status)); + + // If location is IPv6, must adjust UCX config + if (listen_addr.ss_family == AF_INET6) { + status = ucp_config_modify(ucp_config, "AF_PRIO", "inet6"); + RETURN_NOT_OK(FromUcsStatus("ucp_config_modify", status)); + } + + // Allow application to override UCP config + if (options.builder_hook) options.builder_hook(ucp_config); + + std::memset(&ucp_params, 0, sizeof(ucp_params)); + ucp_params.field_mask = + UCP_PARAM_FIELD_FEATURES | UCP_PARAM_FIELD_MT_WORKERS_SHARED; + ucp_params.features = UCP_FEATURE_AM | UCP_FEATURE_WAKEUP; + ucp_params.mt_workers_shared = UCS_THREAD_MODE_MULTI; + + ucp_context_h ucp_context; + status = ucp_init(&ucp_params, ucp_config, &ucp_context); + ucp_config_release(ucp_config); + RETURN_NOT_OK(FromUcsStatus("ucp_init", status)); + ucp_context_.reset(new UcpContext(ucp_context)); + } + + { + // Create one worker to listen for incoming connections. + ucp_worker_params_t worker_params; + ucs_status_t status; + + 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_MULTI; + ucp_worker_h worker; + status = ucp_worker_create(ucp_context_->get(), &worker_params, &worker); + RETURN_NOT_OK(FromUcsStatus("ucp_worker_create", status)); + worker_conn_.reset(new UcpWorker(ucp_context_, worker)); + } + + // Start listening for connections. + { + ucp_listener_params_t params; + ucs_status_t status; + + params.field_mask = + UCP_LISTENER_PARAM_FIELD_SOCK_ADDR | UCP_LISTENER_PARAM_FIELD_CONN_HANDLER; + params.sockaddr.addr = reinterpret_cast<const sockaddr*>(&listen_addr); + params.sockaddr.addrlen = addrlen; + params.conn_handler.cb = HandleIncomingConnection; + params.conn_handler.arg = this; + + status = ucp_listener_create(worker_conn_->get(), ¶ms, &listener_); + RETURN_NOT_OK(FromUcsStatus("ucp_listener_create", status)); + + // Get the real address/port + ucp_listener_attr_t attr; + attr.field_mask = UCP_LISTENER_ATTR_FIELD_SOCKADDR; + status = ucp_listener_query(listener_, &attr); + RETURN_NOT_OK(FromUcsStatus("ucp_listener_query", status)); + + std::string raw_uri = "ucx://"; + if (uri.host().find(':') != std::string::npos) { + // IPv6 host + raw_uri += '['; + raw_uri += uri.host(); + raw_uri += ']'; + } else { + raw_uri += uri.host(); + } + raw_uri += ":"; + raw_uri += std::to_string( + ntohs(reinterpret_cast<const sockaddr_in*>(&attr.sockaddr)->sin_port)); + std::string listen_str; + ARROW_UNUSED(SockaddrToString(attr.sockaddr).Value(&listen_str)); + FLIGHT_LOG(DEBUG) << "Listening on " << listen_str; + RETURN_NOT_OK(Location::Parse(raw_uri, &location_)); + } + + { + listening_.store(true); + std::thread listener_thread(&UcxServerImpl::DriveConnections, this); + listener_thread_.swap(listener_thread); + } + + return Status::OK(); + } + + Status Shutdown() override { + if (!listening_.load()) return Status::OK(); + Status status; + + // Wait for current RPCs to finish + listening_.store(false); + RETURN_NOT_OK( + FromUcsStatus("ucp_worker_signal", ucp_worker_signal(worker_conn_->get()))); + status &= Wait(); + + { + // Reject all pending connections + std::unique_lock<std::mutex> guard(pending_connections_mutex_); + while (!pending_connections_.empty()) { + status &= + FromUcsStatus("ucp_listener_reject", + ucp_listener_reject(listener_, pending_connections_.front())); + pending_connections_.pop(); + } + ucp_listener_destroy(listener_); + worker_conn_.reset(); + } + + status &= rpc_pool_->Shutdown(); + rpc_pool_.reset(); + + ucp_context_.reset(); + return status; + } + + Status Shutdown(const std::chrono::system_clock::time_point& deadline) override { + // TODO(lidavidm): implement shutdown with deadline + return Shutdown(); + } + + Status Wait() override { + std::unique_lock<std::mutex> guard(join_mutex_); + try { + listener_thread_.join(); + } catch (const std::system_error& e) { + if (e.code() != std::errc::invalid_argument) { + return Status::UnknownError("Could not Wait(): ", e.what()); + } + // Else, server wasn't running anyways + } + return Status::OK(); + } + + Location location() const override { return location_; } + + private: + struct ClientWorker { + std::shared_ptr<UcpWorker> worker; + std::unique_ptr<UcpCallDriver> driver; + }; + + Status SendStatus(UcpCallDriver* driver, const Status& status) { + ARROW_ASSIGN_OR_RAISE(auto headers, HeadersFrame::Make(status, {})); + auto payload = std::move(headers).GetBuffer(); + RETURN_NOT_OK( + driver->SendFrame(FrameType::kHeaders, payload->data(), payload->size())); + return Status::OK(); + } + + Status HandleGetFlightInfo(UcpCallDriver* driver) { + UcxServerCallContext context; + + ARROW_ASSIGN_OR_RAISE(auto frame, driver->ReadNextFrame()); + SERVER_RETURN_NOT_OK(driver, driver->ExpectFrameType(*frame, FrameType::kBuffer)); + FlightDescriptor descriptor; + SERVER_RETURN_NOT_OK(driver, + FlightDescriptor::Deserialize(util::string_view(*frame->buffer)) + .Value(&descriptor)); + + std::unique_ptr<FlightInfo> info; + std::string response; + SERVER_RETURN_NOT_OK(driver, base_->GetFlightInfo(context, descriptor, &info)); + SERVER_RETURN_NOT_OK(driver, info->SerializeToString().Value(&response)); + RETURN_NOT_OK(driver->SendFrame(FrameType::kBuffer, + reinterpret_cast<const uint8_t*>(response.data()), + static_cast<int64_t>(response.size()))); + RETURN_NOT_OK(SendStatus(driver, Status::OK())); + return Status::OK(); + } + + Status HandleDoGet(UcpCallDriver* driver) { + UcxServerCallContext context; + + ARROW_ASSIGN_OR_RAISE(auto frame, driver->ReadNextFrame()); + SERVER_RETURN_NOT_OK(driver, driver->ExpectFrameType(*frame, FrameType::kBuffer)); + Ticket ticket; + SERVER_RETURN_NOT_OK(driver, Ticket::Deserialize(frame->view()).Value(&ticket)); + + GetServerStream stream(driver); + auto status = DoGet(context, std::move(ticket), &stream); + RETURN_NOT_OK(SendStatus(driver, status)); + return Status::OK(); + } + + Status HandleDoPut(UcpCallDriver* driver) { + UcxServerCallContext context; + + PutServerStream stream(driver); + auto status = DoPut(context, &stream); + RETURN_NOT_OK(SendStatus(driver, status)); + // Must drain any unread messages, or the next call will get confused + internal::FlightData ignored; + while (stream.ReadData(&ignored)) { + } + return Status::OK(); + } + + Status HandleDoExchange(UcpCallDriver* driver) { + UcxServerCallContext context; + + ExchangeServerStream stream(driver); + auto status = DoExchange(context, &stream); + RETURN_NOT_OK(SendStatus(driver, status)); + // Must drain any unread messages, or the next call will get confused + internal::FlightData ignored; + while (stream.ReadData(&ignored)) { + } + return Status::OK(); + } + + Status HandleOneCall(UcpCallDriver* driver, Frame* frame) { + SERVER_RETURN_NOT_OK(driver, driver->ExpectFrameType(*frame, FrameType::kHeaders)); + ARROW_ASSIGN_OR_RAISE(auto headers, HeadersFrame::Parse(std::move(frame->buffer))); + ARROW_ASSIGN_OR_RAISE(auto method, headers.Get(":method:")); + if (method == kMethodGetFlightInfo) { + return HandleGetFlightInfo(driver); + } else if (method == kMethodDoExchange) { + return HandleDoExchange(driver); + } else if (method == kMethodDoGet) { + return HandleDoGet(driver); + } else if (method == kMethodDoPut) { + return HandleDoPut(driver); + } + RETURN_NOT_OK(SendStatus(driver, Status::NotImplemented(method))); + return Status::OK(); + } + + void WorkerLoop(ucp_conn_request_h request) { + std::string peer = "unknown:" + std::to_string(counter_++); + { + ucp_conn_request_attr_t request_attr; + std::memset(&request_attr, 0, sizeof(request_attr)); + request_attr.field_mask = UCP_CONN_REQUEST_ATTR_FIELD_CLIENT_ADDR; + if (ucp_conn_request_query(request, &request_attr) == UCS_OK) { + ARROW_UNUSED(SockaddrToString(request_attr.client_address).Value(&peer)); + } + } + FLIGHT_LOG_PEER(DEBUG, peer) << "Received connection request"; + + auto maybe_worker = CreateWorker(); + if (!maybe_worker.ok()) { + FLIGHT_LOG_PEER(WARNING, peer) + << "Failed to create worker" << maybe_worker.status().ToString(); + auto status = ucp_listener_reject(listener_, request); + if (status != UCS_OK) { + FLIGHT_LOG_PEER(WARNING, peer) + << FromUcsStatus("ucp_listener_reject", status).ToString(); + } + return; + } + auto worker = maybe_worker.MoveValueUnsafe(); + + // Create an endpoint to the client, using the data worker + { + ucs_status_t status; + ucp_ep_params_t params; + std::memset(¶ms, 0, sizeof(params)); + params.field_mask = UCP_EP_PARAM_FIELD_CONN_REQUEST; + params.conn_request = request; + + ucp_ep_h client_endpoint; + + status = ucp_ep_create(worker->worker->get(), ¶ms, &client_endpoint); + if (status != UCS_OK) { + FLIGHT_LOG_PEER(WARNING, peer) + << "Failed to create endpoint: " + << FromUcsStatus("ucp_ep_create", status).ToString(); + return; + } + worker->driver.reset(new UcpCallDriver(worker->worker, client_endpoint)); + worker->driver->set_memory_manager(memory_manager_); + peer = worker->driver->peer(); + } + + while (listening_.load()) { + auto maybe_frame = worker->driver->ReadNextFrame(); + if (!maybe_frame.ok()) { + if (!maybe_frame.status().IsCancelled()) { + FLIGHT_LOG_PEER(WARNING, peer) + << "Failed to read next message: " << maybe_frame.status().ToString(); + } + break; + } + + auto status = HandleOneCall(worker->driver.get(), maybe_frame->get()); + if (!status.ok()) { + FLIGHT_LOG_PEER(WARNING, peer) << "Call failed: " << status.ToString(); + break; + } + } + + // Clean up + auto status = worker->driver->Close(); + if (!status.ok()) { + FLIGHT_LOG_PEER(WARNING, peer) << "Failed to close worker: " << status.ToString(); + } + worker->worker.reset(); + FLIGHT_LOG_PEER(DEBUG, peer) << "Disconnected"; + } + + void DriveConnections() { + while (listening_.load()) { + while (ucp_worker_progress(worker_conn_->get())) { + } + { + // Check for connect requests in queue + std::unique_lock<std::mutex> guard(pending_connections_mutex_); + while (!pending_connections_.empty()) { + ucp_conn_request_h request = pending_connections_.front(); + pending_connections_.pop(); + + auto submitted = rpc_pool_->Submit([this, request]() { WorkerLoop(request); }); + if (!submitted.ok()) { + ARROW_LOG(WARNING) << "Failed to submit task to handle client " + << submitted.status().ToString(); + } + } + } + + if (!listening_.load()) break; + auto status = ucp_worker_wait(worker_conn_->get()); + if (status != UCS_OK) { + FLIGHT_LOG(WARNING) << FromUcsStatus("ucp_worker_wait", status).ToString(); + } + } + } + + void EnqueueClient(ucp_conn_request_h connection_request) { + std::unique_lock<std::mutex> guard(pending_connections_mutex_); + pending_connections_.push(connection_request); + guard.unlock(); + } + + arrow::Result<std::shared_ptr<ClientWorker>> CreateWorker() { + auto worker = std::make_shared<ClientWorker>(); + + 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_SINGLE; + + ucp_worker_h ucp_worker; + auto status = ucp_worker_create(ucp_context_->get(), &worker_params, &ucp_worker); + RETURN_NOT_OK(FromUcsStatus("ucp_worker_create", status)); + worker->worker.reset(new UcpWorker(ucp_context_, ucp_worker)); + + // Set up Active Message (AM) handler + ucp_am_handler_param_t handler_params; + std::memset(&handler_params, 0, sizeof(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 = worker.get(); + + status = ucp_worker_set_am_recv_handler(worker->worker->get(), &handler_params); + RETURN_NOT_OK(FromUcsStatus("ucp_worker_set_am_recv_handler", status)); + return worker; + } + + /// Callback handler. A new client has connected to the server. + static void HandleIncomingConnection(ucp_conn_request_h connection_request, + void* data) { + UcxServerImpl* server = reinterpret_cast<UcxServerImpl*>(data); Review comment: Wondering how the lifecycle of `server` object is guaranteed safe for this callback. ########## File path: cpp/src/arrow/flight/transport/ucx/util_internal.h ########## @@ -0,0 +1,61 @@ +// 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. + +#pragma once + +#include <arpa/inet.h> +#include <ucp/api/ucp.h> +#include <string> + +#include "arrow/flight/visibility.h" +#include "arrow/util/uri.h" + +namespace arrow { +namespace flight { +namespace transport { +namespace ucx { + +static inline void UInt32ToBytesBe(const uint32_t in, uint8_t* out) { + out[0] = static_cast<uint8_t>((in >> 24) & 0xFF); + out[1] = static_cast<uint8_t>((in >> 16) & 0xFF); + out[2] = static_cast<uint8_t>((in >> 8) & 0xFF); + out[3] = static_cast<uint8_t>(in & 0xFF); +} + +static inline uint32_t BytesToUInt32Be(const uint8_t* in) { + return static_cast<uint32_t>(in[3]) | (static_cast<uint32_t>(in[2]) << 8) | + (static_cast<uint32_t>(in[1]) << 16) | (static_cast<uint32_t>(in[0]) << 24); +} Review comment: `util::ToBigEndian(util::SafeLoadAs<uint32_t>(in))`? ########## File path: cpp/src/arrow/flight/transport/ucx/util_internal.h ########## @@ -0,0 +1,61 @@ +// 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. + +#pragma once + +#include <arpa/inet.h> +#include <ucp/api/ucp.h> +#include <string> + +#include "arrow/flight/visibility.h" +#include "arrow/util/uri.h" + +namespace arrow { +namespace flight { +namespace transport { +namespace ucx { + +static inline void UInt32ToBytesBe(const uint32_t in, uint8_t* out) { + out[0] = static_cast<uint8_t>((in >> 24) & 0xFF); + out[1] = static_cast<uint8_t>((in >> 16) & 0xFF); + out[2] = static_cast<uint8_t>((in >> 8) & 0xFF); + out[3] = static_cast<uint8_t>(in & 0xFF); +} Review comment: Will `util::SafeStore(out, util::ToBigEndian(in))` do the trick? Or use `htonl`? ########## File path: 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 { + +class UcxClientImpl; + +namespace { + +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(), ¶ms, &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); + 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(); }); Review comment: This looks a bit fragile. If `outgoing_` contains erros, will DriveWork() sleep immediately after wakeup as the stop_wait flag is false (line297), and cause `received_cv_.wait()` stucks here? ########## File path: cpp/src/arrow/flight/transport/ucx/ucx_server.cc ########## @@ -0,0 +1,647 @@ +// 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_NOT_OK(driver_->SendFrame(FrameType::kBuffer, payload.data(), payload.size())); + return Status::OK(); + } + + 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"); + } +}; +} // namespace + +class ARROW_FLIGHT_EXPORT 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) { + // TODO: this pool should be resized to match CPU cores + ARROW_ASSIGN_OR_RAISE(rpc_pool_, arrow::internal::ThreadPool::Make(8)); + + struct sockaddr_storage listen_addr; + ARROW_ASSIGN_OR_RAISE(auto addrlen, UriToSockaddr(uri, &listen_addr)); + + // Init UCX + { + ucp_config_t* ucp_config; + ucp_params_t ucp_params; + ucs_status_t status; + + status = ucp_config_read(nullptr, nullptr, &ucp_config); + RETURN_NOT_OK(FromUcsStatus("ucp_config_read", status)); + + // If location is IPv6, must adjust UCX config + if (listen_addr.ss_family == AF_INET6) { + status = ucp_config_modify(ucp_config, "AF_PRIO", "inet6"); + RETURN_NOT_OK(FromUcsStatus("ucp_config_modify", status)); + } + + // Allow application to override UCP config + if (options.builder_hook) options.builder_hook(ucp_config); + + std::memset(&ucp_params, 0, sizeof(ucp_params)); + ucp_params.field_mask = + UCP_PARAM_FIELD_FEATURES | UCP_PARAM_FIELD_MT_WORKERS_SHARED; + ucp_params.features = UCP_FEATURE_AM | UCP_FEATURE_WAKEUP; + ucp_params.mt_workers_shared = UCS_THREAD_MODE_MULTI; + + ucp_context_h ucp_context; + status = ucp_init(&ucp_params, ucp_config, &ucp_context); + ucp_config_release(ucp_config); + RETURN_NOT_OK(FromUcsStatus("ucp_init", status)); + ucp_context_.reset(new UcpContext(ucp_context)); + } + + { + // Create one worker to listen for incoming connections. + ucp_worker_params_t worker_params; + ucs_status_t status; + + 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_MULTI; + ucp_worker_h worker; + status = ucp_worker_create(ucp_context_->get(), &worker_params, &worker); + RETURN_NOT_OK(FromUcsStatus("ucp_worker_create", status)); + worker_conn_.reset(new UcpWorker(ucp_context_, worker)); + } + + // Start listening for connections. + { + ucp_listener_params_t params; + ucs_status_t status; + + params.field_mask = + UCP_LISTENER_PARAM_FIELD_SOCK_ADDR | UCP_LISTENER_PARAM_FIELD_CONN_HANDLER; + params.sockaddr.addr = reinterpret_cast<const sockaddr*>(&listen_addr); + params.sockaddr.addrlen = addrlen; + params.conn_handler.cb = HandleIncomingConnection; + params.conn_handler.arg = this; + + status = ucp_listener_create(worker_conn_->get(), ¶ms, &listener_); + RETURN_NOT_OK(FromUcsStatus("ucp_listener_create", status)); + + // Get the real address/port + ucp_listener_attr_t attr; + attr.field_mask = UCP_LISTENER_ATTR_FIELD_SOCKADDR; + status = ucp_listener_query(listener_, &attr); + RETURN_NOT_OK(FromUcsStatus("ucp_listener_query", status)); + + std::string raw_uri = "ucx://"; + if (uri.host().find(':') != std::string::npos) { + // IPv6 host + raw_uri += '['; + raw_uri += uri.host(); + raw_uri += ']'; + } else { + raw_uri += uri.host(); + } + raw_uri += ":"; + raw_uri += std::to_string( + ntohs(reinterpret_cast<const sockaddr_in*>(&attr.sockaddr)->sin_port)); + std::string listen_str; + ARROW_UNUSED(SockaddrToString(attr.sockaddr).Value(&listen_str)); + FLIGHT_LOG(DEBUG) << "Listening on " << listen_str; + RETURN_NOT_OK(Location::Parse(raw_uri, &location_)); + } + + { + listening_.store(true); + std::thread listener_thread(&UcxServerImpl::DriveConnections, this); + listener_thread_.swap(listener_thread); + } + + return Status::OK(); + } + + Status Shutdown() override { + if (!listening_.load()) return Status::OK(); + Status status; + + // Wait for current RPCs to finish + listening_.store(false); + RETURN_NOT_OK( + FromUcsStatus("ucp_worker_signal", ucp_worker_signal(worker_conn_->get()))); + status &= Wait(); + + { + // Reject all pending connections + std::unique_lock<std::mutex> guard(pending_connections_mutex_); + while (!pending_connections_.empty()) { + status &= + FromUcsStatus("ucp_listener_reject", + ucp_listener_reject(listener_, pending_connections_.front())); + pending_connections_.pop(); + } + ucp_listener_destroy(listener_); + worker_conn_.reset(); + } + + status &= rpc_pool_->Shutdown(); + rpc_pool_.reset(); + + ucp_context_.reset(); + return status; + } + + Status Shutdown(const std::chrono::system_clock::time_point& deadline) override { + // TODO(lidavidm): implement shutdown with deadline + return Shutdown(); + } + + Status Wait() override { + std::unique_lock<std::mutex> guard(join_mutex_); + try { + listener_thread_.join(); + } catch (const std::system_error& e) { + if (e.code() != std::errc::invalid_argument) { + return Status::UnknownError("Could not Wait(): ", e.what()); + } + // Else, server wasn't running anyways + } + return Status::OK(); + } + + Location location() const override { return location_; } + + private: + struct ClientWorker { + std::shared_ptr<UcpWorker> worker; + std::unique_ptr<UcpCallDriver> driver; + }; + + Status SendStatus(UcpCallDriver* driver, const Status& status) { + ARROW_ASSIGN_OR_RAISE(auto headers, HeadersFrame::Make(status, {})); + auto payload = std::move(headers).GetBuffer(); + RETURN_NOT_OK( + driver->SendFrame(FrameType::kHeaders, payload->data(), payload->size())); + return Status::OK(); + } + + Status HandleGetFlightInfo(UcpCallDriver* driver) { + UcxServerCallContext context; + + ARROW_ASSIGN_OR_RAISE(auto frame, driver->ReadNextFrame()); + SERVER_RETURN_NOT_OK(driver, driver->ExpectFrameType(*frame, FrameType::kBuffer)); + FlightDescriptor descriptor; + SERVER_RETURN_NOT_OK(driver, + FlightDescriptor::Deserialize(util::string_view(*frame->buffer)) + .Value(&descriptor)); + + std::unique_ptr<FlightInfo> info; + std::string response; + SERVER_RETURN_NOT_OK(driver, base_->GetFlightInfo(context, descriptor, &info)); + SERVER_RETURN_NOT_OK(driver, info->SerializeToString().Value(&response)); + RETURN_NOT_OK(driver->SendFrame(FrameType::kBuffer, + reinterpret_cast<const uint8_t*>(response.data()), + static_cast<int64_t>(response.size()))); + RETURN_NOT_OK(SendStatus(driver, Status::OK())); + return Status::OK(); + } + + Status HandleDoGet(UcpCallDriver* driver) { + UcxServerCallContext context; + + ARROW_ASSIGN_OR_RAISE(auto frame, driver->ReadNextFrame()); + SERVER_RETURN_NOT_OK(driver, driver->ExpectFrameType(*frame, FrameType::kBuffer)); + Ticket ticket; + SERVER_RETURN_NOT_OK(driver, Ticket::Deserialize(frame->view()).Value(&ticket)); + + GetServerStream stream(driver); + auto status = DoGet(context, std::move(ticket), &stream); + RETURN_NOT_OK(SendStatus(driver, status)); + return Status::OK(); + } + + Status HandleDoPut(UcpCallDriver* driver) { + UcxServerCallContext context; + + PutServerStream stream(driver); + auto status = DoPut(context, &stream); + RETURN_NOT_OK(SendStatus(driver, status)); + // Must drain any unread messages, or the next call will get confused + internal::FlightData ignored; + while (stream.ReadData(&ignored)) { + } + return Status::OK(); + } + + Status HandleDoExchange(UcpCallDriver* driver) { + UcxServerCallContext context; + + ExchangeServerStream stream(driver); + auto status = DoExchange(context, &stream); + RETURN_NOT_OK(SendStatus(driver, status)); + // Must drain any unread messages, or the next call will get confused + internal::FlightData ignored; + while (stream.ReadData(&ignored)) { + } + return Status::OK(); + } + + Status HandleOneCall(UcpCallDriver* driver, Frame* frame) { + SERVER_RETURN_NOT_OK(driver, driver->ExpectFrameType(*frame, FrameType::kHeaders)); + ARROW_ASSIGN_OR_RAISE(auto headers, HeadersFrame::Parse(std::move(frame->buffer))); + ARROW_ASSIGN_OR_RAISE(auto method, headers.Get(":method:")); + if (method == kMethodGetFlightInfo) { + return HandleGetFlightInfo(driver); + } else if (method == kMethodDoExchange) { + return HandleDoExchange(driver); + } else if (method == kMethodDoGet) { + return HandleDoGet(driver); + } else if (method == kMethodDoPut) { + return HandleDoPut(driver); + } + RETURN_NOT_OK(SendStatus(driver, Status::NotImplemented(method))); + return Status::OK(); + } + + void WorkerLoop(ucp_conn_request_h request) { + std::string peer = "unknown:" + std::to_string(counter_++); + { + ucp_conn_request_attr_t request_attr; + std::memset(&request_attr, 0, sizeof(request_attr)); + request_attr.field_mask = UCP_CONN_REQUEST_ATTR_FIELD_CLIENT_ADDR; + if (ucp_conn_request_query(request, &request_attr) == UCS_OK) { + ARROW_UNUSED(SockaddrToString(request_attr.client_address).Value(&peer)); + } + } + FLIGHT_LOG_PEER(DEBUG, peer) << "Received connection request"; + + auto maybe_worker = CreateWorker(); + if (!maybe_worker.ok()) { + FLIGHT_LOG_PEER(WARNING, peer) + << "Failed to create worker" << maybe_worker.status().ToString(); + auto status = ucp_listener_reject(listener_, request); + if (status != UCS_OK) { + FLIGHT_LOG_PEER(WARNING, peer) + << FromUcsStatus("ucp_listener_reject", status).ToString(); + } + return; + } + auto worker = maybe_worker.MoveValueUnsafe(); + + // Create an endpoint to the client, using the data worker + { + ucs_status_t status; + ucp_ep_params_t params; + std::memset(¶ms, 0, sizeof(params)); + params.field_mask = UCP_EP_PARAM_FIELD_CONN_REQUEST; + params.conn_request = request; + + ucp_ep_h client_endpoint; + + status = ucp_ep_create(worker->worker->get(), ¶ms, &client_endpoint); + if (status != UCS_OK) { + FLIGHT_LOG_PEER(WARNING, peer) + << "Failed to create endpoint: " + << FromUcsStatus("ucp_ep_create", status).ToString(); + return; + } + worker->driver.reset(new UcpCallDriver(worker->worker, client_endpoint)); + worker->driver->set_memory_manager(memory_manager_); + peer = worker->driver->peer(); + } + + while (listening_.load()) { + auto maybe_frame = worker->driver->ReadNextFrame(); + if (!maybe_frame.ok()) { + if (!maybe_frame.status().IsCancelled()) { + FLIGHT_LOG_PEER(WARNING, peer) + << "Failed to read next message: " << maybe_frame.status().ToString(); + } + break; + } + + auto status = HandleOneCall(worker->driver.get(), maybe_frame->get()); + if (!status.ok()) { + FLIGHT_LOG_PEER(WARNING, peer) << "Call failed: " << status.ToString(); + break; + } + } + + // Clean up + auto status = worker->driver->Close(); + if (!status.ok()) { + FLIGHT_LOG_PEER(WARNING, peer) << "Failed to close worker: " << status.ToString(); + } + worker->worker.reset(); + FLIGHT_LOG_PEER(DEBUG, peer) << "Disconnected"; + } + + void DriveConnections() { + while (listening_.load()) { + while (ucp_worker_progress(worker_conn_->get())) { + } + { + // Check for connect requests in queue + std::unique_lock<std::mutex> guard(pending_connections_mutex_); + while (!pending_connections_.empty()) { + ucp_conn_request_h request = pending_connections_.front(); + pending_connections_.pop(); + + auto submitted = rpc_pool_->Submit([this, request]() { WorkerLoop(request); }); + if (!submitted.ok()) { + ARROW_LOG(WARNING) << "Failed to submit task to handle client " + << submitted.status().ToString(); + } + } + } + + if (!listening_.load()) break; + auto status = ucp_worker_wait(worker_conn_->get()); + if (status != UCS_OK) { + FLIGHT_LOG(WARNING) << FromUcsStatus("ucp_worker_wait", status).ToString(); + } + } + } + + void EnqueueClient(ucp_conn_request_h connection_request) { + std::unique_lock<std::mutex> guard(pending_connections_mutex_); + pending_connections_.push(connection_request); + guard.unlock(); + } + + arrow::Result<std::shared_ptr<ClientWorker>> CreateWorker() { + auto worker = std::make_shared<ClientWorker>(); + + 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_SINGLE; + + ucp_worker_h ucp_worker; + auto status = ucp_worker_create(ucp_context_->get(), &worker_params, &ucp_worker); + RETURN_NOT_OK(FromUcsStatus("ucp_worker_create", status)); + worker->worker.reset(new UcpWorker(ucp_context_, ucp_worker)); + + // Set up Active Message (AM) handler + ucp_am_handler_param_t handler_params; + std::memset(&handler_params, 0, sizeof(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 = worker.get(); + + status = ucp_worker_set_am_recv_handler(worker->worker->get(), &handler_params); + RETURN_NOT_OK(FromUcsStatus("ucp_worker_set_am_recv_handler", status)); + return worker; + } + + /// Callback handler. A new client has connected to the server. Review comment: `///` ->`//` ? ########## File path: 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 { + +class UcxClientImpl; + +namespace { + +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(), ¶ms, &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); + 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() { + while (true) { + { + std::unique_lock<std::mutex> guard(driver_mutex_); + working_cv_.wait(guard, [this] { + return finished_ || incoming_.is_valid() || outgoing_.is_valid(); Review comment: `finished_` is checked in stop condition but looks it doesn't notify the CV. Shall we notify CV when `finished_` becomes true? As spurious wakeup can happen at any time, will `incoming_` and `outgoing_` hold stale values? ########## File path: cpp/src/arrow/flight/transport/ucx/ucx_server.cc ########## @@ -0,0 +1,647 @@ +// 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_NOT_OK(driver_->SendFrame(FrameType::kBuffer, payload.data(), payload.size())); + return Status::OK(); + } + + 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"); + } +}; +} // namespace + +class ARROW_FLIGHT_EXPORT UcxServerImpl Review comment: Is it necessary to export this class? ########## File path: cpp/src/arrow/flight/transport/ucx/util_internal.cc ########## @@ -0,0 +1,251 @@ +// 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 { + +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, info->ai_addr, info->ai_addrlen); Review comment: Should we copy from `cur_info`? ########## File path: cpp/src/arrow/flight/transport/ucx/ucx_server.cc ########## @@ -0,0 +1,647 @@ +// 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_NOT_OK(driver_->SendFrame(FrameType::kBuffer, payload.data(), payload.size())); + return Status::OK(); Review comment: Same as `return driver_->SendFrame(...)`? -- 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]
