-
Notifications
You must be signed in to change notification settings - Fork 1
Expand file tree
/
Copy pathlib.rs
More file actions
2182 lines (1964 loc) · 83.5 KB
/
lib.rs
File metadata and controls
2182 lines (1964 loc) · 83.5 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
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
Copyright 2024-2025 The Spice.ai OSS Authors
Licensed 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
https://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.
*/
use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use arrow::array::{Array, AsArray, RecordBatch, StringArray, TimestampMicrosecondArray};
use arrow::compute;
use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit};
use data_generation::config::{DatasetConfig as GenerationDatasetConfig, TargetConfig};
use data_generation::dataset::simple_sequence::SimpleSequenceDataset;
use data_generation::dataset::tpch::TpchDataset;
use data_generation::dataset::{Dataset, MutationConfig};
use data_generation::storage::{DataStorage, ReadResult};
use std::collections::{BTreeMap, HashMap, HashSet, VecDeque};
use std::path::Path;
use std::sync::Arc as StdArc;
use std::sync::Mutex as StdMutex;
use std::sync::atomic::{AtomicI64, AtomicU64, Ordering};
use std::time::Instant;
use system_adapter_protocol::DatasetConfig as ProtocolDatasetConfig;
use tokio::sync::watch;
use tokio::task::{JoinHandle, JoinSet};
use tokio_util::sync::CancellationToken;
use tracing::{debug, error, info, trace, warn};
use crate::sink::{InsertOp, Sink};
pub mod sink;
/// Column name appended by the ETL pipeline to every batch.
const CREATED_AT_COLUMN: &str = "__created_at";
/// Internal columns that must be stripped before writing to the sink.
const INTERNAL_COLUMNS: &[&str] = &["_op", "_op_index"];
/// Target and maximum number of rows per output batch.
///
/// Smaller input batches from a [`ReadResult`] are concatenated together until
/// this threshold is reached, and larger input batches are split so no output
/// batch exceeds this size.
const TARGET_BATCH_ROWS_ENV: &str = "SPICEBENCH_TARGET_BATCH_ROWS";
const DEFAULT_TARGET_BATCH_ROWS: usize = 8_192 * 4;
fn target_batch_rows() -> usize {
static VALUE: std::sync::OnceLock<usize> = std::sync::OnceLock::new();
*VALUE.get_or_init(|| {
std::env::var(TARGET_BATCH_ROWS_ENV)
.ok()
.and_then(|v| v.parse::<usize>().ok())
.filter(|&v| v > 0)
.unwrap_or(DEFAULT_TARGET_BATCH_ROWS)
})
}
/// Maximum number of in-flight sink writes allowed per table task when the
/// current segment set is insert-only.
const MAX_IN_FLIGHT_TABLE_WRITES: usize = 1;
/// Maximum number of concurrent source logical-batch reads per ETL table task.
const MAX_IN_FLIGHT_SOURCE_BATCH_READS: usize = 2;
/// Maximum number of concurrent split-part downloads per logical source batch.
const MAX_IN_FLIGHT_SOURCE_PART_READS: usize = 2;
/// Returns a new schema with the `__created_at` timestamp column appended.
fn schema_with_created_at(schema: &SchemaRef) -> SchemaRef {
let mut fields: Vec<_> = schema.fields().iter().cloned().collect();
fields.push(Arc::new(Field::new(
CREATED_AT_COLUMN,
DataType::Timestamp(TimeUnit::Microsecond, None),
true,
)));
Arc::new(Schema::new(fields))
}
fn schema_without_internal_columns(schema: &SchemaRef) -> SchemaRef {
let fields: Vec<_> = schema
.fields()
.iter()
.filter(|f| !INTERNAL_COLUMNS.contains(&f.name().as_str()))
.cloned()
.collect();
Arc::new(Schema::new(fields))
}
/// Returns the current wall-clock time as microseconds since the UNIX epoch.
///
/// Call this **once per input batch** and pass the result to every
/// [`append_created_at`] invocation for that batch so that all segments
/// (splits by `_op`) share the same timestamp.
fn now_micros() -> i64 {
SystemTime::now()
.duration_since(UNIX_EPOCH)
.expect("system time before UNIX epoch")
.as_micros() as i64
}
/// Appends a `__created_at` column with the supplied `created_at_us` timestamp
/// (microsecond UTC) to every row in the batch and stores the value in
/// `last_created_at`.
fn append_created_at(
batch: &RecordBatch,
output_schema: &SchemaRef,
created_at_us: i64,
) -> anyhow::Result<RecordBatch> {
let timestamps = TimestampMicrosecondArray::from_value(created_at_us, batch.num_rows());
let mut columns: Vec<_> = batch.columns().to_vec();
columns.push(Arc::new(timestamps));
Ok(RecordBatch::try_new(Arc::clone(output_schema), columns)?)
}
fn build_partition_columns(dataset_columns: Vec<String>) -> Vec<String> {
let mut columns = Vec::with_capacity(1 + dataset_columns.len());
columns.push(CREATED_AT_COLUMN.to_string());
for column in dataset_columns {
if column == CREATED_AT_COLUMN || columns.iter().any(|existing| existing == &column) {
continue;
}
columns.push(column);
}
columns
}
fn record_timing_sample_ms(samples: &StdArc<StdMutex<Vec<u64>>>, elapsed: Duration) {
let elapsed_ms = elapsed.as_millis().min(u128::from(u64::MAX)) as u64;
let mut values = samples.lock().expect("timing samples lock poisoned");
values.push(elapsed_ms);
}
fn take_median_sample_ms(samples: &StdArc<StdMutex<Vec<u64>>>) -> Option<(f64, usize)> {
let mut values = {
let mut guard = samples.lock().expect("timing samples lock poisoned");
if guard.is_empty() {
return None;
}
std::mem::take(&mut *guard)
};
values.sort_unstable();
let count = values.len();
let mid = count / 2;
let median_ms = if count % 2 == 1 {
values[mid] as f64
} else {
(values[mid - 1] as f64 + values[mid] as f64) / 2.0
};
Some((median_ms, count))
}
/// Compacts any `Utf8View` / `BinaryView` columns in a [`RecordBatch`] by
/// calling [`GenericByteViewArray::gc()`] to coalesce their scattered data
/// buffers into a single contiguous buffer.
///
/// When multiple small batches (e.g. from a Parquet reader) are concatenated
/// via [`arrow::compute::concat_batches`], each `Utf8View` column accumulates
/// every source batch's data buffers without compacting them. A subsequent
/// `slice()` of the concatenated batch retains **all** backing buffers
/// (because `StringViewArray` cannot determine which buffers are still
/// referenced), causing even a single-row slice to serialize at nearly the
/// full batch size in Arrow IPC. This leads to spurious "single-row batch
/// exceeds max ingest payload" errors in downstream sinks.
///
/// This function is a no-op for batches that contain no view-typed columns.
fn gc_byte_view_columns(batch: &RecordBatch) -> anyhow::Result<RecordBatch> {
let schema = batch.schema();
let mut needs_gc = false;
for field in schema.fields() {
if matches!(field.data_type(), DataType::Utf8View | DataType::BinaryView) {
needs_gc = true;
break;
}
}
if !needs_gc {
return Ok(batch.clone());
}
let columns: Vec<Arc<dyn Array>> = schema
.fields()
.iter()
.enumerate()
.map(|(i, field)| -> Arc<dyn Array> {
let col = batch.column(i);
match field.data_type() {
DataType::Utf8View => {
let view_array = col.as_string_view();
Arc::new(view_array.gc())
}
DataType::BinaryView => {
let view_array = col.as_binary_view();
Arc::new(view_array.gc())
}
_ => Arc::clone(col),
}
})
.collect();
RecordBatch::try_new(schema, columns)
.map_err(|e| anyhow::anyhow!("Failed to rebuild batch after gc: {e}"))
}
/// Concatenates small input batches and splits large input batches so each
/// resulting batch has at most [`target_batch_rows()`] rows.
///
/// This reduces per-batch overhead in downstream partitioning and S3 writes.
fn coalesce_batches(batches: &[RecordBatch]) -> anyhow::Result<Vec<RecordBatch>> {
if batches.is_empty() {
return Ok(Vec::new());
}
let schema = batches[0].schema();
let mut result = Vec::new();
let mut pending: Vec<RecordBatch> = Vec::new();
let mut pending_rows: usize = 0;
for batch in batches {
let mut offset = 0usize;
let total_rows = batch.num_rows();
while offset < total_rows {
let chunk_rows = std::cmp::min(target_batch_rows(), total_rows - offset);
let chunk = if offset == 0 && chunk_rows == total_rows {
batch.clone()
} else {
batch.slice(offset, chunk_rows)
};
offset += chunk_rows;
if pending_rows > 0 && pending_rows + chunk_rows > target_batch_rows() {
let merged = if pending.len() == 1 {
pending.remove(0)
} else {
let concatenated = arrow::compute::concat_batches(&schema, &pending)
.map_err(|e| anyhow::anyhow!("Failed to concat input batches: {e}"))?;
gc_byte_view_columns(&concatenated)?
};
result.push(merged);
pending.clear();
pending_rows = 0;
}
if chunk_rows == target_batch_rows() && pending_rows == 0 {
result.push(chunk);
continue;
}
pending_rows += chunk_rows;
pending.push(chunk);
if pending_rows == target_batch_rows() {
let merged = if pending.len() == 1 {
pending.remove(0)
} else {
let concatenated =
arrow::compute::concat_batches(&schema, &pending).map_err(|e| {
anyhow::anyhow!("Failed to concat input batches at threshold: {e}")
})?;
gc_byte_view_columns(&concatenated)?
};
result.push(merged);
pending.clear();
pending_rows = 0;
}
}
}
// Flush any remainder (below the threshold).
if !pending.is_empty() {
let merged = if pending.len() == 1 {
pending.remove(0)
} else {
let concatenated = arrow::compute::concat_batches(&schema, &pending)
.map_err(|e| anyhow::anyhow!("Failed to concat trailing input batches: {e}"))?;
gc_byte_view_columns(&concatenated)?
};
result.push(merged);
}
Ok(result)
}
/// Removes and returns the next batch ID (greater than `after_batch_id`) that
/// still has pending work for `table_name`.
///
/// When `max_batch_id` is `Some(max)`, candidates with a batch ID greater than
/// `max` are not considered. This is used to prevent coalescing across
/// checkpoint interval boundaries.
///
/// This is used by the ETL runner to coalesce very small reads across multiple
/// source batch IDs for the same table while ensuring consumed IDs are not
/// replayed in later steps.
fn reserve_next_batch_id_for_table(
work_state: &mut PipelineWorkState,
table_name: &str,
after_batch_id: u64,
max_batch_id: Option<u64>,
) -> Option<(u64, bool)> {
let mut found: Option<(u64, bool)> = None;
let start = after_batch_id.saturating_add(1);
for (candidate_batch_id, tables) in work_state.steps.range_mut(start..) {
if max_batch_id.is_some_and(|max| *candidate_batch_id > max) {
break;
}
if let Some(pos) = tables.iter().position(|t| t == table_name) {
tables.remove(pos);
found = Some((*candidate_batch_id, tables.is_empty()));
break;
}
}
if let Some((batch_id, remove_entry)) = found {
if remove_entry {
work_state.steps.remove(&batch_id);
}
Some((batch_id, remove_entry))
} else {
None
}
}
async fn read_logical_batch(
data_storage: &Arc<dyn DataStorage>,
table_name: &str,
batch_id: u64,
) -> Result<Option<ReadResult>, String> {
let mut part_ids = data_storage
.read_batch_parts(table_name, batch_id)
.await
.map_err(|e| format!("read {table_name} batch {batch_id} parts: {e}"))?;
if part_ids.is_empty() {
return data_storage
.read_batch(table_name, batch_id, None)
.await
.map_err(|e| format!("read {table_name} batch {batch_id}: {e}"));
}
part_ids.sort_unstable();
let table_name_owned = table_name.to_string();
let mut join_set: JoinSet<(usize, Result<ReadResult, String>)> = JoinSet::new();
let mut scheduled_part_ids: VecDeque<usize> = VecDeque::new();
let mut completed_parts: HashMap<usize, ReadResult> = HashMap::new();
let mut part_iter = part_ids.into_iter();
let max_in_flight = MAX_IN_FLIGHT_SOURCE_PART_READS.max(1);
let mut merged_batches: Vec<RecordBatch> = Vec::new();
let mut rows_read: u64 = 0;
let mut bytes_read: u64 = 0;
let mut key_columns: Option<Vec<String>> = None;
while !join_set.is_empty() || part_iter.len() > 0 {
while join_set.len() < max_in_flight {
let Some(part_id) = part_iter.next() else {
break;
};
scheduled_part_ids.push_back(part_id);
let data_storage = Arc::clone(data_storage);
let table_name = table_name_owned.clone();
join_set.spawn(async move {
let read_result = match data_storage.read_batch(&table_name, batch_id, Some(part_id)).await {
Ok(Some(result)) => Ok(result),
Ok(None) => Err(format!(
"Missing object for {table_name} batch {batch_id} part {part_id} listed in metadata"
)),
Err(e) => Err(format!("read {table_name} batch {batch_id} part {part_id}: {e}")),
};
(part_id, read_result)
});
}
let Some(joined) = join_set.join_next().await else {
break;
};
let (part_id, read_result) = joined
.map_err(|e| format!("join error reading {table_name} batch {batch_id} part: {e}"))?;
let read_result = read_result?;
completed_parts.insert(part_id, read_result);
while let Some(next_part_id) = scheduled_part_ids.front().copied() {
let Some(next_result) = completed_parts.remove(&next_part_id) else {
break;
};
scheduled_part_ids.pop_front();
if let Some(existing_keys) = &key_columns {
if existing_keys != &next_result.key_columns {
warn!(
table = %table_name,
batch_id,
part_id = next_part_id,
"Key columns changed across split parts; using keys from first part"
);
}
} else {
key_columns = Some(next_result.key_columns.clone());
}
rows_read += next_result.rows_read;
bytes_read += next_result.bytes_read;
merged_batches.extend(next_result.batches);
}
}
Ok(Some(ReadResult {
batches: merged_batches,
rows_read,
bytes_read,
key_columns: key_columns.unwrap_or_default(),
}))
}
/// Reads source data for `table_name` starting at `start_batch_id`, then keeps
/// reserving and reading subsequent batch IDs for that table until at least
/// [`target_batch_rows()`] rows have been accumulated (or no further work exists).
///
/// When `max_batch_id` is `Some(max)`, coalescing will not reserve batch IDs
/// beyond `max`. This prevents reads from crossing a checkpoint interval
/// boundary, ensuring checkpoint results are deterministic regardless of
/// the configured [`target_batch_rows()`] value.
///
/// Returns `(raw_batches, key_columns, table_finished, consumed_work_units, rows_read)` where
/// `table_finished=true` means a read returned `None` and the table should be
/// marked as fully consumed. `consumed_work_units` counts how many table+batch
/// work items were consumed from the shared plan (including coalesced reserve
/// pulls), and `rows_read` is the total source rows read for this task.
async fn read_batches_until_min_rows(
data_storage: &Arc<dyn DataStorage>,
work_state: &Arc<StdMutex<PipelineWorkState>>,
logical_steps_consumed: &StdArc<AtomicU64>,
table_name: &str,
start_batch_id: u64,
max_batch_id: Option<u64>,
) -> Result<(Vec<RecordBatch>, Vec<String>, bool, u64, u64), String> {
let mut all_batches: Vec<RecordBatch> = Vec::new();
let mut total_rows: usize = 0;
let mut key_columns: Option<Vec<String>> = None;
let mut reserve_cursor = start_batch_id;
let mut table_finished = false;
let mut read_any = false;
let mut consumed_work_units: u64 = 1;
let mut rows_read: u64 = 0;
let table_name_owned = table_name.to_string();
let mut can_reserve_more = true;
let mut join_set: JoinSet<(u64, Result<Option<ReadResult>, String>)> = JoinSet::new();
let mut scheduled_batch_ids: VecDeque<u64> = VecDeque::new();
let mut completed_batches: HashMap<u64, Option<ReadResult>> = HashMap::new();
let mut reserved_removed_step: HashMap<u64, bool> = HashMap::new();
let restore_unconsumed_reservations =
|scheduled_batch_ids: &VecDeque<u64>,
reserved_removed_step: &HashMap<u64, bool>,
consumed_work_units: &mut u64| {
if scheduled_batch_ids.is_empty() {
return;
}
let mut restored_removed_steps = 0u64;
let mut state = work_state.lock().expect("work_state lock poisoned");
for &batch_id in scheduled_batch_ids {
let tables = state.steps.entry(batch_id).or_default();
if !tables.iter().any(|t| t == table_name) {
tables.push(table_name.to_string());
}
if reserved_removed_step
.get(&batch_id)
.copied()
.unwrap_or(false)
{
restored_removed_steps += 1;
}
*consumed_work_units = consumed_work_units.saturating_sub(1);
}
if restored_removed_steps > 0 {
logical_steps_consumed.fetch_sub(restored_removed_steps, Ordering::Relaxed);
}
};
scheduled_batch_ids.push_back(start_batch_id);
reserved_removed_step.insert(start_batch_id, false);
{
let data_storage = Arc::clone(data_storage);
let table_name = table_name_owned.clone();
join_set.spawn(async move {
let result = read_logical_batch(&data_storage, &table_name, start_batch_id).await;
(start_batch_id, result)
});
}
'read_loop: while !join_set.is_empty() || !completed_batches.is_empty() {
while can_reserve_more
&& join_set.len() < MAX_IN_FLIGHT_SOURCE_BATCH_READS.max(1)
&& total_rows < target_batch_rows()
&& !table_finished
{
let reservation = {
let mut state = work_state.lock().expect("work_state lock poisoned");
reserve_next_batch_id_for_table(
&mut state,
table_name,
reserve_cursor,
max_batch_id,
)
};
let Some((next_batch_id, removed_step_entry)) = reservation else {
can_reserve_more = false;
break;
};
if removed_step_entry {
logical_steps_consumed.fetch_add(1, Ordering::Relaxed);
}
reserve_cursor = next_batch_id;
consumed_work_units += 1;
scheduled_batch_ids.push_back(next_batch_id);
reserved_removed_step.insert(next_batch_id, removed_step_entry);
let data_storage = Arc::clone(data_storage);
let table_name = table_name_owned.clone();
join_set.spawn(async move {
let result = read_logical_batch(&data_storage, &table_name, next_batch_id).await;
(next_batch_id, result)
});
}
let Some(joined) = join_set.join_next().await else {
break;
};
let (batch_id, read_result) =
joined.map_err(|e| format!("join error reading {table_name} batch: {e}"))?;
let read_result = read_result?;
completed_batches.insert(batch_id, read_result);
while let Some(next_batch_id) = scheduled_batch_ids.front().copied() {
let Some(next_read_result) = completed_batches.remove(&next_batch_id) else {
break;
};
scheduled_batch_ids.pop_front();
reserved_removed_step.remove(&next_batch_id);
match next_read_result {
Some(result) => {
if let Some(existing_keys) = &key_columns {
if existing_keys != &result.key_columns {
warn!(
table = %table_name,
batch_id = next_batch_id,
"Key columns changed across source batches while coalescing; using keys from first read"
);
}
} else {
key_columns = Some(result.key_columns.clone());
}
total_rows += result.num_rows();
rows_read += result.num_rows() as u64;
all_batches.extend(result.batches);
read_any = true;
if total_rows >= target_batch_rows() {
join_set.abort_all();
restore_unconsumed_reservations(
&scheduled_batch_ids,
&reserved_removed_step,
&mut consumed_work_units,
);
break 'read_loop;
}
}
None => {
if !read_any {
join_set.abort_all();
restore_unconsumed_reservations(
&scheduled_batch_ids,
&reserved_removed_step,
&mut consumed_work_units,
);
return Ok((Vec::new(), Vec::new(), true, consumed_work_units, rows_read));
}
table_finished = true;
join_set.abort_all();
break 'read_loop;
}
}
}
}
Ok((
all_batches,
key_columns.unwrap_or_default(),
table_finished,
consumed_work_units,
rows_read,
))
}
/// Removes internal bookkeeping columns (`_op`, `_op_index`) from a
/// [`RecordBatch`] so they are not persisted to the sink.
fn strip_internal_columns(batch: &RecordBatch) -> anyhow::Result<RecordBatch> {
let schema = batch.schema();
let indices_to_keep: Vec<usize> = schema
.fields()
.iter()
.enumerate()
.filter(|(_, f)| !INTERNAL_COLUMNS.contains(&f.name().as_str()))
.map(|(i, _)| i)
.collect();
if indices_to_keep.len() == schema.fields().len() {
return Ok(batch.clone());
}
let new_fields: Vec<_> = indices_to_keep
.iter()
.map(|&i| schema.field(i).clone())
.collect();
let new_columns: Vec<_> = indices_to_keep
.iter()
.map(|&i| batch.column(i).clone())
.collect();
Ok(RecordBatch::try_new(
Arc::new(Schema::new(new_fields)),
new_columns,
)?)
}
/// Returns `true` when the batch is insert-only and can skip operation
/// segmentation.
///
/// Fast-path rules:
/// - If the mutation config specifies non-zero update or delete ratios,
/// the batch cannot be guaranteed to be insert-only.
/// - If `_op` is missing, treat the batch as insert-only.
/// - If `_op` exists, every non-null value must be `"c"`.
fn batch_is_insert_only(batch: &RecordBatch, mutations: &MutationConfig) -> anyhow::Result<bool> {
// Naive check: if the dataset was generated with non-zero update or delete
// ratios, batches may contain `_op` values other than "c".
if mutations.etl_type() != data_generation::dataset::EtlType::Changes {
return Ok(true);
}
let op_idx = match batch.schema().index_of("_op") {
Ok(idx) => idx,
Err(_) => return Ok(true),
};
let op_array = batch
.column(op_idx)
.as_any()
.downcast_ref::<StringArray>()
.ok_or_else(|| anyhow::anyhow!("_op column is not a StringArray"))?;
for row_idx in 0..op_array.len() {
if op_array.is_null(row_idx) || op_array.value(row_idx) != "c" {
return Ok(false);
}
}
Ok(true)
}
/// A sub-batch of rows sharing the same operation type, derived from the
/// `_op` column values.
struct OpSegment {
/// The sink operation for this segment.
op: InsertOp,
/// The `RecordBatch` containing only the rows for this segment, with
/// internal columns (`_op`, `_op_index`) already stripped.
batch: RecordBatch,
}
/// Sorts rows by `_op_index`, then splits the batch into consecutive segments
/// of the same `_op` value. Each segment is returned as an [`OpSegment`]
/// with internal columns stripped.
///
/// If the batch has no `_op` / `_op_index` columns (e.g. a pure-insert
/// initial batch), a single `Insert` segment covering all rows is returned.
fn split_batch_by_op(
batch: &RecordBatch,
key_columns: &[String],
) -> anyhow::Result<Vec<OpSegment>> {
let schema = batch.schema();
// If there is no _op column, treat the whole batch as an insert.
let op_idx = match schema.index_of("_op") {
Ok(idx) => idx,
Err(_) => {
let stripped = strip_internal_columns(batch)?;
return Ok(vec![OpSegment {
op: InsertOp::Insert,
batch: stripped,
}]);
}
};
// Sort by _op_index to ensure correct replay order.
let sorted_batch = if let Ok(oi_idx) = schema.index_of("_op_index") {
let op_index_col = batch.column(oi_idx);
let sort_indices = compute::sort_to_indices(op_index_col, None, None)?;
let columns: Vec<_> = batch
.columns()
.iter()
.map(|c| compute::take(c.as_ref(), &sort_indices, None).map_err(|e| e.into()))
.collect::<anyhow::Result<Vec<_>>>()?;
RecordBatch::try_new(batch.schema(), columns)?
} else {
batch.clone()
};
let op_array = sorted_batch
.column(op_idx)
.as_any()
.downcast_ref::<StringArray>()
.ok_or_else(|| anyhow::anyhow!("_op column is not a StringArray"))?;
let num_rows = sorted_batch.num_rows();
if num_rows == 0 {
return Ok(Vec::new());
}
// Strip internal columns once for the sorted batch, then slice this
// projected batch per op-run to avoid rebuilding schema/column vectors for
// every segment.
let projected_batch = strip_internal_columns(&sorted_batch)?;
// Walk through rows and group consecutive runs of the same operation.
let mut segments = Vec::new();
let mut run_start = 0usize;
let mut current_op = op_array.value(0);
for i in 1..num_rows {
let row_op = op_array.value(i);
if row_op != current_op {
// Flush the current run.
let stripped = projected_batch.slice(run_start, i - run_start);
segments.push(OpSegment {
op: op_str_to_insert_op(current_op, key_columns),
batch: stripped,
});
run_start = i;
current_op = row_op;
}
}
// Flush the final run.
let stripped = projected_batch.slice(run_start, num_rows - run_start);
segments.push(OpSegment {
op: op_str_to_insert_op(current_op, key_columns),
batch: stripped,
});
Ok(segments)
}
/// Builds sink op-segments for a batch, using a cheap insert-only fast path.
///
/// If all operations are create/insert (`_op == "c"` for every row), avoids
/// sorting and run-based segmentation and returns a single insert segment.
fn build_segments_for_batch(
batch: &RecordBatch,
key_columns: &[String],
mutations: &MutationConfig,
) -> anyhow::Result<Vec<OpSegment>> {
if batch_is_insert_only(batch, mutations)? {
let stripped = strip_internal_columns(batch)?;
return Ok(vec![OpSegment {
op: InsertOp::Insert,
batch: stripped,
}]);
}
split_batch_by_op(batch, key_columns)
}
/// Maps a `_op` column value (`"c"`, `"u"`, `"d"`) to an [`InsertOp`].
fn op_str_to_insert_op(op: &str, key_columns: &[String]) -> InsertOp {
match op {
"u" => InsertOp::Update {
key_columns: key_columns.to_vec(),
},
"d" => InsertOp::Delete {
key_columns: key_columns.to_vec(),
},
// "c" and anything else default to Insert.
_ => InsertOp::Insert,
}
}
async fn write_segments_for_batch(
data_sink: Arc<dyn Sink>,
table_name: &str,
batch_id: u64,
batch_ts: i64,
segments: Vec<OpSegment>,
output_schema: &SchemaRef,
partition_columns: &[String],
) -> Result<(), String> {
let table_name_owned = table_name.to_string();
// Log per-operation row counts for data reconciliation.
{
let mut insert_rows: usize = 0;
let mut update_rows: usize = 0;
let mut delete_rows: usize = 0;
for seg in &segments {
let n = seg.batch.num_rows();
match &seg.op {
InsertOp::Insert => insert_rows += n,
InsertOp::Update { .. } => update_rows += n,
InsertOp::Delete { .. } => delete_rows += n,
}
}
tracing::info!(
table = %table_name,
batch_id,
segments = segments.len(),
insert_rows,
update_rows,
delete_rows,
"Writing segments for batch",
);
}
let insert_only = segments
.iter()
.all(|segment| matches!(segment.op, InsertOp::Insert));
if !insert_only {
for segment in segments {
let output_batch =
append_created_at(&segment.batch, output_schema, batch_ts).map_err(|e| {
format!("append __created_at to {table_name_owned} batch {batch_id}: {e}")
})?;
data_sink
.write(
&table_name_owned,
batch_id,
output_batch,
segment.op,
partition_columns.to_vec(),
)
.await
.map_err(|e| format!("write {table_name_owned} batch {batch_id}: {e:#}"))?;
}
return Ok(());
}
let mut join_set: JoinSet<Result<(), String>> = JoinSet::new();
for segment in segments {
while join_set.len() >= MAX_IN_FLIGHT_TABLE_WRITES {
let result = join_set
.join_next()
.await
.ok_or_else(|| format!("No in-flight write task available for {table_name_owned}"))
.and_then(|r| {
r.map_err(|e| {
format!(
"Sink write task panicked for {table_name_owned} batch {batch_id}: {e}"
)
})
})?;
result?;
}
let data_sink = Arc::clone(&data_sink);
let table_name = table_name_owned.clone();
let partition_columns = partition_columns.to_vec();
let output_schema = Arc::clone(output_schema);
join_set.spawn(async move {
let output_batch = append_created_at(&segment.batch, &output_schema, batch_ts)
.map_err(|e| {
format!("append __created_at to {table_name} batch {batch_id}: {e}")
})?;
data_sink
.write(
&table_name,
batch_id,
output_batch,
segment.op,
partition_columns,
)
.await
.map_err(|e| format!("write {table_name} batch {batch_id}: {e}"))
});
}
while let Some(result) = join_set.join_next().await {
let inner = result.map_err(|e| {
format!("Sink write task panicked for {table_name_owned} batch {batch_id}: {e}")
})?;
inner?;
}
Ok(())
}
/// Specifies which dataset implementation to use for the ETL pipeline.
#[derive(Debug, Clone)]
pub enum DatasetSource {
/// A simple auto-incrementing integer sequence dataset.
SimpleSequence,
/// The TPC-H benchmark dataset generated via DuckDB.
Tpch,
}
impl DatasetSource {
/// Creates a [`DatasetSource`] from a dataset type string (e.g. from version metadata).
///
/// Supported values: `"tpch"`, `"simple_sequence"`.
pub fn from_dataset_type(dataset_type: &str) -> anyhow::Result<Self> {
match dataset_type {
"tpch" => Ok(DatasetSource::Tpch),
"simple_sequence" => Ok(DatasetSource::SimpleSequence),
other => {
anyhow::bail!("Unknown dataset type: {other}. Use 'tpch' or 'simple_sequence'.")
}
}
}
/// Create an [`Arc<dyn Dataset>`] for this source variant using the given
/// configuration.
///
/// Delegates to the [`Dataset::create`] factory method on the concrete type.
pub fn create(
&self,
config: &GenerationDatasetConfig,
mutations: &MutationConfig,
storage: Arc<dyn DataStorage>,
) -> anyhow::Result<Arc<dyn Dataset>> {
match self {
DatasetSource::SimpleSequence => {
SimpleSequenceDataset::create(config, mutations, storage)
}
DatasetSource::Tpch => TpchDataset::create(config, mutations, storage),
}
}
}
/// The current state of an [`ETLPipeline`].
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum PipelineState {
/// The pipeline has been created with a dataset, source, and target but has
/// not yet started processing.
NotStarted,
/// The pipeline has been initialized: the first batch for every table has
/// been ETL'd into the target so the system adapter can discover initial
/// data.
Initialized,
/// The pipeline is actively rehydrating batches (in order of batch ID) from
/// the configured [`Source`] into the configured [`Target`].
Running,
/// The pipeline has processed the requested number of steps and is waiting
/// for [`continue_pipeline`](ETLPipeline::continue_pipeline) to be called.
Paused,
/// The pipeline has completed, was cancelled, or encountered an error in its
/// background task.
Stopped(StopReason),
}
/// Why the pipeline entered the [`PipelineState::Stopped`] state.
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum StopReason {
/// All batches for every table were processed successfully.
Completed,
/// The pipeline was cancelled via its [`CancellationToken`].