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



##########
File path: cpp/src/arrow/flight/transport/grpc/grpc_client.cc
##########
@@ -0,0 +1,872 @@
+// 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/grpc/grpc_client.h"
+
+#include <map>
+#include <memory>
+#include <mutex>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include "arrow/util/config.h"
+#ifdef GRPCPP_PP_INCLUDE
+#include <grpcpp/grpcpp.h>
+#if defined(GRPC_NAMESPACE_FOR_TLS_CREDENTIALS_OPTIONS)
+#include <grpcpp/security/tls_credentials_options.h>
+#endif
+#else
+#include <grpc++/grpc++.h>
+#endif
+
+#include <grpc/grpc_security_constants.h>
+
+#include "arrow/buffer.h"
+#include "arrow/device.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/uri.h"
+
+#include "arrow/flight/client.h"
+#include "arrow/flight/client_auth.h"
+#include "arrow/flight/client_header_internal.h"
+#include "arrow/flight/client_middleware.h"
+#include "arrow/flight/internal.h"
+#include "arrow/flight/middleware.h"
+#include "arrow/flight/serialization_internal.h"
+#include "arrow/flight/transport.h"
+#include "arrow/flight/types.h"
+
+namespace arrow {
+
+namespace flight {
+namespace transport {
+namespace grpc {
+
+namespace {
+namespace pb = arrow::flight::protocol;
+
+struct ClientRpc {
+  ::grpc::ClientContext context;
+
+  explicit ClientRpc(const FlightCallOptions& options) {
+    if (options.timeout.count() >= 0) {
+      std::chrono::system_clock::time_point deadline =
+          
std::chrono::time_point_cast<std::chrono::system_clock::time_point::duration>(
+              std::chrono::system_clock::now() + options.timeout);
+      context.set_deadline(deadline);
+    }
+    for (auto header : options.headers) {
+      context.AddMetadata(header.first, header.second);
+    }
+  }
+
+  /// \brief Add an auth token via an auth handler
+  Status SetToken(ClientAuthHandler* auth_handler) {
+    if (auth_handler) {
+      std::string token;
+      RETURN_NOT_OK(auth_handler->GetToken(&token));
+      context.AddMetadata(internal::kGrpcAuthHeader, token);
+    }
+    return Status::OK();
+  }
+};
+
+class GrpcAddClientHeaders : public AddCallHeaders {
+ public:
+  explicit GrpcAddClientHeaders(std::multimap<::grpc::string, ::grpc::string>* 
metadata)
+      : metadata_(metadata) {}
+  ~GrpcAddClientHeaders() override = default;
+
+  void AddHeader(const std::string& key, const std::string& value) override {
+    metadata_->insert(std::make_pair(key, value));
+  }
+
+ private:
+  std::multimap<::grpc::string, ::grpc::string>* metadata_;
+};
+
+class GrpcClientInterceptorAdapter : public ::grpc::experimental::Interceptor {
+ public:
+  explicit GrpcClientInterceptorAdapter(
+      std::vector<std::unique_ptr<ClientMiddleware>> middleware)
+      : middleware_(std::move(middleware)), received_headers_(false) {}
+
+  void Intercept(::grpc::experimental::InterceptorBatchMethods* methods) {
+    using InterceptionHookPoints = 
::grpc::experimental::InterceptionHookPoints;
+    if (methods->QueryInterceptionHookPoint(
+            InterceptionHookPoints::PRE_SEND_INITIAL_METADATA)) {
+      GrpcAddClientHeaders add_headers(methods->GetSendInitialMetadata());
+      for (const auto& middleware : middleware_) {
+        middleware->SendingHeaders(&add_headers);
+      }
+    }
+
+    if (methods->QueryInterceptionHookPoint(
+            InterceptionHookPoints::POST_RECV_INITIAL_METADATA)) {
+      if (!methods->GetRecvInitialMetadata()->empty()) {
+        ReceivedHeaders(*methods->GetRecvInitialMetadata());
+      }
+    }
+
+    if 
(methods->QueryInterceptionHookPoint(InterceptionHookPoints::POST_RECV_STATUS)) 
{
+      DCHECK_NE(nullptr, methods->GetRecvStatus());
+      DCHECK_NE(nullptr, methods->GetRecvTrailingMetadata());
+      ReceivedHeaders(*methods->GetRecvTrailingMetadata());
+      const Status status = 
internal::FromGrpcStatus(*methods->GetRecvStatus());
+      for (const auto& middleware : middleware_) {
+        middleware->CallCompleted(status);
+      }
+    }
+
+    methods->Proceed();
+  }
+
+ private:
+  void ReceivedHeaders(
+      const std::multimap<::grpc::string_ref, ::grpc::string_ref>& metadata) {
+    if (received_headers_) {
+      return;
+    }
+    received_headers_ = true;
+    CallHeaders headers;
+    for (const auto& entry : metadata) {
+      headers.insert({util::string_view(entry.first.data(), 
entry.first.length()),
+                      util::string_view(entry.second.data(), 
entry.second.length())});
+    }
+    for (const auto& middleware : middleware_) {
+      middleware->ReceivedHeaders(headers);
+    }
+  }
+
+  std::vector<std::unique_ptr<ClientMiddleware>> middleware_;
+  // When communicating with a gRPC-Java server, the server may not
+  // send back headers if the call fails right away. Instead, the
+  // headers will be consolidated into the trailers. We don't want to
+  // call the client middleware callback twice, so instead track
+  // whether we saw headers - if not, then we need to check trailers.
+  bool received_headers_;
+};
+
+class GrpcClientInterceptorAdapterFactory
+    : public ::grpc::experimental::ClientInterceptorFactoryInterface {
+ public:
+  GrpcClientInterceptorAdapterFactory(
+      std::vector<std::shared_ptr<ClientMiddlewareFactory>> middleware)
+      : middleware_(middleware) {}
+
+  ::grpc::experimental::Interceptor* CreateClientInterceptor(
+      ::grpc::experimental::ClientRpcInfo* info) override {
+    std::vector<std::unique_ptr<ClientMiddleware>> middleware;
+
+    FlightMethod flight_method = FlightMethod::Invalid;
+    util::string_view method(info->method());
+    if (method.ends_with("/Handshake")) {
+      flight_method = FlightMethod::Handshake;
+    } else if (method.ends_with("/ListFlights")) {
+      flight_method = FlightMethod::ListFlights;
+    } else if (method.ends_with("/GetFlightInfo")) {
+      flight_method = FlightMethod::GetFlightInfo;
+    } else if (method.ends_with("/GetSchema")) {
+      flight_method = FlightMethod::GetSchema;
+    } else if (method.ends_with("/DoGet")) {
+      flight_method = FlightMethod::DoGet;
+    } else if (method.ends_with("/DoPut")) {
+      flight_method = FlightMethod::DoPut;
+    } else if (method.ends_with("/DoExchange")) {
+      flight_method = FlightMethod::DoExchange;
+    } else if (method.ends_with("/DoAction")) {
+      flight_method = FlightMethod::DoAction;
+    } else if (method.ends_with("/ListActions")) {
+      flight_method = FlightMethod::ListActions;
+    } else {
+      DCHECK(false) << "Unknown Flight method: " << info->method();
+    }
+
+    const CallInfo flight_info{flight_method};
+    for (auto& factory : middleware_) {
+      std::unique_ptr<ClientMiddleware> instance;
+      factory->StartCall(flight_info, &instance);
+      if (instance) {
+        middleware.push_back(std::move(instance));
+      }
+    }
+    return new GrpcClientInterceptorAdapter(std::move(middleware));
+  }
+
+ private:
+  std::vector<std::shared_ptr<ClientMiddlewareFactory>> middleware_;
+};
+
+class GrpcClientAuthSender : public ClientAuthSender {
+ public:
+  explicit GrpcClientAuthSender(
+      std::shared_ptr<
+          ::grpc::ClientReaderWriter<pb::HandshakeRequest, 
pb::HandshakeResponse>>
+          stream)
+      : stream_(stream) {}
+
+  Status Write(const std::string& token) override {
+    pb::HandshakeRequest response;
+    response.set_payload(token);
+    if (stream_->Write(response)) {
+      return Status::OK();
+    }
+    return internal::FromGrpcStatus(stream_->Finish());
+  }
+
+ private:
+  std::shared_ptr<::grpc::ClientReaderWriter<pb::HandshakeRequest, 
pb::HandshakeResponse>>
+      stream_;
+};
+
+class GrpcClientAuthReader : public ClientAuthReader {
+ public:
+  explicit GrpcClientAuthReader(
+      std::shared_ptr<
+          ::grpc::ClientReaderWriter<pb::HandshakeRequest, 
pb::HandshakeResponse>>
+          stream)
+      : stream_(stream) {}
+
+  Status Read(std::string* token) override {
+    pb::HandshakeResponse request;
+    if (stream_->Read(&request)) {
+      *token = std::move(*request.mutable_payload());
+      return Status::OK();
+    }
+    return internal::FromGrpcStatus(stream_->Finish());
+  }
+
+ private:
+  std::shared_ptr<::grpc::ClientReaderWriter<pb::HandshakeRequest, 
pb::HandshakeResponse>>
+      stream_;
+};
+
+/// \brief The base of the ClientDataStream implementation for gRPC.
+template <typename Stream, typename ReadPayload>
+class FinishableDataStream : public internal::ClientDataStream {
+ public:
+  FinishableDataStream(std::shared_ptr<ClientRpc> rpc, std::shared_ptr<Stream> 
stream,
+                       std::shared_ptr<MemoryManager> memory_manager)
+      : rpc_(std::move(rpc)),
+        stream_(std::move(stream)),
+        memory_manager_(memory_manager ? std::move(memory_manager)
+                                       : 
CPUDevice::Instance()->default_memory_manager()),
+        finished_(false) {}
+
+  void TryCancel() override { rpc_->context.TryCancel(); }
+
+ protected:
+  Status Finish() override {
+    if (finished_) {
+      return server_status_;
+    }
+
+    // Drain the read side, as otherwise gRPC Finish() will hang. We
+    // only call Finish() when the client closes the writer or the
+    // reader finishes, so it's OK to assume the client no longer
+    // wants to read and drain the read side. (If the client wants to
+    // indicate that it is done writing, but not done reading, it
+    // should use DoneWriting.
+    ReadPayload message;
+    while (internal::ReadPayload(stream_.get(), &message)) {
+      // Drain the read side to avoid gRPC hanging in Finish()
+    }
+
+    server_status_ = internal::FromGrpcStatus(stream_->Finish(), 
&rpc_->context);
+    if (!server_status_.ok()) {
+      server_status_ = Status::FromDetailAndArgs(
+          server_status_.code(), server_status_.detail(), 
server_status_.message(),
+          ". gRPC client debug context: ", rpc_->context.debug_error_string());
+    }
+    if (!transport_status_.ok()) {
+      if (server_status_.ok()) {
+        server_status_ = transport_status_;
+      } else {
+        server_status_ = Status::FromDetailAndArgs(
+            server_status_.code(), server_status_.detail(), 
server_status_.message(),
+            ". gRPC client debug context: ", 
rpc_->context.debug_error_string(),
+            ". Additional context: ", transport_status_.ToString());
+      }
+    }
+    finished_ = true;
+
+    return server_status_;
+  }
+
+  std::shared_ptr<ClientRpc> rpc_;
+  std::shared_ptr<Stream> stream_;
+  std::shared_ptr<MemoryManager> memory_manager_;
+  bool finished_;
+  Status server_status_;
+  // A transport-side error that needs to get combined with the server status
+  Status transport_status_;
+};
+
+/// \brief A ClientDataStream implementation for gRPC that manages a
+///   mutex to protect from concurrent reads/writes, and drains the
+///   read side on finish.
+template <typename Stream, typename ReadPayload>
+class WritableDataStream : public FinishableDataStream<Stream, ReadPayload> {
+ public:
+  using Base = FinishableDataStream<Stream, ReadPayload>;
+  WritableDataStream(std::shared_ptr<ClientRpc> rpc, std::shared_ptr<Stream> 
stream,
+                     std::shared_ptr<MemoryManager> memory_manager)
+      : Base(std::move(rpc), std::move(stream), std::move(memory_manager)),
+        read_mutex_(),
+        finish_mutex_(),
+        done_writing_(false) {}
+
+  Status WritesDone() override {
+    // This is only used by the writer side of a stream, so it need
+    // not be protected with a lock.
+    if (done_writing_) {
+      return Status::OK();
+    }
+    done_writing_ = true;
+    if (!stream_->WritesDone()) {
+      // Error happened, try to close the stream to get more detailed info
+      return internal::ClientDataStream::Finish(MakeFlightError(

Review comment:
       If I don't qualify it, it complains it can't find the overload. (Hmm, 
probably I should just rename one of the methods to avoid this confusion.)




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

To unsubscribe, e-mail: [email protected]

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


Reply via email to