26
26
//! # #[tokio::main(flavor="current_thread")]
27
27
//! # async fn main() {
28
28
//! #
29
- //! use std::sync::Arc;
30
- //! use arrow_array::{ArrayRef, Int64Array, RecordBatch, RecordBatchReader};
31
- //! use bytes::Bytes;
32
- //! use parquet::arrow::{AsyncArrowWriter, arrow_reader::ParquetRecordBatchReaderBuilder};
33
- //!
29
+ //! # use std::sync::Arc;
30
+ //! # use arrow_array::{ArrayRef, Int64Array, RecordBatch, RecordBatchReader};
31
+ //! # use bytes::Bytes;
32
+ //! # use parquet::arrow::{AsyncArrowWriter, arrow_reader::ParquetRecordBatchReaderBuilder};
33
+ //! #
34
34
//! let col = Arc::new(Int64Array::from_iter_values([1, 2, 3])) as ArrayRef;
35
35
//! let to_write = RecordBatch::try_from_iter([("col", col)]).unwrap();
36
36
//!
37
37
//! let mut buffer = Vec::new();
38
- //! let mut writer =
39
- //! AsyncArrowWriter::try_new(&mut buffer, to_write.schema(), 0, None).unwrap();
38
+ //! let mut writer = AsyncArrowWriter::try_new(&mut buffer, to_write.schema(), None).unwrap();
40
39
//! writer.write(&to_write).await.unwrap();
41
40
//! writer.close().await.unwrap();
42
41
//!
@@ -62,19 +61,19 @@ use arrow_array::RecordBatch;
62
61
use arrow_schema:: SchemaRef ;
63
62
use tokio:: io:: { AsyncWrite , AsyncWriteExt } ;
64
63
65
- /// Async arrow writer.
64
+ /// Encodes [`RecordBatch`] to parquet, outputting to an [`AsyncWrite`]
66
65
///
67
- /// It is implemented based on the sync writer [`ArrowWriter`] with an inner buffer.
68
- /// The buffered data will be flushed to the writer provided by caller when the
69
- /// buffer's threshold is exceeded.
66
+ /// ## Memory Usage
70
67
///
71
- /// ## Memory Limiting
68
+ /// This writer eagerly writes data as soon as possible to the underlying [`AsyncWrite`],
69
+ /// permitting fine-grained control over buffering and I/O scheduling. However, the columnar
70
+ /// nature of parquet forces data for an entire row group to be buffered in memory, before
71
+ /// it can be flushed. Depending on the data and the configured row group size, this buffering
72
+ /// may be substantial.
72
73
///
73
- /// The nature of parquet forces buffering of an entire row group before it can be flushed
74
- /// to the underlying writer. This buffering may exceed the configured buffer size
75
- /// of [`AsyncArrowWriter`]. Memory usage can be limited by prematurely flushing the row group,
76
- /// although this will have implications for file size and query performance. See [ArrowWriter]
77
- /// for more information.
74
+ /// Memory usage can be limited by calling [`Self::flush`] to flush the in progress row group,
75
+ /// although this will likely increase overall file size and reduce query performance.
76
+ /// See [ArrowWriter] for more information.
78
77
///
79
78
/// ```no_run
80
79
/// # use tokio::fs::File;
@@ -96,50 +95,30 @@ pub struct AsyncArrowWriter<W> {
96
95
97
96
/// Async writer provided by caller
98
97
async_writer : W ,
99
-
100
- /// Trigger forced flushing once buffer size reaches this value
101
- buffer_size : usize ,
102
98
}
103
99
104
100
impl < W : AsyncWrite + Unpin + Send > AsyncArrowWriter < W > {
105
- /// Try to create a new Async Arrow Writer.
106
- ///
107
- /// `buffer_size` determines the minimum number of bytes to buffer before flushing
108
- /// to the underlying [`AsyncWrite`]. However, the nature of writing parquet may
109
- /// force buffering of data in excess of this within the underlying [`ArrowWriter`].
110
- /// See the documentation on [`ArrowWriter`] for more details
101
+ /// Try to create a new Async Arrow Writer
111
102
pub fn try_new (
112
103
writer : W ,
113
104
arrow_schema : SchemaRef ,
114
- buffer_size : usize ,
115
105
props : Option < WriterProperties > ,
116
106
) -> Result < Self > {
117
107
let options = ArrowWriterOptions :: new ( ) . with_properties ( props. unwrap_or_default ( ) ) ;
118
- Self :: try_new_with_options ( writer, arrow_schema, buffer_size , options)
108
+ Self :: try_new_with_options ( writer, arrow_schema, options)
119
109
}
120
110
121
- /// Try to create a new Async Arrow Writer with [`ArrowWriterOptions`].
122
- ///
123
- /// `buffer_size` determines the minimum number of bytes to buffer before flushing
124
- /// to the underlying [`AsyncWrite`]. However, the nature of writing parquet may
125
- /// force buffering of data in excess of this within the underlying [`ArrowWriter`].
126
- /// See the documentation on [`ArrowWriter`] for more details
111
+ /// Try to create a new Async Arrow Writer with [`ArrowWriterOptions`]
127
112
pub fn try_new_with_options (
128
113
writer : W ,
129
114
arrow_schema : SchemaRef ,
130
- buffer_size : usize ,
131
115
options : ArrowWriterOptions ,
132
116
) -> Result < Self > {
133
- let sync_writer = ArrowWriter :: try_new_with_options (
134
- Vec :: with_capacity ( buffer_size) ,
135
- arrow_schema,
136
- options,
137
- ) ?;
117
+ let sync_writer = ArrowWriter :: try_new_with_options ( Vec :: new ( ) , arrow_schema, options) ?;
138
118
139
119
Ok ( Self {
140
120
sync_writer,
141
121
async_writer : writer,
142
- buffer_size,
143
122
} )
144
123
}
145
124
@@ -168,14 +147,18 @@ impl<W: AsyncWrite + Unpin + Send> AsyncArrowWriter<W> {
168
147
/// After every sync write by the inner [ArrowWriter], the inner buffer will be
169
148
/// checked and flush if at least half full
170
149
pub async fn write ( & mut self , batch : & RecordBatch ) -> Result < ( ) > {
150
+ let before = self . sync_writer . flushed_row_groups ( ) . len ( ) ;
171
151
self . sync_writer . write ( batch) ?;
172
- self . try_flush ( false ) . await
152
+ if before != self . sync_writer . flushed_row_groups ( ) . len ( ) {
153
+ self . do_write ( ) . await ?;
154
+ }
155
+ Ok ( ( ) )
173
156
}
174
157
175
158
/// Flushes all buffered rows into a new row group
176
159
pub async fn flush ( & mut self ) -> Result < ( ) > {
177
160
self . sync_writer . flush ( ) ?;
178
- self . try_flush ( false ) . await ?;
161
+ self . do_write ( ) . await ?;
179
162
180
163
Ok ( ( ) )
181
164
}
@@ -194,19 +177,15 @@ impl<W: AsyncWrite + Unpin + Send> AsyncArrowWriter<W> {
194
177
let metadata = self . sync_writer . finish ( ) ?;
195
178
196
179
// Force to flush the remaining data.
197
- self . try_flush ( true ) . await ?;
180
+ self . do_write ( ) . await ?;
198
181
self . async_writer . shutdown ( ) . await ?;
199
182
200
183
Ok ( metadata)
201
184
}
202
185
203
- /// Flush the buffered data into the `async_writer`
204
- async fn try_flush ( & mut self , force : bool ) -> Result < ( ) > {
186
+ /// Flush the data written by `sync_writer` into the `async_writer`
187
+ async fn do_write ( & mut self ) -> Result < ( ) > {
205
188
let buffer = self . sync_writer . inner_mut ( ) ;
206
- if !force && ( buffer. is_empty ( ) || buffer. len ( ) < self . buffer_size ) {
207
- // no need to flush
208
- return Ok ( ( ) ) ;
209
- }
210
189
211
190
self . async_writer
212
191
. write_all ( buffer. as_slice ( ) )
@@ -254,8 +233,7 @@ mod tests {
254
233
let to_write = RecordBatch :: try_from_iter ( [ ( "col" , col) ] ) . unwrap ( ) ;
255
234
256
235
let mut buffer = Vec :: new ( ) ;
257
- let mut writer =
258
- AsyncArrowWriter :: try_new ( & mut buffer, to_write. schema ( ) , 0 , None ) . unwrap ( ) ;
236
+ let mut writer = AsyncArrowWriter :: try_new ( & mut buffer, to_write. schema ( ) , None ) . unwrap ( ) ;
259
237
writer. write ( & to_write) . await . unwrap ( ) ;
260
238
writer. close ( ) . await . unwrap ( ) ;
261
239
@@ -283,7 +261,6 @@ mod tests {
283
261
let mut async_writer = AsyncArrowWriter :: try_new (
284
262
& mut async_buffer,
285
263
reader. schema ( ) ,
286
- 1024 ,
287
264
Some ( write_props. clone ( ) ) ,
288
265
)
289
266
. unwrap ( ) ;
@@ -345,54 +322,6 @@ mod tests {
345
322
}
346
323
}
347
324
348
- #[ tokio:: test]
349
- async fn test_async_writer_with_buffer_flush_threshold ( ) {
350
- let write_props = WriterProperties :: builder ( )
351
- . set_max_row_group_size ( 2048 )
352
- . build ( ) ;
353
- let expect_encode_size = {
354
- let reader = get_test_reader ( ) ;
355
- let mut buffer = Vec :: new ( ) ;
356
- let mut async_writer = AsyncArrowWriter :: try_new (
357
- & mut buffer,
358
- reader. schema ( ) ,
359
- 0 ,
360
- Some ( write_props. clone ( ) ) ,
361
- )
362
- . unwrap ( ) ;
363
- for record_batch in reader {
364
- let record_batch = record_batch. unwrap ( ) ;
365
- async_writer. write ( & record_batch) . await . unwrap ( ) ;
366
- }
367
- async_writer. close ( ) . await . unwrap ( ) ;
368
- buffer. len ( )
369
- } ;
370
-
371
- let test_buffer_flush_thresholds = vec ! [ 0 , 1024 , 40 * 1024 , 50 * 1024 , 100 * 1024 ] ;
372
-
373
- for buffer_flush_threshold in test_buffer_flush_thresholds {
374
- let reader = get_test_reader ( ) ;
375
- let mut test_async_sink = TestAsyncSink {
376
- sink : Vec :: new ( ) ,
377
- min_accept_bytes : buffer_flush_threshold,
378
- expect_total_bytes : expect_encode_size,
379
- } ;
380
- let mut async_writer = AsyncArrowWriter :: try_new (
381
- & mut test_async_sink,
382
- reader. schema ( ) ,
383
- buffer_flush_threshold * 2 ,
384
- Some ( write_props. clone ( ) ) ,
385
- )
386
- . unwrap ( ) ;
387
-
388
- for record_batch in reader {
389
- let record_batch = record_batch. unwrap ( ) ;
390
- async_writer. write ( & record_batch) . await . unwrap ( ) ;
391
- }
392
- async_writer. close ( ) . await . unwrap ( ) ;
393
- }
394
- }
395
-
396
325
#[ tokio:: test]
397
326
async fn test_async_writer_file ( ) {
398
327
let col = Arc :: new ( Int64Array :: from_iter_values ( [ 1 , 2 , 3 ] ) ) as ArrayRef ;
@@ -406,7 +335,7 @@ mod tests {
406
335
let temp = tempfile:: tempfile ( ) . unwrap ( ) ;
407
336
408
337
let file = tokio:: fs:: File :: from_std ( temp. try_clone ( ) . unwrap ( ) ) ;
409
- let mut writer = AsyncArrowWriter :: try_new ( file, to_write. schema ( ) , 0 , None ) . unwrap ( ) ;
338
+ let mut writer = AsyncArrowWriter :: try_new ( file, to_write. schema ( ) , None ) . unwrap ( ) ;
410
339
writer. write ( & to_write) . await . unwrap ( ) ;
411
340
writer. close ( ) . await . unwrap ( ) ;
412
341
@@ -430,36 +359,33 @@ mod tests {
430
359
// build a record batch
431
360
let batch = RecordBatch :: try_new ( Arc :: new ( schema) , vec ! [ Arc :: new( a) ] ) . unwrap ( ) ;
432
361
433
- for buffer_size in [ 0 , 8 , 1024 ] {
434
- let temp = tempfile:: tempfile ( ) . unwrap ( ) ;
435
- let file = tokio:: fs:: File :: from_std ( temp. try_clone ( ) . unwrap ( ) ) ;
436
- let mut writer =
437
- AsyncArrowWriter :: try_new ( file, batch. schema ( ) , buffer_size, None ) . unwrap ( ) ;
438
-
439
- // starts empty
440
- assert_eq ! ( writer. in_progress_size( ) , 0 ) ;
441
- assert_eq ! ( writer. in_progress_rows( ) , 0 ) ;
442
- assert_eq ! ( writer. bytes_written( ) , 4 ) ; // Initial Parquet header
443
- writer. write ( & batch) . await . unwrap ( ) ;
444
-
445
- // updated on write
446
- let initial_size = writer. in_progress_size ( ) ;
447
- assert ! ( initial_size > 0 ) ;
448
- assert_eq ! ( writer. in_progress_rows( ) , batch. num_rows( ) ) ;
449
-
450
- // updated on second write
451
- writer. write ( & batch) . await . unwrap ( ) ;
452
- assert ! ( writer. in_progress_size( ) > initial_size) ;
453
- assert_eq ! ( writer. in_progress_rows( ) , batch. num_rows( ) * 2 ) ;
454
-
455
- // in progress tracking is cleared, but the overall data written is updated
456
- let pre_flush_bytes_written = writer. bytes_written ( ) ;
457
- writer. flush ( ) . await . unwrap ( ) ;
458
- assert_eq ! ( writer. in_progress_size( ) , 0 ) ;
459
- assert_eq ! ( writer. in_progress_rows( ) , 0 ) ;
460
- assert ! ( writer. bytes_written( ) > pre_flush_bytes_written) ;
461
-
462
- writer. close ( ) . await . unwrap ( ) ;
463
- }
362
+ let temp = tempfile:: tempfile ( ) . unwrap ( ) ;
363
+ let file = tokio:: fs:: File :: from_std ( temp. try_clone ( ) . unwrap ( ) ) ;
364
+ let mut writer = AsyncArrowWriter :: try_new ( file, batch. schema ( ) , None ) . unwrap ( ) ;
365
+
366
+ // starts empty
367
+ assert_eq ! ( writer. in_progress_size( ) , 0 ) ;
368
+ assert_eq ! ( writer. in_progress_rows( ) , 0 ) ;
369
+ assert_eq ! ( writer. bytes_written( ) , 4 ) ; // Initial Parquet header
370
+ writer. write ( & batch) . await . unwrap ( ) ;
371
+
372
+ // updated on write
373
+ let initial_size = writer. in_progress_size ( ) ;
374
+ assert ! ( initial_size > 0 ) ;
375
+ assert_eq ! ( writer. in_progress_rows( ) , batch. num_rows( ) ) ;
376
+
377
+ // updated on second write
378
+ writer. write ( & batch) . await . unwrap ( ) ;
379
+ assert ! ( writer. in_progress_size( ) > initial_size) ;
380
+ assert_eq ! ( writer. in_progress_rows( ) , batch. num_rows( ) * 2 ) ;
381
+
382
+ // in progress tracking is cleared, but the overall data written is updated
383
+ let pre_flush_bytes_written = writer. bytes_written ( ) ;
384
+ writer. flush ( ) . await . unwrap ( ) ;
385
+ assert_eq ! ( writer. in_progress_size( ) , 0 ) ;
386
+ assert_eq ! ( writer. in_progress_rows( ) , 0 ) ;
387
+ assert ! ( writer. bytes_written( ) > pre_flush_bytes_written) ;
388
+
389
+ writer. close ( ) . await . unwrap ( ) ;
464
390
}
465
391
}
0 commit comments