-
Notifications
You must be signed in to change notification settings - Fork 304
Expand file tree
/
Copy pathshuffle_writer.rs
More file actions
866 lines (778 loc) · 31.1 KB
/
shuffle_writer.rs
File metadata and controls
866 lines (778 loc) · 31.1 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! Defines the External shuffle repartition plan.
use crate::metrics::ShufflePartitionerMetrics;
use crate::partitioners::{
EmptySchemaShufflePartitioner, MultiPartitionShuffleRepartitioner, ShufflePartitioner,
SinglePartitionShufflePartitioner,
};
use crate::{CometPartitioning, CompressionCodec};
use async_trait::async_trait;
use datafusion::common::exec_datafusion_err;
use datafusion::physical_expr::{EquivalenceProperties, Partitioning};
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::EmptyRecordBatchStream;
use datafusion::{
arrow::{datatypes::SchemaRef, error::ArrowError},
error::Result,
execution::context::TaskContext,
physical_plan::{
metrics::{ExecutionPlanMetricsSet, MetricsSet},
stream::RecordBatchStreamAdapter,
DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream,
},
};
use futures::{StreamExt, TryFutureExt, TryStreamExt};
use std::{
any::Any,
fmt,
fmt::{Debug, Formatter},
sync::Arc,
};
/// The shuffle writer operator maps each input partition to M output partitions based on a
/// partitioning scheme. No guarantees are made about the order of the resulting partitions.
#[derive(Debug)]
pub struct ShuffleWriterExec {
/// Input execution plan
input: Arc<dyn ExecutionPlan>,
/// Partitioning scheme to use
partitioning: CometPartitioning,
/// Output data file path
output_data_file: String,
/// Output index file path
output_index_file: String,
/// Metrics
metrics: ExecutionPlanMetricsSet,
/// Cache for expensive-to-compute plan properties
cache: Arc<PlanProperties>,
/// The compression codec to use when compressing shuffle blocks
codec: CompressionCodec,
tracing_enabled: bool,
/// Size of the write buffer in bytes
write_buffer_size: usize,
/// Maximum number of buffered batches before spilling, 0 = disabled
batch_spill_limit: usize,
}
impl ShuffleWriterExec {
/// Create a new ShuffleWriterExec
#[allow(clippy::too_many_arguments)]
pub fn try_new(
input: Arc<dyn ExecutionPlan>,
partitioning: CometPartitioning,
codec: CompressionCodec,
output_data_file: String,
output_index_file: String,
tracing_enabled: bool,
write_buffer_size: usize,
batch_spill_limit: usize,
) -> Result<Self> {
let cache = Arc::new(PlanProperties::new(
EquivalenceProperties::new(Arc::clone(&input.schema())),
Partitioning::UnknownPartitioning(1),
EmissionType::Final,
Boundedness::Bounded,
));
Ok(ShuffleWriterExec {
input,
partitioning,
metrics: ExecutionPlanMetricsSet::new(),
output_data_file,
output_index_file,
cache,
codec,
tracing_enabled,
write_buffer_size,
batch_spill_limit,
})
}
}
impl DisplayAs for ShuffleWriterExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(
f,
"ShuffleWriterExec: partitioning={:?}, compression={:?}",
self.partitioning, self.codec
)
}
DisplayFormatType::TreeRender => unimplemented!(),
}
}
}
#[async_trait]
impl ExecutionPlan for ShuffleWriterExec {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}
fn name(&self) -> &str {
"ShuffleWriterExec"
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn properties(&self) -> &Arc<PlanProperties> {
&self.cache
}
/// Get the schema for this execution plan
fn schema(&self) -> SchemaRef {
self.input.schema()
}
fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![&self.input]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
match children.len() {
1 => Ok(Arc::new(ShuffleWriterExec::try_new(
Arc::clone(&children[0]),
self.partitioning.clone(),
self.codec.clone(),
self.output_data_file.clone(),
self.output_index_file.clone(),
self.tracing_enabled,
self.write_buffer_size,
self.batch_spill_limit,
)?)),
_ => panic!("ShuffleWriterExec wrong number of children"),
}
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
let input = self.input.execute(partition, Arc::clone(&context))?;
let metrics = ShufflePartitionerMetrics::new(&self.metrics, 0);
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
futures::stream::once(
external_shuffle(
input,
partition,
self.output_data_file.clone(),
self.output_index_file.clone(),
self.partitioning.clone(),
metrics,
context,
self.codec.clone(),
self.tracing_enabled,
self.write_buffer_size,
self.batch_spill_limit,
)
.map_err(|e| ArrowError::ExternalError(Box::new(e))),
)
.try_flatten(),
)))
}
}
#[allow(clippy::too_many_arguments)]
async fn external_shuffle(
mut input: SendableRecordBatchStream,
partition: usize,
output_data_file: String,
output_index_file: String,
partitioning: CometPartitioning,
metrics: ShufflePartitionerMetrics,
context: Arc<TaskContext>,
codec: CompressionCodec,
tracing_enabled: bool,
write_buffer_size: usize,
batch_spill_limit: usize,
) -> Result<SendableRecordBatchStream> {
let schema = input.schema();
let mut repartitioner: Box<dyn ShufflePartitioner> = match &partitioning {
_ if schema.fields().is_empty() => {
log::debug!("found empty schema, overriding {partitioning:?} partitioning with EmptySchemaShufflePartitioner");
Box::new(EmptySchemaShufflePartitioner::try_new(
output_data_file,
output_index_file,
Arc::clone(&schema),
partitioning.partition_count(),
metrics,
codec,
)?)
}
any if any.partition_count() == 1 => Box::new(SinglePartitionShufflePartitioner::try_new(
output_data_file,
output_index_file,
Arc::clone(&schema),
metrics,
context.session_config().batch_size(),
codec,
write_buffer_size,
)?),
_ => Box::new(MultiPartitionShuffleRepartitioner::try_new(
partition,
output_data_file,
output_index_file,
Arc::clone(&schema),
partitioning,
metrics,
context.runtime_env(),
context.session_config().batch_size(),
codec,
tracing_enabled,
write_buffer_size,
batch_spill_limit,
)?),
};
while let Some(batch) = input.next().await {
// Await the repartitioner to insert the batch and shuffle the rows
// into the corresponding partition buffer.
// Otherwise, pull the next batch from the input stream might overwrite the
// current batch in the repartitioner.
repartitioner
.insert_batch(batch?)
.await
.map_err(|err| exec_datafusion_err!("Error inserting batch: {err}"))?;
}
repartitioner
.shuffle_write()
.map_err(|err| exec_datafusion_err!("Error in shuffle write: {err}"))?;
// shuffle writer always has empty output
Ok(Box::pin(EmptyRecordBatchStream::new(Arc::clone(&schema))) as SendableRecordBatchStream)
}
#[cfg(test)]
mod test {
use super::*;
use crate::{read_ipc_compressed, ShuffleBlockWriter};
use arrow::array::{Array, StringArray, StringBuilder};
use arrow::datatypes::{DataType, Field, Schema};
use arrow::record_batch::RecordBatch;
use arrow::row::{RowConverter, SortField};
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::source::DataSourceExec;
use datafusion::execution::config::SessionConfig;
use datafusion::execution::runtime_env::{RuntimeEnv, RuntimeEnvBuilder};
use datafusion::physical_expr::expressions::{col, Column};
use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr};
use datafusion::physical_plan::common::collect;
use datafusion::physical_plan::metrics::Time;
use datafusion::prelude::SessionContext;
use itertools::Itertools;
use std::io::Cursor;
use tokio::runtime::Runtime;
#[test]
#[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx`
fn roundtrip_ipc() {
let batch = create_batch(8192);
for codec in &[
CompressionCodec::None,
CompressionCodec::Zstd(1),
CompressionCodec::Snappy,
CompressionCodec::Lz4Frame,
] {
let mut output = vec![];
let mut cursor = Cursor::new(&mut output);
let writer =
ShuffleBlockWriter::try_new(batch.schema().as_ref(), codec.clone()).unwrap();
let length = writer
.write_batch(&batch, &mut cursor, &Time::default())
.unwrap();
assert_eq!(length, output.len());
let ipc_without_length_prefix = &output[16..];
let batch2 = read_ipc_compressed(ipc_without_length_prefix).unwrap();
assert_eq!(batch, batch2);
}
}
#[test]
#[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx`
fn test_single_partition_shuffle_writer() {
shuffle_write_test(1000, 100, 1, None);
shuffle_write_test(10000, 10, 1, None);
}
#[test]
#[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx`
fn test_insert_larger_batch() {
shuffle_write_test(10000, 1, 16, None);
}
#[test]
#[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx`
fn test_insert_smaller_batch() {
shuffle_write_test(1000, 1, 16, None);
shuffle_write_test(1000, 10, 16, None);
}
#[test]
#[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx`
fn test_large_number_of_partitions() {
shuffle_write_test(10000, 10, 200, Some(10 * 1024 * 1024));
shuffle_write_test(10000, 10, 2000, Some(10 * 1024 * 1024));
}
#[test]
#[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx`
fn test_large_number_of_partitions_spilling() {
shuffle_write_test(10000, 100, 200, Some(10 * 1024 * 1024));
}
#[tokio::test]
async fn shuffle_partitioner_memory() {
let batch = create_batch(900);
assert_eq!(8316, batch.get_array_memory_size()); // Not stable across Arrow versions
let memory_limit = 512 * 1024;
let num_partitions = 2;
let runtime_env = create_runtime(memory_limit);
let metrics_set = ExecutionPlanMetricsSet::new();
let mut repartitioner = MultiPartitionShuffleRepartitioner::try_new(
0,
"/tmp/data.out".to_string(),
"/tmp/index.out".to_string(),
batch.schema(),
CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions),
ShufflePartitionerMetrics::new(&metrics_set, 0),
runtime_env,
1024,
CompressionCodec::Lz4Frame,
false,
1024 * 1024, // write_buffer_size: 1MB default
0, // batch_spill_limit: disabled
)
.unwrap();
repartitioner.insert_batch(batch.clone()).await.unwrap();
{
let partition_writers = repartitioner.partition_writers();
assert_eq!(partition_writers.len(), 2);
assert!(!partition_writers[0].has_spill_file());
assert!(!partition_writers[1].has_spill_file());
}
repartitioner.spill().unwrap();
// after spill, there should be spill files
{
let partition_writers = repartitioner.partition_writers();
assert!(partition_writers[0].has_spill_file());
assert!(partition_writers[1].has_spill_file());
}
// insert another batch after spilling
repartitioner.insert_batch(batch.clone()).await.unwrap();
}
fn create_runtime(memory_limit: usize) -> Arc<RuntimeEnv> {
Arc::new(
RuntimeEnvBuilder::new()
.with_memory_limit(memory_limit, 1.0)
.build()
.unwrap(),
)
}
fn shuffle_write_test(
batch_size: usize,
num_batches: usize,
num_partitions: usize,
memory_limit: Option<usize>,
) {
let batch = create_batch(batch_size);
let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default(
col("a", batch.schema().as_ref()).unwrap(),
)])
.unwrap();
let sort_fields: Vec<SortField> = batch
.columns()
.iter()
.zip(&lex_ordering)
.map(|(array, sort_expr)| {
SortField::new_with_options(array.data_type().clone(), sort_expr.options)
})
.collect();
let row_converter = RowConverter::new(sort_fields).unwrap();
let owned_rows = if num_partitions == 1 {
vec![]
} else {
// Determine range boundaries based on create_batch implementation. We just divide the
// domain of values in the batch equally to find partition bounds.
let bounds_strings = {
let mut boundaries = Vec::with_capacity(num_partitions - 1);
let step = batch_size as f64 / num_partitions as f64;
for i in 1..(num_partitions) {
boundaries.push(Some((step * i as f64).round().to_string()));
}
boundaries
};
let bounds_array: Arc<dyn Array> = Arc::new(StringArray::from(bounds_strings));
let bounds_rows = row_converter
.convert_columns(vec![bounds_array].as_slice())
.unwrap();
let owned_rows_vec = bounds_rows.iter().map(|row| row.owned()).collect_vec();
owned_rows_vec
};
for partitioning in [
CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions),
CometPartitioning::RangePartitioning(
lex_ordering,
num_partitions,
Arc::new(row_converter),
owned_rows,
),
CometPartitioning::RoundRobin(num_partitions, 0),
] {
let batches = (0..num_batches).map(|_| batch.clone()).collect::<Vec<_>>();
let partitions = &[batches];
let exec = ShuffleWriterExec::try_new(
Arc::new(DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(partitions, batch.schema(), None).unwrap(),
))),
partitioning,
CompressionCodec::Zstd(1),
"/tmp/data.out".to_string(),
"/tmp/index.out".to_string(),
false,
1024 * 1024, // write_buffer_size: 1MB default
0, // batch_spill_limit: disabled
)
.unwrap();
// 10MB memory should be enough for running this test
let config = SessionConfig::new();
let mut runtime_env_builder = RuntimeEnvBuilder::new();
runtime_env_builder = match memory_limit {
Some(limit) => runtime_env_builder.with_memory_limit(limit, 1.0),
None => runtime_env_builder,
};
let runtime_env = Arc::new(runtime_env_builder.build().unwrap());
let ctx = SessionContext::new_with_config_rt(config, runtime_env);
let task_ctx = ctx.task_ctx();
let stream = exec.execute(0, task_ctx).unwrap();
let rt = Runtime::new().unwrap();
rt.block_on(collect(stream)).unwrap();
}
}
fn create_batch(batch_size: usize) -> RecordBatch {
let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, true)]));
let mut b = StringBuilder::new();
for i in 0..batch_size {
b.append_value(format!("{i}"));
}
let array = b.finish();
RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap()
}
#[test]
#[cfg_attr(miri, ignore)]
fn test_round_robin_deterministic() {
// Test that round robin partitioning produces identical results when run multiple times
use std::fs;
use std::io::Read;
let batch_size = 1000;
let num_batches = 10;
let num_partitions = 8;
let batch = create_batch(batch_size);
let batches = (0..num_batches).map(|_| batch.clone()).collect::<Vec<_>>();
// Run shuffle twice and compare results
for run in 0..2 {
let data_file = format!("/tmp/rr_data_{}.out", run);
let index_file = format!("/tmp/rr_index_{}.out", run);
let partitions = std::slice::from_ref(&batches);
let exec = ShuffleWriterExec::try_new(
Arc::new(DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(partitions, batch.schema(), None).unwrap(),
))),
CometPartitioning::RoundRobin(num_partitions, 0),
CompressionCodec::Zstd(1),
data_file.clone(),
index_file.clone(),
false,
1024 * 1024,
0,
)
.unwrap();
let config = SessionConfig::new();
let runtime_env = Arc::new(
RuntimeEnvBuilder::new()
.with_memory_limit(10 * 1024 * 1024, 1.0)
.build()
.unwrap(),
);
let session_ctx = Arc::new(SessionContext::new_with_config_rt(config, runtime_env));
let task_ctx = Arc::new(TaskContext::from(session_ctx.as_ref()));
// Execute the shuffle
futures::executor::block_on(async {
let mut stream = exec.execute(0, Arc::clone(&task_ctx)).unwrap();
while stream.next().await.is_some() {}
});
if run == 1 {
// Compare data files
let mut data0 = Vec::new();
fs::File::open("/tmp/rr_data_0.out")
.unwrap()
.read_to_end(&mut data0)
.unwrap();
let mut data1 = Vec::new();
fs::File::open("/tmp/rr_data_1.out")
.unwrap()
.read_to_end(&mut data1)
.unwrap();
assert_eq!(
data0, data1,
"Round robin shuffle data should be identical across runs"
);
// Compare index files
let mut index0 = Vec::new();
fs::File::open("/tmp/rr_index_0.out")
.unwrap()
.read_to_end(&mut index0)
.unwrap();
let mut index1 = Vec::new();
fs::File::open("/tmp/rr_index_1.out")
.unwrap()
.read_to_end(&mut index1)
.unwrap();
assert_eq!(
index0, index1,
"Round robin shuffle index should be identical across runs"
);
}
}
// Clean up
let _ = fs::remove_file("/tmp/rr_data_0.out");
let _ = fs::remove_file("/tmp/rr_index_0.out");
let _ = fs::remove_file("/tmp/rr_data_1.out");
let _ = fs::remove_file("/tmp/rr_index_1.out");
}
/// Test that batch coalescing in BufBatchWriter reduces output size by
/// writing fewer, larger IPC blocks instead of many small ones.
#[test]
#[cfg_attr(miri, ignore)]
fn test_batch_coalescing_reduces_size() {
use crate::writers::BufBatchWriter;
use arrow::array::Int32Array;
// Create a wide schema to amplify per-block schema overhead
let fields: Vec<Field> = (0..20)
.map(|i| Field::new(format!("col_{i}"), DataType::Int32, false))
.collect();
let schema = Arc::new(Schema::new(fields));
// Create many small batches (50 rows each)
let small_batches: Vec<RecordBatch> = (0..100)
.map(|batch_idx| {
let columns: Vec<Arc<dyn Array>> = (0..20)
.map(|col_idx| {
let values: Vec<i32> = (0..50)
.map(|row| batch_idx * 50 + row + col_idx * 1000)
.collect();
Arc::new(Int32Array::from(values)) as Arc<dyn Array>
})
.collect();
RecordBatch::try_new(Arc::clone(&schema), columns).unwrap()
})
.collect();
let codec = CompressionCodec::Lz4Frame;
let encode_time = Time::default();
let write_time = Time::default();
// Write with coalescing (batch_size=8192)
let mut coalesced_output = Vec::new();
{
let mut writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone()).unwrap();
let mut buf_writer = BufBatchWriter::new(
&mut writer,
Cursor::new(&mut coalesced_output),
1024 * 1024,
8192,
);
for batch in &small_batches {
buf_writer.write(batch, &encode_time, &write_time).unwrap();
}
buf_writer.flush(&encode_time, &write_time).unwrap();
}
// Write without coalescing (batch_size=1)
let mut uncoalesced_output = Vec::new();
{
let mut writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone()).unwrap();
let mut buf_writer = BufBatchWriter::new(
&mut writer,
Cursor::new(&mut uncoalesced_output),
1024 * 1024,
1,
);
for batch in &small_batches {
buf_writer.write(batch, &encode_time, &write_time).unwrap();
}
buf_writer.flush(&encode_time, &write_time).unwrap();
}
// Coalesced output should be smaller due to fewer IPC schema blocks
assert!(
coalesced_output.len() < uncoalesced_output.len(),
"Coalesced output ({} bytes) should be smaller than uncoalesced ({} bytes)",
coalesced_output.len(),
uncoalesced_output.len()
);
// Verify both roundtrip correctly by reading all IPC blocks
let coalesced_rows = read_all_ipc_blocks(&coalesced_output);
let uncoalesced_rows = read_all_ipc_blocks(&uncoalesced_output);
assert_eq!(
coalesced_rows, 5000,
"Coalesced should contain all 5000 rows"
);
assert_eq!(
uncoalesced_rows, 5000,
"Uncoalesced should contain all 5000 rows"
);
}
/// Read all IPC blocks from a byte buffer written by BufBatchWriter/ShuffleBlockWriter,
/// returning the total number of rows.
fn read_all_ipc_blocks(data: &[u8]) -> usize {
let mut offset = 0;
let mut total_rows = 0;
while offset < data.len() {
// First 8 bytes are the IPC length (little-endian u64)
let ipc_length =
u64::from_le_bytes(data[offset..offset + 8].try_into().unwrap()) as usize;
// Skip the 8-byte length prefix; the next 8 bytes are field_count + codec header
let block_start = offset + 8;
let block_end = block_start + ipc_length;
// read_ipc_compressed expects data starting after the 16-byte header
// (i.e., after length + field_count), at the codec tag
let ipc_data = &data[block_start + 8..block_end];
let batch = read_ipc_compressed(ipc_data).unwrap();
total_rows += batch.num_rows();
offset = block_end;
}
total_rows
}
#[test]
#[cfg_attr(miri, ignore)]
fn test_empty_schema_shuffle_writer() {
use std::fs;
use std::io::Read;
let num_rows = 1000;
let num_batches = 5;
let num_partitions = 10;
let schema = Arc::new(Schema::new(Vec::<Field>::new()));
let batch = RecordBatch::try_new_with_options(
Arc::clone(&schema),
vec![],
&arrow::array::RecordBatchOptions::new().with_row_count(Some(num_rows)),
)
.unwrap();
let batches = (0..num_batches).map(|_| batch.clone()).collect::<Vec<_>>();
let partitions = &[batches];
let dir = tempfile::tempdir().unwrap();
let data_file = dir.path().join("data.out");
let index_file = dir.path().join("index.out");
let exec = ShuffleWriterExec::try_new(
Arc::new(DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(partitions, Arc::clone(&schema), None).unwrap(),
))),
CometPartitioning::RoundRobin(num_partitions, 0),
CompressionCodec::Zstd(1),
data_file.to_str().unwrap().to_string(),
index_file.to_str().unwrap().to_string(),
false,
1024 * 1024,
0,
)
.unwrap();
let config = SessionConfig::new();
let runtime_env = Arc::new(RuntimeEnvBuilder::new().build().unwrap());
let ctx = SessionContext::new_with_config_rt(config, runtime_env);
let task_ctx = ctx.task_ctx();
let stream = exec.execute(0, task_ctx).unwrap();
let rt = Runtime::new().unwrap();
rt.block_on(collect(stream)).unwrap();
// Verify data file is non-empty (contains IPC batch with row count)
let mut data = Vec::new();
fs::File::open(&data_file)
.unwrap()
.read_to_end(&mut data)
.unwrap();
assert!(!data.is_empty(), "Data file should contain IPC data");
// Verify row count survives roundtrip
let total_rows = read_all_ipc_blocks(&data);
assert_eq!(
total_rows,
num_rows * num_batches,
"Row count should survive roundtrip"
);
// Verify index file structure: num_partitions + 1 offsets
let mut index_data = Vec::new();
fs::File::open(&index_file)
.unwrap()
.read_to_end(&mut index_data)
.unwrap();
let expected_index_size = (num_partitions + 1) * 8;
assert_eq!(index_data.len(), expected_index_size);
// First offset should be 0
let first_offset = i64::from_le_bytes(index_data[0..8].try_into().unwrap());
assert_eq!(first_offset, 0);
// Second offset should equal data file length (partition 0 holds all data)
let data_len = data.len() as i64;
let second_offset = i64::from_le_bytes(index_data[8..16].try_into().unwrap());
assert_eq!(second_offset, data_len);
// All remaining offsets should equal data file length (empty partitions)
for i in 2..=num_partitions {
let offset = i64::from_le_bytes(index_data[i * 8..(i + 1) * 8].try_into().unwrap());
assert_eq!(
offset, data_len,
"Partition {i} offset should equal data length"
);
}
}
#[test]
#[cfg_attr(miri, ignore)]
fn test_empty_schema_shuffle_writer_zero_rows() {
use std::fs;
use std::io::Read;
let num_partitions = 4;
let schema = Arc::new(Schema::new(Vec::<Field>::new()));
let batch = RecordBatch::try_new_with_options(
Arc::clone(&schema),
vec![],
&arrow::array::RecordBatchOptions::new().with_row_count(Some(0)),
)
.unwrap();
let batches = vec![batch];
let partitions = &[batches];
let dir = tempfile::tempdir().unwrap();
let data_file = dir.path().join("data.out");
let index_file = dir.path().join("index.out");
let exec = ShuffleWriterExec::try_new(
Arc::new(DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(partitions, Arc::clone(&schema), None).unwrap(),
))),
CometPartitioning::RoundRobin(num_partitions, 0),
CompressionCodec::Zstd(1),
data_file.to_str().unwrap().to_string(),
index_file.to_str().unwrap().to_string(),
false,
1024 * 1024,
0,
)
.unwrap();
let config = SessionConfig::new();
let runtime_env = Arc::new(RuntimeEnvBuilder::new().build().unwrap());
let ctx = SessionContext::new_with_config_rt(config, runtime_env);
let task_ctx = ctx.task_ctx();
let stream = exec.execute(0, task_ctx).unwrap();
let rt = Runtime::new().unwrap();
rt.block_on(collect(stream)).unwrap();
// Data file should be empty (no rows to write)
let mut data = Vec::new();
fs::File::open(&data_file)
.unwrap()
.read_to_end(&mut data)
.unwrap();
assert!(data.is_empty(), "Data file should be empty with zero rows");
// Index file should have all-zero offsets
let mut index_data = Vec::new();
fs::File::open(&index_file)
.unwrap()
.read_to_end(&mut index_data)
.unwrap();
let expected_index_size = (num_partitions + 1) * 8;
assert_eq!(index_data.len(), expected_index_size);
for i in 0..=num_partitions {
let offset = i64::from_le_bytes(index_data[i * 8..(i + 1) * 8].try_into().unwrap());
assert_eq!(offset, 0, "All offsets should be 0 with zero rows");
}
}
}