/
integration_test.rs
389 lines (329 loc) · 12.8 KB
/
integration_test.rs
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
// 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::collections::HashMap;
use std::convert::TryFrom;
use std::pin::Pin;
use std::sync::Arc;
use arrow::{
array::ArrayRef,
buffer::Buffer,
datatypes::Schema,
datatypes::SchemaRef,
ipc::{self, reader},
record_batch::RecordBatch,
};
use arrow_flight::{
flight_descriptor::DescriptorType, flight_service_server::FlightService,
flight_service_server::FlightServiceServer, Action, ActionType, Criteria, Empty,
FlightData, FlightDescriptor, FlightEndpoint, FlightInfo, HandshakeRequest,
HandshakeResponse, IpcMessage, PutResult, SchemaAsIpc, SchemaResult, Ticket,
};
use futures::{channel::mpsc, sink::SinkExt, Stream, StreamExt};
use std::convert::TryInto;
use tokio::sync::Mutex;
use tonic::{transport::Server, Request, Response, Status, Streaming};
type TonicStream<T> = Pin<Box<dyn Stream<Item = T> + Send + Sync + 'static>>;
type Error = Box<dyn std::error::Error + Send + Sync + 'static>;
type Result<T = (), E = Error> = std::result::Result<T, E>;
pub async fn scenario_setup(port: u16) -> Result {
let addr = super::listen_on(port).await?;
let service = FlightServiceImpl {
server_location: format!("grpc+tcp://{}", addr),
..Default::default()
};
let svc = FlightServiceServer::new(service);
let server = Server::builder().add_service(svc).serve(addr);
// NOTE: Log output used in tests to signal server is ready
println!("Server listening on localhost:{}", addr.port());
server.await?;
Ok(())
}
#[derive(Debug, Clone)]
struct IntegrationDataset {
schema: Schema,
chunks: Vec<RecordBatch>,
}
#[derive(Clone, Default)]
pub struct FlightServiceImpl {
server_location: String,
uploaded_chunks: Arc<Mutex<HashMap<String, IntegrationDataset>>>,
}
impl FlightServiceImpl {
fn endpoint_from_path(&self, path: &str) -> FlightEndpoint {
super::endpoint(path, &self.server_location)
}
}
#[tonic::async_trait]
impl FlightService for FlightServiceImpl {
type HandshakeStream = TonicStream<Result<HandshakeResponse, Status>>;
type ListFlightsStream = TonicStream<Result<FlightInfo, Status>>;
type DoGetStream = TonicStream<Result<FlightData, Status>>;
type DoPutStream = TonicStream<Result<PutResult, Status>>;
type DoActionStream = TonicStream<Result<arrow_flight::Result, Status>>;
type ListActionsStream = TonicStream<Result<ActionType, Status>>;
type DoExchangeStream = TonicStream<Result<FlightData, Status>>;
async fn get_schema(
&self,
_request: Request<FlightDescriptor>,
) -> Result<Response<SchemaResult>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn do_get(
&self,
request: Request<Ticket>,
) -> Result<Response<Self::DoGetStream>, Status> {
let ticket = request.into_inner();
let key = String::from_utf8(ticket.ticket.to_vec())
.map_err(|e| Status::invalid_argument(format!("Invalid ticket: {:?}", e)))?;
let uploaded_chunks = self.uploaded_chunks.lock().await;
let flight = uploaded_chunks.get(&key).ok_or_else(|| {
Status::not_found(format!("Could not find flight. {}", key))
})?;
let options = arrow::ipc::writer::IpcWriteOptions::default();
let schema =
std::iter::once(Ok(SchemaAsIpc::new(&flight.schema, &options).into()));
let batches = flight
.chunks
.iter()
.enumerate()
.flat_map(|(counter, batch)| {
let (dictionary_flight_data, mut batch_flight_data) =
arrow_flight::utils::flight_data_from_arrow_batch(batch, &options);
// Only the record batch's FlightData gets app_metadata
let metadata = counter.to_string().into_bytes();
batch_flight_data.app_metadata = metadata;
dictionary_flight_data
.into_iter()
.chain(std::iter::once(batch_flight_data))
.map(Ok)
});
let output = futures::stream::iter(schema.chain(batches).collect::<Vec<_>>());
Ok(Response::new(Box::pin(output) as Self::DoGetStream))
}
async fn handshake(
&self,
_request: Request<Streaming<HandshakeRequest>>,
) -> Result<Response<Self::HandshakeStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn list_flights(
&self,
_request: Request<Criteria>,
) -> Result<Response<Self::ListFlightsStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn get_flight_info(
&self,
request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
let descriptor = request.into_inner();
match descriptor.r#type {
t if t == DescriptorType::Path as i32 => {
let path = &descriptor.path;
if path.is_empty() {
return Err(Status::invalid_argument("Invalid path"));
}
let uploaded_chunks = self.uploaded_chunks.lock().await;
let flight = uploaded_chunks.get(&path[0]).ok_or_else(|| {
Status::not_found(format!("Could not find flight. {}", path[0]))
})?;
let endpoint = self.endpoint_from_path(&path[0]);
let total_records: usize =
flight.chunks.iter().map(|chunk| chunk.num_rows()).sum();
let options = arrow::ipc::writer::IpcWriteOptions::default();
let message = SchemaAsIpc::new(&flight.schema, &options)
.try_into()
.expect(
"Could not generate schema bytes from schema stored by a DoPut; \
this should be impossible",
);
let IpcMessage(schema) = message;
let info = FlightInfo {
schema,
flight_descriptor: Some(descriptor.clone()),
endpoint: vec![endpoint],
total_records: total_records as i64,
total_bytes: -1,
};
Ok(Response::new(info))
}
other => Err(Status::unimplemented(format!("Request type: {}", other))),
}
}
async fn do_put(
&self,
request: Request<Streaming<FlightData>>,
) -> Result<Response<Self::DoPutStream>, Status> {
let mut input_stream = request.into_inner();
let flight_data = input_stream
.message()
.await?
.ok_or_else(|| Status::invalid_argument("Must send some FlightData"))?;
let descriptor = flight_data
.flight_descriptor
.clone()
.ok_or_else(|| Status::invalid_argument("Must have a descriptor"))?;
if descriptor.r#type != DescriptorType::Path as i32 || descriptor.path.is_empty()
{
return Err(Status::invalid_argument("Must specify a path"));
}
let key = descriptor.path[0].clone();
let schema = Schema::try_from(&flight_data)
.map_err(|e| Status::invalid_argument(format!("Invalid schema: {:?}", e)))?;
let schema_ref = Arc::new(schema.clone());
let (response_tx, response_rx) = mpsc::channel(10);
let uploaded_chunks = self.uploaded_chunks.clone();
tokio::spawn(async {
let mut error_tx = response_tx.clone();
if let Err(e) = save_uploaded_chunks(
uploaded_chunks,
schema_ref,
input_stream,
response_tx,
schema,
key,
)
.await
{
error_tx.send(Err(e)).await.expect("Error sending error")
}
});
Ok(Response::new(Box::pin(response_rx) as Self::DoPutStream))
}
async fn do_action(
&self,
_request: Request<Action>,
) -> Result<Response<Self::DoActionStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn list_actions(
&self,
_request: Request<Empty>,
) -> Result<Response<Self::ListActionsStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn do_exchange(
&self,
_request: Request<Streaming<FlightData>>,
) -> Result<Response<Self::DoExchangeStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
}
async fn send_app_metadata(
tx: &mut mpsc::Sender<Result<PutResult, Status>>,
app_metadata: &[u8],
) -> Result<(), Status> {
tx.send(Ok(PutResult {
app_metadata: app_metadata.to_vec(),
}))
.await
.map_err(|e| Status::internal(format!("Could not send PutResult: {:?}", e)))
}
async fn record_batch_from_message(
message: ipc::Message<'_>,
data_body: &Buffer,
schema_ref: SchemaRef,
dictionaries_by_id: &HashMap<i64, ArrayRef>,
) -> Result<RecordBatch, Status> {
let ipc_batch = message.header_as_record_batch().ok_or_else(|| {
Status::internal("Could not parse message header as record batch")
})?;
let arrow_batch_result = reader::read_record_batch(
data_body,
ipc_batch,
schema_ref,
dictionaries_by_id,
None,
&message.version(),
);
arrow_batch_result.map_err(|e| {
Status::internal(format!("Could not convert to RecordBatch: {:?}", e))
})
}
async fn dictionary_from_message(
message: ipc::Message<'_>,
data_body: &Buffer,
schema_ref: SchemaRef,
dictionaries_by_id: &mut HashMap<i64, ArrayRef>,
) -> Result<(), Status> {
let ipc_batch = message.header_as_dictionary_batch().ok_or_else(|| {
Status::internal("Could not parse message header as dictionary batch")
})?;
let dictionary_batch_result = reader::read_dictionary(
data_body,
ipc_batch,
&schema_ref,
dictionaries_by_id,
&message.version(),
);
dictionary_batch_result.map_err(|e| {
Status::internal(format!("Could not convert to Dictionary: {:?}", e))
})
}
async fn save_uploaded_chunks(
uploaded_chunks: Arc<Mutex<HashMap<String, IntegrationDataset>>>,
schema_ref: Arc<Schema>,
mut input_stream: Streaming<FlightData>,
mut response_tx: mpsc::Sender<Result<PutResult, Status>>,
schema: Schema,
key: String,
) -> Result<(), Status> {
let mut chunks = vec![];
let mut uploaded_chunks = uploaded_chunks.lock().await;
let mut dictionaries_by_id = HashMap::new();
while let Some(Ok(data)) = input_stream.next().await {
let message = arrow::ipc::root_as_message(&data.data_header[..])
.map_err(|e| Status::internal(format!("Could not parse message: {:?}", e)))?;
match message.header_type() {
ipc::MessageHeader::Schema => {
return Err(Status::internal(
"Not expecting a schema when messages are read",
))
}
ipc::MessageHeader::RecordBatch => {
send_app_metadata(&mut response_tx, &data.app_metadata).await?;
let batch = record_batch_from_message(
message,
&Buffer::from(data.data_body),
schema_ref.clone(),
&dictionaries_by_id,
)
.await?;
chunks.push(batch);
}
ipc::MessageHeader::DictionaryBatch => {
dictionary_from_message(
message,
&Buffer::from(data.data_body),
schema_ref.clone(),
&mut dictionaries_by_id,
)
.await?;
}
t => {
return Err(Status::internal(format!(
"Reading types other than record batches not yet supported, \
unable to read {:?}",
t
)));
}
}
}
let dataset = IntegrationDataset { schema, chunks };
uploaded_chunks.insert(key, dataset);
Ok(())
}