1
1
//! Abstractions and implementations for writing data to delta tables
2
2
3
3
use std:: collections:: HashMap ;
4
+ use std:: future:: Future ;
5
+ use std:: sync:: OnceLock ;
4
6
5
7
use arrow_array:: RecordBatch ;
6
8
use arrow_schema:: { ArrowError , SchemaRef as ArrowSchemaRef } ;
@@ -9,25 +11,50 @@ use delta_kernel::expressions::Scalar;
9
11
use futures:: { StreamExt , TryStreamExt } ;
10
12
use indexmap:: IndexMap ;
11
13
use object_store:: { path:: Path , ObjectStore } ;
12
- use parquet:: arrow:: ArrowWriter ;
14
+ use parquet:: arrow:: AsyncArrowWriter ;
13
15
use parquet:: basic:: Compression ;
14
16
use parquet:: file:: properties:: WriterProperties ;
17
+ use tokio:: task:: { JoinError , JoinSet } ;
15
18
use tracing:: debug;
16
19
17
20
use crate :: crate_version;
18
21
use crate :: errors:: { DeltaResult , DeltaTableError } ;
19
22
use crate :: kernel:: { Add , PartitionsExt } ;
20
23
use crate :: storage:: ObjectStoreRef ;
24
+ use crate :: writer:: async_utils:: AsyncShareableBuffer ;
21
25
use crate :: writer:: record_batch:: { divide_by_partition_values, PartitionResult } ;
22
26
use crate :: writer:: stats:: create_add;
23
27
use crate :: writer:: utils:: {
24
28
arrow_schema_without_partitions, next_data_path, record_batch_without_partitions,
25
- ShareableBuffer ,
26
29
} ;
27
30
28
31
// TODO databricks often suggests a file size of 100mb, should we set this default?
29
32
const DEFAULT_TARGET_FILE_SIZE : usize = 104_857_600 ;
30
33
const DEFAULT_WRITE_BATCH_SIZE : usize = 1024 ;
34
+ const DEFAULT_UPLOAD_PART_SIZE : usize = 1024 * 1024 * 5 ;
35
+
36
+ fn upload_part_size ( ) -> usize {
37
+ static UPLOAD_SIZE : OnceLock < usize > = OnceLock :: new ( ) ;
38
+ * UPLOAD_SIZE . get_or_init ( || {
39
+ std:: env:: var ( "DELTARS_UPLOAD_PART_SIZE" )
40
+ . ok ( )
41
+ . and_then ( |s| s. parse :: < usize > ( ) . ok ( ) )
42
+ . map ( |size| {
43
+ if size < DEFAULT_UPLOAD_PART_SIZE {
44
+ // Minimum part size in GCS and S3
45
+ debug ! ( "DELTARS_UPLOAD_PART_SIZE must be at least 5MB, therefore falling back on default of 5MB." ) ;
46
+ DEFAULT_UPLOAD_PART_SIZE
47
+ } else if size > 1024 * 1024 * 1024 * 5 {
48
+ // Maximum part size in GCS and S3
49
+ debug ! ( "DELTARS_UPLOAD_PART_SIZE must not be higher than 5GB, therefore capping it at 5GB." ) ;
50
+ 1024 * 1024 * 1024 * 5
51
+ } else {
52
+ size
53
+ }
54
+ } )
55
+ . unwrap_or ( DEFAULT_UPLOAD_PART_SIZE )
56
+ } )
57
+ }
31
58
32
59
#[ derive( thiserror:: Error , Debug ) ]
33
60
enum WriteError {
@@ -122,7 +149,6 @@ impl WriterConfig {
122
149
}
123
150
}
124
151
125
- #[ derive( Debug ) ]
126
152
/// A parquet writer implementation tailored to the needs of writing data to a delta table.
127
153
pub struct DeltaWriter {
128
154
/// An object store pointing at Delta table root
@@ -286,13 +312,12 @@ impl PartitionWriterConfig {
286
312
/// This writer takes in table data as RecordBatches and writes it out to partitioned parquet files.
287
313
/// It buffers data in memory until it reaches a certain size, then writes it out to optimize file sizes.
288
314
/// When you complete writing you get back a list of Add actions that can be used to update the Delta table commit log.
289
- #[ derive( Debug ) ]
290
315
pub struct PartitionWriter {
291
316
object_store : ObjectStoreRef ,
292
317
writer_id : uuid:: Uuid ,
293
318
config : PartitionWriterConfig ,
294
- buffer : ShareableBuffer ,
295
- arrow_writer : ArrowWriter < ShareableBuffer > ,
319
+ buffer : AsyncShareableBuffer ,
320
+ arrow_writer : AsyncArrowWriter < AsyncShareableBuffer > ,
296
321
part_counter : usize ,
297
322
files_written : Vec < Add > ,
298
323
/// Num index cols to collect stats for
@@ -309,8 +334,8 @@ impl PartitionWriter {
309
334
num_indexed_cols : i32 ,
310
335
stats_columns : Option < Vec < String > > ,
311
336
) -> DeltaResult < Self > {
312
- let buffer = ShareableBuffer :: default ( ) ;
313
- let arrow_writer = ArrowWriter :: try_new (
337
+ let buffer = AsyncShareableBuffer :: default ( ) ;
338
+ let arrow_writer = AsyncArrowWriter :: try_new (
314
339
buffer. clone ( ) ,
315
340
config. file_schema . clone ( ) ,
316
341
Some ( config. writer_properties . clone ( ) ) ,
@@ -340,9 +365,11 @@ impl PartitionWriter {
340
365
)
341
366
}
342
367
343
- fn reset_writer ( & mut self ) -> DeltaResult < ( ArrowWriter < ShareableBuffer > , ShareableBuffer ) > {
344
- let new_buffer = ShareableBuffer :: default ( ) ;
345
- let arrow_writer = ArrowWriter :: try_new (
368
+ fn reset_writer (
369
+ & mut self ,
370
+ ) -> DeltaResult < ( AsyncArrowWriter < AsyncShareableBuffer > , AsyncShareableBuffer ) > {
371
+ let new_buffer = AsyncShareableBuffer :: default ( ) ;
372
+ let arrow_writer = AsyncArrowWriter :: try_new (
346
373
new_buffer. clone ( ) ,
347
374
self . config . file_schema . clone ( ) ,
348
375
Some ( self . config . writer_properties . clone ( ) ) ,
@@ -353,20 +380,20 @@ impl PartitionWriter {
353
380
) )
354
381
}
355
382
356
- fn write_batch ( & mut self , batch : & RecordBatch ) -> DeltaResult < ( ) > {
357
- Ok ( self . arrow_writer . write ( batch) ?)
383
+ async fn write_batch ( & mut self , batch : & RecordBatch ) -> DeltaResult < ( ) > {
384
+ Ok ( self . arrow_writer . write ( batch) . await ?)
358
385
}
359
386
360
387
async fn flush_arrow_writer ( & mut self ) -> DeltaResult < ( ) > {
361
388
// replace counter / buffers and close the current writer
362
389
let ( writer, buffer) = self . reset_writer ( ) ?;
363
- let metadata = writer. close ( ) ?;
390
+ let metadata = writer. close ( ) . await ?;
364
391
// don't write empty file
365
392
if metadata. num_rows == 0 {
366
393
return Ok ( ( ) ) ;
367
394
}
368
395
369
- let buffer = match buffer. into_inner ( ) {
396
+ let mut buffer = match buffer. into_inner ( ) . await {
370
397
Some ( buffer) => Bytes :: from ( buffer) ,
371
398
None => return Ok ( ( ) ) , // Nothing to write
372
399
} ;
@@ -376,7 +403,33 @@ impl PartitionWriter {
376
403
let file_size = buffer. len ( ) as i64 ;
377
404
378
405
// write file to object store
379
- self . object_store . put ( & path, buffer. into ( ) ) . await ?;
406
+ let mut multi_part_upload = self . object_store . put_multipart ( & path) . await ?;
407
+ let part_size = upload_part_size ( ) ;
408
+ let mut tasks = JoinSet :: new ( ) ;
409
+ let max_concurrent_tasks = 10 ; // TODO: make configurable
410
+
411
+ while buffer. len ( ) > part_size {
412
+ let part = buffer. split_to ( part_size) ;
413
+ let upload_future = multi_part_upload. put_part ( part. into ( ) ) ;
414
+
415
+ // wait until one spot frees up before spawning new task
416
+ if tasks. len ( ) >= max_concurrent_tasks {
417
+ tasks. join_next ( ) . await ;
418
+ }
419
+ tasks. spawn ( upload_future) ;
420
+ }
421
+
422
+ if !buffer. is_empty ( ) {
423
+ let upload_future = multi_part_upload. put_part ( buffer. into ( ) ) ;
424
+ tasks. spawn ( upload_future) ;
425
+ }
426
+
427
+ // wait for all remaining tasks to complete
428
+ while let Some ( result) = tasks. join_next ( ) . await {
429
+ result. map_err ( |e| DeltaTableError :: generic ( e. to_string ( ) ) ) ??;
430
+ }
431
+
432
+ multi_part_upload. complete ( ) . await ?;
380
433
381
434
self . files_written . push (
382
435
create_add (
@@ -412,9 +465,9 @@ impl PartitionWriter {
412
465
let max_offset = batch. num_rows ( ) ;
413
466
for offset in ( 0 ..max_offset) . step_by ( self . config . write_batch_size ) {
414
467
let length = usize:: min ( self . config . write_batch_size , max_offset - offset) ;
415
- self . write_batch ( & batch. slice ( offset, length) ) ?;
468
+ self . write_batch ( & batch. slice ( offset, length) ) . await ?;
416
469
// flush currently buffered data to disk once we meet or exceed the target file size.
417
- let estimated_size = self . buffer . len ( ) + self . arrow_writer . in_progress_size ( ) ;
470
+ let estimated_size = self . buffer . len ( ) . await + self . arrow_writer . in_progress_size ( ) ;
418
471
if estimated_size >= self . config . target_file_size {
419
472
debug ! (
420
473
"Writing file with estimated size {:?} to disk." ,
0 commit comments