slatedb 0.9.2

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
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
//! SlateDB makes use of [mpsc::channel] to offload work to background tasks when
//! possible. Examples of these tasks include [crate::mem_table_flush],
//! [crate::checkpoint], and [crate::compactor]. Each task's lifecycle is fairly
//! similar; they:
//!
//! 1. Receive messages and perform some work
//! 2. Perform work based on a fixed schedule
//! 3. Manage error states when failures occur
//! 4. Drain messages on shutdown
//! 5. Clean up resources on shutdown
//!
//! [MessageDispatcher], [MessageHandlerExecutor], and [MessageHandler] standardize
//! this pattern in a single set of event loops.
//!
//! - [MessageHandlerExecutor] spawns background tasks, monitors them for completion, and
//!   updates [crate::db_state::DbState::closed_result] accordingly.
//! - [MessageDispatcher] runs an event loop for a single background task. It receives
//!   messages and ticks, and passes them to the [MessageHandler] for processing.
//! - [MessageHandler] receives callbacks from the dispatcher based on lifetime events
//!   (messages, ticks, etc.) and performs work based on those events.
//!
//! SlateDB instantiates a [MessageHandlerExecutor] and calls
//! [MessageHandlerExecutor::add_handler] for each [MessageHandler]. The DB then calls
//! [MessageHandlerExecutor::monitor_on] to start the event loop and monitor tasks until
//! shutdown.
//!
//! ## Example
//!
//! ```ignore
//! # use slatedb::dispatcher::{MessageHandlerExecutor, MessageHandler, MessageFactory};
//! # use slatedb::error::SlateDBError;
//! # use slatedb::clock::DefaultSystemClock;
//! # use slatedb::utils::WatchableOnceCell;
//! # use std::sync::Arc;
//! # use std::time::Duration;
//! # use tokio::sync::mpsc;
//! # use tokio::runtime::Handle;
//! # use futures::stream::BoxStream;
//! # use futures::StreamExt;
//! # #[tokio::main]
//! # async fn main() {
//! enum Message {
//!     Say(String),
//! }
//!
//! struct PrintMessageHandler;
//!
//! #[async_trait::async_trait]
//! impl MessageHandler<Message> for PrintMessageHandler {
//!     fn tickers(&mut self) -> Vec<(Duration, Box<MessageFactory<Message>>)> {
//!         let mut tickers = Vec::new();
//!         tickers.push((Duration::from_secs(1), Box::new(|| Message::Say("tick".to_string()))));
//!         tickers
//!     }
//!
//!     async fn handle(
//!         &mut self,
//!         message: Message,
//!     ) -> Result<(), SlateDBError> {
//!         match message {
//!             Message::Say(msg) => println!("{}", msg),
//!         }
//!         Ok(())
//!     }
//!
//!     async fn cleanup(
//!         &mut self,
//!         mut messages: BoxStream<'async_trait, Message>,
//!         result: Result<(), SlateDBError>,
//!     ) -> Result<(), SlateDBError> {
//!         match result {
//!             Ok(_) | Err(SlateDBError::Closed) => {
//!                 while let Some(m) = messages.next().await {
//!                     let _ = self.handle(m).await;
//!                 }
//!             },
//!             // skipping drain messages on unclean shutdown
//!             _ => {},
//!         }
//!         Ok(())
//!     }
//! }
//!
//! let clock = Arc::new(DefaultSystemClock::default());
//! let (tx, rx) = mpsc::unbounded_channel();
//! let closed_result = WatchableOnceCell::new();
//! let handle = Handle::current();
//! let task_executor = MessageHandlerExecutor::new(
//!     closed_result,
//!     clock,
//! );
//! task_executor.add_handler(
//!     "print_message_handler".to_string(),
//!     Box::new(PrintMessageHandler),
//!     rx,
//!     &handle,
//! ).expect("failed to add handler");
//! let join_handle = task_executor.monitor_on(&handle)
//!     .expect("failed to start monitoring");
//! tx.send(Message::Say("hello".to_string())).unwrap();
//! task_executor.shutdown_task("print_message_handler").await.ok();
//! join_handle.await.ok();
//! # }
//! ```

use std::{future::Future, panic::AssertUnwindSafe, pin::Pin, sync::Arc, time::Duration};

use async_trait::async_trait;
use crossbeam_skiplist::SkipMap;
use fail_parallel::{fail_point, FailPointRegistry};
use futures::{
    future::BoxFuture,
    stream::{BoxStream, FuturesUnordered},
    FutureExt, StreamExt,
};
use log::error;
use parking_lot::Mutex;
use tokio::{runtime::Handle, sync::mpsc, task::JoinHandle};
use tokio_util::{sync::CancellationToken, task::JoinMap};

use crate::{
    clock::{SystemClock, SystemClockTicker},
    error::SlateDBError,
    utils::{panic_string, split_join_result, split_unwind_result, WatchableOnceCell},
};

/// A factory for creating messages when a [MessageDispatcherTicker] ticks.
pub(crate) type MessageFactory<T> = dyn Fn() -> T + Send;

/// A dispatcher that invokes [MessageHandler] callbacks when events occur.
///
/// [MessageDispatcher] receives a [MessageHandler] and [mpsc::UnboundedReceiver<T>].
/// Messages sent to the receiver are passed to the [MessageHandler] for processing.
///
/// [MessageDispatcher::run] is the primary entry point for running the dispatcher;
/// it is responsible for running the main event loop. The function receives messages
/// from the [mpsc::UnboundedReceiver<T>] and from any [MessageDispatcherTicker]s, and
/// passes them to the [MessageHandler] for processing.
struct MessageDispatcher<T: Send + std::fmt::Debug> {
    handler: Box<dyn MessageHandler<T>>,
    rx: mpsc::UnboundedReceiver<T>,
    clock: Arc<dyn SystemClock>,
    cancellation_token: CancellationToken,
    #[allow(dead_code)]
    fp_registry: Arc<FailPointRegistry>,
}

impl<T: Send + std::fmt::Debug> MessageDispatcher<T> {
    /// Creates a new [MessageDispatcher]. Messages sent to the channel are passed to
    /// the [MessageHandler] for processing.
    ///
    /// ## Arguments
    ///
    /// * `handler`: The [MessageHandler] to use for processing messages.
    /// * `rx`: The [mpsc::UnboundedReceiver<T>] to use for receiving messages.
    /// * `clock`: The [SystemClock] to use for time.
    /// * `cancellation_token`: The [CancellationToken] to use for shutdown.
    #[allow(dead_code)]
    fn new(
        handler: Box<dyn MessageHandler<T>>,
        rx: mpsc::UnboundedReceiver<T>,
        clock: Arc<dyn SystemClock>,
        cancellation_token: CancellationToken,
    ) -> Self {
        Self {
            handler,
            rx,
            clock,
            cancellation_token,
            fp_registry: Arc::new(FailPointRegistry::new()),
        }
    }

    /// Creates a new [MessageDispatcher]. Messages sent to the channel are passed to
    /// the [MessageHandler] for processing. A fail point registry is provided for
    /// fail points.
    ///
    /// ## Arguments
    ///
    /// * `handler`: The [MessageHandler] to use for processing messages.
    /// * `rx`: The [mpsc::UnboundedReceiver<T>] to use for receiving messages.
    /// * `clock`: The [SystemClock] to use for time.
    /// * `cancellation_token`: The [CancellationToken] to use for shutdown.
    /// * `fp_registry`: The [FailPointRegistry] to use for fail points.
    #[allow(dead_code)]
    fn with_fp_registry(mut self, fp_registry: Arc<FailPointRegistry>) -> Self {
        self.fp_registry = fp_registry;
        self
    }

    /// Runs the main event loop for the dispatcher. This is where messages and ticker
    /// events are processed.
    ///
    /// [MessageDispatcher::run] contains a message loop with the following control
    /// flow:
    ///
    /// 1. Break immediately if the task is cancelled and returns `Ok(())`.
    /// 2. Else, if there is a message, read it and invoke [MessageHandler::handle].
    /// 3. Else, if there is a ticker event, read it and invoke [MessageHandler::handle].
    ///
    /// ## Returns
    ///
    /// A [Result] containing `Ok(())` on clean shutdown, or an error if the handler
    /// fails for any reason.
    async fn run(&mut self) -> Result<(), SlateDBError> {
        let mut tickers = self
            .handler
            .tickers()
            .into_iter()
            .map(|(dur, factory)| MessageDispatcherTicker::new(self.clock.ticker(dur), factory))
            .collect::<Vec<_>>();
        let mut ticker_futures: FuturesUnordered<_> =
            tickers.iter_mut().map(|t| t.tick()).collect();
        loop {
            fail_point!(Arc::clone(&self.fp_registry), "dispatcher-run-loop", |_| {
                Err(SlateDBError::Fenced)
            });
            tokio::select! {
                biased;
                // stop the loop if we're in an error state or cancelled
                _ = self.cancellation_token.cancelled() => {
                    break;
                }
                // if no errors, prioritize messages
                Some(message) = self.rx.recv() => {
                    self.handler.handle(message).await?;
                },
                // if no messages, check tickers
                Some((message, ticker)) = ticker_futures.next() => {
                    self.handler.handle(message).await?;
                    ticker_futures.push(ticker.tick());
                },
            }
        }
        Ok(())
    }

    /// Tells the handler to clean up any resources.
    ///
    /// If cleanup fails, the error is returned.
    ///
    /// ## Arguments
    ///
    /// * `result`: The value of [crate::db_state::DbState::closed_result] when
    ///   [MessageDispatcher::run] returns.
    ///
    /// ## Returns
    ///
    /// The [Result] after cleaning up resources.
    async fn cleanup(&mut self, result: Result<(), SlateDBError>) -> Result<(), SlateDBError> {
        fail_point!(Arc::clone(&self.fp_registry), "dispatcher-cleanup", |_| {
            Err(SlateDBError::Fenced)
        });
        self.rx.close();
        let messages = futures::stream::unfold(&mut self.rx, |rx| async move {
            rx.recv().await.map(|message| (message, rx))
        });
        self.handler.cleanup(Box::pin(messages), result).await
    }
}

/// A ticker that generates messages at regular intervals.
///
/// On each [MessageDispatcherTicker::tick], the message factory is called to generate a
/// message, and the message is returned as a [Future].
struct MessageDispatcherTicker<'a, T: Send> {
    inner: SystemClockTicker<'a>,
    message_factory: Box<MessageFactory<T>>,
}

impl<'a, T: Send> MessageDispatcherTicker<'a, T> {
    /// Creates a new [MessageDispatcherTicker].
    ///
    /// ## Arguments
    ///
    /// * `inner`: The [SystemClockTicker] to use.
    /// * `message_factory`: A factory for generating messages.
    ///
    /// ## Returns
    ///
    /// The new [MessageDispatcherTicker].
    fn new(inner: SystemClockTicker<'a>, message_factory: Box<MessageFactory<T>>) -> Self {
        Self {
            inner,
            message_factory,
        }
    }

    /// Returns a [Future] that resolves when the ticker ticks, and returns the message
    /// generated by the message factory.
    ///
    /// ## Returns
    ///
    /// A [Future] that resolves when the ticker ticks.
    fn tick(&mut self) -> Pin<Box<dyn Future<Output = (T, &mut Self)> + Send + '_>> {
        let message = (self.message_factory)();
        Box::pin(async move {
            self.inner.tick().await;
            (message, self)
        })
    }
}

/// [MessageDispatcher] event loop callbacks are implemented in a [MessageHandler].
/// Handlers are responsible for:
///
/// 1. Processing messages ([MessageHandler::handle])
/// 2. Defining ticker schedules (when to tick a certain message) ([MessageHandler::tickers])
/// 3. Cleaning up resources ([MessageHandler::cleanup])
///
/// It is safe to return errors on failure or panic; the [MessageDispatcher] and
/// [MessageHandlerExecutor] will handle them appropriately.
#[async_trait]
pub(crate) trait MessageHandler<T: Send>: Send {
    /// Defines message ticker schedules. [MessageDispatcher::run] instantiates a
    /// [MessageDispatcherTicker] for each ticker defined here. Whenever each ticker
    /// ticks, the message factory generates a message, and [MessageDispatcher] sends the
    /// message to this [MessageHandler].
    ///
    /// ## Returns
    ///
    /// A vector of tuples containing the duration when a message should be sent to the
    /// [MessageDispatcher], and a message factory to generate a new message on each tick.
    fn tickers(&mut self) -> Vec<(Duration, Box<MessageFactory<T>>)> {
        vec![]
    }

    /// Handles a message. Messages can come from either a channel or a ticker. See
    /// [crate::dispatcher] for details.
    ///
    /// ## Arguments
    ///
    /// * `message`: The message to handle.
    /// * `error`: An optional error to pass to the handler. If set, this argument
    ///   signals to the [MessageHandler] that the database is in an error state during
    ///   shutdown.
    ///
    /// ## Returns
    ///
    /// The [Result] after handling the message.
    async fn handle(&mut self, message: T) -> Result<(), SlateDBError>;

    /// Cleans up resources. This method should not be used to continue processing
    /// messages or writing to the database.
    ///
    /// ## Arguments
    ///
    /// * `messages`: An iterator of messages still in the channel after
    ///   [MessageDispatcher::run] returns. If a handler fails, this iterator will not
    ///   include the message that triggered the failure.
    /// * `result`: The value of [crate::db_state::DbState::closed_result] when
    ///   [MessageDispatcher::run] returns.
    ///
    /// ## Returns
    ///
    /// The [Result] after cleaning up resources.
    async fn cleanup(
        &mut self,
        messages: BoxStream<'async_trait, T>,
        result: Result<(), SlateDBError>,
    ) -> Result<(), SlateDBError>;
}

/// A builder data structure for [MessageHandlerExecutor]. The executor creates a
/// [MessageHandlerFuture] for each handler it receives in
/// [MessageHandlerExecutor::add_handler]. The future is used to run the handler's
/// dispatcher ([MessageDispatcher::run]) and handle cleanup
/// ([MessageDispatcher::cleanup]).
struct MessageHandlerFuture {
    /// The name of the task.
    name: String,
    /// A future that runs the handler's dispatcher and handles cleanup.
    future: BoxFuture<'static, Result<(), SlateDBError>>,
    /// A cancellation token used to cancel the handler's dispatcher.
    token: CancellationToken,
    /// A runtime handle used to spawn the future.
    handle: Handle,
}

impl std::fmt::Debug for MessageHandlerFuture {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("MessageHandlerFuture")
            .field("name", &self.name)
            .finish()
    }
}

/// The [MessageHandlerExecutor] is responsible for spawning, monitoring, and shutting
/// down [MessageDispatcher]s. Think of it as a dispatcher pool.
///
/// Each dispatcher is associated with a name, which is used to identify the dispatcher
/// in the [MessageHandlerExecutor]. As dispatchers complete, their results are stored
/// in the [MessageHandlerExecutor]. The first completed dispatcher's results are also
/// used for [crate::db_state::DbState::closed_result].
///
/// The executor also catches panics, and will convert them to an appropriate
/// [SlateDBError].
#[derive(Debug)]
pub(crate) struct MessageHandlerExecutor {
    /// A vector of futures for each dispatcher. Set to `None` when executor starts.
    futures: Mutex<Option<Vec<MessageHandlerFuture>>>,
    /// A map of cancellation tokens for each dispatcher.
    tokens: SkipMap<String, CancellationToken>,
    /// A map of (task name, results) for dispatchers that have returned.
    results: Arc<SkipMap<String, WatchableOnceCell<Result<(), SlateDBError>>>>,
    /// A watchable cell that stores the final result of the database lifecycle.
    /// Ok(()) indicates a clean shutdown; Err(e) indicates an error.
    closed_result: WatchableOnceCell<Result<(), SlateDBError>>,
    /// A system clock for time keeping.
    clock: Arc<dyn SystemClock>,
    /// A fail point registry for fail points.
    #[allow(dead_code)]
    fp_registry: Arc<FailPointRegistry>,
}

impl MessageHandlerExecutor {
    /// Creates a new [MessageHandlerExecutor].
    ///
    /// ## Arguments
    ///
    /// * `closed_result`: A [WatchableOnceCell] that stores the database close result.
    /// * `clock`: A [SystemClock] to use for tickers and timekeeping.
    ///
    /// ## Returns
    ///
    /// The new [MessageHandlerExecutor].
    pub(crate) fn new(
        closed_result: WatchableOnceCell<Result<(), SlateDBError>>,
        clock: Arc<dyn SystemClock>,
    ) -> Self {
        Self {
            futures: Mutex::new(Some(vec![])),
            closed_result,
            clock,
            tokens: SkipMap::new(),
            results: Arc::new(SkipMap::new()),
            fp_registry: Arc::new(FailPointRegistry::new()),
        }
    }

    /// Sets a custom [FailPointRegistry] for the executor.
    ///
    /// ## Arguments
    ///
    /// * `fp_registry`: A [FailPointRegistry] to use for fail points.
    ///
    /// ## Returns
    ///
    /// The [MessageHandlerExecutor] with the custom [FailPointRegistry].
    #[allow(dead_code)]
    pub(crate) fn with_fp_registry(mut self, fp_registry: Arc<FailPointRegistry>) -> Self {
        self.fp_registry = fp_registry;
        self
    }

    /// Adds a new [MessageHandlerFuture] for the provided [MessageHandler] on the given
    /// [Handle]. The future calls [MessageDispatcher::run] on the dispatcher. When
    /// [MessageDispatcher::run] returns, [MessageDispatcher::cleanup] is called. The
    /// future is _not_ spawned in this method. Instead, it is added to the executor's
    /// `futures` list and will be spawned when [MessageHandlerExecutor::monitor_on] is
    /// called.
    ///
    /// ## Arguments
    ///
    /// * `name`: The name of the dispatcher.
    /// * `handler`: The [MessageHandler] to use for handling messages.
    /// * `rx`: The [mpsc::UnboundedReceiver] to use for receiving messages on the
    ///   handler's behalf.
    /// * `handle`: The [Handle] to use for spawning the dispatcher.
    ///
    /// ## Returns
    ///
    /// - `Ok(())` if the dispatcher was successfully spawned.
    /// - Err([SlateDBError::BackgroundTaskExists]) if a dispatcher with the same name is
    ///   already added.
    /// - Err([SlateDBError::BackgroundTaskExecutorStarted]) if the executor is already
    ///   started.
    #[allow(clippy::panic)] // for failpoint
    pub(crate) fn add_handler<T: Send + std::fmt::Debug + 'static>(
        &self,
        name: String,
        handler: Box<dyn MessageHandler<T>>,
        rx: mpsc::UnboundedReceiver<T>,
        handle: &Handle,
    ) -> Result<(), SlateDBError> {
        let token = CancellationToken::new();
        let mut dispatcher = MessageDispatcher::new(handler, rx, self.clock.clone(), token.clone())
            .with_fp_registry(self.fp_registry.clone());
        let this_closed_result = self.closed_result.clone();
        let this_name = name.clone();
        #[allow(unused_variables)]
        let this_fp_registry = self.fp_registry.clone();
        // future that runs the dispatcher and handles cleanup
        let task_future = async move {
            // catch dispatcher panics using catch_unwind
            let run_unwind_result = AssertUnwindSafe(dispatcher.run()).catch_unwind().await;
            let (run_result, run_maybe_panic) =
                split_unwind_result(this_name.clone(), run_unwind_result);
            if let Err(ref err) = run_result {
                error!(
                    "background task panicked unexpectedly. [task_name={}, error={:?}, panic={:?}]",
                    this_name,
                    err,
                    run_maybe_panic.map(|p| panic_string(&p))
                );
            }
            fail_point!(
                this_fp_registry.clone(),
                "executor-wrapper-before-write",
                |_| {
                    panic!("failpoint: executor-wrapper-before-write");
                }
            );
            // set result in db state (first writer wins)
            this_closed_result.write(run_result.clone());
            // re-read the result since it might have already been set by another task
            let final_result = this_closed_result
                .reader()
                .read()
                .expect("error state was unexpectedly empty");
            let cleanup_unwind_result = AssertUnwindSafe(dispatcher.cleanup(final_result))
                .catch_unwind()
                .await;
            let (cleanup_result, cleanup_maybe_panic) =
                split_unwind_result(this_name.clone(), cleanup_unwind_result);
            if let Err(err) = cleanup_result {
                error!(
                    "background task failed to clean up on shutdown [name={}, error={:?}, panic={:?}]",
                    this_name.clone(),
                    err,
                    cleanup_maybe_panic.map(|p| panic_string(&p))
                );
            }
            run_result
        };
        let mut guard = self.futures.lock();
        if let Some(task_definitions) = guard.as_mut() {
            if task_definitions.iter().any(|t| t.name == name) {
                return Err(SlateDBError::BackgroundTaskExists(name));
            }
            task_definitions.push(MessageHandlerFuture {
                name,
                future: Box::pin(task_future),
                token,
                handle: handle.clone(),
            });
            Ok(())
        } else {
            Err(SlateDBError::BackgroundTaskExecutorStarted)
        }
    }

    /// Starts all [MessageHandlerFuture]s and spawns a task that monitors for completed
    /// dispatchers. As dispatchers complete, their results are stored in the executor's
    /// `results` map.
    ///
    /// ## Arguments
    ///
    /// * `handle`: The [Handle] to spawn the monitor on.
    ///
    /// ## Returns
    ///
    /// - Ok([JoinHandle]) if the monitor was successfully spawned.
    /// - Err([SlateDBError::BackgroundTaskExecutorStarted]) if the executor is already
    ///   started.
    pub(crate) fn monitor_on(&self, handle: &Handle) -> Result<JoinHandle<()>, SlateDBError> {
        let mut task_definitions = {
            let mut guard = self.futures.lock();
            if let Some(task_definitions) = guard.take() {
                task_definitions
            } else {
                return Err(SlateDBError::BackgroundTaskExecutorStarted);
            }
        };
        let mut tasks = JoinMap::new();
        for task_definition in task_definitions.drain(..) {
            self.tokens
                .insert(task_definition.name.clone(), task_definition.token.clone());
            self.results
                .insert(task_definition.name.clone(), WatchableOnceCell::new());
            tasks.spawn_on(
                task_definition.name.clone(),
                task_definition.future,
                &task_definition.handle,
            );
        }
        let this_closed_result = self.closed_result.clone();
        let this_results = self.results.clone();
        let this_tokens = self
            .tokens
            .iter()
            .map(|e| e.value().clone())
            .collect::<Vec<_>>();
        // future that runs until all tasks are completed
        let monitor_future = async move {
            while !tasks.is_empty() {
                if let Some((name, join_result)) = tasks.join_next().await {
                    let (task_result, task_maybe_panic) =
                        split_join_result(name.clone(), join_result);
                    if let Err(ref err) = task_result {
                        error!(
                            "background task failed [name={}, error={:?}, panic={:?}]",
                            name,
                            err,
                            task_maybe_panic.map(|p| panic_string(&p))
                        );
                    }
                    this_closed_result.write(task_result.clone());
                    let entry = this_results
                        .get(&name)
                        .expect("result cell isn't set when expected");
                    let result_cell = entry.value();
                    result_cell.write(task_result.clone());
                    if task_result.is_err() {
                        // db is in an error state, so cancel all other tasks
                        this_tokens.iter().for_each(|t| t.cancel());
                    }
                }
            }
        };
        Ok(handle.spawn(monitor_future))
    }

    /// Cancels a task by name.
    ///
    /// ## Arguments
    ///
    /// * `name`: The name of the task to cancel.
    pub(crate) fn cancel_task(&self, name: &str) {
        if let Some(entry) = self.tokens.get(name) {
            entry.value().cancel();
        }
    }

    /// Waits for a task to complete.
    ///
    /// ## Arguments
    ///
    /// * `name`: The name of the task to wait for.
    ///
    /// ## Returns
    ///
    /// The result of the task.
    pub(crate) async fn join_task(&self, name: &str) -> Result<(), SlateDBError> {
        if let Some(entry) = self.results.get(name) {
            return entry.value().reader().await_value().await;
        }
        Ok(())
    }

    /// Cancels a task and waits for it to complete.
    ///
    /// ## Arguments
    ///
    /// * `name`: The name of the task to cancel and wait for.
    ///
    /// ## Returns
    ///
    /// The result of the task.
    pub(crate) async fn shutdown_task(&self, name: &str) -> Result<(), SlateDBError> {
        self.cancel_task(name);
        self.join_task(name).await
    }
}

#[cfg(all(test, feature = "test-util"))]
mod test {
    use super::{MessageDispatcher, MessageHandler};
    use crate::clock::{DefaultSystemClock, MockSystemClock, SystemClock};
    use crate::dispatcher::{MessageFactory, MessageHandlerExecutor};
    use crate::error::SlateDBError;
    use crate::utils::WatchableOnceCell;
    use fail_parallel::FailPointRegistry;
    use futures::stream::BoxStream;
    use futures::StreamExt;
    use std::collections::VecDeque;
    use std::sync::{Arc, Mutex};
    use std::time::Duration;
    use tokio::runtime::Handle;
    use tokio::sync::mpsc;
    use tokio::task::yield_now;
    use tokio::time::timeout;
    use tokio_util::sync::CancellationToken;

    #[derive(Debug, Clone, PartialEq, Eq)]
    enum TestMessage {
        Channel(i32),
        Tick(i32),
    }

    #[derive(Debug, Clone, Copy, PartialEq, Eq)]
    enum Phase {
        Pre,
        Cleanup,
    }

    #[derive(Clone)]
    struct TestHandler {
        log: Arc<Mutex<Vec<(Phase, TestMessage)>>>,
        cleanup_called: WatchableOnceCell<Result<(), SlateDBError>>,
        tickers: Vec<(Duration, u8)>,
        clock: Arc<dyn SystemClock>,
        clock_schedule: VecDeque<Duration>,
    }

    impl TestHandler {
        fn new(
            log: Arc<Mutex<Vec<(Phase, TestMessage)>>>,
            cleanup_called: WatchableOnceCell<Result<(), SlateDBError>>,
            clock: Arc<dyn SystemClock>,
        ) -> Self {
            Self {
                log,
                cleanup_called,
                tickers: vec![],
                clock,
                clock_schedule: VecDeque::new(),
            }
        }

        fn add_ticker(mut self, d: Duration, id: u8) -> Self {
            self.tickers.push((d, id));
            self
        }

        /// Add a clock schedule to the handler. The clock will pop the first duration from the
        /// schedule and advance the clock by that duration after each message is processed.
        fn add_clock_schedule(mut self, ts: u64) -> Self {
            self.clock_schedule.push_back(Duration::from_millis(ts));
            self
        }
    }

    #[async_trait::async_trait]
    impl MessageHandler<TestMessage> for TestHandler {
        fn tickers(&mut self) -> Vec<(Duration, Box<MessageFactory<TestMessage>>)> {
            let mut tickers: Vec<(Duration, Box<MessageFactory<_>>)> = vec![];
            for (interval, id) in self.tickers.iter() {
                let id = *id as i32;
                tickers.push((*interval, Box::new(move || TestMessage::Tick(id))));
            }
            tickers
        }

        async fn handle(&mut self, message: TestMessage) -> Result<(), SlateDBError> {
            self.log.lock().unwrap().push((Phase::Pre, message));
            if let Some(advance_duration) = self.clock_schedule.pop_front() {
                self.clock.advance(advance_duration).await;
            }
            Ok(())
        }

        async fn cleanup(
            &mut self,
            mut messages: futures::stream::BoxStream<'async_trait, TestMessage>,
            result: Result<(), SlateDBError>,
        ) -> Result<(), SlateDBError> {
            self.cleanup_called.write(result);
            while let Some(m) = messages.next().await {
                self.log.lock().unwrap().push((Phase::Cleanup, m));
            }
            Ok(())
        }
    }

    async fn wait_for_message_count(log: Arc<Mutex<Vec<(Phase, TestMessage)>>>, count: usize) {
        timeout(Duration::from_secs(30), async move {
            while log.lock().unwrap().len() < count {
                yield_now().await;
            }
        })
        .await
        .expect("timeout waiting for message count");
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_dispatcher_run_happy_path() {
        let log = Arc::new(Mutex::new(Vec::<(Phase, TestMessage)>::new()));
        let (tx, rx) = mpsc::unbounded_channel();
        let clock = Arc::new(MockSystemClock::new());
        let handler = TestHandler::new(log.clone(), WatchableOnceCell::new(), clock.clone())
            .add_ticker(Duration::from_millis(5), 1)
            .add_clock_schedule(5); // Advance clock by 5ms after first message
        let cancellation_token = CancellationToken::new();
        let mut dispatcher = MessageDispatcher::new(
            Box::new(handler),
            rx,
            clock.clone(),
            cancellation_token.clone(),
        );
        let join = tokio::spawn(async move { dispatcher.run().await });

        // Send a message successfully, then trigger a tick before processing the next message
        tx.send(TestMessage::Channel(10)).unwrap();
        wait_for_message_count(log.clone(), 2).await;
        tx.send(TestMessage::Channel(20)).unwrap();
        wait_for_message_count(log.clone(), 3).await;

        // Cancel and wait for cleanup to start
        cancellation_token.cancel();

        // Ensure run() completes and returns clean shutdown
        let result = timeout(Duration::from_secs(30), join)
            .await
            .expect("dispatcher did not stop in time")
            .expect("join failed");

        // Verify final state
        assert!(matches!(result, Ok(())));
        let messages = log.lock().unwrap().clone();
        assert_eq!(
            messages,
            vec![
                (Phase::Pre, TestMessage::Channel(10)),
                (Phase::Pre, TestMessage::Tick(1)),
                (Phase::Pre, TestMessage::Channel(20))
            ]
        );
    }

    #[cfg(feature = "test-util")]
    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_executor_propagates_handler_error_drains_messages() {
        let log = Arc::new(Mutex::new(Vec::<(Phase, TestMessage)>::new()));
        let cleanup_called = WatchableOnceCell::new();
        let mut cleanup_reader = cleanup_called.reader();
        let (tx, rx) = mpsc::unbounded_channel();
        let clock = Arc::new(MockSystemClock::new());
        let handler = TestHandler::new(log.clone(), cleanup_called.clone(), clock.clone());
        let closed_result = WatchableOnceCell::new();
        let fp_registry = Arc::new(FailPointRegistry::default());
        let task_executor = MessageHandlerExecutor::new(closed_result.clone(), clock.clone())
            .with_fp_registry(fp_registry.clone());
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "1*off->return").unwrap();
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-cleanup", "pause").unwrap();
        task_executor
            .add_handler(
                "test".to_string(),
                Box::new(handler),
                rx,
                &Handle::current(),
            )
            .expect("spawn failed");
        task_executor
            .monitor_on(&Handle::current())
            .expect("failed to monitor executor");

        // Send a message successfully.
        tx.send(TestMessage::Channel(42)).unwrap();
        // Wait for the first message to be processed.
        wait_for_message_count(log.clone(), 1).await;
        // Send another message. The panic is guaranteed to be triggered before this
        // message is processed since the `dispatcher-run-loop` failpoint is set to
        // `1*off->return`, one message has been processed, and the point is hit before
        // the message selector.
        tx.send(TestMessage::Channel(77)).unwrap();
        // Now allow cleanup to proceed.
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-cleanup", "off").unwrap();

        // Wait for cleanup to start (unclean shutdown)
        let _ = cleanup_reader.await_value().await;

        // Wait for the dispatcher to stop
        let result = timeout(Duration::from_secs(30), task_executor.join_task("test"))
            .await
            .expect("dispatcher did not stop in time");

        // Verify final state
        assert!(matches!(result, Err(SlateDBError::Fenced)));
        assert!(matches!(
            closed_result.reader().read(),
            Some(Err(SlateDBError::Fenced))
        ));
        assert!(matches!(
            cleanup_reader.read(),
            Some(Err(SlateDBError::Fenced))
        ));
        let messages = log.lock().unwrap().clone();
        assert_eq!(
            messages,
            vec![
                (Phase::Pre, TestMessage::Channel(42)),
                (Phase::Cleanup, TestMessage::Channel(77)),
            ]
        );
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_dispatcher_prioritizes_messages_over_tickers() {
        let log = Arc::new(Mutex::new(Vec::<(Phase, TestMessage)>::new()));
        let (tx, rx) = mpsc::unbounded_channel();
        let clock = Arc::new(MockSystemClock::new());
        let handler = TestHandler::new(log.clone(), WatchableOnceCell::new(), clock.clone())
            .add_ticker(Duration::from_millis(5), 1);
        let cancellation_token = CancellationToken::new();
        let fp_registry = Arc::new(FailPointRegistry::default());
        let mut dispatcher = MessageDispatcher::new(
            Box::new(handler),
            rx,
            clock.clone(),
            cancellation_token.clone(),
        )
        .with_fp_registry(fp_registry.clone());
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "pause").unwrap();
        let join = tokio::spawn(async move { dispatcher.run().await });

        // Trigger a tick and a message
        clock.advance(Duration::from_millis(5)).await;
        tx.send(TestMessage::Channel(99)).unwrap();
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "off").unwrap();
        wait_for_message_count(log.clone(), 2).await;

        // Shutdown cleanly
        cancellation_token.cancel();
        let result = timeout(Duration::from_secs(30), join)
            .await
            .expect("dispatcher did not stop in time")
            .expect("join failed");

        // Verify final state
        assert!(matches!(result, Ok(())));
        let messages = log.lock().unwrap().clone();
        assert_eq!(
            messages,
            vec![
                (Phase::Pre, TestMessage::Channel(99)),
                (Phase::Pre, TestMessage::Tick(1)),
            ]
        );
    }

    // This test simulates the following timeline:
    // immediate tick(3) (tickers always return first ticks immediately)
    // immediate tick(5)
    // advance clock to 5ms
    // tick(5)
    // advance clock to 7ms
    // tick(7)
    // advance clock to 10ms
    // tick(5)
    // advance clock to 14ms
    // tick(7)
    // advance clock to 15ms
    // tick(5)
    // advance clock to 20ms
    // tick(5)
    // advance clock to 21ms
    // tick(7)
    #[cfg(feature = "test-util")]
    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_dispatcher_supports_multiple_tickers() {
        let log = Arc::new(Mutex::new(Vec::<(Phase, TestMessage)>::new()));
        let (_tx, rx) = mpsc::unbounded_channel::<TestMessage>();
        let clock = Arc::new(MockSystemClock::new());
        let handler = TestHandler::new(log.clone(), WatchableOnceCell::new(), clock.clone())
            .add_ticker(Duration::from_millis(5), 1)
            .add_ticker(Duration::from_millis(7), 2)
            .add_clock_schedule(0) // 0ms (initial two ticks are immediate, so don't advance clock until second tick)
            .add_clock_schedule(5) // 5ms
            .add_clock_schedule(2) // 7ms
            .add_clock_schedule(3) // 10ms
            .add_clock_schedule(4) // 14ms
            .add_clock_schedule(1) // 15ms
            .add_clock_schedule(5) // 20ms
            .add_clock_schedule(1); // 21ms
        let cancellation_token = CancellationToken::new();
        let fp_registry = Arc::new(FailPointRegistry::default());
        let mut dispatcher = MessageDispatcher::new(
            Box::new(handler),
            rx,
            clock.clone(),
            cancellation_token.clone(),
        )
        .with_fp_registry(fp_registry.clone());
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "pause").unwrap();
        let join = tokio::spawn(async move { dispatcher.run().await });
        assert_eq!(log.lock().unwrap().len(), 0);
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "off").unwrap();

        wait_for_message_count(log.clone(), 9).await;
        assert_eq!(
            log.lock().unwrap().clone(),
            vec![
                (Phase::Pre, TestMessage::Tick(1)), // immediate tick
                (Phase::Pre, TestMessage::Tick(2)), // immediate tick
                (Phase::Pre, TestMessage::Tick(1)), // 5
                (Phase::Pre, TestMessage::Tick(2)), // 7
                (Phase::Pre, TestMessage::Tick(1)), // 10
                (Phase::Pre, TestMessage::Tick(2)), // 14
                (Phase::Pre, TestMessage::Tick(1)), // 15
                (Phase::Pre, TestMessage::Tick(1)), // 20
                (Phase::Pre, TestMessage::Tick(2)), // 21
            ]
        );

        // Shutdown and wait for cleanup to start
        cancellation_token.cancel();

        let result = timeout(Duration::from_secs(30), join)
            .await
            .expect("dispatcher did not stop in time")
            .expect("join failed");

        // Verify final state
        assert!(matches!(result, Ok(())));
        assert_eq!(log.lock().unwrap().len(), 9);
    }

    // This test simulates the following timeline:
    // immediate tick(3) (tickers always return first ticks immediately)
    // immediate tick(5)
    // advance clock to 3ms
    // tick(3)
    // advance clock to 5ms
    // tick(5)
    // advance clock to 6ms
    // tick(3)
    // advance clock to 9ms
    // tick(3)
    // advance clock to 10ms
    // tick(5)
    // advance clock to 12ms
    // tick(3)
    // clock = 15ms: tick(3), tick(5)
    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_dispatcher_supports_overlapping_tickers() {
        let log = Arc::new(Mutex::new(Vec::<(Phase, TestMessage)>::new()));
        let (_tx, rx) = mpsc::unbounded_channel::<TestMessage>();
        let clock = Arc::new(MockSystemClock::new());
        let handler = TestHandler::new(log.clone(), WatchableOnceCell::new(), clock.clone())
            .add_ticker(Duration::from_millis(3), 3)
            .add_ticker(Duration::from_millis(5), 5)
            .add_clock_schedule(0) // 0 (initial two ticks are immediate, so don't advance clock until second tick)
            .add_clock_schedule(3) // 3
            .add_clock_schedule(2) // 5
            .add_clock_schedule(1) // 6
            .add_clock_schedule(3) // 9
            .add_clock_schedule(1) // 10
            .add_clock_schedule(2) // 12
            .add_clock_schedule(3); // 15
        let cancellation_token = CancellationToken::new();
        let fp_registry = Arc::new(FailPointRegistry::default());
        let mut dispatcher = MessageDispatcher::new(
            Box::new(handler),
            rx,
            clock.clone(),
            cancellation_token.clone(),
        )
        .with_fp_registry(fp_registry.clone());
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "pause").unwrap();
        let join = tokio::spawn(async move { dispatcher.run().await });
        assert_eq!(log.lock().unwrap().len(), 0);
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "off").unwrap();

        wait_for_message_count(log.clone(), 10).await;
        assert_eq!(
            log.lock().unwrap().clone()[..8],
            vec![
                (Phase::Pre, TestMessage::Tick(3)), // immediate tick
                (Phase::Pre, TestMessage::Tick(5)), // immediate tick
                (Phase::Pre, TestMessage::Tick(3)), // 3
                (Phase::Pre, TestMessage::Tick(5)), // 5
                (Phase::Pre, TestMessage::Tick(3)), // 6
                (Phase::Pre, TestMessage::Tick(3)), // 9
                (Phase::Pre, TestMessage::Tick(5)), // 10
                (Phase::Pre, TestMessage::Tick(3)), // 12
            ]
        );
        // expect two back-to-back ticks at 15
        let mut last_two_ticks = log.lock().unwrap().clone()[8..].to_vec();
        last_two_ticks.sort_by(|a, b| match (a.1.clone(), b.1.clone()) {
            (TestMessage::Tick(a), TestMessage::Tick(b)) => a.cmp(&b),
            _ => panic!("expected ticks"),
        });
        assert_eq!(
            last_two_ticks,
            vec![
                (Phase::Pre, TestMessage::Tick(3)),
                (Phase::Pre, TestMessage::Tick(5))
            ]
        );

        // Shutdown cleanly
        cancellation_token.cancel();
        let result = timeout(Duration::from_secs(30), join)
            .await
            .expect("dispatcher did not stop in time")
            .expect("join failed");
        assert!(matches!(result, Ok(())));
        assert_eq!(log.lock().unwrap().len(), 10);
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_dispatcher_supports_identical_tickers() {
        let log = Arc::new(Mutex::new(Vec::<(Phase, TestMessage)>::new()));
        let (_tx, rx) = mpsc::unbounded_channel::<TestMessage>();
        let clock = Arc::new(MockSystemClock::new());
        let handler = TestHandler::new(log.clone(), WatchableOnceCell::new(), clock.clone())
            .add_ticker(Duration::from_millis(3), 1)
            .add_ticker(Duration::from_millis(3), 2)
            .add_clock_schedule(0) // 0 (initial two ticks are immediate, so don't advance clock until second tick)
            .add_clock_schedule(3) // 3
            .add_clock_schedule(0) // 3 (process second)
            .add_clock_schedule(3) // 6
            .add_clock_schedule(0) // 6 (process second)
            .add_clock_schedule(3) // 9
            .add_clock_schedule(0) // 9 (process second)
            .add_clock_schedule(3) // 12
            .add_clock_schedule(0) // 12 (process second)
            .add_clock_schedule(3) // 15
            .add_clock_schedule(0) // 15 (process second)
            .add_clock_schedule(3) // 18
            .add_clock_schedule(0) // 18 (process second)
            .add_clock_schedule(3) // 21
            .add_clock_schedule(0); // 21 (process second)
        let cancellation_token = CancellationToken::new();
        let fp_registry = Arc::new(FailPointRegistry::default());
        let mut dispatcher = MessageDispatcher::new(
            Box::new(handler),
            rx,
            clock.clone(),
            cancellation_token.clone(),
        )
        .with_fp_registry(fp_registry.clone());
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "pause").unwrap();
        let join = tokio::spawn(async move { dispatcher.run().await });
        assert_eq!(log.lock().unwrap().len(), 0);
        fail_parallel::cfg(fp_registry.clone(), "dispatcher-run-loop", "off").unwrap();

        wait_for_message_count(log.clone(), 16).await;
        assert_eq!(
            log.lock().unwrap().clone(),
            vec![
                (Phase::Pre, TestMessage::Tick(1)), // immediate tick
                (Phase::Pre, TestMessage::Tick(2)), // immediate tick
                (Phase::Pre, TestMessage::Tick(1)), // 3
                (Phase::Pre, TestMessage::Tick(2)), // 3
                (Phase::Pre, TestMessage::Tick(1)), // 6
                (Phase::Pre, TestMessage::Tick(2)), // 6
                (Phase::Pre, TestMessage::Tick(1)), // 9
                (Phase::Pre, TestMessage::Tick(2)), // 9
                (Phase::Pre, TestMessage::Tick(1)), // 12
                (Phase::Pre, TestMessage::Tick(2)), // 12
                (Phase::Pre, TestMessage::Tick(1)), // 15
                (Phase::Pre, TestMessage::Tick(2)), // 15
                (Phase::Pre, TestMessage::Tick(1)), // 18
                (Phase::Pre, TestMessage::Tick(2)), // 18
                (Phase::Pre, TestMessage::Tick(1)), // 21
                (Phase::Pre, TestMessage::Tick(2)), // 21
            ]
        );

        // Shutdown cleanly
        cancellation_token.cancel();
        let result = timeout(Duration::from_secs(30), join)
            .await
            .expect("dispatcher did not stop in time")
            .expect("join failed");
        assert!(matches!(result, Ok(())));
        assert_eq!(log.lock().unwrap().len(), 16);
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_executor_catches_panic_from_run_loop() {
        #[derive(Clone)]
        struct PanicHandler {
            cleanup_called: WatchableOnceCell<Result<(), SlateDBError>>,
        }

        #[async_trait::async_trait]
        impl MessageHandler<u8> for PanicHandler {
            fn tickers(&mut self) -> Vec<(Duration, Box<MessageFactory<u8>>)> {
                vec![]
            }

            async fn handle(&mut self, _message: u8) -> Result<(), SlateDBError> {
                panic!("intentional panic in handler");
            }

            async fn cleanup(
                &mut self,
                mut messages: BoxStream<'async_trait, u8>,
                result: Result<(), SlateDBError>,
            ) -> Result<(), SlateDBError> {
                // Record the shutdown result and drain any pending messages
                self.cleanup_called.write(result);
                while messages.next().await.is_some() {}
                Ok(())
            }
        }

        let cleanup_called = WatchableOnceCell::new();
        let mut cleanup_reader = cleanup_called.reader();
        let (tx, rx) = mpsc::unbounded_channel::<u8>();
        let clock = Arc::new(DefaultSystemClock::new());
        let handler = PanicHandler { cleanup_called };
        let closed_result = WatchableOnceCell::new();
        let task_executor = MessageHandlerExecutor::new(closed_result.clone(), clock.clone());

        // Spawn the panic task
        task_executor
            .add_handler(
                "test".to_string(),
                Box::new(handler),
                rx,
                &Handle::current(),
            )
            .expect("failed to spawn task");

        // Monitor the executor
        task_executor
            .monitor_on(&Handle::current())
            .expect("failed to monitor executor");

        // Trigger panic inside the run loop via handle()
        tx.send(1u8).unwrap();

        // Wait for cleanup to observe the panic result
        let _ = timeout(Duration::from_secs(30), cleanup_reader.await_value())
            .await
            .expect("timeout waiting for cleanup result");

        // Join dispatcher and verify panic was converted and propagated
        let result = timeout(Duration::from_secs(30), task_executor.join_task("test"))
            .await
            .expect("dispatcher did not stop in time");

        // Check final state
        assert!(matches!(result, Err(SlateDBError::BackgroundTaskPanic(_))));
        assert!(matches!(
            closed_result.reader().read(),
            Some(Err(SlateDBError::BackgroundTaskPanic(_)))
        ));
        assert!(matches!(
            cleanup_reader.read(),
            Some(Err(SlateDBError::BackgroundTaskPanic(_)))
        ));
    }

    #[tokio::test(flavor = "multi_thread", worker_threads = 2)]
    async fn test_executor_panic_in_wrapper() {
        let log = Arc::new(Mutex::new(Vec::<(Phase, TestMessage)>::new()));
        let cleanup_called = WatchableOnceCell::new();
        let cleanup_reader = cleanup_called.reader();
        let (tx, rx) = mpsc::unbounded_channel::<TestMessage>();
        drop(tx); // no messages needed
        let clock = Arc::new(DefaultSystemClock::new());
        let handler = TestHandler::new(log.clone(), cleanup_called.clone(), clock.clone());
        let closed_result = WatchableOnceCell::new();
        let fp_registry = Arc::new(FailPointRegistry::default());
        let task_executor = MessageHandlerExecutor::new(closed_result.clone(), clock.clone())
            .with_fp_registry(fp_registry.clone());

        fail_parallel::cfg(
            fp_registry.clone(),
            "executor-wrapper-before-write",
            "panic",
        )
        .unwrap();

        // Spawn the task and monitor
        task_executor
            .add_handler(
                "test".to_string(),
                Box::new(handler),
                rx,
                &Handle::current(),
            )
            .expect("failed to spawn task");
        task_executor
            .monitor_on(&Handle::current())
            .expect("failed to monitor executor");

        // Cancel to exit run() cleanly, then `async move`` wrapper panics at the fail point
        task_executor.cancel_task("test");

        // Wait for task result
        let result = timeout(Duration::from_secs(30), task_executor.join_task("test"))
            .await
            .expect("dispatcher did not stop in time");

        // Assertions: panic result, closed_result set to panic, cleanup not called
        assert!(matches!(result, Err(SlateDBError::BackgroundTaskPanic(_))));
        assert!(matches!(
            closed_result.reader().read(),
            Some(Err(SlateDBError::BackgroundTaskPanic(_)))
        ));
        assert!(cleanup_reader.read().is_none());
    }
}