chitchat 0.10.1

Cluster membership library using gossip with Scuttlebutt reconciliation.
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
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
#![allow(clippy::type_complexity)]
#![allow(clippy::derive_partial_eq_without_eq)]

mod configuration;
mod delta;
mod digest;
mod failure_detector;
mod listener;
mod message;
pub(crate) mod serialize;
mod server;
mod state;
pub mod transport;
mod types;

use std::collections::{BTreeMap, HashMap, HashSet};
use std::iter::once;
use std::net::SocketAddr;
use std::num::NonZeroUsize;

use delta::Delta;
use failure_detector::FailureDetector;
pub use failure_detector::FailureDetectorConfig;
pub use listener::ListenerHandle;
pub use serialize::{Deserializable, Serializable};
use tokio::sync::watch;
use tokio_stream::wrappers::WatchStream;
use tracing::{error, info, warn};

pub use self::configuration::{CatchupCallback, ChitchatConfig};
pub use self::state::{ClusterStateSnapshot, NodeState};
use crate::digest::Digest;
pub use crate::message::ChitchatMessage;
pub use crate::server::{ChitchatHandle, spawn_chitchat};
use crate::state::ClusterState;
pub use crate::types::{ChitchatId, DeletionStatus, Heartbeat, Version, VersionedValue};

/// Maximum UDP datagram payload size (in bytes).
///
/// Note that 65KB typically won't fit in a single IP packet,
/// so long messages will be sent over several IP fragments of MTU size.
///
/// We pick a large payload size because at the moment because
/// we send the self digest "in full".
/// An Ethernet frame size of 1400B would limit us to 20 nodes
/// or so.
pub(crate) const MAX_UDP_DATAGRAM_PAYLOAD_SIZE: usize = 65_507;

/// To prevent dead nodes from being recorded again after deletion,
/// we keep a local memory of the last nodes that were garbage collected.
pub(crate) const GARBAGE_COLLECTED_NODE_HISTORY_SIZE: NonZeroUsize =
    NonZeroUsize::new(500).unwrap();

pub struct Chitchat {
    config: ChitchatConfig,
    cluster_state: ClusterState,
    failure_detector: FailureDetector,
    /// Notifies listeners when a change has occurred in the set of live nodes.
    previous_live_nodes: HashMap<ChitchatId, Version>,
    live_nodes_watcher_tx: watch::Sender<BTreeMap<ChitchatId, NodeState>>,
    live_nodes_watcher_rx: watch::Receiver<BTreeMap<ChitchatId, NodeState>>,
}

impl Chitchat {
    pub fn with_chitchat_id_and_seeds(
        config: ChitchatConfig,
        seed_addrs: watch::Receiver<HashSet<SocketAddr>>,
        initial_key_values: Vec<(String, String)>,
    ) -> Self {
        let failure_detector = FailureDetector::new(config.failure_detector_config.clone());
        let previous_live_nodes = HashMap::new();
        let (live_nodes_watcher_tx, live_nodes_watcher_rx) = watch::channel(BTreeMap::new());
        let mut chitchat = Chitchat {
            config,
            cluster_state: ClusterState::with_seed_addrs(seed_addrs),
            failure_detector,
            previous_live_nodes,
            live_nodes_watcher_tx,
            live_nodes_watcher_rx,
        };

        let self_node_state = chitchat.self_node_state();

        // Immediately mark the node as alive to ensure it responds to SYN messages.
        self_node_state.inc_heartbeat();

        // Set initial key/value pairs.
        for (key, value) in initial_key_values {
            self_node_state.set(key, value);
        }
        chitchat
    }

    pub(crate) fn create_syn_message(&self) -> ChitchatMessage {
        let scheduled_for_deletion: HashSet<_> = self.scheduled_for_deletion_nodes().collect();
        let digest = self.compute_digest(&scheduled_for_deletion);
        ChitchatMessage::Syn {
            cluster_id: self.config.cluster_id.clone(),
            digest,
        }
    }

    /// Digest contains important information about the list of members in
    /// the cluster.
    fn report_heartbeats_in_digest(&mut self, digest: &Digest) {
        for (chitchat_id, node_digest) in &digest.node_digests {
            self.report_heartbeat(chitchat_id, node_digest.heartbeat);
        }
    }

    fn process_delta(&mut self, delta: Delta) {
        let was_reset_triggered = self.cluster_state.apply_delta(delta);
        if was_reset_triggered {
            if let Some(catchup_callback) = &self.config.catchup_callback {
                info!("executing catch-up callback");
                catchup_callback();
            }
        }
    }

    pub(crate) fn process_message(&mut self, msg: ChitchatMessage) -> Option<ChitchatMessage> {
        self.update_self_heartbeat();

        match msg {
            ChitchatMessage::Syn { cluster_id, digest } => {
                if cluster_id != self.cluster_id() {
                    warn!(
                        our_cluster_id=%self.cluster_id(),
                        their_cluster_id=%cluster_id,
                        "received SYN message addressed to a different cluster"
                    );
                    return Some(ChitchatMessage::BadCluster);
                }
                self.report_heartbeats_in_digest(&digest);
                let scheduled_for_deletion: HashSet<_> =
                    self.scheduled_for_deletion_nodes().collect();
                let self_digest = self.compute_digest(&scheduled_for_deletion);
                let delta_mtu = MAX_UDP_DATAGRAM_PAYLOAD_SIZE - 1 - self_digest.serialized_len();
                let delta = self.cluster_state.compute_partial_delta_respecting_mtu(
                    &digest,
                    delta_mtu,
                    &scheduled_for_deletion,
                );
                Some(ChitchatMessage::SynAck {
                    digest: self_digest,
                    delta,
                })
            }
            ChitchatMessage::SynAck { digest, delta } => {
                self.report_heartbeats_in_digest(&digest);
                self.process_delta(delta);
                let scheduled_for_deletion =
                    self.scheduled_for_deletion_nodes().collect::<HashSet<_>>();
                let delta = self.cluster_state.compute_partial_delta_respecting_mtu(
                    &digest,
                    MAX_UDP_DATAGRAM_PAYLOAD_SIZE - 1,
                    &scheduled_for_deletion,
                );
                Some(ChitchatMessage::Ack { delta })
            }
            ChitchatMessage::Ack { delta } => {
                self.process_delta(delta);
                None
            }
            ChitchatMessage::BadCluster => {
                warn!("message rejected by peer: wrong cluster");
                None
            }
            #[cfg(test)]
            ChitchatMessage::PanicForTest => {
                panic!("panic message received");
            }
        }
    }

    fn gc_keys_marked_for_deletion(&mut self) {
        self.cluster_state
            .gc_keys_marked_for_deletion(self.config.marked_for_deletion_grace_period);
    }

    /// Reports heartbeats to the failure detector for nodes in the delta for which we received an
    /// update.
    fn report_heartbeat(&mut self, chitchat_id: &ChitchatId, heartbeat: Heartbeat) {
        if chitchat_id == self.self_chitchat_id() {
            return;
        }

        let should_init_if_absent = self
            .cluster_state
            .last_heartbeat_if_deleted(chitchat_id)
            .map(|last_heartbeat| last_heartbeat < heartbeat)
            .unwrap_or(true);

        let node_state = if should_init_if_absent {
            self.cluster_state.node_state_mut_or_init(chitchat_id)
        } else if let Some(node_state) = self.cluster_state.node_state_mut(chitchat_id) {
            node_state
        } else {
            return;
        };

        if node_state.try_set_heartbeat(heartbeat) {
            self.failure_detector.report_heartbeat(chitchat_id);
        }
    }

    /// Marks the node as dead or alive depending on the new phi values and updates the live nodes
    /// watcher accordingly.
    pub(crate) fn update_nodes_liveness(&mut self) {
        for chitchat_id in self.cluster_state.nodes() {
            if chitchat_id != self.self_chitchat_id() {
                self.failure_detector.update_node_liveness(chitchat_id);
            }
        }
        let current_live_nodes = self
            .live_nodes()
            .flat_map(|chitchat_id| {
                if let Some(node_state) = self.node_state(chitchat_id) {
                    return Some((chitchat_id.clone(), node_state.max_version()));
                }
                warn!("node state for {chitchat_id:?} is absent");
                None
            })
            .collect::<HashMap<_, _>>();

        if self.previous_live_nodes != current_live_nodes {
            let live_nodes = current_live_nodes
                .keys()
                .cloned()
                .flat_map(|chitchat_id| {
                    let node_state = self.node_state(&chitchat_id)?;
                    if let Some(liveness_extra_predicate) = &self.config.extra_liveness_predicate {
                        if !liveness_extra_predicate(node_state) {
                            return None;
                        }
                    }
                    Some((chitchat_id, node_state.clone()))
                })
                .collect::<BTreeMap<_, _>>();
            self.previous_live_nodes = current_live_nodes;

            if self.live_nodes_watcher_tx.send(live_nodes).is_err() {
                error!(current_node = ?self.self_chitchat_id(), "error while reporting membership change event.")
            }
        }
        // Perform garbage collection.
        let garbage_collected_nodes = self.failure_detector.garbage_collect();
        for chitchat_id in &garbage_collected_nodes {
            if chitchat_id != self.self_chitchat_id() {
                self.cluster_state.remove_node(chitchat_id);
            } else {
                error!("self node was marked dead, please report");
            }
        }
    }

    pub fn node_states(&self) -> &BTreeMap<ChitchatId, NodeState> {
        self.cluster_state.node_states()
    }

    pub fn node_state(&self, chitchat_id: &ChitchatId) -> Option<&NodeState> {
        self.cluster_state.node_state(chitchat_id)
    }

    pub fn self_node_state(&mut self) -> &mut NodeState {
        self.cluster_state
            .node_state_mut_or_init(&self.config.chitchat_id)
    }

    /// Returns the set of nodes considered alive by the failure detector. It includes the
    /// current node (also called "self node"), which is always considered alive.
    pub fn live_nodes(&self) -> impl Iterator<Item = &ChitchatId> {
        once(self.self_chitchat_id()).chain(self.failure_detector.live_nodes())
    }

    /// Returns a watch stream for monitoring changes in the cluster.
    ///
    /// The stream will emit a new value whenever a node:
    /// - joins the cluster
    /// - leaves the cluster
    /// - updates its max version
    ///
    /// Heartbeats are not notified.
    pub fn live_nodes_watch_stream(&self) -> WatchStream<BTreeMap<ChitchatId, NodeState>> {
        WatchStream::new(self.live_nodes_watcher_rx.clone())
    }

    pub fn live_nodes_watcher(&self) -> watch::Receiver<BTreeMap<ChitchatId, NodeState>> {
        self.live_nodes_watcher_rx.clone()
    }

    /// Returns the set of nodes considered dead by the failure detector.
    pub fn dead_nodes(&self) -> impl Iterator<Item = &ChitchatId> {
        self.failure_detector.dead_nodes()
    }

    /// Returns the set of nodes considered dead by the failure detector.
    pub fn scheduled_for_deletion_nodes(&self) -> impl Iterator<Item = &ChitchatId> {
        self.failure_detector.scheduled_for_deletion_nodes()
    }

    /// Returns the set of seed nodes.
    pub fn seed_nodes(&self) -> HashSet<SocketAddr> {
        self.cluster_state.seed_addrs()
    }

    pub fn cluster_id(&self) -> &str {
        &self.config.cluster_id
    }

    /// Returns the current node's Chitchat ID.
    pub fn self_chitchat_id(&self) -> &ChitchatId {
        &self.config.chitchat_id
    }

    /// Returns a serializable snapshot of the cluster state.
    pub fn state_snapshot(&self) -> ClusterStateSnapshot {
        ClusterStateSnapshot::from(&self.cluster_state)
    }

    /// Fast forward the entire node state.
    ///
    /// This method is meant to be called as a follow up to a
    /// [`ChitchatConfig::catchup_callback`], to communicate back the caught up
    /// state to Chitchat.
    ///
    /// Updated key-values will see their listeners called. The order of calls
    /// is arbitrary.
    ///
    /// Existing key-values that are not present in `key_values` will be deleted
    /// (not marked with a tombstone).
    ///
    /// A node state that doesn't exist will be created, except if it was
    /// recently deleted and garbage collected. We leave the responsibility of
    /// forcing the garbage collected node's recreation to the regular chitchat
    /// heartbeat protocol.
    pub fn reset_node_state_if_update(
        &mut self,
        chitchat_id: &ChitchatId,
        key_values: impl Iterator<Item = (String, VersionedValue)>,
        max_version: Version,
        last_gc_version: Version,
    ) {
        let should_init_if_absent = self
            .cluster_state
            .last_heartbeat_if_deleted(chitchat_id)
            .is_none();

        let node_state = if should_init_if_absent {
            self.cluster_state.node_state_mut_or_init(chitchat_id)
        } else if let Some(node_state) = self.cluster_state.node_state_mut(chitchat_id) {
            node_state
        } else {
            info!("skip reset_node_state because the node was recently garbage collected");
            return;
        };

        if node_state.max_version() >= max_version {
            info!("attempted to reset node, but node is already up to date");
            return;
        }

        if max_version < node_state.last_gc_version() {
            // It is possible to have gc_version > max_version when we are catching up.
            // If we reach this point, we probably went through a reset via chitchat gossip.
            //
            // Now we are trying to reset our state via grpc gossip, but the new state is already
            // be out of date.
            warn!(
                node_max_version = node_state.max_version(),
                node_last_gc_version = node_state.last_gc_version(),
                delta_max_version = max_version,
                delta_last_gc_version = last_gc_version,
                "attempted to reset node with an obsolete state"
            );
            return;
        }

        let monotonic_property_before = node_state.monotonic_property();

        // We make sure that the node is listed in the failure detector,
        // so that we won't forget to GC the state.
        //
        // We don't report the heartbeat however, to make sure that we
        // avoid identifying resetted node as alive.
        self.failure_detector
            .get_or_create_sampling_window(chitchat_id);

        // We don't want to call listeners for keys that are already up to date so we must do this
        // dance instead of clearing the node state and then setting the new values.
        let mut previous_keys: HashSet<String> = node_state
            .key_values_including_deleted()
            .map(|(key, _)| key.to_string())
            .collect();
        for (key, value) in key_values {
            previous_keys.remove(&key);
            node_state.set_versioned_value(key, value)
        }
        for key in previous_keys {
            node_state.remove_key_value_internal(&key);
        }
        node_state.set_last_gc_version(last_gc_version);

        let monotonic_property_after = node_state.monotonic_property();

        assert!(monotonic_property_after > monotonic_property_before);
    }

    pub(crate) fn update_self_heartbeat(&mut self) {
        self.self_node_state().inc_heartbeat();
    }

    pub(crate) fn cluster_state(&self) -> &ClusterState {
        &self.cluster_state
    }

    /// Computes the node's digest.
    fn compute_digest(&self, scheduled_for_deletion_nodes: &HashSet<&ChitchatId>) -> Digest {
        self.cluster_state
            .compute_digest(scheduled_for_deletion_nodes)
    }

    /// Subscribes a callback that will be called every time a key matching the supplied prefix
    /// is inserted or updated.
    ///
    /// Disclaimer:
    /// The callback is required to be as light as possible.
    /// In particular,
    /// - it should not access the cluster state (as it is locked at the moment of the execution of
    ///   the callback.
    /// - it should be fast: the callback is executed in an async context.
    ///
    /// The callback is called with a [`KeyChangeEvent`] that contains the key stripped of the
    /// prefix, the new value and the node that owns this key/value.
    ///
    /// Deleted keys are not notified.
    #[must_use]
    pub fn subscribe_event(
        &self,
        key_prefix: impl ToString,
        callback: impl Fn(KeyChangeEvent) + 'static + Send + Sync,
    ) -> ListenerHandle {
        self.cluster_state()
            .listeners
            .subscribe_event(key_prefix, callback)
    }
}

#[derive(Debug, Copy, Clone, Eq, PartialEq)]
pub struct KeyChangeEvent<'a> {
    /// The matching key without the prefix used to subscribe to the event.
    pub key: &'a str,
    /// The new value.
    pub value: &'a str,
    /// The node for which the event was triggered.
    pub node: &'a ChitchatId,
}

impl KeyChangeEvent<'_> {
    fn strip_key_prefix(&self, prefix: &str) -> Option<KeyChangeEvent> {
        let key_without_prefix = self.key.strip_prefix(prefix)?;
        Some(KeyChangeEvent {
            key: key_without_prefix,
            value: self.value,
            node: self.node,
        })
    }
}

#[cfg(test)]
mod tests {
    use std::ops::{Add, RangeInclusive};
    use std::sync::Arc;
    use std::sync::atomic::{AtomicUsize, Ordering};
    use std::time::Duration;

    use rand::RngExt;
    use rand::distr::Alphanumeric;
    use tokio::sync::Mutex;
    use tokio::time;
    use tokio_stream::StreamExt;

    use super::*;
    use crate::server::{ChitchatHandle, spawn_chitchat};
    use crate::transport::{ChannelTransport, Transport};

    const DEAD_NODE_GRACE_PERIOD: Duration = Duration::from_secs(20);

    fn run_chitchat_handshake(initiating_node: &mut Chitchat, peer_node: &mut Chitchat) {
        let syn_message = initiating_node.create_syn_message();
        let syn_ack_message = peer_node.process_message(syn_message).unwrap();
        let ack_message = initiating_node.process_message(syn_ack_message).unwrap();
        assert!(peer_node.process_message(ack_message).is_none());
    }

    /// Checks that all of the non-deleted key-values pairs are the same in
    /// lhs and rhs.
    ///
    /// This does NOT check for deleted KVs.
    #[track_caller]
    fn assert_cluster_state_eq(lhs: &NodeState, rhs: &NodeState) {
        assert_eq!(lhs.num_key_values(), rhs.num_key_values());
        for (key, value) in lhs.key_values() {
            assert_eq!(rhs.get(key), Some(value));
        }
    }

    #[track_caller]
    fn assert_nodes_sync(nodes: &[&Chitchat]) {
        let first_node_states = nodes[0].cluster_state.node_states();
        for other_node in nodes.iter().skip(1) {
            let node_states = other_node.cluster_state.node_states();
            assert_eq!(first_node_states.len(), node_states.len());
            for (key, value) in first_node_states {
                assert_cluster_state_eq(value, node_states.get(key).unwrap());
            }
        }
    }

    async fn start_node_with_config(
        transport: &dyn Transport,
        config: ChitchatConfig,
    ) -> ChitchatHandle {
        let initial_kvs: Vec<(String, String)> = Vec::new();
        spawn_chitchat(config, initial_kvs, transport)
            .await
            .unwrap()
    }

    async fn start_node(
        chitchat_id: ChitchatId,
        seeds: &[String],
        transport: &dyn Transport,
    ) -> ChitchatHandle {
        let config = ChitchatConfig {
            chitchat_id: chitchat_id.clone(),
            cluster_id: "default-cluster".to_string(),
            gossip_interval: Duration::from_millis(100),
            listen_addr: chitchat_id.gossip_advertise_addr,
            seed_nodes: seeds.to_vec(),
            failure_detector_config: FailureDetectorConfig {
                dead_node_grace_period: DEAD_NODE_GRACE_PERIOD,
                phi_threshold: 5.0,
                initial_interval: Duration::from_millis(100),
                ..Default::default()
            },
            marked_for_deletion_grace_period: Duration::from_secs(3_600),
            catchup_callback: None,
            extra_liveness_predicate: None,
        };
        start_node_with_config(transport, config).await
    }

    async fn setup_nodes(
        port_range: RangeInclusive<u16>,
        transport: &dyn Transport,
    ) -> Vec<ChitchatHandle> {
        let chitchat_ids: Vec<ChitchatId> = port_range.map(ChitchatId::for_local_test).collect();
        let node_without_seed = start_node(chitchat_ids[0].clone(), &[], transport).await;
        let mut chitchat_handlers: Vec<ChitchatHandle> = vec![node_without_seed];
        for chitchat_id in &chitchat_ids[1..] {
            let seeds = chitchat_ids
                .iter()
                .filter(|&peer_id| peer_id != chitchat_id)
                .map(|peer_id| peer_id.gossip_advertise_addr.to_string())
                .collect::<Vec<_>>();
            chitchat_handlers.push(start_node(chitchat_id.clone(), &seeds, transport).await);
        }
        chitchat_handlers
    }

    async fn shutdown_nodes(nodes: Vec<ChitchatHandle>) -> anyhow::Result<()> {
        for node in nodes {
            node.shutdown().await?;
        }
        Ok(())
    }

    async fn wait_for_chitchat_state(
        chitchat: Arc<Mutex<Chitchat>>,
        expected_nodes: &[ChitchatId],
    ) {
        let expected_nodes = expected_nodes.iter().collect::<HashSet<_>>();
        let mut live_nodes_watcher =
            chitchat
                .lock()
                .await
                .live_nodes_watch_stream()
                .skip_while(|live_nodes| {
                    if live_nodes.len() != expected_nodes.len() {
                        return true;
                    }
                    let live_nodes = live_nodes.keys().collect::<HashSet<_>>();
                    live_nodes != expected_nodes
                });
        tokio::time::timeout(Duration::from_secs(60), async move {
            live_nodes_watcher.next().await.unwrap();
        })
        .await
        .unwrap();
    }

    #[test]
    fn test_chitchat_handshake() {
        let node_config1 = ChitchatConfig::for_test(10_001);
        let empty_seeds = watch::channel(Default::default()).1;
        let mut node1 = Chitchat::with_chitchat_id_and_seeds(
            node_config1,
            empty_seeds.clone(),
            vec![
                ("key1a".to_string(), "1".to_string()),
                ("key2a".to_string(), "2".to_string()),
            ],
        );
        let node_config2 = ChitchatConfig::for_test(10_002);
        let mut node2 = Chitchat::with_chitchat_id_and_seeds(
            node_config2,
            empty_seeds,
            vec![
                ("key1b".to_string(), "1".to_string()),
                ("key2b".to_string(), "2".to_string()),
            ],
        );
        run_chitchat_handshake(&mut node1, &mut node2);
        assert_nodes_sync(&[&node1, &node2]);
        // useless handshake
        run_chitchat_handshake(&mut node1, &mut node2);
        assert_nodes_sync(&[&node1, &node2]);
        {
            let state1 = node1.self_node_state();
            state1.set("key1a", "3");
            state1.set("key1c", "4");
        }
        run_chitchat_handshake(&mut node1, &mut node2);
        assert_nodes_sync(&[&node1, &node2]);
    }

    #[test]
    fn test_chitchat_dead_node_liveness() {
        let node_config1 = ChitchatConfig::for_test(10_001);
        let empty_seeds = watch::channel(Default::default()).1;
        let mut node1 =
            Chitchat::with_chitchat_id_and_seeds(node_config1, empty_seeds.clone(), Vec::new());
        let chitchat_id = ChitchatId::for_local_test(10u16);
        node1.reset_node_state_if_update(&chitchat_id, std::iter::empty(), 10_000, 10u64);
        node1.report_heartbeat(&chitchat_id, Heartbeat(10_000u64));
        node1.report_heartbeat(&chitchat_id, Heartbeat(10_000u64));
        node1.update_nodes_liveness();
        let live_nodes: HashSet<&ChitchatId> = node1.live_nodes().collect();
        assert_eq!(live_nodes.len(), 1);
    }

    #[tokio::test]
    async fn test_chitchat_no_need_to_reset_if_last_gc_version_is_higher() {
        // This test checks what happens if a node is trailing behind too much,
        // needs a reset, and a single delta would:
        // - not increase its max version after reset.
        // - not even bring the state to a max_version >= last_gc_version
        let _ = tracing_subscriber::fmt::try_init();
        tokio::time::pause();
        let node_config1 = ChitchatConfig::for_test(10_001);
        let empty_seeds = watch::channel(Default::default()).1;
        let mut node1 =
            Chitchat::with_chitchat_id_and_seeds(node_config1, empty_seeds.clone(), vec![]);
        let node_config2 = ChitchatConfig::for_test(10_002);
        let mut node2 = Chitchat::with_chitchat_id_and_seeds(node_config2, empty_seeds, vec![]);
        // Because of compression, we need a lot of keys to reach the MTU.
        for i in 0..20_000 {
            let key = format!("k{}", i);
            node1.self_node_state().set(&key, "first_value");
        }
        for _ in 0..2 {
            run_chitchat_handshake(&mut node1, &mut node2);
        }

        assert_nodes_sync(&[&node1, &node2]);

        node1.self_node_state().delete("k1");

        // Advance time before triggering the GC of that deleted key
        tokio::time::advance(Duration::from_secs(3_600 * 3)).await;
        node1.gc_keys_marked_for_deletion();

        for _ in 0..2 {
            run_chitchat_handshake(&mut node1, &mut node2);
        }

        assert_nodes_sync(&[&node1, &node2]);
    }

    #[tokio::test]
    async fn test_live_node_channel() {
        let transport = ChannelTransport::with_mtu(MAX_UDP_DATAGRAM_PAYLOAD_SIZE);
        let nodes = setup_nodes(20001..=20005, &transport).await;
        let node2 = nodes.get(1).unwrap();
        let mut live_nodes_stream = node2.chitchat().lock().await.live_nodes_watch_stream();
        let live_members = loop {
            let live_nodes = live_nodes_stream.next().await.unwrap();
            if live_nodes.len() == 5 {
                break live_nodes;
            }
        };
        for node in &nodes {
            assert!(live_members.contains_key(node.chitchat_id()));
        }
        shutdown_nodes(nodes).await.unwrap();
    }

    #[tokio::test]
    async fn test_live_node_channel_with_extra_predicate() {
        let transport = ChannelTransport::with_mtu(MAX_UDP_DATAGRAM_PAYLOAD_SIZE);
        let chitchat_ids: Vec<ChitchatId> = (1..=3).map(ChitchatId::for_local_test).collect();
        let make_config = |chitchat_id: &ChitchatId| ChitchatConfig {
            chitchat_id: chitchat_id.clone(),
            cluster_id: "default-cluster".to_string(),
            gossip_interval: Duration::from_millis(100),
            listen_addr: chitchat_id.gossip_advertise_addr,
            seed_nodes: vec![chitchat_ids[0].gossip_advertise_addr.to_string()],
            failure_detector_config: FailureDetectorConfig {
                dead_node_grace_period: DEAD_NODE_GRACE_PERIOD,
                phi_threshold: 5.0,
                initial_interval: Duration::from_millis(100),
                ..Default::default()
            },
            marked_for_deletion_grace_period: Duration::from_secs(3_600),
            catchup_callback: None,
            extra_liveness_predicate: Some(Box::new(|node_state| {
                node_state.get("READY") == Some("true")
            })),
        };
        let mut nodes = Vec::new();
        for chitchat_id in &chitchat_ids {
            let config = make_config(chitchat_id);
            let chitchat_handle = start_node_with_config(&transport, config).await;
            nodes.push(chitchat_handle);
        }

        let mut num_live_nodes = 0;
        assert!(
            tokio::time::timeout(Duration::from_secs(1), async {
                let mut live_nodes_stream =
                    nodes[2].chitchat().lock().await.live_nodes_watch_stream();
                loop {
                    let live_nodes = live_nodes_stream.next().await.unwrap();
                    num_live_nodes = live_nodes.len();
                    if live_nodes.len() == 3 {
                        break live_nodes;
                    }
                }
            })
            .await
            .is_err()
        );
        assert_eq!(num_live_nodes, 0);

        nodes[0]
            .chitchat()
            .lock()
            .await
            .self_node_state()
            .set("READY", "true");
        nodes[1]
            .chitchat()
            .lock()
            .await
            .self_node_state()
            .set("READY", "true");
        nodes[2]
            .chitchat()
            .lock()
            .await
            .self_node_state()
            .set("READY", "true");

        let mut live_nodes_stream = nodes[2].chitchat().lock().await.live_nodes_watch_stream();
        let live_members = loop {
            let live_nodes = live_nodes_stream.next().await.unwrap();
            if live_nodes.len() == 3 {
                break live_nodes;
            }
        };
        for node in &nodes {
            assert!(live_members.contains_key(node.chitchat_id()));
        }

        nodes[0]
            .chitchat()
            .lock()
            .await
            .self_node_state()
            .delete("READY");

        let live_members = loop {
            let live_nodes = live_nodes_stream.next().await.unwrap();
            if live_nodes.len() == 2 {
                break live_nodes;
            }
        };
        assert!(live_members.contains_key(&chitchat_ids[1]));
        assert!(live_members.contains_key(&chitchat_ids[2]));

        nodes[1]
            .chitchat()
            .lock()
            .await
            .self_node_state()
            .set("READY", "false");

        let live_members = loop {
            let live_nodes = live_nodes_stream.next().await.unwrap();
            if live_nodes.len() == 1 {
                break live_nodes;
            }
        };

        assert!(live_members.contains_key(&chitchat_ids[2]));

        shutdown_nodes(nodes).await.unwrap();
    }

    #[tokio::test]
    async fn test_multiple_nodes() -> anyhow::Result<()> {
        let transport = ChannelTransport::with_mtu(MAX_UDP_DATAGRAM_PAYLOAD_SIZE);
        let nodes = setup_nodes(20001..=20005, &transport).await;

        let node2 = nodes.get(1).unwrap();
        assert_eq!(node2.chitchat_id().advertise_port(), 20002);
        wait_for_chitchat_state(
            node2.chitchat(),
            &[
                ChitchatId::for_local_test(20001),
                ChitchatId::for_local_test(20002),
                ChitchatId::for_local_test(20003),
                ChitchatId::for_local_test(20004),
                ChitchatId::for_local_test(20005),
            ],
        )
        .await;

        shutdown_nodes(nodes).await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_node_goes_from_live_to_down_to_live() -> anyhow::Result<()> {
        let transport = ChannelTransport::with_mtu(MAX_UDP_DATAGRAM_PAYLOAD_SIZE);
        let mut nodes = setup_nodes(30001..=30006, &transport).await;
        wait_for_chitchat_state(
            nodes[0].chitchat(),
            &[
                ChitchatId::for_local_test(30001),
                ChitchatId::for_local_test(30002),
                ChitchatId::for_local_test(30003),
                ChitchatId::for_local_test(30004),
                ChitchatId::for_local_test(30005),
                ChitchatId::for_local_test(30006),
            ],
        )
        .await;

        // Take node 3 down.
        let node3 = nodes.remove(2);
        assert_eq!(node3.chitchat_id().advertise_port(), 30003);
        node3.shutdown().await.unwrap();

        let node2 = nodes.get(1).unwrap();
        assert_eq!(node2.chitchat_id().advertise_port(), 30002);
        wait_for_chitchat_state(
            node2.chitchat(),
            &[
                ChitchatId::for_local_test(30001),
                ChitchatId::for_local_test(30002),
                ChitchatId::for_local_test(30004),
                ChitchatId::for_local_test(30005),
                ChitchatId::for_local_test(30006),
            ],
        )
        .await;

        // Restart node 3.
        let node_3 = ChitchatId::for_local_test(30003);
        nodes.push(
            start_node(
                node_3,
                &[ChitchatId::for_local_test(30_001)
                    .gossip_advertise_addr
                    .to_string()],
                &transport,
            )
            .await,
        );
        wait_for_chitchat_state(
            nodes[0].chitchat(),
            &[
                ChitchatId::for_local_test(30001),
                ChitchatId::for_local_test(30002),
                ChitchatId::for_local_test(30003),
                ChitchatId::for_local_test(30004),
                ChitchatId::for_local_test(30005),
                ChitchatId::for_local_test(30006),
            ],
        )
        .await;

        shutdown_nodes(nodes).await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_dead_node_kvs_are_gossiped_too_when_node_joins() -> anyhow::Result<()> {
        let transport = ChannelTransport::with_mtu(MAX_UDP_DATAGRAM_PAYLOAD_SIZE);
        // starting 2 nodes.
        let mut nodes = setup_nodes(40001..=40002, &transport).await;

        // Let's add a key to node1.
        let node1_id = {
            let node1 = nodes.first().unwrap();
            let node1_chitchat = node1.chitchat();
            node1_chitchat
                .lock()
                .await
                .self_node_state()
                .set("test_key", "test_val");
            node1.chitchat_id().clone()
        };

        {
            let node2 = nodes.get(1).unwrap();
            assert_eq!(node2.chitchat_id().advertise_port(), 40002);
            wait_for_chitchat_state(
                node2.chitchat(),
                &[
                    ChitchatId::for_local_test(40001),
                    ChitchatId::for_local_test(40002),
                ],
            )
            .await;
            let node2_chitchat = node2.chitchat();
            // We have received node1's key
            let value = node2_chitchat
                .lock()
                .await
                .node_state(&node1_id)
                .unwrap()
                .get("test_key")
                .unwrap()
                .to_string();
            assert_eq!(&value, "test_val");
        }

        // Take node 1 down.
        let node1 = nodes.remove(0);
        assert_eq!(node1.chitchat_id().advertise_port(), 40_001);
        node1.shutdown().await.unwrap();

        // Node 2 has detected that node 1 is missing.
        let node_id2 = {
            let node2 = nodes.first().unwrap();
            assert_eq!(node2.chitchat_id().advertise_port(), 40_002);
            wait_for_chitchat_state(node2.chitchat(), &[ChitchatId::for_local_test(40_002)]).await;
            node2.chitchat_id().clone()
        };

        // Restart node at localhost:40001 with new name
        let mut new_config = ChitchatConfig::for_test(40_001);
        new_config.chitchat_id.node_id = "new_node".to_string();
        let new_chitchat_id = new_config.chitchat_id.clone();
        let seed_addr = ChitchatId::for_local_test(40_002).gossip_advertise_addr;
        new_config.seed_nodes = vec![seed_addr.to_string()];
        let new_node_chitchat_handle = spawn_chitchat(new_config, Vec::new(), &transport)
            .await
            .unwrap();
        let new_node_chitchat = new_node_chitchat_handle.chitchat();
        wait_for_chitchat_state(
            new_node_chitchat.clone(),
            &[new_chitchat_id.clone(), node_id2.clone()],
        )
        .await;

        {
            let new_node_chitchat_guard = new_node_chitchat.lock().await;
            let test_val = new_node_chitchat_guard
                .node_state(&node1_id)
                .unwrap()
                .get("test_key")
                .unwrap();
            assert_eq!(test_val, "test_val");

            // Let's check that node1 is seen as dead.
            let dead_nodes: HashSet<&ChitchatId> = new_node_chitchat_guard.dead_nodes().collect();
            assert_eq!(dead_nodes.len(), 1);
            assert!(dead_nodes.contains(&node1_id));
        }

        nodes.push(new_node_chitchat_handle);
        shutdown_nodes(nodes).await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_network_partition_nodes() -> anyhow::Result<()> {
        let transport = ChannelTransport::with_mtu(MAX_UDP_DATAGRAM_PAYLOAD_SIZE);
        let port_range = 11_001u16..=11_006;
        let nodes = setup_nodes(port_range.clone(), &transport).await;

        // Check nodes know each other.
        for node in &nodes {
            let expected_peers: Vec<ChitchatId> = port_range
                .clone()
                .map(ChitchatId::for_local_test)
                .collect::<Vec<_>>();
            wait_for_chitchat_state(node.chitchat(), &expected_peers).await;
        }
        shutdown_nodes(nodes).await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_dead_node_garbage_collection() -> anyhow::Result<()> {
        let transport = ChannelTransport::with_mtu(MAX_UDP_DATAGRAM_PAYLOAD_SIZE);
        let mut nodes = setup_nodes(60001..=60006, &transport).await;
        let node2 = nodes.get(1).unwrap();
        assert_eq!(node2.chitchat_id().advertise_port(), 60002);
        wait_for_chitchat_state(
            node2.chitchat(),
            &[
                ChitchatId::for_local_test(60_001),
                ChitchatId::for_local_test(60_002),
                ChitchatId::for_local_test(60_003),
                ChitchatId::for_local_test(60_004),
                ChitchatId::for_local_test(60_005),
                ChitchatId::for_local_test(60_006),
            ],
        )
        .await;

        // Take node 3 down.
        let node3 = nodes.remove(2);
        assert_eq!(node3.chitchat_id().advertise_port(), 60003);
        node3.shutdown().await.unwrap();

        let node2 = nodes.get(1).unwrap();
        assert_eq!(node2.chitchat_id().advertise_port(), 60002);
        wait_for_chitchat_state(
            node2.chitchat(),
            &[
                ChitchatId::for_local_test(60_001),
                ChitchatId::for_local_test(60_002),
                ChitchatId::for_local_test(60_004),
                ChitchatId::for_local_test(60_005),
                ChitchatId::for_local_test(60_006),
            ],
        )
        .await;

        // Dead node should still be known to the cluster.
        let dead_chitchat_id = ChitchatId::for_local_test(60003);
        for node in &nodes {
            assert!(
                node.chitchat()
                    .lock()
                    .await
                    .node_state(&dead_chitchat_id)
                    .is_some()
            );
        }

        // Wait a bit more than `dead_node_grace_period` since all nodes will not
        // notice cluster change at the same time.
        let wait_for = DEAD_NODE_GRACE_PERIOD.add(Duration::from_secs(5));
        time::sleep(wait_for).await;

        // Dead node should no longer be known to the cluster.
        for node in &nodes {
            assert!(
                node.chitchat()
                    .lock()
                    .await
                    .node_state(&dead_chitchat_id)
                    .is_none()
            );
        }

        shutdown_nodes(nodes).await?;
        Ok(())
    }

    #[test]
    fn test_chitchat_listener() {
        let node_config1 = ChitchatConfig::for_test(10_001);
        let empty_seeds = watch::channel(Default::default()).1;
        let mut node1 = Chitchat::with_chitchat_id_and_seeds(
            node_config1,
            empty_seeds.clone(),
            vec![("self1:suffix1".to_string(), "hello1".to_string())],
        );
        let counter_self_key: Arc<AtomicUsize> = Default::default();
        let counter_other_key: Arc<AtomicUsize> = Default::default();

        let counter_self_key_clone = counter_self_key.clone();
        node1
            .subscribe_event("self1:", move |evt| {
                assert_eq!(evt.key, "suffix1");
                assert_eq!(evt.value, "updated");
                counter_self_key_clone.fetch_add(1, Ordering::SeqCst);
            })
            .forever();
        let counter_other_key_clone = counter_other_key.clone();
        node1
            .subscribe_event("other:", move |evt| {
                assert_eq!(evt.key, "suffix");
                assert_eq!(evt.value, "hello");
                counter_other_key_clone.fetch_add(1, Ordering::SeqCst);
            })
            .forever();

        let counter_self_key_clone = counter_self_key.clone();
        node1
            .subscribe_event("self2:", move |evt| {
                assert_eq!(evt.key, "suffix2");
                assert_eq!(evt.value, "hello2");
                counter_self_key_clone.fetch_add(1, Ordering::SeqCst);
            })
            .forever();

        let node_config2 = ChitchatConfig::for_test(10_002);
        let mut node2 = Chitchat::with_chitchat_id_and_seeds(
            node_config2,
            empty_seeds,
            vec![("other:suffix".to_string(), "hello".to_string())],
        );

        assert_eq!(counter_self_key.load(Ordering::SeqCst), 0);
        assert_eq!(counter_other_key.load(Ordering::SeqCst), 0);

        run_chitchat_handshake(&mut node1, &mut node2);
        assert_nodes_sync(&[&node1, &node2]);

        assert_eq!(counter_self_key.load(Ordering::SeqCst), 0);
        assert_eq!(counter_other_key.load(Ordering::SeqCst), 1);

        // useless handshake
        run_chitchat_handshake(&mut node1, &mut node2);

        assert_eq!(counter_self_key.load(Ordering::SeqCst), 0);
        assert_eq!(counter_other_key.load(Ordering::SeqCst), 1);

        node1.self_node_state().set("self2:suffix2", "hello2");
        assert_eq!(counter_self_key.load(Ordering::SeqCst), 1);
        assert_eq!(counter_other_key.load(Ordering::SeqCst), 1);

        run_chitchat_handshake(&mut node1, &mut node2);

        assert_eq!(counter_self_key.load(Ordering::SeqCst), 1);
        assert_eq!(counter_other_key.load(Ordering::SeqCst), 1);

        node1.self_node_state().set("self1:suffix1", "updated");
        assert_eq!(counter_self_key.load(Ordering::SeqCst), 2);

        node1.self_node_state().delete("self1:suffix1");
        node2.self_node_state().delete("other:suffix");

        run_chitchat_handshake(&mut node1, &mut node2);

        assert_eq!(counter_self_key.load(Ordering::SeqCst), 2);
        assert_eq!(counter_other_key.load(Ordering::SeqCst), 1);
    }

    #[tokio::test]
    async fn test_maybe_trigger_catchup_callback() {
        let catchup_callback_counter = Arc::new(AtomicUsize::new(0));
        let catchup_callback_counter_clone = catchup_callback_counter.clone();

        let mut config = ChitchatConfig::for_test(10_001);
        config.catchup_callback = Some(Box::new(move || {
            catchup_callback_counter_clone.fetch_add(1, Ordering::Release);
        }));
        let (_seed_addrs_rx, seed_addrs_tx) = watch::channel(Default::default());

        let cluster_id = config.cluster_id.clone();
        let mut node: Chitchat =
            Chitchat::with_chitchat_id_and_seeds(config, seed_addrs_tx, Vec::new());

        // We dismiss delta about nodes we don't know about yet.
        // Let's add our peer to be able to run this test.
        let mut digest = Digest::default();
        let peer_id = ChitchatId::for_local_test(10_002);
        digest.add_node(peer_id, Heartbeat::default(), 10, 30);
        let _ = node.process_message(ChitchatMessage::Syn { cluster_id, digest });

        let delta = Delta::default();
        node.process_delta(delta);

        let mut delta = Delta::default();
        let chitchat_id = ChitchatId::for_local_test(10_002);
        delta.add_node(
            chitchat_id.clone(),
            1000u64, // last gc version
            0u64,
        );
        delta.add_kv(&chitchat_id, "key", "value", 1, false);
        node.process_delta(delta);

        assert_eq!(catchup_callback_counter.load(Ordering::Acquire), 1);
    }

    #[tokio::test]
    async fn test_reset_node_state() {
        let config = ChitchatConfig::for_test(10_001);
        let (_seed_addrs_rx, seed_addrs_tx) = watch::channel(Default::default());
        let mut node = Chitchat::with_chitchat_id_and_seeds(config, seed_addrs_tx, Vec::new());

        let chitchat_id = ChitchatId::for_local_test(10_002);
        node.reset_node_state_if_update(
            &chitchat_id,
            [(
                "foo".to_string(),
                VersionedValue::new("bar".to_string(), 1, false),
            )]
            .into_iter(),
            1,
            1337,
        );
        node.failure_detector.contains_node(&chitchat_id);

        let node_state = node.cluster_state.node_state(&chitchat_id).unwrap();
        assert_eq!(node_state.num_key_values(), 1);
        assert_eq!(node_state.get("foo"), Some("bar"));
        assert_eq!(node_state.max_version(), 1);
        assert_eq!(node_state.last_gc_version(), 1337);

        let chitchat_id = ChitchatId::for_local_test(10_003);
        let node_state = node.cluster_state.node_state_mut_or_init(&chitchat_id);
        node_state.set("foo", "bar");
        node_state.set("qux", "baz");
        node_state.set("toto", "titi");

        node.reset_node_state_if_update(
            &chitchat_id,
            [
                (
                    "qux".to_string(),
                    VersionedValue::new("baz".to_string(), 2, false),
                ),
                (
                    "toto".to_string(),
                    VersionedValue::new("tutu".to_string(), 4, false),
                ),
            ]
            .into_iter(),
            4,
            1337,
        );
        let node_state = node.cluster_state.node_state(&chitchat_id).unwrap();
        assert_eq!(node_state.num_key_values(), 2);
        assert_eq!(node_state.get("qux"), Some("baz"));
        assert_eq!(node_state.get("toto"), Some("tutu"));
        assert_eq!(node_state.max_version(), 4);
        assert_eq!(node_state.last_gc_version(), 1337);

        let chitchat_id = ChitchatId::for_local_test(10_004);
        let node_state = node.cluster_state.node_state_mut_or_init(&chitchat_id);
        node_state.set("foo", "bar");
        node_state.set("qux", "baz");
        node_state.set("toto", "titi");

        node.reset_node_state_if_update(
            &chitchat_id,
            [
                (
                    "foo".to_string(),
                    VersionedValue::new("bar".to_string(), 1, false),
                ),
                (
                    "qux".to_string(),
                    VersionedValue::new("baz".to_string(), 2, false),
                ),
            ]
            .into_iter(),
            2,
            1337,
        );
        let node_state = node.cluster_state.node_state(&chitchat_id).unwrap();
        assert_eq!(node_state.num_key_values(), 3);
        assert_eq!(node_state.get("foo"), Some("bar"));
        assert_eq!(node_state.get("qux"), Some("baz"));
        assert_eq!(node_state.get("toto"), Some("titi"));
        assert_eq!(node_state.max_version(), 3);
    }

    #[tokio::test]
    async fn test_reset_garbage_collected_node_state() {
        tokio::time::pause();
        let mut config = ChitchatConfig::for_test(10_006);
        config.failure_detector_config.dead_node_grace_period = Duration::from_secs(4);
        let (_seed_addrs_rx, seed_addrs_tx) = watch::channel(Default::default());
        let mut node = Chitchat::with_chitchat_id_and_seeds(config, seed_addrs_tx, Vec::new());

        let chitchat_id = ChitchatId::for_local_test(10_007);
        let node_state = node.cluster_state.node_state_mut_or_init(&chitchat_id);
        node_state.set("foo", "bar");
        node_state.set("qux", "baz");
        node_state.set("toto", "titi");

        node.report_heartbeat(&chitchat_id, Heartbeat(1));
        tokio::time::advance(Duration::from_millis(50)).await;
        node.report_heartbeat(&chitchat_id, Heartbeat(2));
        tokio::time::advance(Duration::from_millis(50)).await;
        node.report_heartbeat(&chitchat_id, Heartbeat(3));
        node.update_nodes_liveness();
        assert!(
            node.live_nodes()
                .collect::<Vec<_>>()
                .contains(&&chitchat_id)
        );
        assert!(node.cluster_state.node_state(&chitchat_id).is_some());

        tokio::time::advance(Duration::from_secs(60)).await;
        node.update_nodes_liveness();
        assert!(
            node.dead_nodes()
                .collect::<Vec<_>>()
                .contains(&&chitchat_id)
        );
        assert!(node.cluster_state.node_state(&chitchat_id).is_some());

        tokio::time::advance(Duration::from_secs(5)).await;
        node.update_nodes_liveness();
        assert!(node.cluster_state.node_state(&chitchat_id).is_none());

        // resetting GCed node should not bring it back
        node.reset_node_state_if_update(
            &chitchat_id,
            [(
                "foo".to_string(),
                VersionedValue::new("bar".to_string(), 1, false),
            )]
            .into_iter(),
            2,
            1337,
        );
        assert!(node.cluster_state.node_state(&chitchat_id).is_none());
    }

    // There was a bug in process_message:
    // When node_states is large and the node receives a SYN with an empty digest,
    // the MTU delta was incorrectly computed based on the received empty digest,
    // whereas it should have used self_digest instead.
    //
    #[tokio::test]
    async fn test_process_syn() {
        // Prepare node
        let config = ChitchatConfig::for_test(10_006);

        fn id(i: usize) -> ChitchatId {
            ChitchatId {
                node_id: "a".to_string().repeat(1000),
                generation_id: i as u64,
                gossip_advertise_addr: SocketAddr::from(([127, 0, 0, 1], 10000u16 + i as u16)),
            }
        }

        fn random_string(len: usize) -> String {
            rand::rng()
                .sample_iter(&Alphanumeric)
                .take(len)
                .map(char::from)
                .collect()
        }

        let (_seed_addrs_rx, seed_addrs_tx) = watch::channel(Default::default());

        let mut node = Chitchat::with_chitchat_id_and_seeds(config, seed_addrs_tx, Vec::new());

        // Add node states that form the digest with a serialized size close to the maximum MTU.

        let mut digest = Digest::default();
        let mut delta = Delta::default();
        for i in 0..55 {
            digest.add_node(id(i), Heartbeat(1), 0, 0);
            delta.add_node(id(i), 0, 0);
            delta.add_kv(&id(i), "key", &random_string(1000), 1, false);
        }
        node.report_heartbeats_in_digest(&digest);
        node.process_delta(delta);

        // Process a SYN message with an empty foreign digest

        let ack = node
            .process_message(ChitchatMessage::Syn {
                cluster_id: node.config.cluster_id.clone(),
                digest: Digest::default(),
            })
            .unwrap();

        // Verify that the serialized reply fits within the max MTU.

        let mut buf = Vec::new();
        ack.serialize(&mut buf);
        assert!(buf.len() < MAX_UDP_DATAGRAM_PAYLOAD_SIZE);
        let ChitchatMessage::SynAck { delta, .. } = ack else {
            panic!("Expected SynAck, got {:?}", ack);
        };
        assert_eq!(delta.node_deltas.len(), 4);
    }
}