parquet/arrow/async_writer/
mod.rs1#[cfg(feature = "object_store")]
59mod store;
60#[cfg(feature = "object_store")]
61pub use store::*;
62
63use crate::{
64 arrow::arrow_writer::ArrowWriterOptions,
65 arrow::ArrowWriter,
66 errors::{ParquetError, Result},
67 file::{metadata::RowGroupMetaData, properties::WriterProperties},
68 format::{FileMetaData, KeyValue},
69};
70use arrow_array::RecordBatch;
71use arrow_schema::SchemaRef;
72use bytes::Bytes;
73use futures::future::BoxFuture;
74use futures::FutureExt;
75use std::mem;
76use tokio::io::{AsyncWrite, AsyncWriteExt};
77
78pub trait AsyncFileWriter: Send {
80 fn write(&mut self, bs: Bytes) -> BoxFuture<'_, Result<()>>;
87
88 fn complete(&mut self) -> BoxFuture<'_, Result<()>>;
92}
93
94impl AsyncFileWriter for Box<dyn AsyncFileWriter + '_> {
95 fn write(&mut self, bs: Bytes) -> BoxFuture<'_, Result<()>> {
96 self.as_mut().write(bs)
97 }
98
99 fn complete(&mut self) -> BoxFuture<'_, Result<()>> {
100 self.as_mut().complete()
101 }
102}
103
104impl<T: AsyncWrite + Unpin + Send> AsyncFileWriter for T {
105 fn write(&mut self, bs: Bytes) -> BoxFuture<'_, Result<()>> {
106 async move {
107 self.write_all(&bs).await?;
108 Ok(())
109 }
110 .boxed()
111 }
112
113 fn complete(&mut self) -> BoxFuture<'_, Result<()>> {
114 async move {
115 self.flush().await?;
116 self.shutdown().await?;
117 Ok(())
118 }
119 .boxed()
120 }
121}
122
123pub struct AsyncArrowWriter<W> {
152 sync_writer: ArrowWriter<Vec<u8>>,
154
155 async_writer: W,
157}
158
159impl<W: AsyncFileWriter> AsyncArrowWriter<W> {
160 pub fn try_new(
162 writer: W,
163 arrow_schema: SchemaRef,
164 props: Option<WriterProperties>,
165 ) -> Result<Self> {
166 let options = ArrowWriterOptions::new().with_properties(props.unwrap_or_default());
167 Self::try_new_with_options(writer, arrow_schema, options)
168 }
169
170 pub fn try_new_with_options(
172 writer: W,
173 arrow_schema: SchemaRef,
174 options: ArrowWriterOptions,
175 ) -> Result<Self> {
176 let sync_writer = ArrowWriter::try_new_with_options(Vec::new(), arrow_schema, options)?;
177
178 Ok(Self {
179 sync_writer,
180 async_writer: writer,
181 })
182 }
183
184 pub fn flushed_row_groups(&self) -> &[RowGroupMetaData] {
186 self.sync_writer.flushed_row_groups()
187 }
188
189 pub fn memory_size(&self) -> usize {
193 self.sync_writer.memory_size()
194 }
195
196 pub fn in_progress_size(&self) -> usize {
200 self.sync_writer.in_progress_size()
201 }
202
203 pub fn in_progress_rows(&self) -> usize {
205 self.sync_writer.in_progress_rows()
206 }
207
208 pub fn bytes_written(&self) -> usize {
210 self.sync_writer.bytes_written()
211 }
212
213 pub async fn write(&mut self, batch: &RecordBatch) -> Result<()> {
218 let before = self.sync_writer.flushed_row_groups().len();
219 self.sync_writer.write(batch)?;
220 if before != self.sync_writer.flushed_row_groups().len() {
221 self.do_write().await?;
222 }
223 Ok(())
224 }
225
226 pub async fn flush(&mut self) -> Result<()> {
228 self.sync_writer.flush()?;
229 self.do_write().await?;
230
231 Ok(())
232 }
233
234 pub fn append_key_value_metadata(&mut self, kv_metadata: KeyValue) {
238 self.sync_writer.append_key_value_metadata(kv_metadata);
239 }
240
241 pub async fn finish(&mut self) -> Result<FileMetaData> {
249 let metadata = self.sync_writer.finish()?;
250
251 self.do_write().await?;
253 self.async_writer.complete().await?;
254
255 Ok(metadata)
256 }
257
258 pub async fn close(mut self) -> Result<FileMetaData> {
262 self.finish().await
263 }
264
265 pub fn into_inner(self) -> W {
272 self.async_writer
273 }
274
275 async fn do_write(&mut self) -> Result<()> {
282 let buffer = mem::take(self.sync_writer.inner_mut());
283
284 self.async_writer
285 .write(Bytes::from(buffer))
286 .await
287 .map_err(|e| ParquetError::External(Box::new(e)))?;
288
289 Ok(())
290 }
291}
292
293#[cfg(test)]
294mod tests {
295 use arrow::datatypes::{DataType, Field, Schema};
296 use arrow_array::{ArrayRef, BinaryArray, Int32Array, Int64Array, RecordBatchReader};
297 use bytes::Bytes;
298 use std::sync::Arc;
299 use tokio::pin;
300
301 use crate::arrow::arrow_reader::{ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder};
302
303 use super::*;
304
305 fn get_test_reader() -> ParquetRecordBatchReader {
306 let testdata = arrow::util::test_util::parquet_test_data();
307 let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet");
309 let original_data = Bytes::from(std::fs::read(path).unwrap());
310 ParquetRecordBatchReaderBuilder::try_new(original_data)
311 .unwrap()
312 .build()
313 .unwrap()
314 }
315
316 #[tokio::test]
317 async fn test_async_writer() {
318 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
319 let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
320
321 let mut buffer = Vec::new();
322 let mut writer = AsyncArrowWriter::try_new(&mut buffer, to_write.schema(), None).unwrap();
323 writer.write(&to_write).await.unwrap();
324 writer.close().await.unwrap();
325
326 let buffer = Bytes::from(buffer);
327 let mut reader = ParquetRecordBatchReaderBuilder::try_new(buffer)
328 .unwrap()
329 .build()
330 .unwrap();
331 let read = reader.next().unwrap().unwrap();
332
333 assert_eq!(to_write, read);
334 }
335
336 #[tokio::test]
339 async fn test_async_writer_with_sync_writer() {
340 let reader = get_test_reader();
341
342 let write_props = WriterProperties::builder()
343 .set_max_row_group_size(64)
344 .build();
345
346 let mut async_buffer = Vec::new();
347 let mut async_writer = AsyncArrowWriter::try_new(
348 &mut async_buffer,
349 reader.schema(),
350 Some(write_props.clone()),
351 )
352 .unwrap();
353
354 let mut sync_buffer = Vec::new();
355 let mut sync_writer =
356 ArrowWriter::try_new(&mut sync_buffer, reader.schema(), Some(write_props)).unwrap();
357 for record_batch in reader {
358 let record_batch = record_batch.unwrap();
359 async_writer.write(&record_batch).await.unwrap();
360 sync_writer.write(&record_batch).unwrap();
361 }
362 sync_writer.close().unwrap();
363 async_writer.close().await.unwrap();
364
365 assert_eq!(sync_buffer, async_buffer);
366 }
367
368 struct TestAsyncSink {
369 sink: Vec<u8>,
370 min_accept_bytes: usize,
371 expect_total_bytes: usize,
372 }
373
374 impl AsyncWrite for TestAsyncSink {
375 fn poll_write(
376 self: std::pin::Pin<&mut Self>,
377 cx: &mut std::task::Context<'_>,
378 buf: &[u8],
379 ) -> std::task::Poll<std::result::Result<usize, std::io::Error>> {
380 let written_bytes = self.sink.len();
381 if written_bytes + buf.len() < self.expect_total_bytes {
382 assert!(buf.len() >= self.min_accept_bytes);
383 } else {
384 assert_eq!(written_bytes + buf.len(), self.expect_total_bytes);
385 }
386
387 let sink = &mut self.get_mut().sink;
388 pin!(sink);
389 sink.poll_write(cx, buf)
390 }
391
392 fn poll_flush(
393 self: std::pin::Pin<&mut Self>,
394 cx: &mut std::task::Context<'_>,
395 ) -> std::task::Poll<std::result::Result<(), std::io::Error>> {
396 let sink = &mut self.get_mut().sink;
397 pin!(sink);
398 sink.poll_flush(cx)
399 }
400
401 fn poll_shutdown(
402 self: std::pin::Pin<&mut Self>,
403 cx: &mut std::task::Context<'_>,
404 ) -> std::task::Poll<std::result::Result<(), std::io::Error>> {
405 let sink = &mut self.get_mut().sink;
406 pin!(sink);
407 sink.poll_shutdown(cx)
408 }
409 }
410
411 #[tokio::test]
412 async fn test_async_writer_bytes_written() {
413 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
414 let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
415
416 let temp = tempfile::tempfile().unwrap();
417
418 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
419 let mut writer =
420 AsyncArrowWriter::try_new(file.try_clone().await.unwrap(), to_write.schema(), None)
421 .unwrap();
422 writer.write(&to_write).await.unwrap();
423 let _metadata = writer.finish().await.unwrap();
424 let reported = writer.bytes_written();
426
427 let actual = file.metadata().await.unwrap().len() as usize;
429
430 assert_eq!(reported, actual);
431 }
432
433 #[tokio::test]
434 async fn test_async_writer_file() {
435 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
436 let col2 = Arc::new(BinaryArray::from_iter_values(vec![
437 vec![0; 500000],
438 vec![0; 500000],
439 vec![0; 500000],
440 ])) as ArrayRef;
441 let to_write = RecordBatch::try_from_iter([("col", col), ("col2", col2)]).unwrap();
442
443 let temp = tempfile::tempfile().unwrap();
444
445 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
446 let mut writer = AsyncArrowWriter::try_new(file, to_write.schema(), None).unwrap();
447 writer.write(&to_write).await.unwrap();
448 writer.close().await.unwrap();
449
450 let mut reader = ParquetRecordBatchReaderBuilder::try_new(temp)
451 .unwrap()
452 .build()
453 .unwrap();
454 let read = reader.next().unwrap().unwrap();
455
456 assert_eq!(to_write, read);
457 }
458
459 #[tokio::test]
460 async fn in_progress_accounting() {
461 let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
463
464 let a = Int32Array::from_value(0_i32, 512);
466
467 let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
469
470 let temp = tempfile::tempfile().unwrap();
471 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
472 let mut writer = AsyncArrowWriter::try_new(file, batch.schema(), None).unwrap();
473
474 assert_eq!(writer.in_progress_size(), 0);
476 assert_eq!(writer.in_progress_rows(), 0);
477 assert_eq!(writer.bytes_written(), 4); writer.write(&batch).await.unwrap();
479
480 let initial_size = writer.in_progress_size();
482 assert!(initial_size > 0);
483 assert_eq!(writer.in_progress_rows(), batch.num_rows());
484 let initial_memory = writer.memory_size();
485 assert!(
487 initial_size <= initial_memory,
488 "{initial_size} <= {initial_memory}"
489 );
490
491 writer.write(&batch).await.unwrap();
493 assert!(writer.in_progress_size() > initial_size);
494 assert_eq!(writer.in_progress_rows(), batch.num_rows() * 2);
495 assert!(writer.memory_size() > initial_memory);
496 assert!(
497 writer.in_progress_size() <= writer.memory_size(),
498 "in_progress_size {} <= memory_size {}",
499 writer.in_progress_size(),
500 writer.memory_size()
501 );
502
503 let pre_flush_bytes_written = writer.bytes_written();
505 writer.flush().await.unwrap();
506 assert_eq!(writer.in_progress_size(), 0);
507 assert_eq!(writer.memory_size(), 0);
508 assert_eq!(writer.in_progress_rows(), 0);
509 assert!(writer.bytes_written() > pre_flush_bytes_written);
510
511 writer.close().await.unwrap();
512 }
513}