d-engine-core 0.2.3

Pure Raft consensus algorithm - for building custom Raft-based systems
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
use std::time::Duration;

use bytes::Bytes;
use d_engine_proto::server::storage::SnapshotChunk;
use d_engine_proto::server::storage::SnapshotResponse;
use futures::StreamExt;
use futures::stream;
use futures::stream::BoxStream;
use tokio::sync::oneshot;
use tokio::time::Instant;
use tonic::Status;
use tracing::debug;

use super::*;
use crate::Error;
use crate::MockTypeConfig;
use crate::NetworkError;
use crate::SnapshotConfig;
use crate::SnapshotError;
use crate::StorageError;
use crate::create_test_chunk;
use crate::create_test_snapshot_stream;

/// Helper to create a valid snapshot stream
fn create_snapshot_stream(
    chunks: usize,
    chunk_size: usize,
) -> BoxStream<'static, Result<SnapshotChunk>> {
    let chunks: Vec<SnapshotChunk> = (0..chunks)
        .map(|seq| {
            let data = vec![seq as u8; chunk_size];
            create_test_chunk(
                seq as u32,
                &data,
                1, // term
                1, // leader_id
                chunks as u32,
            )
        })
        .collect();

    let stream = create_test_snapshot_stream(chunks);
    Box::pin(
        stream.map(|item| item.map_err(|s| NetworkError::TonicStatusError(Box::new(s)).into())),
    )
}

fn default_snapshot_config() -> SnapshotConfig {
    SnapshotConfig {
        max_bandwidth_mbps: 1,
        sender_yield_every_n_chunks: 2,
        transfer_timeout_in_sec: 1,
        push_timeout_in_ms: 100,
        ..Default::default()
    }
}

// Helper to create snapshot chunks for testing
fn create_snapshot_chunk(
    seq: u32,
    size: usize,
) -> SnapshotChunk {
    SnapshotChunk {
        leader_term: 1,
        leader_id: 1,
        seq,
        total_chunks: 1,
        chunk_checksum: Bytes::new(),
        metadata: if seq == 0 {
            Some(Default::default())
        } else {
            None
        },
        data: Bytes::from(vec![0; size]),
    }
}

#[cfg(test)]
mod run_push_transfer_test {
    use super::*;
    use crate::MockNode;
    use crate::SystemError;

    #[tokio::test]
    async fn test_push_transfer_success() {
        let config = default_snapshot_config();
        let stream = create_snapshot_stream(3, 1024); // 3 chunks of 1KB each

        // Start mock server
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Ok(SnapshotResponse {
                term: 1,
                success: true, // always succeed
                next_chunk: 1,
            }),
            shutdown_rx,
        )
        .await
        .unwrap();

        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1,
            Box::pin(stream),
            channel,
            config,
        )
        .await;

        debug!(?result);

        assert!(result.is_ok());
    }

    // Test: Transfer fails when stream returns an error
    #[tokio::test]
    async fn test_push_transfer_fails_on_stream_error() {
        let config = default_snapshot_config();
        let stream = create_snapshot_stream(2, 512);
        let error_stream = stream::once(async {
            Err(StorageError::IoError(std::io::Error::new(
                std::io::ErrorKind::BrokenPipe,
                "Test error",
            ))
            .into())
        });
        let combined_stream = stream.chain(error_stream).boxed();

        // Start mock server
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Err(Status::unavailable("Service is not ready")),
            shutdown_rx,
        )
        .await
        .unwrap();

        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1,
            combined_stream,
            channel,
            config,
        )
        .await;

        assert!(result.is_err());
    }

    // Test: Transfer respects bandwidth limit
    #[tokio::test]
    async fn test_push_transfer_respects_bandwidth_limit() {
        let mut config = default_snapshot_config();
        config.max_bandwidth_mbps = 1; // 1 MBps
        config.push_queue_size = 1;
        config.push_timeout_in_ms = 2500;

        let stream = create_snapshot_stream(3, 5 * 1024); // 2 chunks of 5KB each (1.25MB total)

        // Start mock server
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Ok(SnapshotResponse {
                term: 1,
                success: true, // always succeed
                next_chunk: 2,
            }),
            shutdown_rx,
        )
        .await
        .unwrap();

        let start = Instant::now();
        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1,
            Box::pin(stream),
            channel,
            config,
        )
        .await;

        debug!(?result);
        assert!(result.is_ok());
        let duration = start.elapsed();
        debug!(?duration);

        // Calculate expected minimum time:
        // Total data = 2 chunks * 5KB = 10KB = 80,000 bits
        // Bandwidth = 1 Mbps = 1,000,000 bps
        // Minimum time = 80,000 / 1,000,000 = 0.08 seconds
        assert!(duration >= Duration::from_micros(80));
    }

    // Test: First chunk validation fails when metadata is missing
    #[tokio::test]
    async fn test_push_transfer_fails_on_invalid_first_chunk_metadata() {
        let config = default_snapshot_config();

        // Create a stream with first chunk missing metadata
        let mut invalid_chunk = create_snapshot_chunk(0, 1024);
        invalid_chunk.metadata = None;
        let stream = stream::iter(vec![Ok(invalid_chunk)]).boxed();

        // Start mock server
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Ok(SnapshotResponse {
                term: 1,
                success: true,
                next_chunk: 1,
            }),
            shutdown_rx,
        )
        .await
        .unwrap();

        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1, stream, channel, config,
        )
        .await;

        assert!(result.is_err());
        assert!(matches!(
            result.unwrap_err(),
            crate::Error::Consensus(crate::ConsensusError::Snapshot(
                SnapshotError::InvalidFirstChunk
            ))
        ));
    }

    // Test: First chunk validation fails when sequence is not 0
    #[tokio::test]
    async fn test_push_transfer_fails_on_invalid_first_chunk_sequence() {
        let config = default_snapshot_config();

        // Create a stream with first chunk having wrong sequence
        let invalid_chunk = create_snapshot_chunk(1, 1024); // seq=1 instead of 0
        let stream = stream::iter(vec![Ok(invalid_chunk)]).boxed();

        // Start mock server
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Ok(SnapshotResponse {
                term: 1,
                success: true,
                next_chunk: 1,
            }),
            shutdown_rx,
        )
        .await
        .unwrap();

        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1, stream, channel, config,
        )
        .await;

        assert!(result.is_err());
        assert!(matches!(
            result.unwrap_err(),
            crate::Error::Consensus(crate::ConsensusError::Snapshot(
                SnapshotError::InvalidFirstChunk
            ))
        ));
    }

    // Test: Transfer fails when stream is empty
    #[tokio::test]
    async fn test_push_transfer_fails_on_empty_stream() {
        let config = default_snapshot_config();

        // Empty stream
        let stream = stream::iter(vec![]).boxed();

        // Start mock server
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Ok(SnapshotResponse {
                term: 1,
                success: true,
                next_chunk: 1,
            }),
            shutdown_rx,
        )
        .await
        .unwrap();

        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1, stream, channel, config,
        )
        .await;

        assert!(result.is_err());
        assert!(matches!(
            result.unwrap_err(),
            crate::Error::Consensus(crate::ConsensusError::Snapshot(
                SnapshotError::EmptySnapshot
            ))
        ));
    }

    // Test: Transfer fails when gRPC call returns error
    #[tokio::test]
    async fn test_push_transfer_fails_on_grpc_error() {
        let config = default_snapshot_config();
        let stream = create_snapshot_stream(3, 1024);

        // Start mock server that returns gRPC error
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Err(Status::internal("Internal server error")),
            shutdown_rx,
        )
        .await
        .unwrap();

        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1,
            Box::pin(stream),
            channel,
            config,
        )
        .await;

        assert!(result.is_err());
        debug!(?result);
        assert!(matches!(
            result.unwrap_err(),
            Error::System(SystemError::Network(NetworkError::TonicStatusError(_)))
        ));
    }

    // Test: Transfer fails when background task fails to send chunk
    #[tokio::test]
    async fn test_push_transfer_fails_on_background_task_error() {
        let mut config = default_snapshot_config();
        config.push_queue_size = 1; // Small queue to cause backpressure
        config.snapshot_push_backoff_in_ms = 0;
        config.snapshot_push_max_retry = 0;

        // Create a stream that will overflow the queue
        let chunks: Vec<Result<SnapshotChunk>> = (0..10)
            .map(|i| Ok(create_snapshot_chunk(i, 1024 * 1024))) // 1MB chunks
            .collect();
        let stream = stream::iter(chunks).boxed();

        // Start mock server that processes slowly
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Ok(SnapshotResponse {
                term: 1,
                success: true,
                next_chunk: 1,
            }),
            shutdown_rx,
        )
        .await
        .unwrap();

        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1, stream, channel, config,
        )
        .await;

        debug!(?result);

        assert!(result.is_err());
        assert!(matches!(
            result.unwrap_err(),
            crate::Error::Consensus(crate::ConsensusError::Snapshot(
                SnapshotError::Backpressure | SnapshotError::ReceiverDisconnected
            ))
        ));
    }

    // Test: Transfer fails when gRPC response indicates failure
    #[tokio::test]
    async fn test_push_transfer_fails_on_remote_rejection() {
        let mut config = default_snapshot_config();
        config.push_timeout_in_ms = 50;
        let stream = create_snapshot_stream(3, 1024);

        // Start mock server that returns failure response
        let (_shutdown_tx, shutdown_rx) = oneshot::channel();
        let (channel, _port) = MockNode::simulate_snapshot_mock_server(
            Ok(SnapshotResponse {
                term: 1,
                success: false, // rejection
                next_chunk: 1,
            }),
            shutdown_rx,
        )
        .await
        .unwrap();

        let result = BackgroundSnapshotTransfer::<MockTypeConfig>::run_push_transfer(
            1,
            Box::pin(stream),
            channel,
            config,
        )
        .await;

        debug!(?result);
        assert!(result.is_err());
        assert!(matches!(
            result.unwrap_err(),
            crate::Error::Consensus(crate::ConsensusError::Snapshot(
                SnapshotError::RemoteRejection
            ))
        ));
    }
}

// ... rest of the file remains unchanged ...