-
Notifications
You must be signed in to change notification settings - Fork 3.4k
/
client.go
1345 lines (1119 loc) · 40.5 KB
/
client.go
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.
package flightsql
import (
"context"
"errors"
"fmt"
"io"
"github.com/apache/arrow/go/v16/arrow"
"github.com/apache/arrow/go/v16/arrow/array"
"github.com/apache/arrow/go/v16/arrow/flight"
pb "github.com/apache/arrow/go/v16/arrow/flight/gen/flight"
"github.com/apache/arrow/go/v16/arrow/ipc"
"github.com/apache/arrow/go/v16/arrow/memory"
"google.golang.org/grpc"
"google.golang.org/protobuf/proto"
"google.golang.org/protobuf/types/known/anypb"
)
// NewClient is a convenience function to automatically construct
// a flight.Client and return a flightsql.Client containing it rather
// than having to manually construct both yourself. It just delegates
// its arguments to flight.NewClientWithMiddleware to create the
// underlying Flight Client.
func NewClient(addr string, auth flight.ClientAuthHandler, middleware []flight.ClientMiddleware, opts ...grpc.DialOption) (*Client, error) {
return NewClientCtx(context.Background(), addr, auth, middleware, opts...)
}
func NewClientCtx(ctx context.Context, addr string, auth flight.ClientAuthHandler, middleware []flight.ClientMiddleware, opts ...grpc.DialOption) (*Client, error) {
cl, err := flight.NewClientWithMiddlewareCtx(ctx, addr, auth, middleware, opts...)
if err != nil {
return nil, err
}
return &Client{cl, memory.DefaultAllocator}, nil
}
// Client wraps a regular Flight RPC Client to provide the FlightSQL
// interface functions and methods.
type Client struct {
Client flight.Client
Alloc memory.Allocator
}
func descForCommand(cmd proto.Message) (*flight.FlightDescriptor, error) {
var any anypb.Any
if err := any.MarshalFrom(cmd); err != nil {
return nil, err
}
data, err := proto.Marshal(&any)
if err != nil {
return nil, err
}
return &flight.FlightDescriptor{
Type: flight.DescriptorCMD,
Cmd: data,
}, nil
}
func flightInfoForCommand(ctx context.Context, cl *Client, cmd proto.Message, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
desc, err := descForCommand(cmd)
if err != nil {
return nil, err
}
return cl.getFlightInfo(ctx, desc, opts...)
}
func pollInfoForCommand(ctx context.Context, cl *Client, cmd proto.Message, retryDescriptor *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.PollInfo, error) {
if retryDescriptor != nil {
return cl.Client.PollFlightInfo(ctx, retryDescriptor, opts...)
}
desc, err := descForCommand(cmd)
if err != nil {
return nil, err
}
return cl.Client.PollFlightInfo(ctx, desc, opts...)
}
func schemaForCommand(ctx context.Context, cl *Client, cmd proto.Message, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
desc, err := descForCommand(cmd)
if err != nil {
return nil, err
}
return cl.getSchema(ctx, desc, opts...)
}
func packAction(actionType string, msg proto.Message) (action pb.Action, err error) {
var cmd anypb.Any
if err = cmd.MarshalFrom(msg); err != nil {
return
}
action.Type = actionType
action.Body, err = proto.Marshal(&cmd)
return
}
func readResult(stream pb.FlightService_DoActionClient, msg proto.Message) error {
var container anypb.Any
res, err := stream.Recv()
if err != nil {
return err
}
if err = proto.Unmarshal(res.Body, &container); err != nil {
return err
}
return container.UnmarshalTo(msg)
}
// Execute executes the desired query on the server and returns a FlightInfo
// object describing where to retrieve the results.
func (c *Client) Execute(ctx context.Context, query string, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
cmd := pb.CommandStatementQuery{Query: query}
return flightInfoForCommand(ctx, c, &cmd, opts...)
}
// ExecutePoll idempotently starts execution of a query/checks for completion.
// To check for completion, pass the FlightDescriptor from the previous call
// to ExecutePoll as the retryDescriptor.
func (c *Client) ExecutePoll(ctx context.Context, query string, retryDescriptor *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.PollInfo, error) {
cmd := pb.CommandStatementQuery{Query: query}
return pollInfoForCommand(ctx, c, &cmd, retryDescriptor, opts...)
}
// GetExecuteSchema gets the schema of the result set of a query without
// executing the query itself.
func (c *Client) GetExecuteSchema(ctx context.Context, query string, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
cmd := pb.CommandStatementQuery{Query: query}
return schemaForCommand(ctx, c, &cmd, opts...)
}
func (c *Client) ExecuteSubstrait(ctx context.Context, plan SubstraitPlan, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
cmd := pb.CommandStatementSubstraitPlan{
Plan: &pb.SubstraitPlan{Plan: plan.Plan, Version: plan.Version}}
return flightInfoForCommand(ctx, c, &cmd, opts...)
}
func (c *Client) ExecuteSubstraitPoll(ctx context.Context, plan SubstraitPlan, retryDescriptor *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.PollInfo, error) {
cmd := pb.CommandStatementSubstraitPlan{
Plan: &pb.SubstraitPlan{Plan: plan.Plan, Version: plan.Version}}
return pollInfoForCommand(ctx, c, &cmd, retryDescriptor, opts...)
}
func (c *Client) GetExecuteSubstraitSchema(ctx context.Context, plan SubstraitPlan, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
cmd := pb.CommandStatementSubstraitPlan{
Plan: &pb.SubstraitPlan{Plan: plan.Plan, Version: plan.Version}}
return schemaForCommand(ctx, c, &cmd, opts...)
}
// ExecuteUpdate is for executing an update query and only returns the number of affected rows.
func (c *Client) ExecuteUpdate(ctx context.Context, query string, opts ...grpc.CallOption) (n int64, err error) {
var (
cmd pb.CommandStatementUpdate
desc *flight.FlightDescriptor
stream pb.FlightService_DoPutClient
res *pb.PutResult
updateResult pb.DoPutUpdateResult
)
cmd.Query = query
if desc, err = descForCommand(&cmd); err != nil {
return
}
if stream, err = c.Client.DoPut(ctx, opts...); err != nil {
return
}
if err = stream.Send(&flight.FlightData{FlightDescriptor: desc}); err != nil {
return
}
if err = stream.CloseSend(); err != nil {
return
}
if res, err = stream.Recv(); err != nil {
return
}
if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil {
return
}
return updateResult.GetRecordCount(), nil
}
func (c *Client) ExecuteSubstraitUpdate(ctx context.Context, plan SubstraitPlan, opts ...grpc.CallOption) (n int64, err error) {
var (
desc *flight.FlightDescriptor
stream pb.FlightService_DoPutClient
res *pb.PutResult
updateResult pb.DoPutUpdateResult
)
cmd := pb.CommandStatementSubstraitPlan{
Plan: &pb.SubstraitPlan{Plan: plan.Plan, Version: plan.Version}}
if desc, err = descForCommand(&cmd); err != nil {
return
}
if stream, err = c.Client.DoPut(ctx, opts...); err != nil {
return
}
if err = stream.Send(&flight.FlightData{FlightDescriptor: desc}); err != nil {
return
}
if err = stream.CloseSend(); err != nil {
return
}
if res, err = stream.Recv(); err != nil {
return
}
if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil {
return
}
return updateResult.GetRecordCount(), nil
}
// GetCatalogs requests the list of catalogs from the server and
// returns a flightInfo object where the response can be retrieved
func (c *Client) GetCatalogs(ctx context.Context, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
return flightInfoForCommand(ctx, c, &pb.CommandGetCatalogs{}, opts...)
}
// GetCatalogsSchema requests the schema of GetCatalogs from the server
func (c *Client) GetCatalogsSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetCatalogs{}, opts...)
}
// GetDBSchemas requests the list of schemas from the database and
// returns a FlightInfo object where the response can be retrieved
func (c *Client) GetDBSchemas(ctx context.Context, cmdOpts *GetDBSchemasOpts, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
return flightInfoForCommand(ctx, c, (*pb.CommandGetDbSchemas)(cmdOpts), opts...)
}
// GetDBSchemasSchema requests the schema of GetDBSchemas from the server
func (c *Client) GetDBSchemasSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetDbSchemas{}, opts...)
}
// DoGet uses the provided flight ticket to request the stream of data.
// It returns a recordbatch reader to stream the results. Release
// should be called on the reader when done.
func (c *Client) DoGet(ctx context.Context, in *flight.Ticket, opts ...grpc.CallOption) (*flight.Reader, error) {
stream, err := c.Client.DoGet(ctx, in, opts...)
if err != nil {
return nil, err
}
return flight.NewRecordReader(stream, ipc.WithAllocator(c.Alloc))
}
// GetTables requests a list of tables from the server, with the provided
// options describing how to make the request (filter patterns, if the schema
// should be returned, etc.). Returns a FlightInfo object where the response
// can be retrieved.
func (c *Client) GetTables(ctx context.Context, reqOptions *GetTablesOpts, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
return flightInfoForCommand(ctx, c, (*pb.CommandGetTables)(reqOptions), opts...)
}
// GetTablesSchema requests the schema of GetTables from the server.
func (c *Client) GetTablesSchema(ctx context.Context, reqOptions *GetTablesOpts, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, (*pb.CommandGetTables)(reqOptions), opts...)
}
// GetPrimaryKeys requests the primary keys for a specific table from the
// server, specified using a TableRef. Returns a FlightInfo object where
// the response can be retrieved.
func (c *Client) GetPrimaryKeys(ctx context.Context, ref TableRef, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
cmd := pb.CommandGetPrimaryKeys{
Catalog: ref.Catalog,
DbSchema: ref.DBSchema,
Table: ref.Table,
}
return flightInfoForCommand(ctx, c, &cmd, opts...)
}
// GetPrimaryKeysSchema requests the schema of GetPrimaryKeys from the server.
func (c *Client) GetPrimaryKeysSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetPrimaryKeys{}, opts...)
}
// GetExportedKeys retrieves a description about the foreign key columns
// that reference the primary key columns of the specified table. Returns
// a FlightInfo object where the response can be retrieved.
func (c *Client) GetExportedKeys(ctx context.Context, ref TableRef, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
cmd := pb.CommandGetExportedKeys{
Catalog: ref.Catalog,
DbSchema: ref.DBSchema,
Table: ref.Table,
}
return flightInfoForCommand(ctx, c, &cmd, opts...)
}
// GetExportedKeysSchema requests the schema of GetExportedKeys from the server.
func (c *Client) GetExportedKeysSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetExportedKeys{}, opts...)
}
// GetImportedKeys returns the foreign key columns for the specified table.
// Returns a FlightInfo object indicating where the response can be retrieved.
func (c *Client) GetImportedKeys(ctx context.Context, ref TableRef, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
cmd := pb.CommandGetImportedKeys{
Catalog: ref.Catalog,
DbSchema: ref.DBSchema,
Table: ref.Table,
}
return flightInfoForCommand(ctx, c, &cmd, opts...)
}
// GetImportedKeysSchema requests the schema of GetImportedKeys from the server.
func (c *Client) GetImportedKeysSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetImportedKeys{}, opts...)
}
// GetCrossReference retrieves a description of the foreign key columns
// in the specified ForeignKey table that reference the primary key or
// columns representing a restraint of the parent table (could be the same
// or a different table). Returns a FlightInfo object indicating where
// the response can be retrieved with DoGet.
func (c *Client) GetCrossReference(ctx context.Context, pkTable, fkTable TableRef, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
cmd := pb.CommandGetCrossReference{
PkCatalog: pkTable.Catalog,
PkDbSchema: pkTable.DBSchema,
PkTable: pkTable.Table,
FkCatalog: fkTable.Catalog,
FkDbSchema: fkTable.DBSchema,
FkTable: fkTable.Table,
}
return flightInfoForCommand(ctx, c, &cmd, opts...)
}
// GetCrossReferenceSchema requests the schema of GetCrossReference from the server.
func (c *Client) GetCrossReferenceSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetCrossReference{}, opts...)
}
// GetTableTypes requests a list of the types of tables available on this
// server. Returns a FlightInfo object indicating where the response can
// be retrieved.
func (c *Client) GetTableTypes(ctx context.Context, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
return flightInfoForCommand(ctx, c, &pb.CommandGetTableTypes{}, opts...)
}
// GetTableTypesSchema requests the schema of GetTableTypes from the server.
func (c *Client) GetTableTypesSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetTableTypes{}, opts...)
}
// GetXdbcTypeInfo requests the information about all the data types supported
// (dataType == nil) or a specific data type. Returns a FlightInfo object
// indicating where the response can be retrieved.
func (c *Client) GetXdbcTypeInfo(ctx context.Context, dataType *int32, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
return flightInfoForCommand(ctx, c, &pb.CommandGetXdbcTypeInfo{DataType: dataType}, opts...)
}
// GetXdbcTypeInfoSchema requests the schema of GetXdbcTypeInfo from the server.
func (c *Client) GetXdbcTypeInfoSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetXdbcTypeInfo{}, opts...)
}
// GetSqlInfo returns a list of the requested SQL information corresponding
// to the values in the info slice. Returns a FlightInfo object indicating
// where the response can be retrieved.
func (c *Client) GetSqlInfo(ctx context.Context, info []SqlInfo, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
cmd := &pb.CommandGetSqlInfo{Info: make([]uint32, len(info))}
for i, v := range info {
cmd.Info[i] = uint32(v)
}
return flightInfoForCommand(ctx, c, cmd, opts...)
}
// GetSqlInfoSchema requests the schema of GetSqlInfo from the server.
func (c *Client) GetSqlInfoSchema(ctx context.Context, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return schemaForCommand(ctx, c, &pb.CommandGetSqlInfo{}, opts...)
}
// Prepare creates a PreparedStatement object for the specified query.
// The resulting PreparedStatement object should be Closed when no longer
// needed. It will maintain a reference to this Client for use to execute
// and use the specified allocator for any allocations it needs to perform.
func (c *Client) Prepare(ctx context.Context, query string, opts ...grpc.CallOption) (prep *PreparedStatement, err error) {
const actionType = CreatePreparedStatementActionType
var (
request pb.ActionCreatePreparedStatementRequest
action pb.Action
stream pb.FlightService_DoActionClient
)
request.Query = query
if action, err = packAction(actionType, &request); err != nil {
return
}
if stream, err = c.Client.DoAction(ctx, &action, opts...); err != nil {
return
}
return parsePreparedStatementResponse(c, c.Alloc, stream)
}
func (c *Client) PrepareSubstrait(ctx context.Context, plan SubstraitPlan, opts ...grpc.CallOption) (stmt *PreparedStatement, err error) {
const actionType = CreatePreparedSubstraitPlanActionType
var (
request pb.ActionCreatePreparedSubstraitPlanRequest
action pb.Action
stream pb.FlightService_DoActionClient
)
request.Plan = &pb.SubstraitPlan{
Plan: plan.Plan,
Version: plan.Version,
}
if action, err = packAction(actionType, &request); err != nil {
return
}
if stream, err = c.Client.DoAction(ctx, &action, opts...); err != nil {
return
}
return parsePreparedStatementResponse(c, c.Alloc, stream)
}
func (c *Client) LoadPreparedStatementFromResult(result *CreatePreparedStatementResult) (*PreparedStatement, error) {
var (
err error
dsSchema, paramSchema *arrow.Schema
)
if result.DatasetSchema != nil {
dsSchema, err = flight.DeserializeSchema(result.DatasetSchema, c.Alloc)
if err != nil {
return nil, err
}
}
if result.ParameterSchema != nil {
paramSchema, err = flight.DeserializeSchema(result.ParameterSchema, c.Alloc)
if err != nil {
return nil, err
}
}
return &PreparedStatement{
client: c,
handle: result.PreparedStatementHandle,
datasetSchema: dsSchema,
paramSchema: paramSchema,
}, nil
}
func parsePreparedStatementResponse(c *Client, mem memory.Allocator, results pb.FlightService_DoActionClient) (*PreparedStatement, error) {
if err := results.CloseSend(); err != nil {
return nil, err
}
res, err := results.Recv()
if err != nil {
return nil, err
}
var (
container anypb.Any
message pb.ActionCreatePreparedStatementResult
dsSchema, paramSchema *arrow.Schema
)
if err = proto.Unmarshal(res.Body, &container); err != nil {
return nil, err
}
if err = container.UnmarshalTo(&message); err != nil {
return nil, err
}
if message.DatasetSchema != nil {
dsSchema, err = flight.DeserializeSchema(message.DatasetSchema, mem)
if err != nil {
return nil, err
}
}
if message.ParameterSchema != nil {
paramSchema, err = flight.DeserializeSchema(message.ParameterSchema, mem)
if err != nil {
return nil, err
}
}
// XXX: assuming server will not return a result and then an error
// (or else we need to also try to clean up the statement)
if err = flight.ReadUntilEOF(results); err != nil {
return nil, err
}
return &PreparedStatement{
client: c,
handle: message.PreparedStatementHandle,
datasetSchema: dsSchema,
paramSchema: paramSchema,
}, nil
}
func (c *Client) getFlightInfo(ctx context.Context, desc *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
return c.Client.GetFlightInfo(ctx, desc, opts...)
}
func (c *Client) getSchema(ctx context.Context, desc *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
return c.Client.GetSchema(ctx, desc, opts...)
}
// Close will close the underlying flight Client in use by this flightsql.Client
func (c *Client) Close() error { return c.Client.Close() }
// Deprecated: In 13.0.0. Use CancelFlightInfo instead if you can
// assume that server requires 13.0.0 or later. Otherwise, you may
// need to use CancelQuery and/or CancelFlightInfo.
func (c *Client) CancelQuery(ctx context.Context, info *flight.FlightInfo, opts ...grpc.CallOption) (cancelResult CancelResult, err error) {
const actionType = CancelQueryActionType
var (
req pb.ActionCancelQueryRequest
result pb.ActionCancelQueryResult
action pb.Action
stream pb.FlightService_DoActionClient
cmdResult anypb.Any
res *pb.Result
)
if req.Info, err = proto.Marshal(info); err != nil {
return
}
if action, err = packAction(actionType, &req); err != nil {
return
}
if stream, err = c.Client.DoAction(ctx, &action, opts...); err != nil {
return
}
defer stream.CloseSend()
if res, err = stream.Recv(); err != nil {
return
}
if err = flight.ReadUntilEOF(stream); err != nil {
return
}
if err = proto.Unmarshal(res.Body, &cmdResult); err != nil {
return
}
if err = cmdResult.UnmarshalTo(&result); err != nil {
return
}
cancelResult = result.GetResult()
return
}
func (c *Client) CancelFlightInfo(ctx context.Context, request *flight.CancelFlightInfoRequest, opts ...grpc.CallOption) (*flight.CancelFlightInfoResult, error) {
return c.Client.CancelFlightInfo(ctx, request, opts...)
}
func (c *Client) RenewFlightEndpoint(ctx context.Context, request *flight.RenewFlightEndpointRequest, opts ...grpc.CallOption) (*flight.FlightEndpoint, error) {
return c.Client.RenewFlightEndpoint(ctx, request, opts...)
}
func (c *Client) SetSessionOptions(ctx context.Context, request *flight.SetSessionOptionsRequest, opts ...grpc.CallOption) (*flight.SetSessionOptionsResult, error) {
return c.Client.SetSessionOptions(ctx, request, opts...)
}
func (c *Client) GetSessionOptions(ctx context.Context, request *flight.GetSessionOptionsRequest, opts ...grpc.CallOption) (*flight.GetSessionOptionsResult, error) {
return c.Client.GetSessionOptions(ctx, request, opts...)
}
func (c *Client) CloseSession(ctx context.Context, request *flight.CloseSessionRequest, opts ...grpc.CallOption) (*flight.CloseSessionResult, error) {
return c.Client.CloseSession(ctx, request, opts...)
}
func (c *Client) BeginTransaction(ctx context.Context, opts ...grpc.CallOption) (*Txn, error) {
request := &pb.ActionBeginTransactionRequest{}
action, err := packAction(BeginTransactionActionType, request)
if err != nil {
return nil, err
}
stream, err := c.Client.DoAction(ctx, &action, opts...)
if err != nil {
return nil, err
}
if err := stream.CloseSend(); err != nil {
return nil, err
}
var txn pb.ActionBeginTransactionResult
if err = readResult(stream, &txn); err != nil {
return nil, err
}
if err = flight.ReadUntilEOF(stream); err != nil {
return nil, err
}
if len(txn.TransactionId) == 0 {
return nil, ErrBadServerTxn
}
return &Txn{c: c, txn: txn.TransactionId}, nil
}
// Savepoint is a handle for a server-side savepoint
type Savepoint []byte
func (sp Savepoint) IsValid() bool { return len(sp) != 0 }
// Transaction is a handle for a server-side transaction
type Transaction []byte
func (tx Transaction) IsValid() bool { return len(tx) != 0 }
var (
ErrInvalidTxn = fmt.Errorf("%w: missing a valid transaction", arrow.ErrInvalid)
ErrInvalidSavepoint = fmt.Errorf("%w: missing a valid savepoint", arrow.ErrInvalid)
ErrBadServerTxn = fmt.Errorf("%w: server returned an empty transaction ID", arrow.ErrInvalid)
ErrBadServerSavepoint = fmt.Errorf("%w: server returned an empty savepoint ID", arrow.ErrInvalid)
)
type Txn struct {
c *Client
txn Transaction
}
func (tx *Txn) ID() Transaction { return tx.txn }
func (tx *Txn) Execute(ctx context.Context, query string, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
cmd := &pb.CommandStatementQuery{Query: query, TransactionId: tx.txn}
return flightInfoForCommand(ctx, tx.c, cmd, opts...)
}
func (tx *Txn) ExecutePoll(ctx context.Context, query string, retryDescriptor *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.PollInfo, error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
// The server should encode the transaction into the retry descriptor
cmd := &pb.CommandStatementQuery{Query: query, TransactionId: tx.txn}
return pollInfoForCommand(ctx, tx.c, cmd, retryDescriptor, opts...)
}
func (tx *Txn) ExecuteSubstrait(ctx context.Context, plan SubstraitPlan, opts ...grpc.CallOption) (*flight.FlightInfo, error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
cmd := &pb.CommandStatementSubstraitPlan{
Plan: &pb.SubstraitPlan{Plan: plan.Plan, Version: plan.Version},
TransactionId: tx.txn}
return flightInfoForCommand(ctx, tx.c, cmd, opts...)
}
func (tx *Txn) ExecuteSubstraitPoll(ctx context.Context, plan SubstraitPlan, retryDescriptor *flight.FlightDescriptor, opts ...grpc.CallOption) (*flight.PollInfo, error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
// The server should encode the transaction into the retry descriptor
cmd := &pb.CommandStatementSubstraitPlan{
Plan: &pb.SubstraitPlan{Plan: plan.Plan, Version: plan.Version},
TransactionId: tx.txn,
}
return pollInfoForCommand(ctx, tx.c, cmd, retryDescriptor, opts...)
}
func (tx *Txn) GetExecuteSchema(ctx context.Context, query string, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
cmd := &pb.CommandStatementQuery{Query: query, TransactionId: tx.txn}
return schemaForCommand(ctx, tx.c, cmd, opts...)
}
func (tx *Txn) GetExecuteSubstraitSchema(ctx context.Context, plan SubstraitPlan, opts ...grpc.CallOption) (*flight.SchemaResult, error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
cmd := &pb.CommandStatementSubstraitPlan{
Plan: &pb.SubstraitPlan{Plan: plan.Plan, Version: plan.Version},
TransactionId: tx.txn}
return schemaForCommand(ctx, tx.c, cmd, opts...)
}
func (tx *Txn) ExecuteUpdate(ctx context.Context, query string, opts ...grpc.CallOption) (n int64, err error) {
if !tx.txn.IsValid() {
return 0, ErrInvalidTxn
}
var (
cmd = &pb.CommandStatementUpdate{
Query: query,
TransactionId: tx.txn,
}
desc *flight.FlightDescriptor
stream pb.FlightService_DoPutClient
res *pb.PutResult
updateResult pb.DoPutUpdateResult
)
if desc, err = descForCommand(cmd); err != nil {
return
}
if stream, err = tx.c.Client.DoPut(ctx, opts...); err != nil {
return
}
if err = stream.Send(&flight.FlightData{FlightDescriptor: desc}); err != nil {
return
}
if err = stream.CloseSend(); err != nil {
return
}
if res, err = stream.Recv(); err != nil {
return
}
if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil {
return
}
return updateResult.GetRecordCount(), nil
}
func (tx *Txn) ExecuteSubstraitUpdate(ctx context.Context, plan SubstraitPlan, opts ...grpc.CallOption) (n int64, err error) {
if !tx.txn.IsValid() {
return 0, ErrInvalidTxn
}
var (
desc *flight.FlightDescriptor
stream pb.FlightService_DoPutClient
res *pb.PutResult
updateResult pb.DoPutUpdateResult
)
cmd := pb.CommandStatementSubstraitPlan{
Plan: &pb.SubstraitPlan{Plan: plan.Plan, Version: plan.Version},
TransactionId: tx.txn,
}
if desc, err = descForCommand(&cmd); err != nil {
return
}
if stream, err = tx.c.Client.DoPut(ctx, opts...); err != nil {
return
}
if err = stream.Send(&flight.FlightData{FlightDescriptor: desc}); err != nil {
return
}
if err = stream.CloseSend(); err != nil {
return
}
if res, err = stream.Recv(); err != nil {
return
}
if err = proto.Unmarshal(res.GetAppMetadata(), &updateResult); err != nil {
return
}
return updateResult.GetRecordCount(), nil
}
func (tx *Txn) Prepare(ctx context.Context, query string, opts ...grpc.CallOption) (prep *PreparedStatement, err error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
const actionType = CreatePreparedStatementActionType
var (
request = pb.ActionCreatePreparedStatementRequest{
Query: query,
TransactionId: tx.txn,
}
action pb.Action
stream pb.FlightService_DoActionClient
)
if action, err = packAction(actionType, &request); err != nil {
return
}
if stream, err = tx.c.Client.DoAction(ctx, &action, opts...); err != nil {
return
}
return parsePreparedStatementResponse(tx.c, tx.c.Alloc, stream)
}
func (tx *Txn) PrepareSubstrait(ctx context.Context, plan SubstraitPlan, opts ...grpc.CallOption) (stmt *PreparedStatement, err error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
const actionType = CreatePreparedSubstraitPlanActionType
var (
request = pb.ActionCreatePreparedSubstraitPlanRequest{
TransactionId: tx.txn,
Plan: &pb.SubstraitPlan{
Plan: plan.Plan,
Version: plan.Version,
},
}
action pb.Action
stream pb.FlightService_DoActionClient
)
if action, err = packAction(actionType, &request); err != nil {
return
}
if stream, err = tx.c.Client.DoAction(ctx, &action, opts...); err != nil {
return
}
return parsePreparedStatementResponse(tx.c, tx.c.Alloc, stream)
}
func (tx *Txn) Commit(ctx context.Context, opts ...grpc.CallOption) error {
if !tx.txn.IsValid() {
return ErrInvalidTxn
}
request := &pb.ActionEndTransactionRequest{
TransactionId: tx.txn,
Action: EndTransactionCommit,
}
action, err := packAction(EndTransactionActionType, request)
if err != nil {
return err
}
stream, err := tx.c.Client.DoAction(ctx, &action, opts...)
if err != nil {
return err
}
if err := stream.CloseSend(); err != nil {
return err
}
tx.txn = nil
return flight.ReadUntilEOF(stream)
}
func (tx *Txn) Rollback(ctx context.Context, opts ...grpc.CallOption) error {
if !tx.txn.IsValid() {
return ErrInvalidTxn
}
request := &pb.ActionEndTransactionRequest{
TransactionId: tx.txn,
Action: EndTransactionRollback,
}
action, err := packAction(EndTransactionActionType, request)
if err != nil {
return err
}
stream, err := tx.c.Client.DoAction(ctx, &action, opts...)
if err != nil {
return err
}
if err := stream.CloseSend(); err != nil {
return err
}
tx.txn = nil
return flight.ReadUntilEOF(stream)
}
func (tx *Txn) BeginSavepoint(ctx context.Context, name string, opts ...grpc.CallOption) (Savepoint, error) {
if !tx.txn.IsValid() {
return nil, ErrInvalidTxn
}
request := &pb.ActionBeginSavepointRequest{
TransactionId: tx.txn,
Name: name,
}
action, err := packAction(BeginSavepointActionType, request)
if err != nil {
return nil, err
}
stream, err := tx.c.Client.DoAction(ctx, &action, opts...)
if err != nil {
return nil, err
}
if err := stream.CloseSend(); err != nil {
return nil, err
}
var savepoint pb.ActionBeginSavepointResult
if err = readResult(stream, &savepoint); err != nil {
return nil, err
}
if err = flight.ReadUntilEOF(stream); err != nil {
return nil, err
}
if len(savepoint.SavepointId) == 0 {
return nil, ErrBadServerSavepoint
}
return Savepoint(savepoint.SavepointId), nil
}
func (tx *Txn) ReleaseSavepoint(ctx context.Context, sp Savepoint, opts ...grpc.CallOption) error {
if !sp.IsValid() {
return ErrInvalidSavepoint
}
request := &pb.ActionEndSavepointRequest{
SavepointId: sp,
Action: EndSavepointRelease,
}
action, err := packAction(EndSavepointActionType, request)
if err != nil {
return err
}
stream, err := tx.c.Client.DoAction(ctx, &action, opts...)
if err != nil {
return err
}
if err := stream.CloseSend(); err != nil {
return err
}
return flight.ReadUntilEOF(stream)
}
func (tx *Txn) RollbackSavepoint(ctx context.Context, sp Savepoint, opts ...grpc.CallOption) error {
if !sp.IsValid() {
return ErrInvalidSavepoint
}
request := &pb.ActionEndSavepointRequest{
SavepointId: sp,
Action: EndSavepointRollback,
}
action, err := packAction(EndSavepointActionType, request)
if err != nil {
return err
}
stream, err := tx.c.Client.DoAction(ctx, &action, opts...)
if err != nil {
return err