slatedb 0.12.1

A cloud native embedded storage engine built on object storage.
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
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
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
//! Parallel L0 flush manifest manifest_writer.
//!
//! The manifest_writer owns ordered retirement of uploaded L0 tables:
//! - restore flush order using sequence ranges
//! - apply ordered in-memory manifest state transitions
//! - persist manifest updates
//! - report durable progress
//! - create checkpoints against manifest-owned state
//!
//! It does not own:
//! - upload execution
//! - flush request semantics
//! - flush waiter bookkeeping

use log::debug;

use super::tracker::TrackerMessage;
use super::uploader::UploadedMemtable;
use crate::checkpoint::CheckpointCreateResult;
use crate::config::CheckpointOptions;
use crate::db::DbInner;
use crate::db_state::SsTableView;
use crate::dispatcher::MessageHandler;
use crate::error::SlateDBError;
use crate::manifest::store::FenceableManifest;
use crate::oracle::Oracle;
use crate::utils::IdGenerator;
use crate::utils::SafeSender;
use async_trait::async_trait;
use futures::stream::BoxStream;
use futures::StreamExt;
use std::cmp;
use std::collections::BTreeMap;
use std::sync::Arc;
use std::time::Duration;
use tokio::runtime::Handle;
use tokio::sync::oneshot;

/// Result reported for a completed flush request.
#[derive(Debug, Clone, PartialEq, Eq)]
pub(crate) struct FlushResult {
    /// Highest durable sequence number covered by the completed flush (inclusive).
    pub(crate) durable_seq: u64,
}

/// Command submitted to the manifest_writer.
enum ManifestWriterCommand {
    /// One uploaded table is ready for ordered retirement.
    Uploaded(Box<UploadedMemtable>),
    /// Wait for a sequence to become durable, then respond with FlushResult.
    AwaitFlush {
        through_seq: Option<u64>,
        sender: oneshot::Sender<Result<FlushResult, SlateDBError>>,
    },
    /// Create a checkpoint against the current durable manifest state.
    CreateCheckpoint {
        through_seq: Option<u64>,
        options: CheckpointOptions,
        sender: oneshot::Sender<Result<CheckpointCreateResult, SlateDBError>>,
    },
    /// Periodic manifest poll to pick up remote changes (e.g. compaction).
    PollManifest,
}

impl std::fmt::Debug for ManifestWriterCommand {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        match self {
            Self::Uploaded(u) => {
                write!(
                    f,
                    "Uploaded(first_seq={}, last_seq={})",
                    u.first_seq, u.last_seq
                )
            }
            Self::AwaitFlush { through_seq, .. } => {
                write!(f, "AwaitFlush({through_seq:?})")
            }
            Self::CreateCheckpoint { through_seq, .. } => {
                write!(f, "CreateCheckpoint({through_seq:?})")
            }
            Self::PollManifest => write!(f, "PollManifest"),
        }
    }
}

pub(super) const MANIFEST_WRITER_TASK_NAME: &str = "l0_manifest_writer";

/// Ordered L0 retirement and manifest update subsystem.
pub(crate) struct ManifestWriter {
    commands_tx: SafeSender<ManifestWriterCommand>,
}

impl ManifestWriter {
    /// Starts the manifest_writer subsystem by registering with the executor.
    pub(crate) fn start(
        db: Arc<DbInner>,
        manifest: FenceableManifest,
        manifest_poll_interval: Duration,
        closed_result: &dyn crate::db_status::ClosedResultWriter,
        executor: &crate::dispatcher::MessageHandlerExecutor,
        tokio_handle: &Handle,
        tracker_tx: SafeSender<TrackerMessage>,
    ) -> Result<Self, SlateDBError> {
        let (commands_tx, commands_rx) =
            SafeSender::unbounded_channel(closed_result.result_reader());
        let handler = ManifestWriterHandler::new(db, manifest, manifest_poll_interval, tracker_tx);
        executor.add_handler(
            MANIFEST_WRITER_TASK_NAME.to_string(),
            Box::new(handler),
            commands_rx,
            tokio_handle,
        )?;
        Ok(Self { commands_tx })
    }

    /// Notifies the manifest_writer that one uploaded table is ready for ordered retirement.
    pub(crate) async fn notify_uploaded(
        &self,
        uploaded_memtable: UploadedMemtable,
    ) -> Result<(), SlateDBError> {
        self.commands_tx
            .send(ManifestWriterCommand::Uploaded(Box::new(uploaded_memtable)))
    }

    /// Sends a flush request to the manifest_writer. The manifest_writer will respond
    /// once all sequences up to and including `through_seq` are durable (or
    /// immediately if `None`).
    pub(crate) fn send_flush(
        &self,
        through_seq: Option<u64>,
        sender: oneshot::Sender<Result<FlushResult, SlateDBError>>,
    ) -> Result<(), SlateDBError> {
        self.commands_tx.send(ManifestWriterCommand::AwaitFlush {
            through_seq,
            sender,
        })
    }

    /// Sends a checkpoint request to the manifest_writer. The manifest_writer will write
    /// the checkpoint once all sequences up to and including `through_seq` are
    /// durable (or immediately if `None`) and respond via `sender`.
    pub(crate) fn send_checkpoint(
        &self,
        through_seq: Option<u64>,
        options: CheckpointOptions,
        sender: oneshot::Sender<Result<CheckpointCreateResult, SlateDBError>>,
    ) -> Result<(), SlateDBError> {
        self.commands_tx
            .send(ManifestWriterCommand::CreateCheckpoint {
                through_seq,
                options,
                sender,
            })
    }

    pub(crate) async fn shutdown(executor: &crate::dispatcher::MessageHandlerExecutor) {
        if let Err(e) = executor.shutdown_task(MANIFEST_WRITER_TASK_NAME).await {
            log::warn!("failed to shutdown l0 manifest writer [error={:?}]", e);
        }
    }
}

struct ManifestWriterHandler {
    db: Arc<DbInner>,
    manifest: FenceableManifest,
    manifest_poll_interval: Duration,
    tracker_tx: SafeSender<TrackerMessage>,
    /// Uploaded memtables waiting for contiguous ordering, keyed by first_seq.
    ready: BTreeMap<u64, UploadedMemtable>,
    /// The first_seq we expect for the next memtable to process.
    next_seq: u64,
    /// Highest last_seq that has been durably written to the manifest (inclusive).
    durable_seq: u64,
    pending_flushes: Vec<PendingFlush>,
    pending_checkpoints: Vec<PendingCheckpoint>,
}

#[async_trait]
impl MessageHandler<ManifestWriterCommand> for ManifestWriterHandler {
    fn tickers(
        &mut self,
    ) -> Vec<(
        Duration,
        Box<crate::dispatcher::MessageFactory<ManifestWriterCommand>>,
    )> {
        vec![(
            self.manifest_poll_interval,
            Box::new(|| ManifestWriterCommand::PollManifest),
        )]
    }

    async fn handle(&mut self, command: ManifestWriterCommand) -> Result<(), SlateDBError> {
        match command {
            ManifestWriterCommand::Uploaded(uploaded_memtable) => {
                self.handle_uploaded(*uploaded_memtable).await
            }
            ManifestWriterCommand::AwaitFlush {
                through_seq,
                sender,
            } => {
                self.handle_flush(through_seq, sender);
                Ok(())
            }
            ManifestWriterCommand::CreateCheckpoint {
                through_seq,
                options,
                sender,
            } => {
                self.handle_create_checkpoint(through_seq, options, sender)
                    .await
            }
            ManifestWriterCommand::PollManifest => self.refresh_manifest_progress().await,
        }
    }

    async fn cleanup(
        &mut self,
        commands: BoxStream<'async_trait, ManifestWriterCommand>,
        result: Result<(), SlateDBError>,
    ) -> Result<(), SlateDBError> {
        let mut commands = commands.fuse();
        let close_result = self.try_graceful_cleanup(&mut commands, &result).await;
        // Drain any commands not consumed by graceful cleanup, collecting
        // flush/checkpoint waiters so they receive a proper error.
        while let Some(command) = commands.next().await {
            self.collect_pending_waiter(command);
        }
        // Any remaining pending waiters must fail with a concrete error.
        let error = result
            .and(close_result.clone())
            .err()
            .unwrap_or(SlateDBError::Closed);
        self.fail_pending_flushes(&error);
        self.fail_pending_checkpoints(&error);
        close_result
    }
}

impl ManifestWriterHandler {
    fn new(
        db: Arc<DbInner>,
        manifest: FenceableManifest,
        manifest_poll_interval: Duration,
        tracker_tx: SafeSender<TrackerMessage>,
    ) -> Self {
        let durable_seq = db.oracle.last_remote_persisted_seq();
        let next_seq = db.oracle.peek_next_seq();
        Self {
            db,
            manifest,
            manifest_poll_interval,
            tracker_tx,
            pending_flushes: Vec::new(),
            ready: BTreeMap::new(),
            next_seq,
            durable_seq,
            pending_checkpoints: Vec::new(),
        }
    }

    async fn handle_uploaded(
        &mut self,
        uploaded_memtable: UploadedMemtable,
    ) -> Result<(), SlateDBError> {
        assert!(
            uploaded_memtable.first_seq >= self.next_seq,
            "uploaded memtable first_seq ({}) is behind next_seq ({})",
            uploaded_memtable.first_seq,
            self.next_seq,
        );
        if self
            .ready
            .insert(uploaded_memtable.first_seq, uploaded_memtable)
            .is_some()
        {
            return Err(SlateDBError::InvalidDBState);
        }
        self.process_ready_work().await
    }

    fn handle_flush(
        &mut self,
        through_seq: Option<u64>,
        sender: oneshot::Sender<Result<FlushResult, SlateDBError>>,
    ) {
        if self.is_durable(through_seq) {
            let _ = sender.send(Ok(self.flush_result()));
        } else {
            self.pending_flushes.push(PendingFlush {
                through_seq,
                sender,
            });
        }
    }

    fn is_durable(&self, through_seq: Option<u64>) -> bool {
        match through_seq {
            None => true,
            Some(seq) => self.durable_seq >= seq,
        }
    }

    fn flush_result(&self) -> FlushResult {
        FlushResult {
            durable_seq: self.durable_seq,
        }
    }

    async fn handle_create_checkpoint(
        &mut self,
        through_seq: Option<u64>,
        options: CheckpointOptions,
        sender: oneshot::Sender<Result<CheckpointCreateResult, SlateDBError>>,
    ) -> Result<(), SlateDBError> {
        if self.is_durable(through_seq) {
            let result = self.write_checkpoint_safely(&options).await;
            let _ = sender.send(result.clone());
            return result.map(|_| ());
        }

        self.pending_checkpoints.push(PendingCheckpoint {
            through_seq,
            options,
            sender,
        });
        self.process_ready_work().await
    }

    async fn process_ready_work(&mut self) -> Result<(), SlateDBError> {
        loop {
            let Some(staged_batch) = self.take_next_ready_batch() else {
                return Ok(());
            };
            let through_seq = staged_batch
                .last()
                .map(|uploaded| uploaded.last_seq)
                .expect("staged batch should not be empty");
            let attached_checkpoints = self.take_satisfied_pending_checkpoints(through_seq);
            self.apply_ready_batch(staged_batch, attached_checkpoints, through_seq)
                .await?;
        }
    }

    fn take_next_ready_batch(&mut self) -> Option<Vec<UploadedMemtable>> {
        let mut next_seq = self.next_seq;
        let mut batch = Vec::new();
        while let Some(uploaded) = self.ready.remove(&next_seq) {
            next_seq = uploaded.last_seq + 1;
            batch.push(uploaded);
        }

        if batch.is_empty() {
            None
        } else {
            self.next_seq = next_seq;
            Some(batch)
        }
    }

    fn take_satisfied_pending_checkpoints(&mut self, through_seq: u64) -> Vec<PendingCheckpoint> {
        let mut satisfied = Vec::new();
        let mut pending = Vec::with_capacity(self.pending_checkpoints.len());
        for checkpoint in self.pending_checkpoints.drain(..) {
            if checkpoint
                .through_seq
                .is_none_or(|required_seq| required_seq <= through_seq)
            {
                satisfied.push(checkpoint);
            } else {
                pending.push(checkpoint);
            }
        }
        self.pending_checkpoints = pending;
        satisfied
    }

    async fn apply_ready_batch(
        &mut self,
        staged_batch: Vec<UploadedMemtable>,
        attached_checkpoints: Vec<PendingCheckpoint>,
        through_seq: u64,
    ) -> Result<(), SlateDBError> {
        self.apply_uploaded_state(&staged_batch)?;

        for uploaded in &staged_batch {
            uploaded.imm_memtable.notify_uploaded(Ok(()));
        }
        self.db
            .db_stats
            .immutable_memtable_flushes
            .increment(staged_batch.len() as u64);

        match self
            .write_manifest_update_safely(
                &attached_checkpoints
                    .iter()
                    .map(|c| &c.options)
                    .collect::<Vec<_>>(),
            )
            .await
        {
            Ok(checkpoint_results) => {
                self.finish_ready_batch(
                    staged_batch,
                    attached_checkpoints,
                    checkpoint_results,
                    through_seq,
                )
                .await
            }
            Err(err) => {
                self.fail_ready_batch(staged_batch, attached_checkpoints, err.clone())
                    .await?;
                Err(err)
            }
        }
    }

    fn apply_uploaded_state(&self, staged_batch: &[UploadedMemtable]) -> Result<(), SlateDBError> {
        let min_active_snapshot_seq = [
            self.db.snapshot_manager.min_active_seq(),
            self.db.txn_manager.min_active_seq(),
        ]
        .into_iter()
        .flatten()
        .min();
        let mut guard = self.db.state.write();
        let manifest = guard.modify(|modifier| {
            for uploaded in staged_batch {
                let uploaded_tracker = uploaded.imm_memtable.sequence_tracker();
                let popped = modifier
                    .state
                    .imm_memtable
                    .pop_back()
                    .expect("expected imm memtable");
                assert!(Arc::ptr_eq(&popped, &uploaded.imm_memtable));
                modifier
                    .state
                    .manifest
                    .value
                    .core
                    .l0
                    .push_front(SsTableView::new(
                        self.db.rand.rng().gen_ulid(self.db.system_clock.as_ref()),
                        uploaded.sst_handle.clone(),
                    ));
                modifier.state.manifest.value.core.replay_after_wal_id =
                    uploaded.imm_memtable.recent_flushed_wal_id();

                let memtable_tick = uploaded.imm_memtable.table().last_tick();
                modifier.state.manifest.value.core.last_l0_clock_tick = cmp::max(
                    modifier.state.manifest.value.core.last_l0_clock_tick,
                    memtable_tick,
                );
                if modifier.state.manifest.value.core.last_l0_clock_tick != memtable_tick {
                    return Err(SlateDBError::InvalidClockTick {
                        last_tick: modifier.state.manifest.value.core.last_l0_clock_tick,
                        next_tick: memtable_tick,
                    });
                }

                assert!(uploaded.last_seq >= modifier.state.manifest.value.core.last_l0_seq);
                modifier.state.manifest.value.core.last_l0_seq = uploaded.last_seq;
                modifier.state.manifest.value.core.recent_snapshot_min_seq =
                    min_active_snapshot_seq.unwrap_or(uploaded.last_seq);

                modifier
                    .state
                    .manifest
                    .value
                    .core
                    .sequence_tracker
                    .extend_from(uploaded_tracker);
            }
            Ok(modifier.state.manifest.clone())
        })?;

        drop(guard);
        self.db.status_manager.report_manifest(manifest.into());
        Ok(())
    }

    async fn write_manifest_update_safely(
        &mut self,
        checkpoint_options: &[&CheckpointOptions],
    ) -> Result<Vec<CheckpointCreateResult>, SlateDBError> {
        loop {
            let result = self.write_manifest_update(checkpoint_options).await;
            if matches!(result, Err(SlateDBError::TransactionalObjectVersionExists)) {
                self.load_manifest().await?;
            } else {
                return result;
            }
        }
    }

    async fn write_manifest_update(
        &mut self,
        checkpoint_options: &[&CheckpointOptions],
    ) -> Result<Vec<CheckpointCreateResult>, SlateDBError> {
        let mut dirty = self.clone_local_manifest_for_write();
        let mut checkpoint_results = Vec::new();
        for options in checkpoint_options {
            let id = self.db.rand.rng().gen_uuid();
            let checkpoint = self.manifest.new_checkpoint(id, options)?;
            let manifest_id = checkpoint.manifest_id;
            dirty.value.core.checkpoints.push(checkpoint);
            checkpoint_results.push(CheckpointCreateResult { id, manifest_id });
        }
        self.manifest.update(dirty).await?;
        Ok(checkpoint_results)
    }

    async fn write_current_manifest_safely(&mut self) -> Result<(), SlateDBError> {
        loop {
            let result = self.write_current_manifest().await;
            if matches!(result, Err(SlateDBError::TransactionalObjectVersionExists)) {
                self.load_manifest().await?;
            } else {
                return result;
            }
        }
    }

    async fn write_current_manifest(&mut self) -> Result<(), SlateDBError> {
        let dirty = self.clone_local_manifest_for_write();
        self.manifest.update(dirty.clone()).await?;
        self.db.status_manager.report_manifest(dirty.into());
        Ok(())
    }

    fn clone_local_manifest_for_write(
        &self,
    ) -> slatedb_txn_obj::DirtyObject<crate::manifest::Manifest> {
        let dirty = {
            let rguard_state = self.db.state.read();
            rguard_state.state().manifest.clone()
        };
        dirty
    }

    async fn load_manifest(&mut self) -> Result<(), SlateDBError> {
        self.manifest.refresh().await?;
        let remote_dirty = self.manifest.prepare_dirty()?;
        self.merge_remote_manifest(remote_dirty);
        Ok(())
    }

    async fn refresh_manifest_progress(&mut self) -> Result<(), SlateDBError> {
        self.manifest.refresh().await?;
        let remote_dirty = self.manifest.prepare_dirty()?;
        self.merge_remote_manifest(remote_dirty);
        let _ = self.tracker_tx.send(TrackerMessage::ManifestRefreshed);
        Ok(())
    }

    fn merge_remote_manifest(
        &self,
        remote_dirty: slatedb_txn_obj::DirtyObject<crate::manifest::Manifest>,
    ) {
        let dirty_manifest = {
            let mut wguard_state = self.db.state.write();
            wguard_state.merge_remote_manifest(remote_dirty);
            let cow = wguard_state.state();
            self.db
                .db_stats
                .l0_sst_count
                .set(cow.core().l0.len() as i64);
            cow.manifest.clone()
        };
        self.db
            .status_manager
            .report_manifest(dirty_manifest.into());
    }

    async fn write_checkpoint_safely(
        &mut self,
        options: &CheckpointOptions,
    ) -> Result<CheckpointCreateResult, SlateDBError> {
        self.load_manifest().await?;
        let mut results = self.write_manifest_update_safely(&[options]).await?;
        Ok(results
            .pop()
            .expect("checkpoint write should return exactly one result"))
    }

    async fn finish_ready_batch(
        &mut self,
        staged_batch: Vec<UploadedMemtable>,
        attached_checkpoints: Vec<PendingCheckpoint>,
        checkpoint_results: Vec<CheckpointCreateResult>,
        through_seq: u64,
    ) -> Result<(), SlateDBError> {
        debug!(
            "l0 flush batch written to manifest [batch_size={}, through_seq={}]",
            staged_batch.len(),
            through_seq,
        );
        self.durable_seq = through_seq;
        for uploaded in &staged_batch {
            uploaded.imm_memtable.table().notify_durable(Ok(()));
            self.db.oracle.advance_durable_seq(uploaded.last_seq);
        }
        self.resolve_pending_flushes();
        for (checkpoint, result) in attached_checkpoints
            .into_iter()
            .zip(checkpoint_results.into_iter())
        {
            debug!("checkpoint created [id={}]", result.id);
            let _ = checkpoint.sender.send(Ok(result));
        }
        let _ = self
            .tracker_tx
            .send(TrackerMessage::FlushComplete { through_seq });
        Ok(())
    }

    fn resolve_pending_flushes(&mut self) {
        let flush_result = self.flush_result();
        let pending = std::mem::take(&mut self.pending_flushes);
        let mut still_pending = Vec::with_capacity(pending.len());
        for flush in pending {
            if self.is_durable(flush.through_seq) {
                let _ = flush.sender.send(Ok(flush_result.clone()));
            } else {
                still_pending.push(flush);
            }
        }
        self.pending_flushes = still_pending;
    }

    async fn fail_ready_batch(
        &mut self,
        staged_batch: Vec<UploadedMemtable>,
        attached_checkpoints: Vec<PendingCheckpoint>,
        err: SlateDBError,
    ) -> Result<(), SlateDBError> {
        for uploaded in staged_batch {
            uploaded
                .imm_memtable
                .table()
                .notify_durable(Err(err.clone()));
        }
        for checkpoint in attached_checkpoints {
            let _ = checkpoint.sender.send(Err(err.clone()));
        }
        Ok(())
    }

    /// Fail remaining flush waiters on exit. Any waiter still pending at
    /// shutdown was never satisfied by a durable epoch advance, so it
    /// always receives an error.
    fn fail_pending_flushes(&mut self, err: &SlateDBError) {
        for flush in self.pending_flushes.drain(..) {
            let _ = flush.sender.send(Err(err.clone()));
        }
    }

    /// Fail remaining checkpoint waiters on exit. Any waiter still pending
    /// at shutdown was never satisfied, so it always receives an error.
    fn fail_pending_checkpoints(&mut self, err: &SlateDBError) {
        for checkpoint in self.pending_checkpoints.drain(..) {
            let _ = checkpoint.sender.send(Err(err.clone()));
        }
    }

    /// Extract flush/checkpoint waiters from a command without processing
    /// uploads. Used during error shutdown to ensure waiters get a proper error.
    fn collect_pending_waiter(&mut self, command: ManifestWriterCommand) {
        match command {
            ManifestWriterCommand::AwaitFlush {
                through_seq,
                sender,
            } => {
                self.pending_flushes.push(PendingFlush {
                    through_seq,
                    sender,
                });
            }
            ManifestWriterCommand::CreateCheckpoint {
                through_seq,
                options,
                sender,
            } => {
                self.pending_checkpoints.push(PendingCheckpoint {
                    through_seq,
                    options,
                    sender,
                });
            }
            _ => {}
        }
    }

    async fn try_graceful_cleanup(
        &mut self,
        commands: &mut (impl futures::Stream<Item = ManifestWriterCommand> + Unpin),
        result: &Result<(), SlateDBError>,
    ) -> Result<(), SlateDBError> {
        if result.is_ok() {
            while let Some(message) = commands.next().await {
                self.handle(message).await?;
            }
        }

        // Persist the local manifest on shutdown to advance next_wal_sst_id
        // and any other locally updated fields. Skip if fenced since another
        // writer owns the manifest.
        if !matches!(result, Err(SlateDBError::Fenced)) {
            self.write_current_manifest_safely().await?;
        }

        Ok(())
    }
}

struct PendingFlush {
    through_seq: Option<u64>,
    sender: oneshot::Sender<Result<FlushResult, SlateDBError>>,
}

struct PendingCheckpoint {
    through_seq: Option<u64>,
    options: CheckpointOptions,
    sender: oneshot::Sender<Result<CheckpointCreateResult, SlateDBError>>,
}

#[cfg(test)]
mod tests {
    use super::{ManifestWriter, TrackerMessage};
    use crate::config::{CheckpointOptions, Settings};
    use crate::db::DbInner;
    use crate::db_state::{ManifestCore, SsTableId};
    use crate::db_status::{ClosedResultWriter, DbStatusManager};
    use crate::error::SlateDBError;
    use crate::format::sst::SsTableFormat;
    use crate::manifest::store::{FenceableManifest, ManifestStore, StoredManifest};
    use crate::memtable_flusher::uploader::UploadedMemtable;
    use crate::object_stores::ObjectStores;
    use crate::paths::PathResolver;
    use crate::rand::DbRand;
    use crate::tablestore::TableStore;
    use crate::types::RowEntry;
    use crate::utils::{IdGenerator, WatchableOnceCell};
    use fail_parallel::FailPointRegistry;
    use object_store::memory::InMemory;
    use object_store::path::Path;
    use object_store::ObjectStore;
    use slatedb_common::clock::DefaultSystemClock;
    use slatedb_common::clock::SystemClock;
    use slatedb_common::metrics::MetricsRecorderHelper;
    use std::sync::Arc;
    use std::time::Duration;
    use tokio::runtime::Handle;
    use tokio::sync::oneshot;
    use tokio::time::timeout;

    struct StartedManifestWriter {
        writer: ManifestWriter,
        executor: crate::dispatcher::MessageHandlerExecutor,
        tracker_rx: async_channel::Receiver<TrackerMessage>,
        closed_result: WatchableOnceCell<Result<(), SlateDBError>>,
    }

    impl StartedManifestWriter {
        async fn shutdown(&self) {
            ManifestWriter::shutdown(&self.executor).await;
        }

        /// Wait for the executor to report a closed result (error or clean shutdown).
        async fn await_closed(&self) -> Result<(), SlateDBError> {
            self.closed_result.reader().await_value().await
        }
    }

    impl std::ops::Deref for StartedManifestWriter {
        type Target = ManifestWriter;
        fn deref(&self) -> &Self::Target {
            &self.writer
        }
    }

    fn start_manifest_writer(
        inner: Arc<DbInner>,
        manifest: FenceableManifest,
        poll_interval: Duration,
    ) -> StartedManifestWriter {
        let closed_result = WatchableOnceCell::new();
        let system_clock: Arc<dyn SystemClock> = Arc::new(DefaultSystemClock::new());
        let executor = crate::dispatcher::MessageHandlerExecutor::new(
            Arc::new(closed_result.clone()),
            system_clock,
        );
        let (tracker_tx, tracker_rx) =
            crate::utils::SafeSender::unbounded_channel(closed_result.result_reader());
        let writer = ManifestWriter::start(
            inner,
            manifest,
            poll_interval,
            &closed_result,
            &executor,
            &Handle::current(),
            tracker_tx,
        )
        .unwrap();
        executor.monitor_on(&Handle::current()).unwrap();
        StartedManifestWriter {
            writer,
            executor,
            tracker_rx,
            closed_result,
        }
    }

    async fn assert_no_flush_event(
        tracker_rx: &async_channel::Receiver<TrackerMessage>,
        duration: Duration,
    ) {
        let deadline = tokio::time::Instant::now() + duration;
        loop {
            match timeout(deadline - tokio::time::Instant::now(), tracker_rx.recv()).await {
                Err(_) => return, // timed out — no flush event, as expected
                Ok(Ok(TrackerMessage::ManifestRefreshed)) => continue,
                Ok(Ok(TrackerMessage::FlushComplete { .. })) => {
                    panic!("unexpected flushed event")
                }
                Ok(Err(_)) => panic!("tracker channel closed"),
                Ok(Ok(_)) => continue,
            }
        }
    }

    async fn expect_flushed(tracker_rx: &async_channel::Receiver<TrackerMessage>) -> u64 {
        loop {
            let msg = timeout(Duration::from_secs(5), tracker_rx.recv())
                .await
                .expect("timed out waiting for flushed event")
                .expect("tracker channel closed");
            match msg {
                TrackerMessage::FlushComplete { through_seq } => return through_seq,
                _ => continue,
            }
        }
    }

    struct TestHarness {
        inner: Arc<DbInner>,
        manifest: FenceableManifest,
        object_store: Arc<dyn ObjectStore>,
        path: String,
    }

    async fn setup_harness(path: &str, fp_registry: Arc<FailPointRegistry>) -> TestHarness {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        let path = path.to_string();
        let settings = Settings::default();
        let system_clock: Arc<dyn SystemClock> = Arc::new(DefaultSystemClock::new());
        let rand = Arc::new(DbRand::new(42));
        let db_metrics = MetricsRecorderHelper::noop();
        let manifest_store = Arc::new(ManifestStore::new(
            &Path::from(path.clone()),
            Arc::clone(&object_store),
        ));
        let stored_manifest = StoredManifest::create_new_db(
            Arc::clone(&manifest_store),
            ManifestCore::new_with_wal_object_store(None),
            Arc::clone(&system_clock),
        )
        .await
        .unwrap();
        let manifest_dirty = stored_manifest.prepare_dirty().unwrap();
        let table_store = Arc::new(TableStore::new_with_fp_registry(
            ObjectStores::new(Arc::clone(&object_store), None),
            SsTableFormat::default(),
            PathResolver::new(Path::from(path.clone())),
            Arc::clone(&fp_registry),
            None,
        ));
        let status_manager = DbStatusManager::new(0);
        let (write_tx, _) =
            crate::utils::SafeSender::unbounded_channel(status_manager.result_reader());
        let inner = Arc::new(
            DbInner::new(
                settings.clone(),
                Arc::clone(&system_clock),
                Arc::clone(&rand),
                Arc::clone(&table_store),
                manifest_dirty,
                Arc::new(crate::memtable_flusher::MemtableFlusher::new(
                    &WatchableOnceCell::new(),
                )),
                write_tx,
                db_metrics,
                fp_registry,
                None,
                status_manager,
            )
            .await
            .unwrap(),
        );
        let manifest_store = Arc::new(ManifestStore::new(
            &Path::from(path.clone()),
            Arc::clone(&object_store),
        ));
        let stored_manifest =
            StoredManifest::load(manifest_store, Arc::new(DefaultSystemClock::new()))
                .await
                .unwrap();
        let manifest = FenceableManifest::init_writer(
            stored_manifest,
            Duration::from_secs(300),
            Arc::new(DefaultSystemClock::new()),
        )
        .await
        .unwrap();

        TestHarness {
            inner,
            manifest,
            object_store,
            path,
        }
    }

    async fn load_writer_manifest(
        path: &str,
        object_store: Arc<dyn ObjectStore>,
    ) -> FenceableManifest {
        let manifest_store = Arc::new(ManifestStore::new(&Path::from(path), object_store));
        let stored_manifest =
            StoredManifest::load(manifest_store, Arc::new(DefaultSystemClock::new()))
                .await
                .unwrap();
        FenceableManifest::init_writer(
            stored_manifest,
            Duration::from_secs(300),
            Arc::new(DefaultSystemClock::new()),
        )
        .await
        .unwrap()
    }

    async fn latest_manifest_checkpoint_count(
        path: &str,
        object_store: Arc<dyn ObjectStore>,
    ) -> usize {
        let manifest_store = ManifestStore::new(&Path::from(path), object_store);
        let (_, manifest) = manifest_store.read_latest_manifest().await.unwrap();
        manifest.core.checkpoints.len()
    }

    fn freeze_imm(
        inner: &Arc<DbInner>,
        key: &[u8],
        value: &[u8],
    ) -> Arc<crate::mem_table::ImmutableMemtable> {
        let seq = inner.oracle.next_seq();
        let mut guard = inner.state.write();
        guard.memtable().put(RowEntry::new_value(key, value, seq));
        guard.freeze_memtable(0);
        guard.state().imm_memtable.front().cloned().unwrap()
    }

    async fn next_uploaded_memtable(
        inner: &Arc<DbInner>,
        key: &[u8],
        value: &[u8],
    ) -> UploadedMemtable {
        let imm_memtable = freeze_imm(inner, key, value);
        let sst_id = SsTableId::Compacted(inner.rand.rng().gen_ulid(inner.system_clock.as_ref()));
        let sst_handle = inner
            .flush_imm_table(&sst_id, imm_memtable.table(), true)
            .await
            .unwrap();
        let first_seq = imm_memtable.table().first_seq().unwrap();
        let last_seq = imm_memtable.table().last_seq().unwrap();
        UploadedMemtable::new(imm_memtable, sst_handle, first_seq, last_seq)
    }

    #[tokio::test]
    async fn should_emit_flushed_event_for_contiguous_uploads() {
        let harness = setup_harness(
            "/tmp/test_parallel_l0_flush_manifest_writer_flush_event",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let inner = Arc::clone(&harness.inner);
        let started = start_manifest_writer(
            Arc::clone(&inner),
            harness.manifest,
            Duration::from_secs(3600),
        );
        let uploaded = next_uploaded_memtable(&inner, b"k1", b"v1").await;
        started.notify_uploaded(uploaded).await.unwrap();

        let through_seq = expect_flushed(&started.tracker_rx).await;
        assert_eq!(through_seq, 1);

        started.shutdown().await;
    }

    #[tokio::test]
    async fn should_wait_for_missing_seq_before_flushing() {
        let harness = setup_harness(
            "/tmp/test_parallel_l0_flush_manifest_writer_gap",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let inner = Arc::clone(&harness.inner);
        let started = start_manifest_writer(
            Arc::clone(&inner),
            harness.manifest,
            Duration::from_secs(3600),
        );
        let uploaded1 = next_uploaded_memtable(&inner, b"k1", b"v1").await;
        let uploaded2 = next_uploaded_memtable(&inner, b"k2", b"v2").await;
        started.notify_uploaded(uploaded2).await.unwrap();
        assert_no_flush_event(&started.tracker_rx, Duration::from_millis(100)).await;

        started.notify_uploaded(uploaded1).await.unwrap();
        let through_seq = expect_flushed(&started.tracker_rx).await;
        assert_eq!(through_seq, 2);

        started.shutdown().await;
    }

    #[tokio::test]
    async fn should_create_checkpoint_immediately_when_no_barrier_is_required() {
        let harness = setup_harness(
            "/tmp/test_parallel_l0_flush_manifest_writer_checkpoint_immediate",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let before =
            latest_manifest_checkpoint_count(&harness.path, Arc::clone(&harness.object_store))
                .await;

        let inner = Arc::clone(&harness.inner);
        let started = start_manifest_writer(
            Arc::clone(&inner),
            harness.manifest,
            Duration::from_secs(3600),
        );

        let (tx, rx) = tokio::sync::oneshot::channel();
        started
            .send_checkpoint(None, CheckpointOptions::default(), tx)
            .unwrap();
        let checkpoint = timeout(Duration::from_secs(5), rx)
            .await
            .unwrap()
            .unwrap()
            .unwrap();

        let after =
            latest_manifest_checkpoint_count(&harness.path, Arc::clone(&harness.object_store))
                .await;
        assert_eq!(after, before + 1);
        assert!(checkpoint.manifest_id > 0);

        started.shutdown().await;
    }

    #[tokio::test]
    async fn should_wait_for_checkpoint_barrier_and_attach_to_flush_batch() {
        let harness = setup_harness(
            "/tmp/test_parallel_l0_flush_manifest_writer_checkpoint_barrier",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let before =
            latest_manifest_checkpoint_count(&harness.path, Arc::clone(&harness.object_store))
                .await;

        let inner = Arc::clone(&harness.inner);
        let started = start_manifest_writer(
            Arc::clone(&inner),
            harness.manifest,
            Duration::from_secs(3600),
        );
        let uploaded = next_uploaded_memtable(&inner, b"k1", b"v1").await;

        let (tx, rx) = oneshot::channel();
        started
            .send_checkpoint(Some(1), CheckpointOptions::default(), tx)
            .unwrap();

        tokio::task::yield_now().await;
        assert_no_flush_event(&started.tracker_rx, Duration::from_millis(100)).await;

        started.notify_uploaded(uploaded).await.unwrap();

        let through_seq = expect_flushed(&started.tracker_rx).await;
        assert_eq!(through_seq, 1);

        let checkpoint = rx.await.unwrap().unwrap();
        let after =
            latest_manifest_checkpoint_count(&harness.path, Arc::clone(&harness.object_store))
                .await;
        assert_eq!(after, before + 1);
        assert!(checkpoint.manifest_id > 0);

        started.shutdown().await;
    }

    #[tokio::test]
    async fn should_emit_fatal_event_when_manifest_writer_is_fenced() {
        let harness = setup_harness(
            "/tmp/test_parallel_l0_flush_manifest_writer_fenced",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let path = harness.path.clone();
        let object_store = Arc::clone(&harness.object_store);
        let inner = Arc::clone(&harness.inner);
        let started = start_manifest_writer(
            Arc::clone(&inner),
            harness.manifest,
            Duration::from_secs(3600),
        );
        let uploaded = next_uploaded_memtable(&inner, b"k1", b"v1").await;

        let _fence = load_writer_manifest(&path, object_store).await;
        started.notify_uploaded(uploaded).await.unwrap();

        // The manifest writer detects the fence and writes the error to closed_result.
        let result = timeout(Duration::from_secs(5), started.await_closed())
            .await
            .expect("timed out waiting for fenced error");
        assert!(
            matches!(result, Err(SlateDBError::Fenced)),
            "expected Fenced, got {:?}",
            result
        );

        started.shutdown().await;
    }

    #[tokio::test]
    async fn pending_flush_waiter_receives_error_on_fenced_shutdown() {
        let harness = setup_harness(
            "/tmp/test_parallel_l0_flush_manifest_writer_pending_flush_fenced",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let path = harness.path.clone();
        let object_store = Arc::clone(&harness.object_store);
        let inner = Arc::clone(&harness.inner);
        let started = start_manifest_writer(
            Arc::clone(&inner),
            harness.manifest,
            Duration::from_secs(3600),
        );
        let uploaded = next_uploaded_memtable(&inner, b"k1", b"v1").await;

        // Send a flush request for epoch 1, which hasn't been uploaded yet.
        let (tx, rx) = oneshot::channel();
        started.send_flush(Some(1), tx).unwrap();

        // Fence the manifest so the next write fails.
        let _fence = load_writer_manifest(&path, object_store).await;

        // Trigger a manifest write by uploading — this discovers the fence.
        started.notify_uploaded(uploaded).await.unwrap();

        // The pending flush waiter should receive the fencing error.
        let result = timeout(Duration::from_secs(5), rx)
            .await
            .expect("timed out")
            .expect("channel dropped");
        assert!(
            matches!(result, Err(SlateDBError::Fenced)),
            "expected Fenced, got {:?}",
            result
        );

        started.shutdown().await;
    }

    #[tokio::test]
    async fn pending_checkpoint_waiter_receives_error_on_fenced_shutdown() {
        let harness = setup_harness(
            "/tmp/test_parallel_l0_flush_manifest_writer_pending_checkpoint_fenced",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let path = harness.path.clone();
        let object_store = Arc::clone(&harness.object_store);
        let inner = Arc::clone(&harness.inner);
        let started = start_manifest_writer(
            Arc::clone(&inner),
            harness.manifest,
            Duration::from_secs(3600),
        );
        let uploaded = next_uploaded_memtable(&inner, b"k1", b"v1").await;

        // Send a checkpoint request for epoch 1, which hasn't been uploaded yet.
        let (tx, rx) = oneshot::channel();
        started
            .send_checkpoint(Some(1), CheckpointOptions::default(), tx)
            .unwrap();

        // Fence and trigger a manifest write.
        let _fence = load_writer_manifest(&path, object_store).await;
        started.notify_uploaded(uploaded).await.unwrap();

        // The pending checkpoint waiter should receive the fencing error.
        let result = timeout(Duration::from_secs(5), rx)
            .await
            .expect("timed out")
            .expect("channel dropped");
        assert!(
            matches!(result, Err(SlateDBError::Fenced)),
            "expected Fenced, got {:?}",
            result
        );

        started.shutdown().await;
    }

    #[tokio::test]
    async fn flush_waiter_in_channel_receives_error_on_clean_shutdown() {
        let harness = setup_harness(
            "/tmp/test_parallel_l0_flush_manifest_writer_channel_flush_clean",
            Arc::new(FailPointRegistry::new()),
        )
        .await;

        let inner = Arc::clone(&harness.inner);
        let started = start_manifest_writer(
            Arc::clone(&inner),
            harness.manifest,
            Duration::from_secs(3600),
        );

        // Send a flush request for an epoch that will never be uploaded.
        let (tx, rx) = oneshot::channel();
        started.send_flush(Some(1), tx).unwrap();

        // Shut down cleanly — the flush waiter should get Closed.
        started.shutdown().await;

        let result = timeout(Duration::from_secs(5), rx)
            .await
            .expect("timed out")
            .expect("channel dropped");
        assert!(
            matches!(result, Err(SlateDBError::Closed)),
            "expected Closed, got {:?}",
            result
        );
    }
}