datacake-cluster 0.1.0

Eventually consistent state replication as a library (consensus, RPC and conflict resolution) for building your own eventually consistent databases.
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
use std::borrow::Cow;
use std::collections::BTreeMap;
use std::marker::PhantomData;
use std::net::SocketAddr;
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use std::time::Duration;

use anyhow::anyhow;
use crossbeam_channel::{Receiver, Sender};
use crossbeam_utils::atomic::AtomicCell;
use datacake_crdt::{HLCTimestamp, StateChanges};
use puppet::ActorMailbox;
use tokio::sync::Semaphore;
use tokio::time::{interval, MissedTickBehavior};

use crate::keyspace::{
    Del,
    Diff,
    KeyspaceActor,
    KeyspaceGroup,
    KeyspaceTimestamps,
    MultiDel,
    MultiSet,
    READ_REPAIR_SOURCE_ID,
};
use crate::replication::{MembershipChanges, MAX_CONCURRENT_REQUESTS};
use crate::rpc::ReplicationClient;
use crate::storage::ProgressWatcher;
use crate::{Clock, ProgressTracker, PutContext, RpcNetwork, Storage};

const KEYSPACE_SYNC_TIMEOUT: Duration = if cfg!(test) {
    Duration::from_secs(1)
} else {
    Duration::from_secs(5)
};
const MAX_NUMBER_OF_DOCS_PER_FETCH: usize = 50_000;

/// The required context for the actor to run.
pub struct ReplicationCycleContext<S>
where
    S: Storage + Send + Sync + 'static,
{
    /// The time interval which should elapse between each tick.
    pub(crate) repair_interval: Duration,
    /// The ID of the local node.
    /// The cluster keyspace group.
    pub(crate) group: KeyspaceGroup<S>,
    /// The cluster RPC network.
    pub(crate) network: RpcNetwork,
}

impl<S> ReplicationCycleContext<S>
where
    S: Storage + Send + Sync + 'static,
{
    /// Gets the cluster clock.
    pub fn clock(&self) -> &Clock {
        self.group.clock()
    }
}

pub struct NodeChangeInfo {
    changes: Vec<KeyspaceDiff>,
}

#[derive(Clone)]
/// A handle for communicating with the replication cycle actor.
///
/// This handle is cheap to clone.
pub(crate) struct ReplicationHandle {
    tx: Sender<Op>,
    kill_switch: Arc<AtomicBool>,
}

impl ReplicationHandle {
    /// Marks that the cluster has had a membership change.
    pub(crate) fn membership_change(&self, changes: MembershipChanges) {
        let _ = self.tx.send(Op::MembershipChange(changes));
    }

    /// Kills the replication service.
    pub(crate) fn kill(&self) {
        self.kill_switch.store(true, Ordering::Relaxed);
    }
}

/// A enqueued event/operation for the cycle to handle next tick.
enum Op {
    MembershipChange(MembershipChanges),
}

/// Starts the replication cycle task.
///
/// This task is an intermittent task which checks to make sure current node is up to date
/// with what other nodes have seen.
///
/// In theory the difference should be minimal as the task distributor will create a higher
/// consistency unless a node goes down.
pub(crate) async fn start_replication_cycle<S>(
    ctx: ReplicationCycleContext<S>,
) -> ReplicationHandle
where
    S: Storage + Send + Sync + 'static,
{
    let kill_switch = Arc::new(AtomicBool::new(false));
    let (tx, rx) = crossbeam_channel::unbounded();

    tokio::spawn(replication_cycle(ctx, rx, kill_switch.clone()));

    ReplicationHandle { tx, kill_switch }
}

async fn replication_cycle<S>(
    ctx: ReplicationCycleContext<S>,
    rx: Receiver<Op>,
    kill_switch: Arc<AtomicBool>,
) where
    S: Storage + Send + Sync + 'static,
{
    let mut live_members = BTreeMap::new();
    let mut keyspace_tracker = KeyspaceTracker::default();

    let mut interval = interval(ctx.repair_interval);
    interval.set_missed_tick_behavior(MissedTickBehavior::Skip);
    loop {
        interval.tick().await;

        if kill_switch.load(Ordering::Relaxed) {
            break;
        }

        while let Ok(op) = rx.try_recv() {
            match op {
                Op::MembershipChange(changes) => {
                    for node_id in changes.left {
                        live_members.remove(node_id.as_ref());
                        keyspace_tracker.remove_node(node_id.as_ref());
                    }

                    for (node_id, addr) in changes.joined {
                        live_members.insert(node_id.to_string(), addr);
                    }
                },
            }
        }

        read_repair_members(&ctx, &live_members, &mut keyspace_tracker).await;
    }
}

#[derive(Default, Debug)]
struct KeyspaceTracker {
    inner: BTreeMap<String, KeyspaceTimestamps>,
}

impl KeyspaceTracker {
    fn remove_node(&mut self, node_id: &str) {
        self.inner.remove(node_id);
    }

    fn get_diff(
        &mut self,
        node_id: String,
        other: &KeyspaceTimestamps,
    ) -> impl Iterator<Item = Cow<'static, str>> {
        self.inner.entry(node_id).or_default().diff(other)
    }

    fn set_keyspace(&mut self, node_id: String, ts: HLCTimestamp) {
        self.inner
            .entry(node_id.clone())
            .or_default()
            .insert(Cow::Owned(node_id), Arc::new(AtomicCell::new(ts)));
    }
}

/// Polls all `live_members` and works out what entries are missing from the local node.
///
/// This will return a optimised plan of what nodes should have what documents retrieved.
async fn read_repair_members<S>(
    ctx: &ReplicationCycleContext<S>,
    live_members: &BTreeMap<String, SocketAddr>,
    keyspace_tracker: &mut KeyspaceTracker,
) where
    S: Storage + Send + Sync + 'static,
{
    for (node_id, addr) in live_members {
        let res =
            check_node_changes(ctx, node_id.clone(), *addr, keyspace_tracker).await;

        let info = match res {
            Err(e) => {
                error!(
                    error = ?e,
                    target_node_id = %node_id,
                    target_node_addr = %addr,
                    "Failed to poll node changes due to error.",
                );
                continue;
            },
            Ok(info) => info,
        };

        for change in info.changes {
            let res = begin_keyspace_sync(
                ctx,
                change.keyspace.clone(),
                node_id.to_string(),
                *addr,
                change.removed,
                change.modified,
            )
            .await;

            if let Err(e) = res {
                error!(
                    error = ?e,
                    keyspace = %change.keyspace,
                    target_node_id = %node_id,
                    target_node_addr = %addr,
                    "Failed to sync with node."
                );
            } else {
                keyspace_tracker.set_keyspace(node_id.to_string(), change.last_updated);
            }
        }
    }
}

/// Polls the remote node's keyspace timestamps.
///
/// If any timestamps are different to when the node was last polled, a task is created
/// for each keyspace which has changed.
///
/// If a task already exists for a given keyspace, it is checked to see if the task is complete
/// or not, if the task is complete then a new task is created, otherwise, if the task is *not*
/// complete but has taken longer than the allowed timeout period, the existing task is cancelled
/// and restarted.
async fn check_node_changes<S>(
    ctx: &ReplicationCycleContext<S>,
    target_node_id: String,
    target_node_addr: SocketAddr,
    keyspace_tracker: &mut KeyspaceTracker,
) -> Result<NodeChangeInfo, anyhow::Error>
where
    S: Storage + Send + Sync + 'static,
{
    info!(
        target_node_id = %target_node_id,
        target_node_addr = %target_node_addr,
        "Getting keyspace changes on remote node.",
    );

    let channel = ctx.network.get_or_connect(target_node_addr).await?;
    let mut client = ReplicationClient::new(ctx.clock().clone(), channel.clone());
    let keyspace_timestamps = client.poll_keyspace().await?;

    let diff = keyspace_tracker
        .get_diff(target_node_id.clone(), &keyspace_timestamps)
        .map(|ks| ks.to_string())
        .collect::<Vec<_>>();

    let permits = Arc::new(Semaphore::new(MAX_CONCURRENT_REQUESTS));
    let mut tasks = Vec::new();
    for keyspace in diff {
        let permits = permits.clone();
        let target_node_id = target_node_id.clone();
        let group = ctx.group.clone();
        let client = ReplicationClient::new(ctx.clock().clone(), channel.clone());

        let task = tokio::spawn(async move {
            let _permit = permits.acquire();
            get_keyspace_diff(
                keyspace,
                target_node_id.to_string(),
                target_node_addr,
                group,
                client,
            )
            .await
        });
        tasks.push(task);
    }

    let mut changes = Vec::new();
    for task in tasks {
        let diff = task.await?;
        match diff {
            Err(e) => {
                error!(
                    error = ?e.cause,
                    keyspace = %e.keyspace,
                    target_node_id = %e.node_id,
                    target_rpc_addr = %e.node_addr,
                    "Failed to get keyspace diff.",
                );
            },
            Ok(diff) => {
                changes.push(diff);
            },
        }
    }

    Ok(NodeChangeInfo { changes })
}

pub struct KeyspaceDiff {
    keyspace: String,
    modified: StateChanges,
    removed: StateChanges,
    last_updated: HLCTimestamp,
}

#[derive(Debug, thiserror::Error)]
#[error("RPC Error: {cause:?}")]
pub struct GetDiffError {
    cause: tonic::Status,
    keyspace: String,
    node_id: String,
    node_addr: SocketAddr,
}

#[instrument(
    name = "keyspace-diff",
    skip_all,
    fields(
        keyspace = %keyspace_name,
        target_node_id = %target_node_id,
        target_rpc_addr = %target_rpc_addr,
    )
)]
async fn get_keyspace_diff<S>(
    keyspace_name: String,
    target_node_id: String,
    target_rpc_addr: SocketAddr,
    group: KeyspaceGroup<S>,
    mut client: ReplicationClient,
) -> Result<KeyspaceDiff, GetDiffError>
where
    S: Storage + Send + Sync + 'static,
{
    let keyspace = group.get_or_create_keyspace(&keyspace_name).await;
    let (last_updated, set) =
        client
            .get_state(keyspace.name())
            .await
            .map_err(|e| GetDiffError {
                cause: e,
                keyspace: keyspace_name.clone(),
                node_id: target_node_id.clone(),
                node_addr: target_rpc_addr,
            })?;

    let (modified, removed) = keyspace.send(Diff(set)).await;

    Ok(KeyspaceDiff {
        keyspace: keyspace_name.clone(),
        modified,
        removed,
        last_updated,
    })
}

#[instrument(name = "sync-removed-docs", skip_all)]
/// Starts the synchronisation process of syncing the remote node's keyspace
/// to the current node's keyspace.
///
/// The system begins by requesting the keyspace CRDT and gets the diff between
/// the current CRDT and the remote CRDT.
async fn begin_keyspace_sync<S>(
    ctx: &ReplicationCycleContext<S>,
    keyspace_name: String,
    target_node_id: String,
    target_rpc_addr: SocketAddr,
    removed: StateChanges,
    modified: StateChanges,
) -> Result<(), anyhow::Error>
where
    S: Storage + Send + Sync + 'static,
{
    let channel = ctx.network.get_or_connect(target_rpc_addr).await?;
    let keyspace = ctx.group.get_or_create_keyspace(&keyspace_name).await;
    let client = ReplicationClient::new(ctx.clock().clone(), channel.clone());

    let progress_tracker = ProgressTracker::default();
    let ctx = PutContext {
        progress: progress_tracker.clone(),
        remote_node_id: Cow::Owned(target_node_id.clone()),
        remote_addr: target_rpc_addr,
        remote_rpc_channel: channel.clone(),
    };

    // The removal task can operate interdependently of the modified handler.
    // If, in the process of handling removals, the modified handler errors,
    // we simply let the removal task continue on as normal.
    let removal_task = tokio::spawn(handle_removals(keyspace.clone(), removed));

    let res = tokio::spawn(handle_modified(client, keyspace, modified, ctx));

    let mut watcher = ProgressWatcher::new(progress_tracker, KEYSPACE_SYNC_TIMEOUT);
    let mut interval = interval(Duration::from_millis(250));
    loop {
        interval.tick().await;

        if watcher.has_expired() {
            res.abort();
            removal_task.await??;
            return Err(anyhow!("Task timed out and could not be completed."));
        }

        if watcher.is_done() {
            break;
        }
    }

    removal_task.await??;

    Ok(())
}

#[instrument(name = "sync-modified-docs", skip_all)]
/// Fetches all the documents which have changed since the last state fetch.
///
/// These documents are then persisted and the metadata marked accordingly.
async fn handle_modified<S>(
    mut client: ReplicationClient,
    keyspace: ActorMailbox<KeyspaceActor<S>>,
    modified: StateChanges,
    ctx: PutContext,
) -> Result<(), anyhow::Error>
where
    S: Storage + Send + Sync + 'static,
{
    let doc_id_chunks = modified
        .chunks(MAX_NUMBER_OF_DOCS_PER_FETCH)
        .map(|entries| entries.iter().map(|(k, _)| *k).collect::<Vec<_>>());

    for doc_ids in doc_id_chunks {
        let docs = client
            .fetch_docs(keyspace.name(), doc_ids)
            .await?
            .collect::<Vec<_>>();

        let msg = MultiSet {
            source: READ_REPAIR_SOURCE_ID,
            docs,
            ctx: Some(ctx.clone()),
            _marker: PhantomData::<S>::default(),
        };
        keyspace.send(msg).await?;
    }

    Ok(())
}

#[instrument(name = "sync-removed-docs", skip_all)]
/// Removes the marked documents from the persisted storage and then
/// marks the document metadata as a tombstone.
///
/// This does not remove the metadata of the document entirely, instead the document is marked
/// as deleted along with the main data itself, but we keep a history of the deletes we've made.
async fn handle_removals<S>(
    keyspace: ActorMailbox<KeyspaceActor<S>>,
    mut removed: StateChanges,
) -> Result<(), anyhow::Error>
where
    S: Storage + Send + Sync + 'static,
{
    if removed.is_empty() {
        return Ok(());
    }

    if removed.len() == 1 {
        let (doc_id, ts) = removed.remove(0);
        let msg = Del {
            source: READ_REPAIR_SOURCE_ID,
            doc_id,
            ts,
            _marker: PhantomData::<S>::default(),
        };
        keyspace.send(msg).await?;
        return Ok(());
    }

    let msg = MultiDel {
        source: READ_REPAIR_SOURCE_ID,
        key_ts_pairs: removed,
        _marker: PhantomData::<S>::default(),
    };
    keyspace.send(msg).await?;
    Ok(())
}