# Change Data Capture

> Implement CDC by streaming write-ahead log (WAL) SSTs with WalReader

Change data capture (CDC) turns writes in SlateDB into a durable change stream that external systems can consume. SlateDB exposes CDC through the `WalReader`, which reads write-ahead log (WAL) SST files from object storage and returns the same row entries that were written by the database.

This page explains how to build a CDC pipeline with `WalReader`, including ordering, durability, deletes, and resume semantics.

## When to use WAL-based CDC

Use WAL-based CDC when you need:

- A low-latency stream of inserts, updates, and deletes.
- A simple, append-only feed that can be replayed.
- Integration with downstream systems like Kafka, Pulsar, or data lakes.

WAL-based CDC is not a bootstrap or backfill API. If you need an initial full copy of data, combine a backfill with WAL tailing as described below.

## What WalReader reads

SlateDB stores WAL SSTs under the `wal/` directory in the object store. Each WAL file has a monotonically increasing ID and is written in insertion order by sequence number. `WalReader` lists these files in ID order and exposes a `WalFile` for each file in object storage.

Each `WalFile` exposes an `iterator()` that yields `RowEntry` values in order. The iterator is optional because the underlying object might have been deleted between listing and reading (for example, due to GC).

Each row returned by the iterator is a `RowEntry` with:

- `key`: the key bytes
- `value`: a `ValueDeletable` variant (`Value`, `Merge`, or `Tombstone`)
- `seq`: a globally increasing sequence number
- `create_ts` and `expire_ts`: optional timestamps

Deletes appear as `Tombstone`s. If your database uses merge operators, you will see `Merge` values and must apply the merge logic yourself if your sink requires materialized values.

:::note

The database must have a write-ahead log in order to use WAL-based CDC. The write-ahead log is controlled with `Settings::wal_enabled`. `wal_enabled` defaults to `true`, and is feature-gated behind a `wal_disable` feature in Rust. Databases have a WAL unless you have deliberaly disabled it.

:::

## Visibility and durability

`WalReader` only sees WAL SSTs that have been flushed to object storage. WAL flushes occur when:

- The WAL buffer reaches its size threshold ([`l0_sst_size_bytes`](https://docs.rs/slatedb/latest/slatedb/config/struct.Settings.html#structfield.l0_sst_size_bytes) is used to limit WAL SST file sizes as well).
- The configured [`flush_interval`](https://docs.rs/slatedb/latest/slatedb/config/struct.Settings.html#structfield.flush_interval) elapses.
- You explicitly call a WAL or MemTable flush with [`Db::flush_with_options`](https://docs.rs/slatedb/latest/slatedb/struct.Db.html#method.flush_with_options).

Note that a memtable flush also forces any pending WAL data to be flushed first, which guarantees the WAL is durable before L0 data is written.

## CDC architecture

```mermaid
flowchart LR
    A[Writer] --> B[WAL buffer]
    B --> C[WAL SSTs in object store]
    C --> D[WalReader]
    D --> E[CDC sink]
```

`WalReader` does not interpret or compact data. It simply provides a durable, ordered feed of row-level changes.

## Basic tailer loop

This example tails all WAL files, emits row entries, and records a cursor so the stream can resume after restarts.

<!-- could not inline /home/runner/work/slatedb/examples/src/change_data_capture.rs -->

This pattern is safe and idempotent as long as you persist the cursor after each emitted row and skip any rows with `seq <= last_seq` when resuming.

## Backfills plus streaming

If you need a full backfill of data not in the WAL along with the WAL change stream:

1. Tail the WAL, buffering changes.
2. Checkpoint the database using `Admin::create_detached_checkpoint`.
3. Range scan (`Db::scan`) over the cloned database to read old data.
4. Apply the buffered WAL changes in sequence order, filtering rows whose sequence number was in the range scan in (3).
5. Delete the checkpoint using `Admin::delete_checkpoint`.
6. Continue tailing WALs from the stored cursor.

This avoids gaps between the backfill and the live stream. Your sink should be idempotent if the backfill and WAL stream can overlap.

## Resumability and ordering

`WalReader` lists files in wal ID order, and each WAL file stores entries in increasing sequence order. This gives you a total order that is stable across restarts. A minimal cursor includes:

- `wal_id`: the WAL file currently being processed
- `last_seq`: the highest sequence number processed in that file

If you need to fan out to multiple workers, partition by key and keep one cursor per partition. Ensure your sink can handle replays or duplicates. Since WAL files can be deleted between `list` and `iterator` calls, handle `None` by skipping that file or re-listing from a newer cursor.

## Listing costs and polling strategy

The `list()` API can become expensive when WAL retention is high or GC is not keeping up. If the GC is not running, listings can grow without bound. Even with GC, CDC often needs higher retention. Retaining WAL files for just 1 hour can yield tens of thousands of files, which is expensive to list in both cost (object-store listing calls) and time.

If you plan to poll frequently:

1. Use `list()` once to get an initial view (or to recover after a long outage).
2. Track the highest WAL ID you have successfully processed.
3. From then on, poll using `WalReader::get(latest_id + 1)`.

## Deletes, merges, and TTL

- Deletes appear as `ValueDeletable::Tombstone`. Emit a delete event in your CDC sink.
- Merge operands appear as `ValueDeletable::Merge`. If you use merge operators, downstream consumers must either apply the merge or store the operand as-is.
- TTL is represented by `expire_ts`. If your downstream system enforces TTL, honor this field.

## WAL retention and GC

WAL SSTs are garbage collected based on the GC configuration. If your CDC consumer falls behind, the WAL files it needs may be deleted. Set the WAL GC `min_age` option to retain files long enough for your slowest consumer and implement monitoring. Slow consumers should copy the WAL files elsewhere for processing. See [Garbage Collection](/docs/design/gc) for details.

## Using a separate WAL object store

If your database uses a dedicated WAL object store, pass that store to `WalReader::new` rather than the main store.
