arrow-flight 54.3.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
// 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.

use std::sync::{Arc, Mutex};

use arrow_array::RecordBatch;
use arrow_schema::Schema;
use futures::{stream::BoxStream, StreamExt, TryStreamExt};
use tonic::{metadata::MetadataMap, Request, Response, Status, Streaming};

use arrow_flight::{
    encode::FlightDataEncoderBuilder,
    flight_service_server::{FlightService, FlightServiceServer},
    Action, ActionType, Criteria, Empty, FlightData, FlightDescriptor, FlightInfo,
    HandshakeRequest, HandshakeResponse, PollInfo, PutResult, SchemaAsIpc, SchemaResult, Ticket,
};

#[derive(Debug, Clone)]
/// Flight server for testing, with configurable responses
pub struct TestFlightServer {
    /// Shared state to configure responses
    state: Arc<Mutex<State>>,
}

impl TestFlightServer {
    /// Create a `TestFlightServer`
    #[allow(dead_code)]
    pub fn new() -> Self {
        Self {
            state: Arc::new(Mutex::new(State::new())),
        }
    }

    /// Return an [`FlightServiceServer`] that can be used with a
    /// [`Server`](tonic::transport::Server)
    #[allow(dead_code)]
    pub fn service(&self) -> FlightServiceServer<TestFlightServer> {
        // wrap up tonic goop
        FlightServiceServer::new(self.clone())
    }

    /// Specify the response returned from the next call to handshake
    #[allow(dead_code)]
    pub fn set_handshake_response(&self, response: Result<HandshakeResponse, Status>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.handshake_response.replace(response);
    }

    /// Take and return last handshake request sent to the server,
    #[allow(dead_code)]
    pub fn take_handshake_request(&self) -> Option<HandshakeRequest> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .handshake_request
            .take()
    }

    /// Specify the response returned from the next call to get_flight_info
    #[allow(dead_code)]
    pub fn set_get_flight_info_response(&self, response: Result<FlightInfo, Status>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.get_flight_info_response.replace(response);
    }

    /// Take and return last get_flight_info request sent to the server,
    #[allow(dead_code)]
    pub fn take_get_flight_info_request(&self) -> Option<FlightDescriptor> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .get_flight_info_request
            .take()
    }

    /// Specify the response returned from the next call to poll_flight_info
    #[allow(dead_code)]
    pub fn set_poll_flight_info_response(&self, response: Result<PollInfo, Status>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.poll_flight_info_response.replace(response);
    }

    /// Take and return last poll_flight_info request sent to the server,
    #[allow(dead_code)]
    pub fn take_poll_flight_info_request(&self) -> Option<FlightDescriptor> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .poll_flight_info_request
            .take()
    }

    /// Specify the response returned from the next call to `do_get`
    #[allow(dead_code)]
    pub fn set_do_get_response(&self, response: Vec<Result<RecordBatch, Status>>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.do_get_response.replace(response);
    }

    /// Take and return last do_get request send to the server,
    #[allow(dead_code)]
    pub fn take_do_get_request(&self) -> Option<Ticket> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .do_get_request
            .take()
    }

    /// Specify the response returned from the next call to `do_put`
    #[allow(dead_code)]
    pub fn set_do_put_response(&self, response: Vec<Result<PutResult, Status>>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.do_put_response.replace(response);
    }

    /// Take and return last do_put request sent to the server,
    #[allow(dead_code)]
    pub fn take_do_put_request(&self) -> Option<Vec<FlightData>> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .do_put_request
            .take()
    }

    /// Specify the response returned from the next call to `do_exchange`
    #[allow(dead_code)]
    pub fn set_do_exchange_response(&self, response: Vec<Result<FlightData, Status>>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.do_exchange_response.replace(response);
    }

    /// Take and return last do_exchange request send to the server,
    #[allow(dead_code)]
    pub fn take_do_exchange_request(&self) -> Option<Vec<FlightData>> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .do_exchange_request
            .take()
    }

    /// Specify the response returned from the next call to `list_flights`
    #[allow(dead_code)]
    pub fn set_list_flights_response(&self, response: Vec<Result<FlightInfo, Status>>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.list_flights_response.replace(response);
    }

    /// Take and return last list_flights request send to the server,
    #[allow(dead_code)]
    pub fn take_list_flights_request(&self) -> Option<Criteria> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .list_flights_request
            .take()
    }

    /// Specify the response returned from the next call to `get_schema`
    #[allow(dead_code)]
    pub fn set_get_schema_response(&self, response: Result<Schema, Status>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.get_schema_response.replace(response);
    }

    /// Take and return last get_schema request send to the server,
    #[allow(dead_code)]
    pub fn take_get_schema_request(&self) -> Option<FlightDescriptor> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .get_schema_request
            .take()
    }

    /// Specify the response returned from the next call to `list_actions`
    #[allow(dead_code)]
    pub fn set_list_actions_response(&self, response: Vec<Result<ActionType, Status>>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.list_actions_response.replace(response);
    }

    /// Take and return last list_actions request send to the server,
    #[allow(dead_code)]
    pub fn take_list_actions_request(&self) -> Option<Empty> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .list_actions_request
            .take()
    }

    /// Specify the response returned from the next call to `do_action`
    #[allow(dead_code)]
    pub fn set_do_action_response(&self, response: Vec<Result<arrow_flight::Result, Status>>) {
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.do_action_response.replace(response);
    }

    /// Take and return last do_action request send to the server,
    #[allow(dead_code)]
    pub fn take_do_action_request(&self) -> Option<Action> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .do_action_request
            .take()
    }

    /// Returns the last metadata from a request received by the server
    #[allow(dead_code)]
    pub fn take_last_request_metadata(&self) -> Option<MetadataMap> {
        self.state
            .lock()
            .expect("mutex not poisoned")
            .last_request_metadata
            .take()
    }

    /// Save the last request's metadatacom
    fn save_metadata<T>(&self, request: &Request<T>) {
        let metadata = request.metadata().clone();
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.last_request_metadata = Some(metadata);
    }
}

/// mutable state for the TestFlightServer, captures requests and provides responses
#[derive(Debug, Default)]
struct State {
    /// The last handshake request that was received
    pub handshake_request: Option<HandshakeRequest>,
    /// The next response to return from `handshake()`
    pub handshake_response: Option<Result<HandshakeResponse, Status>>,
    /// The last `get_flight_info` request received
    pub get_flight_info_request: Option<FlightDescriptor>,
    /// The next response to return from `get_flight_info`
    pub get_flight_info_response: Option<Result<FlightInfo, Status>>,
    /// The last `poll_flight_info` request received
    pub poll_flight_info_request: Option<FlightDescriptor>,
    /// The next response to return from `poll_flight_info`
    pub poll_flight_info_response: Option<Result<PollInfo, Status>>,
    /// The last do_get request received
    pub do_get_request: Option<Ticket>,
    /// The next response returned from `do_get`
    pub do_get_response: Option<Vec<Result<RecordBatch, Status>>>,
    /// The last do_put request received
    pub do_put_request: Option<Vec<FlightData>>,
    /// The next response returned from `do_put`
    pub do_put_response: Option<Vec<Result<PutResult, Status>>>,
    /// The last do_exchange request received
    pub do_exchange_request: Option<Vec<FlightData>>,
    /// The next response returned from `do_exchange`
    pub do_exchange_response: Option<Vec<Result<FlightData, Status>>>,
    /// The last list_flights request received
    pub list_flights_request: Option<Criteria>,
    /// The next response returned from `list_flights`
    pub list_flights_response: Option<Vec<Result<FlightInfo, Status>>>,
    /// The last get_schema request received
    pub get_schema_request: Option<FlightDescriptor>,
    /// The next response returned from `get_schema`
    pub get_schema_response: Option<Result<Schema, Status>>,
    /// The last list_actions request received
    pub list_actions_request: Option<Empty>,
    /// The next response returned from `list_actions`
    pub list_actions_response: Option<Vec<Result<ActionType, Status>>>,
    /// The last do_action request received
    pub do_action_request: Option<Action>,
    /// The next response returned from `do_action`
    pub do_action_response: Option<Vec<Result<arrow_flight::Result, Status>>>,
    /// The last request headers received
    pub last_request_metadata: Option<MetadataMap>,
}

impl State {
    fn new() -> Self {
        Default::default()
    }
}

/// Implement the FlightService trait
#[tonic::async_trait]
impl FlightService for TestFlightServer {
    type HandshakeStream = BoxStream<'static, Result<HandshakeResponse, Status>>;
    type ListFlightsStream = BoxStream<'static, Result<FlightInfo, Status>>;
    type DoGetStream = BoxStream<'static, Result<FlightData, Status>>;
    type DoPutStream = BoxStream<'static, Result<PutResult, Status>>;
    type DoActionStream = BoxStream<'static, Result<arrow_flight::Result, Status>>;
    type ListActionsStream = BoxStream<'static, Result<ActionType, Status>>;
    type DoExchangeStream = BoxStream<'static, Result<FlightData, Status>>;

    async fn handshake(
        &self,
        request: Request<Streaming<HandshakeRequest>>,
    ) -> Result<Response<Self::HandshakeStream>, Status> {
        self.save_metadata(&request);
        let handshake_request = request.into_inner().message().await?.unwrap();

        let mut state = self.state.lock().expect("mutex not poisoned");
        state.handshake_request = Some(handshake_request);

        let response = state
            .handshake_response
            .take()
            .unwrap_or_else(|| Err(Status::internal("No handshake response configured")))?;

        // turn into a streaming response
        let output = futures::stream::iter(std::iter::once(Ok(response)));
        Ok(Response::new(output.boxed()))
    }

    async fn list_flights(
        &self,
        request: Request<Criteria>,
    ) -> Result<Response<Self::ListFlightsStream>, Status> {
        self.save_metadata(&request);
        let mut state = self.state.lock().expect("mutex not poisoned");

        state.list_flights_request = Some(request.into_inner());

        let flights: Vec<_> = state
            .list_flights_response
            .take()
            .ok_or_else(|| Status::internal("No list_flights response configured"))?;

        let flights_stream = futures::stream::iter(flights);

        Ok(Response::new(flights_stream.boxed()))
    }

    async fn get_flight_info(
        &self,
        request: Request<FlightDescriptor>,
    ) -> Result<Response<FlightInfo>, Status> {
        self.save_metadata(&request);
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.get_flight_info_request = Some(request.into_inner());
        let response = state
            .get_flight_info_response
            .take()
            .unwrap_or_else(|| Err(Status::internal("No get_flight_info response configured")))?;
        Ok(Response::new(response))
    }

    async fn poll_flight_info(
        &self,
        request: Request<FlightDescriptor>,
    ) -> Result<Response<PollInfo>, Status> {
        self.save_metadata(&request);
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.poll_flight_info_request = Some(request.into_inner());
        let response = state
            .poll_flight_info_response
            .take()
            .unwrap_or_else(|| Err(Status::internal("No poll_flight_info response configured")))?;
        Ok(Response::new(response))
    }

    async fn get_schema(
        &self,
        request: Request<FlightDescriptor>,
    ) -> Result<Response<SchemaResult>, Status> {
        self.save_metadata(&request);
        let mut state = self.state.lock().expect("mutex not poisoned");
        state.get_schema_request = Some(request.into_inner());
        let schema = state
            .get_schema_response
            .take()
            .unwrap_or_else(|| Err(Status::internal("No get_schema response configured")))?;

        // encode the schema
        let options = arrow_ipc::writer::IpcWriteOptions::default();
        let response: SchemaResult = SchemaAsIpc::new(&schema, &options)
            .try_into()
            .expect("Error encoding schema");

        Ok(Response::new(response))
    }

    async fn do_get(
        &self,
        request: Request<Ticket>,
    ) -> Result<Response<Self::DoGetStream>, Status> {
        self.save_metadata(&request);
        let mut state = self.state.lock().expect("mutex not poisoned");

        state.do_get_request = Some(request.into_inner());

        let batches: Vec<_> = state
            .do_get_response
            .take()
            .ok_or_else(|| Status::internal("No do_get response configured"))?;

        let batch_stream = futures::stream::iter(batches).map_err(Into::into);

        let stream = FlightDataEncoderBuilder::new()
            .build(batch_stream)
            .map_err(Into::into);

        let mut resp = Response::new(stream.boxed());
        resp.metadata_mut()
            .insert("test-resp-header", "some_val".parse().unwrap());

        Ok(resp)
    }

    async fn do_put(
        &self,
        request: Request<Streaming<FlightData>>,
    ) -> Result<Response<Self::DoPutStream>, Status> {
        self.save_metadata(&request);
        let do_put_request: Vec<_> = request.into_inner().try_collect().await?;

        let mut state = self.state.lock().expect("mutex not poisoned");

        state.do_put_request = Some(do_put_request);

        let response = state
            .do_put_response
            .take()
            .ok_or_else(|| Status::internal("No do_put response configured"))?;

        let stream = futures::stream::iter(response).map_err(Into::into);

        Ok(Response::new(stream.boxed()))
    }

    async fn do_action(
        &self,
        request: Request<Action>,
    ) -> Result<Response<Self::DoActionStream>, Status> {
        self.save_metadata(&request);
        let mut state = self.state.lock().expect("mutex not poisoned");

        state.do_action_request = Some(request.into_inner());

        let results: Vec<_> = state
            .do_action_response
            .take()
            .ok_or_else(|| Status::internal("No do_action response configured"))?;

        let results_stream = futures::stream::iter(results);

        Ok(Response::new(results_stream.boxed()))
    }

    async fn list_actions(
        &self,
        request: Request<Empty>,
    ) -> Result<Response<Self::ListActionsStream>, Status> {
        self.save_metadata(&request);
        let mut state = self.state.lock().expect("mutex not poisoned");

        state.list_actions_request = Some(request.into_inner());

        let actions: Vec<_> = state
            .list_actions_response
            .take()
            .ok_or_else(|| Status::internal("No list_actions response configured"))?;

        let action_stream = futures::stream::iter(actions);

        Ok(Response::new(action_stream.boxed()))
    }

    async fn do_exchange(
        &self,
        request: Request<Streaming<FlightData>>,
    ) -> Result<Response<Self::DoExchangeStream>, Status> {
        self.save_metadata(&request);
        let do_exchange_request: Vec<_> = request.into_inner().try_collect().await?;

        let mut state = self.state.lock().expect("mutex not poisoned");

        state.do_exchange_request = Some(do_exchange_request);

        let response = state
            .do_exchange_response
            .take()
            .ok_or_else(|| Status::internal("No do_exchange response configured"))?;

        let stream = futures::stream::iter(response).map_err(Into::into);

        Ok(Response::new(stream.boxed()))
    }
}