arrow-flight-sql-client 0.4.0

An Apache Arrow Flight SQL client
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
use std::cell::{RefCell, RefMut};

use arrow::{
    datatypes::{Schema, SchemaRef},
    error::{ArrowError, Result},
    ipc::convert,
    ipc::*,
};
use futures::stream;
use prost::Message;
use tonic::{
    codegen::{Body, StdError},
    Streaming,
};

use crate::arrow_flight_protocol::{flight_service_client::FlightServiceClient, *};

use crate::arrow_flight_protocol::flight_descriptor::DescriptorType;
use crate::arrow_flight_protocol::FlightDescriptor;

use std::collections::HashMap;
use std::{convert::TryFrom, ops::Deref};

use crate::arrow_flight_protocol_sql::*;

/// A FlightSQLServiceClient is an endpoint for retrieving or storing Arrow data
/// by FlightSQL protocol.
#[derive(Debug, Clone)]
pub struct FlightSqlServiceClient<T> {
    inner: RefCell<FlightServiceClient<T>>,
}

impl<T> FlightSqlServiceClient<T>
where
    T: tonic::client::GrpcService<tonic::body::BoxBody>,
    T::Error: Into<StdError>,
    T::ResponseBody: Default + Body<Data = bytes::Bytes> + Send + 'static,
    <T::ResponseBody as Body>::Error: Into<StdError> + Send,
{
    /// create FlightSqlServiceClient using FlightServiceClient
    pub fn new(client: RefCell<FlightServiceClient<T>>) -> Self {
        FlightSqlServiceClient { inner: client }
    }

    /// borrow mut FlightServiceClient
    fn mut_client(&self) -> RefMut<'_, FlightServiceClient<T>> {
        self.inner.borrow_mut()
    }

    async fn get_flight_info_for_command<M: ProstMessageExt>(
        &mut self,
        cmd: M,
    ) -> Result<FlightInfo> {
        let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec());
        Ok(self
            .mut_client()
            .get_flight_info(descriptor)
            .await
            .map_err(status_to_arrow_error)?
            .into_inner())
    }

    /// Execute a query on the server.
    pub async fn execute(&mut self, query: String) -> Result<FlightInfo> {
        let cmd = CommandStatementQuery { query };
        self.get_flight_info_for_command(cmd).await
    }

    /// Execute a update query on the server.
    pub async fn execute_update(&mut self, query: String) -> Result<i64> {
        let cmd = CommandStatementUpdate { query };
        let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec());
        let mut result = self
            .mut_client()
            .do_put(stream::iter(vec![FlightData {
                flight_descriptor: Some(descriptor),
                ..Default::default()
            }]))
            .await
            .map_err(status_to_arrow_error)?
            .into_inner();
        let result = result
            .message()
            .await
            .map_err(status_to_arrow_error)?
            .unwrap();
        let any: prost_types::Any =
            prost::Message::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.
    pub async fn get_catalogs(&mut self) -> Result<FlightInfo> {
        self.get_flight_info_for_command(CommandGetCatalogs {})
            .await
    }

    /// Request a list of database schemas.
    pub async fn get_db_schemas(&mut self, request: CommandGetDbSchemas) -> Result<FlightInfo> {
        self.get_flight_info_for_command(request).await
    }

    /// Given a flight ticket and schema, request to be sent the
    /// stream. Returns record batch stream reader
    pub async fn do_get(&mut self, ticket: Ticket) -> Result<Streaming<FlightData>> {
        Ok(self
            .mut_client()
            .do_get(ticket)
            .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> {
        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> {
        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> {
        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> {
        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> {
        self.get_flight_info_for_command(request).await
    }

    /// Request a list of table types.
    pub async fn get_table_types(&mut self) -> Result<FlightInfo> {
        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> {
        let request = CommandGetSqlInfo {
            info: sql_infos.iter().map(|sql_info| *sql_info as u32).collect(),
        };
        self.get_flight_info_for_command(request).await
    }

    /// Create a prepared statement object.
    pub async fn prepare(&mut self, query: String) -> Result<PreparedStatement<'_, T>> {
        let cmd = ActionCreatePreparedStatementRequest { query };
        let action = Action {
            r#type: ACTION_TYPE_CREATE_PREPARED_STATEMENT.to_string(),
            body: cmd.as_any().encode_to_vec(),
        };
        let mut result = self
            .mut_client()
            .do_action(tonic::Request::new(action))
            .await
            .map_err(status_to_arrow_error)?
            .into_inner();
        let result = result
            .message()
            .await
            .map_err(status_to_arrow_error)?
            .unwrap();
        let any: prost_types::Any =
            prost::Message::decode(&*result.body).map_err(decode_error_to_arrow_error)?;
        let prepared_result: ActionCreatePreparedStatementResult = any.unpack()?.unwrap();
        let dataset_schema = Schema::try_from(IpcMessage(prepared_result.dataset_schema))?;
        let parameter_schema = Schema::try_from(IpcMessage(prepared_result.parameter_schema))?;
        Ok(PreparedStatement::new(
            &self.inner,
            prepared_result.prepared_statement_handle,
            dataset_schema,
            parameter_schema,
        ))
    }

    /// Explicitly shut down and clean up the client.
    pub async fn close(&mut self) -> Result<()> {
        Ok(())
    }
}

/// A PreparedStatement
#[derive(Debug, Clone)]
pub struct PreparedStatement<'a, T> {
    inner: &'a RefCell<FlightServiceClient<T>>,
    is_closed: bool,
    parameter_binding: Option<RecordBatch<'a>>,
    handle: Vec<u8>,
    dataset_schema: Schema,
    parameter_schema: Schema,
}

impl<'a, T> PreparedStatement<'a, T>
where
    T: tonic::client::GrpcService<tonic::body::BoxBody>,
    T::Error: Into<StdError>,
    T::ResponseBody: Default + Body<Data = bytes::Bytes> + Send + 'static,
    <T::ResponseBody as Body>::Error: Into<StdError> + Send,
{
    pub(crate) fn new(
        client: &'a RefCell<FlightServiceClient<T>>,
        handle: Vec<u8>,
        dataset_schema: Schema,
        parameter_schema: Schema,
    ) -> Self {
        PreparedStatement {
            inner: client,
            is_closed: false,
            parameter_binding: None,
            handle,
            dataset_schema,
            parameter_schema,
        }
    }
    /// Executes the prepared statement query on the server.
    pub async fn execute(&mut self) -> Result<FlightInfo> {
        if self.is_closed() {
            return Err(ArrowError::IoError("Statement already closed.".to_string()));
        }
        let cmd = CommandPreparedStatementQuery {
            prepared_statement_handle: self.handle.clone(),
        };
        let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec());
        let mut result = self
            .mut_client()
            .do_put(stream::iter(vec![FlightData {
                flight_descriptor: Some(descriptor),
                ..Default::default()
            }]))
            .await
            .map_err(status_to_arrow_error)?
            .into_inner();
        let result = result
            .message()
            .await
            .map_err(status_to_arrow_error)?
            .unwrap();
        let _: prost_types::Any =
            prost::Message::decode(&*result.app_metadata).map_err(decode_error_to_arrow_error)?;
        Err(ArrowError::NotYetImplemented(
            "Not yet implemented".to_string(),
        ))
    }

    /// Executes the prepared statement update query on the server.
    pub async fn execute_update(&self) -> Result<i64> {
        if self.is_closed() {
            return Err(ArrowError::IoError("Statement already closed.".to_string()));
        }
        let cmd = CommandPreparedStatementQuery {
            prepared_statement_handle: self.handle.clone(),
        };
        let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec());
        let mut result = self
            .mut_client()
            .do_put(stream::iter(vec![FlightData {
                flight_descriptor: Some(descriptor),
                ..Default::default()
            }]))
            .await
            .map_err(status_to_arrow_error)?
            .into_inner();
        let result = result
            .message()
            .await
            .map_err(status_to_arrow_error)?
            .unwrap();
        let any: prost_types::Any =
            prost::Message::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 async fn parameter_schema(&self) -> Result<&Schema> {
        Ok(&self.parameter_schema)
    }

    /// Retrieve the ResultSet schema from the query.
    pub async fn dataset_schema(&self) -> Result<&Schema> {
        Ok(&self.dataset_schema)
    }

    /// Set a RecordBatch that contains the parameters that will be bind.
    pub async fn set_parameters(&mut self, parameter_binding: RecordBatch<'a>) -> Result<()> {
        self.parameter_binding = Some(parameter_binding);
        Ok(())
    }

    /// 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<()> {
        if self.is_closed() {
            return Err(ArrowError::IoError("Statement already closed.".to_string()));
        }
        let cmd = ActionClosePreparedStatementRequest {
            prepared_statement_handle: self.handle.clone(),
        };
        let action = Action {
            r#type: ACTION_TYPE_CLOSE_PREPARED_STATEMENT.to_string(),
            body: cmd.as_any().encode_to_vec(),
        };
        let _ = self
            .mut_client()
            .do_action(action)
            .await
            .map_err(status_to_arrow_error)?;
        self.is_closed = true;
        Ok(())
    }

    /// Check if the prepared statement is closed.
    pub fn is_closed(&self) -> bool {
        self.is_closed
    }

    /// borrow mut FlightServiceClient
    fn mut_client(&self) -> RefMut<'_, FlightServiceClient<T>> {
        self.inner.borrow_mut()
    }
}

pub fn decode_error_to_arrow_error(err: prost::DecodeError) -> ArrowError {
    ArrowError::IoError(err.to_string())
}

pub fn arrow_error_to_status(err: arrow::error::ArrowError) -> tonic::Status {
    tonic::Status::internal(format!("{:?}", err))
}

pub fn status_to_arrow_error(status: tonic::Status) -> ArrowError {
    ArrowError::IoError(format!("{:?}", status))
}

pub fn transport_error_to_arrow_erorr(error: tonic::transport::Error) -> ArrowError {
    ArrowError::IoError(format!("{}", error))
}

pub fn arrow_schema_from_flight_info(fi: &FlightInfo) -> Result<Schema> {
    let ipc_message = arrow::ipc::size_prefixed_root_as_message(&fi.schema[4..])
        .map_err(|e| ArrowError::ComputeError(format!("{:?}", e)))?;

    let ipc_schema = ipc_message
        .header_as_schema()
        .ok_or(ArrowError::ComputeError(
            "failed to get schema...".to_string(),
        ))?;

    let arrow_schema = arrow::ipc::convert::fb_to_schema(ipc_schema);

    Ok(arrow_schema)
}

pub enum ArrowFlightData {
    RecordBatch(arrow::record_batch::RecordBatch),
    Schema(arrow::datatypes::Schema),
}

pub fn arrow_data_from_flight_data(
    flight_data: FlightData,
    arrow_schema_ref: &SchemaRef,
) -> Result<ArrowFlightData> {
    let ipc_message = arrow::ipc::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(ArrowError::ComputeError(
                        "Unable to convert flight data header to a record batch".to_string(),
                    ))?;

            let dictionaries_by_field = HashMap::new();
            let record_batch = arrow::ipc::reader::read_record_batch(
                &flight_data.data_body,
                ipc_record_batch,
                arrow_schema_ref.clone(),
                &dictionaries_by_field,
                None,
            )?;
            Ok(ArrowFlightData::RecordBatch(record_batch))
        }
        MessageHeader::Schema => {
            let ipc_schema = ipc_message
                .header_as_schema()
                .ok_or(ArrowError::ComputeError(
                    "Unable to convert flight data header to a schema".to_string(),
                ))?;

            let arrow_schema = arrow::ipc::convert::fb_to_schema(ipc_schema);
            Ok(ArrowFlightData::Schema(arrow_schema))
        }
        MessageHeader::DictionaryBatch => {
            let _ = ipc_message
                .header_as_dictionary_batch()
                .ok_or(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(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(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()
        ))),
    }
}

/// ProstMessageExt are useful utility methods for prost::Message types
pub trait ProstMessageExt: prost::Message + Default + Sized {
    /// type_url for this Message
    fn type_url() -> &'static str;

    /// Convert this Message to prost_types::Any
    fn as_any(&self) -> prost_types::Any;
}

macro_rules! prost_message_ext {
    ($($name:ty,)*) => {
        $(
            impl ProstMessageExt for $name {
                fn type_url() -> &'static str {
                    concat!("type.googleapis.com/arrow.flight.protocol.sql.", stringify!($name))
                }

                fn as_any(&self) -> prost_types::Any {
                    prost_types::Any {
                        type_url: <$name>::type_url().to_string(),
                        value: self.encode_to_vec(),
                    }
                }
            }
        )*
    };
}

// Implement ProstMessageExt for all structs defined in FlightSql.proto
prost_message_ext!(
    ActionClosePreparedStatementRequest,
    ActionCreatePreparedStatementRequest,
    ActionCreatePreparedStatementResult,
    CommandGetCatalogs,
    CommandGetCrossReference,
    CommandGetDbSchemas,
    CommandGetExportedKeys,
    CommandGetImportedKeys,
    CommandGetPrimaryKeys,
    CommandGetSqlInfo,
    CommandGetTableTypes,
    CommandGetTables,
    CommandPreparedStatementQuery,
    CommandPreparedStatementUpdate,
    CommandStatementQuery,
    CommandStatementUpdate,
    DoPutUpdateResult,
    TicketStatementQuery,
);

pub static ACTION_TYPE_CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
pub static ACTION_TYPE_CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";

/// ProstAnyExt are useful utility methods for prost_types::Any
/// The API design is inspired by [rust-protobuf](https://github.com/stepancheg/rust-protobuf/blob/master/protobuf/src/well_known_types_util/any.rs)
pub trait ProstAnyExt {
    /// Check if `Any` contains a message of given type.
    fn is<M: ProstMessageExt>(&self) -> bool;

    /// Extract a message from this `Any`.
    ///
    /// # Returns
    ///
    /// * `Ok(None)` when message type mismatch
    /// * `Err` when parse failed
    fn unpack<M: ProstMessageExt>(&self) -> Result<Option<M>>;

    /// Pack any message into `prost_types::Any` value.
    fn pack<M: ProstMessageExt>(message: &M) -> Result<prost_types::Any>;
}

impl ProstAnyExt for prost_types::Any {
    fn is<M: ProstMessageExt>(&self) -> bool {
        M::type_url() == self.type_url
    }

    fn unpack<M: ProstMessageExt>(&self) -> Result<Option<M>> {
        if !self.is::<M>() {
            return Ok(None);
        }
        let m = prost::Message::decode(&*self.value).map_err(|err| {
            ArrowError::ParseError(format!("Unable to decode Any value: {}", err))
        })?;
        Ok(Some(m))
    }

    fn pack<M: ProstMessageExt>(message: &M) -> Result<prost_types::Any> {
        Ok(message.as_any())
    }
}

/// IpcMessage represents a `Schema` in the format expected in
/// `FlightInfo.schema`
#[derive(Debug)]
pub struct IpcMessage(pub Vec<u8>);

impl Deref for IpcMessage {
    type Target = Vec<u8>;

    fn deref(&self) -> &Self::Target {
        &self.0
    }
}

impl TryFrom<IpcMessage> for Schema {
    type Error = ArrowError;

    fn try_from(value: IpcMessage) -> Result<Self> {
        // CONTINUATION TAKES 4 BYTES
        // SIZE TAKES 4 BYTES (so read msg as size prefixed)
        let msg = size_prefixed_root_as_message(&value.0[4..]).map_err(|err| {
            ArrowError::ParseError(format!(
                "Unable to convert flight info to a message: {}",
                err
            ))
        })?;
        let ipc_schema = msg.header_as_schema().ok_or_else(|| {
            ArrowError::ParseError("Unable to convert flight info to a schema".to_string())
        })?;
        Ok(convert::fb_to_schema(ipc_schema))
    }
}

impl FlightDescriptor {
    pub fn new_cmd(cmd: Vec<u8>) -> Self {
        FlightDescriptor {
            r#type: DescriptorType::Cmd.into(),
            cmd,
            ..Default::default()
        }
    }

    pub fn new_path(path: Vec<String>) -> Self {
        FlightDescriptor {
            r#type: DescriptorType::Path.into(),
            path,
            ..Default::default()
        }
    }
}