File: server.go

package info (click to toggle)
golang-github-apache-arrow-go 18.2.0-1
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid, trixie
  • size: 32,200 kB
  • sloc: asm: 477,547; ansic: 5,369; cpp: 759; sh: 585; makefile: 319; python: 190; sed: 5
file content (1400 lines) | stat: -rw-r--r-- 55,784 bytes parent folder | download
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
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
// 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"
	"fmt"

	"github.com/apache/arrow-go/v18/arrow"
	"github.com/apache/arrow-go/v18/arrow/array"
	"github.com/apache/arrow-go/v18/arrow/flight"
	"github.com/apache/arrow-go/v18/arrow/flight/flightsql/schema_ref"
	pb "github.com/apache/arrow-go/v18/arrow/flight/gen/flight"
	"github.com/apache/arrow-go/v18/arrow/internal/debug"
	"github.com/apache/arrow-go/v18/arrow/ipc"
	"github.com/apache/arrow-go/v18/arrow/memory"
	"google.golang.org/grpc/codes"
	"google.golang.org/grpc/status"
	"google.golang.org/protobuf/proto"
	"google.golang.org/protobuf/types/known/anypb"
)

// the following interfaces wrap the Protobuf commands to avoid
// exposing the Protobuf types themselves in the API.

// StatementQuery represents a Sql Query
type StatementQuery interface {
	GetQuery() string
	GetTransactionId() []byte
}

type statementSubstraitPlan struct {
	*pb.CommandStatementSubstraitPlan
}

func (s *statementSubstraitPlan) GetPlan() SubstraitPlan {
	var (
		plan    []byte
		version string
	)
	if s.Plan != nil {
		plan = s.Plan.Plan
		version = s.Plan.Version
	}
	return SubstraitPlan{
		Plan:    plan,
		Version: version,
	}
}

type StatementSubstraitPlan interface {
	GetTransactionId() []byte
	GetPlan() SubstraitPlan
}

// StatementUpdate represents a SQL update query
type StatementUpdate interface {
	GetQuery() string
	GetTransactionId() []byte
}

// StatementQueryTicket represents a request to execute a query
type StatementQueryTicket interface {
	// GetStatementHandle returns the server-generated opaque
	// identifier for the query
	GetStatementHandle() []byte
}

func GetStatementQueryTicket(ticket *flight.Ticket) (result StatementQueryTicket, err error) {
	var anycmd anypb.Any
	if err = proto.Unmarshal(ticket.Ticket, &anycmd); err != nil {
		return
	}

	var out pb.TicketStatementQuery
	if err = anycmd.UnmarshalTo(&out); err != nil {
		return
	}

	result = &out
	return
}

// PreparedStatementQuery represents a prepared query statement
type PreparedStatementQuery interface {
	// GetPreparedStatementHandle returns the server-generated opaque
	// identifier for the statement
	GetPreparedStatementHandle() []byte
}

// PreparedStatementUpdate represents a prepared update statement
type PreparedStatementUpdate interface {
	// GetPreparedStatementHandle returns the server-generated opaque
	// identifier for the statement
	GetPreparedStatementHandle() []byte
}

// ActionClosePreparedStatementRequest represents a request to close
// a prepared statement
type ActionClosePreparedStatementRequest interface {
	// GetPreparedStatementHandle returns the server-generated opaque
	// identifier for the statement
	GetPreparedStatementHandle() []byte
}

// ActionCreatePreparedStatementRequest represents a request to construct
// a new prepared statement
type ActionCreatePreparedStatementRequest interface {
	GetQuery() string
	GetTransactionId() []byte
}

type ActionCreatePreparedSubstraitPlanRequest interface {
	GetPlan() SubstraitPlan
	GetTransactionId() []byte
}

type createPreparedSubstraitPlanReq struct {
	*pb.ActionCreatePreparedSubstraitPlanRequest
}

func (c *createPreparedSubstraitPlanReq) GetPlan() SubstraitPlan {
	var (
		plan    []byte
		version string
	)
	if c.Plan != nil {
		plan = c.Plan.Plan
		version = c.Plan.Version
	}
	return SubstraitPlan{
		Plan:    plan,
		Version: version,
	}
}

// ActionCreatePreparedStatementResult is the result of creating a new
// prepared statement, optionally including the dataset and parameter
// schemas.
type ActionCreatePreparedStatementResult struct {
	Handle          []byte
	DatasetSchema   *arrow.Schema
	ParameterSchema *arrow.Schema
}

type ActionBeginTransactionRequest interface{}

type ActionBeginSavepointRequest interface {
	GetTransactionId() []byte
	GetName() string
}

type ActionBeginSavepointResult interface {
	GetSavepointId() []byte
}

type ActionBeginTransactionResult interface {
	GetTransactionId() []byte
}

type ActionCancelQueryRequest interface {
	GetInfo() *flight.FlightInfo
}

type cancelQueryRequest struct {
	info *flight.FlightInfo
}

func (c *cancelQueryRequest) GetInfo() *flight.FlightInfo { return c.info }

type cancelQueryServer interface {
	CancelQuery(context.Context, ActionCancelQueryRequest) (CancelResult, error)
}

type ActionEndTransactionRequest interface {
	GetTransactionId() []byte
	GetAction() EndTransactionRequestType
}

type ActionEndSavepointRequest interface {
	GetSavepointId() []byte
	GetAction() EndSavepointRequestType
}

// StatementIngest represents a bulk ingestion request
type StatementIngest interface {
	GetTableDefinitionOptions() *TableDefinitionOptions
	GetTable() string
	GetSchema() string
	GetCatalog() string
	GetTemporary() bool
	GetTransactionId() []byte
	GetOptions() map[string]string
}

type getXdbcTypeInfo struct {
	*pb.CommandGetXdbcTypeInfo
}

func (c *getXdbcTypeInfo) GetDataType() *int32 { return c.DataType }

// GetXdbcTypeInfo represents a request for SQL Data Type information
type GetXdbcTypeInfo interface {
	// GetDataType returns either nil (get for all types)
	// or a specific SQL type ID to fetch information about.
	GetDataType() *int32
}

// GetSqlInfo represents a request for SQL Information
type GetSqlInfo interface {
	// GetInfo returns a slice of SqlInfo ids to return information about
	GetInfo() []uint32
}

type getDBSchemas struct {
	*pb.CommandGetDbSchemas
}

func (c *getDBSchemas) GetCatalog() *string               { return c.Catalog }
func (c *getDBSchemas) GetDBSchemaFilterPattern() *string { return c.DbSchemaFilterPattern }

// GetDBSchemas represents a request for list of database schemas
type GetDBSchemas interface {
	GetCatalog() *string
	GetDBSchemaFilterPattern() *string
}

type getTables struct {
	*pb.CommandGetTables
}

func (c *getTables) GetCatalog() *string                { return c.Catalog }
func (c *getTables) GetDBSchemaFilterPattern() *string  { return c.DbSchemaFilterPattern }
func (c *getTables) GetTableNameFilterPattern() *string { return c.TableNameFilterPattern }

// GetTables represents a request to list the database's tables
type GetTables interface {
	GetCatalog() *string
	GetDBSchemaFilterPattern() *string
	GetTableNameFilterPattern() *string
	GetTableTypes() []string
	GetIncludeSchema() bool
}

func packActionResult(msg proto.Message) (*pb.Result, error) {
	var (
		anycmd anypb.Any
		err    error
	)

	if err = anycmd.MarshalFrom(msg); err != nil {
		return nil, fmt.Errorf("%w: unable to marshal final response", err)
	}

	ret := &pb.Result{}
	if ret.Body, err = proto.Marshal(&anycmd); err != nil {
		return nil, fmt.Errorf("%w: unable to marshal final response", err)
	}
	return ret, nil
}

// BaseServer must be embedded into any FlightSQL Server implementation
// and provides default implementations of all methods returning an
// unimplemented error if called. This allows consumers to gradually
// implement methods as they want instead of requiring all consumers to
// boilerplate the same "unimplemented" methods.
//
// The base implementation also contains handling for registering sql info
// and serving it up in response to GetSqlInfo requests.
type BaseServer struct {
	sqlInfoToResult SqlInfoResultMap
	// Alloc allows specifying a particular allocator to use for any
	// allocations done by the base implementation.
	// Will use memory.DefaultAllocator if nil
	Alloc memory.Allocator
}

func (BaseServer) mustEmbedBaseServer() {}

// RegisterSqlInfo registers a specific result to return for a given sqlinfo
// id. The result must be one of the following types: string, bool, int64,
// int32, []string, or map[int32][]int32.
//
// Once registered, this value will be returned for any SqlInfo requests.
func (b *BaseServer) RegisterSqlInfo(id SqlInfo, result interface{}) error {
	if b.sqlInfoToResult == nil {
		b.sqlInfoToResult = make(SqlInfoResultMap)
	}

	switch result.(type) {
	case string, bool, int64, int32, []string, map[int32][]int32:
		b.sqlInfoToResult[uint32(id)] = result
	default:
		return fmt.Errorf("invalid sql info type '%T' registered for id: %d", result, id)
	}
	return nil
}

func (BaseServer) GetFlightInfoStatement(context.Context, StatementQuery, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoStatement not implemented")
}

func (BaseServer) GetFlightInfoSubstraitPlan(context.Context, StatementSubstraitPlan, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoSubstraitPlan not implemented")
}

func (BaseServer) GetSchemaStatement(context.Context, StatementQuery, *flight.FlightDescriptor) (*flight.SchemaResult, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetSchemaStatement not implemented")
}

func (BaseServer) GetSchemaSubstraitPlan(context.Context, StatementSubstraitPlan, *flight.FlightDescriptor) (*flight.SchemaResult, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetSchemaSubstraitPlan not implemented")
}

func (BaseServer) DoGetStatement(context.Context, StatementQueryTicket) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetStatement not implemented")
}

func (BaseServer) GetFlightInfoPreparedStatement(context.Context, PreparedStatementQuery, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoPreparedStatement not implemented")
}

func (BaseServer) GetSchemaPreparedStatement(context.Context, PreparedStatementQuery, *flight.FlightDescriptor) (*flight.SchemaResult, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetSchemaPreparedStatement not implemented")
}

func (BaseServer) DoGetPreparedStatement(context.Context, PreparedStatementQuery) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetPreparedStatement not implemented")
}

func (BaseServer) GetFlightInfoCatalogs(context.Context, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoCatalogs not implemented")
}

func (BaseServer) DoGetCatalogs(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetCatalogs not implemented")
}

func (BaseServer) GetFlightInfoXdbcTypeInfo(context.Context, GetXdbcTypeInfo, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoXdbcTypeInfo not implemented")
}

func (BaseServer) DoGetXdbcTypeInfo(context.Context, GetXdbcTypeInfo) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetXdbcTypeInfo not implemented")
}

// GetFlightInfoSqlInfo is a base implementation of GetSqlInfo by using any
// registered sqlinfo (by calling RegisterSqlInfo). Will return an error
// if there is no sql info registered, otherwise a FlightInfo for retrieving
// the Sql info.
func (b *BaseServer) GetFlightInfoSqlInfo(_ context.Context, _ GetSqlInfo, desc *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	if len(b.sqlInfoToResult) == 0 {
		return nil, status.Error(codes.NotFound, "no sql information available")
	}

	if b.Alloc == nil {
		b.Alloc = memory.DefaultAllocator
	}

	return &flight.FlightInfo{
		Endpoint:         []*flight.FlightEndpoint{{Ticket: &flight.Ticket{Ticket: desc.Cmd}}},
		FlightDescriptor: desc,
		TotalRecords:     -1,
		TotalBytes:       -1,
		Schema:           flight.SerializeSchema(schema_ref.SqlInfo, b.Alloc),
	}, nil
}

// DoGetSqlInfo returns a flight stream containing the list of sqlinfo results
func (b *BaseServer) DoGetSqlInfo(_ context.Context, cmd GetSqlInfo) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	if b.Alloc == nil {
		b.Alloc = memory.DefaultAllocator
	}

	bldr := array.NewRecordBuilder(b.Alloc, schema_ref.SqlInfo)
	defer bldr.Release()

	nameFieldBldr := bldr.Field(0).(*array.Uint32Builder)
	valFieldBldr := bldr.Field(1).(*array.DenseUnionBuilder)

	// doesn't take ownership, no calls to retain. so we don't need
	// extra releases.
	sqlInfoResultBldr := newSqlInfoResultBuilder(valFieldBldr)

	keys := cmd.GetInfo()

	// populate both the nameFieldBldr and the values for each
	// element on command.info.
	// valueFieldBldr is populated depending on the data type
	// since it's a dense union. The population for each
	// data type is handled by the sqlInfoResultBuilder.
	if len(keys) > 0 {
		for _, info := range keys {
			val, ok := b.sqlInfoToResult[info]
			if !ok {
				return nil, nil, status.Errorf(codes.NotFound, "no information for sql info number %d", info)
			}
			nameFieldBldr.Append(info)
			sqlInfoResultBldr.Append(val)
		}
	} else {
		for k, v := range b.sqlInfoToResult {
			nameFieldBldr.Append(k)
			sqlInfoResultBldr.Append(v)
		}
	}

	batch := bldr.NewRecord()
	defer batch.Release()
	debug.Assert(int(batch.NumRows()) == len(cmd.GetInfo()), "too many rows added to SqlInfo result")

	ch := make(chan flight.StreamChunk)
	rdr, err := array.NewRecordReader(schema_ref.SqlInfo, []arrow.Record{batch})
	if err != nil {
		return nil, nil, status.Errorf(codes.Internal, "error producing record response: %s", err.Error())
	}

	// StreamChunksFromReader will call release on the reader when done
	go flight.StreamChunksFromReader(rdr, ch)
	return schema_ref.SqlInfo, ch, nil
}

func (BaseServer) GetFlightInfoSchemas(context.Context, GetDBSchemas, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoSchemas not implemented")
}

func (BaseServer) DoGetDBSchemas(context.Context, GetDBSchemas) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetDBSchemas not implemented")
}

func (BaseServer) GetFlightInfoTables(context.Context, GetTables, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoTables not implemented")
}

func (BaseServer) DoGetTables(context.Context, GetTables) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetTables not implemented")
}

func (BaseServer) GetFlightInfoTableTypes(context.Context, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Errorf(codes.Unimplemented, "GetFlightInfoTableTypes not implemented")
}

func (BaseServer) DoGetTableTypes(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetTableTypes not implemented")
}

func (BaseServer) GetFlightInfoPrimaryKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Error(codes.Unimplemented, "GetFlightInfoPrimaryKeys not implemented")
}

func (BaseServer) DoGetPrimaryKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetPrimaryKeys not implemented")
}

func (BaseServer) GetFlightInfoExportedKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Error(codes.Unimplemented, "GetFlightInfoExportedKeys not implemented")
}

func (BaseServer) DoGetExportedKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetExportedKeys not implemented")
}

func (BaseServer) GetFlightInfoImportedKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Error(codes.Unimplemented, "GetFlightInfoImportedKeys not implemented")
}

func (BaseServer) DoGetImportedKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetImportedKeys not implemented")
}

func (BaseServer) GetFlightInfoCrossReference(context.Context, CrossTableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	return nil, status.Error(codes.Unimplemented, "GetFlightInfoCrossReference not implemented")
}

func (BaseServer) DoGetCrossReference(context.Context, CrossTableRef) (*arrow.Schema, <-chan flight.StreamChunk, error) {
	return nil, nil, status.Errorf(codes.Unimplemented, "DoGetCrossReference not implemented")
}

func (BaseServer) CreatePreparedStatement(context.Context, ActionCreatePreparedStatementRequest) (res ActionCreatePreparedStatementResult, err error) {
	return res, status.Error(codes.Unimplemented, "CreatePreparedStatement not implemented")
}

func (BaseServer) CreatePreparedSubstraitPlan(context.Context, ActionCreatePreparedSubstraitPlanRequest) (res ActionCreatePreparedStatementResult, err error) {
	return res, status.Error(codes.Unimplemented, "CreatePreparedSubstraitPlan not implemented")
}

func (BaseServer) ClosePreparedStatement(context.Context, ActionClosePreparedStatementRequest) error {
	return status.Error(codes.Unimplemented, "ClosePreparedStatement not implemented")
}

func (BaseServer) DoPutCommandStatementUpdate(context.Context, StatementUpdate) (int64, error) {
	return 0, status.Error(codes.Unimplemented, "DoPutCommandStatementUpdate not implemented")
}

func (BaseServer) DoPutCommandSubstraitPlan(context.Context, StatementSubstraitPlan) (int64, error) {
	return 0, status.Error(codes.Unimplemented, "DoPutCommandSubstraitPlan not implemented")
}

func (BaseServer) DoPutPreparedStatementQuery(context.Context, PreparedStatementQuery, flight.MessageReader, flight.MetadataWriter) ([]byte, error) {
	return nil, status.Error(codes.Unimplemented, "DoPutPreparedStatementQuery not implemented")
}

func (BaseServer) DoPutPreparedStatementUpdate(context.Context, PreparedStatementUpdate, flight.MessageReader) (int64, error) {
	return 0, status.Error(codes.Unimplemented, "DoPutPreparedStatementUpdate not implemented")
}

func (BaseServer) DoPutCommandStatementIngest(context.Context, StatementIngest, flight.MessageReader) (int64, error) {
	return 0, status.Error(codes.Unimplemented, "DoPutCommandStatementIngest not implemented")
}

func (BaseServer) BeginTransaction(context.Context, ActionBeginTransactionRequest) ([]byte, error) {
	return nil, status.Error(codes.Unimplemented, "BeginTransaction not implemented")
}

func (BaseServer) BeginSavepoint(context.Context, ActionBeginSavepointRequest) ([]byte, error) {
	return nil, status.Error(codes.Unimplemented, "BeginSavepoint not implemented")
}

func (BaseServer) CancelFlightInfo(context.Context, *flight.CancelFlightInfoRequest) (flight.CancelFlightInfoResult, error) {
	return flight.CancelFlightInfoResult{Status: flight.CancelStatusUnspecified},
		status.Error(codes.Unimplemented, "CancelFlightInfo not implemented")
}

func (BaseServer) RenewFlightEndpoint(context.Context, *flight.RenewFlightEndpointRequest) (*flight.FlightEndpoint, error) {
	return nil, status.Error(codes.Unimplemented, "RenewFlightEndpoint not implemented")
}

func (BaseServer) PollFlightInfo(context.Context, *flight.FlightDescriptor) (*flight.PollInfo, error) {
	return nil, status.Error(codes.Unimplemented, "PollFlightInfo not implemented")
}

func (BaseServer) PollFlightInfoStatement(context.Context, StatementQuery, *flight.FlightDescriptor) (*flight.PollInfo, error) {
	return nil, status.Error(codes.Unimplemented, "PollFlightInfoStatement not implemented")
}

func (BaseServer) PollFlightInfoSubstraitPlan(context.Context, StatementSubstraitPlan, *flight.FlightDescriptor) (*flight.PollInfo, error) {
	return nil, status.Error(codes.Unimplemented, "PollFlightInfoSubstraitPlan not implemented")
}

func (BaseServer) PollFlightInfoPreparedStatement(context.Context, PreparedStatementQuery, *flight.FlightDescriptor) (*flight.PollInfo, error) {
	return nil, status.Error(codes.Unimplemented, "PollFlightInfoPreparedStatement not implemented")
}

func (BaseServer) EndTransaction(context.Context, ActionEndTransactionRequest) error {
	return status.Error(codes.Unimplemented, "EndTransaction not implemented")
}

func (BaseServer) EndSavepoint(context.Context, ActionEndSavepointRequest) error {
	return status.Error(codes.Unimplemented, "EndSavepoint not implemented")
}

func (BaseServer) SetSessionOptions(context.Context, *flight.SetSessionOptionsRequest) (*flight.SetSessionOptionsResult, error) {
	return nil, status.Error(codes.Unimplemented, "SetSessionOptions not implemented")
}

func (BaseServer) GetSessionOptions(context.Context, *flight.GetSessionOptionsRequest) (*flight.GetSessionOptionsResult, error) {
	return nil, status.Error(codes.Unimplemented, "GetSessionOptions not implemented")
}

func (BaseServer) CloseSession(context.Context, *flight.CloseSessionRequest) (*flight.CloseSessionResult, error) {
	return nil, status.Error(codes.Unimplemented, "CloseSession not implemented")
}

// Server is the required interface for a FlightSQL server. It is implemented by
// BaseServer which must be embedded in any implementation. The default
// implementation by BaseServer for each of these (except GetSqlInfo)
//
// GetFlightInfo* methods should return the FlightInfo object representing where
// to retrieve the results for a given request.
//
// DoGet* methods should return the Schema of the resulting stream along with
// a channel to retrieve stream chunks (each chunk is a record batch and optionally
// a descriptor and app metadata). The channel will be read from until it
// closes, sending each chunk on the stream. Since the channel is returned
// from the method, it should be populated within a goroutine to ensure
// there are no deadlocks.
type Server interface {
	// GetFlightInfoStatement returns a FlightInfo for executing the requested sql query
	GetFlightInfoStatement(context.Context, StatementQuery, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// GetFlightInfoSubstraitPlan returns a FlightInfo for executing the requested substrait plan
	GetFlightInfoSubstraitPlan(context.Context, StatementSubstraitPlan, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// GetSchemaStatement returns the schema of the result set of the requested sql query
	GetSchemaStatement(context.Context, StatementQuery, *flight.FlightDescriptor) (*flight.SchemaResult, error)
	// GetSchemaSubstraitPlan returns the schema of the result set for the requested substrait plan
	GetSchemaSubstraitPlan(context.Context, StatementSubstraitPlan, *flight.FlightDescriptor) (*flight.SchemaResult, error)
	// DoGetStatement returns a stream containing the query results for the
	// requested statement handle that was populated by GetFlightInfoStatement
	DoGetStatement(context.Context, StatementQueryTicket) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoPreparedStatement returns a FlightInfo for executing an already
	// prepared statement with the provided statement handle.
	GetFlightInfoPreparedStatement(context.Context, PreparedStatementQuery, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// GetSchemaPreparedStatement returns the schema of the result set of executing an already
	// prepared statement with the provided statement handle.
	GetSchemaPreparedStatement(context.Context, PreparedStatementQuery, *flight.FlightDescriptor) (*flight.SchemaResult, error)
	// DoGetPreparedStatement returns a stream containing the results from executing
	// a prepared statement query with the provided statement handle.
	DoGetPreparedStatement(context.Context, PreparedStatementQuery) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoCatalogs returns a FlightInfo for the listing of all catalogs
	GetFlightInfoCatalogs(context.Context, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetCatalogs returns the stream containing the list of catalogs
	DoGetCatalogs(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoXdbcTypeInfo returns a FlightInfo for retrieving data type info
	GetFlightInfoXdbcTypeInfo(context.Context, GetXdbcTypeInfo, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetXdbcTypeInfo returns a stream containing the information about the
	// requested supported datatypes
	DoGetXdbcTypeInfo(context.Context, GetXdbcTypeInfo) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoSqlInfo returns a FlightInfo for retrieving SqlInfo from the server
	GetFlightInfoSqlInfo(context.Context, GetSqlInfo, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetSqlInfo returns a stream containing the list of SqlInfo results
	DoGetSqlInfo(context.Context, GetSqlInfo) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoSchemas returns a FlightInfo for requesting a list of schemas
	GetFlightInfoSchemas(context.Context, GetDBSchemas, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetDBSchemas returns a stream containing the list of schemas
	DoGetDBSchemas(context.Context, GetDBSchemas) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoTables returns a FlightInfo for listing the tables available
	GetFlightInfoTables(context.Context, GetTables, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetTables returns a stream containing the list of tables
	DoGetTables(context.Context, GetTables) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoTableTypes returns a FlightInfo for retrieving a list
	// of table types supported
	GetFlightInfoTableTypes(context.Context, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetTableTypes returns a stream containing the data related to the table types
	DoGetTableTypes(context.Context) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoPrimaryKeys returns a FlightInfo for extracting information about primary keys
	GetFlightInfoPrimaryKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetPrimaryKeys returns a stream containing the data related to primary keys
	DoGetPrimaryKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoExportedKeys returns a FlightInfo for extracting information about foreign keys
	GetFlightInfoExportedKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetExportedKeys returns a stream containing the data related to foreign keys
	DoGetExportedKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoImportedKeys returns a FlightInfo for extracting information about imported keys
	GetFlightInfoImportedKeys(context.Context, TableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetImportedKeys returns a stream containing the data related to imported keys
	DoGetImportedKeys(context.Context, TableRef) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// GetFlightInfoCrossReference returns a FlightInfo for extracting data related
	// to primary and foreign keys
	GetFlightInfoCrossReference(context.Context, CrossTableRef, *flight.FlightDescriptor) (*flight.FlightInfo, error)
	// DoGetCrossReference returns a stream of data related to foreign and primary keys
	DoGetCrossReference(context.Context, CrossTableRef) (*arrow.Schema, <-chan flight.StreamChunk, error)
	// DoPutCommandStatementUpdate executes a sql update statement and returns
	// the number of affected rows
	DoPutCommandStatementUpdate(context.Context, StatementUpdate) (int64, error)
	// DoPutCommandSubstraitPlan executes a substrait plan and returns the number
	// of affected rows.
	DoPutCommandSubstraitPlan(context.Context, StatementSubstraitPlan) (int64, error)
	// CreatePreparedStatement constructs a prepared statement from a sql query
	// and returns an opaque statement handle for use.
	CreatePreparedStatement(context.Context, ActionCreatePreparedStatementRequest) (ActionCreatePreparedStatementResult, error)
	// CreatePreparedSubstraitPlan constructs a prepared statement from a substrait
	// plan, and returns an opaque statement handle for use.
	CreatePreparedSubstraitPlan(context.Context, ActionCreatePreparedSubstraitPlanRequest) (ActionCreatePreparedStatementResult, error)
	// ClosePreparedStatement closes the prepared statement identified by the requested
	// opaque statement handle.
	ClosePreparedStatement(context.Context, ActionClosePreparedStatementRequest) error
	// DoPutPreparedStatementQuery binds parameters to a given prepared statement
	// identified by the provided statement handle.
	//
	// The provided MessageReader is a stream of record batches with optional
	// app metadata and flight descriptors to represent the values to bind
	// to the parameters.
	//
	// Currently anything written to the writer will be ignored. It is in the
	// interface for potential future enhancements to avoid having to change
	// the interface in the future.
	DoPutPreparedStatementQuery(context.Context, PreparedStatementQuery, flight.MessageReader, flight.MetadataWriter) ([]byte, error)
	// DoPutPreparedStatementUpdate executes an update SQL Prepared statement
	// for the specified statement handle. The reader allows providing a sequence
	// of uploaded record batches to bind the parameters to. Returns the number
	// of affected records.
	DoPutPreparedStatementUpdate(context.Context, PreparedStatementUpdate, flight.MessageReader) (int64, error)
	// BeginTransaction starts a new transaction and returns the id
	BeginTransaction(context.Context, ActionBeginTransactionRequest) (id []byte, err error)
	// BeginSavepoint initializes a new savepoint and returns the id
	BeginSavepoint(context.Context, ActionBeginSavepointRequest) (id []byte, err error)
	// EndSavepoint releases or rolls back a savepoint
	EndSavepoint(context.Context, ActionEndSavepointRequest) error
	// EndTransaction commits or rolls back a transaction
	EndTransaction(context.Context, ActionEndTransactionRequest) error
	// CancelFlightInfo attempts to explicitly cancel a FlightInfo
	CancelFlightInfo(context.Context, *flight.CancelFlightInfoRequest) (flight.CancelFlightInfoResult, error)
	// RenewFlightEndpoint attempts to extend the expiration of a FlightEndpoint
	RenewFlightEndpoint(context.Context, *flight.RenewFlightEndpointRequest) (*flight.FlightEndpoint, error)
	// PollFlightInfo is a generic handler for PollFlightInfo requests.
	PollFlightInfo(context.Context, *flight.FlightDescriptor) (*flight.PollInfo, error)
	// PollFlightInfoStatement handles polling for query execution.
	PollFlightInfoStatement(context.Context, StatementQuery, *flight.FlightDescriptor) (*flight.PollInfo, error)
	// PollFlightInfoSubstraitPlan handles polling for query execution.
	PollFlightInfoSubstraitPlan(context.Context, StatementSubstraitPlan, *flight.FlightDescriptor) (*flight.PollInfo, error)
	// PollFlightInfoPreparedStatement handles polling for query execution.
	PollFlightInfoPreparedStatement(context.Context, PreparedStatementQuery, *flight.FlightDescriptor) (*flight.PollInfo, error)
	// SetSessionOptions sets option(s) for the current server session.
	SetSessionOptions(context.Context, *flight.SetSessionOptionsRequest) (*flight.SetSessionOptionsResult, error)
	// GetSessionOptions gets option(s) for the current server session.
	GetSessionOptions(context.Context, *flight.GetSessionOptionsRequest) (*flight.GetSessionOptionsResult, error)
	// CloseSession closes/invalidates the current server session.
	CloseSession(context.Context, *flight.CloseSessionRequest) (*flight.CloseSessionResult, error)
	// DoPutCommandStatementIngest executes a bulk ingestion and returns
	// the number of affected rows
	DoPutCommandStatementIngest(context.Context, StatementIngest, flight.MessageReader) (int64, error)

	mustEmbedBaseServer()
}

// NewFlightServer constructs a FlightRPC server from the provided
// FlightSQL Server so that it can be passed to RegisterFlightService.
func NewFlightServer(srv Server) flight.FlightServer {
	return &flightSqlServer{srv: srv, mem: memory.DefaultAllocator}
}

// NewFlightServerWithAllocator constructs a FlightRPC server from
// the provided FlightSQL Server so that it can be passed to
// RegisterFlightService, setting the provided allocator into the server
// for use with any allocations necessary by the routing.
//
// Will default to memory.DefaultAllocator if mem is nil
func NewFlightServerWithAllocator(srv Server, mem memory.Allocator) flight.FlightServer {
	if mem == nil {
		mem = memory.DefaultAllocator
	}
	return &flightSqlServer{srv: srv, mem: mem}
}

// flightSqlServer is a wrapper around a FlightSQL server interface to
// perform routing from FlightRPC to FlightSQL.
type flightSqlServer struct {
	flight.BaseFlightServer
	mem memory.Allocator
	srv Server
}

func (f *flightSqlServer) GetFlightInfo(ctx context.Context, request *flight.FlightDescriptor) (*flight.FlightInfo, error) {
	var (
		anycmd anypb.Any
		cmd    proto.Message
		err    error
	)
	if err = proto.Unmarshal(request.Cmd, &anycmd); err != nil {
		return nil, status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
	}

	if cmd, err = anycmd.UnmarshalNew(); err != nil {
		return nil, status.Errorf(codes.InvalidArgument, "could not unmarshal Any to a command type: %s", err.Error())
	}

	switch cmd := cmd.(type) {
	case *pb.CommandStatementQuery:
		return f.srv.GetFlightInfoStatement(ctx, cmd, request)
	case *pb.CommandStatementSubstraitPlan:
		return f.srv.GetFlightInfoSubstraitPlan(ctx, &statementSubstraitPlan{cmd}, request)
	case *pb.CommandPreparedStatementQuery:
		return f.srv.GetFlightInfoPreparedStatement(ctx, cmd, request)
	case *pb.CommandGetCatalogs:
		return f.srv.GetFlightInfoCatalogs(ctx, request)
	case *pb.CommandGetDbSchemas:
		return f.srv.GetFlightInfoSchemas(ctx, &getDBSchemas{cmd}, request)
	case *pb.CommandGetTables:
		return f.srv.GetFlightInfoTables(ctx, &getTables{cmd}, request)
	case *pb.CommandGetTableTypes:
		return f.srv.GetFlightInfoTableTypes(ctx, request)
	case *pb.CommandGetXdbcTypeInfo:
		return f.srv.GetFlightInfoXdbcTypeInfo(ctx, &getXdbcTypeInfo{cmd}, request)
	case *pb.CommandGetSqlInfo:
		return f.srv.GetFlightInfoSqlInfo(ctx, cmd, request)
	case *pb.CommandGetPrimaryKeys:
		return f.srv.GetFlightInfoPrimaryKeys(ctx, pkToTableRef(cmd), request)
	case *pb.CommandGetExportedKeys:
		return f.srv.GetFlightInfoExportedKeys(ctx, exkToTableRef(cmd), request)
	case *pb.CommandGetImportedKeys:
		return f.srv.GetFlightInfoImportedKeys(ctx, impkToTableRef(cmd), request)
	case *pb.CommandGetCrossReference:
		return f.srv.GetFlightInfoCrossReference(ctx, toCrossTableRef(cmd), request)
	}

	return nil, status.Error(codes.InvalidArgument, "requested command is invalid")
}

func (f *flightSqlServer) PollFlightInfo(ctx context.Context, request *flight.FlightDescriptor) (*flight.PollInfo, error) {
	var (
		anycmd anypb.Any
		cmd    proto.Message
		err    error
	)
	// If we can't parse things, be friendly and defer to the server
	// implementation. This is especially important for this method since
	// the server returns a custom FlightDescriptor for future requests.
	if err = proto.Unmarshal(request.Cmd, &anycmd); err != nil {
		return f.srv.PollFlightInfo(ctx, request)
	}

	if cmd, err = anycmd.UnmarshalNew(); err != nil {
		return f.srv.PollFlightInfo(ctx, request)
	}

	switch cmd := cmd.(type) {
	case *pb.CommandStatementQuery:
		return f.srv.PollFlightInfoStatement(ctx, cmd, request)
	case *pb.CommandStatementSubstraitPlan:
		return f.srv.PollFlightInfoSubstraitPlan(ctx, &statementSubstraitPlan{cmd}, request)
	case *pb.CommandPreparedStatementQuery:
		return f.srv.PollFlightInfoPreparedStatement(ctx, cmd, request)
	}
	// XXX: for now we won't support the other methods

	return f.srv.PollFlightInfo(ctx, request)
}

func (f *flightSqlServer) GetSchema(ctx context.Context, request *flight.FlightDescriptor) (*flight.SchemaResult, error) {
	var (
		anycmd anypb.Any
		cmd    proto.Message
		err    error
	)
	if err = proto.Unmarshal(request.Cmd, &anycmd); err != nil {
		return nil, status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
	}

	if cmd, err = anycmd.UnmarshalNew(); err != nil {
		return nil, status.Errorf(codes.InvalidArgument, "could not unmarshal Any to a command type: %s", err.Error())
	}

	switch cmd := cmd.(type) {
	case *pb.CommandStatementQuery:
		return f.srv.GetSchemaStatement(ctx, cmd, request)
	case *pb.CommandStatementSubstraitPlan:
		return f.srv.GetSchemaSubstraitPlan(ctx, &statementSubstraitPlan{cmd}, request)
	case *pb.CommandPreparedStatementQuery:
		return f.srv.GetSchemaPreparedStatement(ctx, cmd, request)
	case *pb.CommandGetCatalogs:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.Catalogs, f.mem)}, nil
	case *pb.CommandGetDbSchemas:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.DBSchemas, f.mem)}, nil
	case *pb.CommandGetTables:
		if cmd.GetIncludeSchema() {
			return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.TablesWithIncludedSchema, f.mem)}, nil
		}
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.Tables, f.mem)}, nil
	case *pb.CommandGetTableTypes:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.TableTypes, f.mem)}, nil
	case *pb.CommandGetXdbcTypeInfo:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.XdbcTypeInfo, f.mem)}, nil
	case *pb.CommandGetSqlInfo:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.SqlInfo, f.mem)}, nil
	case *pb.CommandGetPrimaryKeys:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.PrimaryKeys, f.mem)}, nil
	case *pb.CommandGetExportedKeys:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.ExportedKeys, f.mem)}, nil
	case *pb.CommandGetImportedKeys:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.ImportedKeys, f.mem)}, nil
	case *pb.CommandGetCrossReference:
		return &flight.SchemaResult{Schema: flight.SerializeSchema(schema_ref.CrossReference, f.mem)}, nil
	}

	return nil, status.Errorf(codes.InvalidArgument, "requested command is invalid: %s", anycmd.GetTypeUrl())
}

func (f *flightSqlServer) DoGet(request *flight.Ticket, stream flight.FlightService_DoGetServer) (err error) {
	var (
		anycmd anypb.Any
		cmd    proto.Message
		cc     <-chan flight.StreamChunk
		sc     *arrow.Schema
	)
	if err = proto.Unmarshal(request.Ticket, &anycmd); err != nil {
		return status.Errorf(codes.InvalidArgument, "unable to parse ticket: %s", err.Error())
	}

	if cmd, err = anycmd.UnmarshalNew(); err != nil {
		return status.Errorf(codes.InvalidArgument, "unable to unmarshal proto.Any: %s", err.Error())
	}

	switch cmd := cmd.(type) {
	case *pb.TicketStatementQuery:
		sc, cc, err = f.srv.DoGetStatement(stream.Context(), cmd)
	case *pb.CommandPreparedStatementQuery:
		sc, cc, err = f.srv.DoGetPreparedStatement(stream.Context(), cmd)
	case *pb.CommandGetCatalogs:
		sc, cc, err = f.srv.DoGetCatalogs(stream.Context())
	case *pb.CommandGetDbSchemas:
		sc, cc, err = f.srv.DoGetDBSchemas(stream.Context(), &getDBSchemas{cmd})
	case *pb.CommandGetTables:
		sc, cc, err = f.srv.DoGetTables(stream.Context(), &getTables{cmd})
	case *pb.CommandGetTableTypes:
		sc, cc, err = f.srv.DoGetTableTypes(stream.Context())
	case *pb.CommandGetXdbcTypeInfo:
		sc, cc, err = f.srv.DoGetXdbcTypeInfo(stream.Context(), &getXdbcTypeInfo{cmd})
	case *pb.CommandGetSqlInfo:
		sc, cc, err = f.srv.DoGetSqlInfo(stream.Context(), cmd)
	case *pb.CommandGetPrimaryKeys:
		sc, cc, err = f.srv.DoGetPrimaryKeys(stream.Context(), pkToTableRef(cmd))
	case *pb.CommandGetExportedKeys:
		sc, cc, err = f.srv.DoGetExportedKeys(stream.Context(), exkToTableRef(cmd))
	case *pb.CommandGetImportedKeys:
		sc, cc, err = f.srv.DoGetImportedKeys(stream.Context(), impkToTableRef(cmd))
	case *pb.CommandGetCrossReference:
		sc, cc, err = f.srv.DoGetCrossReference(stream.Context(), toCrossTableRef(cmd))
	default:
		return status.Error(codes.InvalidArgument, "requested command is invalid")
	}

	if err != nil {
		return err
	}

	wr := flight.NewRecordWriter(stream, ipc.WithSchema(sc))
	defer wr.Close()

	for chunk := range cc {
		if chunk.Err != nil {
			return chunk.Err
		}

		wr.SetFlightDescriptor(chunk.Desc)
		if err = wr.WriteWithAppMetadata(chunk.Data, chunk.AppMetadata); err != nil {
			return err
		}
		chunk.Data.Release()
	}

	return err
}

type putMetadataWriter struct {
	stream flight.FlightService_DoPutServer
}

func (p *putMetadataWriter) WriteMetadata(appMetadata []byte) error {
	return p.stream.Send(&flight.PutResult{AppMetadata: appMetadata})
}

func (f *flightSqlServer) DoPut(stream flight.FlightService_DoPutServer) error {
	rdr, err := flight.NewRecordReader(stream, ipc.WithAllocator(f.mem), ipc.WithDelayReadSchema(true))
	if err != nil {
		return status.Errorf(codes.InvalidArgument, "failed to read input stream: %s", err.Error())
	}
	defer rdr.Release()

	// flight descriptor should have come with the schema message
	request := rdr.LatestFlightDescriptor()

	var (
		anycmd anypb.Any
		cmd    proto.Message
	)
	if err = proto.Unmarshal(request.Cmd, &anycmd); err != nil {
		return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
	}

	if cmd, err = anycmd.UnmarshalNew(); err != nil {
		return status.Errorf(codes.InvalidArgument, "could not unmarshal google.protobuf.Any: %s", err.Error())
	}

	switch cmd := cmd.(type) {
	case *pb.CommandStatementUpdate:
		recordCount, err := f.srv.DoPutCommandStatementUpdate(stream.Context(), cmd)
		if err != nil {
			return err
		}

		result := pb.DoPutUpdateResult{RecordCount: recordCount}
		out := &flight.PutResult{}
		if out.AppMetadata, err = proto.Marshal(&result); err != nil {
			return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error())
		}
		return stream.Send(out)
	case *pb.CommandStatementSubstraitPlan:
		recordCount, err := f.srv.DoPutCommandSubstraitPlan(stream.Context(), &statementSubstraitPlan{cmd})
		if err != nil {
			return err
		}

		result := pb.DoPutUpdateResult{RecordCount: recordCount}
		out := &flight.PutResult{}
		if out.AppMetadata, err = proto.Marshal(&result); err != nil {
			return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error())
		}
		return stream.Send(out)
	case *pb.CommandPreparedStatementQuery:
		handle, err := f.srv.DoPutPreparedStatementQuery(stream.Context(), cmd, rdr, &putMetadataWriter{stream})
		if err != nil {
			return err
		}
		result := pb.DoPutPreparedStatementResult{PreparedStatementHandle: handle}
		out := &flight.PutResult{}
		if out.AppMetadata, err = proto.Marshal(&result); err != nil {
			return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error())
		}
		return stream.Send(out)
	case *pb.CommandPreparedStatementUpdate:
		recordCount, err := f.srv.DoPutPreparedStatementUpdate(stream.Context(), cmd, rdr)
		if err != nil {
			return err
		}

		result := pb.DoPutUpdateResult{RecordCount: recordCount}
		out := &flight.PutResult{}
		if out.AppMetadata, err = proto.Marshal(&result); err != nil {
			return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error())
		}
		return stream.Send(out)
	case *pb.CommandStatementIngest:
		// Even if there was an error, the server may have ingested some records.
		// For this reason we send PutResult{recordCount} no matter what, potentially followed by an error
		// if there was one.
		recordCount, rpcErr := f.srv.DoPutCommandStatementIngest(stream.Context(), cmd, rdr)

		result := pb.DoPutUpdateResult{RecordCount: recordCount}
		out := &flight.PutResult{}
		if out.AppMetadata, err = proto.Marshal(&result); err != nil {
			return status.Errorf(codes.Internal, "failed to marshal PutResult: %s", err.Error())
		}

		// If we fail to send the recordCount, just return an error outright
		if err := stream.Send(out); err != nil {
			return err
		}

		// We successfully sent the recordCount.
		// Send the error if one occurred in the RPC, otherwise this is nil.
		return rpcErr
	default:
		return status.Error(codes.InvalidArgument, "the defined request is invalid")
	}
}

func (f *flightSqlServer) ListActions(_ *flight.Empty, stream flight.FlightService_ListActionsServer) error {
	actions := []string{
		flight.CancelFlightInfoActionType,
		flight.RenewFlightEndpointActionType,
		CreatePreparedStatementActionType,
		ClosePreparedStatementActionType,
		BeginSavepointActionType,
		BeginTransactionActionType,
		CancelQueryActionType,
		CreatePreparedSubstraitPlanActionType,
		EndSavepointActionType,
		EndTransactionActionType,
	}

	for _, a := range actions {
		if err := stream.Send(&flight.ActionType{Type: a}); err != nil {
			return err
		}
	}
	return nil
}

func cancelStatusToCancelResult(status flight.CancelStatus) CancelResult {
	switch status {
	case flight.CancelStatusUnspecified:
		return CancelResultUnspecified
	case flight.CancelStatusCancelled:
		return CancelResultCancelled
	case flight.CancelStatusCancelling:
		return CancelResultCancelling
	case flight.CancelStatusNotCancellable:
		return CancelResultNotCancellable
	default:
		return CancelResultUnspecified
	}
}

func (f *flightSqlServer) DoAction(cmd *flight.Action, stream flight.FlightService_DoActionServer) error {
	var anycmd anypb.Any

	switch cmd.Type {
	case flight.CancelFlightInfoActionType:
		var (
			request flight.CancelFlightInfoRequest
			result  flight.CancelFlightInfoResult
			err     error
		)

		if err = proto.Unmarshal(cmd.Body, &request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal CancelFlightInfoRequest for CancelFlightInfo: %s", err.Error())
		}

		result, err = f.srv.CancelFlightInfo(stream.Context(), &request)
		if err != nil {
			return err
		}

		out := &pb.Result{}
		out.Body, err = proto.Marshal(&result)
		if err != nil {
			return err
		}
		return stream.Send(out)
	case flight.RenewFlightEndpointActionType:
		var (
			request flight.RenewFlightEndpointRequest
			err     error
		)

		if err = proto.Unmarshal(cmd.Body, &request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal FlightEndpoint for RenewFlightEndpoint: %s", err.Error())
		}

		renewedEndpoint, err := f.srv.RenewFlightEndpoint(stream.Context(), &request)
		if err != nil {
			return err
		}

		out := &pb.Result{}
		out.Body, err = proto.Marshal(renewedEndpoint)
		if err != nil {
			return err
		}
		return stream.Send(out)
	case BeginSavepointActionType:
		if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
		}

		var (
			request pb.ActionBeginSavepointRequest
			result  pb.ActionBeginSavepointResult
			id      []byte
			err     error
		)
		if err = anycmd.UnmarshalTo(&request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error())
		}

		if id, err = f.srv.BeginSavepoint(stream.Context(), &request); err != nil {
			return err
		}

		result.SavepointId = id
		out, err := packActionResult(&result)
		if err != nil {
			return err
		}
		return stream.Send(out)
	case BeginTransactionActionType:
		if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
		}

		var (
			request pb.ActionBeginTransactionRequest
			result  pb.ActionBeginTransactionResult
			id      []byte
			err     error
		)
		if err = anycmd.UnmarshalTo(&request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error())
		}

		if id, err = f.srv.BeginTransaction(stream.Context(), &request); err != nil {
			return err
		}

		result.TransactionId = id
		out, err := packActionResult(&result)
		if err != nil {
			return err
		}
		return stream.Send(out)
	case CancelQueryActionType:
		if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
		}

		var (
			//nolint:staticcheck,SA1019 for backward compatibility
			request pb.ActionCancelQueryRequest
			//nolint:staticcheck,SA1019 for backward compatibility
			result pb.ActionCancelQueryResult
			info   flight.FlightInfo
			err    error
		)

		if err = anycmd.UnmarshalTo(&request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error())
		}

		if err = proto.Unmarshal(request.Info, &info); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal FlightInfo for CancelQuery: %s", err)
		}

		if cancel, ok := f.srv.(cancelQueryServer); ok {
			result.Result, err = cancel.CancelQuery(stream.Context(), &cancelQueryRequest{&info})
			if err != nil {
				return err
			}
		} else {
			cancelFlightInfoRequest := flight.CancelFlightInfoRequest{Info: &info}
			cancelFlightInfoResult, err := f.srv.CancelFlightInfo(stream.Context(), &cancelFlightInfoRequest)
			if err != nil {
				return err
			}
			result.Result = cancelStatusToCancelResult(cancelFlightInfoResult.Status)
		}

		out, err := packActionResult(&result)
		if err != nil {
			return err
		}
		return stream.Send(out)
	case CreatePreparedStatementActionType:
		if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
		}

		var (
			request pb.ActionCreatePreparedStatementRequest
			result  pb.ActionCreatePreparedStatementResult
			ret     pb.Result
		)
		if err := anycmd.UnmarshalTo(&request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error())
		}

		output, err := f.srv.CreatePreparedStatement(stream.Context(), &request)
		if err != nil {
			return err
		}

		result.PreparedStatementHandle = output.Handle
		if output.DatasetSchema != nil {
			result.DatasetSchema = flight.SerializeSchema(output.DatasetSchema, f.mem)
		}
		if output.ParameterSchema != nil {
			result.ParameterSchema = flight.SerializeSchema(output.ParameterSchema, f.mem)
		}

		if err := anycmd.MarshalFrom(&result); err != nil {
			return status.Errorf(codes.Internal, "unable to marshal final response: %s", err.Error())
		}

		if ret.Body, err = proto.Marshal(&anycmd); err != nil {
			return status.Errorf(codes.Internal, "unable to marshal result: %s", err.Error())
		}
		return stream.Send(&ret)
	case CreatePreparedSubstraitPlanActionType:
		if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
		}

		var (
			request pb.ActionCreatePreparedSubstraitPlanRequest
			result  pb.ActionCreatePreparedStatementResult
			ret     pb.Result
		)
		if err := anycmd.UnmarshalTo(&request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error())
		}

		output, err := f.srv.CreatePreparedSubstraitPlan(stream.Context(), &createPreparedSubstraitPlanReq{&request})
		if err != nil {
			return err
		}

		result.PreparedStatementHandle = output.Handle
		if output.DatasetSchema != nil {
			result.DatasetSchema = flight.SerializeSchema(output.DatasetSchema, f.mem)
		}
		if output.ParameterSchema != nil {
			result.ParameterSchema = flight.SerializeSchema(output.ParameterSchema, f.mem)
		}

		if err := anycmd.MarshalFrom(&result); err != nil {
			return status.Errorf(codes.Internal, "unable to marshal final response: %s", err.Error())
		}

		if ret.Body, err = proto.Marshal(&anycmd); err != nil {
			return status.Errorf(codes.Internal, "unable to marshal result: %s", err.Error())
		}
		return stream.Send(&ret)
	case ClosePreparedStatementActionType:
		if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
		}

		var request pb.ActionClosePreparedStatementRequest
		if err := anycmd.UnmarshalTo(&request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error())
		}

		if err := f.srv.ClosePreparedStatement(stream.Context(), &request); err != nil {
			return err
		}

		return stream.Send(&pb.Result{})
	case EndTransactionActionType:
		if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
		}

		var request pb.ActionEndTransactionRequest
		if err := anycmd.UnmarshalTo(&request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error())
		}

		if err := f.srv.EndTransaction(stream.Context(), &request); err != nil {
			return err
		}

		return stream.Send(&pb.Result{})
	case EndSavepointActionType:
		if err := proto.Unmarshal(cmd.Body, &anycmd); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to parse command: %s", err.Error())
		}

		var request pb.ActionEndSavepointRequest
		if err := anycmd.UnmarshalTo(&request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal google.protobuf.Any: %s", err.Error())
		}

		if err := f.srv.EndSavepoint(stream.Context(), &request); err != nil {
			return err
		}

		return stream.Send(&pb.Result{})
	case flight.SetSessionOptionsActionType:
		var (
			request flight.SetSessionOptionsRequest
			err     error
		)

		if err = proto.Unmarshal(cmd.Body, &request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal SetSessionOptionsRequest: %s", err.Error())
		}

		response, err := f.srv.SetSessionOptions(stream.Context(), &request)
		if err != nil {
			return err
		}

		out := &pb.Result{}
		out.Body, err = proto.Marshal(response)
		if err != nil {
			return err
		}
		return stream.Send(out)
	case flight.GetSessionOptionsActionType:
		var (
			request flight.GetSessionOptionsRequest
			err     error
		)

		if err = proto.Unmarshal(cmd.Body, &request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal GetSessionOptionsRequest: %s", err.Error())
		}

		response, err := f.srv.GetSessionOptions(stream.Context(), &request)
		if err != nil {
			return err
		}

		out := &pb.Result{}
		out.Body, err = proto.Marshal(response)
		if err != nil {
			return err
		}
		return stream.Send(out)
	case flight.CloseSessionActionType:
		var (
			request flight.CloseSessionRequest
			err     error
		)

		if err = proto.Unmarshal(cmd.Body, &request); err != nil {
			return status.Errorf(codes.InvalidArgument, "unable to unmarshal CloseSessionRequest: %s", err.Error())
		}

		response, err := f.srv.CloseSession(stream.Context(), &request)
		if err != nil {
			return err
		}

		out := &pb.Result{}
		out.Body, err = proto.Marshal(response)
		if err != nil {
			return err
		}
		return stream.Send(out)
	default:
		return status.Error(codes.InvalidArgument, "the defined request is invalid.")
	}
}

var (
	_ Server = (*BaseServer)(nil)
)