github.com/cockroachdb/pebble@v0.0.0-20231214172447-ab4952c5f87b/internal/arenaskl/skl.go (about) 1 /* 2 * Copyright 2017 Dgraph Labs, Inc. and Contributors 3 * Modifications copyright (C) 2017 Andy Kimball and Contributors 4 * 5 * Licensed under the Apache License, Version 2.0 (the "License"); 6 * you may not use this file except in compliance with the License. 7 * You may obtain a copy of the License at 8 * 9 * http://www.apache.org/licenses/LICENSE-2.0 10 * 11 * Unless required by applicable law or agreed to in writing, software 12 * distributed under the License is distributed on an "AS IS" BASIS, 13 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 * See the License for the specific language governing permissions and 15 * limitations under the License. 16 */ 17 18 /* 19 Adapted from RocksDB inline skiplist. 20 21 Key differences: 22 - No optimization for sequential inserts (no "prev"). 23 - No custom comparator. 24 - Support overwrites. This requires care when we see the same key when inserting. 25 For RocksDB or LevelDB, overwrites are implemented as a newer sequence number in the key, so 26 there is no need for values. We don't intend to support versioning. In-place updates of values 27 would be more efficient. 28 - We discard all non-concurrent code. 29 - We do not support Splices. This simplifies the code a lot. 30 - No AllocateNode or other pointer arithmetic. 31 - We combine the findLessThan, findGreaterOrEqual, etc into one function. 32 */ 33 34 /* 35 Further adapted from Badger: https://github.com/dgraph-io/badger. 36 37 Key differences: 38 - Support for previous pointers - doubly linked lists. Note that it's up to higher 39 level code to deal with the intermediate state that occurs during insertion, 40 where node A is linked to node B, but node B is not yet linked back to node A. 41 - Iterator includes mutator functions. 42 */ 43 44 package arenaskl // import "github.com/cockroachdb/pebble/internal/arenaskl" 45 46 import ( 47 "math" 48 "runtime" 49 "sync/atomic" 50 "unsafe" 51 52 "github.com/cockroachdb/errors" 53 "github.com/cockroachdb/pebble/internal/base" 54 "github.com/cockroachdb/pebble/internal/fastrand" 55 ) 56 57 const ( 58 maxHeight = 20 59 maxNodeSize = int(unsafe.Sizeof(node{})) 60 linksSize = int(unsafe.Sizeof(links{})) 61 pValue = 1 / math.E 62 ) 63 64 // ErrRecordExists indicates that an entry with the specified key already 65 // exists in the skiplist. Duplicate entries are not directly supported and 66 // instead must be handled by the user by appending a unique version suffix to 67 // keys. 68 var ErrRecordExists = errors.New("record with this key already exists") 69 70 // Skiplist is a fast, concurrent skiplist implementation that supports forward 71 // and backward iteration. See batchskl.Skiplist for a non-concurrent 72 // skiplist. Keys and values are immutable once added to the skiplist and 73 // deletion is not supported. Instead, higher-level code is expected to add new 74 // entries that shadow existing entries and perform deletion via tombstones. It 75 // is up to the user to process these shadow entries and tombstones 76 // appropriately during retrieval. 77 type Skiplist struct { 78 arena *Arena 79 cmp base.Compare 80 head *node 81 tail *node 82 height atomic.Uint32 // Current height. 1 <= height <= maxHeight. CAS. 83 84 // If set to true by tests, then extra delays are added to make it easier to 85 // detect unusual race conditions. 86 testing bool 87 } 88 89 // Inserter TODO(peter) 90 type Inserter struct { 91 spl [maxHeight]splice 92 height uint32 93 } 94 95 // Add TODO(peter) 96 func (ins *Inserter) Add(list *Skiplist, key base.InternalKey, value []byte) error { 97 return list.addInternal(key, value, ins) 98 } 99 100 var ( 101 probabilities [maxHeight]uint32 102 ) 103 104 func init() { 105 // Precompute the skiplist probabilities so that only a single random number 106 // needs to be generated and so that the optimal pvalue can be used (inverse 107 // of Euler's number). 108 p := float64(1.0) 109 for i := 0; i < maxHeight; i++ { 110 probabilities[i] = uint32(float64(math.MaxUint32) * p) 111 p *= pValue 112 } 113 } 114 115 // NewSkiplist constructs and initializes a new, empty skiplist. All nodes, keys, 116 // and values in the skiplist will be allocated from the given arena. 117 func NewSkiplist(arena *Arena, cmp base.Compare) *Skiplist { 118 skl := &Skiplist{} 119 skl.Reset(arena, cmp) 120 return skl 121 } 122 123 // Reset the skiplist to empty and re-initialize. 124 func (s *Skiplist) Reset(arena *Arena, cmp base.Compare) { 125 // Allocate head and tail nodes. 126 head, err := newRawNode(arena, maxHeight, 0, 0) 127 if err != nil { 128 panic("arenaSize is not large enough to hold the head node") 129 } 130 head.keyOffset = 0 131 132 tail, err := newRawNode(arena, maxHeight, 0, 0) 133 if err != nil { 134 panic("arenaSize is not large enough to hold the tail node") 135 } 136 tail.keyOffset = 0 137 138 // Link all head/tail levels together. 139 headOffset := arena.getPointerOffset(unsafe.Pointer(head)) 140 tailOffset := arena.getPointerOffset(unsafe.Pointer(tail)) 141 for i := 0; i < maxHeight; i++ { 142 head.tower[i].nextOffset.Store(tailOffset) 143 tail.tower[i].prevOffset.Store(headOffset) 144 } 145 146 *s = Skiplist{ 147 arena: arena, 148 cmp: cmp, 149 head: head, 150 tail: tail, 151 } 152 s.height.Store(1) 153 } 154 155 // Height returns the height of the highest tower within any of the nodes that 156 // have ever been allocated as part of this skiplist. 157 func (s *Skiplist) Height() uint32 { return s.height.Load() } 158 159 // Arena returns the arena backing this skiplist. 160 func (s *Skiplist) Arena() *Arena { return s.arena } 161 162 // Size returns the number of bytes that have allocated from the arena. 163 func (s *Skiplist) Size() uint32 { return s.arena.Size() } 164 165 // Add adds a new key if it does not yet exist. If the key already exists, then 166 // Add returns ErrRecordExists. If there isn't enough room in the arena, then 167 // Add returns ErrArenaFull. 168 func (s *Skiplist) Add(key base.InternalKey, value []byte) error { 169 var ins Inserter 170 return s.addInternal(key, value, &ins) 171 } 172 173 func (s *Skiplist) addInternal(key base.InternalKey, value []byte, ins *Inserter) error { 174 if s.findSplice(key, ins) { 175 // Found a matching node, but handle case where it's been deleted. 176 return ErrRecordExists 177 } 178 179 if s.testing { 180 // Add delay to make it easier to test race between this thread 181 // and another thread that sees the intermediate state between 182 // finding the splice and using it. 183 runtime.Gosched() 184 } 185 186 nd, height, err := s.newNode(key, value) 187 if err != nil { 188 return err 189 } 190 191 ndOffset := s.arena.getPointerOffset(unsafe.Pointer(nd)) 192 193 // We always insert from the base level and up. After you add a node in base 194 // level, we cannot create a node in the level above because it would have 195 // discovered the node in the base level. 196 var found bool 197 var invalidateSplice bool 198 for i := 0; i < int(height); i++ { 199 prev := ins.spl[i].prev 200 next := ins.spl[i].next 201 202 if prev == nil { 203 // New node increased the height of the skiplist, so assume that the 204 // new level has not yet been populated. 205 if next != nil { 206 panic("next is expected to be nil, since prev is nil") 207 } 208 209 prev = s.head 210 next = s.tail 211 } 212 213 // +----------------+ +------------+ +----------------+ 214 // | prev | | nd | | next | 215 // | prevNextOffset |---->| | | | 216 // | |<----| prevOffset | | | 217 // | | | nextOffset |---->| | 218 // | | | |<----| nextPrevOffset | 219 // +----------------+ +------------+ +----------------+ 220 // 221 // 1. Initialize prevOffset and nextOffset to point to prev and next. 222 // 2. CAS prevNextOffset to repoint from next to nd. 223 // 3. CAS nextPrevOffset to repoint from prev to nd. 224 for { 225 prevOffset := s.arena.getPointerOffset(unsafe.Pointer(prev)) 226 nextOffset := s.arena.getPointerOffset(unsafe.Pointer(next)) 227 nd.tower[i].init(prevOffset, nextOffset) 228 229 // Check whether next has an updated link to prev. If it does not, 230 // that can mean one of two things: 231 // 1. The thread that added the next node hasn't yet had a chance 232 // to add the prev link (but will shortly). 233 // 2. Another thread has added a new node between prev and next. 234 nextPrevOffset := next.prevOffset(i) 235 if nextPrevOffset != prevOffset { 236 // Determine whether #1 or #2 is true by checking whether prev 237 // is still pointing to next. As long as the atomic operations 238 // have at least acquire/release semantics (no need for 239 // sequential consistency), this works, as it is equivalent to 240 // the "publication safety" pattern. 241 prevNextOffset := prev.nextOffset(i) 242 if prevNextOffset == nextOffset { 243 // Ok, case #1 is true, so help the other thread along by 244 // updating the next node's prev link. 245 next.casPrevOffset(i, nextPrevOffset, prevOffset) 246 } 247 } 248 249 if prev.casNextOffset(i, nextOffset, ndOffset) { 250 // Managed to insert nd between prev and next, so update the next 251 // node's prev link and go to the next level. 252 if s.testing { 253 // Add delay to make it easier to test race between this thread 254 // and another thread that sees the intermediate state between 255 // setting next and setting prev. 256 runtime.Gosched() 257 } 258 259 next.casPrevOffset(i, prevOffset, ndOffset) 260 break 261 } 262 263 // CAS failed. We need to recompute prev and next. It is unlikely to 264 // be helpful to try to use a different level as we redo the search, 265 // because it is unlikely that lots of nodes are inserted between prev 266 // and next. 267 prev, next, found = s.findSpliceForLevel(key, i, prev) 268 if found { 269 if i != 0 { 270 panic("how can another thread have inserted a node at a non-base level?") 271 } 272 273 return ErrRecordExists 274 } 275 invalidateSplice = true 276 } 277 } 278 279 // If we had to recompute the splice for a level, invalidate the entire 280 // cached splice. 281 if invalidateSplice { 282 ins.height = 0 283 } else { 284 // The splice was valid. We inserted a node between spl[i].prev and 285 // spl[i].next. Optimistically update spl[i].prev for use in a subsequent 286 // call to add. 287 for i := uint32(0); i < height; i++ { 288 ins.spl[i].prev = nd 289 } 290 } 291 292 return nil 293 } 294 295 // NewIter returns a new Iterator object. The lower and upper bound parameters 296 // control the range of keys the iterator will return. Specifying for nil for 297 // lower or upper bound disables the check for that boundary. Note that lower 298 // bound is not checked on {SeekGE,First} and upper bound is not check on 299 // {SeekLT,Last}. The user is expected to perform that check. Note that it is 300 // safe for an iterator to be copied by value. 301 func (s *Skiplist) NewIter(lower, upper []byte) *Iterator { 302 it := iterPool.Get().(*Iterator) 303 *it = Iterator{list: s, nd: s.head, lower: lower, upper: upper} 304 return it 305 } 306 307 // NewFlushIter returns a new flushIterator, which is similar to an Iterator 308 // but also sets the current number of the bytes that have been iterated 309 // through. 310 func (s *Skiplist) NewFlushIter(bytesFlushed *uint64) base.InternalIterator { 311 return &flushIterator{ 312 Iterator: Iterator{list: s, nd: s.head}, 313 bytesIterated: bytesFlushed, 314 } 315 } 316 317 func (s *Skiplist) newNode( 318 key base.InternalKey, value []byte, 319 ) (nd *node, height uint32, err error) { 320 height = s.randomHeight() 321 nd, err = newNode(s.arena, height, key, value) 322 if err != nil { 323 return 324 } 325 326 // Try to increase s.height via CAS. 327 listHeight := s.Height() 328 for height > listHeight { 329 if s.height.CompareAndSwap(listHeight, height) { 330 // Successfully increased skiplist.height. 331 break 332 } 333 334 listHeight = s.Height() 335 } 336 337 return 338 } 339 340 func (s *Skiplist) randomHeight() uint32 { 341 rnd := fastrand.Uint32() 342 343 h := uint32(1) 344 for h < maxHeight && rnd <= probabilities[h] { 345 h++ 346 } 347 348 return h 349 } 350 351 func (s *Skiplist) findSplice(key base.InternalKey, ins *Inserter) (found bool) { 352 listHeight := s.Height() 353 var level int 354 355 prev := s.head 356 if ins.height < listHeight { 357 // Our cached height is less than the list height, which means there were 358 // inserts that increased the height of the list. Recompute the splice from 359 // scratch. 360 ins.height = listHeight 361 level = int(ins.height) 362 } else { 363 // Our cached height is equal to the list height. 364 for ; level < int(listHeight); level++ { 365 spl := &ins.spl[level] 366 if s.getNext(spl.prev, level) != spl.next { 367 // One or more nodes have been inserted between the splice at this 368 // level. 369 continue 370 } 371 if spl.prev != s.head && !s.keyIsAfterNode(spl.prev, key) { 372 // Key lies before splice. 373 level = int(listHeight) 374 break 375 } 376 if spl.next != s.tail && s.keyIsAfterNode(spl.next, key) { 377 // Key lies after splice. 378 level = int(listHeight) 379 break 380 } 381 // The splice brackets the key! 382 prev = spl.prev 383 break 384 } 385 } 386 387 for level = level - 1; level >= 0; level-- { 388 var next *node 389 prev, next, found = s.findSpliceForLevel(key, level, prev) 390 if next == nil { 391 next = s.tail 392 } 393 ins.spl[level].init(prev, next) 394 } 395 396 return 397 } 398 399 func (s *Skiplist) findSpliceForLevel( 400 key base.InternalKey, level int, start *node, 401 ) (prev, next *node, found bool) { 402 prev = start 403 404 for { 405 // Assume prev.key < key. 406 next = s.getNext(prev, level) 407 if next == s.tail { 408 // Tail node, so done. 409 break 410 } 411 412 offset, size := next.keyOffset, next.keySize 413 nextKey := s.arena.buf[offset : offset+size] 414 cmp := s.cmp(key.UserKey, nextKey) 415 if cmp < 0 { 416 // We are done for this level, since prev.key < key < next.key. 417 break 418 } 419 if cmp == 0 { 420 // User-key equality. 421 if key.Trailer == next.keyTrailer { 422 // Internal key equality. 423 found = true 424 break 425 } 426 if key.Trailer > next.keyTrailer { 427 // We are done for this level, since prev.key < key < next.key. 428 break 429 } 430 } 431 432 // Keep moving right on this level. 433 prev = next 434 } 435 436 return 437 } 438 439 func (s *Skiplist) keyIsAfterNode(nd *node, key base.InternalKey) bool { 440 ndKey := s.arena.buf[nd.keyOffset : nd.keyOffset+nd.keySize] 441 cmp := s.cmp(ndKey, key.UserKey) 442 if cmp < 0 { 443 return true 444 } 445 if cmp > 0 { 446 return false 447 } 448 // User-key equality. 449 if key.Trailer == nd.keyTrailer { 450 // Internal key equality. 451 return false 452 } 453 return key.Trailer < nd.keyTrailer 454 } 455 456 func (s *Skiplist) getNext(nd *node, h int) *node { 457 offset := nd.tower[h].nextOffset.Load() 458 return (*node)(s.arena.getPointer(offset)) 459 } 460 461 func (s *Skiplist) getPrev(nd *node, h int) *node { 462 offset := nd.tower[h].prevOffset.Load() 463 return (*node)(s.arena.getPointer(offset)) 464 }