Scott Baker | 2d89798 | 2019-09-24 11:50:08 -0700 | [diff] [blame] | 1 | // Copyright 2015 The etcd Authors |
| 2 | // |
| 3 | // Licensed under the Apache License, Version 2.0 (the "License"); |
| 4 | // you may not use this file except in compliance with the License. |
| 5 | // You may obtain a copy of the License at |
| 6 | // |
| 7 | // http://www.apache.org/licenses/LICENSE-2.0 |
| 8 | // |
| 9 | // Unless required by applicable law or agreed to in writing, software |
| 10 | // distributed under the License is distributed on an "AS IS" BASIS, |
| 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 12 | // See the License for the specific language governing permissions and |
| 13 | // limitations under the License. |
| 14 | |
| 15 | package raft |
| 16 | |
| 17 | import ( |
| 18 | "errors" |
| 19 | "sync" |
| 20 | |
| 21 | pb "go.etcd.io/etcd/raft/raftpb" |
| 22 | ) |
| 23 | |
| 24 | // ErrCompacted is returned by Storage.Entries/Compact when a requested |
| 25 | // index is unavailable because it predates the last snapshot. |
| 26 | var ErrCompacted = errors.New("requested index is unavailable due to compaction") |
| 27 | |
| 28 | // ErrSnapOutOfDate is returned by Storage.CreateSnapshot when a requested |
| 29 | // index is older than the existing snapshot. |
| 30 | var ErrSnapOutOfDate = errors.New("requested index is older than the existing snapshot") |
| 31 | |
| 32 | // ErrUnavailable is returned by Storage interface when the requested log entries |
| 33 | // are unavailable. |
| 34 | var ErrUnavailable = errors.New("requested entry at index is unavailable") |
| 35 | |
| 36 | // ErrSnapshotTemporarilyUnavailable is returned by the Storage interface when the required |
| 37 | // snapshot is temporarily unavailable. |
| 38 | var ErrSnapshotTemporarilyUnavailable = errors.New("snapshot is temporarily unavailable") |
| 39 | |
| 40 | // Storage is an interface that may be implemented by the application |
| 41 | // to retrieve log entries from storage. |
| 42 | // |
| 43 | // If any Storage method returns an error, the raft instance will |
| 44 | // become inoperable and refuse to participate in elections; the |
| 45 | // application is responsible for cleanup and recovery in this case. |
| 46 | type Storage interface { |
Scott Baker | 8487c5d | 2019-10-18 12:49:46 -0700 | [diff] [blame^] | 47 | // TODO(tbg): split this into two interfaces, LogStorage and StateStorage. |
| 48 | |
Scott Baker | 2d89798 | 2019-09-24 11:50:08 -0700 | [diff] [blame] | 49 | // InitialState returns the saved HardState and ConfState information. |
| 50 | InitialState() (pb.HardState, pb.ConfState, error) |
| 51 | // Entries returns a slice of log entries in the range [lo,hi). |
| 52 | // MaxSize limits the total size of the log entries returned, but |
| 53 | // Entries returns at least one entry if any. |
| 54 | Entries(lo, hi, maxSize uint64) ([]pb.Entry, error) |
| 55 | // Term returns the term of entry i, which must be in the range |
| 56 | // [FirstIndex()-1, LastIndex()]. The term of the entry before |
| 57 | // FirstIndex is retained for matching purposes even though the |
| 58 | // rest of that entry may not be available. |
| 59 | Term(i uint64) (uint64, error) |
| 60 | // LastIndex returns the index of the last entry in the log. |
| 61 | LastIndex() (uint64, error) |
| 62 | // FirstIndex returns the index of the first log entry that is |
| 63 | // possibly available via Entries (older entries have been incorporated |
| 64 | // into the latest Snapshot; if storage only contains the dummy entry the |
| 65 | // first log entry is not available). |
| 66 | FirstIndex() (uint64, error) |
| 67 | // Snapshot returns the most recent snapshot. |
| 68 | // If snapshot is temporarily unavailable, it should return ErrSnapshotTemporarilyUnavailable, |
| 69 | // so raft state machine could know that Storage needs some time to prepare |
| 70 | // snapshot and call Snapshot later. |
| 71 | Snapshot() (pb.Snapshot, error) |
| 72 | } |
| 73 | |
| 74 | // MemoryStorage implements the Storage interface backed by an |
| 75 | // in-memory array. |
| 76 | type MemoryStorage struct { |
| 77 | // Protects access to all fields. Most methods of MemoryStorage are |
| 78 | // run on the raft goroutine, but Append() is run on an application |
| 79 | // goroutine. |
| 80 | sync.Mutex |
| 81 | |
| 82 | hardState pb.HardState |
| 83 | snapshot pb.Snapshot |
| 84 | // ents[i] has raft log position i+snapshot.Metadata.Index |
| 85 | ents []pb.Entry |
| 86 | } |
| 87 | |
| 88 | // NewMemoryStorage creates an empty MemoryStorage. |
| 89 | func NewMemoryStorage() *MemoryStorage { |
| 90 | return &MemoryStorage{ |
| 91 | // When starting from scratch populate the list with a dummy entry at term zero. |
| 92 | ents: make([]pb.Entry, 1), |
| 93 | } |
| 94 | } |
| 95 | |
| 96 | // InitialState implements the Storage interface. |
| 97 | func (ms *MemoryStorage) InitialState() (pb.HardState, pb.ConfState, error) { |
| 98 | return ms.hardState, ms.snapshot.Metadata.ConfState, nil |
| 99 | } |
| 100 | |
| 101 | // SetHardState saves the current HardState. |
| 102 | func (ms *MemoryStorage) SetHardState(st pb.HardState) error { |
| 103 | ms.Lock() |
| 104 | defer ms.Unlock() |
| 105 | ms.hardState = st |
| 106 | return nil |
| 107 | } |
| 108 | |
| 109 | // Entries implements the Storage interface. |
| 110 | func (ms *MemoryStorage) Entries(lo, hi, maxSize uint64) ([]pb.Entry, error) { |
| 111 | ms.Lock() |
| 112 | defer ms.Unlock() |
| 113 | offset := ms.ents[0].Index |
| 114 | if lo <= offset { |
| 115 | return nil, ErrCompacted |
| 116 | } |
| 117 | if hi > ms.lastIndex()+1 { |
| 118 | raftLogger.Panicf("entries' hi(%d) is out of bound lastindex(%d)", hi, ms.lastIndex()) |
| 119 | } |
| 120 | // only contains dummy entries. |
| 121 | if len(ms.ents) == 1 { |
| 122 | return nil, ErrUnavailable |
| 123 | } |
| 124 | |
| 125 | ents := ms.ents[lo-offset : hi-offset] |
| 126 | return limitSize(ents, maxSize), nil |
| 127 | } |
| 128 | |
| 129 | // Term implements the Storage interface. |
| 130 | func (ms *MemoryStorage) Term(i uint64) (uint64, error) { |
| 131 | ms.Lock() |
| 132 | defer ms.Unlock() |
| 133 | offset := ms.ents[0].Index |
| 134 | if i < offset { |
| 135 | return 0, ErrCompacted |
| 136 | } |
| 137 | if int(i-offset) >= len(ms.ents) { |
| 138 | return 0, ErrUnavailable |
| 139 | } |
| 140 | return ms.ents[i-offset].Term, nil |
| 141 | } |
| 142 | |
| 143 | // LastIndex implements the Storage interface. |
| 144 | func (ms *MemoryStorage) LastIndex() (uint64, error) { |
| 145 | ms.Lock() |
| 146 | defer ms.Unlock() |
| 147 | return ms.lastIndex(), nil |
| 148 | } |
| 149 | |
| 150 | func (ms *MemoryStorage) lastIndex() uint64 { |
| 151 | return ms.ents[0].Index + uint64(len(ms.ents)) - 1 |
| 152 | } |
| 153 | |
| 154 | // FirstIndex implements the Storage interface. |
| 155 | func (ms *MemoryStorage) FirstIndex() (uint64, error) { |
| 156 | ms.Lock() |
| 157 | defer ms.Unlock() |
| 158 | return ms.firstIndex(), nil |
| 159 | } |
| 160 | |
| 161 | func (ms *MemoryStorage) firstIndex() uint64 { |
| 162 | return ms.ents[0].Index + 1 |
| 163 | } |
| 164 | |
| 165 | // Snapshot implements the Storage interface. |
| 166 | func (ms *MemoryStorage) Snapshot() (pb.Snapshot, error) { |
| 167 | ms.Lock() |
| 168 | defer ms.Unlock() |
| 169 | return ms.snapshot, nil |
| 170 | } |
| 171 | |
| 172 | // ApplySnapshot overwrites the contents of this Storage object with |
| 173 | // those of the given snapshot. |
| 174 | func (ms *MemoryStorage) ApplySnapshot(snap pb.Snapshot) error { |
| 175 | ms.Lock() |
| 176 | defer ms.Unlock() |
| 177 | |
| 178 | //handle check for old snapshot being applied |
| 179 | msIndex := ms.snapshot.Metadata.Index |
| 180 | snapIndex := snap.Metadata.Index |
| 181 | if msIndex >= snapIndex { |
| 182 | return ErrSnapOutOfDate |
| 183 | } |
| 184 | |
| 185 | ms.snapshot = snap |
| 186 | ms.ents = []pb.Entry{{Term: snap.Metadata.Term, Index: snap.Metadata.Index}} |
| 187 | return nil |
| 188 | } |
| 189 | |
| 190 | // CreateSnapshot makes a snapshot which can be retrieved with Snapshot() and |
| 191 | // can be used to reconstruct the state at that point. |
| 192 | // If any configuration changes have been made since the last compaction, |
| 193 | // the result of the last ApplyConfChange must be passed in. |
| 194 | func (ms *MemoryStorage) CreateSnapshot(i uint64, cs *pb.ConfState, data []byte) (pb.Snapshot, error) { |
| 195 | ms.Lock() |
| 196 | defer ms.Unlock() |
| 197 | if i <= ms.snapshot.Metadata.Index { |
| 198 | return pb.Snapshot{}, ErrSnapOutOfDate |
| 199 | } |
| 200 | |
| 201 | offset := ms.ents[0].Index |
| 202 | if i > ms.lastIndex() { |
| 203 | raftLogger.Panicf("snapshot %d is out of bound lastindex(%d)", i, ms.lastIndex()) |
| 204 | } |
| 205 | |
| 206 | ms.snapshot.Metadata.Index = i |
| 207 | ms.snapshot.Metadata.Term = ms.ents[i-offset].Term |
| 208 | if cs != nil { |
| 209 | ms.snapshot.Metadata.ConfState = *cs |
| 210 | } |
| 211 | ms.snapshot.Data = data |
| 212 | return ms.snapshot, nil |
| 213 | } |
| 214 | |
| 215 | // Compact discards all log entries prior to compactIndex. |
| 216 | // It is the application's responsibility to not attempt to compact an index |
| 217 | // greater than raftLog.applied. |
| 218 | func (ms *MemoryStorage) Compact(compactIndex uint64) error { |
| 219 | ms.Lock() |
| 220 | defer ms.Unlock() |
| 221 | offset := ms.ents[0].Index |
| 222 | if compactIndex <= offset { |
| 223 | return ErrCompacted |
| 224 | } |
| 225 | if compactIndex > ms.lastIndex() { |
| 226 | raftLogger.Panicf("compact %d is out of bound lastindex(%d)", compactIndex, ms.lastIndex()) |
| 227 | } |
| 228 | |
| 229 | i := compactIndex - offset |
| 230 | ents := make([]pb.Entry, 1, 1+uint64(len(ms.ents))-i) |
| 231 | ents[0].Index = ms.ents[i].Index |
| 232 | ents[0].Term = ms.ents[i].Term |
| 233 | ents = append(ents, ms.ents[i+1:]...) |
| 234 | ms.ents = ents |
| 235 | return nil |
| 236 | } |
| 237 | |
| 238 | // Append the new entries to storage. |
| 239 | // TODO (xiangli): ensure the entries are continuous and |
| 240 | // entries[0].Index > ms.entries[0].Index |
| 241 | func (ms *MemoryStorage) Append(entries []pb.Entry) error { |
| 242 | if len(entries) == 0 { |
| 243 | return nil |
| 244 | } |
| 245 | |
| 246 | ms.Lock() |
| 247 | defer ms.Unlock() |
| 248 | |
| 249 | first := ms.firstIndex() |
| 250 | last := entries[0].Index + uint64(len(entries)) - 1 |
| 251 | |
| 252 | // shortcut if there is no new entry. |
| 253 | if last < first { |
| 254 | return nil |
| 255 | } |
| 256 | // truncate compacted entries |
| 257 | if first > entries[0].Index { |
| 258 | entries = entries[first-entries[0].Index:] |
| 259 | } |
| 260 | |
| 261 | offset := entries[0].Index - ms.ents[0].Index |
| 262 | switch { |
| 263 | case uint64(len(ms.ents)) > offset: |
| 264 | ms.ents = append([]pb.Entry{}, ms.ents[:offset]...) |
| 265 | ms.ents = append(ms.ents, entries...) |
| 266 | case uint64(len(ms.ents)) == offset: |
| 267 | ms.ents = append(ms.ents, entries...) |
| 268 | default: |
| 269 | raftLogger.Panicf("missing log entry [last: %d, append at: %d]", |
| 270 | ms.lastIndex(), entries[0].Index) |
| 271 | } |
| 272 | return nil |
| 273 | } |