lidavidm commented on a change in pull request #12465: URL: https://github.com/apache/arrow/pull/12465#discussion_r813965377
########## File path: cpp/src/arrow/flight/transport_server_impl.cc ########## @@ -0,0 +1,327 @@ +// 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_impl.h" + +#include <unordered_map> + +#include "arrow/buffer.h" +#include "arrow/flight/serialization_internal.h" +#include "arrow/flight/server.h" +#include "arrow/flight/types.h" +#include "arrow/ipc/reader.h" +#include "arrow/result.h" +#include "arrow/status.h" + +namespace arrow { +namespace flight { +namespace internal { + +namespace { +class TransportIpcMessageReader : public ipc::MessageReader { + public: + explicit TransportIpcMessageReader( + std::shared_ptr<internal::PeekableFlightDataReader> peekable_reader, + std::shared_ptr<MemoryManager> memory_manager, + std::shared_ptr<Buffer>* app_metadata) + : peekable_reader_(peekable_reader), + memory_manager_(std::move(memory_manager)), + app_metadata_(app_metadata) {} + + ::arrow::Result<std::unique_ptr<ipc::Message>> ReadNextMessage() override { + if (stream_finished_) { + return nullptr; + } + internal::FlightData* data; + peekable_reader_->Next(&data); + if (!data) { + stream_finished_ = true; + if (first_message_) { Review comment: Hmm, I'll have to check why tests didn't catch this - thanks ########## File path: cpp/src/arrow/flight/transport_server_impl.cc ########## @@ -0,0 +1,327 @@ +// 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_impl.h" + +#include <unordered_map> + +#include "arrow/buffer.h" +#include "arrow/flight/serialization_internal.h" +#include "arrow/flight/server.h" +#include "arrow/flight/types.h" +#include "arrow/ipc/reader.h" +#include "arrow/result.h" +#include "arrow/status.h" + +namespace arrow { +namespace flight { +namespace internal { + +namespace { +class TransportIpcMessageReader : public ipc::MessageReader { + public: + explicit TransportIpcMessageReader( + std::shared_ptr<internal::PeekableFlightDataReader> peekable_reader, + std::shared_ptr<MemoryManager> memory_manager, + std::shared_ptr<Buffer>* app_metadata) + : peekable_reader_(peekable_reader), + memory_manager_(std::move(memory_manager)), + app_metadata_(app_metadata) {} + + ::arrow::Result<std::unique_ptr<ipc::Message>> ReadNextMessage() override { + if (stream_finished_) { + return nullptr; + } + internal::FlightData* data; + peekable_reader_->Next(&data); + if (!data) { + stream_finished_ = true; + if (first_message_) { + return Status::Invalid( + "Client provided malformed message or did not provide message"); + } + return nullptr; + } + if (data->body && + ARROW_PREDICT_FALSE(!data->body->device()->Equals(*memory_manager_->device()))) { + ARROW_ASSIGN_OR_RAISE(data->body, Buffer::ViewOrCopy(data->body, memory_manager_)); + } + *app_metadata_ = std::move(data->app_metadata); + return data->OpenMessage(); + } + + protected: + std::shared_ptr<internal::PeekableFlightDataReader> peekable_reader_; + std::shared_ptr<MemoryManager> memory_manager_; + // A reference to TransportDataStream.app_metadata_. That class + // can't access the app metadata because when it Peek()s the stream, + // it may be looking at a dictionary batch, not the record + // batch. Updating it here ensures the reader is always updated with + // the last metadata message read. + std::shared_ptr<Buffer>* app_metadata_; + bool first_message_ = true; + bool stream_finished_ = false; +}; + +/// \brief Adapt TransportDataStream to the FlightMessageReader +/// interface for DoPut. +class TransportMessageReader final : public FlightMessageReader { + public: + explicit TransportMessageReader(TransportDataStream* stream, + std::shared_ptr<MemoryManager> memory_manager) + : peekable_reader_(new internal::PeekableFlightDataReader(stream)), + memory_manager_(std::move(memory_manager)) {} + + Status Init() { + // Peek the first message to get the descriptor. + internal::FlightData* data; + peekable_reader_->Peek(&data); + if (!data) { + return Status::IOError("Stream finished before first message sent"); + } + if (!data->descriptor) { + return Status::IOError("Descriptor missing on first message"); + } + descriptor_ = *data->descriptor; + // If there's a schema (=DoPut), also Open(). + if (data->metadata) { + return EnsureDataStarted(); + } + peekable_reader_->Next(&data); + return Status::OK(); + } + + const FlightDescriptor& descriptor() const override { return descriptor_; } + + arrow::Result<std::shared_ptr<Schema>> GetSchema() override { + RETURN_NOT_OK(EnsureDataStarted()); + return batch_reader_->schema(); + } + + Status Next(FlightStreamChunk* out) override { + internal::FlightData* data; + peekable_reader_->Peek(&data); + if (!data) { + out->app_metadata = nullptr; + out->data = nullptr; + return Status::OK(); + } + + if (!data->metadata) { + // Metadata-only (data->metadata is the IPC header) + out->app_metadata = data->app_metadata; + out->data = nullptr; + peekable_reader_->Next(&data); + return Status::OK(); + } + + if (!batch_reader_) { + RETURN_NOT_OK(EnsureDataStarted()); + // re-peek here since EnsureDataStarted() advances the stream + return Next(out); + } + RETURN_NOT_OK(batch_reader_->ReadNext(&out->data)); + out->app_metadata = std::move(app_metadata_); + return Status::OK(); + } + + private: + /// Ensure we are set up to read data. + Status EnsureDataStarted() { + if (!batch_reader_) { + // peek() until we find the first data message; discard metadata + if (!peekable_reader_->SkipToData()) { + return Status::IOError("Client never sent a data message"); + } + auto message_reader = + std::unique_ptr<ipc::MessageReader>(new TransportIpcMessageReader( + peekable_reader_, memory_manager_, &app_metadata_)); + ARROW_ASSIGN_OR_RAISE( + batch_reader_, ipc::RecordBatchStreamReader::Open(std::move(message_reader))); + } + return Status::OK(); + } + + FlightDescriptor descriptor_; + std::shared_ptr<internal::PeekableFlightDataReader> peekable_reader_; + std::shared_ptr<MemoryManager> memory_manager_; + std::shared_ptr<RecordBatchReader> batch_reader_; + std::shared_ptr<Buffer> app_metadata_; +}; + +// TODO(ARROW-10787): this should use the same writer/ipc trick as client +class TransportMessageWriter final : public FlightMessageWriter { + public: + explicit TransportMessageWriter(TransportDataStream* stream) + : stream_(stream), ipc_options_(::arrow::ipc::IpcWriteOptions::Defaults()) {} + + Status Begin(const std::shared_ptr<Schema>& schema, + const ipc::IpcWriteOptions& options) override { + if (started_) { + return Status::Invalid("This writer has already been started."); + } + started_ = true; + ipc_options_ = options; + + RETURN_NOT_OK(mapper_.AddSchemaFields(*schema)); + FlightPayload schema_payload; + RETURN_NOT_OK(ipc::GetSchemaPayload(*schema, ipc_options_, mapper_, + &schema_payload.ipc_message)); + return WritePayload(schema_payload); + } + + Status WriteRecordBatch(const RecordBatch& batch) override { + return WriteWithMetadata(batch, nullptr); + } + + Status WriteMetadata(std::shared_ptr<Buffer> app_metadata) override { + FlightPayload payload{}; + payload.app_metadata = app_metadata; + return WritePayload(payload); + } + + Status WriteWithMetadata(const RecordBatch& batch, + std::shared_ptr<Buffer> app_metadata) override { + RETURN_NOT_OK(CheckStarted()); + RETURN_NOT_OK(EnsureDictionariesWritten(batch)); + FlightPayload payload{}; + if (app_metadata) { + payload.app_metadata = app_metadata; + } + RETURN_NOT_OK(ipc::GetRecordBatchPayload(batch, ipc_options_, &payload.ipc_message)); + RETURN_NOT_OK(WritePayload(payload)); + ++stats_.num_record_batches; + return Status::OK(); + } + + Status Close() override { + // It's fine to Close() without writing data + return Status::OK(); + } + + ipc::WriteStats stats() const override { return stats_; } + + private: + Status WritePayload(const FlightPayload& payload) { + RETURN_NOT_OK(stream_->WriteData(payload)); + ++stats_.num_messages; + return Status::OK(); + } + + Status CheckStarted() { + if (!started_) { + return Status::Invalid("This writer is not started. Call Begin() with a schema"); + } + return Status::OK(); + } + + Status EnsureDictionariesWritten(const RecordBatch& batch) { + if (dictionaries_written_) { + return Status::OK(); + } + dictionaries_written_ = true; + ARROW_ASSIGN_OR_RAISE(const auto dictionaries, + ipc::CollectDictionaries(batch, mapper_)); + for (const auto& pair : dictionaries) { + FlightPayload payload{}; + RETURN_NOT_OK(ipc::GetDictionaryPayload(pair.first, pair.second, ipc_options_, + &payload.ipc_message)); + RETURN_NOT_OK(WritePayload(payload)); + ++stats_.num_dictionary_batches; + } + return Status::OK(); + } + + TransportDataStream* stream_; + ::arrow::ipc::IpcWriteOptions ipc_options_; + ipc::DictionaryFieldMapper mapper_; + ipc::WriteStats stats_; + bool started_ = false; + bool dictionaries_written_ = false; +}; + +/// \brief Adapt TransportDataStream to the FlightMetadataWriter +/// interface for DoPut. +class TransportMetadataWriter final : public FlightMetadataWriter { + public: + explicit TransportMetadataWriter(TransportDataStream* stream) : stream_(stream) {} + + Status WriteMetadata(const Buffer& buffer) override { + return stream_->WritePutMetadata(buffer); + } + + private: + TransportDataStream* stream_; +}; +} // namespace + +Status FlightServiceImpl::DoGet(const ServerCallContext& context, const Ticket& ticket, + TransportDataStream* stream) { + std::unique_ptr<FlightDataStream> data_stream; + RETURN_NOT_OK(service_->DoGet(context, ticket, &data_stream)); + + if (!data_stream) return Status::KeyError("No data in this flight"); + + // Write the schema as the first message in the stream + FlightPayload schema_payload; + RETURN_NOT_OK(data_stream->GetSchemaPayload(&schema_payload)); + auto status = stream->WriteData(schema_payload); + // Connection terminated + if (status.IsIOError()) return Status::OK(); + RETURN_NOT_OK(status); + + // Consume data stream and write out payloads + while (true) { + FlightPayload payload; + RETURN_NOT_OK(data_stream->Next(&payload)); + // End of stream + if (payload.ipc_message.metadata == nullptr) break; + auto status = stream->WriteData(payload); + // Ignore IOError (used to signal that client disconnected; there's nothing + // we can do - e.g. see WritePayload in serialization_internal.cc) + if (status.IsIOError()) return Status::OK(); + RETURN_NOT_OK(status); + } + RETURN_NOT_OK(stream->WritesDone()); + return Status::OK(); +} + +Status FlightServiceImpl::DoPut(const ServerCallContext& context, + TransportDataStream* stream) { + std::unique_ptr<FlightMessageReader> reader( + new TransportMessageReader(stream, memory_manager_)); + std::unique_ptr<FlightMetadataWriter> writer(new TransportMetadataWriter(stream)); + RETURN_NOT_OK(reinterpret_cast<TransportMessageReader*>(reader.get())->Init()); Review comment: Ah, yes, we can just declare the unique_ptr as the subclass and its type will be converted when we std::move it, thanks! ########## File path: cpp/src/arrow/flight/client.cc ########## @@ -428,22 +298,18 @@ class GrpcClientAuthReader : public ClientAuthReader { stream_; }; -// An ipc::MessageReader that adapts any readable gRPC stream -// returning FlightData. -template <typename Reader> -class GrpcIpcMessageReader : public ipc::MessageReader { +/// \brief An ipc::MessageReader adapting the Flight ClientDataStream interface. +/// +/// In order to support app_metadata and reuse the existing IPC +/// infrastructure, this takes a pointer to a buffer (provided by the +/// FlightStreamReader implementation) and upon reading a message, +/// updates that buffer with the one read from the server. +class IpcMessageReader : public ipc::MessageReader { Review comment: Some classes are general. I would like to split them up further in the future, e.g. move out all the gRPC stuff into `arrow/flight/transport/grpc` like how the UCX transport is structured. ########## File path: cpp/src/arrow/flight/test_util.h ########## @@ -82,24 +82,43 @@ std::unique_ptr<FlightServerBase> ExampleTestServer(); // Helper to initialize a server and matching client with callbacks to // populate options. template <typename T, typename... Args> -Status MakeServer(std::unique_ptr<FlightServerBase>* server, +Status MakeServer(const Location& location, std::unique_ptr<FlightServerBase>* server, std::unique_ptr<FlightClient>* client, std::function<Status(FlightServerOptions*)> make_server_options, std::function<Status(FlightClientOptions*)> make_client_options, Args&&... server_args) { - Location location; - RETURN_NOT_OK(Location::ForGrpcTcp("localhost", 0, &location)); *server = arrow::internal::make_unique<T>(std::forward<Args>(server_args)...); FlightServerOptions server_options(location); RETURN_NOT_OK(make_server_options(&server_options)); RETURN_NOT_OK((*server)->Init(server_options)); Location real_location; - RETURN_NOT_OK(Location::ForGrpcTcp("localhost", (*server)->port(), &real_location)); + if ((*server)->port() > 0) { Review comment: I'll have to double check if this is still necessary, this is from my initial exploration when I didn't use UCX's listener feature (and hence there was no concept of a 'port') ########## File path: cpp/src/arrow/flight/client.cc ########## @@ -477,15 +334,9 @@ class GrpcIpcMessageReader : public ipc::MessageReader { } private: - // The RPC context lifetime must be coupled to the ClientReader - std::shared_ptr<ClientRpc> rpc_; + std::shared_ptr<internal::ClientDataStream> stream_; + std::shared_ptr<internal::PeekableFlightDataReader> peekable_reader_; std::shared_ptr<MemoryManager> memory_manager_; - // Guard reads with a mutex to prevent concurrent reads if the write - // side calls Finish(). Nullable as DoGet doesn't need this. - std::shared_ptr<std::mutex> read_mutex_; Review comment: The lock was pushed down into the WritableDataStream implementation below to limit how many classes have to manage it -- 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]
