tonbo 0.4.0-a1

Embedded database for serverless and edge runtimes, storing data as Parquet on S3
Documentation
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
//! Compaction executor contracts and scaffolding for major compaction.
//!
//! Design note: the executor is responsible for constructing target descriptors/config
//! rather than receiving pre-baked outputs from `DB::run_compaction_task`. This keeps
//! compaction policy (target level/paths/compression/tuning) local to the executor,
//! allows swapping executors without changing DB plumbing, and mirrors the autonomy we
//! want for remote/serverless compactors. The DB only supplies planner output and input
//! descriptors; the executor decides how to materialize new SSTs and report edits.

use std::{
    collections::HashMap,
    pin::Pin,
    sync::{
        Arc,
        atomic::{AtomicU64, Ordering},
    },
};

use fusio::dynamic::MaybeSendFuture;
use tracing::Instrument;
use ulid::Ulid;

use crate::{
    compaction::planner::CompactionTask,
    manifest::{ManifestError, SstEntry, VersionEdit, WalSegmentRef},
    mvcc::Timestamp,
    ondisk::{
        merge::cleanup_descriptors,
        sstable::{SsTableConfig, SsTableDescriptor, SsTableId, SsTableMerger},
    },
};

/// Lease/ownership token used when delegating compaction to a remote worker.
#[derive(Debug, Clone)]
#[allow(dead_code)]
pub(crate) struct CompactionLease {
    /// Unique identifier for the leased job.
    pub(crate) id: Ulid,
    /// Human-readable owner/worker identifier for observability.
    pub(crate) owner: String,
    /// Lease time-to-live in milliseconds; renew before expiry to retain ownership.
    pub(crate) ttl_ms: u64,
}

/// Execution context for a single planned compaction.

#[derive(Debug, Clone)]
pub(crate) struct CompactionJob {
    /// Planner output describing which level/inputs to merge.
    pub(crate) task: CompactionTask,
    /// Resolved input SST descriptors (including stats/paths) for the task.
    pub(crate) inputs: Vec<SsTableDescriptor>,
    /// Optional lease token when jobs are handed out to remote executors.
    pub(crate) lease: Option<CompactionLease>,
}

/// Outcome of a successful compaction run.

#[derive(Debug, Clone)]
pub(crate) struct CompactionOutcome {
    /// Newly produced SST entries to be added to the target level.
    pub(crate) add_ssts: Vec<SstEntry>,
    /// SST identifiers that should be removed once compaction finishes.
    pub(crate) remove_ssts: Vec<SsTableDescriptor>,
    /// Target level receiving the new SSTs.
    pub(crate) target_level: u32,
    /// Complete WAL segment set that remains visible after the compaction.
    pub(crate) wal_segments: Option<Vec<WalSegmentRef>>,
    /// Optional watermark for tombstone visibility.
    pub(crate) tombstone_watermark: Option<u64>,
    /// SST outputs produced by the executor (useful for GC hints/tests).
    pub(crate) outputs: Vec<SsTableDescriptor>,
    /// Obsolete SST ids eligible for GC.
    pub(crate) obsolete_sst_ids: Vec<SsTableId>,
    /// Optional WAL floor advancement hint (last visible segment).
    pub(crate) wal_floor: Option<WalSegmentRef>,
    /// WAL segments made obsolete by this compaction (for GC hints).
    pub(crate) obsolete_wal_segments: Vec<WalSegmentRef>,
}

impl CompactionOutcome {
    /// Build manifest edits representing the outcome.
    pub(crate) fn to_version_edits(&self) -> Vec<VersionEdit> {
        let mut edits = Vec::new();
        let mut level_for_sst: HashMap<SsTableId, u32> = HashMap::new();
        for desc in &self.remove_ssts {
            level_for_sst.insert(desc.id().clone(), desc.level() as u32);
        }

        let mut remove_by_level: HashMap<u32, Vec<SsTableId>> = HashMap::new();
        for desc in &self.remove_ssts {
            remove_by_level
                .entry(desc.level() as u32)
                .or_default()
                .push(desc.id().clone());
        }
        for sst_id in &self.obsolete_sst_ids {
            if let Some(level) = level_for_sst.get(sst_id).copied() {
                remove_by_level
                    .entry(level)
                    .or_default()
                    .push(sst_id.clone());
            }
        }

        for (level, mut sst_ids) in remove_by_level {
            sst_ids.sort_by_key(SsTableId::raw);
            sst_ids.dedup();
            if !sst_ids.is_empty() {
                edits.push(VersionEdit::RemoveSsts { level, sst_ids });
            }
        }
        if !self.add_ssts.is_empty() {
            edits.push(VersionEdit::AddSsts {
                level: self.target_level,
                entries: self.add_ssts.clone(),
            });
        }
        let wal_segments = self
            .wal_segments
            .clone()
            .or_else(|| self.wal_floor.as_ref().map(|floor| vec![floor.clone()]));
        if let Some(segments) = wal_segments {
            edits.push(VersionEdit::SetWalSegments { segments });
        }
        if let Some(watermark) = self.tombstone_watermark {
            edits.push(VersionEdit::SetTombstoneWatermark { watermark });
        }
        edits
    }

    /// Build a compaction outcome from finished SST descriptors, validating required paths.
    pub(crate) fn from_outputs(
        outputs: Vec<SsTableDescriptor>,
        remove_ssts: Vec<SsTableDescriptor>,
        target_level: u32,
        wal_segments: Option<Vec<WalSegmentRef>>,
    ) -> Result<Self, CompactionError> {
        let mut add_ssts = Vec::with_capacity(outputs.len());
        let mut max_commit: Option<Timestamp> = None;
        for desc in &outputs {
            let data_path = desc
                .data_path()
                .cloned()
                .ok_or(CompactionError::MissingPath("data"))?;
            let delete_path = desc.delete_path().cloned();
            if let Some(stats) = desc.stats()
                && let Some(ts) = stats.max_commit_ts
            {
                max_commit = match max_commit {
                    Some(current) if current >= ts => Some(current),
                    _ => Some(ts),
                };
            }
            let entry = SstEntry::new(
                desc.id().clone(),
                desc.stats().cloned(),
                desc.wal_ids().map(|ids| ids.to_vec()),
                data_path,
                delete_path,
            );
            add_ssts.push(entry);
        }
        Ok(Self {
            add_ssts,
            remove_ssts,
            target_level,
            wal_segments,
            outputs,
            obsolete_sst_ids: Vec::new(),
            wal_floor: None,
            tombstone_watermark: max_commit.map(|ts| ts.get()),
            obsolete_wal_segments: Vec::new(),
        })
    }
}

/// Errors that can surface while executing compaction.
#[derive(Debug, thiserror::Error)]
pub(crate) enum CompactionError {
    /// Planner or manifest interaction failed.
    #[error(transparent)]
    Manifest(#[from] ManifestError),
    /// CAS conflict while publishing manifest edits.
    #[error("manifest CAS conflict")]
    CasConflict,
    /// An expected storage path was missing from an SST descriptor.
    #[error("compaction output missing path for {0}")]
    MissingPath(&'static str),
    /// SST execution failed.
    #[error(transparent)]
    Sst(#[from] crate::ondisk::sstable::SsTableError),
    /// Executor invoked without any inputs to merge.
    #[error("compaction executor received no inputs")]
    NoInputs,
}

/// Async trait for orchestrating a major compaction over SST inputs.
pub(crate) trait CompactionExecutor {
    /// Execute a compaction job and return a manifest edit describing the change.
    fn execute(
        &self,
        job: CompactionJob,
    ) -> Pin<Box<dyn MaybeSendFuture<Output = Result<CompactionOutcome, CompactionError>> + '_>>;

    /// Best-effort cleanup hook for outputs produced during execution. Used when manifest
    /// publication fails (e.g., CAS conflict) so temporary artifacts do not leak.
    fn cleanup_outputs<'a>(
        &'a self,
        outputs: &'a [SsTableDescriptor],
    ) -> Pin<Box<dyn MaybeSendFuture<Output = Result<(), CompactionError>> + 'a>>;
}

/// Local executor that merges SST inputs on the current host.
#[derive(Debug, Clone)]
pub(crate) struct LocalCompactionExecutor {
    config: Arc<SsTableConfig>,
    next_id: Arc<AtomicU64>,
    max_output_rows: Option<usize>,
    max_output_bytes: Option<usize>,
    #[cfg(test)]
    corrupt_descriptors: bool,
}

const DEFAULT_BASE_OUTPUT_BYTES: usize = 128 * 1024 * 1024; // 128 MiB
const DEFAULT_LEVEL_MULTIPLIER: usize = 10;
const DEFAULT_OUTPUT_HARD_CAP_BYTES: usize = 512 * 1024 * 1024; // 512 MiB safety cap

impl LocalCompactionExecutor {
    /// Build a local executor that will use `config` for outputs and allocate SST ids starting at
    /// `start_id`.
    #[cfg(test)]
    pub(crate) fn new(config: Arc<SsTableConfig>, start_id: u64) -> Self {
        Self::with_id_allocator(config, Arc::new(AtomicU64::new(start_id)))
    }

    /// Build a local executor that draws SST ids from a shared allocator.
    pub(crate) fn with_id_allocator(config: Arc<SsTableConfig>, next_id: Arc<AtomicU64>) -> Self {
        Self {
            config,
            next_id,
            max_output_rows: None,
            max_output_bytes: None,
            #[cfg(test)]
            corrupt_descriptors: false,
        }
    }

    /// Cap the number of bytes per output SST. Prevents oversized single files when splitting.
    pub(crate) fn with_max_output_bytes(mut self, max_output_bytes: usize) -> Self {
        self.max_output_bytes = Some(max_output_bytes.max(1));
        self
    }

    /// Cap the number of rows per output SST. Prevents oversized single files when splitting.
    pub(crate) fn with_max_output_rows(mut self, max_output_rows: usize) -> Self {
        self.max_output_rows = Some(max_output_rows.max(1));
        self
    }

    /// Test-only hook to corrupt descriptors before outcome construction and exercise cleanup.
    #[cfg(all(test, feature = "tokio"))]
    pub(crate) fn with_corrupt_descriptors_for_test(mut self) -> Self {
        self.corrupt_descriptors = true;
        self
    }

    fn default_output_bytes_for_level(level: usize) -> usize {
        let mut size = DEFAULT_BASE_OUTPUT_BYTES;
        for _ in 0..level {
            size = size.saturating_mul(DEFAULT_LEVEL_MULTIPLIER);
            if size >= DEFAULT_OUTPUT_HARD_CAP_BYTES {
                return DEFAULT_OUTPUT_HARD_CAP_BYTES;
            }
        }
        size.min(DEFAULT_OUTPUT_HARD_CAP_BYTES)
    }

    fn output_caps_for_level(&self, level: usize) -> (Option<usize>, Option<usize>) {
        let rows_cap = self.max_output_rows;
        // Only apply hard cap to defaults; respect explicit caller overrides.
        let bytes_cap = self
            .max_output_bytes
            .or_else(|| Some(Self::default_output_bytes_for_level(level)));
        (rows_cap, bytes_cap)
    }

    fn alloc_descriptor(&self, level: usize) -> SsTableDescriptor {
        let id = self.next_id.fetch_add(1, Ordering::Relaxed);
        SsTableDescriptor::new(SsTableId::new(id), level)
    }
}

impl CompactionExecutor for LocalCompactionExecutor {
    fn execute(
        &self,
        job: CompactionJob,
    ) -> Pin<Box<dyn MaybeSendFuture<Output = Result<CompactionOutcome, CompactionError>> + '_>>
    {
        let source_level = job.task.source_level;
        let target_level = job.task.target_level;
        let input_count = job.inputs.len();
        let span = tracing::info_span!(
            "compaction::execute",
            component = "compaction",
            source_level,
            target_level,
            input_count,
        );
        Box::pin(
            async move {
                if job.inputs.is_empty() {
                    return Err(CompactionError::NoInputs);
                }
                let _ = job.lease.as_ref();
                let target = self.alloc_descriptor(job.task.target_level);
                let (max_rows, max_bytes) = self.output_caps_for_level(job.task.target_level);
                let merger =
                    SsTableMerger::new(Arc::clone(&self.config), job.inputs.clone(), target)
                        .with_output_id_allocator(Arc::clone(&self.next_id))
                        .with_output_caps(max_rows, max_bytes);
                let merged = {
                    #[cfg(feature = "tokio")]
                    {
                        merger
                            .execute(fusio::executor::tokio::TokioExecutor::default())
                            .await?
                    }
                    #[cfg(not(feature = "tokio"))]
                    {
                        merger.execute(fusio::executor::NoopExecutor).await?
                    }
                };
                let descriptors: Vec<_> =
                    merged.iter().map(|sst| sst.descriptor().clone()).collect();
                let descriptors_for_outcome = {
                    #[cfg(test)]
                    {
                        if self.corrupt_descriptors {
                            let mut corrupted = descriptors.clone();
                            if let Some(first) = corrupted.first_mut() {
                                *first = SsTableDescriptor::new(first.id().clone(), first.level());
                            }
                            corrupted
                        } else {
                            descriptors.clone()
                        }
                    }
                    #[cfg(not(test))]
                    {
                        descriptors.clone()
                    }
                };
                match CompactionOutcome::from_outputs(
                    descriptors_for_outcome,
                    job.inputs,
                    job.task.target_level as u32,
                    None,
                ) {
                    Ok(outcome) => Ok(outcome),
                    Err(err) => {
                        // Best-effort cleanup for partial outputs; surface original error.
                        let _ = self.cleanup_outputs(&descriptors).await;
                        Err(err)
                    }
                }
            }
            .instrument(span),
        )
    }

    fn cleanup_outputs<'a>(
        &'a self,
        outputs: &'a [SsTableDescriptor],
    ) -> Pin<Box<dyn MaybeSendFuture<Output = Result<(), CompactionError>> + 'a>> {
        Box::pin(async move {
            cleanup_descriptors(&self.config, outputs).await;
            Ok(())
        })
    }
}

#[cfg(all(test, feature = "tokio"))]
mod tests {
    use std::sync::Arc;

    use arrow_schema::{DataType, Field, Schema};
    use fusio::{disk::LocalFs, dynamic::DynFs, path::Path};
    use futures::StreamExt;
    use tempfile::tempdir;
    use typed_arrow_dyn::{DynCell, DynRow};

    use super::*;
    use crate::{
        compaction::planner::{CompactionInput, CompactionTask},
        id::FileIdGenerator,
        inmem::immutable::memtable::segment_from_batch_with_key_name,
        ondisk::sstable::{
            SsTableBuilder, SsTableConfig, SsTableDescriptor, SsTableId, SsTableStats,
        },
        schema::SchemaBuilder,
        test::build_batch,
    };

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn local_executor_cleans_outputs_when_outcome_build_fails() {
        let schema = Arc::new(Schema::new(vec![
            Field::new("id", DataType::Utf8, false),
            Field::new("v", DataType::Int32, false),
        ]));
        let tmpdir = tempdir().expect("tempdir");
        let mode_cfg = SchemaBuilder::from_schema(Arc::clone(&schema))
            .primary_key("id")
            .build()
            .expect("schema builder");
        let extractor = Arc::clone(&mode_cfg.extractor);
        let fs: Arc<dyn DynFs> = Arc::new(LocalFs {});
        let cfg = Arc::new(
            SsTableConfig::new(
                Arc::clone(&mode_cfg.schema),
                fs,
                Path::from(tmpdir.path().to_string_lossy().to_string()),
            )
            .with_key_extractor(extractor),
        );

        let batch = build_batch(
            Arc::clone(&schema),
            vec![
                DynRow(vec![Some(DynCell::Str("a".into())), Some(DynCell::I32(1))]),
                DynRow(vec![Some(DynCell::Str("b".into())), Some(DynCell::I32(2))]),
            ],
        )
        .expect("batch");
        let immutable =
            segment_from_batch_with_key_name(batch, "id").expect("immutable segment from batch");
        let mut builder = SsTableBuilder::new(
            Arc::clone(&cfg),
            SsTableDescriptor::new(SsTableId::new(1), 0),
        );
        builder.add_immutable(&immutable).expect("stage seg");
        let input = builder
            .finish(fusio::executor::NoopExecutor)
            .await
            .expect("sst");

        let task = CompactionTask {
            source_level: 0,
            target_level: 1,
            input: vec![CompactionInput {
                level: 0,
                sst_id: input.descriptor().id().clone(),
            }],
            key_range: None,
        };
        let job = CompactionJob {
            task,
            inputs: vec![input.descriptor().clone()],
            lease: None,
        };

        let executor =
            LocalCompactionExecutor::new(Arc::clone(&cfg), 10).with_corrupt_descriptors_for_test();
        let result = executor.execute(job).await;
        match result {
            Err(CompactionError::MissingPath("data")) => {}
            other => panic!("expected missing path error, got {other:?}"),
        }

        let level_dir = cfg.level_dir(1).expect("level dir");
        let mut entries = cfg
            .fs()
            .list(&level_dir)
            .await
            .expect("level dir should exist");
        let mut count = 0usize;
        while let Some(item) = entries.next().await.transpose().expect("stream ok") {
            let name = item.path.as_ref();
            if name.ends_with(".parquet")
                || name.ends_with(".mvcc.parquet")
                || name.ends_with(".delete.parquet")
            {
                count += 1;
            }
        }
        assert_eq!(count, 0, "expected compaction outputs to be cleaned");

        drop(tmpdir);
    }

    #[test]
    fn outcome_builds_version_edits() {
        let remove = SsTableDescriptor::new(SsTableId::new(7), 0);
        let wal_id = FileIdGenerator::default().generate();
        let add = SstEntry::new(
            SsTableId::new(9),
            Some(SsTableStats::default()),
            Some(vec![wal_id]),
            Path::from("L1/000000000000000009.parquet"),
            None,
        );
        let outcome = CompactionOutcome {
            add_ssts: vec![add.clone()],
            remove_ssts: vec![remove.clone()],
            target_level: 1,
            wal_segments: Some(vec![WalSegmentRef::new(0, wal_id, 0, 0)]),
            tombstone_watermark: Some(42),
            outputs: vec![],
            obsolete_sst_ids: vec![],
            wal_floor: None,
            obsolete_wal_segments: Vec::new(),
        };
        let edits = outcome.to_version_edits();
        assert_eq!(edits.len(), 4);
        assert!(edits.iter().any(|edit| matches!(
            edit,
            VersionEdit::RemoveSsts { sst_ids, .. } if sst_ids.contains(remove.id())
        )));
        assert!(edits.iter().any(|edit| matches!(
            edit,
            VersionEdit::AddSsts { level, entries } if *level == 1 && entries.first().map(|e| e.sst_id()) == Some(add.sst_id())
        )));
        assert!(edits.iter().any(|edit| matches!(
            edit,
            VersionEdit::SetWalSegments { segments } if segments.len() == 1
        )));
        assert!(edits.iter().any(|edit| matches!(
            edit,
            VersionEdit::SetTombstoneWatermark { watermark } if *watermark == 42
        )));
    }

    #[test]
    fn from_outputs_builds_entries_and_watermark() {
        let wal_id = FileIdGenerator::default().generate();
        let stats = SsTableStats {
            rows: 2,
            bytes: 10,
            tombstones: 1,
            min_key: None,
            max_key: None,
            min_commit_ts: None,
            max_commit_ts: Some(Timestamp::new(7)),
        };
        let output = SsTableDescriptor::new(SsTableId::new(11), 1)
            .with_stats(stats.clone())
            .with_wal_ids(Some(vec![wal_id]))
            .with_storage_paths(
                Path::from("L1/000000000000000011.parquet"),
                Some(Path::from("L1/000000000000000011.delete.parquet")),
            );
        let remove = SsTableDescriptor::new(SsTableId::new(5), 0);
        let outcome = CompactionOutcome::from_outputs(
            vec![output.clone()],
            vec![remove.clone()],
            1,
            Some(vec![WalSegmentRef::new(0, wal_id, 0, 0)]),
        )
        .expect("outcome");
        assert_eq!(outcome.add_ssts.len(), 1);
        assert_eq!(outcome.remove_ssts.len(), 1);
        assert_eq!(outcome.target_level, 1);
        assert_eq!(outcome.tombstone_watermark, Some(7));
        assert_eq!(
            outcome
                .add_ssts
                .first()
                .and_then(|entry| entry.stats())
                .map(|s| s.max_commit_ts),
            Some(stats.max_commit_ts)
        );
    }

    #[test]
    fn from_outputs_fails_on_missing_paths() {
        let desc = SsTableDescriptor::new(SsTableId::new(3), 0);
        let err = CompactionOutcome::from_outputs(vec![desc], Vec::new(), 0, None)
            .expect_err("missing paths");
        assert!(matches!(err, CompactionError::MissingPath("data")));
    }

    #[test]
    fn default_output_caps_are_applied() {
        let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Utf8, false)]));
        let tmpdir = tempfile::tempdir().expect("temp dir");
        let mode_cfg = SchemaBuilder::from_schema(Arc::clone(&schema))
            .primary_key("id")
            .build()
            .expect("schema builder");
        let fs: Arc<dyn DynFs> = Arc::new(fusio::disk::LocalFs {});
        let cfg = Arc::new(
            SsTableConfig::new(
                Arc::clone(&mode_cfg.schema),
                fs,
                Path::from(tmpdir.path().to_string_lossy().to_string()),
            )
            .with_key_extractor(Arc::clone(&mode_cfg.extractor)),
        );
        let exec = LocalCompactionExecutor::new(cfg, 0);
        let (rows_cap, bytes_cap) = exec.output_caps_for_level(1);
        assert!(rows_cap.is_none());
        let bytes = bytes_cap.expect("default bytes cap");
        assert!(bytes <= DEFAULT_OUTPUT_HARD_CAP_BYTES);
        // Keep tmpdir alive for the duration of the test.
        drop(tmpdir);
    }

    #[test]
    fn outcome_emits_per_level_removals() {
        let remove_l1 = SsTableDescriptor::new(SsTableId::new(1), 1);
        let remove_l2 = SsTableDescriptor::new(SsTableId::new(2), 2);
        let outcome = CompactionOutcome {
            add_ssts: Vec::new(),
            remove_ssts: vec![remove_l1.clone(), remove_l2.clone()],
            target_level: 2,
            wal_segments: None,
            tombstone_watermark: None,
            outputs: vec![],
            obsolete_sst_ids: vec![remove_l2.id().clone()],
            wal_floor: None,
            obsolete_wal_segments: Vec::new(),
        };

        let edits = outcome.to_version_edits();
        let mut levels: Vec<u32> = Vec::new();
        let mut remove_ids: Vec<Vec<SsTableId>> = Vec::new();
        for edit in edits {
            if let VersionEdit::RemoveSsts { level, sst_ids } = edit {
                levels.push(level);
                remove_ids.push(sst_ids);
            }
        }
        assert_eq!(levels.len(), 2);
        assert!(levels.contains(&1));
        assert!(levels.contains(&2));
        assert!(remove_ids.iter().any(|ids| ids.contains(remove_l1.id())));
        assert!(remove_ids.iter().any(|ids| ids.contains(remove_l2.id())));
    }

    #[test]
    fn to_version_edits_dedups_remove_and_falls_back_to_floor() {
        let wal_id = FileIdGenerator::default().generate();
        let wal_floor = WalSegmentRef::new(7, wal_id, 0, 10);
        let remove = SsTableDescriptor::new(SsTableId::new(1), 0);
        let outcome = CompactionOutcome {
            add_ssts: Vec::new(),
            remove_ssts: vec![remove.clone()],
            target_level: 0,
            wal_segments: None,
            tombstone_watermark: None,
            outputs: Vec::new(),
            obsolete_sst_ids: vec![remove.id().clone()],
            wal_floor: Some(wal_floor.clone()),
            obsolete_wal_segments: Vec::new(),
        };
        let edits = outcome.to_version_edits();
        assert_eq!(edits.len(), 2);
        assert!(matches!(
            &edits[0],
            VersionEdit::RemoveSsts { sst_ids, .. }
                if sst_ids.len() == 1 && sst_ids.contains(remove.id())
        ));
        assert!(matches!(
            &edits[1],
            VersionEdit::SetWalSegments { segments }
                if segments.len() == 1 && segments[0].seq() == wal_floor.seq()
        ));
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 1)]
    async fn local_executor_merges_outputs() {
        let schema = Arc::new(Schema::new(vec![
            Field::new("id", DataType::Utf8, false),
            Field::new("v", DataType::Int32, false),
        ]));
        let tmpdir = tempfile::tempdir().expect("temp dir");
        let mode_cfg = SchemaBuilder::from_schema(Arc::clone(&schema))
            .primary_key("id")
            .build()
            .expect("schema builder");
        let fs: Arc<dyn DynFs> = Arc::new(fusio::disk::LocalFs {});
        let cfg = Arc::new(
            SsTableConfig::new(
                Arc::clone(&mode_cfg.schema),
                fs,
                Path::from(tmpdir.path().to_string_lossy().to_string()),
            )
            .with_key_extractor(Arc::clone(&mode_cfg.extractor)),
        );
        let exec = LocalCompactionExecutor::new(cfg, 100);
        let batch = crate::test::build_batch(
            Arc::clone(&schema),
            vec![DynRow(vec![
                Some(DynCell::Str("a".into())),
                Some(DynCell::I32(1)),
            ])],
        )
        .expect("batch");
        let immutable =
            segment_from_batch_with_key_name(batch, "id").expect("immutable segment from batch");
        let mut builder = SsTableBuilder::new(
            Arc::clone(&exec.config),
            SsTableDescriptor::new(SsTableId::new(1), 0),
        );
        builder.add_immutable(&immutable).expect("stage seg");
        let input = builder
            .finish(fusio::executor::NoopExecutor)
            .await
            .expect("sst")
            .descriptor()
            .clone();
        let job = CompactionJob {
            task: CompactionTask {
                source_level: 0,
                target_level: 1,
                input: vec![CompactionInput {
                    level: 0,
                    sst_id: SsTableId::new(1),
                }],
                key_range: None,
            },
            inputs: vec![input],
            lease: None,
        };
        let out = exec.execute(job).await.expect("outcome");
        assert_eq!(out.add_ssts.len(), 1);
        assert_eq!(out.target_level, 1);
    }
}