arrow-flight 49.0.0

Apache Arrow Flight
Documentation
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
// 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.

//! Integration test for "mid level" Client

mod common {
    pub mod server;
    pub mod trailers_layer;
}
use arrow_array::{RecordBatch, UInt64Array};
use arrow_flight::{
    decode::FlightRecordBatchStream, encode::FlightDataEncoderBuilder, error::FlightError, Action,
    ActionType, Criteria, Empty, FlightClient, FlightData, FlightDescriptor, FlightInfo,
    HandshakeRequest, HandshakeResponse, PutResult, Ticket,
};
use arrow_schema::{DataType, Field, Schema};
use bytes::Bytes;
use common::{server::TestFlightServer, trailers_layer::TrailersLayer};
use futures::{Future, StreamExt, TryStreamExt};
use tokio::{net::TcpListener, task::JoinHandle};
use tonic::{
    transport::{Channel, Uri},
    Status,
};

use std::{net::SocketAddr, sync::Arc, time::Duration};

const DEFAULT_TIMEOUT_SECONDS: u64 = 30;

#[tokio::test]
async fn test_handshake() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();
        let request_payload = Bytes::from("foo-request-payload");
        let response_payload = Bytes::from("bar-response-payload");

        let request = HandshakeRequest {
            payload: request_payload.clone(),
            protocol_version: 0,
        };

        let response = HandshakeResponse {
            payload: response_payload.clone(),
            protocol_version: 0,
        };

        test_server.set_handshake_response(Ok(response));
        let response = client.handshake(request_payload).await.unwrap();
        assert_eq!(response, response_payload);
        assert_eq!(test_server.take_handshake_request(), Some(request));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_handshake_error() {
    do_test(|test_server, mut client| async move {
        let request_payload = "foo-request-payload".to_string().into_bytes();
        let e = Status::unauthenticated("DENIED");
        test_server.set_handshake_response(Err(e.clone()));

        let response = client.handshake(request_payload).await.unwrap_err();
        expect_status(response, e);
    })
    .await;
}

/// Verifies that all headers sent from the the client are in the request_metadata
fn ensure_metadata(client: &FlightClient, test_server: &TestFlightServer) {
    let client_metadata = client.metadata().clone().into_headers();
    assert!(!client_metadata.is_empty());
    let metadata = test_server
        .take_last_request_metadata()
        .expect("No headers in server")
        .into_headers();

    for (k, v) in &client_metadata {
        assert_eq!(
            metadata.get(k).as_ref(),
            Some(&v),
            "Missing / Mismatched metadata {k:?} sent {client_metadata:?} got {metadata:?}"
        );
    }
}

fn test_flight_info(request: &FlightDescriptor) -> FlightInfo {
    FlightInfo {
        schema: Bytes::new(),
        endpoint: vec![],
        flight_descriptor: Some(request.clone()),
        total_bytes: 123,
        total_records: 456,
        ordered: false,
    }
}

#[tokio::test]
async fn test_get_flight_info() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();
        let request = FlightDescriptor::new_cmd(b"My Command".to_vec());

        let expected_response = test_flight_info(&request);
        test_server.set_get_flight_info_response(Ok(expected_response.clone()));

        let response = client.get_flight_info(request.clone()).await.unwrap();

        assert_eq!(response, expected_response);
        assert_eq!(test_server.take_get_flight_info_request(), Some(request));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_get_flight_info_error() {
    do_test(|test_server, mut client| async move {
        let request = FlightDescriptor::new_cmd(b"My Command".to_vec());

        let e = Status::unauthenticated("DENIED");
        test_server.set_get_flight_info_response(Err(e.clone()));

        let response = client.get_flight_info(request.clone()).await.unwrap_err();
        expect_status(response, e);
    })
    .await;
}

// TODO more negative  tests (like if there are endpoints defined, etc)

#[tokio::test]
async fn test_do_get() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();
        let ticket = Ticket {
            ticket: Bytes::from("my awesome flight ticket"),
        };

        let batch = RecordBatch::try_from_iter(vec![(
            "col",
            Arc::new(UInt64Array::from_iter([1, 2, 3, 4])) as _,
        )])
        .unwrap();

        let response = vec![Ok(batch.clone())];
        test_server.set_do_get_response(response);
        let mut response_stream = client
            .do_get(ticket.clone())
            .await
            .expect("error making request");

        assert_eq!(
            response_stream
                .headers()
                .get("test-resp-header")
                .expect("header exists")
                .to_str()
                .unwrap(),
            "some_val",
        );

        // trailers are not available before stream exhaustion
        assert!(response_stream.trailers().is_none());

        let expected_response = vec![batch];
        let response: Vec<_> = (&mut response_stream)
            .try_collect()
            .await
            .expect("Error streaming data");
        assert_eq!(response, expected_response);

        assert_eq!(
            response_stream
                .trailers()
                .expect("stream exhausted")
                .get("test-trailer")
                .expect("trailer exists")
                .to_str()
                .unwrap(),
            "trailer_val",
        );

        assert_eq!(test_server.take_do_get_request(), Some(ticket));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_get_error() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();
        let ticket = Ticket {
            ticket: Bytes::from("my awesome flight ticket"),
        };

        let response = client.do_get(ticket.clone()).await.unwrap_err();

        let e = Status::internal("No do_get response configured");
        expect_status(response, e);
        // server still got the request
        assert_eq!(test_server.take_do_get_request(), Some(ticket));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_get_error_in_record_batch_stream() {
    do_test(|test_server, mut client| async move {
        let ticket = Ticket {
            ticket: Bytes::from("my awesome flight ticket"),
        };

        let batch = RecordBatch::try_from_iter(vec![(
            "col",
            Arc::new(UInt64Array::from_iter([1, 2, 3, 4])) as _,
        )])
        .unwrap();

        let e = Status::data_loss("she's dead jim");

        let expected_response = vec![Ok(batch), Err(e.clone())];

        test_server.set_do_get_response(expected_response);

        let response_stream = client
            .do_get(ticket.clone())
            .await
            .expect("error making request");

        let response: Result<Vec<_>, FlightError> = response_stream.try_collect().await;

        let response = response.unwrap_err();
        expect_status(response, e);
        // server still got the request
        assert_eq!(test_server.take_do_get_request(), Some(ticket));
    })
    .await;
}

#[tokio::test]
async fn test_do_put() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        // encode the batch as a stream of FlightData
        let input_flight_data = test_flight_data().await;

        let expected_response = vec![
            PutResult {
                app_metadata: Bytes::from("foo-metadata1"),
            },
            PutResult {
                app_metadata: Bytes::from("bar-metadata2"),
            },
        ];

        test_server.set_do_put_response(expected_response.clone().into_iter().map(Ok).collect());

        let input_stream = futures::stream::iter(input_flight_data.clone()).map(Ok);

        let response_stream = client
            .do_put(input_stream)
            .await
            .expect("error making request");

        let response: Vec<_> = response_stream
            .try_collect()
            .await
            .expect("Error streaming data");

        assert_eq!(response, expected_response);
        assert_eq!(test_server.take_do_put_request(), Some(input_flight_data));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_put_error_server() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let input_flight_data = test_flight_data().await;

        let input_stream = futures::stream::iter(input_flight_data.clone()).map(Ok);

        let response = client.do_put(input_stream).await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        let e = Status::internal("No do_put response configured");
        expect_status(response, e);
        // server still got the request
        assert_eq!(test_server.take_do_put_request(), Some(input_flight_data));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_put_error_stream_server() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let input_flight_data = test_flight_data().await;

        let e = Status::invalid_argument("bad arg");

        let response = vec![
            Ok(PutResult {
                app_metadata: Bytes::from("foo-metadata"),
            }),
            Err(e.clone()),
        ];

        test_server.set_do_put_response(response);

        let input_stream = futures::stream::iter(input_flight_data.clone()).map(Ok);

        let response_stream = client
            .do_put(input_stream)
            .await
            .expect("error making request");

        let response: Result<Vec<_>, _> = response_stream.try_collect().await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        expect_status(response, e);
        // server still got the request
        assert_eq!(test_server.take_do_put_request(), Some(input_flight_data));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_put_error_client() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let e = Status::invalid_argument("bad arg: client");

        // input stream to client sends good FlightData followed by an error
        let input_flight_data = test_flight_data().await;
        let input_stream = futures::stream::iter(input_flight_data.clone())
            .map(Ok)
            .chain(futures::stream::iter(vec![Err(FlightError::from(
                e.clone(),
            ))]));

        // server responds with one good message
        let response = vec![Ok(PutResult {
            app_metadata: Bytes::from("foo-metadata"),
        })];
        test_server.set_do_put_response(response);

        let response_stream = client
            .do_put(input_stream)
            .await
            .expect("error making request");

        let response: Result<Vec<_>, _> = response_stream.try_collect().await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        // expect to the error made from the client
        expect_status(response, e);
        // server still got the request messages until the client sent the error
        assert_eq!(test_server.take_do_put_request(), Some(input_flight_data));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_put_error_client_and_server() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let e_client = Status::invalid_argument("bad arg: client");
        let e_server = Status::invalid_argument("bad arg: server");

        // input stream to client sends good FlightData followed by an error
        let input_flight_data = test_flight_data().await;
        let input_stream = futures::stream::iter(input_flight_data.clone())
            .map(Ok)
            .chain(futures::stream::iter(vec![Err(FlightError::from(
                e_client.clone(),
            ))]));

        // server responds with an error (e.g. because it got truncated data)
        let response = vec![Err(e_server)];
        test_server.set_do_put_response(response);

        let response_stream = client
            .do_put(input_stream)
            .await
            .expect("error making request");

        let response: Result<Vec<_>, _> = response_stream.try_collect().await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        // expect to the error made from the client (not the server)
        expect_status(response, e_client);
        // server still got the request messages until the client sent the error
        assert_eq!(test_server.take_do_put_request(), Some(input_flight_data));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_exchange() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        // encode the batch as a stream of FlightData
        let input_flight_data = test_flight_data().await;
        let output_flight_data = test_flight_data2().await;

        test_server
            .set_do_exchange_response(output_flight_data.clone().into_iter().map(Ok).collect());

        let response_stream = client
            .do_exchange(futures::stream::iter(input_flight_data.clone()))
            .await
            .expect("error making request");

        let response: Vec<_> = response_stream
            .try_collect()
            .await
            .expect("Error streaming data");

        let expected_stream = futures::stream::iter(output_flight_data).map(Ok);

        let expected_batches: Vec<_> =
            FlightRecordBatchStream::new_from_flight_data(expected_stream)
                .try_collect()
                .await
                .unwrap();

        assert_eq!(response, expected_batches);
        assert_eq!(
            test_server.take_do_exchange_request(),
            Some(input_flight_data)
        );
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_exchange_error() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let input_flight_data = test_flight_data().await;

        let response = client
            .do_exchange(futures::stream::iter(input_flight_data.clone()))
            .await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        let e = Status::internal("No do_exchange response configured");
        expect_status(response, e);
        // server still got the request
        assert_eq!(
            test_server.take_do_exchange_request(),
            Some(input_flight_data)
        );
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_exchange_error_stream() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let input_flight_data = test_flight_data().await;

        let e = Status::invalid_argument("the error");
        let response = test_flight_data2()
            .await
            .into_iter()
            .enumerate()
            .map(|(i, m)| {
                if i == 0 {
                    Ok(m)
                } else {
                    // make all messages after the first an error
                    Err(e.clone())
                }
            })
            .collect();

        test_server.set_do_exchange_response(response);

        let response_stream = client
            .do_exchange(futures::stream::iter(input_flight_data.clone()))
            .await
            .expect("error making request");

        let response: Result<Vec<_>, _> = response_stream.try_collect().await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        expect_status(response, e);
        // server still got the request
        assert_eq!(
            test_server.take_do_exchange_request(),
            Some(input_flight_data)
        );
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_get_schema() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let schema = Schema::new(vec![Field::new("foo", DataType::Int64, true)]);

        let request = FlightDescriptor::new_cmd("my command");
        test_server.set_get_schema_response(Ok(schema.clone()));

        let response = client
            .get_schema(request.clone())
            .await
            .expect("error making request");

        let expected_schema = schema;
        let expected_request = request;

        assert_eq!(response, expected_schema);
        assert_eq!(
            test_server.take_get_schema_request(),
            Some(expected_request)
        );

        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_get_schema_error() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();
        let request = FlightDescriptor::new_cmd("my command");

        let e = Status::unauthenticated("DENIED");
        test_server.set_get_schema_response(Err(e.clone()));

        let response = client.get_schema(request).await.unwrap_err();
        expect_status(response, e);
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_list_flights() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let infos = vec![
            test_flight_info(&FlightDescriptor::new_cmd("foo")),
            test_flight_info(&FlightDescriptor::new_cmd("bar")),
        ];

        let response = infos.iter().map(|i| Ok(i.clone())).collect();
        test_server.set_list_flights_response(response);

        let response_stream = client
            .list_flights("query")
            .await
            .expect("error making request");

        let expected_response = infos;
        let response: Vec<_> = response_stream
            .try_collect()
            .await
            .expect("Error streaming data");

        let expected_request = Some(Criteria {
            expression: "query".into(),
        });

        assert_eq!(response, expected_response);
        assert_eq!(test_server.take_list_flights_request(), expected_request);
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_list_flights_error() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let response = client.list_flights("query").await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        let e = Status::internal("No list_flights response configured");
        expect_status(response, e);
        // server still got the request
        let expected_request = Some(Criteria {
            expression: "query".into(),
        });
        assert_eq!(test_server.take_list_flights_request(), expected_request);
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_list_flights_error_in_stream() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let e = Status::data_loss("she's dead jim");

        let response = vec![
            Ok(test_flight_info(&FlightDescriptor::new_cmd("foo"))),
            Err(e.clone()),
        ];
        test_server.set_list_flights_response(response);

        let response_stream = client
            .list_flights("other query")
            .await
            .expect("error making request");

        let response: Result<Vec<_>, FlightError> = response_stream.try_collect().await;

        let response = response.unwrap_err();
        expect_status(response, e);
        // server still got the request
        let expected_request = Some(Criteria {
            expression: "other query".into(),
        });
        assert_eq!(test_server.take_list_flights_request(), expected_request);
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_list_actions() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let actions = vec![
            ActionType {
                r#type: "type 1".into(),
                description: "awesomeness".into(),
            },
            ActionType {
                r#type: "type 2".into(),
                description: "more awesomeness".into(),
            },
        ];

        let response = actions.iter().map(|i| Ok(i.clone())).collect();
        test_server.set_list_actions_response(response);

        let response_stream = client.list_actions().await.expect("error making request");

        let expected_response = actions;
        let response: Vec<_> = response_stream
            .try_collect()
            .await
            .expect("Error streaming data");

        assert_eq!(response, expected_response);
        assert_eq!(test_server.take_list_actions_request(), Some(Empty {}));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_list_actions_error() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let response = client.list_actions().await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        let e = Status::internal("No list_actions response configured");
        expect_status(response, e);
        // server still got the request
        assert_eq!(test_server.take_list_actions_request(), Some(Empty {}));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_list_actions_error_in_stream() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let e = Status::data_loss("she's dead jim");

        let response = vec![
            Ok(ActionType {
                r#type: "type 1".into(),
                description: "awesomeness".into(),
            }),
            Err(e.clone()),
        ];
        test_server.set_list_actions_response(response);

        let response_stream = client.list_actions().await.expect("error making request");

        let response: Result<Vec<_>, FlightError> = response_stream.try_collect().await;

        let response = response.unwrap_err();
        expect_status(response, e);
        // server still got the request
        assert_eq!(test_server.take_list_actions_request(), Some(Empty {}));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_action() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let bytes = vec![Bytes::from("foo"), Bytes::from("blarg")];

        let response = bytes
            .iter()
            .cloned()
            .map(arrow_flight::Result::new)
            .map(Ok)
            .collect();
        test_server.set_do_action_response(response);

        let request = Action::new("action type", "action body");

        let response_stream = client
            .do_action(request.clone())
            .await
            .expect("error making request");

        let expected_response = bytes;
        let response: Vec<_> = response_stream
            .try_collect()
            .await
            .expect("Error streaming data");

        assert_eq!(response, expected_response);
        assert_eq!(test_server.take_do_action_request(), Some(request));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_action_error() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let request = Action::new("action type", "action body");

        let response = client.do_action(request.clone()).await;
        let response = match response {
            Ok(_) => panic!("unexpected success"),
            Err(e) => e,
        };

        let e = Status::internal("No do_action response configured");
        expect_status(response, e);
        // server still got the request
        assert_eq!(test_server.take_do_action_request(), Some(request));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

#[tokio::test]
async fn test_do_action_error_in_stream() {
    do_test(|test_server, mut client| async move {
        client.add_header("foo-header", "bar-header-value").unwrap();

        let e = Status::data_loss("she's dead jim");

        let request = Action::new("action type", "action body");

        let response = vec![Ok(arrow_flight::Result::new("foo")), Err(e.clone())];
        test_server.set_do_action_response(response);

        let response_stream = client
            .do_action(request.clone())
            .await
            .expect("error making request");

        let response: Result<Vec<_>, FlightError> = response_stream.try_collect().await;

        let response = response.unwrap_err();
        expect_status(response, e);
        // server still got the request
        assert_eq!(test_server.take_do_action_request(), Some(request));
        ensure_metadata(&client, &test_server);
    })
    .await;
}

async fn test_flight_data() -> Vec<FlightData> {
    let batch = RecordBatch::try_from_iter(vec![(
        "col",
        Arc::new(UInt64Array::from_iter([1, 2, 3, 4])) as _,
    )])
    .unwrap();

    // encode the batch as a stream of FlightData
    FlightDataEncoderBuilder::new()
        .build(futures::stream::iter(vec![Ok(batch)]))
        .try_collect()
        .await
        .unwrap()
}

async fn test_flight_data2() -> Vec<FlightData> {
    let batch = RecordBatch::try_from_iter(vec![(
        "col2",
        Arc::new(UInt64Array::from_iter([10, 23, 33])) as _,
    )])
    .unwrap();

    // encode the batch as a stream of FlightData
    FlightDataEncoderBuilder::new()
        .build(futures::stream::iter(vec![Ok(batch)]))
        .try_collect()
        .await
        .unwrap()
}

/// Runs the future returned by the function,  passing it a test server and client
async fn do_test<F, Fut>(f: F)
where
    F: Fn(TestFlightServer, FlightClient) -> Fut,
    Fut: Future<Output = ()>,
{
    let test_server = TestFlightServer::new();
    let fixture = TestFixture::new(&test_server).await;
    let client = FlightClient::new(fixture.channel().await);

    // run the test function
    f(test_server, client).await;

    // cleanly shutdown the test fixture
    fixture.shutdown_and_wait().await
}

fn expect_status(error: FlightError, expected: Status) {
    let status = if let FlightError::Tonic(status) = error {
        status
    } else {
        panic!("Expected FlightError::Tonic, got: {error:?}");
    };

    assert_eq!(
        status.code(),
        expected.code(),
        "Got {status:?} want {expected:?}"
    );
    assert_eq!(
        status.message(),
        expected.message(),
        "Got {status:?} want {expected:?}"
    );
    assert_eq!(
        status.details(),
        expected.details(),
        "Got {status:?} want {expected:?}"
    );
}

/// Creates and manages a running TestServer with a background task
struct TestFixture {
    /// channel to send shutdown command
    shutdown: Option<tokio::sync::oneshot::Sender<()>>,

    /// Address the server is listening on
    addr: SocketAddr,

    // handle for the server task
    handle: Option<JoinHandle<Result<(), tonic::transport::Error>>>,
}

impl TestFixture {
    /// create a new test fixture from the server
    pub async fn new(test_server: &TestFlightServer) -> Self {
        // let OS choose a a free port
        let listener = TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap();

        println!("Listening on {addr}");

        // prepare the shutdown channel
        let (tx, rx) = tokio::sync::oneshot::channel();

        let server_timeout = Duration::from_secs(DEFAULT_TIMEOUT_SECONDS);

        let shutdown_future = async move {
            rx.await.ok();
        };

        let serve_future = tonic::transport::Server::builder()
            .timeout(server_timeout)
            .layer(TrailersLayer)
            .add_service(test_server.service())
            .serve_with_incoming_shutdown(
                tokio_stream::wrappers::TcpListenerStream::new(listener),
                shutdown_future,
            );

        // Run the server in its own background task
        let handle = tokio::task::spawn(serve_future);

        Self {
            shutdown: Some(tx),
            addr,
            handle: Some(handle),
        }
    }

    /// Return a [`Channel`] connected to the TestServer
    pub async fn channel(&self) -> Channel {
        let url = format!("http://{}", self.addr);
        let uri: Uri = url.parse().expect("Valid URI");
        Channel::builder(uri)
            .timeout(Duration::from_secs(DEFAULT_TIMEOUT_SECONDS))
            .connect()
            .await
            .expect("error connecting to server")
    }

    /// Stops the test server and waits for the server to shutdown
    pub async fn shutdown_and_wait(mut self) {
        if let Some(shutdown) = self.shutdown.take() {
            shutdown.send(()).expect("server quit early");
        }
        if let Some(handle) = self.handle.take() {
            println!("Waiting on server to finish");
            handle
                .await
                .expect("task join error (panic?)")
                .expect("Server Error found at shutdown");
        }
    }
}

impl Drop for TestFixture {
    fn drop(&mut self) {
        if let Some(shutdown) = self.shutdown.take() {
            shutdown.send(()).ok();
        }
        if self.handle.is_some() {
            // tests should properly clean up TestFixture
            println!("TestFixture::Drop called prior to `shutdown_and_wait`");
        }
    }
}