-
Notifications
You must be signed in to change notification settings - Fork 3.4k
/
client.cc
462 lines (386 loc) · 15.7 KB
/
client.cc
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
// 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/client.h"
// Platform-specific defines
#include "arrow/flight/platform.h"
#include <memory>
#include <sstream>
#include <string>
#include <utility>
#ifdef GRPCPP_PP_INCLUDE
#include <grpcpp/grpcpp.h>
#else
#include <grpc++/grpc++.h>
#endif
#include "arrow/buffer.h"
#include "arrow/ipc/reader.h"
#include "arrow/ipc/writer.h"
#include "arrow/memory_pool.h"
#include "arrow/record_batch.h"
#include "arrow/status.h"
#include "arrow/type.h"
#include "arrow/util/logging.h"
#include "arrow/util/uri.h"
#include "arrow/flight/client_auth.h"
#include "arrow/flight/internal.h"
#include "arrow/flight/serialization-internal.h"
#include "arrow/flight/types.h"
namespace pb = arrow::flight::protocol;
namespace arrow {
class MemoryPool;
namespace flight {
FlightCallOptions::FlightCallOptions() : timeout(-1) {}
struct ClientRpc {
grpc::ClientContext context;
explicit ClientRpc(const FlightCallOptions& options) {
/// XXX workaround until we have a handshake in Connect
context.set_wait_for_ready(true);
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);
}
}
Status IOError(const std::string& error_message) {
std::stringstream ss;
ss << error_message << context.debug_error_string();
return Status::IOError(ss.str());
}
/// \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::AUTH_HEADER, token);
}
return Status::OK();
}
};
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.release_payload());
return Status::OK();
}
return internal::FromGrpcStatus(stream_->Finish());
}
private:
std::shared_ptr<grpc::ClientReaderWriter<pb::HandshakeRequest, pb::HandshakeResponse>>
stream_;
};
class FlightIpcMessageReader : public ipc::MessageReader {
public:
FlightIpcMessageReader(std::unique_ptr<ClientRpc> rpc,
std::unique_ptr<grpc::ClientReader<pb::FlightData>> stream)
: rpc_(std::move(rpc)), stream_(std::move(stream)), stream_finished_(false) {}
Status ReadNextMessage(std::unique_ptr<ipc::Message>* out) override {
if (stream_finished_) {
*out = nullptr;
return Status::OK();
}
internal::FlightData data;
if (!internal::ReadPayload(stream_.get(), &data)) {
// Stream is completed
stream_finished_ = true;
*out = nullptr;
return OverrideWithServerError(Status::OK());
}
// Validate IPC message
auto st = data.OpenMessage(out);
if (!st.ok()) {
return OverrideWithServerError(std::move(st));
}
return Status::OK();
}
protected:
Status OverrideWithServerError(Status&& st) {
// Get the gRPC status if not OK, to propagate any server error message
RETURN_NOT_OK(internal::FromGrpcStatus(stream_->Finish()));
return std::move(st);
}
// The RPC context lifetime must be coupled to the ClientReader
std::unique_ptr<ClientRpc> rpc_;
std::unique_ptr<grpc::ClientReader<pb::FlightData>> stream_;
bool stream_finished_;
};
/// A IpcPayloadWriter implementation that writes to a DoPut stream
class DoPutPayloadWriter : public ipc::internal::IpcPayloadWriter {
public:
DoPutPayloadWriter(const FlightDescriptor& descriptor, std::unique_ptr<ClientRpc> rpc,
std::unique_ptr<protocol::PutResult> response,
std::unique_ptr<grpc::ClientWriter<pb::FlightData>> writer)
: descriptor_(descriptor),
rpc_(std::move(rpc)),
response_(std::move(response)),
writer_(std::move(writer)),
first_payload_(true) {}
~DoPutPayloadWriter() override = default;
Status Start() override { return Status::OK(); }
Status WritePayload(const ipc::internal::IpcPayload& ipc_payload) override {
FlightPayload payload;
payload.ipc_message = ipc_payload;
if (first_payload_) {
// First Flight message needs to encore the Flight descriptor
DCHECK_EQ(ipc_payload.type, ipc::Message::SCHEMA);
std::string str_descr;
{
pb::FlightDescriptor pb_descr;
RETURN_NOT_OK(internal::ToProto(descriptor_, &pb_descr));
if (!pb_descr.SerializeToString(&str_descr)) {
return Status::UnknownError("Failed to serialized Flight descriptor");
}
}
RETURN_NOT_OK(Buffer::FromString(str_descr, &payload.descriptor));
first_payload_ = false;
}
if (!internal::WritePayload(payload, writer_.get())) {
return rpc_->IOError("Could not write record batch to stream: ");
}
return Status::OK();
}
Status Close() override {
bool finished_writes = writer_->WritesDone();
RETURN_NOT_OK(internal::FromGrpcStatus(writer_->Finish()));
if (!finished_writes) {
return Status::UnknownError(
"Could not finish writing record batches before closing");
}
return Status::OK();
}
protected:
// TODO: there isn't a way to access this as a user.
const FlightDescriptor descriptor_;
std::unique_ptr<ClientRpc> rpc_;
std::unique_ptr<protocol::PutResult> response_;
std::unique_ptr<grpc::ClientWriter<pb::FlightData>> writer_;
bool first_payload_;
};
class FlightClient::FlightClientImpl {
public:
Status Connect(const Location& location, const FlightClientOptions& options) {
const std::string& scheme = location.scheme();
std::stringstream grpc_uri;
std::shared_ptr<grpc::ChannelCredentials> creds;
if (scheme == kSchemeGrpc || scheme == kSchemeGrpcTcp || scheme == kSchemeGrpcTls) {
grpc_uri << location.uri_->host() << ":" << location.uri_->port_text();
if (scheme == "grpc+tls") {
grpc::SslCredentialsOptions ssl_options;
if (!options.tls_root_certs.empty()) {
ssl_options.pem_root_certs = options.tls_root_certs;
}
creds = grpc::SslCredentials(ssl_options);
} else {
creds = grpc::InsecureChannelCredentials();
}
} else if (scheme == kSchemeGrpcUnix) {
grpc_uri << "unix://" << location.uri_->path();
creds = grpc::InsecureChannelCredentials();
} else {
return Status::NotImplemented("Flight scheme " + scheme + " is not supported.");
}
grpc::ChannelArguments args;
// Try to reconnect quickly at first, in case the server is still starting up
args.SetInt(GRPC_ARG_INITIAL_RECONNECT_BACKOFF_MS, 100);
// Receive messages of any size
args.SetMaxReceiveMessageSize(-1);
stub_ = pb::FlightService::NewStub(
grpc::CreateCustomChannel(grpc_uri.str(), creds, args));
return Status::OK();
}
Status Authenticate(const FlightCallOptions& options,
std::unique_ptr<ClientAuthHandler> auth_handler) {
auth_handler_ = std::move(auth_handler);
ClientRpc rpc(options);
std::shared_ptr<grpc::ClientReaderWriter<pb::HandshakeRequest, pb::HandshakeResponse>>
stream = stub_->Handshake(&rpc.context);
GrpcClientAuthSender outgoing{stream};
GrpcClientAuthReader incoming{stream};
RETURN_NOT_OK(auth_handler_->Authenticate(&outgoing, &incoming));
RETURN_NOT_OK(internal::FromGrpcStatus(stream->Finish()));
return Status::OK();
}
Status ListFlights(const FlightCallOptions& options, const Criteria& criteria,
std::unique_ptr<FlightListing>* listing) {
// TODO(wesm): populate criteria
pb::Criteria pb_criteria;
ClientRpc rpc(options);
RETURN_NOT_OK(rpc.SetToken(auth_handler_.get()));
std::unique_ptr<grpc::ClientReader<pb::FlightInfo>> stream(
stub_->ListFlights(&rpc.context, pb_criteria));
std::vector<FlightInfo> flights;
pb::FlightInfo pb_info;
while (stream->Read(&pb_info)) {
FlightInfo::Data info_data;
RETURN_NOT_OK(internal::FromProto(pb_info, &info_data));
flights.emplace_back(std::move(info_data));
}
listing->reset(new SimpleFlightListing(std::move(flights)));
return internal::FromGrpcStatus(stream->Finish());
}
Status DoAction(const FlightCallOptions& options, const Action& action,
std::unique_ptr<ResultStream>* results) {
pb::Action pb_action;
RETURN_NOT_OK(internal::ToProto(action, &pb_action));
ClientRpc rpc(options);
RETURN_NOT_OK(rpc.SetToken(auth_handler_.get()));
std::unique_ptr<grpc::ClientReader<pb::Result>> stream(
stub_->DoAction(&rpc.context, pb_action));
pb::Result pb_result;
std::vector<Result> materialized_results;
while (stream->Read(&pb_result)) {
Result result;
RETURN_NOT_OK(internal::FromProto(pb_result, &result));
materialized_results.emplace_back(std::move(result));
}
*results = std::unique_ptr<ResultStream>(
new SimpleResultStream(std::move(materialized_results)));
return internal::FromGrpcStatus(stream->Finish());
}
Status ListActions(const FlightCallOptions& options, std::vector<ActionType>* types) {
pb::Empty empty;
ClientRpc rpc(options);
RETURN_NOT_OK(rpc.SetToken(auth_handler_.get()));
std::unique_ptr<grpc::ClientReader<pb::ActionType>> stream(
stub_->ListActions(&rpc.context, empty));
pb::ActionType pb_type;
ActionType type;
while (stream->Read(&pb_type)) {
RETURN_NOT_OK(internal::FromProto(pb_type, &type));
types->emplace_back(std::move(type));
}
return internal::FromGrpcStatus(stream->Finish());
}
Status GetFlightInfo(const FlightCallOptions& options,
const FlightDescriptor& descriptor,
std::unique_ptr<FlightInfo>* info) {
pb::FlightDescriptor pb_descriptor;
pb::FlightInfo pb_response;
RETURN_NOT_OK(internal::ToProto(descriptor, &pb_descriptor));
ClientRpc rpc(options);
RETURN_NOT_OK(rpc.SetToken(auth_handler_.get()));
Status s = internal::FromGrpcStatus(
stub_->GetFlightInfo(&rpc.context, pb_descriptor, &pb_response));
RETURN_NOT_OK(s);
FlightInfo::Data info_data;
RETURN_NOT_OK(internal::FromProto(pb_response, &info_data));
info->reset(new FlightInfo(std::move(info_data)));
return Status::OK();
}
Status DoGet(const FlightCallOptions& options, const Ticket& ticket,
std::unique_ptr<RecordBatchReader>* out) {
pb::Ticket pb_ticket;
internal::ToProto(ticket, &pb_ticket);
std::unique_ptr<ClientRpc> rpc(new ClientRpc(options));
RETURN_NOT_OK(rpc->SetToken(auth_handler_.get()));
std::unique_ptr<grpc::ClientReader<pb::FlightData>> stream(
stub_->DoGet(&rpc->context, pb_ticket));
std::unique_ptr<ipc::MessageReader> message_reader(
new FlightIpcMessageReader(std::move(rpc), std::move(stream)));
return ipc::RecordBatchStreamReader::Open(std::move(message_reader), out);
}
Status DoPut(const FlightCallOptions& options, const FlightDescriptor& descriptor,
const std::shared_ptr<Schema>& schema,
std::unique_ptr<ipc::RecordBatchWriter>* out) {
std::unique_ptr<ClientRpc> rpc(new ClientRpc(options));
RETURN_NOT_OK(rpc->SetToken(auth_handler_.get()));
std::unique_ptr<protocol::PutResult> response(new protocol::PutResult);
std::unique_ptr<grpc::ClientWriter<pb::FlightData>> writer(
stub_->DoPut(&rpc->context, response.get()));
std::unique_ptr<ipc::internal::IpcPayloadWriter> payload_writer(
new DoPutPayloadWriter(descriptor, std::move(rpc), std::move(response),
std::move(writer)));
return ipc::internal::OpenRecordBatchWriter(std::move(payload_writer), schema, out);
}
private:
std::unique_ptr<pb::FlightService::Stub> stub_;
std::shared_ptr<ClientAuthHandler> auth_handler_;
};
FlightClient::FlightClient() { impl_.reset(new FlightClientImpl); }
FlightClient::~FlightClient() {}
Status FlightClient::Connect(const Location& location,
std::unique_ptr<FlightClient>* client) {
return Connect(location, {}, client);
}
Status FlightClient::Connect(const Location& location, const FlightClientOptions& options,
std::unique_ptr<FlightClient>* client) {
client->reset(new FlightClient);
return (*client)->impl_->Connect(location, options);
}
Status FlightClient::Authenticate(const FlightCallOptions& options,
std::unique_ptr<ClientAuthHandler> auth_handler) {
return impl_->Authenticate(options, std::move(auth_handler));
}
Status FlightClient::DoAction(const FlightCallOptions& options, const Action& action,
std::unique_ptr<ResultStream>* results) {
return impl_->DoAction(options, action, results);
}
Status FlightClient::ListActions(const FlightCallOptions& options,
std::vector<ActionType>* actions) {
return impl_->ListActions(options, actions);
}
Status FlightClient::GetFlightInfo(const FlightCallOptions& options,
const FlightDescriptor& descriptor,
std::unique_ptr<FlightInfo>* info) {
return impl_->GetFlightInfo(options, descriptor, info);
}
Status FlightClient::ListFlights(std::unique_ptr<FlightListing>* listing) {
return ListFlights({}, {}, listing);
}
Status FlightClient::ListFlights(const FlightCallOptions& options,
const Criteria& criteria,
std::unique_ptr<FlightListing>* listing) {
return impl_->ListFlights(options, criteria, listing);
}
Status FlightClient::DoGet(const FlightCallOptions& options, const Ticket& ticket,
std::unique_ptr<RecordBatchReader>* stream) {
return impl_->DoGet(options, ticket, stream);
}
Status FlightClient::DoPut(const FlightCallOptions& options,
const FlightDescriptor& descriptor,
const std::shared_ptr<Schema>& schema,
std::unique_ptr<ipc::RecordBatchWriter>* stream) {
return impl_->DoPut(options, descriptor, schema, stream);
}
} // namespace flight
} // namespace arrow