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 async fn do_write(&mut self) -> Result<()> {
272 let buffer = mem::take(self.sync_writer.inner_mut());
273
274 self.async_writer
275 .write(Bytes::from(buffer))
276 .await
277 .map_err(|e| ParquetError::External(Box::new(e)))?;
278
279 Ok(())
280 }
281}
282
283#[cfg(test)]
284mod tests {
285 use arrow::datatypes::{DataType, Field, Schema};
286 use arrow_array::{ArrayRef, BinaryArray, Int32Array, Int64Array, RecordBatchReader};
287 use bytes::Bytes;
288 use std::sync::Arc;
289 use tokio::pin;
290
291 use crate::arrow::arrow_reader::{ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder};
292
293 use super::*;
294
295 fn get_test_reader() -> ParquetRecordBatchReader {
296 let testdata = arrow::util::test_util::parquet_test_data();
297 let path = format!("{}/alltypes_tiny_pages_plain.parquet", testdata);
299 let original_data = Bytes::from(std::fs::read(path).unwrap());
300 ParquetRecordBatchReaderBuilder::try_new(original_data)
301 .unwrap()
302 .build()
303 .unwrap()
304 }
305
306 #[tokio::test]
307 async fn test_async_writer() {
308 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
309 let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
310
311 let mut buffer = Vec::new();
312 let mut writer = AsyncArrowWriter::try_new(&mut buffer, to_write.schema(), None).unwrap();
313 writer.write(&to_write).await.unwrap();
314 writer.close().await.unwrap();
315
316 let buffer = Bytes::from(buffer);
317 let mut reader = ParquetRecordBatchReaderBuilder::try_new(buffer)
318 .unwrap()
319 .build()
320 .unwrap();
321 let read = reader.next().unwrap().unwrap();
322
323 assert_eq!(to_write, read);
324 }
325
326 #[tokio::test]
329 async fn test_async_writer_with_sync_writer() {
330 let reader = get_test_reader();
331
332 let write_props = WriterProperties::builder()
333 .set_max_row_group_size(64)
334 .build();
335
336 let mut async_buffer = Vec::new();
337 let mut async_writer = AsyncArrowWriter::try_new(
338 &mut async_buffer,
339 reader.schema(),
340 Some(write_props.clone()),
341 )
342 .unwrap();
343
344 let mut sync_buffer = Vec::new();
345 let mut sync_writer =
346 ArrowWriter::try_new(&mut sync_buffer, reader.schema(), Some(write_props)).unwrap();
347 for record_batch in reader {
348 let record_batch = record_batch.unwrap();
349 async_writer.write(&record_batch).await.unwrap();
350 sync_writer.write(&record_batch).unwrap();
351 }
352 sync_writer.close().unwrap();
353 async_writer.close().await.unwrap();
354
355 assert_eq!(sync_buffer, async_buffer);
356 }
357
358 struct TestAsyncSink {
359 sink: Vec<u8>,
360 min_accept_bytes: usize,
361 expect_total_bytes: usize,
362 }
363
364 impl AsyncWrite for TestAsyncSink {
365 fn poll_write(
366 self: std::pin::Pin<&mut Self>,
367 cx: &mut std::task::Context<'_>,
368 buf: &[u8],
369 ) -> std::task::Poll<std::result::Result<usize, std::io::Error>> {
370 let written_bytes = self.sink.len();
371 if written_bytes + buf.len() < self.expect_total_bytes {
372 assert!(buf.len() >= self.min_accept_bytes);
373 } else {
374 assert_eq!(written_bytes + buf.len(), self.expect_total_bytes);
375 }
376
377 let sink = &mut self.get_mut().sink;
378 pin!(sink);
379 sink.poll_write(cx, buf)
380 }
381
382 fn poll_flush(
383 self: std::pin::Pin<&mut Self>,
384 cx: &mut std::task::Context<'_>,
385 ) -> std::task::Poll<std::result::Result<(), std::io::Error>> {
386 let sink = &mut self.get_mut().sink;
387 pin!(sink);
388 sink.poll_flush(cx)
389 }
390
391 fn poll_shutdown(
392 self: std::pin::Pin<&mut Self>,
393 cx: &mut std::task::Context<'_>,
394 ) -> std::task::Poll<std::result::Result<(), std::io::Error>> {
395 let sink = &mut self.get_mut().sink;
396 pin!(sink);
397 sink.poll_shutdown(cx)
398 }
399 }
400
401 #[tokio::test]
402 async fn test_async_writer_bytes_written() {
403 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
404 let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
405
406 let temp = tempfile::tempfile().unwrap();
407
408 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
409 let mut writer =
410 AsyncArrowWriter::try_new(file.try_clone().await.unwrap(), to_write.schema(), None)
411 .unwrap();
412 writer.write(&to_write).await.unwrap();
413 let _metadata = writer.finish().await.unwrap();
414 let reported = writer.bytes_written();
416
417 let actual = file.metadata().await.unwrap().len() as usize;
419
420 assert_eq!(reported, actual);
421 }
422
423 #[tokio::test]
424 async fn test_async_writer_file() {
425 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
426 let col2 = Arc::new(BinaryArray::from_iter_values(vec![
427 vec![0; 500000],
428 vec![0; 500000],
429 vec![0; 500000],
430 ])) as ArrayRef;
431 let to_write = RecordBatch::try_from_iter([("col", col), ("col2", col2)]).unwrap();
432
433 let temp = tempfile::tempfile().unwrap();
434
435 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
436 let mut writer = AsyncArrowWriter::try_new(file, to_write.schema(), None).unwrap();
437 writer.write(&to_write).await.unwrap();
438 writer.close().await.unwrap();
439
440 let mut reader = ParquetRecordBatchReaderBuilder::try_new(temp)
441 .unwrap()
442 .build()
443 .unwrap();
444 let read = reader.next().unwrap().unwrap();
445
446 assert_eq!(to_write, read);
447 }
448
449 #[tokio::test]
450 async fn in_progress_accounting() {
451 let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
453
454 let a = Int32Array::from_value(0_i32, 512);
456
457 let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
459
460 let temp = tempfile::tempfile().unwrap();
461 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
462 let mut writer = AsyncArrowWriter::try_new(file, batch.schema(), None).unwrap();
463
464 assert_eq!(writer.in_progress_size(), 0);
466 assert_eq!(writer.in_progress_rows(), 0);
467 assert_eq!(writer.bytes_written(), 4); writer.write(&batch).await.unwrap();
469
470 let initial_size = writer.in_progress_size();
472 assert!(initial_size > 0);
473 assert_eq!(writer.in_progress_rows(), batch.num_rows());
474 let initial_memory = writer.memory_size();
475 assert!(
477 initial_size <= initial_memory,
478 "{initial_size} <= {initial_memory}"
479 );
480
481 writer.write(&batch).await.unwrap();
483 assert!(writer.in_progress_size() > initial_size);
484 assert_eq!(writer.in_progress_rows(), batch.num_rows() * 2);
485 assert!(writer.memory_size() > initial_memory);
486 assert!(
487 writer.in_progress_size() <= writer.memory_size(),
488 "in_progress_size {} <= memory_size {}",
489 writer.in_progress_size(),
490 writer.memory_size()
491 );
492
493 let pre_flush_bytes_written = writer.bytes_written();
495 writer.flush().await.unwrap();
496 assert_eq!(writer.in_progress_size(), 0);
497 assert_eq!(writer.memory_size(), 0);
498 assert_eq!(writer.in_progress_rows(), 0);
499 assert!(writer.bytes_written() > pre_flush_bytes_written);
500
501 writer.close().await.unwrap();
502 }
503}