-
Notifications
You must be signed in to change notification settings - Fork 169
/
storage.go
310 lines (271 loc) · 10.2 KB
/
storage.go
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
// 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.
package raft
import (
"errors"
"sync"
pb "go.etcd.io/raft/v3/raftpb"
)
// ErrCompacted is returned by Storage.Entries/Compact when a requested
// index is unavailable because it predates the last snapshot.
var ErrCompacted = errors.New("requested index is unavailable due to compaction")
// ErrSnapOutOfDate is returned by Storage.CreateSnapshot when a requested
// index is older than the existing snapshot.
var ErrSnapOutOfDate = errors.New("requested index is older than the existing snapshot")
// ErrUnavailable is returned by Storage interface when the requested log entries
// are unavailable.
var ErrUnavailable = errors.New("requested entry at index is unavailable")
// ErrSnapshotTemporarilyUnavailable is returned by the Storage interface when the required
// snapshot is temporarily unavailable.
var ErrSnapshotTemporarilyUnavailable = errors.New("snapshot is temporarily unavailable")
// Storage is an interface that may be implemented by the application
// to retrieve log entries from storage.
//
// 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.
type Storage interface {
// TODO(tbg): split this into two interfaces, LogStorage and StateStorage.
// InitialState returns the saved HardState and ConfState information.
InitialState() (pb.HardState, pb.ConfState, error)
// Entries returns a slice of consecutive log entries in the range [lo, hi),
// starting from lo. The maxSize limits the total size of the log entries
// returned, but Entries returns at least one entry if any.
//
// The caller of Entries owns the returned slice, and may append to it. The
// individual entries in the slice must not be mutated, neither by the Storage
// implementation nor the caller. Note that raft may forward these entries
// back to the application via Ready struct, so the corresponding handler must
// not mutate entries either (see comments in Ready struct).
//
// Since the caller may append to the returned slice, Storage implementation
// must protect its state from corruption that such appends may cause. For
// example, common ways to do so are:
// - allocate the slice before returning it (safest option),
// - return a slice protected by Go full slice expression, which causes
// copying on appends (see MemoryStorage).
//
// Returns ErrCompacted if entry lo has been compacted, or ErrUnavailable if
// encountered an unavailable entry in [lo, hi).
Entries(lo, hi, maxSize uint64) ([]pb.Entry, error)
// Term returns the term of entry i, which must be in the range
// [FirstIndex()-1, LastIndex()]. The term of the entry before
// FirstIndex is retained for matching purposes even though the
// rest of that entry may not be available.
Term(i uint64) (uint64, error)
// LastIndex returns the index of the last entry in the log.
LastIndex() (uint64, error)
// FirstIndex returns the index of the first log entry that is
// possibly available via Entries (older entries have been incorporated
// into the latest Snapshot; if storage only contains the dummy entry the
// first log entry is not available).
FirstIndex() (uint64, error)
// Snapshot returns the most recent snapshot.
// If snapshot is temporarily unavailable, it should return ErrSnapshotTemporarilyUnavailable,
// so raft state machine could know that Storage needs some time to prepare
// snapshot and call Snapshot later.
Snapshot() (pb.Snapshot, error)
}
type inMemStorageCallStats struct {
initialState, firstIndex, lastIndex, entries, term, snapshot int
}
// MemoryStorage implements the Storage interface backed by an
// in-memory array.
type MemoryStorage struct {
// Protects access to all fields. Most methods of MemoryStorage are
// run on the raft goroutine, but Append() is run on an application
// goroutine.
sync.Mutex
hardState pb.HardState
snapshot pb.Snapshot
// ents[i] has raft log position i+snapshot.Metadata.Index
ents []pb.Entry
callStats inMemStorageCallStats
}
// NewMemoryStorage creates an empty MemoryStorage.
func NewMemoryStorage() *MemoryStorage {
return &MemoryStorage{
// When starting from scratch populate the list with a dummy entry at term zero.
ents: make([]pb.Entry, 1),
}
}
// InitialState implements the Storage interface.
func (ms *MemoryStorage) InitialState() (pb.HardState, pb.ConfState, error) {
ms.callStats.initialState++
return ms.hardState, ms.snapshot.Metadata.ConfState, nil
}
// SetHardState saves the current HardState.
func (ms *MemoryStorage) SetHardState(st pb.HardState) error {
ms.Lock()
defer ms.Unlock()
ms.hardState = st
return nil
}
// Entries implements the Storage interface.
func (ms *MemoryStorage) Entries(lo, hi, maxSize uint64) ([]pb.Entry, error) {
ms.Lock()
defer ms.Unlock()
ms.callStats.entries++
offset := ms.ents[0].Index
if lo <= offset {
return nil, ErrCompacted
}
if hi > ms.lastIndex()+1 {
getLogger().Panicf("entries' hi(%d) is out of bound lastindex(%d)", hi, ms.lastIndex())
}
// only contains dummy entries.
if len(ms.ents) == 1 {
return nil, ErrUnavailable
}
ents := limitSize(ms.ents[lo-offset:hi-offset], entryEncodingSize(maxSize))
// NB: use the full slice expression to limit what the caller can do with the
// returned slice. For example, an append will reallocate and copy this slice
// instead of corrupting the neighbouring ms.ents.
return ents[:len(ents):len(ents)], nil
}
// Term implements the Storage interface.
func (ms *MemoryStorage) Term(i uint64) (uint64, error) {
ms.Lock()
defer ms.Unlock()
ms.callStats.term++
offset := ms.ents[0].Index
if i < offset {
return 0, ErrCompacted
}
if int(i-offset) >= len(ms.ents) {
return 0, ErrUnavailable
}
return ms.ents[i-offset].Term, nil
}
// LastIndex implements the Storage interface.
func (ms *MemoryStorage) LastIndex() (uint64, error) {
ms.Lock()
defer ms.Unlock()
ms.callStats.lastIndex++
return ms.lastIndex(), nil
}
func (ms *MemoryStorage) lastIndex() uint64 {
return ms.ents[0].Index + uint64(len(ms.ents)) - 1
}
// FirstIndex implements the Storage interface.
func (ms *MemoryStorage) FirstIndex() (uint64, error) {
ms.Lock()
defer ms.Unlock()
ms.callStats.firstIndex++
return ms.firstIndex(), nil
}
func (ms *MemoryStorage) firstIndex() uint64 {
return ms.ents[0].Index + 1
}
// Snapshot implements the Storage interface.
func (ms *MemoryStorage) Snapshot() (pb.Snapshot, error) {
ms.Lock()
defer ms.Unlock()
ms.callStats.snapshot++
return ms.snapshot, nil
}
// ApplySnapshot overwrites the contents of this Storage object with
// those of the given snapshot.
func (ms *MemoryStorage) ApplySnapshot(snap pb.Snapshot) error {
ms.Lock()
defer ms.Unlock()
//handle check for old snapshot being applied
msIndex := ms.snapshot.Metadata.Index
snapIndex := snap.Metadata.Index
if msIndex >= snapIndex {
return ErrSnapOutOfDate
}
ms.snapshot = snap
ms.ents = []pb.Entry{{Term: snap.Metadata.Term, Index: snap.Metadata.Index}}
return nil
}
// CreateSnapshot makes a snapshot which can be retrieved with Snapshot() and
// can be used to reconstruct the state at that point.
// If any configuration changes have been made since the last compaction,
// the result of the last ApplyConfChange must be passed in.
func (ms *MemoryStorage) CreateSnapshot(i uint64, cs *pb.ConfState, data []byte) (pb.Snapshot, error) {
ms.Lock()
defer ms.Unlock()
if i <= ms.snapshot.Metadata.Index {
return pb.Snapshot{}, ErrSnapOutOfDate
}
offset := ms.ents[0].Index
if i > ms.lastIndex() {
getLogger().Panicf("snapshot %d is out of bound lastindex(%d)", i, ms.lastIndex())
}
ms.snapshot.Metadata.Index = i
ms.snapshot.Metadata.Term = ms.ents[i-offset].Term
if cs != nil {
ms.snapshot.Metadata.ConfState = *cs
}
ms.snapshot.Data = data
return ms.snapshot, nil
}
// Compact discards all log entries prior to compactIndex.
// It is the application's responsibility to not attempt to compact an index
// greater than raftLog.applied.
func (ms *MemoryStorage) Compact(compactIndex uint64) error {
ms.Lock()
defer ms.Unlock()
offset := ms.ents[0].Index
if compactIndex <= offset {
return ErrCompacted
}
if compactIndex > ms.lastIndex() {
getLogger().Panicf("compact %d is out of bound lastindex(%d)", compactIndex, ms.lastIndex())
}
i := compactIndex - offset
// NB: allocate a new slice instead of reusing the old ms.ents. Entries in
// ms.ents are immutable, and can be referenced from outside MemoryStorage
// through slices returned by ms.Entries().
ents := make([]pb.Entry, 1, uint64(len(ms.ents))-i)
ents[0].Index = ms.ents[i].Index
ents[0].Term = ms.ents[i].Term
ents = append(ents, ms.ents[i+1:]...)
ms.ents = ents
return nil
}
// Append the new entries to storage.
// TODO (xiangli): ensure the entries are continuous and
// entries[0].Index > ms.entries[0].Index
func (ms *MemoryStorage) Append(entries []pb.Entry) error {
if len(entries) == 0 {
return nil
}
ms.Lock()
defer ms.Unlock()
first := ms.firstIndex()
last := entries[0].Index + uint64(len(entries)) - 1
// shortcut if there is no new entry.
if last < first {
return nil
}
// truncate compacted entries
if first > entries[0].Index {
entries = entries[first-entries[0].Index:]
}
offset := entries[0].Index - ms.ents[0].Index
switch {
case uint64(len(ms.ents)) > offset:
// NB: full slice expression protects ms.ents at index >= offset from
// rewrites, as they may still be referenced from outside MemoryStorage.
ms.ents = append(ms.ents[:offset:offset], entries...)
case uint64(len(ms.ents)) == offset:
ms.ents = append(ms.ents, entries...)
default:
getLogger().Panicf("missing log entry [last: %d, append at: %d]",
ms.lastIndex(), entries[0].Index)
}
return nil
}