github.com/petermattis/pebble@v0.0.0-20190905164901-ab51a2166067/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/petermattis/pebble/internal/arenaskl" 45 46 import ( 47 "encoding/binary" 48 "errors" 49 "math" 50 "runtime" 51 "sync" 52 "sync/atomic" 53 "time" 54 "unsafe" 55 56 "github.com/petermattis/pebble/internal/base" 57 "golang.org/x/exp/rand" 58 ) 59 60 const ( 61 maxHeight = 20 62 maxNodeSize = int(unsafe.Sizeof(node{})) 63 linksSize = int(unsafe.Sizeof(links{})) 64 pValue = 1 / math.E 65 ) 66 67 // ErrRecordExists indicates that an entry with the specified key already 68 // exists in the skiplist. Duplicate entries are not directly supported and 69 // instead must be handled by the user by appending a unique version suffix to 70 // keys. 71 var ErrRecordExists = errors.New("record with this key already exists") 72 73 // Skiplist is a fast, cocnurrent skiplist implementation that supports forward 74 // and backward iteration. See batchskl.Skiplist for a non-concurrent 75 // skiplist. Keys and values are immutable once added to the skiplist and 76 // deletion is not supported. Instead, higher-level code is expected to add new 77 // entries that shadow existing entries and perform deletion via tombstones. It 78 // is up to the user to process these shadow entries and tombstones 79 // appropriately during retrieval. 80 type Skiplist struct { 81 arena *Arena 82 cmp base.Compare 83 head *node 84 tail *node 85 height uint32 // Current height. 1 <= height <= maxHeight. CAS. 86 87 rand struct { 88 sync.Mutex 89 src rand.PCGSource 90 } 91 92 // If set to true by tests, then extra delays are added to make it easier to 93 // detect unusual race conditions. 94 testing bool 95 } 96 97 // Inserter TODO(peter) 98 type Inserter struct { 99 spl [maxHeight]splice 100 height uint32 101 } 102 103 // Add TODO(peter) 104 func (ins *Inserter) Add(list *Skiplist, key base.InternalKey, value []byte) error { 105 return list.addInternal(key, value, ins) 106 } 107 108 var ( 109 probabilities [maxHeight]uint32 110 ) 111 112 func init() { 113 // Precompute the skiplist probabilities so that only a single random number 114 // needs to be generated and so that the optimal pvalue can be used (inverse 115 // of Euler's number). 116 p := float64(1.0) 117 for i := 0; i < maxHeight; i++ { 118 probabilities[i] = uint32(float64(math.MaxUint32) * p) 119 p *= pValue 120 } 121 } 122 123 // NewSkiplist constructs and initializes a new, empty skiplist. All nodes, keys, 124 // and values in the skiplist will be allocated from the given arena. 125 func NewSkiplist(arena *Arena, cmp base.Compare) *Skiplist { 126 skl := &Skiplist{} 127 skl.Reset(arena, cmp) 128 return skl 129 } 130 131 // Reset the skiplist to empty and re-initialize. 132 func (s *Skiplist) Reset(arena *Arena, cmp base.Compare) { 133 // Allocate head and tail nodes. 134 head, err := newRawNode(arena, maxHeight, 0, 0) 135 if err != nil { 136 panic("arenaSize is not large enough to hold the head node") 137 } 138 head.keyOffset = 0 139 140 tail, err := newRawNode(arena, maxHeight, 0, 0) 141 if err != nil { 142 panic("arenaSize is not large enough to hold the tail node") 143 } 144 tail.keyOffset = 0 145 146 // Link all head/tail levels together. 147 headOffset := arena.getPointerOffset(unsafe.Pointer(head)) 148 tailOffset := arena.getPointerOffset(unsafe.Pointer(tail)) 149 for i := 0; i < maxHeight; i++ { 150 head.tower[i].nextOffset = tailOffset 151 tail.tower[i].prevOffset = headOffset 152 } 153 154 *s = Skiplist{ 155 arena: arena, 156 cmp: cmp, 157 head: head, 158 tail: tail, 159 height: 1, 160 } 161 s.rand.src.Seed(uint64(time.Now().UnixNano())) 162 } 163 164 // Height returns the height of the highest tower within any of the nodes that 165 // have ever been allocated as part of this skiplist. 166 func (s *Skiplist) Height() uint32 { return atomic.LoadUint32(&s.height) } 167 168 // Arena returns the arena backing this skiplist. 169 func (s *Skiplist) Arena() *Arena { return s.arena } 170 171 // Size returns the number of bytes that have allocated from the arena. 172 func (s *Skiplist) Size() uint32 { return s.arena.Size() } 173 174 // Add adds a new key if it does not yet exist. If the key already exists, then 175 // Add returns ErrRecordExists. If there isn't enough room in the arena, then 176 // Add returns ErrArenaFull. 177 func (s *Skiplist) Add(key base.InternalKey, value []byte) error { 178 var ins Inserter 179 return s.addInternal(key, value, &ins) 180 } 181 182 func (s *Skiplist) addInternal(key base.InternalKey, value []byte, ins *Inserter) error { 183 if s.findSplice(key, ins) { 184 // Found a matching node, but handle case where it's been deleted. 185 return ErrRecordExists 186 } 187 188 if s.testing { 189 // Add delay to make it easier to test race between this thread 190 // and another thread that sees the intermediate state between 191 // finding the splice and using it. 192 runtime.Gosched() 193 } 194 195 nd, height, err := s.newNode(key, value) 196 if err != nil { 197 return err 198 } 199 200 ndOffset := s.arena.getPointerOffset(unsafe.Pointer(nd)) 201 202 // We always insert from the base level and up. After you add a node in base 203 // level, we cannot create a node in the level above because it would have 204 // discovered the node in the base level. 205 var found bool 206 var invalidateSplice bool 207 for i := 0; i < int(height); i++ { 208 prev := ins.spl[i].prev 209 next := ins.spl[i].next 210 211 if prev == nil { 212 // New node increased the height of the skiplist, so assume that the 213 // new level has not yet been populated. 214 if next != nil { 215 panic("next is expected to be nil, since prev is nil") 216 } 217 218 prev = s.head 219 next = s.tail 220 } 221 222 // +----------------+ +------------+ +----------------+ 223 // | prev | | nd | | next | 224 // | prevNextOffset |---->| | | | 225 // | |<----| prevOffset | | | 226 // | | | nextOffset |---->| | 227 // | | | |<----| nextPrevOffset | 228 // +----------------+ +------------+ +----------------+ 229 // 230 // 1. Initialize prevOffset and nextOffset to point to prev and next. 231 // 2. CAS prevNextOffset to repoint from next to nd. 232 // 3. CAS nextPrevOffset to repoint from prev to nd. 233 for { 234 prevOffset := s.arena.getPointerOffset(unsafe.Pointer(prev)) 235 nextOffset := s.arena.getPointerOffset(unsafe.Pointer(next)) 236 nd.tower[i].init(prevOffset, nextOffset) 237 238 // Check whether next has an updated link to prev. If it does not, 239 // that can mean one of two things: 240 // 1. The thread that added the next node hasn't yet had a chance 241 // to add the prev link (but will shortly). 242 // 2. Another thread has added a new node between prev and next. 243 nextPrevOffset := next.prevOffset(i) 244 if nextPrevOffset != prevOffset { 245 // Determine whether #1 or #2 is true by checking whether prev 246 // is still pointing to next. As long as the atomic operations 247 // have at least acquire/release semantics (no need for 248 // sequential consistency), this works, as it is equivalent to 249 // the "publication safety" pattern. 250 prevNextOffset := prev.nextOffset(i) 251 if prevNextOffset == nextOffset { 252 // Ok, case #1 is true, so help the other thread along by 253 // updating the next node's prev link. 254 next.casPrevOffset(i, nextPrevOffset, prevOffset) 255 } 256 } 257 258 if prev.casNextOffset(i, nextOffset, ndOffset) { 259 // Managed to insert nd between prev and next, so update the next 260 // node's prev link and go to the next level. 261 if s.testing { 262 // Add delay to make it easier to test race between this thread 263 // and another thread that sees the intermediate state between 264 // setting next and setting prev. 265 runtime.Gosched() 266 } 267 268 next.casPrevOffset(i, prevOffset, ndOffset) 269 break 270 } 271 272 // CAS failed. We need to recompute prev and next. It is unlikely to 273 // be helpful to try to use a different level as we redo the search, 274 // because it is unlikely that lots of nodes are inserted between prev 275 // and next. 276 prev, next, found = s.findSpliceForLevel(key, i, prev) 277 if found { 278 if i != 0 { 279 panic("how can another thread have inserted a node at a non-base level?") 280 } 281 282 return ErrRecordExists 283 } 284 invalidateSplice = true 285 } 286 } 287 288 // If we had to recompute the splice for a level, invalidate the entire 289 // cached splice. 290 if invalidateSplice { 291 ins.height = 0 292 } else { 293 // The splice was valid. We inserted a node between spl[i].prev and 294 // spl[i].next. Optimistically update spl[i].prev for use in a subsequent 295 // call to add. 296 for i := uint32(0); i < height; i++ { 297 ins.spl[i].prev = nd 298 } 299 } 300 301 return nil 302 } 303 304 // NewIter returns a new Iterator object. The lower and upper bound parameters 305 // control the range of keys the iterator will return. Specifying for nil for 306 // lower or upper bound disables the check for that boundary. Note that lower 307 // bound is not checked on {SeekGE,First} and upper bound is not check on 308 // {SeekLT,Last}. The user is expected to perform that check. Note that it is 309 // safe for an iterator to be copied by value. 310 func (s *Skiplist) NewIter(lower, upper []byte) *Iterator { 311 it := iterPool.Get().(*Iterator) 312 *it = Iterator{list: s, nd: s.head, lower: lower, upper: upper} 313 return it 314 } 315 316 // NewFlushIter returns a new flushIterator, which is similar to an Iterator 317 // but also sets the current number of the bytes that have been iterated 318 // through. 319 func (s *Skiplist) NewFlushIter(bytesFlushed *uint64) *flushIterator { 320 return &flushIterator{ 321 Iterator: Iterator{list: s, nd: s.head}, 322 bytesIterated: bytesFlushed, 323 } 324 } 325 326 func (s *Skiplist) newNode( 327 key base.InternalKey, value []byte, 328 ) (nd *node, height uint32, err error) { 329 height = s.randomHeight() 330 nd, err = newNode(s.arena, height, key, value) 331 if err != nil { 332 return 333 } 334 335 // Try to increase s.height via CAS. 336 listHeight := s.Height() 337 for height > listHeight { 338 if atomic.CompareAndSwapUint32(&s.height, listHeight, height) { 339 // Successfully increased skiplist.height. 340 break 341 } 342 343 listHeight = s.Height() 344 } 345 346 return 347 } 348 349 func (s *Skiplist) randomHeight() uint32 { 350 s.rand.Lock() 351 rnd := uint32(s.rand.src.Uint64()) 352 s.rand.Unlock() 353 354 h := uint32(1) 355 for h < maxHeight && rnd <= probabilities[h] { 356 h++ 357 } 358 359 return h 360 } 361 362 func (s *Skiplist) findSplice(key base.InternalKey, ins *Inserter) (found bool) { 363 listHeight := s.Height() 364 var level int 365 366 prev, next := s.head, (*node)(nil) 367 if ins.height < listHeight { 368 // Our cached height is less than the list height, which means there were 369 // inserts that increased the height of the list. Recompute the splice from 370 // scratch. 371 ins.height = listHeight 372 level = int(ins.height) 373 } else { 374 // Our cached height is equal to the list height. 375 for ; level < int(listHeight); level++ { 376 spl := &ins.spl[level] 377 if s.getNext(spl.prev, level) != spl.next { 378 // One or more nodes have been inserted between the splice at this 379 // level. 380 continue 381 } 382 if spl.prev != s.head && !s.keyIsAfterNode(spl.prev, key) { 383 // Key lies before splice. 384 level = int(listHeight) 385 break 386 } 387 if spl.next != s.tail && s.keyIsAfterNode(spl.next, key) { 388 // Key lies after splice. 389 level = int(listHeight) 390 break 391 } 392 // The splice brackets the key! 393 prev, next = spl.prev, spl.next 394 break 395 } 396 } 397 398 for level = level - 1; level >= 0; level-- { 399 prev, next, found = s.findSpliceForLevel(key, level, prev) 400 if next == nil { 401 next = s.tail 402 } 403 ins.spl[level].init(prev, next) 404 } 405 406 return 407 } 408 409 func (s *Skiplist) findSpliceForLevel( 410 key base.InternalKey, level int, start *node, 411 ) (prev, next *node, found bool) { 412 prev = start 413 414 for { 415 // Assume prev.key < key. 416 next = s.getNext(prev, level) 417 if next == s.tail { 418 // Tail node, so done. 419 break 420 } 421 422 offset, size := next.keyOffset, next.keySize 423 nextKey := s.arena.buf[offset : offset+size] 424 n := size - 8 425 cmp := s.cmp(key.UserKey, nextKey[:n]) 426 if cmp < 0 { 427 // We are done for this level, since prev.key < key < next.key. 428 break 429 } 430 if cmp == 0 { 431 // User-key equality. 432 var nextTrailer uint64 433 if n >= 0 { 434 nextTrailer = binary.LittleEndian.Uint64(nextKey[n:]) 435 } else { 436 nextTrailer = uint64(base.InternalKeyKindInvalid) 437 } 438 if key.Trailer == nextTrailer { 439 // Internal key equality. 440 found = true 441 break 442 } 443 if key.Trailer > nextTrailer { 444 // We are done for this level, since prev.key < key < next.key. 445 break 446 } 447 } 448 449 // Keep moving right on this level. 450 prev = next 451 } 452 453 return 454 } 455 456 func (s *Skiplist) keyIsAfterNode(nd *node, key base.InternalKey) bool { 457 ndKey := s.arena.buf[nd.keyOffset : nd.keyOffset+nd.keySize] 458 n := nd.keySize - 8 459 cmp := s.cmp(ndKey[:n], key.UserKey) 460 if cmp < 0 { 461 return true 462 } 463 if cmp > 0 { 464 return false 465 } 466 // User-key equality. 467 var ndTrailer uint64 468 if n >= 0 { 469 ndTrailer = binary.LittleEndian.Uint64(ndKey[n:]) 470 } else { 471 ndTrailer = uint64(base.InternalKeyKindInvalid) 472 } 473 if key.Trailer == ndTrailer { 474 // Internal key equality. 475 return false 476 } 477 return key.Trailer < ndTrailer 478 } 479 480 func (s *Skiplist) getNext(nd *node, h int) *node { 481 offset := atomic.LoadUint32(&nd.tower[h].nextOffset) 482 return (*node)(s.arena.getPointer(offset)) 483 } 484 485 func (s *Skiplist) getPrev(nd *node, h int) *node { 486 offset := atomic.LoadUint32(&nd.tower[h].prevOffset) 487 return (*node)(s.arena.getPointer(offset)) 488 }