github.com/cockroachdb/pebble@v0.0.0-20231214172447-ab4952c5f87b/compaction_iter.go (about) 1 // Copyright 2018 The LevelDB-Go and Pebble Authors. All rights reserved. Use 2 // of this source code is governed by a BSD-style license that can be found in 3 // the LICENSE file. 4 5 package pebble 6 7 import ( 8 "bytes" 9 "encoding/binary" 10 "fmt" 11 "io" 12 "sort" 13 "strconv" 14 15 "github.com/cockroachdb/errors" 16 "github.com/cockroachdb/pebble/internal/base" 17 "github.com/cockroachdb/pebble/internal/bytealloc" 18 "github.com/cockroachdb/pebble/internal/keyspan" 19 "github.com/cockroachdb/pebble/internal/rangekey" 20 "github.com/cockroachdb/redact" 21 ) 22 23 // compactionIter provides a forward-only iterator that encapsulates the logic 24 // for collapsing entries during compaction. It wraps an internal iterator and 25 // collapses entries that are no longer necessary because they are shadowed by 26 // newer entries. The simplest example of this is when the internal iterator 27 // contains two keys: a.PUT.2 and a.PUT.1. Instead of returning both entries, 28 // compactionIter collapses the second entry because it is no longer 29 // necessary. The high-level structure for compactionIter is to iterate over 30 // its internal iterator and output 1 entry for every user-key. There are four 31 // complications to this story. 32 // 33 // 1. Eliding Deletion Tombstones 34 // 35 // Consider the entries a.DEL.2 and a.PUT.1. These entries collapse to 36 // a.DEL.2. Do we have to output the entry a.DEL.2? Only if a.DEL.2 possibly 37 // shadows an entry at a lower level. If we're compacting to the base-level in 38 // the LSM tree then a.DEL.2 is definitely not shadowing an entry at a lower 39 // level and can be elided. 40 // 41 // We can do slightly better than only eliding deletion tombstones at the base 42 // level by observing that we can elide a deletion tombstone if there are no 43 // sstables that contain the entry's key. This check is performed by 44 // elideTombstone. 45 // 46 // 2. Merges 47 // 48 // The MERGE operation merges the value for an entry with the existing value 49 // for an entry. The logical value of an entry can be composed of a series of 50 // merge operations. When compactionIter sees a MERGE, it scans forward in its 51 // internal iterator collapsing MERGE operations for the same key until it 52 // encounters a SET or DELETE operation. For example, the keys a.MERGE.4, 53 // a.MERGE.3, a.MERGE.2 will be collapsed to a.MERGE.4 and the values will be 54 // merged using the specified Merger. 55 // 56 // An interesting case here occurs when MERGE is combined with SET. Consider 57 // the entries a.MERGE.3 and a.SET.2. The collapsed key will be a.SET.3. The 58 // reason that the kind is changed to SET is because the SET operation acts as 59 // a barrier preventing further merging. This can be seen better in the 60 // scenario a.MERGE.3, a.SET.2, a.MERGE.1. The entry a.MERGE.1 may be at lower 61 // (older) level and not involved in the compaction. If the compaction of 62 // a.MERGE.3 and a.SET.2 produced a.MERGE.3, a subsequent compaction with 63 // a.MERGE.1 would merge the values together incorrectly. 64 // 65 // 3. Snapshots 66 // 67 // Snapshots are lightweight point-in-time views of the DB state. At its core, 68 // a snapshot is a sequence number along with a guarantee from Pebble that it 69 // will maintain the view of the database at that sequence number. Part of this 70 // guarantee is relatively straightforward to achieve. When reading from the 71 // database Pebble will ignore sequence numbers that are larger than the 72 // snapshot sequence number. The primary complexity with snapshots occurs 73 // during compaction: the collapsing of entries that are shadowed by newer 74 // entries is at odds with the guarantee that Pebble will maintain the view of 75 // the database at the snapshot sequence number. Rather than collapsing entries 76 // up to the next user key, compactionIter can only collapse entries up to the 77 // next snapshot boundary. That is, every snapshot boundary potentially causes 78 // another entry for the same user-key to be emitted. Another way to view this 79 // is that snapshots define stripes and entries are collapsed within stripes, 80 // but not across stripes. Consider the following scenario: 81 // 82 // a.PUT.9 83 // a.DEL.8 84 // a.PUT.7 85 // a.DEL.6 86 // a.PUT.5 87 // 88 // In the absence of snapshots these entries would be collapsed to 89 // a.PUT.9. What if there is a snapshot at sequence number 7? The entries can 90 // be divided into two stripes and collapsed within the stripes: 91 // 92 // a.PUT.9 a.PUT.9 93 // a.DEL.8 ---> 94 // a.PUT.7 95 // -- -- 96 // a.DEL.6 ---> a.DEL.6 97 // a.PUT.5 98 // 99 // All of the rules described earlier still apply, but they are confined to 100 // operate within a snapshot stripe. Snapshots only affect compaction when the 101 // snapshot sequence number lies within the range of sequence numbers being 102 // compacted. In the above example, a snapshot at sequence number 10 or at 103 // sequence number 5 would not have any effect. 104 // 105 // 4. Range Deletions 106 // 107 // Range deletions provide the ability to delete all of the keys (and values) 108 // in a contiguous range. Range deletions are stored indexed by their start 109 // key. The end key of the range is stored in the value. In order to support 110 // lookup of the range deletions which overlap with a particular key, the range 111 // deletion tombstones need to be fragmented whenever they overlap. This 112 // fragmentation is performed by keyspan.Fragmenter. The fragments are then 113 // subject to the rules for snapshots. For example, consider the two range 114 // tombstones [a,e)#1 and [c,g)#2: 115 // 116 // 2: c-------g 117 // 1: a-------e 118 // 119 // These tombstones will be fragmented into: 120 // 121 // 2: c---e---g 122 // 1: a---c---e 123 // 124 // Do we output the fragment [c,e)#1? Since it is covered by [c-e]#2 the answer 125 // depends on whether it is in a new snapshot stripe. 126 // 127 // In addition to the fragmentation of range tombstones, compaction also needs 128 // to take the range tombstones into consideration when outputting normal 129 // keys. Just as with point deletions, a range deletion covering an entry can 130 // cause the entry to be elided. 131 // 132 // A note on the stability of keys and values. 133 // 134 // The stability guarantees of keys and values returned by the iterator tree 135 // that backs a compactionIter is nuanced and care must be taken when 136 // referencing any returned items. 137 // 138 // Keys and values returned by exported functions (i.e. First, Next, etc.) have 139 // lifetimes that fall into two categories: 140 // 141 // Lifetime valid for duration of compaction. Range deletion keys and values are 142 // stable for the duration of the compaction, due to way in which a 143 // compactionIter is typically constructed (i.e. via (*compaction).newInputIter, 144 // which wraps the iterator over the range deletion block in a noCloseIter, 145 // preventing the release of the backing memory until the compaction is 146 // finished). 147 // 148 // Lifetime limited to duration of sstable block liveness. Point keys (SET, DEL, 149 // etc.) and values must be cloned / copied following the return from the 150 // exported function, and before a subsequent call to Next advances the iterator 151 // and mutates the contents of the returned key and value. 152 type compactionIter struct { 153 equal Equal 154 merge Merge 155 iter internalIterator 156 err error 157 // `key.UserKey` is set to `keyBuf` caused by saving `i.iterKey.UserKey` 158 // and `key.Trailer` is set to `i.iterKey.Trailer`. This is the 159 // case on return from all public methods -- these methods return `key`. 160 // Additionally, it is the internal state when the code is moving to the 161 // next key so it can determine whether the user key has changed from 162 // the previous key. 163 key InternalKey 164 // keyTrailer is updated when `i.key` is updated and holds the key's 165 // original trailer (eg, before any sequence-number zeroing or changes to 166 // key kind). 167 keyTrailer uint64 168 value []byte 169 valueCloser io.Closer 170 // Temporary buffer used for storing the previous user key in order to 171 // determine when iteration has advanced to a new user key and thus a new 172 // snapshot stripe. 173 keyBuf []byte 174 // Temporary buffer used for storing the previous value, which may be an 175 // unsafe, i.iter-owned slice that could be altered when the iterator is 176 // advanced. 177 valueBuf []byte 178 // Is the current entry valid? 179 valid bool 180 iterKey *InternalKey 181 iterValue []byte 182 iterStripeChange stripeChangeType 183 // `skip` indicates whether the remaining skippable entries in the current 184 // snapshot stripe should be skipped or processed. An example of a non- 185 // skippable entry is a range tombstone as we need to return it from the 186 // `compactionIter`, even if a key covering its start key has already been 187 // seen in the same stripe. `skip` has no effect when `pos == iterPosNext`. 188 // 189 // TODO(jackson): If we use keyspan.InterleavingIter for range deletions, 190 // like we do for range keys, the only remaining 'non-skippable' key is 191 // the invalid key. We should be able to simplify this logic and remove this 192 // field. 193 skip bool 194 // `pos` indicates the iterator position at the top of `Next()`. Its type's 195 // (`iterPos`) values take on the following meanings in the context of 196 // `compactionIter`. 197 // 198 // - `iterPosCur`: the iterator is at the last key returned. 199 // - `iterPosNext`: the iterator has already been advanced to the next 200 // candidate key. For example, this happens when processing merge operands, 201 // where we advance the iterator all the way into the next stripe or next 202 // user key to ensure we've seen all mergeable operands. 203 // - `iterPosPrev`: this is invalid as compactionIter is forward-only. 204 pos iterPos 205 // `snapshotPinned` indicates whether the last point key returned by the 206 // compaction iterator was only returned because an open snapshot prevents 207 // its elision. This field only applies to point keys, and not to range 208 // deletions or range keys. 209 // 210 // For MERGE, it is possible that doing the merge is interrupted even when 211 // the next point key is in the same stripe. This can happen if the loop in 212 // mergeNext gets interrupted by sameStripeNonSkippable. 213 // sameStripeNonSkippable occurs due to RANGEDELs that sort before 214 // SET/MERGE/DEL with the same seqnum, so the RANGEDEL does not necessarily 215 // delete the subsequent SET/MERGE/DEL keys. 216 snapshotPinned bool 217 // forceObsoleteDueToRangeDel is set to true in a subset of the cases that 218 // snapshotPinned is true. This value is true when the point is obsolete due 219 // to a RANGEDEL but could not be deleted due to a snapshot. 220 // 221 // NB: it may seem that the additional cases that snapshotPinned captures 222 // are harmless in that they can also be used to mark a point as obsolete 223 // (it is merely a duplication of some logic that happens in 224 // Writer.AddWithForceObsolete), but that is not quite accurate as of this 225 // writing -- snapshotPinned originated in stats collection and for a 226 // sequence MERGE, SET, where the MERGE cannot merge with the (older) SET 227 // due to a snapshot, the snapshotPinned value for the SET is true. 228 // 229 // TODO(sumeer,jackson): improve the logic of snapshotPinned and reconsider 230 // whether we need forceObsoleteDueToRangeDel. 231 forceObsoleteDueToRangeDel bool 232 // The index of the snapshot for the current key within the snapshots slice. 233 curSnapshotIdx int 234 curSnapshotSeqNum uint64 235 // The snapshot sequence numbers that need to be maintained. These sequence 236 // numbers define the snapshot stripes (see the Snapshots description 237 // above). The sequence numbers are in ascending order. 238 snapshots []uint64 239 // frontiers holds a heap of user keys that affect compaction behavior when 240 // they're exceeded. Before a new key is returned, the compaction iterator 241 // advances the frontier, notifying any code that subscribed to be notified 242 // when a key was reached. The primary use today is within the 243 // implementation of compactionOutputSplitters in compaction.go. Many of 244 // these splitters wait for the compaction iterator to call Advance(k) when 245 // it's returning a new key. If the key that they're waiting for is 246 // surpassed, these splitters update internal state recording that they 247 // should request a compaction split next time they're asked in 248 // [shouldSplitBefore]. 249 frontiers frontiers 250 // Reference to the range deletion tombstone fragmenter (e.g., 251 // `compaction.rangeDelFrag`). 252 rangeDelFrag *keyspan.Fragmenter 253 rangeKeyFrag *keyspan.Fragmenter 254 // The fragmented tombstones. 255 tombstones []keyspan.Span 256 // The fragmented range keys. 257 rangeKeys []keyspan.Span 258 // Byte allocator for the tombstone keys. 259 alloc bytealloc.A 260 allowZeroSeqNum bool 261 elideTombstone func(key []byte) bool 262 elideRangeTombstone func(start, end []byte) bool 263 ineffectualSingleDeleteCallback func(userKey []byte) 264 singleDeleteInvariantViolationCallback func(userKey []byte) 265 // The on-disk format major version. This informs the types of keys that 266 // may be written to disk during a compaction. 267 formatVersion FormatMajorVersion 268 stats struct { 269 // count of DELSIZED keys that were missized. 270 countMissizedDels uint64 271 } 272 } 273 274 func newCompactionIter( 275 cmp Compare, 276 equal Equal, 277 formatKey base.FormatKey, 278 merge Merge, 279 iter internalIterator, 280 snapshots []uint64, 281 rangeDelFrag *keyspan.Fragmenter, 282 rangeKeyFrag *keyspan.Fragmenter, 283 allowZeroSeqNum bool, 284 elideTombstone func(key []byte) bool, 285 elideRangeTombstone func(start, end []byte) bool, 286 ineffectualSingleDeleteCallback func(userKey []byte), 287 singleDeleteInvariantViolationCallback func(userKey []byte), 288 formatVersion FormatMajorVersion, 289 ) *compactionIter { 290 i := &compactionIter{ 291 equal: equal, 292 merge: merge, 293 iter: iter, 294 snapshots: snapshots, 295 frontiers: frontiers{cmp: cmp}, 296 rangeDelFrag: rangeDelFrag, 297 rangeKeyFrag: rangeKeyFrag, 298 allowZeroSeqNum: allowZeroSeqNum, 299 elideTombstone: elideTombstone, 300 elideRangeTombstone: elideRangeTombstone, 301 ineffectualSingleDeleteCallback: ineffectualSingleDeleteCallback, 302 singleDeleteInvariantViolationCallback: singleDeleteInvariantViolationCallback, 303 formatVersion: formatVersion, 304 } 305 i.rangeDelFrag.Cmp = cmp 306 i.rangeDelFrag.Format = formatKey 307 i.rangeDelFrag.Emit = i.emitRangeDelChunk 308 i.rangeKeyFrag.Cmp = cmp 309 i.rangeKeyFrag.Format = formatKey 310 i.rangeKeyFrag.Emit = i.emitRangeKeyChunk 311 return i 312 } 313 314 func (i *compactionIter) First() (*InternalKey, []byte) { 315 if i.err != nil { 316 return nil, nil 317 } 318 var iterValue LazyValue 319 i.iterKey, iterValue = i.iter.First() 320 i.iterValue, _, i.err = iterValue.Value(nil) 321 if i.err != nil { 322 return nil, nil 323 } 324 if i.iterKey != nil { 325 i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(i.iterKey.SeqNum(), i.snapshots) 326 } 327 i.pos = iterPosNext 328 i.iterStripeChange = newStripeNewKey 329 return i.Next() 330 } 331 332 func (i *compactionIter) Next() (*InternalKey, []byte) { 333 if i.err != nil { 334 return nil, nil 335 } 336 337 // Close the closer for the current value if one was open. 338 if i.closeValueCloser() != nil { 339 return nil, nil 340 } 341 342 // Prior to this call to `Next()` we are in one of four situations with 343 // respect to `iterKey` and related state: 344 // 345 // - `!skip && pos == iterPosNext`: `iterKey` is already at the next key. 346 // - `!skip && pos == iterPosCurForward`: We are at the key that has been returned. 347 // To move forward we advance by one key, even if that lands us in the same 348 // snapshot stripe. 349 // - `skip && pos == iterPosCurForward`: We are at the key that has been returned. 350 // To move forward we skip skippable entries in the stripe. 351 // - `skip && pos == iterPosNext && i.iterStripeChange == sameStripeNonSkippable`: 352 // This case may occur when skipping within a snapshot stripe and we 353 // encounter either: 354 // a) an invalid key kind; The previous call will have returned 355 // whatever key it was processing and deferred handling of the 356 // invalid key to this invocation of Next(). We're responsible for 357 // ignoring skip=true and falling into the invalid key kind case 358 // down below. 359 // b) an interleaved range delete; This is a wart of the current code 360 // structure. While skipping within a snapshot stripe, a range 361 // delete interleaved at its start key and sequence number 362 // interrupts the sequence of point keys. After we return the range 363 // delete to the caller, we need to pick up skipping at where we 364 // left off, so we preserve skip=true. 365 // TODO(jackson): This last case is confusing and can be removed if we 366 // interleave range deletions at the maximal sequence number using the 367 // keyspan interleaving iterator. This is the treatment given to range 368 // keys today. 369 if i.pos == iterPosCurForward { 370 if i.skip { 371 i.skipInStripe() 372 } else { 373 i.nextInStripe() 374 } 375 } else if i.skip { 376 if i.iterStripeChange != sameStripeNonSkippable { 377 panic(errors.AssertionFailedf("compaction iterator has skip=true, but iterator is at iterPosNext")) 378 } 379 } 380 381 i.pos = iterPosCurForward 382 i.valid = false 383 384 for i.iterKey != nil { 385 // If we entered a new snapshot stripe with the same key, any key we 386 // return on this iteration is only returned because the open snapshot 387 // prevented it from being elided or merged with the key returned for 388 // the previous stripe. Mark it as pinned so that the compaction loop 389 // can correctly populate output tables' pinned statistics. We might 390 // also set snapshotPinned=true down below if we observe that the key is 391 // deleted by a range deletion in a higher stripe or that this key is a 392 // tombstone that could be elided if only it were in the last snapshot 393 // stripe. 394 i.snapshotPinned = i.iterStripeChange == newStripeSameKey 395 396 if i.iterKey.Kind() == InternalKeyKindRangeDelete || rangekey.IsRangeKey(i.iterKey.Kind()) { 397 // Return the span so the compaction can use it for file truncation and add 398 // it to the relevant fragmenter. We do not set `skip` to true before 399 // returning as there may be a forthcoming point key with the same user key 400 // and sequence number. Such a point key must be visible (i.e., not skipped 401 // over) since we promise point keys are not deleted by range tombstones at 402 // the same sequence number. 403 // 404 // Although, note that `skip` may already be true before reaching here 405 // due to an earlier key in the stripe. Then it is fine to leave it set 406 // to true, as the earlier key must have had a higher sequence number. 407 // 408 // NOTE: there is a subtle invariant violation here in that calling 409 // saveKey and returning a reference to the temporary slice violates 410 // the stability guarantee for range deletion keys. A potential 411 // mediation could return the original iterKey and iterValue 412 // directly, as the backing memory is guaranteed to be stable until 413 // the compaction completes. The violation here is only minor in 414 // that the caller immediately clones the range deletion InternalKey 415 // when passing the key to the deletion fragmenter (see the 416 // call-site in compaction.go). 417 // TODO(travers): address this violation by removing the call to 418 // saveKey and instead return the original iterKey and iterValue. 419 // This goes against the comment on i.key in the struct, and 420 // therefore warrants some investigation. 421 i.saveKey() 422 // TODO(jackson): Handle tracking pinned statistics for range keys 423 // and range deletions. This would require updating 424 // emitRangeDelChunk and rangeKeyCompactionTransform to update 425 // statistics when they apply their own snapshot striping logic. 426 i.snapshotPinned = false 427 i.value = i.iterValue 428 i.valid = true 429 return &i.key, i.value 430 } 431 432 // TODO(sumeer): we could avoid calling Covers if i.iterStripeChange == 433 // sameStripeSameKey since that check has already been done in 434 // nextInStripeHelper. However, we also need to handle the case of 435 // CoversInvisibly below. 436 if cover := i.rangeDelFrag.Covers(*i.iterKey, i.curSnapshotSeqNum); cover == keyspan.CoversVisibly { 437 // A pending range deletion deletes this key. Skip it. 438 i.saveKey() 439 i.skipInStripe() 440 continue 441 } else if cover == keyspan.CoversInvisibly { 442 // i.iterKey would be deleted by a range deletion if there weren't 443 // any open snapshots. Mark it as pinned. 444 // 445 // NB: there are multiple places in this file where we call 446 // i.rangeDelFrag.Covers and this is the only one where we are writing 447 // to i.snapshotPinned. Those other cases occur in mergeNext where the 448 // caller is deciding whether the value should be merged or not, and the 449 // key is in the same snapshot stripe. Hence, snapshotPinned is by 450 // definition false in those cases. 451 i.snapshotPinned = true 452 i.forceObsoleteDueToRangeDel = true 453 } else { 454 i.forceObsoleteDueToRangeDel = false 455 } 456 457 switch i.iterKey.Kind() { 458 case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: 459 if i.elideTombstone(i.iterKey.UserKey) { 460 if i.curSnapshotIdx == 0 { 461 // If we're at the last snapshot stripe and the tombstone 462 // can be elided skip skippable keys in the same stripe. 463 i.saveKey() 464 if i.key.Kind() == InternalKeyKindSingleDelete { 465 i.skipDueToSingleDeleteElision() 466 } else { 467 i.skipInStripe() 468 if !i.skip && i.iterStripeChange != newStripeNewKey { 469 panic(errors.AssertionFailedf("pebble: skipInStripe in last stripe disabled skip without advancing to new key")) 470 } 471 } 472 if i.iterStripeChange == newStripeSameKey { 473 panic(errors.AssertionFailedf("pebble: skipInStripe in last stripe found a new stripe within the same key")) 474 } 475 continue 476 } else { 477 // We're not at the last snapshot stripe, so the tombstone 478 // can NOT yet be elided. Mark it as pinned, so that it's 479 // included in table statistics appropriately. 480 i.snapshotPinned = true 481 } 482 } 483 484 switch i.iterKey.Kind() { 485 case InternalKeyKindDelete: 486 i.saveKey() 487 i.value = i.iterValue 488 i.valid = true 489 i.skip = true 490 return &i.key, i.value 491 492 case InternalKeyKindDeleteSized: 493 // We may skip subsequent keys because of this tombstone. Scan 494 // ahead to see just how much data this tombstone drops and if 495 // the tombstone's value should be updated accordingly. 496 return i.deleteSizedNext() 497 498 case InternalKeyKindSingleDelete: 499 if i.singleDeleteNext() { 500 return &i.key, i.value 501 } else if i.err != nil { 502 return nil, nil 503 } 504 continue 505 506 default: 507 panic(errors.AssertionFailedf( 508 "unexpected kind %s", redact.SafeString(i.iterKey.Kind().String()))) 509 } 510 511 case InternalKeyKindSet, InternalKeyKindSetWithDelete: 512 // The key we emit for this entry is a function of the current key 513 // kind, and whether this entry is followed by a DEL/SINGLEDEL 514 // entry. setNext() does the work to move the iterator forward, 515 // preserving the original value, and potentially mutating the key 516 // kind. 517 i.setNext() 518 if i.err != nil { 519 return nil, nil 520 } 521 return &i.key, i.value 522 523 case InternalKeyKindMerge: 524 // Record the snapshot index before mergeNext as merging 525 // advances the iterator, adjusting curSnapshotIdx. 526 origSnapshotIdx := i.curSnapshotIdx 527 var valueMerger ValueMerger 528 valueMerger, i.err = i.merge(i.iterKey.UserKey, i.iterValue) 529 var change stripeChangeType 530 if i.err == nil { 531 change = i.mergeNext(valueMerger) 532 } 533 var needDelete bool 534 if i.err == nil { 535 // includesBase is true whenever we've transformed the MERGE record 536 // into a SET. 537 var includesBase bool 538 switch i.key.Kind() { 539 case InternalKeyKindSet, InternalKeyKindSetWithDelete: 540 includesBase = true 541 case InternalKeyKindMerge: 542 default: 543 panic(errors.AssertionFailedf( 544 "unexpected kind %s", redact.SafeString(i.key.Kind().String()))) 545 } 546 i.value, needDelete, i.valueCloser, i.err = finishValueMerger(valueMerger, includesBase) 547 } 548 if i.err == nil { 549 if needDelete { 550 i.valid = false 551 if i.closeValueCloser() != nil { 552 return nil, nil 553 } 554 continue 555 } 556 // A non-skippable entry does not necessarily cover later merge 557 // operands, so we must not zero the current merge result's seqnum. 558 // 559 // For example, suppose the forthcoming two keys are a range 560 // tombstone, `[a, b)#3`, and a merge operand, `a#3`. Recall that 561 // range tombstones do not cover point keys at the same seqnum, so 562 // `a#3` is not deleted. The range tombstone will be seen first due 563 // to its larger value type. Since it is a non-skippable key, the 564 // current merge will not include `a#3`. If we zeroed the current 565 // merge result's seqnum, then it would conflict with the upcoming 566 // merge including `a#3`, whose seqnum will also be zeroed. 567 if change != sameStripeNonSkippable { 568 i.maybeZeroSeqnum(origSnapshotIdx) 569 } 570 return &i.key, i.value 571 } 572 if i.err != nil { 573 i.valid = false 574 // TODO(sumeer): why is MarkCorruptionError only being called for 575 // MERGE? 576 i.err = base.MarkCorruptionError(i.err) 577 } 578 return nil, nil 579 580 default: 581 i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) 582 i.valid = false 583 return nil, nil 584 } 585 } 586 587 return nil, nil 588 } 589 590 func (i *compactionIter) closeValueCloser() error { 591 if i.valueCloser == nil { 592 return nil 593 } 594 595 i.err = i.valueCloser.Close() 596 i.valueCloser = nil 597 if i.err != nil { 598 i.valid = false 599 } 600 return i.err 601 } 602 603 // snapshotIndex returns the index of the first sequence number in snapshots 604 // which is greater than or equal to seq. 605 func snapshotIndex(seq uint64, snapshots []uint64) (int, uint64) { 606 index := sort.Search(len(snapshots), func(i int) bool { 607 return snapshots[i] > seq 608 }) 609 if index >= len(snapshots) { 610 return index, InternalKeySeqNumMax 611 } 612 return index, snapshots[index] 613 } 614 615 // skipInStripe skips over skippable keys in the same stripe and user key. It 616 // may set i.err, in which case i.iterKey will be nil. 617 func (i *compactionIter) skipInStripe() { 618 i.skip = true 619 // TODO(sumeer): we can avoid the overhead of calling i.rangeDelFrag.Covers, 620 // in this case of nextInStripe, since we are skipping all of them anyway. 621 for i.nextInStripe() == sameStripeSkippable { 622 if i.err != nil { 623 panic(i.err) 624 } 625 } 626 // Reset skip if we landed outside the original stripe. Otherwise, we landed 627 // in the same stripe on a non-skippable key. In that case we should preserve 628 // `i.skip == true` such that later keys in the stripe will continue to be 629 // skipped. 630 if i.iterStripeChange == newStripeNewKey || i.iterStripeChange == newStripeSameKey { 631 i.skip = false 632 } 633 } 634 635 func (i *compactionIter) iterNext() bool { 636 var iterValue LazyValue 637 i.iterKey, iterValue = i.iter.Next() 638 i.iterValue, _, i.err = iterValue.Value(nil) 639 if i.err != nil { 640 i.iterKey = nil 641 } 642 return i.iterKey != nil 643 } 644 645 // stripeChangeType indicates how the snapshot stripe changed relative to the 646 // previous key. If no change, it also indicates whether the current entry is 647 // skippable. If the snapshot stripe changed, it also indicates whether the new 648 // stripe was entered because the iterator progressed onto an entirely new key 649 // or entered a new stripe within the same key. 650 type stripeChangeType int 651 652 const ( 653 newStripeNewKey stripeChangeType = iota 654 newStripeSameKey 655 sameStripeSkippable 656 sameStripeNonSkippable 657 ) 658 659 // nextInStripe advances the iterator and returns one of the above const ints 660 // indicating how its state changed. 661 // 662 // All sameStripeSkippable keys that are covered by a RANGEDEL will be skipped 663 // and not returned. 664 // 665 // Calls to nextInStripe must be preceded by a call to saveKey to retain a 666 // temporary reference to the original key, so that forward iteration can 667 // proceed with a reference to the original key. Care should be taken to avoid 668 // overwriting or mutating the saved key or value before they have been returned 669 // to the caller of the exported function (i.e. the caller of Next, First, etc.) 670 // 671 // nextInStripe may set i.err, in which case the return value will be 672 // newStripeNewKey, and i.iterKey will be nil. 673 func (i *compactionIter) nextInStripe() stripeChangeType { 674 i.iterStripeChange = i.nextInStripeHelper() 675 return i.iterStripeChange 676 } 677 678 // nextInStripeHelper is an internal helper for nextInStripe; callers should use 679 // nextInStripe and not call nextInStripeHelper. 680 func (i *compactionIter) nextInStripeHelper() stripeChangeType { 681 origSnapshotIdx := i.curSnapshotIdx 682 for { 683 if !i.iterNext() { 684 return newStripeNewKey 685 } 686 key := i.iterKey 687 688 if !i.equal(i.key.UserKey, key.UserKey) { 689 i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(key.SeqNum(), i.snapshots) 690 return newStripeNewKey 691 } 692 693 // If i.key and key have the same user key, then 694 // 1. i.key must not have had a zero sequence number (or it would've be the last 695 // key with its user key). 696 // 2. i.key must have a strictly larger sequence number 697 // There's an exception in that either key may be a range delete. Range 698 // deletes may share a sequence number with a point key if the keys were 699 // ingested together. Range keys may also share the sequence number if they 700 // were ingested, but range keys are interleaved into the compaction 701 // iterator's input iterator at the maximal sequence number so their 702 // original sequence number will not be observed here. 703 if prevSeqNum := base.SeqNumFromTrailer(i.keyTrailer); (prevSeqNum == 0 || prevSeqNum <= key.SeqNum()) && 704 i.key.Kind() != InternalKeyKindRangeDelete && key.Kind() != InternalKeyKindRangeDelete { 705 prevKey := i.key 706 prevKey.Trailer = i.keyTrailer 707 panic(errors.AssertionFailedf("pebble: invariant violation: %s and %s out of order", prevKey, key)) 708 } 709 710 i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(key.SeqNum(), i.snapshots) 711 switch key.Kind() { 712 case InternalKeyKindRangeDelete: 713 // Range tombstones need to be exposed by the compactionIter to the upper level 714 // `compaction` object, so return them regardless of whether they are in the same 715 // snapshot stripe. 716 if i.curSnapshotIdx == origSnapshotIdx { 717 return sameStripeNonSkippable 718 } 719 return newStripeSameKey 720 case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete: 721 // Range keys are interleaved at the max sequence number for a given user 722 // key, so we should not see any more range keys in this stripe. 723 panic("unreachable") 724 case InternalKeyKindInvalid: 725 if i.curSnapshotIdx == origSnapshotIdx { 726 return sameStripeNonSkippable 727 } 728 return newStripeSameKey 729 case InternalKeyKindDelete, InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindSingleDelete, 730 InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized: 731 // Fall through 732 default: 733 i.iterKey = nil 734 i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) 735 i.valid = false 736 return newStripeNewKey 737 } 738 if i.curSnapshotIdx == origSnapshotIdx { 739 // Same snapshot. 740 if i.rangeDelFrag.Covers(*i.iterKey, i.curSnapshotSeqNum) == keyspan.CoversVisibly { 741 continue 742 } 743 return sameStripeSkippable 744 } 745 return newStripeSameKey 746 } 747 } 748 749 func (i *compactionIter) setNext() { 750 // Save the current key. 751 i.saveKey() 752 i.value = i.iterValue 753 i.valid = true 754 i.maybeZeroSeqnum(i.curSnapshotIdx) 755 756 // There are two cases where we can early return and skip the remaining 757 // records in the stripe: 758 // - If the DB does not SETWITHDEL. 759 // - If this key is already a SETWITHDEL. 760 if i.formatVersion < FormatSetWithDelete || 761 i.iterKey.Kind() == InternalKeyKindSetWithDelete { 762 i.skip = true 763 return 764 } 765 766 // We are iterating forward. Save the current value. 767 i.valueBuf = append(i.valueBuf[:0], i.iterValue...) 768 i.value = i.valueBuf 769 770 // Else, we continue to loop through entries in the stripe looking for a 771 // DEL. Note that we may stop *before* encountering a DEL, if one exists. 772 // 773 // NB: nextInStripe will skip sameStripeSkippable keys that are visibly 774 // covered by a RANGEDEL. This can include DELs -- this is fine since such 775 // DELs don't need to be combined with SET to make SETWITHDEL. 776 for { 777 switch i.nextInStripe() { 778 case newStripeNewKey, newStripeSameKey: 779 i.pos = iterPosNext 780 return 781 case sameStripeNonSkippable: 782 i.pos = iterPosNext 783 // We iterated onto a key that we cannot skip. We can 784 // conservatively transform the original SET into a SETWITHDEL 785 // as an indication that there *may* still be a DEL/SINGLEDEL 786 // under this SET, even if we did not actually encounter one. 787 // 788 // This is safe to do, as: 789 // 790 // - in the case that there *is not* actually a DEL/SINGLEDEL 791 // under this entry, any SINGLEDEL above this now-transformed 792 // SETWITHDEL will become a DEL when the two encounter in a 793 // compaction. The DEL will eventually be elided in a 794 // subsequent compaction. The cost for ensuring correctness is 795 // that this entry is kept around for an additional compaction 796 // cycle(s). 797 // 798 // - in the case there *is* indeed a DEL/SINGLEDEL under us 799 // (but in a different stripe or sstable), then we will have 800 // already done the work to transform the SET into a 801 // SETWITHDEL, and we will skip any additional iteration when 802 // this entry is encountered again in a subsequent compaction. 803 // 804 // Ideally, this codepath would be smart enough to handle the 805 // case of SET <- RANGEDEL <- ... <- DEL/SINGLEDEL <- .... 806 // This requires preserving any RANGEDEL entries we encounter 807 // along the way, then emitting the original (possibly 808 // transformed) key, followed by the RANGEDELs. This requires 809 // a sizable refactoring of the existing code, as nextInStripe 810 // currently returns a sameStripeNonSkippable when it 811 // encounters a RANGEDEL. 812 // TODO(travers): optimize to handle the RANGEDEL case if it 813 // turns out to be a performance problem. 814 i.key.SetKind(InternalKeyKindSetWithDelete) 815 816 // By setting i.skip=true, we are saying that after the 817 // non-skippable key is emitted (which is likely a RANGEDEL), 818 // the remaining point keys that share the same user key as this 819 // saved key should be skipped. 820 i.skip = true 821 return 822 case sameStripeSkippable: 823 // We're still in the same stripe. If this is a 824 // DEL/SINGLEDEL/DELSIZED, we stop looking and emit a SETWITHDEL. 825 // Subsequent keys are eligible for skipping. 826 switch i.iterKey.Kind() { 827 case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: 828 i.key.SetKind(InternalKeyKindSetWithDelete) 829 i.skip = true 830 return 831 case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindSetWithDelete: 832 // Do nothing 833 default: 834 i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) 835 i.valid = false 836 } 837 default: 838 panic("pebble: unexpected stripeChangeType: " + strconv.Itoa(int(i.iterStripeChange))) 839 } 840 } 841 } 842 843 func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { 844 // Save the current key. 845 i.saveKey() 846 i.valid = true 847 848 // Loop looking for older values in the current snapshot stripe and merge 849 // them. 850 for { 851 if i.nextInStripe() != sameStripeSkippable { 852 i.pos = iterPosNext 853 return i.iterStripeChange 854 } 855 if i.err != nil { 856 panic(i.err) 857 } 858 // NB: MERGE#10+RANGEDEL#9 stays a MERGE, since nextInStripe skips 859 // sameStripeSkippable keys that are visibly covered by a RANGEDEL. There 860 // may be MERGE#7 that is invisibly covered and will be preserved, but 861 // there is no risk that MERGE#10 and MERGE#7 will get merged in the 862 // future as the RANGEDEL still exists and will be used in user-facing 863 // reads that see MERGE#10, and will also eventually cause MERGE#7 to be 864 // deleted in a compaction. 865 key := i.iterKey 866 switch key.Kind() { 867 case InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized: 868 // We've hit a deletion tombstone. Return everything up to this point and 869 // then skip entries until the next snapshot stripe. We change the kind 870 // of the result key to a Set so that it shadows keys in lower 871 // levels. That is, MERGE+DEL -> SETWITHDEL. 872 // 873 // We do the same for SingleDelete since SingleDelete is only 874 // permitted (with deterministic behavior) for keys that have been 875 // set once since the last SingleDelete/Delete, so everything 876 // older is acceptable to shadow. Note that this is slightly 877 // different from singleDeleteNext() which implements stricter 878 // semantics in terms of applying the SingleDelete to the single 879 // next Set. But those stricter semantics are not observable to 880 // the end-user since Iterator interprets SingleDelete as Delete. 881 // We could do something more complicated here and consume only a 882 // single Set, and then merge in any following Sets, but that is 883 // complicated wrt code and unnecessary given the narrow permitted 884 // use of SingleDelete. 885 i.key.SetKind(InternalKeyKindSetWithDelete) 886 i.skip = true 887 return sameStripeSkippable 888 889 case InternalKeyKindSet, InternalKeyKindSetWithDelete: 890 // We've hit a Set or SetWithDel value. Merge with the existing 891 // value and return. We change the kind of the resulting key to a 892 // Set so that it shadows keys in lower levels. That is: 893 // MERGE + (SET*) -> SET. 894 i.err = valueMerger.MergeOlder(i.iterValue) 895 if i.err != nil { 896 i.valid = false 897 return sameStripeSkippable 898 } 899 i.key.SetKind(InternalKeyKindSet) 900 i.skip = true 901 return sameStripeSkippable 902 903 case InternalKeyKindMerge: 904 // We've hit another Merge value. Merge with the existing value and 905 // continue looping. 906 i.err = valueMerger.MergeOlder(i.iterValue) 907 if i.err != nil { 908 i.valid = false 909 return sameStripeSkippable 910 } 911 912 default: 913 i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) 914 i.valid = false 915 return sameStripeSkippable 916 } 917 } 918 } 919 920 // singleDeleteNext processes a SingleDelete point tombstone. A SingleDelete, or 921 // SINGLEDEL, is unique in that it deletes exactly 1 internal key. It's a 922 // performance optimization when the client knows a user key has not been 923 // overwritten, allowing the elision of the tombstone earlier, avoiding write 924 // amplification. 925 // 926 // singleDeleteNext returns a boolean indicating whether or not the caller 927 // should yield the SingleDelete key to the consumer of the compactionIter. If 928 // singleDeleteNext returns false, the caller may consume/elide the 929 // SingleDelete. 930 func (i *compactionIter) singleDeleteNext() bool { 931 // Save the current key. 932 i.saveKey() 933 i.value = i.iterValue 934 i.valid = true 935 936 // Loop until finds a key to be passed to the next level. 937 for { 938 // If we find a key that can't be skipped, return true so that the 939 // caller yields the SingleDelete to the caller. 940 if i.nextInStripe() != sameStripeSkippable { 941 // This defers additional error checking regarding single delete 942 // invariants to the compaction where the keys with the same user key as 943 // the single delete are in the same stripe. 944 i.pos = iterPosNext 945 return i.err == nil 946 } 947 if i.err != nil { 948 panic(i.err) 949 } 950 // INVARIANT: sameStripeSkippable. 951 key := i.iterKey 952 kind := key.Kind() 953 switch kind { 954 case InternalKeyKindDelete, InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized: 955 if (kind == InternalKeyKindDelete || kind == InternalKeyKindDeleteSized) && 956 i.ineffectualSingleDeleteCallback != nil { 957 i.ineffectualSingleDeleteCallback(i.key.UserKey) 958 } 959 // We've hit a Delete, DeleteSized, SetWithDelete, transform 960 // the SingleDelete into a full Delete. 961 i.key.SetKind(InternalKeyKindDelete) 962 i.skip = true 963 return true 964 965 case InternalKeyKindSet, InternalKeyKindMerge: 966 // This SingleDelete deletes the Set/Merge, and we can now elide the 967 // SingleDel as well. We advance past the Set and return false to 968 // indicate to the main compaction loop that we should NOT yield the 969 // current SingleDel key to the compaction loop. 970 // 971 // NB: singleDeleteNext was called with i.pos == iterPosCurForward, and 972 // after the call to nextInStripe, we are still at iterPosCurForward, 973 // since we are at the key after the Set/Merge that was single deleted. 974 change := i.nextInStripe() 975 switch change { 976 case sameStripeSkippable, newStripeSameKey: 977 // On the same user key. 978 nextKind := i.iterKey.Kind() 979 switch nextKind { 980 case InternalKeyKindSet, InternalKeyKindSetWithDelete, InternalKeyKindMerge: 981 if i.singleDeleteInvariantViolationCallback != nil { 982 // sameStripeSkippable keys returned by nextInStripe() are already 983 // known to not be covered by a RANGEDEL, so it is an invariant 984 // violation. The rare case is newStripeSameKey, where it is a 985 // violation if not covered by a RANGEDEL. 986 if change == sameStripeSkippable || 987 i.rangeDelFrag.Covers(*i.iterKey, i.curSnapshotSeqNum) == keyspan.NoCover { 988 i.singleDeleteInvariantViolationCallback(i.key.UserKey) 989 } 990 } 991 case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete, 992 InternalKeyKindRangeDelete: 993 default: 994 panic(errors.AssertionFailedf( 995 "unexpected internal key kind: %d", errors.Safe(i.iterKey.Kind()))) 996 } 997 case sameStripeNonSkippable: 998 // No ability to check whether there is another Set/Merge below with 999 // the same user key. 1000 // 1001 // TODO(sumeer): once range deletions are interleaved at the maximal 1002 // sequence number, this case will go away. 1003 case newStripeNewKey: 1004 default: 1005 panic("unreachable") 1006 } 1007 i.valid = false 1008 return false 1009 1010 case InternalKeyKindSingleDelete: 1011 // Two single deletes met in a compaction. The first single delete is 1012 // ineffectual. 1013 if i.ineffectualSingleDeleteCallback != nil { 1014 i.ineffectualSingleDeleteCallback(i.key.UserKey) 1015 } 1016 // Continue to apply the second single delete. 1017 continue 1018 1019 default: 1020 i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) 1021 i.valid = false 1022 return false 1023 } 1024 } 1025 } 1026 1027 // skipDueToSingleDeleteElision is called when the SingleDelete is being 1028 // elided because it is in the final snapshot stripe and there are no keys 1029 // with the same user key in lower levels in the LSM (below the files in this 1030 // compaction). 1031 // 1032 // TODO(sumeer): the only difference between singleDeleteNext and 1033 // skipDueToSingleDeleteElision is the fact that the caller knows it will be 1034 // eliding the single delete in the latter case. There are some similar things 1035 // happening in both implementations. My first attempt at combining them into 1036 // a single method was hard to comprehend. Try again. 1037 func (i *compactionIter) skipDueToSingleDeleteElision() { 1038 for { 1039 stripeChange := i.nextInStripe() 1040 if i.err != nil { 1041 panic(i.err) 1042 } 1043 switch stripeChange { 1044 case newStripeNewKey: 1045 // The single delete is only now being elided, meaning it did not elide 1046 // any keys earlier in its descent down the LSM. We stepped onto a new 1047 // user key, meaning that even now at its moment of elision, it still 1048 // hasn't elided any other keys. The single delete was ineffectual (a 1049 // no-op). 1050 if i.ineffectualSingleDeleteCallback != nil { 1051 i.ineffectualSingleDeleteCallback(i.key.UserKey) 1052 } 1053 i.skip = false 1054 return 1055 case newStripeSameKey: 1056 // This should be impossible. If we're eliding a single delete, we 1057 // determined that the tombstone is in the final snapshot stripe, but we 1058 // stepped into a new stripe of the same key. 1059 panic(errors.AssertionFailedf("eliding single delete followed by same key in new stripe")) 1060 case sameStripeNonSkippable: 1061 // There's a key that we cannot skip. There are two possible cases: 1062 // a. The key is invalid. This is an error. 1063 // b. The key is a range deletion. 1064 // The second case may also be an ineffectual single delete. However, it 1065 // is possible that there is a SET that is at the same seqnum as the 1066 // RANGEDEL, and so is not deleted by that RANGEDEL, and will be deleted 1067 // by this single delete. So we cannot be certain that this is an 1068 // ineffectual single delete. 1069 // 1070 // TODO(sumeer): the existing todo to interleave range deletions at the 1071 // maximal sequence number will allow us to address this ambiguity. 1072 // 1073 // TODO(sumeer): by setting skip to true, the compactionIter is making a 1074 // single delete stronger (like a del), which will hide bugs in the use of 1075 // single delete. 1076 i.skip = true 1077 return 1078 case sameStripeSkippable: 1079 kind := i.iterKey.Kind() 1080 switch kind { 1081 case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete: 1082 if i.ineffectualSingleDeleteCallback != nil { 1083 i.ineffectualSingleDeleteCallback(i.key.UserKey) 1084 } 1085 switch kind { 1086 case InternalKeyKindDelete, InternalKeyKindDeleteSized: 1087 i.skipInStripe() 1088 return 1089 case InternalKeyKindSingleDelete: 1090 // Repeat the same with this SingleDelete. We don't want to simply 1091 // call skipInStripe(), since it increases the strength of the 1092 // SingleDel, which hides bugs in the use of single delete. 1093 continue 1094 default: 1095 panic(errors.AssertionFailedf( 1096 "unexpected internal key kind: %d", errors.Safe(i.iterKey.Kind()))) 1097 } 1098 case InternalKeyKindSetWithDelete: 1099 // The SingleDelete should behave like a Delete. 1100 i.skipInStripe() 1101 return 1102 case InternalKeyKindSet, InternalKeyKindMerge: 1103 // This SingleDelete deletes the Set/Merge, and we are eliding the 1104 // SingleDel as well. Step to the next key (this is not deleted by the 1105 // SingleDelete). 1106 // 1107 // NB: skipDueToSingleDeleteElision was called with i.pos == 1108 // iterPosCurForward, and after the call to nextInStripe, we are still 1109 // at iterPosCurForward, since we are at the key after the Set/Merge 1110 // that was single deleted. 1111 change := i.nextInStripe() 1112 if i.err != nil { 1113 panic(i.err) 1114 } 1115 switch change { 1116 case newStripeSameKey: 1117 panic(errors.AssertionFailedf("eliding single delete followed by same key in new stripe")) 1118 case newStripeNewKey: 1119 case sameStripeSkippable: 1120 // On the same key. 1121 nextKind := i.iterKey.Kind() 1122 switch nextKind { 1123 case InternalKeyKindSet, InternalKeyKindSetWithDelete, InternalKeyKindMerge: 1124 if i.singleDeleteInvariantViolationCallback != nil { 1125 i.singleDeleteInvariantViolationCallback(i.key.UserKey) 1126 } 1127 case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete, 1128 InternalKeyKindRangeDelete: 1129 default: 1130 panic(errors.AssertionFailedf( 1131 "unexpected internal key kind: %d", errors.Safe(i.iterKey.Kind()))) 1132 } 1133 case sameStripeNonSkippable: 1134 // No ability to check whether there is another Set/Merge below with 1135 // the same user key. 1136 // 1137 // TODO(sumeer): once range deletions are interleaved at the maximal 1138 // sequence number, this case will go away. 1139 default: 1140 panic("unreachable") 1141 } 1142 // Whether in same stripe or new stripe, this key is not consumed by 1143 // the SingleDelete. 1144 i.skip = false 1145 return 1146 default: 1147 panic(errors.AssertionFailedf( 1148 "unexpected internal key kind: %d", errors.Safe(i.iterKey.Kind()))) 1149 } 1150 default: 1151 panic("unreachable") 1152 } 1153 } 1154 } 1155 1156 // deleteSizedNext processes a DELSIZED point tombstone. Unlike ordinary DELs, 1157 // these tombstones carry a value that's a varint indicating the size of the 1158 // entry (len(key)+len(value)) that the tombstone is expected to delete. 1159 // 1160 // When a deleteSizedNext is encountered, we skip ahead to see which keys, if 1161 // any, are elided as a result of the tombstone. 1162 func (i *compactionIter) deleteSizedNext() (*base.InternalKey, []byte) { 1163 i.saveKey() 1164 i.valid = true 1165 i.skip = true 1166 1167 // The DELSIZED tombstone may have no value at all. This happens when the 1168 // tombstone has already deleted the key that the user originally predicted. 1169 // In this case, we still peek forward in case there's another DELSIZED key 1170 // with a lower sequence number, in which case we'll adopt its value. 1171 if len(i.iterValue) == 0 { 1172 i.value = i.valueBuf[:0] 1173 } else { 1174 i.valueBuf = append(i.valueBuf[:0], i.iterValue...) 1175 i.value = i.valueBuf 1176 } 1177 1178 // Loop through all the keys within this stripe that are skippable. 1179 i.pos = iterPosNext 1180 for i.nextInStripe() == sameStripeSkippable { 1181 if i.err != nil { 1182 panic(i.err) 1183 } 1184 switch i.iterKey.Kind() { 1185 case InternalKeyKindDelete, InternalKeyKindDeleteSized, InternalKeyKindSingleDelete: 1186 // We encountered a tombstone (DEL, or DELSIZED) that's deleted by 1187 // the original DELSIZED tombstone. This can happen in two cases: 1188 // 1189 // (1) These tombstones were intended to delete two distinct values, 1190 // and this DELSIZED has already dropped the relevant key. For 1191 // example: 1192 // 1193 // a.DELSIZED.9 a.SET.7 a.DELSIZED.5 a.SET.4 1194 // 1195 // If a.DELSIZED.9 has already deleted a.SET.7, its size has 1196 // already been zeroed out. In this case, we want to adopt the 1197 // value of the DELSIZED with the lower sequence number, in 1198 // case the a.SET.4 key has not yet been elided. 1199 // 1200 // (2) This DELSIZED was missized. The user thought they were 1201 // deleting a key with this user key, but this user key had 1202 // already been deleted. 1203 // 1204 // We can differentiate these two cases by examining the length of 1205 // the DELSIZED's value. A DELSIZED's value holds the size of both 1206 // the user key and value that it intends to delete. For any user 1207 // key with a length > 0, a DELSIZED that has not deleted a key must 1208 // have a value with a length > 0. 1209 // 1210 // We treat both cases the same functionally, adopting the identity 1211 // of the lower-sequence numbered tombstone. However in the second 1212 // case, we also increment the stat counting missized tombstones. 1213 if len(i.value) > 0 { 1214 // The original DELSIZED key was missized. The key that the user 1215 // thought they were deleting does not exist. 1216 i.stats.countMissizedDels++ 1217 } 1218 i.valueBuf = append(i.valueBuf[:0], i.iterValue...) 1219 i.value = i.valueBuf 1220 if i.iterKey.Kind() != InternalKeyKindDeleteSized { 1221 // Convert the DELSIZED to a DEL—The DEL/SINGLEDEL we're eliding 1222 // may not have deleted the key(s) it was intended to yet. The 1223 // ordinary DEL compaction heuristics are better suited at that, 1224 // plus we don't want to count it as a missized DEL. We early 1225 // exit in this case, after skipping the remainder of the 1226 // snapshot stripe. 1227 i.key.SetKind(InternalKeyKindDelete) 1228 // NB: We skipInStripe now, rather than returning leaving 1229 // i.skip=true and returning early, because Next() requires 1230 // that i.skip=true only if i.iterPos = iterPosCurForward. 1231 // 1232 // Ignore any error caused by skipInStripe since it does not affect 1233 // the key/value being returned here, and the next call to Next() will 1234 // expose it. 1235 i.skipInStripe() 1236 return &i.key, i.value 1237 } 1238 // Continue, in case we uncover another DELSIZED or a key this 1239 // DELSIZED deletes. 1240 1241 case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindSetWithDelete: 1242 // If the DELSIZED is value-less, it already deleted the key that it 1243 // was intended to delete. This is possible with a sequence like: 1244 // 1245 // DELSIZED.8 SET.7 SET.3 1246 // 1247 // The DELSIZED only describes the size of the SET.7, which in this 1248 // case has already been elided. We don't count it as a missizing, 1249 // instead converting the DELSIZED to a DEL. Skip the remainder of 1250 // the snapshot stripe and return. 1251 if len(i.value) == 0 { 1252 i.key.SetKind(InternalKeyKindDelete) 1253 // NB: We skipInStripe now, rather than returning leaving 1254 // i.skip=true and returning early, because Next() requires 1255 // that i.skip=true only if i.iterPos = iterPosCurForward. 1256 // 1257 // Ignore any error caused by skipInStripe since it does not affect 1258 // the key/value being returned here, and the next call to Next() will 1259 // expose it. 1260 i.skipInStripe() 1261 return &i.key, i.value 1262 } 1263 // The deleted key is not a DEL, DELSIZED, and the DELSIZED in i.key 1264 // has a positive size. 1265 expectedSize, n := binary.Uvarint(i.value) 1266 if n != len(i.value) { 1267 i.err = base.CorruptionErrorf("DELSIZED holds invalid value: %x", errors.Safe(i.value)) 1268 i.valid = false 1269 return nil, nil 1270 } 1271 elidedSize := uint64(len(i.iterKey.UserKey)) + uint64(len(i.iterValue)) 1272 if elidedSize != expectedSize { 1273 // The original DELSIZED key was missized. It's unclear what to 1274 // do. The user-provided size was wrong, so it's unlikely to be 1275 // accurate or meaningful. We could: 1276 // 1277 // 1. return the DELSIZED with the original user-provided size unmodified 1278 // 2. return the DELZIZED with a zeroed size to reflect that a key was 1279 // elided, even if it wasn't the anticipated size. 1280 // 3. subtract the elided size from the estimate and re-encode. 1281 // 4. convert the DELSIZED into a value-less DEL, so that 1282 // ordinary DEL heuristics apply. 1283 // 1284 // We opt for (4) under the rationale that we can't rely on the 1285 // user-provided size for accuracy, so ordinary DEL heuristics 1286 // are safer. 1287 i.stats.countMissizedDels++ 1288 i.key.SetKind(InternalKeyKindDelete) 1289 i.value = i.valueBuf[:0] 1290 // NB: We skipInStripe now, rather than returning leaving 1291 // i.skip=true and returning early, because Next() requires 1292 // that i.skip=true only if i.iterPos = iterPosCurForward. 1293 // 1294 // Ignore any error caused by skipInStripe since it does not affect 1295 // the key/value being returned here, and the next call to Next() will 1296 // expose it. 1297 i.skipInStripe() 1298 return &i.key, i.value 1299 } 1300 // NB: We remove the value regardless of whether the key was sized 1301 // appropriately. The size encoded is 'consumed' the first time it 1302 // meets a key that it deletes. 1303 i.value = i.valueBuf[:0] 1304 1305 default: 1306 i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) 1307 i.valid = false 1308 return nil, nil 1309 } 1310 } 1311 // Reset skip if we landed outside the original stripe. Otherwise, we landed 1312 // in the same stripe on a non-skippable key. In that case we should preserve 1313 // `i.skip == true` such that later keys in the stripe will continue to be 1314 // skipped. 1315 if i.iterStripeChange == newStripeNewKey || i.iterStripeChange == newStripeSameKey { 1316 i.skip = false 1317 } 1318 if i.err != nil { 1319 return nil, nil 1320 } 1321 return &i.key, i.value 1322 } 1323 1324 func (i *compactionIter) saveKey() { 1325 i.keyBuf = append(i.keyBuf[:0], i.iterKey.UserKey...) 1326 i.key.UserKey = i.keyBuf 1327 i.key.Trailer = i.iterKey.Trailer 1328 i.keyTrailer = i.iterKey.Trailer 1329 i.frontiers.Advance(i.key.UserKey) 1330 } 1331 1332 func (i *compactionIter) cloneKey(key []byte) []byte { 1333 i.alloc, key = i.alloc.Copy(key) 1334 return key 1335 } 1336 1337 func (i *compactionIter) Key() InternalKey { 1338 return i.key 1339 } 1340 1341 func (i *compactionIter) Value() []byte { 1342 return i.value 1343 } 1344 1345 func (i *compactionIter) Valid() bool { 1346 return i.valid 1347 } 1348 1349 func (i *compactionIter) Error() error { 1350 return i.err 1351 } 1352 1353 func (i *compactionIter) Close() error { 1354 err := i.iter.Close() 1355 if i.err == nil { 1356 i.err = err 1357 } 1358 1359 // Close the closer for the current value if one was open. 1360 if i.valueCloser != nil { 1361 i.err = firstError(i.err, i.valueCloser.Close()) 1362 i.valueCloser = nil 1363 } 1364 1365 return i.err 1366 } 1367 1368 // Tombstones returns a list of pending range tombstones in the fragmenter 1369 // up to the specified key, or all pending range tombstones if key = nil. 1370 func (i *compactionIter) Tombstones(key []byte) []keyspan.Span { 1371 if key == nil { 1372 i.rangeDelFrag.Finish() 1373 } else { 1374 // The specified end key is exclusive; no versions of the specified 1375 // user key (including range tombstones covering that key) should 1376 // be flushed yet. 1377 i.rangeDelFrag.TruncateAndFlushTo(key) 1378 } 1379 tombstones := i.tombstones 1380 i.tombstones = nil 1381 return tombstones 1382 } 1383 1384 // RangeKeys returns a list of pending fragmented range keys up to the specified 1385 // key, or all pending range keys if key = nil. 1386 func (i *compactionIter) RangeKeys(key []byte) []keyspan.Span { 1387 if key == nil { 1388 i.rangeKeyFrag.Finish() 1389 } else { 1390 // The specified end key is exclusive; no versions of the specified 1391 // user key (including range tombstones covering that key) should 1392 // be flushed yet. 1393 i.rangeKeyFrag.TruncateAndFlushTo(key) 1394 } 1395 rangeKeys := i.rangeKeys 1396 i.rangeKeys = nil 1397 return rangeKeys 1398 } 1399 1400 func (i *compactionIter) emitRangeDelChunk(fragmented keyspan.Span) { 1401 // Apply the snapshot stripe rules, keeping only the latest tombstone for 1402 // each snapshot stripe. 1403 currentIdx := -1 1404 keys := fragmented.Keys[:0] 1405 for _, k := range fragmented.Keys { 1406 idx, _ := snapshotIndex(k.SeqNum(), i.snapshots) 1407 if currentIdx == idx { 1408 continue 1409 } 1410 if idx == 0 && i.elideRangeTombstone(fragmented.Start, fragmented.End) { 1411 // This is the last snapshot stripe and the range tombstone 1412 // can be elided. 1413 break 1414 } 1415 1416 keys = append(keys, k) 1417 if idx == 0 { 1418 // This is the last snapshot stripe. 1419 break 1420 } 1421 currentIdx = idx 1422 } 1423 if len(keys) > 0 { 1424 i.tombstones = append(i.tombstones, keyspan.Span{ 1425 Start: fragmented.Start, 1426 End: fragmented.End, 1427 Keys: keys, 1428 }) 1429 } 1430 } 1431 1432 func (i *compactionIter) emitRangeKeyChunk(fragmented keyspan.Span) { 1433 // Elision of snapshot stripes happens in rangeKeyCompactionTransform, so no need to 1434 // do that here. 1435 if len(fragmented.Keys) > 0 { 1436 i.rangeKeys = append(i.rangeKeys, fragmented) 1437 } 1438 } 1439 1440 // maybeZeroSeqnum attempts to set the seqnum for the current key to 0. Doing 1441 // so improves compression and enables an optimization during forward iteration 1442 // to skip some key comparisons. The seqnum for an entry can be zeroed if the 1443 // entry is on the bottom snapshot stripe and on the bottom level of the LSM. 1444 func (i *compactionIter) maybeZeroSeqnum(snapshotIdx int) { 1445 if !i.allowZeroSeqNum { 1446 // TODO(peter): allowZeroSeqNum applies to the entire compaction. We could 1447 // make the determination on a key by key basis, similar to what is done 1448 // for elideTombstone. Need to add a benchmark for compactionIter to verify 1449 // that isn't too expensive. 1450 return 1451 } 1452 if snapshotIdx > 0 { 1453 // This is not the last snapshot 1454 return 1455 } 1456 i.key.SetSeqNum(base.SeqNumZero) 1457 } 1458 1459 // A frontier is used to monitor a compaction's progression across the user 1460 // keyspace. 1461 // 1462 // A frontier hold a user key boundary that it's concerned with in its `key` 1463 // field. If/when the compaction iterator returns an InternalKey with a user key 1464 // _k_ such that k ≥ frontier.key, the compaction iterator invokes the 1465 // frontier's `reached` function, passing _k_ as its argument. 1466 // 1467 // The `reached` function returns a new value to use as the key. If `reached` 1468 // returns nil, the frontier is forgotten and its `reached` method will not be 1469 // invoked again, unless the user calls [Update] to set a new key. 1470 // 1471 // A frontier's key may be updated outside the context of a `reached` 1472 // invocation at any time, through its Update method. 1473 type frontier struct { 1474 // container points to the containing *frontiers that was passed to Init 1475 // when the frontier was initialized. 1476 container *frontiers 1477 1478 // key holds the frontier's current key. If nil, this frontier is inactive 1479 // and its reached func will not be invoked. The value of this key may only 1480 // be updated by the `frontiers` type, or the Update method. 1481 key []byte 1482 1483 // reached is invoked to inform a frontier that its key has been reached. 1484 // It's invoked with the user key that reached the limit. The `key` argument 1485 // is guaranteed to be ≥ the frontier's key. 1486 // 1487 // After reached is invoked, the frontier's key is updated to the return 1488 // value of `reached`. Note bene, the frontier is permitted to update its 1489 // key to a user key ≤ the argument `key`. 1490 // 1491 // If a frontier is set to key k1, and reached(k2) is invoked (k2 ≥ k1), the 1492 // frontier will receive reached(k2) calls until it returns nil or a key 1493 // `k3` such that k2 < k3. This property is useful for frontiers that use 1494 // `reached` invocations to drive iteration through collections of keys that 1495 // may contain multiple keys that are both < k2 and ≥ k1. 1496 reached func(key []byte) (next []byte) 1497 } 1498 1499 // Init initializes the frontier with the provided key and reached callback. 1500 // The frontier is attached to the provided *frontiers and the provided reached 1501 // func will be invoked when the *frontiers is advanced to a key ≥ this 1502 // frontier's key. 1503 func (f *frontier) Init( 1504 frontiers *frontiers, initialKey []byte, reached func(key []byte) (next []byte), 1505 ) { 1506 *f = frontier{ 1507 container: frontiers, 1508 key: initialKey, 1509 reached: reached, 1510 } 1511 if initialKey != nil { 1512 f.container.push(f) 1513 } 1514 } 1515 1516 // String implements fmt.Stringer. 1517 func (f *frontier) String() string { 1518 return string(f.key) 1519 } 1520 1521 // Update replaces the existing frontier's key with the provided key. The 1522 // frontier's reached func will be invoked when the new key is reached. 1523 func (f *frontier) Update(key []byte) { 1524 c := f.container 1525 prevKeyIsNil := f.key == nil 1526 f.key = key 1527 if prevKeyIsNil { 1528 if key != nil { 1529 c.push(f) 1530 } 1531 return 1532 } 1533 1534 // Find the frontier within the heap (it must exist within the heap because 1535 // f.key was != nil). If the frontier key is now nil, remove it from the 1536 // heap. Otherwise, fix up its position. 1537 for i := 0; i < len(c.items); i++ { 1538 if c.items[i] == f { 1539 if key != nil { 1540 c.fix(i) 1541 } else { 1542 n := c.len() - 1 1543 c.swap(i, n) 1544 c.down(i, n) 1545 c.items = c.items[:n] 1546 } 1547 return 1548 } 1549 } 1550 panic("unreachable") 1551 } 1552 1553 // frontiers is used to track progression of a task (eg, compaction) across the 1554 // keyspace. Clients that want to be informed when the task advances to a key ≥ 1555 // some frontier may register a frontier, providing a callback. The task calls 1556 // `Advance(k)` with each user key encountered, which invokes the `reached` func 1557 // on all tracked frontiers with `key`s ≤ k. 1558 // 1559 // Internally, frontiers is implemented as a simple heap. 1560 type frontiers struct { 1561 cmp Compare 1562 items []*frontier 1563 } 1564 1565 // String implements fmt.Stringer. 1566 func (f *frontiers) String() string { 1567 var buf bytes.Buffer 1568 for i := 0; i < len(f.items); i++ { 1569 if i > 0 { 1570 fmt.Fprint(&buf, ", ") 1571 } 1572 fmt.Fprintf(&buf, "%s: %q", f.items[i], f.items[i].key) 1573 } 1574 return buf.String() 1575 } 1576 1577 // Advance notifies all member frontiers with keys ≤ k. 1578 func (f *frontiers) Advance(k []byte) { 1579 for len(f.items) > 0 && f.cmp(k, f.items[0].key) >= 0 { 1580 // This frontier has been reached. Invoke the closure and update with 1581 // the next frontier. 1582 f.items[0].key = f.items[0].reached(k) 1583 if f.items[0].key == nil { 1584 // This was the final frontier that this user was concerned with. 1585 // Remove it from the heap. 1586 f.pop() 1587 } else { 1588 // Fix up the heap root. 1589 f.fix(0) 1590 } 1591 } 1592 } 1593 1594 func (f *frontiers) len() int { 1595 return len(f.items) 1596 } 1597 1598 func (f *frontiers) less(i, j int) bool { 1599 return f.cmp(f.items[i].key, f.items[j].key) < 0 1600 } 1601 1602 func (f *frontiers) swap(i, j int) { 1603 f.items[i], f.items[j] = f.items[j], f.items[i] 1604 } 1605 1606 // fix, up and down are copied from the go stdlib. 1607 1608 func (f *frontiers) fix(i int) { 1609 if !f.down(i, f.len()) { 1610 f.up(i) 1611 } 1612 } 1613 1614 func (f *frontiers) push(ff *frontier) { 1615 n := len(f.items) 1616 f.items = append(f.items, ff) 1617 f.up(n) 1618 } 1619 1620 func (f *frontiers) pop() *frontier { 1621 n := f.len() - 1 1622 f.swap(0, n) 1623 f.down(0, n) 1624 item := f.items[n] 1625 f.items = f.items[:n] 1626 return item 1627 } 1628 1629 func (f *frontiers) up(j int) { 1630 for { 1631 i := (j - 1) / 2 // parent 1632 if i == j || !f.less(j, i) { 1633 break 1634 } 1635 f.swap(i, j) 1636 j = i 1637 } 1638 } 1639 1640 func (f *frontiers) down(i0, n int) bool { 1641 i := i0 1642 for { 1643 j1 := 2*i + 1 1644 if j1 >= n || j1 < 0 { // j1 < 0 after int overflow 1645 break 1646 } 1647 j := j1 // left child 1648 if j2 := j1 + 1; j2 < n && f.less(j2, j1) { 1649 j = j2 // = 2*i + 2 // right child 1650 } 1651 if !f.less(j, i) { 1652 break 1653 } 1654 f.swap(i, j) 1655 i = j 1656 } 1657 return i > i0 1658 }