-
Notifications
You must be signed in to change notification settings - Fork 400
/
storage.rs
813 lines (724 loc) · 27.7 KB
/
storage.rs
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
//! Represents the storage trait and example implementation.
//!
//! The storage trait is used to house and eventually serialize the state of the system.
//! Custom implementations of this are normal and this is likely to be a key integration
//! point for your distributed storage.
// Copyright 2019 TiKV Project Authors. Licensed under Apache-2.0.
// Copyright 2015 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::cmp;
use std::sync::{Arc, RwLock, RwLockReadGuard, RwLockWriteGuard};
use crate::eraftpb::*;
use crate::errors::{Error, Result, StorageError};
use crate::util::limit_size;
use getset::{Getters, Setters};
/// Holds both the hard state (commit index, vote leader, term) and the configuration state
/// (Current node IDs)
#[derive(Debug, Clone, Default, Getters, Setters)]
pub struct RaftState {
/// Contains the last meta information including commit index, the vote leader, and the vote term.
pub hard_state: HardState,
/// Records the current node IDs like `[1, 2, 3]` in the cluster. Every Raft node must have a
/// unique ID in the cluster;
pub conf_state: ConfState,
}
impl RaftState {
/// Create a new RaftState.
pub fn new(hard_state: HardState, conf_state: ConfState) -> RaftState {
RaftState {
hard_state,
conf_state,
}
}
/// Indicates the `RaftState` is initialized or not.
pub fn initialized(&self) -> bool {
self.conf_state != ConfState::default()
}
}
/// Records the context of the caller who calls entries() of Storage trait.
#[derive(Debug, Clone, Copy)]
pub struct GetEntriesContext(pub(crate) GetEntriesFor);
impl GetEntriesContext {
/// Used for callers out of raft. Caller can customize if it supports async.
pub fn empty(can_async: bool) -> Self {
GetEntriesContext(GetEntriesFor::Empty(can_async))
}
/// Check if the caller's context support fetching entries asynchronously.
pub fn can_async(&self) -> bool {
match self.0 {
GetEntriesFor::SendAppend { .. } => true,
GetEntriesFor::Empty(can_async) => can_async,
_ => false,
}
}
}
#[derive(Debug, Clone, Copy)]
pub(crate) enum GetEntriesFor {
// for sending entries to followers
SendAppend {
/// the peer id to which the entries are going to send
to: u64,
/// the term when the request is issued
term: u64,
/// whether to exhaust all the entries
aggressively: bool,
},
// for getting committed entries in a ready
GenReady,
// for getting entries to check pending conf when transferring leader
TransferLeader,
// for getting entries to check pending conf when forwarding commit index by vote messages
CommitByVote,
// It's not called by the raft itself
Empty(bool),
}
/// Storage saves all the information about the current Raft implementation, including Raft Log,
/// commit index, the leader to vote for, etc.
///
/// If any Storage method returns an error, the raft instance will
/// become inoperable and refuse to participate in elections; the
/// application is responsible for cleanup and recovery in this case.
pub trait Storage {
/// `initial_state` is called when Raft is initialized. This interface will return a `RaftState`
/// which contains `HardState` and `ConfState`.
///
/// `RaftState` could be initialized or not. If it's initialized it means the `Storage` is
/// created with a configuration, and its last index and term should be greater than 0.
fn initial_state(&self) -> Result<RaftState>;
/// Returns a slice of log entries in the range `[low, high)`.
/// max_size limits the total size of the log entries returned if not `None`, however
/// the slice of entries returned will always have length at least 1 if entries are
/// found in the range.
///
/// Entries are supported to be fetched asynchronously depending on the context. Async is optional.
/// Storage should check context.can_async() first and decide whether to fetch entries asynchronously
/// based on its own implementation. If the entries are fetched asynchronously, storage should return
/// LogTemporarilyUnavailable, and application needs to call `on_entries_fetched(context)` to trigger
/// re-fetch of the entries after the storage finishes fetching the entries.
///
/// # Panics
///
/// Panics if `high` is higher than `Storage::last_index(&self) + 1`.
fn entries(
&self,
low: u64,
high: u64,
max_size: impl Into<Option<u64>>,
context: GetEntriesContext,
) -> Result<Vec<Entry>>;
/// Returns the term of entry idx, which must be in the range
/// [first_index()-1, last_index()]. The term of the entry before
/// first_index is retained for matching purpose even though the
/// rest of that entry may not be available.
fn term(&self, idx: u64) -> Result<u64>;
/// Returns the index of the first log entry that is possible available via entries, which will
/// always equal to `truncated index` plus 1.
///
/// New created (but not initialized) `Storage` can be considered as truncated at 0 so that 1
/// will be returned in this case.
fn first_index(&self) -> Result<u64>;
/// The index of the last entry replicated in the `Storage`.
fn last_index(&self) -> Result<u64>;
/// Returns the most recent snapshot.
///
/// If snapshot is temporarily unavailable, it should return SnapshotTemporarilyUnavailable,
/// so raft state machine could know that Storage needs some time to prepare
/// snapshot and call snapshot later.
/// A snapshot's index must not less than the `request_index`.
/// `to` indicates which peer is requesting the snapshot.
fn snapshot(&self, request_index: u64, to: u64) -> Result<Snapshot>;
}
/// The Memory Storage Core instance holds the actual state of the storage struct. To access this
/// value, use the `rl` and `wl` functions on the main MemStorage implementation.
#[derive(Default)]
pub struct MemStorageCore {
raft_state: RaftState,
// entries[i] has raft log position i+snapshot.get_metadata().index
entries: Vec<Entry>,
// Metadata of the last snapshot received.
snapshot_metadata: SnapshotMetadata,
// If it is true, the next snapshot will return a
// SnapshotTemporarilyUnavailable error.
trigger_snap_unavailable: bool,
// Peers that are fetching entries asynchronously.
trigger_log_unavailable: bool,
// Stores get entries context.
get_entries_context: Option<GetEntriesContext>,
}
impl MemStorageCore {
/// Saves the current HardState.
pub fn set_hardstate(&mut self, hs: HardState) {
self.raft_state.hard_state = hs;
}
/// Get the hard state.
pub fn hard_state(&self) -> &HardState {
&self.raft_state.hard_state
}
/// Get the mut hard state.
pub fn mut_hard_state(&mut self) -> &mut HardState {
&mut self.raft_state.hard_state
}
/// Commit to an index.
///
/// # Panics
///
/// Panics if there is no such entry in raft logs.
pub fn commit_to(&mut self, index: u64) -> Result<()> {
assert!(
self.has_entry_at(index),
"commit_to {} but the entry does not exist",
index
);
let diff = (index - self.entries[0].index) as usize;
self.raft_state.hard_state.commit = index;
self.raft_state.hard_state.term = self.entries[diff].term;
Ok(())
}
/// Saves the current conf state.
pub fn set_conf_state(&mut self, cs: ConfState) {
self.raft_state.conf_state = cs;
}
#[inline]
fn has_entry_at(&self, index: u64) -> bool {
!self.entries.is_empty() && index >= self.first_index() && index <= self.last_index()
}
fn first_index(&self) -> u64 {
match self.entries.first() {
Some(e) => e.index,
None => self.snapshot_metadata.index + 1,
}
}
fn last_index(&self) -> u64 {
match self.entries.last() {
Some(e) => e.index,
None => self.snapshot_metadata.index,
}
}
/// Overwrites the contents of this Storage object with those of the given snapshot.
///
/// # Panics
///
/// Panics if the snapshot index is less than the storage's first index.
pub fn apply_snapshot(&mut self, mut snapshot: Snapshot) -> Result<()> {
let mut meta = snapshot.take_metadata();
let index = meta.index;
if self.first_index() > index {
return Err(Error::Store(StorageError::SnapshotOutOfDate));
}
self.snapshot_metadata = meta.clone();
self.raft_state.hard_state.term = cmp::max(self.raft_state.hard_state.term, meta.term);
self.raft_state.hard_state.commit = index;
self.entries.clear();
// Update conf states.
self.raft_state.conf_state = meta.take_conf_state();
Ok(())
}
fn snapshot(&self) -> Snapshot {
let mut snapshot = Snapshot::default();
// We assume all entries whose indexes are less than `hard_state.commit`
// have been applied, so use the latest commit index to construct the snapshot.
// TODO: This is not true for async ready.
let meta = snapshot.mut_metadata();
meta.index = self.raft_state.hard_state.commit;
meta.term = match meta.index.cmp(&self.snapshot_metadata.index) {
cmp::Ordering::Equal => self.snapshot_metadata.term,
cmp::Ordering::Greater => {
let offset = self.entries[0].index;
self.entries[(meta.index - offset) as usize].term
}
cmp::Ordering::Less => {
panic!(
"commit {} < snapshot_metadata.index {}",
meta.index, self.snapshot_metadata.index
);
}
};
meta.set_conf_state(self.raft_state.conf_state.clone());
snapshot
}
/// Discards all log entries prior to compact_index.
/// It is the application's responsibility to not attempt to compact an index
/// greater than RaftLog.applied.
///
/// # Panics
///
/// Panics if `compact_index` is higher than `Storage::last_index(&self) + 1`.
pub fn compact(&mut self, compact_index: u64) -> Result<()> {
if compact_index <= self.first_index() {
// Don't need to treat this case as an error.
return Ok(());
}
if compact_index > self.last_index() + 1 {
panic!(
"compact not received raft logs: {}, last index: {}",
compact_index,
self.last_index()
);
}
if let Some(entry) = self.entries.first() {
let offset = compact_index - entry.index;
self.entries.drain(..offset as usize);
}
Ok(())
}
/// Append the new entries to storage.
///
/// # Panics
///
/// Panics if `ents` contains compacted entries, or there's a gap between `ents` and the last
/// received entry in the storage.
pub fn append(&mut self, ents: &[Entry]) -> Result<()> {
if ents.is_empty() {
return Ok(());
}
if self.first_index() > ents[0].index {
panic!(
"overwrite compacted raft logs, compacted: {}, append: {}",
self.first_index() - 1,
ents[0].index,
);
}
if self.last_index() + 1 < ents[0].index {
panic!(
"raft logs should be continuous, last index: {}, new appended: {}",
self.last_index(),
ents[0].index,
);
}
// Remove all entries overwritten by `ents`.
let diff = ents[0].index - self.first_index();
self.entries.drain(diff as usize..);
self.entries.extend_from_slice(ents);
Ok(())
}
/// Commit to `idx` and set configuration to the given states. Only used for tests.
pub fn commit_to_and_set_conf_states(&mut self, idx: u64, cs: Option<ConfState>) -> Result<()> {
self.commit_to(idx)?;
if let Some(cs) = cs {
self.raft_state.conf_state = cs;
}
Ok(())
}
/// Trigger a SnapshotTemporarilyUnavailable error.
pub fn trigger_snap_unavailable(&mut self) {
self.trigger_snap_unavailable = true;
}
/// Set a LogTemporarilyUnavailable error.
pub fn trigger_log_unavailable(&mut self, v: bool) {
self.trigger_log_unavailable = v;
}
/// Take get entries context.
pub fn take_get_entries_context(&mut self) -> Option<GetEntriesContext> {
self.get_entries_context.take()
}
}
/// `MemStorage` is a thread-safe but incomplete implementation of `Storage`, mainly for tests.
///
/// A real `Storage` should save both raft logs and applied data. However `MemStorage` only
/// contains raft logs. So you can call `MemStorage::append` to persist new received unstable raft
/// logs and then access them with `Storage` APIs. The only exception is `Storage::snapshot`. There
/// is no data in `Snapshot` returned by `MemStorage::snapshot` because applied data is not stored
/// in `MemStorage`.
#[derive(Clone, Default)]
pub struct MemStorage {
core: Arc<RwLock<MemStorageCore>>,
}
impl MemStorage {
/// Returns a new memory storage value.
pub fn new() -> MemStorage {
MemStorage {
..Default::default()
}
}
/// Create a new `MemStorage` with a given `Config`. The given `Config` will be used to
/// initialize the storage.
///
/// You should use the same input to initialize all nodes.
pub fn new_with_conf_state<T>(conf_state: T) -> MemStorage
where
ConfState: From<T>,
{
let store = MemStorage::new();
store.initialize_with_conf_state(conf_state);
store
}
/// Initialize a `MemStorage` with a given `Config`.
///
/// You should use the same input to initialize all nodes.
pub fn initialize_with_conf_state<T>(&self, conf_state: T)
where
ConfState: From<T>,
{
assert!(!self.initial_state().unwrap().initialized());
let mut core = self.wl();
// Setting initial state is very important to build a correct raft, as raft algorithm
// itself only guarantees logs consistency. Typically, you need to ensure either all start
// states are the same on all nodes, or new nodes always catch up logs by snapshot first.
//
// In practice, we choose the second way by assigning non-zero index to first index. Here
// we choose the first way for historical reason and easier to write tests.
core.raft_state.conf_state = ConfState::from(conf_state);
}
/// Opens up a read lock on the storage and returns a guard handle. Use this
/// with functions that don't require mutation.
pub fn rl(&self) -> RwLockReadGuard<'_, MemStorageCore> {
self.core.read().unwrap()
}
/// Opens up a write lock on the storage and returns guard handle. Use this
/// with functions that take a mutable reference to self.
pub fn wl(&self) -> RwLockWriteGuard<'_, MemStorageCore> {
self.core.write().unwrap()
}
}
impl Storage for MemStorage {
/// Implements the Storage trait.
fn initial_state(&self) -> Result<RaftState> {
Ok(self.rl().raft_state.clone())
}
/// Implements the Storage trait.
fn entries(
&self,
low: u64,
high: u64,
max_size: impl Into<Option<u64>>,
context: GetEntriesContext,
) -> Result<Vec<Entry>> {
let max_size = max_size.into();
let mut core = self.wl();
if low < core.first_index() {
return Err(Error::Store(StorageError::Compacted));
}
if high > core.last_index() + 1 {
panic!(
"index out of bound (last: {}, high: {})",
core.last_index() + 1,
high
);
}
if core.trigger_log_unavailable && context.can_async() {
core.get_entries_context = Some(context);
return Err(Error::Store(StorageError::LogTemporarilyUnavailable));
}
let offset = core.entries[0].index;
let lo = (low - offset) as usize;
let hi = (high - offset) as usize;
let mut ents = core.entries[lo..hi].to_vec();
limit_size(&mut ents, max_size);
Ok(ents)
}
/// Implements the Storage trait.
fn term(&self, idx: u64) -> Result<u64> {
let core = self.rl();
if idx == core.snapshot_metadata.index {
return Ok(core.snapshot_metadata.term);
}
let offset = core.first_index();
if idx < offset {
return Err(Error::Store(StorageError::Compacted));
}
if idx > core.last_index() {
return Err(Error::Store(StorageError::Unavailable));
}
Ok(core.entries[(idx - offset) as usize].term)
}
/// Implements the Storage trait.
fn first_index(&self) -> Result<u64> {
Ok(self.rl().first_index())
}
/// Implements the Storage trait.
fn last_index(&self) -> Result<u64> {
Ok(self.rl().last_index())
}
/// Implements the Storage trait.
fn snapshot(&self, request_index: u64, _to: u64) -> Result<Snapshot> {
let mut core = self.wl();
if core.trigger_snap_unavailable {
core.trigger_snap_unavailable = false;
Err(Error::Store(StorageError::SnapshotTemporarilyUnavailable))
} else {
let mut snap = core.snapshot();
if snap.get_metadata().index < request_index {
snap.mut_metadata().index = request_index;
}
Ok(snap)
}
}
}
#[cfg(test)]
mod test {
use std::panic::{self, AssertUnwindSafe};
use protobuf::Message as PbMessage;
use crate::eraftpb::{ConfState, Entry, Snapshot};
use crate::errors::{Error as RaftError, StorageError};
use super::{GetEntriesContext, MemStorage, Storage};
fn new_entry(index: u64, term: u64) -> Entry {
let mut e = Entry::default();
e.term = term;
e.index = index;
e
}
fn size_of<T: PbMessage>(m: &T) -> u32 {
m.compute_size()
}
fn new_snapshot(index: u64, term: u64, voters: Vec<u64>) -> Snapshot {
let mut s = Snapshot::default();
s.mut_metadata().index = index;
s.mut_metadata().term = term;
s.mut_metadata().mut_conf_state().voters = voters;
s
}
#[test]
fn test_storage_term() {
let ents = vec![new_entry(3, 3), new_entry(4, 4), new_entry(5, 5)];
let mut tests = vec![
(2, Err(RaftError::Store(StorageError::Compacted))),
(3, Ok(3)),
(4, Ok(4)),
(5, Ok(5)),
(6, Err(RaftError::Store(StorageError::Unavailable))),
];
for (i, (idx, wterm)) in tests.drain(..).enumerate() {
let storage = MemStorage::new();
storage.wl().entries = ents.clone();
let t = storage.term(idx);
if t != wterm {
panic!("#{}: expect res {:?}, got {:?}", i, wterm, t);
}
}
}
#[test]
fn test_storage_entries() {
let ents = vec![
new_entry(3, 3),
new_entry(4, 4),
new_entry(5, 5),
new_entry(6, 6),
];
let max_u64 = u64::max_value();
let mut tests = vec![
(
2,
6,
max_u64,
Err(RaftError::Store(StorageError::Compacted)),
),
(3, 4, max_u64, Ok(vec![new_entry(3, 3)])),
(4, 5, max_u64, Ok(vec![new_entry(4, 4)])),
(4, 6, max_u64, Ok(vec![new_entry(4, 4), new_entry(5, 5)])),
(
4,
7,
max_u64,
Ok(vec![new_entry(4, 4), new_entry(5, 5), new_entry(6, 6)]),
),
// even if maxsize is zero, the first entry should be returned
(4, 7, 0, Ok(vec![new_entry(4, 4)])),
// limit to 2
(
4,
7,
u64::from(size_of(&ents[1]) + size_of(&ents[2])),
Ok(vec![new_entry(4, 4), new_entry(5, 5)]),
),
(
4,
7,
u64::from(size_of(&ents[1]) + size_of(&ents[2]) + size_of(&ents[3]) / 2),
Ok(vec![new_entry(4, 4), new_entry(5, 5)]),
),
(
4,
7,
u64::from(size_of(&ents[1]) + size_of(&ents[2]) + size_of(&ents[3]) - 1),
Ok(vec![new_entry(4, 4), new_entry(5, 5)]),
),
// all
(
4,
7,
u64::from(size_of(&ents[1]) + size_of(&ents[2]) + size_of(&ents[3])),
Ok(vec![new_entry(4, 4), new_entry(5, 5), new_entry(6, 6)]),
),
];
for (i, (lo, hi, maxsize, wentries)) in tests.drain(..).enumerate() {
let storage = MemStorage::new();
storage.wl().entries = ents.clone();
let e = storage.entries(lo, hi, maxsize, GetEntriesContext::empty(false));
if e != wentries {
panic!("#{}: expect entries {:?}, got {:?}", i, wentries, e);
}
}
}
#[test]
fn test_storage_last_index() {
let ents = vec![new_entry(3, 3), new_entry(4, 4), new_entry(5, 5)];
let storage = MemStorage::new();
storage.wl().entries = ents;
let wresult = Ok(5);
let result = storage.last_index();
if result != wresult {
panic!("want {:?}, got {:?}", wresult, result);
}
storage.wl().append(&[new_entry(6, 5)]).unwrap();
let wresult = Ok(6);
let result = storage.last_index();
if result != wresult {
panic!("want {:?}, got {:?}", wresult, result);
}
}
#[test]
fn test_storage_first_index() {
let ents = vec![new_entry(3, 3), new_entry(4, 4), new_entry(5, 5)];
let storage = MemStorage::new();
storage.wl().entries = ents;
assert_eq!(storage.first_index(), Ok(3));
storage.wl().compact(4).unwrap();
assert_eq!(storage.first_index(), Ok(4));
}
#[test]
fn test_storage_compact() {
let ents = vec![new_entry(3, 3), new_entry(4, 4), new_entry(5, 5)];
let mut tests = vec![(2, 3, 3, 3), (3, 3, 3, 3), (4, 4, 4, 2), (5, 5, 5, 1)];
for (i, (idx, windex, wterm, wlen)) in tests.drain(..).enumerate() {
let storage = MemStorage::new();
storage.wl().entries = ents.clone();
storage.wl().compact(idx).unwrap();
let index = storage.first_index().unwrap();
if index != windex {
panic!("#{}: want {}, index {}", i, windex, index);
}
let term = if let Ok(v) =
storage.entries(index, index + 1, 1, GetEntriesContext::empty(false))
{
v.first().map_or(0, |e| e.term)
} else {
0
};
if term != wterm {
panic!("#{}: want {}, term {}", i, wterm, term);
}
let last = storage.last_index().unwrap();
let len = storage
.entries(index, last + 1, 100, GetEntriesContext::empty(false))
.unwrap()
.len();
if len != wlen {
panic!("#{}: want {}, term {}", i, wlen, len);
}
}
}
#[test]
fn test_storage_create_snapshot() {
let ents = vec![new_entry(3, 3), new_entry(4, 4), new_entry(5, 5)];
let nodes = vec![1, 2, 3];
let mut conf_state = ConfState::default();
conf_state.voters = nodes.clone();
let unavailable = Err(RaftError::Store(
StorageError::SnapshotTemporarilyUnavailable,
));
let mut tests = vec![
(4, Ok(new_snapshot(4, 4, nodes.clone())), 0),
(5, Ok(new_snapshot(5, 5, nodes.clone())), 5),
(5, Ok(new_snapshot(6, 5, nodes)), 6),
(5, unavailable, 6),
];
for (i, (idx, wresult, windex)) in tests.drain(..).enumerate() {
let storage = MemStorage::new();
storage.wl().entries = ents.clone();
storage.wl().raft_state.hard_state.commit = idx;
storage.wl().raft_state.hard_state.term = idx;
storage.wl().raft_state.conf_state = conf_state.clone();
if wresult.is_err() {
storage.wl().trigger_snap_unavailable();
}
let result = storage.snapshot(windex, 0);
if result != wresult {
panic!("#{}: want {:?}, got {:?}", i, wresult, result);
}
}
}
#[test]
fn test_storage_append() {
let ents = vec![new_entry(3, 3), new_entry(4, 4), new_entry(5, 5)];
let mut tests = vec![
(
vec![new_entry(3, 3), new_entry(4, 4), new_entry(5, 5)],
Some(vec![new_entry(3, 3), new_entry(4, 4), new_entry(5, 5)]),
),
(
vec![new_entry(3, 3), new_entry(4, 6), new_entry(5, 6)],
Some(vec![new_entry(3, 3), new_entry(4, 6), new_entry(5, 6)]),
),
(
vec![
new_entry(3, 3),
new_entry(4, 4),
new_entry(5, 5),
new_entry(6, 5),
],
Some(vec![
new_entry(3, 3),
new_entry(4, 4),
new_entry(5, 5),
new_entry(6, 5),
]),
),
// overwrite compacted raft logs is not allowed
(
vec![new_entry(2, 3), new_entry(3, 3), new_entry(4, 5)],
None,
),
// truncate the existing entries and append
(
vec![new_entry(4, 5)],
Some(vec![new_entry(3, 3), new_entry(4, 5)]),
),
// direct append
(
vec![new_entry(6, 6)],
Some(vec![
new_entry(3, 3),
new_entry(4, 4),
new_entry(5, 5),
new_entry(6, 6),
]),
),
];
for (i, (entries, wentries)) in tests.drain(..).enumerate() {
let storage = MemStorage::new();
storage.wl().entries = ents.clone();
let res = panic::catch_unwind(AssertUnwindSafe(|| storage.wl().append(&entries)));
if let Some(wentries) = wentries {
let _ = res.unwrap();
let e = &storage.wl().entries;
if *e != wentries {
panic!("#{}: want {:?}, entries {:?}", i, wentries, e);
}
} else {
res.unwrap_err();
}
}
}
#[test]
fn test_storage_apply_snapshot() {
let nodes = vec![1, 2, 3];
let storage = MemStorage::new();
// Apply snapshot successfully
let snap = new_snapshot(4, 4, nodes.clone());
storage.wl().apply_snapshot(snap).unwrap();
// Apply snapshot fails due to StorageError::SnapshotOutOfDate
let snap = new_snapshot(3, 3, nodes);
storage.wl().apply_snapshot(snap).unwrap_err();
}
}