blob: 1bff5a7bdcb937c70784ae71902f6e978ab1b5eb [file] [log] [blame]
Holger Hildebrandtfa074992020-03-27 15:42:06 +00001// 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
15package raft
16
17import pb "go.etcd.io/etcd/raft/raftpb"
18
19// unstable.entries[i] has raft log position i+unstable.offset.
20// Note that unstable.offset may be less than the highest log
21// position in storage; this means that the next write to storage
22// might need to truncate the log before persisting unstable.entries.
23type unstable struct {
24 // the incoming unstable snapshot, if any.
25 snapshot *pb.Snapshot
26 // all entries that have not yet been written to storage.
27 entries []pb.Entry
28 offset uint64
29
30 logger Logger
31}
32
33// maybeFirstIndex returns the index of the first possible entry in entries
34// if it has a snapshot.
35func (u *unstable) maybeFirstIndex() (uint64, bool) {
36 if u.snapshot != nil {
37 return u.snapshot.Metadata.Index + 1, true
38 }
39 return 0, false
40}
41
42// maybeLastIndex returns the last index if it has at least one
43// unstable entry or snapshot.
44func (u *unstable) maybeLastIndex() (uint64, bool) {
45 if l := len(u.entries); l != 0 {
46 return u.offset + uint64(l) - 1, true
47 }
48 if u.snapshot != nil {
49 return u.snapshot.Metadata.Index, true
50 }
51 return 0, false
52}
53
54// maybeTerm returns the term of the entry at index i, if there
55// is any.
56func (u *unstable) maybeTerm(i uint64) (uint64, bool) {
57 if i < u.offset {
58 if u.snapshot != nil && u.snapshot.Metadata.Index == i {
59 return u.snapshot.Metadata.Term, true
60 }
61 return 0, false
62 }
63
64 last, ok := u.maybeLastIndex()
65 if !ok {
66 return 0, false
67 }
68 if i > last {
69 return 0, false
70 }
71
72 return u.entries[i-u.offset].Term, true
73}
74
75func (u *unstable) stableTo(i, t uint64) {
76 gt, ok := u.maybeTerm(i)
77 if !ok {
78 return
79 }
80 // if i < offset, term is matched with the snapshot
81 // only update the unstable entries if term is matched with
82 // an unstable entry.
83 if gt == t && i >= u.offset {
84 u.entries = u.entries[i+1-u.offset:]
85 u.offset = i + 1
86 u.shrinkEntriesArray()
87 }
88}
89
90// shrinkEntriesArray discards the underlying array used by the entries slice
91// if most of it isn't being used. This avoids holding references to a bunch of
92// potentially large entries that aren't needed anymore. Simply clearing the
93// entries wouldn't be safe because clients might still be using them.
94func (u *unstable) shrinkEntriesArray() {
95 // We replace the array if we're using less than half of the space in
96 // it. This number is fairly arbitrary, chosen as an attempt to balance
97 // memory usage vs number of allocations. It could probably be improved
98 // with some focused tuning.
99 const lenMultiple = 2
100 if len(u.entries) == 0 {
101 u.entries = nil
102 } else if len(u.entries)*lenMultiple < cap(u.entries) {
103 newEntries := make([]pb.Entry, len(u.entries))
104 copy(newEntries, u.entries)
105 u.entries = newEntries
106 }
107}
108
109func (u *unstable) stableSnapTo(i uint64) {
110 if u.snapshot != nil && u.snapshot.Metadata.Index == i {
111 u.snapshot = nil
112 }
113}
114
115func (u *unstable) restore(s pb.Snapshot) {
116 u.offset = s.Metadata.Index + 1
117 u.entries = nil
118 u.snapshot = &s
119}
120
121func (u *unstable) truncateAndAppend(ents []pb.Entry) {
122 after := ents[0].Index
123 switch {
124 case after == u.offset+uint64(len(u.entries)):
125 // after is the next index in the u.entries
126 // directly append
127 u.entries = append(u.entries, ents...)
128 case after <= u.offset:
129 u.logger.Infof("replace the unstable entries from index %d", after)
130 // The log is being truncated to before our current offset
131 // portion, so set the offset and replace the entries
132 u.offset = after
133 u.entries = ents
134 default:
135 // truncate to after and copy to u.entries
136 // then append
137 u.logger.Infof("truncate the unstable entries before index %d", after)
138 u.entries = append([]pb.Entry{}, u.slice(u.offset, after)...)
139 u.entries = append(u.entries, ents...)
140 }
141}
142
143func (u *unstable) slice(lo uint64, hi uint64) []pb.Entry {
144 u.mustCheckOutOfBounds(lo, hi)
145 return u.entries[lo-u.offset : hi-u.offset]
146}
147
148// u.offset <= lo <= hi <= u.offset+len(u.entries)
149func (u *unstable) mustCheckOutOfBounds(lo, hi uint64) {
150 if lo > hi {
151 u.logger.Panicf("invalid unstable.slice %d > %d", lo, hi)
152 }
153 upper := u.offset + uint64(len(u.entries))
154 if lo < u.offset || hi > upper {
155 u.logger.Panicf("unstable.slice[%d,%d) out of bound [%d,%d]", lo, hi, u.offset, upper)
156 }
157}