laminar-core 0.26.0

Core streaming engine for LaminarDB - operators, checkpoint barriers, and streaming primitives
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
//! Multi-instance integration tests for the cluster control plane.
//!
//! Each test spawns a real N-node `MiniCluster` — N chitchat
//! instances on loopback UDP plus per-node `ClusterController`s —
//! and exercises a specific property. Timeouts are generous to
//! accommodate gossip propagation variability.
//!
//! Feature-gated on `cluster`. Run with:
//!     cargo test -p laminar-core --features cluster \
//!         --test cluster_integration

#![cfg(feature = "cluster")]

use std::collections::BTreeMap;
use std::sync::Arc;
use std::time::{Duration, Instant};

use laminar_core::cluster::control::{
    AssignmentSnapshot, AssignmentSnapshotStore, BarrierAck, BarrierAnnouncement, QuorumOutcome,
};
use laminar_core::cluster::discovery::NodeId;
use laminar_core::cluster::testing::{FaultyObjectStore, MiniCluster, ObjectStoreFault};
use object_store::local::LocalFileSystem;
use object_store::ObjectStore;

const CONVERGENCE_DEADLINE: Duration = Duration::from_secs(8);
/// After killing a node, how long to wait for the remaining nodes to
/// observe its `Left` announcement via gossip. 5 s covers the
/// 500 ms gossip_discovery watcher interval plus a margin.
const FAILOVER_DEADLINE: Duration = Duration::from_secs(5);

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn three_node_cluster_converges() {
    let cluster = MiniCluster::spawn(3).await;
    cluster
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("cluster must converge");
    cluster.shutdown().await;
}

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn leader_is_consistent_across_nodes() {
    let cluster = MiniCluster::spawn(3).await;
    cluster
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("convergence");

    // Every node's view must agree on the leader (lowest ID).
    // Node IDs are 1, 2, 3 → leader is 1.
    let leaders: Vec<_> = cluster
        .nodes
        .iter()
        .map(|n| n.controller.current_leader())
        .collect();

    for (i, l) in leaders.iter().enumerate() {
        assert_eq!(
            *l,
            Some(cluster.nodes[0].instance_id),
            "node {i} disagrees on leader; got {l:?}"
        );
    }

    // Exactly one `is_leader() == true`.
    let leader_count = cluster
        .nodes
        .iter()
        .filter(|n| n.controller.is_leader())
        .count();
    assert_eq!(
        leader_count, 1,
        "exactly one node must self-identify as leader"
    );
    assert!(
        cluster.nodes[0].controller.is_leader(),
        "node 0 (lowest id) is leader"
    );

    cluster.shutdown().await;
}

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn barrier_announce_then_follower_observes_and_acks() {
    // End-to-end barrier round-trip across real chitchat gossip:
    //   1. Leader announces a barrier via ClusterKv.
    //   2. Followers observe via gossip (~100–500 ms).
    //   3. Followers ack via ClusterKv.
    //   4. Leader waits for quorum and sees all acks.
    let cluster = MiniCluster::spawn(3).await;
    cluster
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("convergence");

    let leader = &cluster.nodes[0];
    let followers: Vec<_> = cluster.nodes.iter().skip(1).collect();
    let follower_ids: Vec<_> = followers.iter().map(|n| n.instance_id).collect();
    let announcement = BarrierAnnouncement {
        epoch: 7,
        checkpoint_id: 101,
        phase: laminar_core::cluster::control::Phase::Prepare,
        flags: 0,
        min_watermark_ms: None,
    };

    // Step 1: leader announces.
    leader
        .controller
        .announce_barrier(&announcement)
        .await
        .expect("announce");

    // Step 2+3: each follower observes the leader's KV and acks.
    //   Retry loop tolerates gossip propagation delay.
    for follower in &followers {
        let mut observed = None;
        let start = std::time::Instant::now();
        while start.elapsed() < Duration::from_secs(4) {
            match follower.controller.observe_barrier().await {
                Ok(Some(ann)) if ann.epoch == 7 => {
                    observed = Some(ann);
                    break;
                }
                _ => tokio::time::sleep(Duration::from_millis(50)).await,
            }
        }
        observed.unwrap_or_else(|| {
            panic!(
                "follower {} never observed the barrier",
                follower.instance_id.0
            )
        });
        follower
            .controller
            .ack_barrier(&BarrierAck {
                epoch: 7,
                ok: true,
                error: None,
                local_watermark_ms: None,
            })
            .await
            .expect("ack");
    }

    // Step 4: leader observes quorum. Quorum is just the followers;
    // the leader does not ack itself in this test (the orchestration
    // in the eventual checkpoint flow will — but the BarrierCoordinator
    // doesn't enforce that semantically).
    let outcome = leader
        .controller
        .wait_for_quorum(7, &follower_ids, Duration::from_secs(6))
        .await;
    match outcome {
        QuorumOutcome::Reached { mut acks, .. } => {
            acks.sort_by_key(|n| n.0);
            let mut expected = follower_ids.clone();
            expected.sort_by_key(|n| n.0);
            assert_eq!(acks, expected);
        }
        other => panic!("expected Reached, got {other:?}"),
    }

    cluster.shutdown().await;
}

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn leader_failover_on_kill() {
    // Kill the original leader (lowest id); verify the next-lowest-id
    // node takes over within FAILOVER_DEADLINE and every surviving
    // node agrees on the new leader.
    let mut cluster = MiniCluster::spawn(3).await;
    cluster
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("convergence");

    assert!(
        cluster.nodes[0].controller.is_leader(),
        "node 0 (lowest id) starts as leader"
    );

    // Graceful kill: the stopped discovery tells peers it's Left.
    // Surviving nodes' current_leader() must switch to node 1.
    let original_leader = cluster.nodes.remove(0);
    let original_id = original_leader.instance_id;
    original_leader.kill().await;

    let start = Instant::now();
    let expected = cluster.nodes[0].instance_id;
    loop {
        let leaders: Vec<_> = cluster
            .nodes
            .iter()
            .map(|n| n.controller.current_leader())
            .collect();
        let all_agree = leaders.iter().all(|l| *l == Some(expected));
        let correct_self_id =
            cluster.nodes[0].controller.is_leader() && !cluster.nodes[1].controller.is_leader();
        if all_agree && correct_self_id {
            break;
        }
        if start.elapsed() > FAILOVER_DEADLINE {
            panic!(
                "leader failover incomplete after {:?}: got leaders {leaders:?}, expected {expected:?} (original {original_id:?})",
                FAILOVER_DEADLINE
            );
        }
        tokio::time::sleep(Duration::from_millis(100)).await;
    }

    cluster.shutdown().await;
}

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn network_partition_produces_two_leaders() {
    // Partition the cluster into [1,2] and [3,4]. Each side sees the
    // other as unreachable, marks them suspected, and its own lowest-
    // live-id instance takes leader role. This is the split-brain
    // symptom the commit-manifest CAS fence resolves at durable
    // commit time (see design doc §7).
    let cluster = MiniCluster::spawn_partitionable(4).await;
    cluster
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("initial convergence");

    // Pre-partition invariant: one leader, all agree.
    let pre = cluster.nodes[0].controller.current_leader();
    assert_eq!(pre, Some(cluster.nodes[0].instance_id));

    let addrs = cluster.addrs();
    let rules = cluster
        .rules
        .as_ref()
        .expect("partitionable cluster has rules");
    rules.partition(&addrs[0..2], &addrs[2..4]);

    // Wait for each side to re-derive its local leader.
    // Side A (nodes 0,1): leader becomes node 0 (id 1). Side B
    // (nodes 2,3): leader becomes node 2 (id 3).
    let expected_side_a = cluster.nodes[0].instance_id;
    let expected_side_b = cluster.nodes[2].instance_id;

    let start = Instant::now();
    loop {
        let side_a = cluster.nodes[0].controller.current_leader();
        let side_a_peer = cluster.nodes[1].controller.current_leader();
        let side_b = cluster.nodes[2].controller.current_leader();
        let side_b_peer = cluster.nodes[3].controller.current_leader();

        let converged = side_a == Some(expected_side_a)
            && side_a_peer == Some(expected_side_a)
            && side_b == Some(expected_side_b)
            && side_b_peer == Some(expected_side_b);

        if converged {
            // Two leaders — exactly the split-brain we expected.
            break;
        }
        if start.elapsed() > FAILOVER_DEADLINE {
            panic!(
                "partition did not produce split leaders within {:?}: \
                 side_a={side_a:?} side_a_peer={side_a_peer:?} \
                 side_b={side_b:?} side_b_peer={side_b_peer:?}",
                FAILOVER_DEADLINE
            );
        }
        tokio::time::sleep(Duration::from_millis(100)).await;
    }

    // Healing restores a single leader across all four nodes.
    rules.heal();
    let start = Instant::now();
    loop {
        let leaders: Vec<_> = cluster
            .nodes
            .iter()
            .map(|n| n.controller.current_leader())
            .collect();
        if leaders.iter().all(|l| *l == Some(expected_side_a)) {
            break;
        }
        if start.elapsed() > FAILOVER_DEADLINE {
            panic!("partition did not heal in {FAILOVER_DEADLINE:?}: {leaders:?}");
        }
        tokio::time::sleep(Duration::from_millis(100)).await;
    }

    cluster.shutdown().await;
}

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn snapshot_save_fails_under_object_store_fault_and_recovers() {
    // Phase C.5 failure-matrix row: "object store unavailable".
    //
    // FaultyObjectStore wraps a real LocalFileSystem. Toggling the
    // fault mode mid-test simulates a backing-store outage:
    //   1. Normal save succeeds.
    //   2. set_fault(FailWrites) → subsequent save returns Generic
    //      error, but the prior snapshot on disk is unaffected.
    //   3. set_fault(FailReads) → load also errors (NotFound injection).
    //   4. set_fault(None) → next save + load succeed; the new
    //      snapshot is visible.
    let dir = tempfile::tempdir().expect("tempdir");
    let inner: Arc<dyn ObjectStore> =
        Arc::new(LocalFileSystem::new_with_prefix(dir.path()).unwrap());
    let faulty = Arc::new(FaultyObjectStore::new(inner));
    let store = Arc::new(AssignmentSnapshotStore::new(
        Arc::clone(&faulty) as Arc<dyn ObjectStore>
    ));

    // Step 1: save baseline via CAS-create.
    let mut v1_map = BTreeMap::new();
    v1_map.insert(0u32, NodeId(1));
    let v1 = AssignmentSnapshot::empty().next(v1_map);
    store.save_if_absent(&v1).await.expect("baseline save");

    // Step 2: turn writes off; the next rotate must fail.
    faulty.set_fault(ObjectStoreFault::FailWrites);
    let mut v2_map = BTreeMap::new();
    v2_map.insert(0u32, NodeId(2));
    let v2 = v1.next(v2_map);
    let write_err = store
        .save_if_version(&v2, v1.version)
        .await
        .expect_err("write should fail");
    assert!(
        format!("{write_err}").to_lowercase().contains("injected")
            || matches!(
                write_err,
                laminar_core::cluster::control::SnapshotError::Io(_)
            ),
        "unexpected error shape: {write_err:?}"
    );

    // Step 3: load still sees the v1 snapshot in Normal / writes-off
    // mode (reads are fine).
    let still_v1 = store.load().await.expect("load ok").expect("present");
    assert_eq!(still_v1.version, 1);

    // Step 4: fail reads; load surfaces either Err or Ok(None).
    // Under the versioned layout, `load` uses `list` first and
    // tolerates a missing listing as "empty store"; backends that
    // model FailReads as a failed get (without failing list) return
    // Ok(None), while backends that fail the list stream return Err.
    // Both are legitimate read-fault behaviors.
    faulty.set_fault(ObjectStoreFault::FailReads);
    match store.load().await {
        Err(_) | Ok(None) => {}
        Ok(Some(s)) => panic!("expected read fault, got Ok(Some({s:?}))"),
    }

    // Step 5: heal; rotate v1 → v2, verify visible.
    faulty.set_fault(ObjectStoreFault::None);
    assert!(matches!(
        store
            .save_if_version(&v2, v1.version)
            .await
            .expect("rotate after heal"),
        laminar_core::cluster::control::RotateOutcome::Rotated,
    ));
    let loaded = store.load().await.unwrap().unwrap();
    assert_eq!(loaded.version, 2);
    assert_eq!(loaded.vnodes.get(&0), Some(&NodeId(2)));
}

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn assignment_snapshot_survives_full_cluster_restart() {
    // 1. Spin up a cluster backed by a shared on-disk object store.
    // 2. Leader writes an assignment snapshot via its controller.
    // 3. Shut the cluster down entirely (all nodes die).
    // 4. Spin a fresh cluster pointing at the same object-store dir.
    // 5. Verify the fresh leader reads the prior snapshot back
    //    unchanged.
    //
    // This is the cold-recovery contract from the design doc §7:
    // "the assignment snapshot lives on object store so full-cluster
    // restart doesn't lose state."
    let dir = tempfile::tempdir().expect("tempdir");
    let fs_a: Arc<dyn ObjectStore> =
        Arc::new(LocalFileSystem::new_with_prefix(dir.path()).unwrap());
    let store_a = Arc::new(AssignmentSnapshotStore::new(fs_a));

    // ── First cluster ────────────────────────────────────────────
    let cluster_a = MiniCluster::spawn_with_snapshot(3, Arc::clone(&store_a)).await;
    cluster_a
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("initial convergence");

    let leader = &cluster_a.nodes[0];
    assert!(leader.controller.is_leader());

    let mut vnodes = BTreeMap::new();
    vnodes.insert(0u32, NodeId(1));
    vnodes.insert(1u32, NodeId(2));
    vnodes.insert(2u32, NodeId(3));
    let snapshot = AssignmentSnapshot::empty().next(vnodes);

    leader
        .controller
        .snapshot_store()
        .expect("snapshot store installed")
        .save_if_absent(&snapshot)
        .await
        .expect("save");

    cluster_a.shutdown().await;

    // ── Fresh cluster against the same directory ─────────────────
    let fs_b: Arc<dyn ObjectStore> =
        Arc::new(LocalFileSystem::new_with_prefix(dir.path()).unwrap());
    let store_b = Arc::new(AssignmentSnapshotStore::new(fs_b));
    let cluster_b = MiniCluster::spawn_with_snapshot(3, Arc::clone(&store_b)).await;
    cluster_b
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("restart convergence");

    let leader_b = &cluster_b.nodes[0];
    assert!(leader_b.controller.is_leader());

    let loaded = leader_b
        .controller
        .snapshot_store()
        .expect("snapshot store installed")
        .load()
        .await
        .expect("load")
        .expect("snapshot present");

    assert_eq!(loaded.version, snapshot.version);
    assert_eq!(loaded.vnodes, snapshot.vnodes);

    cluster_b.shutdown().await;
}

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn fresh_node_can_join_running_cluster() {
    // Diagnostic: validates that join_node() itself works, distinct
    // from the rejoin-with-same-id case. If this passes but
    // killed_node_can_rejoin fails, the bug is specific to same-id
    // rejoin (chitchat generation collision, state reconciliation,
    // etc.).
    let mut cluster = MiniCluster::spawn(3).await;
    cluster
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("initial convergence");

    cluster.join_node(NodeId(99)).await;

    let start = Instant::now();
    loop {
        let all_see_four = cluster
            .nodes
            .iter()
            .all(|n| n.controller.live_instances().len() == 4);
        if all_see_four {
            break;
        }
        if start.elapsed() > FAILOVER_DEADLINE {
            let dumps: Vec<_> = cluster
                .nodes
                .iter()
                .map(|n| (n.instance_id, n.controller.live_instances()))
                .collect();
            panic!("fresh join didn't converge: {dumps:?}");
        }
        tokio::time::sleep(Duration::from_millis(100)).await;
    }
    cluster.shutdown().await;
}

// Same-id rejoin is a known-hard case with chitchat 0.10: after a
// peer is announced Left, its entry lingers in cluster state until
// phi-accrual fires + marked_for_deletion_grace_period elapses.
// A fresh instance with the same `node_id` string but a higher
// generation can race with the stale entry, and chitchat's state
// merge rules sometimes keep the Left record. Production flows
// around this by either (a) restarting with a distinct logical id
// (common in k8s pod rollovers — each pod gets a fresh UUID), or
// (b) waiting for explicit reaping. The `fresh_node_can_join_running_cluster`
// test above validates the common-case (a) path. Re-enable this
// test once we add an explicit reap primitive to the harness.
#[ignore = "known chitchat same-id rejoin limitation; see fresh_node_can_join_running_cluster for the (a)-path test"]
#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn killed_node_can_rejoin() {
    let mut cluster = MiniCluster::spawn(3).await;
    cluster
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("initial convergence");

    let killed_id = cluster.nodes[0].instance_id;
    let killed = cluster.nodes.remove(0);
    killed.kill().await;

    // Allow phi-accrual + GC of the Left record before rejoining.
    tokio::time::sleep(Duration::from_secs(5)).await;

    cluster.join_node(killed_id).await;
    let rejoined_idx = cluster
        .nodes
        .iter()
        .position(|n| n.instance_id == killed_id)
        .expect("rejoined node present");
    cluster.nodes.swap(0, rejoined_idx);

    let start = Instant::now();
    loop {
        let leaders: Vec<_> = cluster
            .nodes
            .iter()
            .map(|n| n.controller.current_leader())
            .collect();
        if leaders.iter().all(|l| *l == Some(killed_id)) {
            break;
        }
        if start.elapsed() > Duration::from_secs(15) {
            panic!("rejoined leader never surfaced: {leaders:?}");
        }
        tokio::time::sleep(Duration::from_millis(100)).await;
    }

    cluster.shutdown().await;
}

#[tokio::test(flavor = "multi_thread", worker_threads = 4)]
async fn quorum_times_out_when_follower_silent() {
    // Negative-path regression for the barrier protocol: if a
    // follower never acks, the leader's wait_for_quorum times out.
    // The checkpoint coordinator interprets this as "abort the epoch,
    // retry on the next tick".
    let cluster = MiniCluster::spawn(3).await;
    cluster
        .wait_for_convergence(CONVERGENCE_DEADLINE)
        .await
        .expect("convergence");

    let leader = &cluster.nodes[0];
    let silent = cluster.nodes[1].instance_id;
    let acker = &cluster.nodes[2];
    let expected_acks = vec![silent, acker.instance_id];

    leader
        .controller
        .announce_barrier(&BarrierAnnouncement {
            epoch: 42,
            checkpoint_id: 1,
            phase: laminar_core::cluster::control::Phase::Prepare,
            flags: 0,
            min_watermark_ms: None,
        })
        .await
        .unwrap();

    // Wait briefly for gossip to deliver the announcement, then have
    // only one follower ack. The other stays silent.
    tokio::time::sleep(Duration::from_millis(300)).await;
    acker
        .controller
        .ack_barrier(&BarrierAck {
            epoch: 42,
            ok: true,
            error: None,
            local_watermark_ms: None,
        })
        .await
        .unwrap();

    // Leader should time out waiting for the silent follower.
    let outcome = leader
        .controller
        .wait_for_quorum(42, &expected_acks, Duration::from_secs(2))
        .await;
    match outcome {
        QuorumOutcome::TimedOut { got, missing } => {
            assert_eq!(got, vec![acker.instance_id]);
            assert_eq!(missing, vec![silent]);
        }
        other => panic!("expected TimedOut, got {other:?}"),
    }

    cluster.shutdown().await;
}