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
//! Parallel L0 flush SST uploader.
//!
//! The uploader is intentionally narrow in scope:
//! - build an SST from an immutable memtable
//! - upload the SST to object storage
//! - report successful completion
//!
//! It does not own:
//! - manifest mutation
//! - checkpoint creation
//! - flush waiter bookkeeping
//! - timeout enforcement

use super::tracker::TrackerMessage;
use crate::db::DbInner;
use crate::db_state::{SsTableHandle, SsTableId};
use crate::db_status::ClosedResultWriter;
use crate::dispatcher::{MessageHandler, MessageHandlerExecutor};
use crate::error::SlateDBError;
use crate::format::sst::EncodedSsTable;
use crate::mem_table::ImmutableMemtable;
use crate::utils::SafeSender;
use async_trait::async_trait;
use futures::stream::BoxStream;
use futures::StreamExt;
use log::warn;
use std::sync::Arc;
use std::time::Duration;
use tokio::runtime::Handle;

const UPLOADER_TASK_NAME: &str = "l0_sst_uploader";

/// One immutable-memtable upload request submitted to the uploader.
pub(crate) struct UploadJob {
    /// Immutable memtable to build into an SST.
    pub(crate) imm_memtable: Arc<ImmutableMemtable>,
    /// Preallocated SST id to use for the uploaded table.
    pub(crate) sst_id: SsTableId,
}

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

impl UploadJob {
    /// Creates a new upload job.
    pub(crate) fn new(imm_memtable: Arc<ImmutableMemtable>, sst_id: SsTableId) -> Self {
        Self {
            imm_memtable,
            sst_id,
        }
    }
}

#[derive(Clone)]
/// Result of a successfully uploaded immutable memtable.
pub(crate) struct UploadedMemtable {
    /// Same immutable memtable that was uploaded.
    pub(crate) imm_memtable: Arc<ImmutableMemtable>,
    /// Handle for the uploaded SST in object storage.
    pub(crate) sst_handle: SsTableHandle,
    /// Lowest sequence number present in the immutable memtable.
    pub(crate) first_seq: u64,
    /// Highest sequence number present in the immutable memtable.
    pub(crate) last_seq: u64,
}

impl UploadedMemtable {
    #[cfg(test)]
    pub(crate) fn new(
        imm_memtable: Arc<ImmutableMemtable>,
        sst_handle: SsTableHandle,
        first_seq: u64,
        last_seq: u64,
    ) -> Self {
        assert!(first_seq <= last_seq);
        Self {
            imm_memtable,
            sst_handle,
            first_seq,
            last_seq,
        }
    }
}

/// Wrapper around the upload job channel. Owns the tx channel and registers
/// workers with the executor on [`start`](Self::start).
#[derive(Clone)]
pub(crate) struct Uploader {
    jobs_tx: SafeSender<UploadJob>,
}

impl Uploader {
    pub(crate) fn start(
        db: Arc<DbInner>,
        closed_result: &dyn ClosedResultWriter,
        tracker_tx: SafeSender<TrackerMessage>,
        executor: &MessageHandlerExecutor,
        tokio_handle: &Handle,
    ) -> Result<Self, SlateDBError> {
        let (jobs_tx, jobs_rx) = SafeSender::unbounded_channel(closed_result.result_reader());
        let uploader = Uploader { jobs_tx };
        let handlers = Self::build_handlers(db, tracker_tx);
        executor.add_handlers(
            UPLOADER_TASK_NAME.to_string(),
            handlers,
            jobs_rx,
            tokio_handle,
        )?;
        Ok(uploader)
    }

    pub(crate) fn build_handlers(
        db: Arc<DbInner>,
        tracker_tx: SafeSender<TrackerMessage>,
    ) -> Vec<Box<dyn MessageHandler<UploadJob>>> {
        let parallelism = db.settings.l0_flush_parallelism;
        let retry_backoff = db.settings.manifest_poll_interval;
        (0..parallelism)
            .map(|_| {
                Box::new(UploadHandler::new(
                    Arc::clone(&db),
                    tracker_tx.clone(),
                    retry_backoff,
                )) as Box<dyn MessageHandler<UploadJob>>
            })
            .collect()
    }

    /// Submits a new upload job.
    pub(crate) fn submit(&self, job: UploadJob) -> Result<(), SlateDBError> {
        self.jobs_tx.send(job)
    }

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

/// MessageHandler that builds and uploads one SST per job.
pub(crate) struct UploadHandler {
    db: Arc<DbInner>,
    tracker_tx: SafeSender<TrackerMessage>,
    retry_backoff: Duration,
}

impl UploadHandler {
    pub(crate) fn new(
        db: Arc<DbInner>,
        tracker_tx: SafeSender<TrackerMessage>,
        retry_backoff: Duration,
    ) -> Self {
        Self {
            db,
            tracker_tx,
            retry_backoff,
        }
    }

    async fn upload_with_retry(&self, job: &UploadJob) -> Result<UploadedMemtable, SlateDBError> {
        loop {
            let encoded_sst = self.db.build_imm_sst(job.imm_memtable.table()).await?;
            match self.try_upload_once(job, encoded_sst).await {
                Ok(success) => return Ok(success),
                Err(_) => {
                    self.db.system_clock.sleep(self.retry_backoff).await;
                }
            }
        }
    }

    async fn try_upload_once(
        &self,
        job: &UploadJob,
        encoded_sst: EncodedSsTable,
    ) -> Result<UploadedMemtable, SlateDBError> {
        // TODO: consider changing the low-level upload path so failed uploads
        // return ownership of the built SST. That would let the worker build
        // once and retry uploads without rebuilding.
        let first_seq = job
            .imm_memtable
            .table()
            .first_seq()
            .expect("flush of l0 with no entries");
        let last_seq = job
            .imm_memtable
            .table()
            .last_seq()
            .expect("flush of l0 with no entries");
        let written_bytes = encoded_sst.remaining_len() as u64;
        let sst_handle = self
            .db
            .upload_compacted_sst(&job.sst_id, job.imm_memtable.table(), encoded_sst, true)
            .await?;
        self.db.db_stats.l0_flush_bytes.increment(written_bytes);
        Ok(UploadedMemtable {
            imm_memtable: Arc::clone(&job.imm_memtable),
            sst_handle,
            first_seq,
            last_seq,
        })
    }
}

#[async_trait]
impl MessageHandler<UploadJob> for UploadHandler {
    async fn handle(&mut self, job: UploadJob) -> Result<(), SlateDBError> {
        let success = self.upload_with_retry(&job).await?;
        self.tracker_tx
            .send(TrackerMessage::UploadComplete(success))?;
        Ok(())
    }

    async fn cleanup(
        &mut self,
        mut messages: BoxStream<'async_trait, UploadJob>,
        result: Result<(), SlateDBError>,
    ) -> Result<(), SlateDBError> {
        // On clean shutdown, drain remaining jobs.
        if result.is_ok() {
            while let Some(job) = messages.next().await {
                let success = self.upload_with_retry(&job).await?;
                self.tracker_tx
                    .send(TrackerMessage::UploadComplete(success))?;
            }
        }
        Ok(())
    }
}

#[cfg(test)]
mod tests {
    use super::{TrackerMessage, UploadJob, Uploader};
    use crate::config::Settings;
    use crate::db::DbInner;
    use crate::db_state::{ManifestCore, SsTableId, SsTableView};
    use crate::db_status::{ClosedResultWriter, DbStatusManager};
    use crate::error::SlateDBError;
    use crate::format::sst::SsTableFormat;
    use crate::iter::RowEntryIterator;
    use crate::object_stores::ObjectStores;
    use crate::paths::PathResolver;
    use crate::rand::DbRand;
    use crate::sst_iter::{SstIterator, SstIteratorOptions};
    use crate::tablestore::TableStore;
    use crate::types::{RowEntry, ValueDeletable};
    use crate::utils::IdGenerator;
    use crate::utils::WatchableOnceCell;
    use bytes::Bytes;
    use fail_parallel::FailPointRegistry;
    use object_store::memory::InMemory;
    use object_store::path::Path;
    use object_store::ObjectStore;
    use slatedb_common::clock::{DefaultSystemClock, SystemClock};
    use slatedb_common::metrics::MetricsRecorderHelper;
    use std::sync::Arc;
    use std::time::Duration;
    use tokio::runtime::Handle;
    use tokio::time::timeout;

    async fn setup_db(path: &str, fp_registry: Arc<FailPointRegistry>) -> Arc<DbInner> {
        let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
        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(crate::manifest::store::ManifestStore::new(
            &Path::from(path),
            Arc::clone(&object_store),
        ));
        let stored_manifest = crate::manifest::store::StoredManifest::create_new_db(
            manifest_store,
            ManifestCore::new_with_wal_object_store(None),
            Arc::clone(&system_clock),
        )
        .await
        .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)),
            fp_registry.clone(),
            None,
        ));
        let status_manager = DbStatusManager::new(0);
        let (write_tx, _) =
            crate::utils::SafeSender::unbounded_channel(status_manager.result_reader());
        Arc::new(
            DbInner::new(
                settings,
                Arc::clone(&system_clock),
                Arc::clone(&rand),
                Arc::clone(&table_store),
                stored_manifest.prepare_dirty().unwrap(),
                Arc::new(crate::memtable_flusher::MemtableFlusher::new(
                    &status_manager,
                )),
                write_tx,
                db_metrics,
                fp_registry,
                None,
                status_manager,
            )
            .await
            .unwrap(),
        )
    }

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

    fn next_upload_job(db: &DbInner, key: &[u8], value: &[u8], seq: u64) -> UploadJob {
        let imm_memtable = freeze_imm(db, key, value, seq);
        let sst_id = SsTableId::Compacted(db.rand.rng().gen_ulid(db.system_clock.as_ref()));
        UploadJob::new(imm_memtable, sst_id)
    }

    struct TestUploader {
        uploader: Uploader,
        tracker_rx: async_channel::Receiver<TrackerMessage>,
        executor: Arc<crate::dispatcher::MessageHandlerExecutor>,
        closed_result: WatchableOnceCell<Result<(), SlateDBError>>,
    }

    impl TestUploader {
        /// 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
        }

        async fn shutdown(&self) {
            Uploader::shutdown(&self.executor).await;
        }
    }

    impl std::ops::Deref for TestUploader {
        type Target = Uploader;
        fn deref(&self) -> &Self::Target {
            &self.uploader
        }
    }

    fn start_test_uploader(db: &Arc<DbInner>) -> TestUploader {
        let closed_result: WatchableOnceCell<Result<(), SlateDBError>> = WatchableOnceCell::new();
        let system_clock: Arc<dyn SystemClock> = Arc::new(DefaultSystemClock::new());
        let (tracker_tx, tracker_rx) =
            crate::utils::SafeSender::unbounded_channel(closed_result.result_reader());
        let executor = Arc::new(crate::dispatcher::MessageHandlerExecutor::new(
            Arc::new(closed_result.clone()),
            system_clock,
        ));
        let uploader = Uploader::start(
            Arc::clone(db),
            &closed_result,
            tracker_tx,
            &executor,
            &Handle::current(),
        )
        .unwrap();
        executor.monitor_on(&Handle::current()).unwrap();
        TestUploader {
            uploader,
            tracker_rx,
            executor,
            closed_result,
        }
    }

    #[tokio::test]
    async fn should_emit_uploaded_event_for_successful_job() {
        let db = setup_db(
            "/tmp/test_parallel_l0_flush_uploader_success",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let job = next_upload_job(&db, b"key", b"value", 1);

        let test = start_test_uploader(&db);
        test.submit(job).unwrap();

        let msg = timeout(Duration::from_secs(5), test.tracker_rx.recv())
            .await
            .unwrap()
            .unwrap();
        let TrackerMessage::UploadComplete(event) = msg else {
            panic!("expected UploadComplete");
        };
        assert_eq!(event.first_seq, 1);
        assert_eq!(event.last_seq, 1);

        // Verify the uploaded SST contains the expected key-value entry.
        let sst_view =
            SsTableView::identity(db.table_store.open_sst(&event.sst_handle.id).await.unwrap());
        let mut iter = SstIterator::new_owned_initialized(
            ..,
            sst_view,
            Arc::clone(&db.table_store),
            SstIteratorOptions::default(),
        )
        .await
        .unwrap()
        .expect("expected non-empty SST");
        let entry = iter
            .next()
            .await
            .unwrap()
            .expect("expected at least one entry");
        assert_eq!(entry.key.as_ref(), b"key");
        assert_eq!(entry.value, ValueDeletable::Value(Bytes::from("value")));
        assert_eq!(entry.seq, 1);
        assert!(
            iter.next().await.unwrap().is_none(),
            "expected exactly one entry"
        );

        test.shutdown().await;
    }

    #[tokio::test]
    async fn should_retry_upload_failures_until_success() {
        let fp_registry = Arc::new(FailPointRegistry::new());
        fail_parallel::cfg(
            Arc::clone(&fp_registry),
            "write-compacted-sst-io-error",
            "1*off->return",
        )
        .unwrap();
        let db = setup_db("/tmp/test_parallel_l0_flush_uploader_retry", fp_registry).await;
        let job = next_upload_job(&db, b"key", b"value", 1);

        let test = start_test_uploader(&db);
        test.submit(job).unwrap();

        let msg = timeout(Duration::from_secs(5), test.tracker_rx.recv())
            .await
            .unwrap()
            .unwrap();
        let TrackerMessage::UploadComplete(event) = msg else {
            panic!("expected UploadComplete");
        };
        assert_eq!(event.first_seq, 1);
        assert_eq!(event.last_seq, 1);

        test.shutdown().await;
    }

    #[tokio::test]
    async fn should_emit_fatal_event_for_build_failure() {
        let db = setup_db(
            "/tmp/test_parallel_l0_flush_uploader_build_failure",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        {
            let mut guard = db.state.write();
            guard.memtable().put(crate::types::RowEntry::new_merge(
                b"key",
                b"merge_operand",
                1,
            ));
            guard.freeze_memtable(0);
        }
        let imm_memtable = db
            .state
            .read()
            .state()
            .imm_memtable
            .front()
            .cloned()
            .unwrap();
        let sst_id = SsTableId::Compacted(db.rand.rng().gen_ulid(db.system_clock.as_ref()));
        let job = UploadJob::new(imm_memtable, sst_id);

        let test = start_test_uploader(&db);
        test.submit(job).unwrap();

        // The worker returns a fatal error. Wait for the executor to
        // propagate it to closed_result.
        let result = timeout(Duration::from_secs(5), test.await_closed())
            .await
            .expect("timed out waiting for fatal error");
        assert!(result.is_err());
        assert!(
            !matches!(result, Err(SlateDBError::Closed)),
            "expected specific error, got Closed"
        );
    }

    #[tokio::test]
    async fn should_process_multiple_jobs() {
        let db = setup_db(
            "/tmp/test_parallel_l0_flush_uploader_multiple",
            Arc::new(FailPointRegistry::new()),
        )
        .await;
        let job1 = next_upload_job(&db, b"key1", b"value1", 1);
        let job2 = next_upload_job(&db, b"key2", b"value2", 2);

        let test = start_test_uploader(&db);
        test.submit(job1).unwrap();
        test.submit(job2).unwrap();

        // Collect both upload completions.
        let mut uploaded_seqs = Vec::new();
        for _ in 0..2 {
            let msg = timeout(Duration::from_secs(5), test.tracker_rx.recv())
                .await
                .unwrap()
                .unwrap();
            if let TrackerMessage::UploadComplete(uploaded) = msg {
                uploaded_seqs.push(uploaded.last_seq);
            }
        }
        uploaded_seqs.sort();
        assert_eq!(uploaded_seqs, vec![1, 2]);

        test.shutdown().await;
    }

    #[tokio::test]
    async fn submit_should_fail_after_worker_fatal_error() {
        let db = setup_db(
            "/tmp/test_parallel_l0_flush_uploader_submit_after_fatal",
            Arc::new(FailPointRegistry::new()),
        )
        .await;

        // Create a merge entry without a merge operator — this causes a
        // fatal build error in the worker.
        {
            let mut guard = db.state.write();
            guard
                .memtable()
                .put(crate::types::RowEntry::new_merge(b"key", b"operand", 1));
            guard.freeze_memtable(0);
        }
        let imm_memtable = db
            .state
            .read()
            .state()
            .imm_memtable
            .front()
            .cloned()
            .unwrap();
        let sst_id = SsTableId::Compacted(db.rand.rng().gen_ulid(db.system_clock.as_ref()));
        let bad_job = UploadJob::new(imm_memtable, sst_id);

        let test = start_test_uploader(&db);
        test.submit(bad_job).unwrap();

        // Wait for the executor to propagate the error to closed_result.
        let result = timeout(Duration::from_secs(5), test.await_closed())
            .await
            .expect("timed out waiting for fatal error");
        assert!(result.is_err());

        // Subsequent submits should fail with the specific error.
        let err = test
            .submit(next_upload_job(&db, b"key2", b"value2", 2))
            .unwrap_err();
        assert!(
            !matches!(err, SlateDBError::Closed),
            "expected specific error, got Closed"
        );
    }
}