github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/kv/kvserver/raftentry/ring_buffer.go (about) 1 // Copyright 2018 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package raftentry 12 13 import ( 14 "math/bits" 15 16 "go.etcd.io/etcd/raft/raftpb" 17 ) 18 19 // ringBuf is a ring buffer of raft entries. 20 type ringBuf struct { 21 buf []raftpb.Entry 22 head int 23 len int 24 } 25 26 const ( 27 shrinkThreshold = 8 // shrink buf if len(buf)/len is above this. 28 minBufSize = 16 /* entries */ 29 ) 30 31 // add adds ents to the ringBuf keeping track of how much was actually added 32 // given that ents may overlap with existing entries or may be rejected from 33 // the buffer. ents must not be empty. 34 func (b *ringBuf) add(ents []raftpb.Entry) (addedBytes, addedEntries int32) { 35 if afterCache := b.len > 0 && ents[0].Index > last(b).index(b)+1; afterCache { 36 // If ents is non-contiguous and later than the currently cached range then 37 // remove the current entries and add ents in their place. 38 removedBytes, removedEntries := b.clearTo(last(b).index(b) + 1) 39 addedBytes, addedEntries = -1*removedBytes, -1*removedEntries 40 } 41 before, after, ok := computeExtension(b, ents[0].Index, ents[len(ents)-1].Index) 42 if !ok { 43 return 44 } 45 extend(b, before, after) 46 it := first(b) 47 if before == 0 && after != b.len { // skip unchanged prefix 48 it, _ = iterateFrom(b, ents[0].Index) // safe by construction 49 } 50 firstNewAfter := len(ents) - after 51 for i, e := range ents { 52 if i < before || i >= firstNewAfter { 53 addedEntries++ 54 addedBytes += int32(e.Size()) 55 } else { 56 addedBytes += int32(e.Size() - it.entry(b).Size()) 57 } 58 it = it.push(b, e) 59 } 60 return 61 } 62 63 // truncateFrom clears all entries from the ringBuf with index equal to or 64 // greater than lo. The method returns the aggregate size and count of entries 65 // removed. 66 func (b *ringBuf) truncateFrom(lo uint64) (removedBytes, removedEntries int32) { 67 it, ok := iterateFrom(b, lo) 68 for ok { 69 removedBytes += int32(it.entry(b).Size()) 70 removedEntries++ 71 it.clear(b) 72 it, ok = it.next(b) 73 } 74 b.len -= int(removedEntries) 75 if b.len < (len(b.buf) / shrinkThreshold) { 76 realloc(b, 0, b.len) 77 } 78 return 79 } 80 81 // clearTo clears all entries from the ringBuf with index less than hi. The 82 // method returns the aggregate size and count of entries removed. 83 func (b *ringBuf) clearTo(hi uint64) (removedBytes, removedEntries int32) { 84 if b.len == 0 || hi < first(b).index(b) { 85 return 86 } 87 it, ok := first(b), true 88 firstIndex := it.index(b) 89 for ok && it.index(b) < hi { 90 removedBytes += int32(it.entry(b).Size()) 91 removedEntries++ 92 it.clear(b) 93 it, ok = it.next(b) 94 } 95 offset := int(hi - firstIndex) 96 if offset > b.len { 97 offset = b.len 98 } 99 b.len = b.len - offset 100 b.head = (b.head + offset) % len(b.buf) 101 if b.len < (len(b.buf) / shrinkThreshold) { 102 realloc(b, 0, b.len) 103 } 104 return 105 } 106 107 func (b *ringBuf) get(index uint64) (e raftpb.Entry, ok bool) { 108 it, ok := iterateFrom(b, index) 109 if !ok { 110 return e, ok 111 } 112 return *it.entry(b), ok 113 } 114 115 func (b *ringBuf) scan( 116 ents []raftpb.Entry, lo, hi, maxBytes uint64, 117 ) (_ []raftpb.Entry, bytes uint64, nextIdx uint64, exceededMaxBytes bool) { 118 var it iterator 119 nextIdx = lo 120 it, ok := iterateFrom(b, lo) 121 for ok && !exceededMaxBytes && it.index(b) < hi { 122 e := it.entry(b) 123 s := uint64(e.Size()) 124 exceededMaxBytes = bytes+s > maxBytes 125 if exceededMaxBytes && len(ents) > 0 { 126 break 127 } 128 bytes += s 129 ents = append(ents, *e) 130 nextIdx++ 131 it, ok = it.next(b) 132 } 133 return ents, bytes, nextIdx, exceededMaxBytes 134 } 135 136 // reallocs b.buf into a new buffer of newSize leaving before zero value entries 137 // at the front of b. 138 func realloc(b *ringBuf, before, newLen int) { 139 newBuf := make([]raftpb.Entry, reallocLen(newLen)) 140 if b.head+b.len > len(b.buf) { 141 n := copy(newBuf[before:], b.buf[b.head:]) 142 copy(newBuf[before+n:], b.buf[:(b.head+b.len)%len(b.buf)]) 143 } else { 144 copy(newBuf[before:], b.buf[b.head:b.head+b.len]) 145 } 146 b.buf = newBuf 147 b.head = 0 148 b.len = newLen 149 } 150 151 // reallocLen returns a new length which is a power-of-two greater than or equal 152 // to need and at least minBufSize. 153 func reallocLen(need int) (newLen int) { 154 if need <= minBufSize { 155 return minBufSize 156 } 157 return 1 << uint(bits.Len(uint(need))) 158 } 159 160 // extend takes a number of entries before and after the current cached values 161 // to increase the length of b. The before-length prefix of b will now be zero 162 // valued entries. 163 func extend(b *ringBuf, before, after int) { 164 size := before + b.len + after 165 if size > len(b.buf) { 166 realloc(b, before, size) 167 } else { 168 b.head = (b.head - before) % len(b.buf) 169 if b.head < 0 { 170 b.head += len(b.buf) 171 } 172 } 173 b.len = size 174 } 175 176 // computeExtension returns the number of entries in [lo, hi] which will be 177 // added before and after the current range of the cache. Note that lo and hi 178 // here are inclusive indices for the range being added and that before and 179 // after are counts, not indices, of number of entries which precede and follow 180 // the currently cached range. If [lo, hi] is not overlapping or directly 181 // adjacent to the current cache bounds, ok will be false. 182 func computeExtension(b *ringBuf, lo, hi uint64) (before, after int, ok bool) { 183 if b.len == 0 { 184 return 0, int(hi) - int(lo) + 1, true 185 } 186 first, last := first(b).index(b), last(b).index(b) 187 if lo > (last+1) || hi < (first-1) { // gap case 188 return 0, 0, false 189 } 190 if lo < first { 191 before = int(first) - int(lo) 192 } 193 if hi > last { 194 after = int(hi) - int(last) 195 } 196 return before, after, true 197 } 198 199 // iterator indexes into a ringBuf. A value of -1 is not valid. 200 type iterator int 201 202 func iterateFrom(b *ringBuf, index uint64) (_ iterator, ok bool) { 203 if b.len == 0 { 204 return -1, false 205 } 206 offset := int(index) - int(first(b).index(b)) 207 if offset < 0 || offset >= b.len { 208 return -1, false 209 } 210 return iterator((b.head + offset) % len(b.buf)), true 211 } 212 213 // first returns an iterator pointing to the first entry of the ringBuf. 214 // If b is empty, the returned iterator is not valid. 215 func first(b *ringBuf) iterator { 216 return iterator(b.head) 217 } 218 219 // last returns an iterator pointing to the last element in b. 220 // It is unsafe to call last if b has an empty buffer. 221 func last(b *ringBuf) iterator { 222 return iterator((b.head + b.len - 1) % len(b.buf)) 223 } 224 225 func (it iterator) valid(b *ringBuf) bool { 226 return it >= 0 && int(it) < len(b.buf) 227 } 228 229 // index returns the index of the entry at iterator's curent position. 230 func (it iterator) index(b *ringBuf) uint64 { 231 return b.buf[it].Index 232 } 233 234 // entry returns the entry at iterator's curent position. 235 func (it iterator) entry(b *ringBuf) *raftpb.Entry { 236 return &b.buf[it] 237 } 238 239 // clear zeroes the current value in b. 240 func (it iterator) clear(b *ringBuf) { 241 b.buf[it] = raftpb.Entry{} 242 } 243 244 // next returns an iterator which points to the next element in b. 245 // If it is invalid or points to the last element in b, (-1, false) is returned. 246 func (it iterator) next(b *ringBuf) (_ iterator, ok bool) { 247 if !it.valid(b) || it == last(b) { 248 return -1, false 249 } 250 return iterator(int(it+1) % len(b.buf)), true 251 } 252 253 // push sets the iterator's current value in b to e and calls next 254 // It is the caller's responsibility to ensure that b has space for the new 255 // entry. 256 func (it iterator) push(b *ringBuf, e raftpb.Entry) iterator { 257 b.buf[it] = e 258 it, _ = it.next(b) 259 return it 260 }