-
Notifications
You must be signed in to change notification settings - Fork 793
/
client.rs
778 lines (712 loc) · 28.3 KB
/
client.rs
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
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
// 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.
//! A FlightSQL Client [`FlightSqlServiceClient`]
use base64::prelude::BASE64_STANDARD;
use base64::Engine;
use bytes::Bytes;
use std::collections::HashMap;
use std::str::FromStr;
use tonic::metadata::AsciiMetadataKey;
use crate::decode::FlightRecordBatchStream;
use crate::encode::FlightDataEncoderBuilder;
use crate::error::FlightError;
use crate::flight_service_client::FlightServiceClient;
use crate::sql::gen::action_end_transaction_request::EndTransaction;
use crate::sql::server::{
BEGIN_TRANSACTION, CLOSE_PREPARED_STATEMENT, CREATE_PREPARED_STATEMENT, END_TRANSACTION,
};
use crate::sql::{
ActionBeginTransactionRequest, ActionBeginTransactionResult,
ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
ActionCreatePreparedStatementResult, ActionEndTransactionRequest, Any, CommandGetCatalogs,
CommandGetCrossReference, CommandGetDbSchemas, CommandGetExportedKeys, CommandGetImportedKeys,
CommandGetPrimaryKeys, CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
CommandGetXdbcTypeInfo, CommandPreparedStatementQuery, CommandPreparedStatementUpdate,
CommandStatementIngest, CommandStatementQuery, CommandStatementUpdate,
DoPutPreparedStatementResult, DoPutUpdateResult, ProstMessageExt, SqlInfo,
};
use crate::streams::FallibleRequestStream;
use crate::trailers::extract_lazy_trailers;
use crate::{
Action, FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse,
IpcMessage, PutResult, Ticket,
};
use arrow_array::RecordBatch;
use arrow_buffer::Buffer;
use arrow_ipc::convert::fb_to_schema;
use arrow_ipc::reader::read_record_batch;
use arrow_ipc::{root_as_message, MessageHeader};
use arrow_schema::{ArrowError, Schema, SchemaRef};
use futures::{stream, Stream, TryStreamExt};
use prost::Message;
use tonic::transport::Channel;
use tonic::{IntoRequest, IntoStreamingRequest, Streaming};
/// A FlightSQLServiceClient is an endpoint for retrieving or storing Arrow data
/// by FlightSQL protocol.
#[derive(Debug, Clone)]
pub struct FlightSqlServiceClient<T> {
token: Option<String>,
headers: HashMap<String, String>,
flight_client: FlightServiceClient<T>,
}
/// A FlightSql protocol client that can run queries against FlightSql servers
/// This client is in the "experimental" stage. It is not guaranteed to follow the spec in all instances.
/// Github issues are welcomed.
impl FlightSqlServiceClient<Channel> {
/// Creates a new FlightSql client that connects to a server over an arbitrary tonic `Channel`
pub fn new(channel: Channel) -> Self {
Self::new_from_inner(FlightServiceClient::new(channel))
}
/// Creates a new higher level client with the provided lower level client
pub fn new_from_inner(inner: FlightServiceClient<Channel>) -> Self {
Self {
token: None,
flight_client: inner,
headers: HashMap::default(),
}
}
/// Return a reference to the underlying [`FlightServiceClient`]
pub fn inner(&self) -> &FlightServiceClient<Channel> {
&self.flight_client
}
/// Return a mutable reference to the underlying [`FlightServiceClient`]
pub fn inner_mut(&mut self) -> &mut FlightServiceClient<Channel> {
&mut self.flight_client
}
/// Consume this client and return the underlying [`FlightServiceClient`]
pub fn into_inner(self) -> FlightServiceClient<Channel> {
self.flight_client
}
/// Set auth token to the given value.
pub fn set_token(&mut self, token: String) {
self.token = Some(token);
}
/// Clear the auth token.
pub fn clear_token(&mut self) {
self.token = None;
}
/// Share the bearer token with potentially different `DoGet` clients
pub fn token(&self) -> Option<&String> {
self.token.as_ref()
}
/// Set header value.
pub fn set_header(&mut self, key: impl Into<String>, value: impl Into<String>) {
let key: String = key.into();
let value: String = value.into();
self.headers.insert(key, value);
}
async fn get_flight_info_for_command<M: ProstMessageExt>(
&mut self,
cmd: M,
) -> Result<FlightInfo, ArrowError> {
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec());
let req = self.set_request_headers(descriptor.into_request())?;
let fi = self
.flight_client
.get_flight_info(req)
.await
.map_err(status_to_arrow_error)?
.into_inner();
Ok(fi)
}
/// Execute a query on the server.
pub async fn execute(
&mut self,
query: String,
transaction_id: Option<Bytes>,
) -> Result<FlightInfo, ArrowError> {
let cmd = CommandStatementQuery {
query,
transaction_id,
};
self.get_flight_info_for_command(cmd).await
}
/// Perform a `handshake` with the server, passing credentials and establishing a session.
///
/// If the server returns an "authorization" header, it is automatically parsed and set as
/// a token for future requests. Any other data returned by the server in the handshake
/// response is returned as a binary blob.
pub async fn handshake(&mut self, username: &str, password: &str) -> Result<Bytes, ArrowError> {
let cmd = HandshakeRequest {
protocol_version: 0,
payload: Default::default(),
};
let mut req = tonic::Request::new(stream::iter(vec![cmd]));
let val = BASE64_STANDARD.encode(format!("{username}:{password}"));
let val = format!("Basic {val}")
.parse()
.map_err(|_| ArrowError::ParseError("Cannot parse header".to_string()))?;
req.metadata_mut().insert("authorization", val);
let req = self.set_request_headers(req)?;
let resp = self
.flight_client
.handshake(req)
.await
.map_err(|e| ArrowError::IpcError(format!("Can't handshake {e}")))?;
if let Some(auth) = resp.metadata().get("authorization") {
let auth = auth
.to_str()
.map_err(|_| ArrowError::ParseError("Can't read auth header".to_string()))?;
let bearer = "Bearer ";
if !auth.starts_with(bearer) {
Err(ArrowError::ParseError("Invalid auth header!".to_string()))?;
}
let auth = auth[bearer.len()..].to_string();
self.token = Some(auth);
}
let responses: Vec<HandshakeResponse> = resp
.into_inner()
.try_collect()
.await
.map_err(|_| ArrowError::ParseError("Can't collect responses".to_string()))?;
let resp = match responses.as_slice() {
[resp] => resp.payload.clone(),
[] => Bytes::new(),
_ => Err(ArrowError::ParseError(
"Multiple handshake responses".to_string(),
))?,
};
Ok(resp)
}
/// Execute a update query on the server, and return the number of records affected
pub async fn execute_update(
&mut self,
query: String,
transaction_id: Option<Bytes>,
) -> Result<i64, ArrowError> {
let cmd = CommandStatementUpdate {
query,
transaction_id,
};
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec());
let req = self.set_request_headers(
stream::iter(vec![FlightData {
flight_descriptor: Some(descriptor),
..Default::default()
}])
.into_request(),
)?;
let mut result = self
.flight_client
.do_put(req)
.await
.map_err(status_to_arrow_error)?
.into_inner();
let result = result
.message()
.await
.map_err(status_to_arrow_error)?
.unwrap();
let any = Any::decode(&*result.app_metadata).map_err(decode_error_to_arrow_error)?;
let result: DoPutUpdateResult = any.unpack()?.unwrap();
Ok(result.record_count)
}
/// Execute a bulk ingest on the server and return the number of records added
pub async fn execute_ingest<S>(
&mut self,
command: CommandStatementIngest,
stream: S,
) -> Result<i64, ArrowError>
where
S: Stream<Item = crate::error::Result<RecordBatch>> + Send + 'static,
{
let (sender, receiver) = futures::channel::oneshot::channel();
let descriptor = FlightDescriptor::new_cmd(command.as_any().encode_to_vec());
let flight_data = FlightDataEncoderBuilder::new()
.with_flight_descriptor(Some(descriptor))
.build(stream);
// Intercept client errors and send them to the one shot channel above
let flight_data = Box::pin(flight_data);
let flight_data: FallibleRequestStream<FlightData, FlightError> =
FallibleRequestStream::new(sender, flight_data);
let req = self.set_request_headers(flight_data.into_streaming_request())?;
let mut result = self
.flight_client
.do_put(req)
.await
.map_err(status_to_arrow_error)?
.into_inner();
// check if the there were any errors in the input stream provided note
// if receiver.await fails, it means the sender was dropped and there is
// no message to return.
if let Ok(msg) = receiver.await {
return Err(ArrowError::ExternalError(Box::new(msg)));
}
let result = result
.message()
.await
.map_err(status_to_arrow_error)?
.unwrap();
let any = Any::decode(&*result.app_metadata).map_err(decode_error_to_arrow_error)?;
let result: DoPutUpdateResult = any.unpack()?.unwrap();
Ok(result.record_count)
}
/// Request a list of catalogs as tabular FlightInfo results
pub async fn get_catalogs(&mut self) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(CommandGetCatalogs {})
.await
}
/// Request a list of database schemas as tabular FlightInfo results
pub async fn get_db_schemas(
&mut self,
request: CommandGetDbSchemas,
) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(request).await
}
/// Given a flight ticket, request to be sent the stream. Returns record batch stream reader
pub async fn do_get(
&mut self,
ticket: impl IntoRequest<Ticket>,
) -> Result<FlightRecordBatchStream, ArrowError> {
let req = self.set_request_headers(ticket.into_request())?;
let (md, response_stream, _ext) = self
.flight_client
.do_get(req)
.await
.map_err(status_to_arrow_error)?
.into_parts();
let (response_stream, trailers) = extract_lazy_trailers(response_stream);
Ok(FlightRecordBatchStream::new_from_flight_data(
response_stream.map_err(FlightError::Tonic),
)
.with_headers(md)
.with_trailers(trailers))
}
/// Push a stream to the flight service associated with a particular flight stream.
pub async fn do_put(
&mut self,
request: impl tonic::IntoStreamingRequest<Message = FlightData>,
) -> Result<Streaming<PutResult>, ArrowError> {
let req = self.set_request_headers(request.into_streaming_request())?;
Ok(self
.flight_client
.do_put(req)
.await
.map_err(status_to_arrow_error)?
.into_inner())
}
/// DoAction allows a flight client to do a specific action against a flight service
pub async fn do_action(
&mut self,
request: impl IntoRequest<Action>,
) -> Result<Streaming<crate::Result>, ArrowError> {
let req = self.set_request_headers(request.into_request())?;
Ok(self
.flight_client
.do_action(req)
.await
.map_err(status_to_arrow_error)?
.into_inner())
}
/// Request a list of tables.
pub async fn get_tables(
&mut self,
request: CommandGetTables,
) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(request).await
}
/// Request the primary keys for a table.
pub async fn get_primary_keys(
&mut self,
request: CommandGetPrimaryKeys,
) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(request).await
}
/// Retrieves a description about the foreign key columns that reference the
/// primary key columns of the given table.
pub async fn get_exported_keys(
&mut self,
request: CommandGetExportedKeys,
) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(request).await
}
/// Retrieves the foreign key columns for the given table.
pub async fn get_imported_keys(
&mut self,
request: CommandGetImportedKeys,
) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(request).await
}
/// Retrieves a description of the foreign key columns in the given foreign key
/// table that reference the primary key or the columns representing a unique
/// constraint of the parent table (could be the same or a different table).
pub async fn get_cross_reference(
&mut self,
request: CommandGetCrossReference,
) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(request).await
}
/// Request a list of table types.
pub async fn get_table_types(&mut self) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(CommandGetTableTypes {})
.await
}
/// Request a list of SQL information.
pub async fn get_sql_info(
&mut self,
sql_infos: Vec<SqlInfo>,
) -> Result<FlightInfo, ArrowError> {
let request = CommandGetSqlInfo {
info: sql_infos.iter().map(|sql_info| *sql_info as u32).collect(),
};
self.get_flight_info_for_command(request).await
}
/// Request XDBC SQL information.
pub async fn get_xdbc_type_info(
&mut self,
request: CommandGetXdbcTypeInfo,
) -> Result<FlightInfo, ArrowError> {
self.get_flight_info_for_command(request).await
}
/// Create a prepared statement object.
pub async fn prepare(
&mut self,
query: String,
transaction_id: Option<Bytes>,
) -> Result<PreparedStatement<Channel>, ArrowError> {
let cmd = ActionCreatePreparedStatementRequest {
query,
transaction_id,
};
let action = Action {
r#type: CREATE_PREPARED_STATEMENT.to_string(),
body: cmd.as_any().encode_to_vec().into(),
};
let req = self.set_request_headers(action.into_request())?;
let mut result = self
.flight_client
.do_action(req)
.await
.map_err(status_to_arrow_error)?
.into_inner();
let result = result
.message()
.await
.map_err(status_to_arrow_error)?
.unwrap();
let any = Any::decode(&*result.body).map_err(decode_error_to_arrow_error)?;
let prepared_result: ActionCreatePreparedStatementResult = any.unpack()?.unwrap();
let dataset_schema = match prepared_result.dataset_schema.len() {
0 => Schema::empty(),
_ => Schema::try_from(IpcMessage(prepared_result.dataset_schema))?,
};
let parameter_schema = match prepared_result.parameter_schema.len() {
0 => Schema::empty(),
_ => Schema::try_from(IpcMessage(prepared_result.parameter_schema))?,
};
Ok(PreparedStatement::new(
self.clone(),
prepared_result.prepared_statement_handle,
dataset_schema,
parameter_schema,
))
}
/// Request to begin a transaction.
pub async fn begin_transaction(&mut self) -> Result<Bytes, ArrowError> {
let cmd = ActionBeginTransactionRequest {};
let action = Action {
r#type: BEGIN_TRANSACTION.to_string(),
body: cmd.as_any().encode_to_vec().into(),
};
let req = self.set_request_headers(action.into_request())?;
let mut result = self
.flight_client
.do_action(req)
.await
.map_err(status_to_arrow_error)?
.into_inner();
let result = result
.message()
.await
.map_err(status_to_arrow_error)?
.unwrap();
let any = Any::decode(&*result.body).map_err(decode_error_to_arrow_error)?;
let begin_result: ActionBeginTransactionResult = any.unpack()?.unwrap();
Ok(begin_result.transaction_id)
}
/// Request to commit/rollback a transaction.
pub async fn end_transaction(
&mut self,
transaction_id: Bytes,
action: EndTransaction,
) -> Result<(), ArrowError> {
let cmd = ActionEndTransactionRequest {
transaction_id,
action: action as i32,
};
let action = Action {
r#type: END_TRANSACTION.to_string(),
body: cmd.as_any().encode_to_vec().into(),
};
let req = self.set_request_headers(action.into_request())?;
let _ = self
.flight_client
.do_action(req)
.await
.map_err(status_to_arrow_error)?
.into_inner();
Ok(())
}
/// Explicitly shut down and clean up the client.
pub async fn close(&mut self) -> Result<(), ArrowError> {
// TODO: consume self instead of &mut self to explicitly prevent reuse?
Ok(())
}
fn set_request_headers<T>(
&self,
mut req: tonic::Request<T>,
) -> Result<tonic::Request<T>, ArrowError> {
for (k, v) in &self.headers {
let k = AsciiMetadataKey::from_str(k.as_str()).map_err(|e| {
ArrowError::ParseError(format!("Cannot convert header key \"{k}\": {e}"))
})?;
let v = v.parse().map_err(|e| {
ArrowError::ParseError(format!("Cannot convert header value \"{v}\": {e}"))
})?;
req.metadata_mut().insert(k, v);
}
if let Some(token) = &self.token {
let val = format!("Bearer {token}").parse().map_err(|e| {
ArrowError::ParseError(format!("Cannot convert token to header value: {e}"))
})?;
req.metadata_mut().insert("authorization", val);
}
Ok(req)
}
}
/// A PreparedStatement
#[derive(Debug, Clone)]
pub struct PreparedStatement<T> {
flight_sql_client: FlightSqlServiceClient<T>,
parameter_binding: Option<RecordBatch>,
handle: Bytes,
dataset_schema: Schema,
parameter_schema: Schema,
}
impl PreparedStatement<Channel> {
pub(crate) fn new(
flight_client: FlightSqlServiceClient<Channel>,
handle: impl Into<Bytes>,
dataset_schema: Schema,
parameter_schema: Schema,
) -> Self {
PreparedStatement {
flight_sql_client: flight_client,
parameter_binding: None,
handle: handle.into(),
dataset_schema,
parameter_schema,
}
}
/// Executes the prepared statement query on the server.
pub async fn execute(&mut self) -> Result<FlightInfo, ArrowError> {
self.write_bind_params().await?;
let cmd = CommandPreparedStatementQuery {
prepared_statement_handle: self.handle.clone(),
};
let result = self
.flight_sql_client
.get_flight_info_for_command(cmd)
.await?;
Ok(result)
}
/// Executes the prepared statement update query on the server.
pub async fn execute_update(&mut self) -> Result<i64, ArrowError> {
self.write_bind_params().await?;
let cmd = CommandPreparedStatementUpdate {
prepared_statement_handle: self.handle.clone(),
};
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec());
let mut result = self
.flight_sql_client
.do_put(stream::iter(vec![FlightData {
flight_descriptor: Some(descriptor),
..Default::default()
}]))
.await?;
let result = result
.message()
.await
.map_err(status_to_arrow_error)?
.unwrap();
let any = Any::decode(&*result.app_metadata).map_err(decode_error_to_arrow_error)?;
let result: DoPutUpdateResult = any.unpack()?.unwrap();
Ok(result.record_count)
}
/// Retrieve the parameter schema from the query.
pub fn parameter_schema(&self) -> Result<&Schema, ArrowError> {
Ok(&self.parameter_schema)
}
/// Retrieve the ResultSet schema from the query.
pub fn dataset_schema(&self) -> Result<&Schema, ArrowError> {
Ok(&self.dataset_schema)
}
/// Set a RecordBatch that contains the parameters that will be bind.
pub fn set_parameters(&mut self, parameter_binding: RecordBatch) -> Result<(), ArrowError> {
self.parameter_binding = Some(parameter_binding);
Ok(())
}
/// Submit parameters to the server, if any have been set on this prepared statement instance
/// Updates our stored prepared statement handle with the handle given by the server response.
async fn write_bind_params(&mut self) -> Result<(), ArrowError> {
if let Some(ref params_batch) = self.parameter_binding {
let cmd = CommandPreparedStatementQuery {
prepared_statement_handle: self.handle.clone(),
};
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec());
let flight_stream_builder = FlightDataEncoderBuilder::new()
.with_flight_descriptor(Some(descriptor))
.with_schema(params_batch.schema());
let flight_data = flight_stream_builder
.build(futures::stream::iter(
self.parameter_binding.clone().map(Ok),
))
.try_collect::<Vec<_>>()
.await
.map_err(flight_error_to_arrow_error)?;
// Attempt to update the stored handle with any updated handle in the DoPut result.
// Older servers do not respond with a result for DoPut, so skip this step when
// the stream closes with no response.
if let Some(result) = self
.flight_sql_client
.do_put(stream::iter(flight_data))
.await?
.message()
.await
.map_err(status_to_arrow_error)?
{
if let Some(handle) = self.unpack_prepared_statement_handle(&result)? {
self.handle = handle;
}
}
}
Ok(())
}
/// Decodes the app_metadata stored in a [`PutResult`] as a
/// [`DoPutPreparedStatementResult`] and then returns
/// the inner prepared statement handle as [`Bytes`]
fn unpack_prepared_statement_handle(
&self,
put_result: &PutResult,
) -> Result<Option<Bytes>, ArrowError> {
let result: DoPutPreparedStatementResult =
Message::decode(&*put_result.app_metadata).map_err(decode_error_to_arrow_error)?;
Ok(result.prepared_statement_handle)
}
/// Close the prepared statement, so that this PreparedStatement can not used
/// anymore and server can free up any resources.
pub async fn close(mut self) -> Result<(), ArrowError> {
let cmd = ActionClosePreparedStatementRequest {
prepared_statement_handle: self.handle.clone(),
};
let action = Action {
r#type: CLOSE_PREPARED_STATEMENT.to_string(),
body: cmd.as_any().encode_to_vec().into(),
};
let _ = self.flight_sql_client.do_action(action).await?;
Ok(())
}
}
fn decode_error_to_arrow_error(err: prost::DecodeError) -> ArrowError {
ArrowError::IpcError(err.to_string())
}
fn status_to_arrow_error(status: tonic::Status) -> ArrowError {
ArrowError::IpcError(format!("{status:?}"))
}
fn flight_error_to_arrow_error(err: FlightError) -> ArrowError {
match err {
FlightError::Arrow(e) => e,
e => ArrowError::ExternalError(Box::new(e)),
}
}
/// A polymorphic structure to natively represent different types of data contained in `FlightData`
pub enum ArrowFlightData {
/// A record batch
RecordBatch(RecordBatch),
/// A schema
Schema(Schema),
}
/// Extract `Schema` or `RecordBatch`es from the `FlightData` wire representation
pub fn arrow_data_from_flight_data(
flight_data: FlightData,
arrow_schema_ref: &SchemaRef,
) -> Result<ArrowFlightData, ArrowError> {
let ipc_message = root_as_message(&flight_data.data_header[..])
.map_err(|err| ArrowError::ParseError(format!("Unable to get root as message: {err:?}")))?;
match ipc_message.header_type() {
MessageHeader::RecordBatch => {
let ipc_record_batch = ipc_message.header_as_record_batch().ok_or_else(|| {
ArrowError::ComputeError(
"Unable to convert flight data header to a record batch".to_string(),
)
})?;
let dictionaries_by_field = HashMap::new();
let record_batch = read_record_batch(
&Buffer::from_bytes(flight_data.data_body.into()),
ipc_record_batch,
arrow_schema_ref.clone(),
&dictionaries_by_field,
None,
&ipc_message.version(),
)?;
Ok(ArrowFlightData::RecordBatch(record_batch))
}
MessageHeader::Schema => {
let ipc_schema = ipc_message.header_as_schema().ok_or_else(|| {
ArrowError::ComputeError(
"Unable to convert flight data header to a schema".to_string(),
)
})?;
let arrow_schema = fb_to_schema(ipc_schema);
Ok(ArrowFlightData::Schema(arrow_schema))
}
MessageHeader::DictionaryBatch => {
let _ = ipc_message.header_as_dictionary_batch().ok_or_else(|| {
ArrowError::ComputeError(
"Unable to convert flight data header to a dictionary batch".to_string(),
)
})?;
Err(ArrowError::NotYetImplemented(
"no idea on how to convert an ipc dictionary batch to an arrow type".to_string(),
))
}
MessageHeader::Tensor => {
let _ = ipc_message.header_as_tensor().ok_or_else(|| {
ArrowError::ComputeError(
"Unable to convert flight data header to a tensor".to_string(),
)
})?;
Err(ArrowError::NotYetImplemented(
"no idea on how to convert an ipc tensor to an arrow type".to_string(),
))
}
MessageHeader::SparseTensor => {
let _ = ipc_message.header_as_sparse_tensor().ok_or_else(|| {
ArrowError::ComputeError(
"Unable to convert flight data header to a sparse tensor".to_string(),
)
})?;
Err(ArrowError::NotYetImplemented(
"no idea on how to convert an ipc sparse tensor to an arrow type".to_string(),
))
}
_ => Err(ArrowError::ComputeError(format!(
"Unable to convert message with header_type: '{:?}' to arrow data",
ipc_message.header_type()
))),
}
}