arrow_integration_testing/flight_server_scenarios/
integration_test.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Integration tests for the Flight server.
19
20use core::str;
21use std::collections::HashMap;
22use std::pin::Pin;
23use std::sync::Arc;
24
25use arrow::{
26    array::ArrayRef,
27    buffer::Buffer,
28    datatypes::Schema,
29    datatypes::SchemaRef,
30    ipc::{self, reader, writer},
31    record_batch::RecordBatch,
32};
33use arrow_flight::{
34    flight_descriptor::DescriptorType, flight_service_server::FlightService,
35    flight_service_server::FlightServiceServer, Action, ActionType, Criteria, Empty, FlightData,
36    FlightDescriptor, FlightEndpoint, FlightInfo, HandshakeRequest, HandshakeResponse, IpcMessage,
37    PollInfo, PutResult, SchemaAsIpc, SchemaResult, Ticket,
38};
39use futures::{channel::mpsc, sink::SinkExt, Stream, StreamExt};
40use tokio::sync::Mutex;
41use tonic::{transport::Server, Request, Response, Status, Streaming};
42
43type TonicStream<T> = Pin<Box<dyn Stream<Item = T> + Send + Sync + 'static>>;
44
45type Error = Box<dyn std::error::Error + Send + Sync + 'static>;
46type Result<T = (), E = Error> = std::result::Result<T, E>;
47
48/// Run a scenario that tests integration testing.
49pub async fn scenario_setup(port: u16) -> Result {
50    let addr = super::listen_on(port).await?;
51    let resolved_port = addr.port();
52
53    let service = FlightServiceImpl {
54        // See https://github.com/apache/arrow-rs/issues/6577
55        // C# had trouble resolving addressed like 0.0.0.0:port
56        // server_location: format!("grpc+tcp://{addr}"),
57        server_location: format!("grpc+tcp://localhost:{resolved_port}"),
58        ..Default::default()
59    };
60    let svc = FlightServiceServer::new(service);
61
62    let server = Server::builder().add_service(svc).serve(addr);
63
64    // NOTE: Log output used in tests to signal server is ready
65    println!("Server listening on localhost:{}", addr.port());
66    server.await?;
67    Ok(())
68}
69
70#[derive(Debug, Clone)]
71struct IntegrationDataset {
72    schema: Schema,
73    chunks: Vec<RecordBatch>,
74}
75
76/// Flight service implementation for integration testing
77#[derive(Clone, Default)]
78pub struct FlightServiceImpl {
79    server_location: String,
80    uploaded_chunks: Arc<Mutex<HashMap<String, IntegrationDataset>>>,
81}
82
83impl FlightServiceImpl {
84    fn endpoint_from_path(&self, path: &str) -> FlightEndpoint {
85        super::endpoint(path, &self.server_location)
86    }
87}
88
89#[tonic::async_trait]
90impl FlightService for FlightServiceImpl {
91    type HandshakeStream = TonicStream<Result<HandshakeResponse, Status>>;
92    type ListFlightsStream = TonicStream<Result<FlightInfo, Status>>;
93    type DoGetStream = TonicStream<Result<FlightData, Status>>;
94    type DoPutStream = TonicStream<Result<PutResult, Status>>;
95    type DoActionStream = TonicStream<Result<arrow_flight::Result, Status>>;
96    type ListActionsStream = TonicStream<Result<ActionType, Status>>;
97    type DoExchangeStream = TonicStream<Result<FlightData, Status>>;
98
99    async fn get_schema(
100        &self,
101        _request: Request<FlightDescriptor>,
102    ) -> Result<Response<SchemaResult>, Status> {
103        Err(Status::unimplemented("Not yet implemented"))
104    }
105
106    async fn do_get(
107        &self,
108        request: Request<Ticket>,
109    ) -> Result<Response<Self::DoGetStream>, Status> {
110        let ticket = request.into_inner();
111
112        let key = str::from_utf8(&ticket.ticket)
113            .map_err(|e| Status::invalid_argument(format!("Invalid ticket: {e:?}")))?;
114
115        let uploaded_chunks = self.uploaded_chunks.lock().await;
116
117        let flight = uploaded_chunks
118            .get(key)
119            .ok_or_else(|| Status::not_found(format!("Could not find flight. {key}")))?;
120
121        let options = arrow::ipc::writer::IpcWriteOptions::default();
122        let mut dictionary_tracker = writer::DictionaryTracker::new(false);
123        let data_gen = writer::IpcDataGenerator::default();
124        let data = IpcMessage(
125            data_gen
126                .schema_to_bytes_with_dictionary_tracker(
127                    &flight.schema,
128                    &mut dictionary_tracker,
129                    &options,
130                )
131                .ipc_message
132                .into(),
133        );
134        let schema_flight_data = FlightData {
135            data_header: data.0,
136            ..Default::default()
137        };
138
139        let schema = std::iter::once(Ok(schema_flight_data));
140
141        let batches = flight
142            .chunks
143            .iter()
144            .enumerate()
145            .flat_map(|(counter, batch)| {
146                let (encoded_dictionaries, encoded_batch) = data_gen
147                    .encode(
148                        batch,
149                        &mut dictionary_tracker,
150                        &options,
151                        &mut Default::default(),
152                    )
153                    .expect("DictionaryTracker configured above to not error on replacement");
154
155                let dictionary_flight_data = encoded_dictionaries.into_iter().map(Into::into);
156                let mut batch_flight_data: FlightData = encoded_batch.into();
157
158                // Only the record batch's FlightData gets app_metadata
159                let metadata = counter.to_string().into();
160                batch_flight_data.app_metadata = metadata;
161
162                dictionary_flight_data
163                    .chain(std::iter::once(batch_flight_data))
164                    .map(Ok)
165            });
166
167        let output = futures::stream::iter(schema.chain(batches).collect::<Vec<_>>());
168
169        Ok(Response::new(Box::pin(output) as Self::DoGetStream))
170    }
171
172    async fn handshake(
173        &self,
174        _request: Request<Streaming<HandshakeRequest>>,
175    ) -> Result<Response<Self::HandshakeStream>, Status> {
176        Err(Status::unimplemented("Not yet implemented"))
177    }
178
179    async fn list_flights(
180        &self,
181        _request: Request<Criteria>,
182    ) -> Result<Response<Self::ListFlightsStream>, Status> {
183        Err(Status::unimplemented("Not yet implemented"))
184    }
185
186    async fn get_flight_info(
187        &self,
188        request: Request<FlightDescriptor>,
189    ) -> Result<Response<FlightInfo>, Status> {
190        let descriptor = request.into_inner();
191
192        match descriptor.r#type {
193            t if t == DescriptorType::Path as i32 => {
194                let path = &descriptor.path;
195                if path.is_empty() {
196                    return Err(Status::invalid_argument("Invalid path"));
197                }
198
199                let uploaded_chunks = self.uploaded_chunks.lock().await;
200                let flight = uploaded_chunks.get(&path[0]).ok_or_else(|| {
201                    Status::not_found(format!("Could not find flight. {}", path[0]))
202                })?;
203
204                let endpoint = self.endpoint_from_path(&path[0]);
205
206                let total_records: usize = flight.chunks.iter().map(|chunk| chunk.num_rows()).sum();
207
208                let options = arrow::ipc::writer::IpcWriteOptions::default();
209                let message = SchemaAsIpc::new(&flight.schema, &options)
210                    .try_into()
211                    .expect(
212                        "Could not generate schema bytes from schema stored by a DoPut; \
213                         this should be impossible",
214                    );
215                let IpcMessage(schema) = message;
216
217                let info = FlightInfo {
218                    schema,
219                    flight_descriptor: Some(descriptor.clone()),
220                    endpoint: vec![endpoint],
221                    total_records: total_records as i64,
222                    total_bytes: -1,
223                    ordered: false,
224                    app_metadata: vec![].into(),
225                };
226
227                Ok(Response::new(info))
228            }
229            other => Err(Status::unimplemented(format!("Request type: {other}"))),
230        }
231    }
232
233    async fn poll_flight_info(
234        &self,
235        _request: Request<FlightDescriptor>,
236    ) -> Result<Response<PollInfo>, Status> {
237        Err(Status::unimplemented("Not yet implemented"))
238    }
239
240    async fn do_put(
241        &self,
242        request: Request<Streaming<FlightData>>,
243    ) -> Result<Response<Self::DoPutStream>, Status> {
244        let mut input_stream = request.into_inner();
245        let flight_data = input_stream
246            .message()
247            .await?
248            .ok_or_else(|| Status::invalid_argument("Must send some FlightData"))?;
249
250        let descriptor = flight_data
251            .flight_descriptor
252            .clone()
253            .ok_or_else(|| Status::invalid_argument("Must have a descriptor"))?;
254
255        if descriptor.r#type != DescriptorType::Path as i32 || descriptor.path.is_empty() {
256            return Err(Status::invalid_argument("Must specify a path"));
257        }
258
259        let key = descriptor.path[0].clone();
260
261        let schema = Schema::try_from(&flight_data)
262            .map_err(|e| Status::invalid_argument(format!("Invalid schema: {e:?}")))?;
263        let schema_ref = Arc::new(schema.clone());
264
265        let (response_tx, response_rx) = mpsc::channel(10);
266
267        let uploaded_chunks = self.uploaded_chunks.clone();
268
269        tokio::spawn(async {
270            let mut error_tx = response_tx.clone();
271            if let Err(e) = save_uploaded_chunks(
272                uploaded_chunks,
273                schema_ref,
274                input_stream,
275                response_tx,
276                schema,
277                key,
278            )
279            .await
280            {
281                error_tx.send(Err(e)).await.expect("Error sending error")
282            }
283        });
284
285        Ok(Response::new(Box::pin(response_rx) as Self::DoPutStream))
286    }
287
288    async fn do_action(
289        &self,
290        _request: Request<Action>,
291    ) -> Result<Response<Self::DoActionStream>, Status> {
292        Err(Status::unimplemented("Not yet implemented"))
293    }
294
295    async fn list_actions(
296        &self,
297        _request: Request<Empty>,
298    ) -> Result<Response<Self::ListActionsStream>, Status> {
299        Err(Status::unimplemented("Not yet implemented"))
300    }
301
302    async fn do_exchange(
303        &self,
304        _request: Request<Streaming<FlightData>>,
305    ) -> Result<Response<Self::DoExchangeStream>, Status> {
306        Err(Status::unimplemented("Not yet implemented"))
307    }
308}
309
310async fn send_app_metadata(
311    tx: &mut mpsc::Sender<Result<PutResult, Status>>,
312    app_metadata: &[u8],
313) -> Result<(), Status> {
314    tx.send(Ok(PutResult {
315        app_metadata: app_metadata.to_vec().into(),
316    }))
317    .await
318    .map_err(|e| Status::internal(format!("Could not send PutResult: {e:?}")))
319}
320
321async fn record_batch_from_message(
322    message: ipc::Message<'_>,
323    data_body: &Buffer,
324    schema_ref: SchemaRef,
325    dictionaries_by_id: &HashMap<i64, ArrayRef>,
326) -> Result<RecordBatch, Status> {
327    let ipc_batch = message
328        .header_as_record_batch()
329        .ok_or_else(|| Status::internal("Could not parse message header as record batch"))?;
330
331    let arrow_batch_result = reader::read_record_batch(
332        data_body,
333        ipc_batch,
334        schema_ref,
335        dictionaries_by_id,
336        None,
337        &message.version(),
338    );
339
340    arrow_batch_result
341        .map_err(|e| Status::internal(format!("Could not convert to RecordBatch: {e:?}")))
342}
343
344async fn dictionary_from_message(
345    message: ipc::Message<'_>,
346    data_body: &Buffer,
347    schema_ref: SchemaRef,
348    dictionaries_by_id: &mut HashMap<i64, ArrayRef>,
349) -> Result<(), Status> {
350    let ipc_batch = message
351        .header_as_dictionary_batch()
352        .ok_or_else(|| Status::internal("Could not parse message header as dictionary batch"))?;
353
354    let dictionary_batch_result = reader::read_dictionary(
355        data_body,
356        ipc_batch,
357        &schema_ref,
358        dictionaries_by_id,
359        &message.version(),
360    );
361    dictionary_batch_result
362        .map_err(|e| Status::internal(format!("Could not convert to Dictionary: {e:?}")))
363}
364
365async fn save_uploaded_chunks(
366    uploaded_chunks: Arc<Mutex<HashMap<String, IntegrationDataset>>>,
367    schema_ref: Arc<Schema>,
368    mut input_stream: Streaming<FlightData>,
369    mut response_tx: mpsc::Sender<Result<PutResult, Status>>,
370    schema: Schema,
371    key: String,
372) -> Result<(), Status> {
373    let mut chunks = vec![];
374    let mut uploaded_chunks = uploaded_chunks.lock().await;
375
376    let mut dictionaries_by_id = HashMap::new();
377
378    while let Some(Ok(data)) = input_stream.next().await {
379        let message = arrow::ipc::root_as_message(&data.data_header[..])
380            .map_err(|e| Status::internal(format!("Could not parse message: {e:?}")))?;
381
382        match message.header_type() {
383            ipc::MessageHeader::Schema => {
384                return Err(Status::internal(
385                    "Not expecting a schema when messages are read",
386                ))
387            }
388            ipc::MessageHeader::RecordBatch => {
389                send_app_metadata(&mut response_tx, &data.app_metadata).await?;
390
391                let batch = record_batch_from_message(
392                    message,
393                    &Buffer::from(data.data_body.as_ref()),
394                    schema_ref.clone(),
395                    &dictionaries_by_id,
396                )
397                .await?;
398
399                chunks.push(batch);
400            }
401            ipc::MessageHeader::DictionaryBatch => {
402                dictionary_from_message(
403                    message,
404                    &Buffer::from(data.data_body.as_ref()),
405                    schema_ref.clone(),
406                    &mut dictionaries_by_id,
407                )
408                .await?;
409            }
410            t => {
411                return Err(Status::internal(format!(
412                    "Reading types other than record batches not yet supported, \
413                                              unable to read {t:?}"
414                )));
415            }
416        }
417    }
418
419    let dataset = IntegrationDataset { schema, chunks };
420    uploaded_chunks.insert(key, dataset);
421
422    Ok(())
423}