-
Notifications
You must be signed in to change notification settings - Fork 11
/
Copy pathparquet.rs
398 lines (357 loc) · 12.8 KB
/
parquet.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
390
391
392
393
394
395
396
397
398
use fmt::Debug;
use std::any::Any;
use std::fmt;
use std::sync::Arc;
use std::task::{Context, Poll};
use tokio::{
sync::mpsc::{channel, Receiver, Sender},
task,
};
use tokio_stream::wrappers::ReceiverStream;
use crate::clients::CachedFile;
use arrow::datatypes::{Schema, SchemaRef};
use arrow::error::{ArrowError, Result as ArrowResult};
use arrow::record_batch::RecordBatch;
use arrow_parquet::arrow::{ArrowReader, ParquetFileArrowReader};
use arrow_parquet::file::reader::{FileReader, Length, SerializedFileReader};
use async_trait::async_trait;
use datafusion::error::{DataFusionError, Result as DataFusionResult};
use datafusion::physical_plan::ExecutionPlan;
use datafusion::physical_plan::Partitioning;
use datafusion::physical_plan::{RecordBatchStream, SendableRecordBatchStream};
use futures::stream::{Stream, StreamExt};
/// Execution plan for scanning a Parquet file
#[derive(Debug, Clone)]
pub struct ParquetExec {
files: Vec<CachedFile>,
/// Schema before projection is applied
file_schema: SchemaRef,
/// Schema after projection is applied
projected_schema: SchemaRef,
/// Projection for which columns to load
projection: Vec<usize>,
/// Batch size
batch_size: usize,
}
impl ParquetExec {
/// Create a new Parquet reader execution plan
pub fn new(
files: Vec<CachedFile>,
projection: Option<Vec<usize>>,
batch_size: usize,
schema: SchemaRef,
) -> Self {
let projection = match projection {
Some(p) => p,
None => (0..schema.fields().len()).collect(),
};
let projected_schema = Schema::new(
projection
.iter()
.map(|col| schema.field(*col).clone())
.collect(),
);
Self {
files,
file_schema: schema,
projected_schema: Arc::new(projected_schema),
projection,
batch_size,
}
}
/// Read the footer and schedule the downloads of all the required chunks
async fn init_file(
&self,
partition: usize,
) -> DataFusionResult<Arc<SerializedFileReader<CachedFile>>> {
let end_dl_chunk_start = Self::download_footer(self.files[partition].clone());
let file_schema = self.file_schema.clone();
let file = self.files[partition].clone();
let projection = self.projection.clone();
// Reading the footer is blocking so it should be started on a specific thread
tokio::task::spawn_blocking(move || {
let file_reader = Arc::new(
SerializedFileReader::new(file.clone())
.map_err(|e| DataFusionError::ParquetError(e))?,
);
let mut arrow_reader = ParquetFileArrowReader::new(file_reader.clone());
let schema_eq = file_schema
.fields()
.iter()
.zip(arrow_reader.get_schema()?.fields().iter())
.all(|x| {
x.0.name() == x.1.name()
&& x.0.data_type() == x.1.data_type()
&& x.0.is_nullable() == x.1.is_nullable()
});
if !schema_eq {
return Err(DataFusionError::Plan(format!(
"Expected and parsed schema fields are not equal: \n{:?} \n!= \n{:?}",
file_schema.fields(),
arrow_reader.get_schema()?.fields()
)));
}
// prefetch usefull byte ranges
let metadata = file_reader.metadata();
for i in 0..metadata.num_row_groups() {
for proj in &projection {
let rg_metadata = metadata.row_group(i);
let col_metadata = rg_metadata.column(*proj);
let (start, length) = col_metadata.byte_range();
if start < end_dl_chunk_start {
file.prefetch(start, length as usize);
}
}
}
Ok(file_reader)
})
.await
.unwrap()
}
// returns the start of the downloaded chunk
fn download_footer(file: CachedFile) -> u64 {
let end_length = 1024 * 1024;
let (end_start, end_length) = match file.len().checked_sub(end_length) {
Some(val) => (val, end_length),
None => (0, file.len()),
};
file.prefetch(end_start, end_length as usize);
end_start
}
}
#[async_trait]
impl ExecutionPlan for ParquetExec {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.projected_schema.clone()
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
// this is a leaf node and has no children
vec![]
}
/// Get the output partitioning of this plan
fn output_partitioning(&self) -> Partitioning {
Partitioning::UnknownPartitioning(self.files.len())
}
fn with_new_children(
&self,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> DataFusionResult<Arc<dyn ExecutionPlan>> {
if children.is_empty() {
Ok(Arc::new(self.clone()))
} else {
Err(DataFusionError::Internal(format!(
"Children cannot be replaced in {:?}",
self
)))
}
}
async fn execute(
&self,
partition: usize,
) -> DataFusionResult<SendableRecordBatchStream> {
let parquet_reader = self.init_file(partition).await?;
let (response_tx, response_rx): (
Sender<ArrowResult<RecordBatch>>,
Receiver<ArrowResult<RecordBatch>>,
) = channel(2);
let projection = self.projection.clone();
let batch_size = self.batch_size;
task::spawn_blocking(move || {
if let Err(e) = read_file(parquet_reader, projection, batch_size, response_tx)
{
println!("Parquet reader thread terminated due to error: {:?}", e);
}
});
Ok(Box::pin(ParquetStream {
schema: self.projected_schema.clone(),
response_rx: ReceiverStream::new(response_rx),
}))
}
}
fn send_result(
response_tx: &Sender<ArrowResult<RecordBatch>>,
result: ArrowResult<RecordBatch>,
) -> DataFusionResult<()> {
response_tx
.blocking_send(result)
.map_err(|e| DataFusionError::Execution(e.to_string()))?;
Ok(())
}
fn read_file(
file_reader: Arc<SerializedFileReader<CachedFile>>,
projection: Vec<usize>,
batch_size: usize,
response_tx: Sender<ArrowResult<RecordBatch>>,
) -> DataFusionResult<()> {
let mut arrow_reader = ParquetFileArrowReader::new(file_reader.clone());
let mut batch_reader =
arrow_reader.get_record_reader_by_columns(projection.clone(), batch_size)?;
loop {
match batch_reader.next() {
Some(Ok(batch)) => send_result(&response_tx, Ok(batch))?,
None => {
break;
}
Some(Err(e)) => {
let err_msg = format!("Error reading batch from file: {}", e.to_string());
// send error to operator
send_result(
&response_tx,
Err(ArrowError::ParquetError(err_msg.clone())),
)?;
// terminate thread with error
return Err(DataFusionError::Execution(err_msg));
}
}
}
Ok(())
}
struct ParquetStream {
schema: SchemaRef,
response_rx: ReceiverStream<ArrowResult<RecordBatch>>,
}
impl Stream for ParquetStream {
type Item = ArrowResult<RecordBatch>;
fn poll_next(
mut self: std::pin::Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
self.response_rx.poll_next_unpin(cx)
}
}
impl RecordBatchStream for ParquetStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
#[cfg(test)]
mod tests {
use std::env;
use std::fs;
use std::io::SeekFrom;
use std::path::PathBuf;
use super::*;
use crate::clients::Downloader;
use crate::clients::RangeCache;
use crate::error::Result as BuzzResult;
use arrow::array::*;
use arrow::datatypes::{DataType, Field, Schema};
use arrow_parquet::arrow::ArrowWriter;
use async_trait::async_trait;
use tokio::fs::File as TokioFile;
use tokio::io::{AsyncReadExt, AsyncSeekExt};
#[tokio::test]
async fn test_small_parquet() {
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)]));
let rec_batch = RecordBatch::try_new(
schema.clone(),
vec![Arc::new(Int32Array::from(vec![1, 2, 3]))],
)
.unwrap();
let filename = "test_small_parquet.parquet";
let results = write_and_exec(&rec_batch, filename).await;
assert_eq!(results.len(), 1);
assert_eq!(format!("{:?}", results[0]), format!("{:?}", rec_batch));
}
#[tokio::test]
async fn test_parquet_two_columns() {
let schema = Arc::new(Schema::new(vec![
Field::new("a", DataType::Int32, false),
Field::new("b", DataType::Utf8, false),
]));
let rec_batch = RecordBatch::try_new(
schema.clone(),
vec![
Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])),
Arc::new(StringArray::from(vec![
"hello", "from", "the", "other", "side",
])),
],
)
.unwrap();
let filename = "test_2col_parquet.parquet";
let results = write_and_exec(&rec_batch, filename).await;
assert_eq!(results.len(), 1);
assert_eq!(format!("{:?}", results[0]), format!("{:?}", rec_batch));
}
// Without the threaded_scheduler the async machine gets stuck because of blocking call
#[tokio::test(flavor = "multi_thread")]
async fn test_larger_parquet() {
let schema =
Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, false)]));
let rec_batch = RecordBatch::try_new(
schema.clone(),
vec![Arc::new(Float64Array::from(
(0..200_000).map(|val| val as f64 * 0.1).collect::<Vec<_>>(),
))],
)
.unwrap();
let filename = "test_larger_parquet.parquet";
let results = write_and_exec(&rec_batch, filename).await;
assert_eq!(
results.iter().map(|rb| rb.num_rows()).sum::<usize>(),
200_000
);
}
/// Write the given `rec_batch` as a parquet file then make it into an exec plan
async fn write_and_exec(rec_batch: &RecordBatch, filename: &str) -> Vec<RecordBatch> {
let (tmp_file, path) = get_temp_file(filename);
let mut writer =
ArrowWriter::try_new(tmp_file.try_clone().unwrap(), rec_batch.schema(), None)
.unwrap();
writer.write(&rec_batch).unwrap();
writer.close().unwrap();
let file = CachedFile::new(
path.into_os_string().into_string().unwrap(),
tmp_file.metadata().unwrap().len(),
Arc::new(RangeCache::new().await),
"file_downloader".to_owned(),
|| Arc::new(FileDownloader {}),
);
let exec_plan = ParquetExec::new(vec![file], None, 2048, rec_batch.schema());
datafusion::physical_plan::collect(Arc::new(exec_plan))
.await
.unwrap()
}
/// A downloader that simply reads from file system (file_id is the file path)
#[derive(Clone)]
struct FileDownloader {}
#[async_trait]
impl Downloader for FileDownloader {
async fn download(
&self,
file_id: String,
start: u64,
length: usize,
) -> BuzzResult<Vec<u8>> {
let mut result = vec![0; length];
let mut f = TokioFile::open(&file_id).await?;
f.seek(SeekFrom::Start(start)).await?;
f.read_exact(&mut result).await?;
Ok(result)
}
}
/// Returns file handle for a temp file in 'target' directory with an empty content
fn get_temp_file(file_name: &str) -> (fs::File, PathBuf) {
// build tmp path to a file in "target/debug/testdata"
let mut path_buf = env::current_dir().unwrap();
path_buf.push("target");
path_buf.push("debug");
path_buf.push("testdata");
fs::create_dir_all(&path_buf).unwrap();
path_buf.push(file_name);
// create empty file (truncate)
fs::File::create(path_buf.as_path()).unwrap();
// return file handle for both read and write
let file = fs::OpenOptions::new()
.read(true)
.write(true)
.open(path_buf.as_path());
assert!(file.is_ok());
(file.unwrap(), path_buf)
}
}