-
Notifications
You must be signed in to change notification settings - Fork 796
/
server.rs
1064 lines (975 loc) · 41 KB
/
server.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
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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.
//! Helper trait [`FlightSqlService`] for implementing a [`FlightService`] that implements FlightSQL.
use std::pin::Pin;
use futures::{stream::Peekable, Stream, StreamExt};
use prost::Message;
use tonic::{Request, Response, Status, Streaming};
use super::{
ActionBeginSavepointRequest, ActionBeginSavepointResult, ActionBeginTransactionRequest,
ActionBeginTransactionResult, ActionCancelQueryRequest, ActionCancelQueryResult,
ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
ActionCreatePreparedStatementResult, ActionCreatePreparedSubstraitPlanRequest,
ActionEndSavepointRequest, ActionEndTransactionRequest, Any, Command, CommandGetCatalogs,
CommandGetCrossReference, CommandGetDbSchemas, CommandGetExportedKeys, CommandGetImportedKeys,
CommandGetPrimaryKeys, CommandGetSqlInfo, CommandGetTableTypes, CommandGetTables,
CommandGetXdbcTypeInfo, CommandPreparedStatementQuery, CommandPreparedStatementUpdate,
CommandStatementIngest, CommandStatementQuery, CommandStatementSubstraitPlan,
CommandStatementUpdate, DoPutPreparedStatementResult, DoPutUpdateResult, ProstMessageExt,
SqlInfo, TicketStatementQuery,
};
use crate::{
flight_service_server::FlightService, gen::PollInfo, Action, ActionType, Criteria, Empty,
FlightData, FlightDescriptor, FlightInfo, HandshakeRequest, HandshakeResponse, PutResult,
SchemaResult, Ticket,
};
pub(crate) static CREATE_PREPARED_STATEMENT: &str = "CreatePreparedStatement";
pub(crate) static CLOSE_PREPARED_STATEMENT: &str = "ClosePreparedStatement";
pub(crate) static CREATE_PREPARED_SUBSTRAIT_PLAN: &str = "CreatePreparedSubstraitPlan";
pub(crate) static BEGIN_TRANSACTION: &str = "BeginTransaction";
pub(crate) static END_TRANSACTION: &str = "EndTransaction";
pub(crate) static BEGIN_SAVEPOINT: &str = "BeginSavepoint";
pub(crate) static END_SAVEPOINT: &str = "EndSavepoint";
pub(crate) static CANCEL_QUERY: &str = "CancelQuery";
/// Implements FlightSqlService to handle the flight sql protocol
#[tonic::async_trait]
pub trait FlightSqlService: Sync + Send + Sized + 'static {
/// When impl FlightSqlService, you can always set FlightService to Self
type FlightService: FlightService;
/// Accept authentication and return a token
/// <https://arrow.apache.org/docs/format/Flight.html#authentication>
async fn do_handshake(
&self,
_request: Request<Streaming<HandshakeRequest>>,
) -> Result<
Response<Pin<Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send>>>,
Status,
> {
Err(Status::unimplemented(
"Handshake has no default implementation",
))
}
/// Implementors may override to handle additional calls to do_get()
async fn do_get_fallback(
&self,
_request: Request<Ticket>,
message: Any,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(format!(
"do_get: The defined request is invalid: {}",
message.type_url
)))
}
/// Get a FlightInfo for executing a SQL query.
async fn get_flight_info_statement(
&self,
_query: CommandStatementQuery,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_statement has no default implementation",
))
}
/// Get a FlightInfo for executing a substrait plan.
async fn get_flight_info_substrait_plan(
&self,
_query: CommandStatementSubstraitPlan,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_substrait_plan has no default implementation",
))
}
/// Get a FlightInfo for executing an already created prepared statement.
async fn get_flight_info_prepared_statement(
&self,
_query: CommandPreparedStatementQuery,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_prepared_statement has no default implementation",
))
}
/// Get a FlightInfo for listing catalogs.
async fn get_flight_info_catalogs(
&self,
_query: CommandGetCatalogs,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_catalogs has no default implementation",
))
}
/// Get a FlightInfo for listing schemas.
async fn get_flight_info_schemas(
&self,
_query: CommandGetDbSchemas,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_schemas has no default implementation",
))
}
/// Get a FlightInfo for listing tables.
async fn get_flight_info_tables(
&self,
_query: CommandGetTables,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_tables has no default implementation",
))
}
/// Get a FlightInfo to extract information about the table types.
async fn get_flight_info_table_types(
&self,
_query: CommandGetTableTypes,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_table_types has no default implementation",
))
}
/// Get a FlightInfo for retrieving other information (See SqlInfo).
async fn get_flight_info_sql_info(
&self,
_query: CommandGetSqlInfo,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_sql_info has no default implementation",
))
}
/// Get a FlightInfo to extract information about primary and foreign keys.
async fn get_flight_info_primary_keys(
&self,
_query: CommandGetPrimaryKeys,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_primary_keys has no default implementation",
))
}
/// Get a FlightInfo to extract information about exported keys.
async fn get_flight_info_exported_keys(
&self,
_query: CommandGetExportedKeys,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_exported_keys has no default implementation",
))
}
/// Get a FlightInfo to extract information about imported keys.
async fn get_flight_info_imported_keys(
&self,
_query: CommandGetImportedKeys,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_imported_keys has no default implementation",
))
}
/// Get a FlightInfo to extract information about cross reference.
async fn get_flight_info_cross_reference(
&self,
_query: CommandGetCrossReference,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_cross_reference has no default implementation",
))
}
/// Get a FlightInfo to extract information about the supported XDBC types.
async fn get_flight_info_xdbc_type_info(
&self,
_query: CommandGetXdbcTypeInfo,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(
"get_flight_info_xdbc_type_info has no default implementation",
))
}
/// Implementors may override to handle additional calls to get_flight_info()
async fn get_flight_info_fallback(
&self,
cmd: Command,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented(format!(
"get_flight_info: The defined request is invalid: {}",
cmd.type_url()
)))
}
// do_get
/// Get a FlightDataStream containing the query results.
async fn do_get_statement(
&self,
_ticket: TicketStatementQuery,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_statement has no default implementation",
))
}
/// Get a FlightDataStream containing the prepared statement query results.
async fn do_get_prepared_statement(
&self,
_query: CommandPreparedStatementQuery,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_prepared_statement has no default implementation",
))
}
/// Get a FlightDataStream containing the list of catalogs.
async fn do_get_catalogs(
&self,
_query: CommandGetCatalogs,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_catalogs has no default implementation",
))
}
/// Get a FlightDataStream containing the list of schemas.
async fn do_get_schemas(
&self,
_query: CommandGetDbSchemas,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_schemas has no default implementation",
))
}
/// Get a FlightDataStream containing the list of tables.
async fn do_get_tables(
&self,
_query: CommandGetTables,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_tables has no default implementation",
))
}
/// Get a FlightDataStream containing the data related to the table types.
async fn do_get_table_types(
&self,
_query: CommandGetTableTypes,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_table_types has no default implementation",
))
}
/// Get a FlightDataStream containing the list of SqlInfo results.
async fn do_get_sql_info(
&self,
_query: CommandGetSqlInfo,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_sql_info has no default implementation",
))
}
/// Get a FlightDataStream containing the data related to the primary and foreign keys.
async fn do_get_primary_keys(
&self,
_query: CommandGetPrimaryKeys,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_primary_keys has no default implementation",
))
}
/// Get a FlightDataStream containing the data related to the exported keys.
async fn do_get_exported_keys(
&self,
_query: CommandGetExportedKeys,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_exported_keys has no default implementation",
))
}
/// Get a FlightDataStream containing the data related to the imported keys.
async fn do_get_imported_keys(
&self,
_query: CommandGetImportedKeys,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_imported_keys has no default implementation",
))
}
/// Get a FlightDataStream containing the data related to the cross reference.
async fn do_get_cross_reference(
&self,
_query: CommandGetCrossReference,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_cross_reference has no default implementation",
))
}
/// Get a FlightDataStream containing the data related to the supported XDBC types.
async fn do_get_xdbc_type_info(
&self,
_query: CommandGetXdbcTypeInfo,
_request: Request<Ticket>,
) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
Err(Status::unimplemented(
"do_get_xdbc_type_info has no default implementation",
))
}
// do_put
/// Implementors may override to handle additional calls to do_put()
async fn do_put_fallback(
&self,
_request: Request<PeekableFlightDataStream>,
message: Any,
) -> Result<Response<<Self as FlightService>::DoPutStream>, Status> {
Err(Status::unimplemented(format!(
"do_put: The defined request is invalid: {}",
message.type_url
)))
}
/// Execute an update SQL statement.
async fn do_put_statement_update(
&self,
_ticket: CommandStatementUpdate,
_request: Request<PeekableFlightDataStream>,
) -> Result<i64, Status> {
Err(Status::unimplemented(
"do_put_statement_update has no default implementation",
))
}
/// Execute a bulk ingestion.
async fn do_put_statement_ingest(
&self,
_ticket: CommandStatementIngest,
_request: Request<PeekableFlightDataStream>,
) -> Result<i64, Status> {
Err(Status::unimplemented(
"do_put_statement_ingest has no default implementation",
))
}
/// Bind parameters to given prepared statement.
///
/// Returns an opaque handle that the client should pass
/// back to the server during subsequent requests with this
/// prepared statement.
async fn do_put_prepared_statement_query(
&self,
_query: CommandPreparedStatementQuery,
_request: Request<PeekableFlightDataStream>,
) -> Result<DoPutPreparedStatementResult, Status> {
Err(Status::unimplemented(
"do_put_prepared_statement_query has no default implementation",
))
}
/// Execute an update SQL prepared statement.
async fn do_put_prepared_statement_update(
&self,
_query: CommandPreparedStatementUpdate,
_request: Request<PeekableFlightDataStream>,
) -> Result<i64, Status> {
Err(Status::unimplemented(
"do_put_prepared_statement_update has no default implementation",
))
}
/// Execute a substrait plan
async fn do_put_substrait_plan(
&self,
_query: CommandStatementSubstraitPlan,
_request: Request<PeekableFlightDataStream>,
) -> Result<i64, Status> {
Err(Status::unimplemented(
"do_put_substrait_plan has no default implementation",
))
}
// do_action
/// Implementors may override to handle additional calls to do_action()
async fn do_action_fallback(
&self,
request: Request<Action>,
) -> Result<Response<<Self as FlightService>::DoActionStream>, Status> {
Err(Status::invalid_argument(format!(
"do_action: The defined request is invalid: {:?}",
request.get_ref().r#type
)))
}
/// Add custom actions to list_actions() result
async fn list_custom_actions(&self) -> Option<Vec<Result<ActionType, Status>>> {
None
}
/// Create a prepared statement from given SQL statement.
async fn do_action_create_prepared_statement(
&self,
_query: ActionCreatePreparedStatementRequest,
_request: Request<Action>,
) -> Result<ActionCreatePreparedStatementResult, Status> {
Err(Status::unimplemented(
"do_action_create_prepared_statement has no default implementation",
))
}
/// Close a prepared statement.
async fn do_action_close_prepared_statement(
&self,
_query: ActionClosePreparedStatementRequest,
_request: Request<Action>,
) -> Result<(), Status> {
Err(Status::unimplemented(
"do_action_close_prepared_statement has no default implementation",
))
}
/// Create a prepared substrait plan.
async fn do_action_create_prepared_substrait_plan(
&self,
_query: ActionCreatePreparedSubstraitPlanRequest,
_request: Request<Action>,
) -> Result<ActionCreatePreparedStatementResult, Status> {
Err(Status::unimplemented(
"do_action_create_prepared_substrait_plan has no default implementation",
))
}
/// Begin a transaction
async fn do_action_begin_transaction(
&self,
_query: ActionBeginTransactionRequest,
_request: Request<Action>,
) -> Result<ActionBeginTransactionResult, Status> {
Err(Status::unimplemented(
"do_action_begin_transaction has no default implementation",
))
}
/// End a transaction
async fn do_action_end_transaction(
&self,
_query: ActionEndTransactionRequest,
_request: Request<Action>,
) -> Result<(), Status> {
Err(Status::unimplemented(
"do_action_end_transaction has no default implementation",
))
}
/// Begin a savepoint
async fn do_action_begin_savepoint(
&self,
_query: ActionBeginSavepointRequest,
_request: Request<Action>,
) -> Result<ActionBeginSavepointResult, Status> {
Err(Status::unimplemented(
"do_action_begin_savepoint has no default implementation",
))
}
/// End a savepoint
async fn do_action_end_savepoint(
&self,
_query: ActionEndSavepointRequest,
_request: Request<Action>,
) -> Result<(), Status> {
Err(Status::unimplemented(
"do_action_end_savepoint has no default implementation",
))
}
/// Cancel a query
async fn do_action_cancel_query(
&self,
_query: ActionCancelQueryRequest,
_request: Request<Action>,
) -> Result<ActionCancelQueryResult, Status> {
Err(Status::unimplemented(
"do_action_cancel_query has no default implementation",
))
}
/// do_exchange
/// Implementors may override to handle additional calls to do_exchange()
async fn do_exchange_fallback(
&self,
_request: Request<Streaming<FlightData>>,
) -> Result<Response<<Self as FlightService>::DoExchangeStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
/// Register a new SqlInfo result, making it available when calling GetSqlInfo.
async fn register_sql_info(&self, id: i32, result: &SqlInfo);
}
/// Implements the lower level interface to handle FlightSQL
#[tonic::async_trait]
impl<T: 'static> FlightService for T
where
T: FlightSqlService + Send,
{
type HandshakeStream =
Pin<Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + 'static>>;
type ListFlightsStream =
Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + 'static>>;
type DoGetStream = Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + 'static>>;
type DoPutStream = Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + 'static>>;
type DoActionStream =
Pin<Box<dyn Stream<Item = Result<super::super::Result, Status>> + Send + 'static>>;
type ListActionsStream =
Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + 'static>>;
type DoExchangeStream =
Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + 'static>>;
async fn handshake(
&self,
request: Request<Streaming<HandshakeRequest>>,
) -> Result<Response<Self::HandshakeStream>, Status> {
let res = self.do_handshake(request).await?;
Ok(res)
}
async fn list_flights(
&self,
_request: Request<Criteria>,
) -> Result<Response<Self::ListFlightsStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn get_flight_info(
&self,
request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
let message = Any::decode(&*request.get_ref().cmd).map_err(decode_error_to_status)?;
match Command::try_from(message).map_err(arrow_error_to_status)? {
Command::CommandStatementQuery(token) => {
self.get_flight_info_statement(token, request).await
}
Command::CommandPreparedStatementQuery(handle) => {
self.get_flight_info_prepared_statement(handle, request)
.await
}
Command::CommandStatementSubstraitPlan(handle) => {
self.get_flight_info_substrait_plan(handle, request).await
}
Command::CommandGetCatalogs(token) => {
self.get_flight_info_catalogs(token, request).await
}
Command::CommandGetDbSchemas(token) => {
return self.get_flight_info_schemas(token, request).await
}
Command::CommandGetTables(token) => self.get_flight_info_tables(token, request).await,
Command::CommandGetTableTypes(token) => {
self.get_flight_info_table_types(token, request).await
}
Command::CommandGetSqlInfo(token) => {
self.get_flight_info_sql_info(token, request).await
}
Command::CommandGetPrimaryKeys(token) => {
self.get_flight_info_primary_keys(token, request).await
}
Command::CommandGetExportedKeys(token) => {
self.get_flight_info_exported_keys(token, request).await
}
Command::CommandGetImportedKeys(token) => {
self.get_flight_info_imported_keys(token, request).await
}
Command::CommandGetCrossReference(token) => {
self.get_flight_info_cross_reference(token, request).await
}
Command::CommandGetXdbcTypeInfo(token) => {
self.get_flight_info_xdbc_type_info(token, request).await
}
cmd => self.get_flight_info_fallback(cmd, request).await,
}
}
async fn poll_flight_info(
&self,
_request: Request<FlightDescriptor>,
) -> Result<Response<PollInfo>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn get_schema(
&self,
_request: Request<FlightDescriptor>,
) -> Result<Response<SchemaResult>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn do_get(
&self,
request: Request<Ticket>,
) -> Result<Response<Self::DoGetStream>, Status> {
let msg: Any =
Message::decode(&*request.get_ref().ticket).map_err(decode_error_to_status)?;
match Command::try_from(msg).map_err(arrow_error_to_status)? {
Command::TicketStatementQuery(command) => self.do_get_statement(command, request).await,
Command::CommandPreparedStatementQuery(command) => {
self.do_get_prepared_statement(command, request).await
}
Command::CommandGetCatalogs(command) => self.do_get_catalogs(command, request).await,
Command::CommandGetDbSchemas(command) => self.do_get_schemas(command, request).await,
Command::CommandGetTables(command) => self.do_get_tables(command, request).await,
Command::CommandGetTableTypes(command) => {
self.do_get_table_types(command, request).await
}
Command::CommandGetSqlInfo(command) => self.do_get_sql_info(command, request).await,
Command::CommandGetPrimaryKeys(command) => {
self.do_get_primary_keys(command, request).await
}
Command::CommandGetExportedKeys(command) => {
self.do_get_exported_keys(command, request).await
}
Command::CommandGetImportedKeys(command) => {
self.do_get_imported_keys(command, request).await
}
Command::CommandGetCrossReference(command) => {
self.do_get_cross_reference(command, request).await
}
Command::CommandGetXdbcTypeInfo(command) => {
self.do_get_xdbc_type_info(command, request).await
}
cmd => self.do_get_fallback(request, cmd.into_any()).await,
}
}
async fn do_put(
&self,
request: Request<Streaming<FlightData>>,
) -> Result<Response<Self::DoPutStream>, Status> {
// See issue #4658: https://github.com/apache/arrow-rs/issues/4658
// To dispatch to the correct `do_put` method, we cannot discard the first message,
// as it may contain the Arrow schema, which the `do_put` handler may need.
// To allow the first message to be reused by the `do_put` handler,
// we wrap this stream in a `Peekable` one, which allows us to peek at
// the first message without discarding it.
let mut request = request.map(PeekableFlightDataStream::new);
let cmd = Pin::new(request.get_mut()).peek().await.unwrap().clone()?;
let message =
Any::decode(&*cmd.flight_descriptor.unwrap().cmd).map_err(decode_error_to_status)?;
match Command::try_from(message).map_err(arrow_error_to_status)? {
Command::CommandStatementUpdate(command) => {
let record_count = self.do_put_statement_update(command, request).await?;
let result = DoPutUpdateResult { record_count };
let output = futures::stream::iter(vec![Ok(PutResult {
app_metadata: result.as_any().encode_to_vec().into(),
})]);
Ok(Response::new(Box::pin(output)))
}
Command::CommandStatementIngest(command) => {
let record_count = self.do_put_statement_ingest(command, request).await?;
let result = DoPutUpdateResult { record_count };
let output = futures::stream::iter(vec![Ok(PutResult {
app_metadata: result.as_any().encode_to_vec().into(),
})]);
Ok(Response::new(Box::pin(output)))
}
Command::CommandPreparedStatementQuery(command) => {
let result = self
.do_put_prepared_statement_query(command, request)
.await?;
let output = futures::stream::iter(vec![Ok(PutResult {
app_metadata: result.encode_to_vec().into(),
})]);
Ok(Response::new(Box::pin(output)))
}
Command::CommandStatementSubstraitPlan(command) => {
let record_count = self.do_put_substrait_plan(command, request).await?;
let result = DoPutUpdateResult { record_count };
let output = futures::stream::iter(vec![Ok(PutResult {
app_metadata: result.as_any().encode_to_vec().into(),
})]);
Ok(Response::new(Box::pin(output)))
}
Command::CommandPreparedStatementUpdate(command) => {
let record_count = self
.do_put_prepared_statement_update(command, request)
.await?;
let result = DoPutUpdateResult { record_count };
let output = futures::stream::iter(vec![Ok(PutResult {
app_metadata: result.as_any().encode_to_vec().into(),
})]);
Ok(Response::new(Box::pin(output)))
}
cmd => self.do_put_fallback(request, cmd.into_any()).await,
}
}
async fn list_actions(
&self,
_request: Request<Empty>,
) -> Result<Response<Self::ListActionsStream>, Status> {
let create_prepared_statement_action_type = ActionType {
r#type: CREATE_PREPARED_STATEMENT.to_string(),
description: "Creates a reusable prepared statement resource on the server.\n
Request Message: ActionCreatePreparedStatementRequest\n
Response Message: ActionCreatePreparedStatementResult"
.into(),
};
let close_prepared_statement_action_type = ActionType {
r#type: CLOSE_PREPARED_STATEMENT.to_string(),
description: "Closes a reusable prepared statement resource on the server.\n
Request Message: ActionClosePreparedStatementRequest\n
Response Message: N/A"
.into(),
};
let create_prepared_substrait_plan_action_type = ActionType {
r#type: CREATE_PREPARED_SUBSTRAIT_PLAN.to_string(),
description: "Creates a reusable prepared substrait plan resource on the server.\n
Request Message: ActionCreatePreparedSubstraitPlanRequest\n
Response Message: ActionCreatePreparedStatementResult"
.into(),
};
let begin_transaction_action_type = ActionType {
r#type: BEGIN_TRANSACTION.to_string(),
description: "Begins a transaction.\n
Request Message: ActionBeginTransactionRequest\n
Response Message: ActionBeginTransactionResult"
.into(),
};
let end_transaction_action_type = ActionType {
r#type: END_TRANSACTION.to_string(),
description: "Ends a transaction\n
Request Message: ActionEndTransactionRequest\n
Response Message: N/A"
.into(),
};
let begin_savepoint_action_type = ActionType {
r#type: BEGIN_SAVEPOINT.to_string(),
description: "Begins a savepoint.\n
Request Message: ActionBeginSavepointRequest\n
Response Message: ActionBeginSavepointResult"
.into(),
};
let end_savepoint_action_type = ActionType {
r#type: END_SAVEPOINT.to_string(),
description: "Ends a savepoint\n
Request Message: ActionEndSavepointRequest\n
Response Message: N/A"
.into(),
};
let cancel_query_action_type = ActionType {
r#type: CANCEL_QUERY.to_string(),
description: "Cancels a query\n
Request Message: ActionCancelQueryRequest\n
Response Message: ActionCancelQueryResult"
.into(),
};
let mut actions: Vec<Result<ActionType, Status>> = vec![
Ok(create_prepared_statement_action_type),
Ok(close_prepared_statement_action_type),
Ok(create_prepared_substrait_plan_action_type),
Ok(begin_transaction_action_type),
Ok(end_transaction_action_type),
Ok(begin_savepoint_action_type),
Ok(end_savepoint_action_type),
Ok(cancel_query_action_type),
];
if let Some(mut custom_actions) = self.list_custom_actions().await {
actions.append(&mut custom_actions);
}
let output = futures::stream::iter(actions);
Ok(Response::new(Box::pin(output) as Self::ListActionsStream))
}
async fn do_action(
&self,
request: Request<Action>,
) -> Result<Response<Self::DoActionStream>, Status> {
if request.get_ref().r#type == CREATE_PREPARED_STATEMENT {
let any = Any::decode(&*request.get_ref().body).map_err(decode_error_to_status)?;
let cmd: ActionCreatePreparedStatementRequest = any
.unpack()
.map_err(arrow_error_to_status)?
.ok_or_else(|| {
Status::invalid_argument(
"Unable to unpack ActionCreatePreparedStatementRequest.",
)
})?;
let stmt = self
.do_action_create_prepared_statement(cmd, request)
.await?;
let output = futures::stream::iter(vec![Ok(super::super::gen::Result {
body: stmt.as_any().encode_to_vec().into(),
})]);
return Ok(Response::new(Box::pin(output)));
} else if request.get_ref().r#type == CLOSE_PREPARED_STATEMENT {
let any = Any::decode(&*request.get_ref().body).map_err(decode_error_to_status)?;
let cmd: ActionClosePreparedStatementRequest = any
.unpack()
.map_err(arrow_error_to_status)?
.ok_or_else(|| {
Status::invalid_argument(
"Unable to unpack ActionClosePreparedStatementRequest.",
)
})?;
self.do_action_close_prepared_statement(cmd, request)
.await?;
return Ok(Response::new(Box::pin(futures::stream::empty())));
} else if request.get_ref().r#type == CREATE_PREPARED_SUBSTRAIT_PLAN {
let any = Any::decode(&*request.get_ref().body).map_err(decode_error_to_status)?;
let cmd: ActionCreatePreparedSubstraitPlanRequest = any
.unpack()
.map_err(arrow_error_to_status)?
.ok_or_else(|| {
Status::invalid_argument(
"Unable to unpack ActionCreatePreparedSubstraitPlanRequest.",
)
})?;
self.do_action_create_prepared_substrait_plan(cmd, request)
.await?;
return Ok(Response::new(Box::pin(futures::stream::empty())));
} else if request.get_ref().r#type == BEGIN_TRANSACTION {
let any = Any::decode(&*request.get_ref().body).map_err(decode_error_to_status)?;
let cmd: ActionBeginTransactionRequest = any
.unpack()
.map_err(arrow_error_to_status)?
.ok_or_else(|| {
Status::invalid_argument("Unable to unpack ActionBeginTransactionRequest.")
})?;
let stmt = self.do_action_begin_transaction(cmd, request).await?;
let output = futures::stream::iter(vec![Ok(super::super::gen::Result {
body: stmt.as_any().encode_to_vec().into(),
})]);
return Ok(Response::new(Box::pin(output)));
} else if request.get_ref().r#type == END_TRANSACTION {
let any = Any::decode(&*request.get_ref().body).map_err(decode_error_to_status)?;
let cmd: ActionEndTransactionRequest = any
.unpack()
.map_err(arrow_error_to_status)?
.ok_or_else(|| {
Status::invalid_argument("Unable to unpack ActionEndTransactionRequest.")
})?;
self.do_action_end_transaction(cmd, request).await?;
return Ok(Response::new(Box::pin(futures::stream::empty())));
} else if request.get_ref().r#type == BEGIN_SAVEPOINT {
let any = Any::decode(&*request.get_ref().body).map_err(decode_error_to_status)?;
let cmd: ActionBeginSavepointRequest = any
.unpack()
.map_err(arrow_error_to_status)?
.ok_or_else(|| {
Status::invalid_argument("Unable to unpack ActionBeginSavepointRequest.")
})?;
let stmt = self.do_action_begin_savepoint(cmd, request).await?;
let output = futures::stream::iter(vec![Ok(super::super::gen::Result {
body: stmt.as_any().encode_to_vec().into(),
})]);
return Ok(Response::new(Box::pin(output)));
} else if request.get_ref().r#type == END_SAVEPOINT {
let any = Any::decode(&*request.get_ref().body).map_err(decode_error_to_status)?;
let cmd: ActionEndSavepointRequest = any
.unpack()
.map_err(arrow_error_to_status)?
.ok_or_else(|| {
Status::invalid_argument("Unable to unpack ActionEndSavepointRequest.")
})?;
self.do_action_end_savepoint(cmd, request).await?;
return Ok(Response::new(Box::pin(futures::stream::empty())));
} else if request.get_ref().r#type == CANCEL_QUERY {
let any = Any::decode(&*request.get_ref().body).map_err(decode_error_to_status)?;
let cmd: ActionCancelQueryRequest = any
.unpack()
.map_err(arrow_error_to_status)?
.ok_or_else(|| {
Status::invalid_argument("Unable to unpack ActionCancelQueryRequest.")
})?;
let stmt = self.do_action_cancel_query(cmd, request).await?;
let output = futures::stream::iter(vec![Ok(super::super::gen::Result {
body: stmt.as_any().encode_to_vec().into(),
})]);
return Ok(Response::new(Box::pin(output)));
}
self.do_action_fallback(request).await
}
async fn do_exchange(
&self,
request: Request<Streaming<FlightData>>,
) -> Result<Response<Self::DoExchangeStream>, Status> {
self.do_exchange_fallback(request).await
}
}
fn decode_error_to_status(err: prost::DecodeError) -> Status {
Status::invalid_argument(format!("{err:?}"))
}
fn arrow_error_to_status(err: arrow_schema::ArrowError) -> Status {
Status::internal(format!("{err:?}"))
}
/// A wrapper around [`Streaming<FlightData>`] that allows "peeking" at the
/// message at the front of the stream without consuming it.
///
/// This is needed because sometimes the first message in the stream will contain
/// a [`FlightDescriptor`] in addition to potentially any data, and the dispatch logic
/// must inspect this information.
///
/// # Example
///
/// [`PeekableFlightDataStream::peek`] can be used to peek at the first message without
/// discarding it; otherwise, `PeekableFlightDataStream` can be used as a regular stream.
/// See the following example:
///
/// ```no_run
/// use arrow_array::RecordBatch;
/// use arrow_flight::decode::FlightRecordBatchStream;
/// use arrow_flight::FlightDescriptor;
/// use arrow_flight::error::FlightError;
/// use arrow_flight::sql::server::PeekableFlightDataStream;
/// use tonic::{Request, Status};
/// use futures::TryStreamExt;
///