crabka-client-streams 0.3.6

KIP-1071 Kafka Streams rebalance-protocol client for Apache Kafka in Rust
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
//! `CachingKeyValueStore`: byte-level write-back caching wrapper over a
//! `ByteKeyValueStore`. Ports Kafka `CachingKeyValueStore`.
//!
//! Reads are cache-first: a cache hit (including a tombstone, i.e. a dirty
//! `None`) wins over the underlying store. Writes are write-back: `put`/`delete`
//! only stage a dirty entry in the cache and are pushed THROUGH to the inner
//! store on `flush`. `flush` also returns the drained dirty entries so the
//! caller can forward them downstream (Kafka's flush listener).
//!
//! ## Inner mutability / locking
//!
//! [`ByteKeyValueStore`]'s `put`/`delete` take `&mut self`, so `inner` is held
//! behind a `tokio::sync::Mutex` — its async methods are awaited while the guard
//! is held, which a `std::sync::Mutex` guard cannot do (`await_holding_lock`).
//! The cache, whose ops are synchronous, uses a plain `std::sync::Mutex` whose
//! guard is always dropped before any `.await`. All public methods take `&self`
//! so the store can be shared.
//!
//! ## Merged `range`
//!
//! [`NamedCache::range`] yields the staged entries whose key falls in `[lo, hi)`
//! in ascending memcmp order, so the merge enumerates cache candidates directly
//! off the cache (no shadow key set). Cache entries win on key collision and a
//! cached tombstone hides the inner value.

use std::collections::BTreeMap;
use std::sync::{Arc, Mutex};

use bytes::Bytes;
use tokio::sync::Mutex as AsyncMutex;

use crate::processor::record::RecordContext;
use crate::store::byte::ByteKeyValueStore;
use crate::store::cache::entry::LruCacheEntry;
use crate::store::cache::named::NamedCache;

pub(crate) struct CachingKeyValueStore {
    cache: Arc<Mutex<NamedCache>>,
    inner: AsyncMutex<Box<dyn ByteKeyValueStore>>,
    /// Cache name, captured so `clear` can rebuild an empty [`NamedCache`]
    /// (which has no in-place reset) under the same identity.
    name: String,
}

impl CachingKeyValueStore {
    pub fn new(cache: Arc<Mutex<NamedCache>>, inner: Box<dyn ByteKeyValueStore>) -> Self {
        Self {
            cache,
            inner: AsyncMutex::new(inner),
            name: String::new(),
        }
    }

    /// Like [`new`](Self::new) but records the cache's name for `clear`.
    pub fn with_name(
        cache: Arc<Mutex<NamedCache>>,
        inner: Box<dyn ByteKeyValueStore>,
        name: String,
    ) -> Self {
        Self {
            cache,
            inner: AsyncMutex::new(inner),
            name,
        }
    }

    /// Cache-first read: a cache hit (including a tombstone, a dirty `None`)
    /// wins; otherwise fall through to the inner store.
    pub async fn get(&self, key: &[u8]) -> Option<Bytes> {
        let key = Bytes::copy_from_slice(key);
        // Take the cached value out of the guard before any await.
        let cached = {
            let mut cache = self.cache.lock().unwrap();
            cache.get_promote(&key).map(|e| e.value.clone())
        };
        match cached {
            // Cache hit: `Some(value)` is a live value, `None` is a tombstone —
            // both authoritative, so we never consult the inner store.
            Some(value) => value,
            // Cache miss: fall through to the underlying store.
            None => self.inner.lock().await.get(&key).await,
        }
    }

    /// Write-back put: stage a dirty entry in the cache carrying its context.
    /// The inner store is not touched until `flush`.
    ///
    /// `async` is part of the store-wrapper contract (the typed `put` path is
    /// async end to end) even though a pure cache write needs no `.await`.
    #[allow(clippy::unused_async)]
    pub async fn put(&self, key: Bytes, value: Bytes, ctx: RecordContext) {
        let mut cache = self.cache.lock().unwrap();
        cache.put(key, LruCacheEntry::new(Some(value), true, ctx));
    }

    /// Write-back delete: stage a dirty tombstone (`None`) in the cache.
    #[allow(clippy::unused_async)]
    pub async fn delete(&self, key: Bytes, ctx: RecordContext) {
        let mut cache = self.cache.lock().unwrap();
        cache.delete(key, ctx);
    }

    /// Merged range over `[lo, hi)`: the cache layer is overlaid on the inner
    /// store. Cache entries win on key collision, and a cached tombstone hides
    /// the inner value (the key is omitted). Returns key-sorted `(key, value)`.
    pub async fn range(&self, lo: &[u8], hi: &[u8]) -> Vec<(Bytes, Bytes)> {
        // Seed the merged view from the inner store.
        let mut merged: BTreeMap<Bytes, Bytes> = {
            let inner = self.inner.lock().await;
            inner.range(lo, hi).await.into_iter().collect()
        };
        // Overlay the cache entries staged in `[lo, hi)`, collected under the lock
        // and dropped before any await. Cache wins; a tombstone hides the inner
        // value.
        let cached = {
            let cache = self.cache.lock().unwrap();
            cache.range(lo, hi)
        };
        for (k, e) in cached {
            match e.value {
                // Live value: cache wins over the inner store.
                Some(v) => {
                    merged.insert(k, v);
                }
                // Tombstone: hide the inner value.
                None => {
                    merged.remove(&k);
                }
            }
        }
        merged.into_iter().collect()
    }

    /// Flush: drain dirty entries in insertion order, write each THROUGH to the
    /// inner store (`put` the value, `delete` on a tombstone), clear dirty, and
    /// return the drained entries so the caller can forward them downstream.
    pub async fn flush(&self) -> Vec<(Bytes, LruCacheEntry)> {
        let mut collected: Vec<(Bytes, LruCacheEntry)> = Vec::new();
        {
            let mut cache = self.cache.lock().unwrap();
            let mut listener =
                |k: &Bytes, e: &LruCacheEntry| collected.push((k.clone(), e.clone()));
            cache.flush(&mut listener);
        } // cache guard dropped before any await
        {
            let mut inner = self.inner.lock().await;
            for (k, e) in &collected {
                match &e.value {
                    Some(v) => inner.put(k.clone(), v.clone()).await,
                    None => {
                        inner.delete(k).await;
                    }
                }
            }
        }
        collected
    }

    /// Write straight through to the inner store, bypassing the cache. Used by
    /// the restore path (`apply_changelog`), which replays the committed
    /// changelog into the state below the cache without staging dirty entries.
    pub async fn put_inner(&self, key: Bytes, value: Bytes) {
        self.inner.lock().await.put(key, value).await;
    }

    /// Delete straight through to the inner store, bypassing the cache (restore).
    pub async fn delete_inner(&self, key: &[u8]) {
        self.inner.lock().await.delete(key).await;
    }

    /// Clear both the cache layer and the inner store (EOS rollback reset).
    pub async fn clear(&self) {
        {
            let mut cache = self.cache.lock().unwrap();
            *cache = NamedCache::new(self.name.clone());
        }
        self.inner.lock().await.clear().await;
    }

    /// Merged unbounded scan: every inner entry overlaid with the cache. Cache
    /// entries win on key collision and a cached tombstone hides the inner value.
    /// Returns key-sorted `(key, value)`.
    pub async fn scan_all(&self) -> Vec<(Bytes, Bytes)> {
        let mut merged: BTreeMap<Bytes, Bytes> = {
            let inner = self.inner.lock().await;
            inner.scan_all().await.into_iter().collect()
        };
        let cached = {
            let cache = self.cache.lock().unwrap();
            cache.all()
        };
        for (k, e) in cached {
            match e.value {
                Some(v) => {
                    merged.insert(k, v);
                }
                None => {
                    merged.remove(&k);
                }
            }
        }
        merged.into_iter().collect()
    }

    /// Flush dirty entries in insertion order, capturing the inner OLD value
    /// BEFORE each write-through. For each entry: read `old = inner.get(&k)`,
    /// then write the new value through (`put` / `delete` on a tombstone), and
    /// return `(key, old, new, context)`. `old`/`new` are `Option<Bytes>`
    /// (`None` = absent / tombstone). The typed store uses `old` (the
    /// last-committed value) to build the deduped downstream `Change` and the
    /// context to stamp the forwarded record.
    pub async fn flush_with_old(
        &self,
    ) -> Vec<(Bytes, Option<Bytes>, Option<Bytes>, RecordContext)> {
        // Drain dirty entries under the cache lock, dropping the guard before any
        // await.
        let mut dirty: Vec<(Bytes, LruCacheEntry)> = Vec::new();
        {
            let mut cache = self.cache.lock().unwrap();
            let mut listener = |k: &Bytes, e: &LruCacheEntry| dirty.push((k.clone(), e.clone()));
            cache.flush(&mut listener);
        }
        let mut out = Vec::with_capacity(dirty.len());
        {
            let mut inner = self.inner.lock().await;
            for (k, e) in dirty {
                let old = inner.get(&k).await;
                match &e.value {
                    Some(v) => inner.put(k.clone(), v.clone()).await,
                    None => {
                        inner.delete(&k).await;
                    }
                }
                out.push((k, old, e.value, e.context));
            }
        }
        out
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::store::byte::InMemoryBytes;

    fn ctx() -> RecordContext {
        RecordContext {
            topic: "t".to_string(),
            partition: 0,
            offset: 0,
            timestamp: 0,
        }
    }

    fn cache() -> Arc<Mutex<NamedCache>> {
        Arc::new(Mutex::new(NamedCache::new("s".to_string())))
    }

    fn b(v: &'static [u8]) -> Bytes {
        Bytes::from_static(v)
    }

    #[tokio::test]
    async fn get_returns_cached_before_underlying() {
        let inner = InMemoryBytes::default();
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));

        store.put(b(b"k"), b(b"v"), ctx()).await;

        // Cached only — not yet flushed to inner.
        assert_eq!(store.get(b"k").await, Some(b(b"v")));
    }

    #[tokio::test]
    async fn flush_writes_through_and_returns_entries() {
        let store = CachingKeyValueStore::new(cache(), Box::new(InMemoryBytes::default()));

        store.put(b(b"k"), b(b"v"), ctx()).await;
        let flushed = store.flush().await;

        assert_eq!(flushed.len(), 1);
        assert_eq!(flushed[0].0, b(b"k"));
        assert_eq!(flushed[0].1.value, Some(b(b"v")));

        // Inner now has the write-through value; serve it from the (now-clean)
        // cache or inner — either way `get` returns it.
        assert_eq!(store.get(b"k").await, Some(b(b"v")));
    }

    #[tokio::test]
    async fn tombstone_hides_underlying() {
        // Seed inner with k -> v0.
        let mut inner = InMemoryBytes::default();
        inner.put(b(b"k"), b(b"v0")).await;
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));

        // Sanity: inner value is visible before the delete.
        assert_eq!(store.get(b"k").await, Some(b(b"v0")));

        store.delete(b(b"k"), ctx()).await;
        // Cached tombstone hides the inner value.
        assert_eq!(store.get(b"k").await, None);

        // After flush, the inner store no longer has the key.
        store.flush().await;
        assert_eq!(store.get(b"k").await, None);
    }

    #[tokio::test]
    async fn range_merges_cache_and_underlying_cache_wins() {
        // Inner: a -> 1, c -> 3.
        let mut inner = InMemoryBytes::default();
        inner.put(b(b"a"), b(b"1")).await;
        inner.put(b(b"c"), b(b"3")).await;
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));

        // Cache: b -> 2 (new), a -> 9 (overrides inner).
        store.put(b(b"b"), b(b"2"), ctx()).await;
        store.put(b(b"a"), b(b"9"), ctx()).await;

        // Range [a, d) covers a, b, c.
        let r = store.range(b"a", b"d").await;
        assert_eq!(
            r,
            vec![
                (b(b"a"), b(b"9")), // cache wins over inner's a -> 1
                (b(b"b"), b(b"2")), // cache-only
                (b(b"c"), b(b"3")), // inner-only
            ]
        );
    }

    #[tokio::test]
    async fn range_tombstone_omits_key() {
        let mut inner = InMemoryBytes::default();
        inner.put(b(b"a"), b(b"1")).await;
        inner.put(b(b"b"), b(b"2")).await;
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));

        store.delete(b(b"a"), ctx()).await;

        let r = store.range(b"a", b"c").await;
        assert_eq!(r, vec![(b(b"b"), b(b"2"))]);
    }

    /// A cache miss falls through to the inner store (the `None` arm of `get`).
    #[tokio::test]
    async fn get_falls_through_to_inner_on_miss() {
        let mut inner = InMemoryBytes::default();
        inner.put(b(b"k"), b(b"inner")).await;
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));

        // Nothing staged in the cache for "k": the read falls through to inner.
        assert_eq!(store.get(b"k").await, Some(b(b"inner")));
        // A genuinely-absent key returns None from inner.
        assert_eq!(store.get(b"missing").await, None);
    }

    /// `scan_all` overlays the full cache on the full inner store: cache wins on
    /// collision and a cached tombstone hides the inner value.
    #[tokio::test]
    async fn scan_all_merges_cache_and_underlying() {
        let mut inner = InMemoryBytes::default();
        inner.put(b(b"a"), b(b"1")).await;
        inner.put(b(b"b"), b(b"2")).await;
        inner.put(b(b"c"), b(b"3")).await;
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));

        store.put(b(b"b"), b(b"9"), ctx()).await; // overrides inner b -> 2
        store.put(b(b"d"), b(b"4"), ctx()).await; // cache-only
        store.delete(b(b"c"), ctx()).await; // tombstone hides inner c

        let r = store.scan_all().await;
        assert_eq!(
            r,
            vec![
                (b(b"a"), b(b"1")), // inner-only
                (b(b"b"), b(b"9")), // cache wins
                (b(b"d"), b(b"4")), // cache-only
            ]
        );
    }

    /// `put_inner` / `delete_inner` write straight through, bypassing the cache:
    /// no dirty entry is staged (a subsequent `flush` drains nothing).
    #[tokio::test]
    async fn put_and_delete_inner_bypass_the_cache() {
        let store = CachingKeyValueStore::new(cache(), Box::new(InMemoryBytes::default()));

        store.put_inner(b(b"k"), b(b"v")).await;
        // Visible via the cache-first read (falls through to inner) ...
        assert_eq!(store.get(b"k").await, Some(b(b"v")));
        // ... and no dirty entry was staged.
        assert!(store.flush().await.is_empty());

        store.delete_inner(b"k").await;
        assert_eq!(store.get(b"k").await, None);
        assert!(store.flush().await.is_empty());
    }

    /// `clear` empties both the cache layer (dropping staged dirty entries) and
    /// the inner store.
    #[tokio::test]
    async fn clear_empties_cache_and_inner() {
        let mut inner = InMemoryBytes::default();
        inner.put(b(b"a"), b(b"1")).await;
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));
        store.put(b(b"b"), b(b"2"), ctx()).await; // staged dirty

        store.clear().await;

        // Both the staged entry and the inner value are gone.
        assert_eq!(store.get(b"a").await, None);
        assert_eq!(store.get(b"b").await, None);
        assert!(store.scan_all().await.is_empty());
        // The cleared cache has no dirty entries to flush.
        assert!(store.flush().await.is_empty());
    }

    /// `flush_with_old` reports `old = None` for a key with no prior inner value
    /// and `old = Some(..)` for one that does, then writes both through.
    #[tokio::test]
    async fn flush_with_old_distinguishes_absent_and_present_inner() {
        let mut inner = InMemoryBytes::default();
        inner.put(b(b"present"), b(b"old")).await;
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));

        store.put(b(b"present"), b(b"new"), ctx()).await; // has inner old
        store.put(b(b"fresh"), b(b"v"), ctx()).await; // no inner old

        let mut drained = store.flush_with_old().await;
        // Sort by key for a deterministic assertion (insertion order is preserved
        // by the cache, but make the test independent of it).
        drained.sort_by(|a, b| a.0.cmp(&b.0));
        assert_eq!(drained.len(), 2);

        let fresh = &drained[0];
        assert_eq!(fresh.0, b(b"fresh"));
        assert_eq!(fresh.1, None); // no prior inner value
        assert_eq!(fresh.2, Some(b(b"v")));

        let present = &drained[1];
        assert_eq!(present.0, b(b"present"));
        assert_eq!(present.1, Some(b(b"old"))); // prior inner value captured
        assert_eq!(present.2, Some(b(b"new")));

        // Both write-throughs landed.
        assert_eq!(store.get(b"present").await, Some(b(b"new")));
        assert_eq!(store.get(b"fresh").await, Some(b(b"v")));
    }

    /// `flush_with_old` on a tombstone returns `new = None` and deletes the inner
    /// value through (the tombstone arm of the write-through).
    #[tokio::test]
    async fn flush_with_old_tombstone_deletes_through() {
        let mut inner = InMemoryBytes::default();
        inner.put(b(b"k"), b(b"old")).await;
        let store = CachingKeyValueStore::new(cache(), Box::new(inner));

        store.delete(b(b"k"), ctx()).await;

        let drained = store.flush_with_old().await;
        assert_eq!(drained.len(), 1);
        assert_eq!(drained[0].0, b(b"k"));
        assert_eq!(drained[0].1, Some(b(b"old"))); // inner OLD captured
        assert_eq!(drained[0].2, None); // tombstone

        assert_eq!(store.get(b"k").await, None);
    }
}