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
300 use crate::arrow::arrow_reader::{ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder};
301
302 use super::*;
303
304 fn get_test_reader() -> ParquetRecordBatchReader {
305 let testdata = arrow::util::test_util::parquet_test_data();
306 let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet");
308 let original_data = Bytes::from(std::fs::read(path).unwrap());
309 ParquetRecordBatchReaderBuilder::try_new(original_data)
310 .unwrap()
311 .build()
312 .unwrap()
313 }
314
315 #[tokio::test]
316 async fn test_async_writer() {
317 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
318 let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
319
320 let mut buffer = Vec::new();
321 let mut writer = AsyncArrowWriter::try_new(&mut buffer, to_write.schema(), None).unwrap();
322 writer.write(&to_write).await.unwrap();
323 writer.close().await.unwrap();
324
325 let buffer = Bytes::from(buffer);
326 let mut reader = ParquetRecordBatchReaderBuilder::try_new(buffer)
327 .unwrap()
328 .build()
329 .unwrap();
330 let read = reader.next().unwrap().unwrap();
331
332 assert_eq!(to_write, read);
333 }
334
335 #[tokio::test]
338 async fn test_async_writer_with_sync_writer() {
339 let reader = get_test_reader();
340
341 let write_props = WriterProperties::builder()
342 .set_max_row_group_size(64)
343 .build();
344
345 let mut async_buffer = Vec::new();
346 let mut async_writer = AsyncArrowWriter::try_new(
347 &mut async_buffer,
348 reader.schema(),
349 Some(write_props.clone()),
350 )
351 .unwrap();
352
353 let mut sync_buffer = Vec::new();
354 let mut sync_writer =
355 ArrowWriter::try_new(&mut sync_buffer, reader.schema(), Some(write_props)).unwrap();
356 for record_batch in reader {
357 let record_batch = record_batch.unwrap();
358 async_writer.write(&record_batch).await.unwrap();
359 sync_writer.write(&record_batch).unwrap();
360 }
361 sync_writer.close().unwrap();
362 async_writer.close().await.unwrap();
363
364 assert_eq!(sync_buffer, async_buffer);
365 }
366
367 #[tokio::test]
368 async fn test_async_writer_bytes_written() {
369 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
370 let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
371
372 let temp = tempfile::tempfile().unwrap();
373
374 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
375 let mut writer =
376 AsyncArrowWriter::try_new(file.try_clone().await.unwrap(), to_write.schema(), None)
377 .unwrap();
378 writer.write(&to_write).await.unwrap();
379 let _metadata = writer.finish().await.unwrap();
380 let reported = writer.bytes_written();
382
383 let actual = file.metadata().await.unwrap().len() as usize;
385
386 assert_eq!(reported, actual);
387 }
388
389 #[tokio::test]
390 async fn test_async_writer_file() {
391 let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
392 let col2 = Arc::new(BinaryArray::from_iter_values(vec![
393 vec![0; 500000],
394 vec![0; 500000],
395 vec![0; 500000],
396 ])) as ArrayRef;
397 let to_write = RecordBatch::try_from_iter([("col", col), ("col2", col2)]).unwrap();
398
399 let temp = tempfile::tempfile().unwrap();
400
401 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
402 let mut writer = AsyncArrowWriter::try_new(file, to_write.schema(), None).unwrap();
403 writer.write(&to_write).await.unwrap();
404 writer.close().await.unwrap();
405
406 let mut reader = ParquetRecordBatchReaderBuilder::try_new(temp)
407 .unwrap()
408 .build()
409 .unwrap();
410 let read = reader.next().unwrap().unwrap();
411
412 assert_eq!(to_write, read);
413 }
414
415 #[tokio::test]
416 async fn in_progress_accounting() {
417 let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
419
420 let a = Int32Array::from_value(0_i32, 512);
422
423 let batch = RecordBatch::try_new(Arc::new(schema), vec![Arc::new(a)]).unwrap();
425
426 let temp = tempfile::tempfile().unwrap();
427 let file = tokio::fs::File::from_std(temp.try_clone().unwrap());
428 let mut writer = AsyncArrowWriter::try_new(file, batch.schema(), None).unwrap();
429
430 assert_eq!(writer.in_progress_size(), 0);
432 assert_eq!(writer.in_progress_rows(), 0);
433 assert_eq!(writer.bytes_written(), 4); writer.write(&batch).await.unwrap();
435
436 let initial_size = writer.in_progress_size();
438 assert!(initial_size > 0);
439 assert_eq!(writer.in_progress_rows(), batch.num_rows());
440 let initial_memory = writer.memory_size();
441 assert!(
443 initial_size <= initial_memory,
444 "{initial_size} <= {initial_memory}"
445 );
446
447 writer.write(&batch).await.unwrap();
449 assert!(writer.in_progress_size() > initial_size);
450 assert_eq!(writer.in_progress_rows(), batch.num_rows() * 2);
451 assert!(writer.memory_size() > initial_memory);
452 assert!(
453 writer.in_progress_size() <= writer.memory_size(),
454 "in_progress_size {} <= memory_size {}",
455 writer.in_progress_size(),
456 writer.memory_size()
457 );
458
459 let pre_flush_bytes_written = writer.bytes_written();
461 writer.flush().await.unwrap();
462 assert_eq!(writer.in_progress_size(), 0);
463 assert_eq!(writer.memory_size(), 0);
464 assert_eq!(writer.in_progress_rows(), 0);
465 assert!(writer.bytes_written() > pre_flush_bytes_written);
466
467 writer.close().await.unwrap();
468 }
469}