-
Notifications
You must be signed in to change notification settings - Fork 472
/
iterator.go
1633 lines (1519 loc) · 57.5 KB
/
iterator.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2011 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.
package pebble
import (
"bytes"
"io"
"sync/atomic"
"unsafe"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/fastrand"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/rangekey"
)
// iterPos describes the state of the internal iterator, in terms of whether it
// is at the position returned to the user (cur), one ahead of the position
// returned (next for forward iteration and prev for reverse iteration). The cur
// position is split into two states, for forward and reverse iteration, since
// we need to differentiate for switching directions.
//
// There is subtlety in what is considered the current position of the Iterator.
// The internal iterator exposes a sequence of internal keys. There is not
// always a single internalIterator position corresponding to the position
// returned to the user. Consider the example:
//
// a.MERGE.9 a.MERGE.8 a.MERGE.7 a.SET.6 b.DELETE.9 b.DELETE.5 b.SET.4
// \ /
// \ Iterator.Key() = 'a' /
//
// The Iterator exposes one valid position at user key 'a' and the two exhausted
// positions at the beginning and end of iteration. The underlying
// internalIterator contains 7 valid positions and 2 exhausted positions.
//
// Iterator positioning methods must set iterPos to iterPosCur{Foward,Backward}
// iff the user key at the current internalIterator position equals the
// Iterator.Key returned to the user. This guarantees that a call to nextUserKey
// or prevUserKey will advance to the next or previous iterator position.
// iterPosCur{Forward,Backward} does not make any guarantee about the internal
// iterator position among internal keys with matching user keys, and it will
// vary subtly depending on the particular key kinds encountered. In the above
// example, the iterator returning 'a' may set iterPosCurForward if the internal
// iterator is positioned at any of a.MERGE.9, a.MERGE.8, a.MERGE.7 or a.SET.6.
//
// When setting iterPos to iterPosNext or iterPosPrev, the internal iterator
// must be advanced to the first internalIterator position at a user key greater
// (iterPosNext) or less (iterPosPrev) than the key returned to the user. An
// internalIterator position that's !Valid() must also be considered greater or
// less—depending on the direction of iteration—than the last valid Iterator
// position.
type iterPos int8
const (
iterPosCurForward iterPos = 0
iterPosNext iterPos = 1
iterPosPrev iterPos = -1
iterPosCurReverse iterPos = -2
// For limited iteration. When the iterator is at iterPosCurForwardPaused
// - Next*() call should behave as if the internal iterator is already
// at next (akin to iterPosNext).
// - Prev*() call should behave as if the internal iterator is at the
// current key (akin to iterPosCurForward).
//
// Similar semantics apply to CurReversePaused.
iterPosCurForwardPaused iterPos = 2
iterPosCurReversePaused iterPos = -3
)
// Approximate gap in bytes between samples of data read during iteration.
// This is multiplied with a default ReadSamplingMultiplier of 1 << 4 to yield
// 1 << 20 (1MB). The 1MB factor comes from:
// https://github.com/cockroachdb/pebble/issues/29#issuecomment-494477985
const readBytesPeriod uint64 = 1 << 16
var errReversePrefixIteration = errors.New("pebble: unsupported reverse prefix iteration")
// IteratorMetrics holds per-iterator metrics. These do not change over the
// lifetime of the iterator.
type IteratorMetrics struct {
// The read amplification experienced by this iterator. This is the sum of
// the memtables, the L0 sublevels and the non-empty Ln levels. Higher read
// amplification generally results in slower reads, though allowing higher
// read amplification can also result in faster writes.
ReadAmp int
}
// IteratorStatsKind describes the two kind of iterator stats.
type IteratorStatsKind int8
const (
// InterfaceCall represents calls to Iterator.
InterfaceCall IteratorStatsKind = iota
// InternalIterCall represents calls by Iterator to its internalIterator.
InternalIterCall
// NumStatsKind is the number of kinds, and is used for array sizing.
NumStatsKind
)
// IteratorStats contains iteration stats.
type IteratorStats struct {
// ForwardSeekCount includes SeekGE, SeekPrefixGE, First.
ForwardSeekCount [NumStatsKind]int
// ReverseSeek includes SeekLT, Last.
ReverseSeekCount [NumStatsKind]int
// ForwardStepCount includes Next.
ForwardStepCount [NumStatsKind]int
// ReverseStepCount includes Prev.
ReverseStepCount [NumStatsKind]int
}
var _ redact.SafeFormatter = &IteratorStats{}
// Iterator iterates over a DB's key/value pairs in key order.
//
// An iterator must be closed after use, but it is not necessary to read an
// iterator until exhaustion.
//
// An iterator is not goroutine-safe, but it is safe to use multiple iterators
// concurrently, with each in a dedicated goroutine.
//
// It is also safe to use an iterator concurrently with modifying its
// underlying DB, if that DB permits modification. However, the resultant
// key/value pairs are not guaranteed to be a consistent snapshot of that DB
// at a particular point in time.
//
// If an iterator encounters an error during any operation, it is stored by
// the Iterator and surfaced through the Error method. All absolute
// positioning methods (eg, SeekLT, SeekGT, First, Last, etc) reset any
// accumulated error before positioning. All relative positioning methods (eg,
// Next, Prev) return without advancing if the iterator has an accumulated
// error.
type Iterator struct {
opts IterOptions
cmp Compare
equal Equal
merge Merge
split Split
iter internalIterator
readState *readState
rangeKey *iteratorRangeKeyState
err error
// When iterValidityState=IterValid, key represents the current key, which
// is backed by keyBuf.
key []byte
keyBuf []byte
value []byte
valueBuf []byte
valueCloser io.Closer
// iterKey, iterValue reflect the latest position of iter, except when
// SetBounds is called. In that case, these are explicitly set to nil.
iterKey *InternalKey
iterValue []byte
alloc *iterAlloc
getIterAlloc *getIterAlloc
prefixOrFullSeekKey []byte
readSampling readSampling
stats IteratorStats
// Following fields are only used in Clone.
// Non-nil if this Iterator includes a Batch.
batch *Batch
newIters tableNewIters
seqNum uint64
// Keeping the bools here after all the 8 byte aligned fields shrinks the
// sizeof this struct by 24 bytes.
// INVARIANT:
// iterValidityState==IterAtLimit <=>
// pos==iterPosCurForwardPaused || pos==iterPosCurReversePaused
iterValidityState IterValidityState
// The position of iter. When this is iterPos{Prev,Next} the iter has been
// moved past the current key-value, which can only happen if
// iterValidityState=IterValid, i.e., there is something to return to the
// client for the current position.
pos iterPos
// Relates to the prefixOrFullSeekKey field above.
hasPrefix bool
// Used for deriving the value of SeekPrefixGE(..., trySeekUsingNext),
// and SeekGE/SeekLT optimizations
lastPositioningOp lastPositioningOpKind
}
type iteratorRangeKeyState struct {
// rangeKeyIter is temporarily an iterator into a single global in-memory
// range keys arena. This will need to be reworked when we have a merging
// range key iterator.
rangeKeyIter *rangekey.Iter
iter rangekey.InterleavingIter
// rangeKeyOnly is set to true if at the current iterator position there is
// no point key, only a range key start boundary.
rangeKeyOnly bool
hasRangeKey bool
keys []RangeKeyData
// start and end are the [start, end) boundaries of the current range keys.
start []byte
end []byte
// buf is used to save range-key data before moving the range-key iterator.
// Start and end boundaries, suffixes and values are all copied into buf.
buf []byte
}
type lastPositioningOpKind int8
const (
unknownLastPositionOp lastPositioningOpKind = iota
seekPrefixGELastPositioningOp
seekGELastPositioningOp
seekLTLastPositioningOp
)
// Limited iteration mode. Not for use with prefix iteration.
//
// SeekGE, SeekLT, Prev, Next have WithLimit variants, that pause the iterator
// at the limit in a best-effort manner. The client should behave correctly
// even if the limits are ignored. These limits are not "deep", in that they
// are not passed down to the underlying collection of internalIterators. This
// is because the limits are transient, and apply only until the next
// iteration call. They serve mainly as a way to bound the amount of work when
// two (or more) Iterators are being coordinated at a higher level.
//
// In limited iteration mode:
// - Avoid using Iterator.Valid if the last call was to a *WithLimit() method.
// The return value from the *WithLimit() method provides a more precise
// disposition.
// - The limit is exclusive for forward and inclusive for reverse.
//
//
// Limited iteration mode & range keys
//
// Limited iteration interacts with range-key iteration. When range key
// iteration is enabled, range keys are interleaved at their start boundaries.
// Limited iteration must ensure that if a range key exists within the limit,
// the iterator visits the range key.
//
// During forward limited iteration, this is trivial: An overlapping range key
// must have a start boundary less than the limit, and the range key's start
// boundary will be interleaved and found to be within the limit.
//
// During reverse limited iteration, the tail of the range key may fall within
// the limit. The range key must be surfaced even if the range key's start
// boundary is less than the limit, and if there are no point keys between the
// current iterator position and the limit. To provide this guarantee, reverse
// limited iteration ignores the limit as long as there is a range key
// overlapping the iteration position.
// IterValidityState captures the state of the Iterator.
type IterValidityState int8
const (
// IterExhausted represents an Iterator that is exhausted.
IterExhausted IterValidityState = iota
// IterValid represents an Iterator that is valid.
IterValid
// IterAtLimit represents an Iterator that has a non-exhausted
// internalIterator, but has reached a limit without any key for the
// caller.
IterAtLimit
)
// readSampling stores variables used to sample a read to trigger a read
// compaction
type readSampling struct {
bytesUntilReadSampling uint64
initialSamplePassed bool
pendingCompactions readCompactionQueue
// forceReadSampling is used for testing purposes to force a read sample on every
// call to Iterator.maybeSampleRead()
forceReadSampling bool
}
func (i *Iterator) findNextEntry(limit []byte) {
i.iterValidityState = IterExhausted
i.pos = iterPosCurForward
if i.rangeKey != nil {
i.rangeKey.rangeKeyOnly = false
}
// Close the closer for the current value if one was open.
if i.closeValueCloser() != nil {
return
}
for i.iterKey != nil {
key := *i.iterKey
if i.hasPrefix {
if n := i.split(key.UserKey); !bytes.Equal(i.prefixOrFullSeekKey, key.UserKey[:n]) {
return
}
}
// Compare with limit every time we start at a different user key.
// Note that given the best-effort contract of limit, we could avoid a
// comparison in the common case by doing this only after
// i.nextUserKey is called for the deletes below. However that makes
// the behavior non-deterministic (since the behavior will vary based
// on what has been compacted), which makes it hard to test with the
// metamorphic test. So we forego that performance optimization.
if limit != nil && i.cmp(limit, i.iterKey.UserKey) <= 0 {
i.iterValidityState = IterAtLimit
i.pos = iterPosCurForwardPaused
return
}
switch key.Kind() {
case InternalKeyKindRangeKeySet:
// Save the current key.
i.keyBuf = append(i.keyBuf[:0], key.UserKey...)
i.key = i.keyBuf
i.value = nil
// There may also be a live point key at this userkey that we have
// not yet read. We need to find the next entry with this user key
// to find it. Save the range key so we don't lose it when we Next
// the underlying iterator.
i.saveRangeKey()
pointKeyExists := i.nextPointCurrentUserKey()
if i.err != nil {
i.iterValidityState = IterExhausted
return
}
i.rangeKey.rangeKeyOnly = !pointKeyExists
i.iterValidityState = IterValid
return
case InternalKeyKindDelete, InternalKeyKindSingleDelete:
i.nextUserKey()
continue
case InternalKeyKindSet, InternalKeyKindSetWithDelete:
i.keyBuf = append(i.keyBuf[:0], key.UserKey...)
i.key = i.keyBuf
i.value = i.iterValue
i.iterValidityState = IterValid
i.setRangeKey()
return
case InternalKeyKindMerge:
// Resolving the merge may advance us to the next point key, which
// may be covered by a different set of range keys. Save the range
// key state so we don't lose it.
i.saveRangeKey()
if i.mergeForward(key) {
i.iterValidityState = IterValid
return
}
// The merge didn't yield a valid key, either because the value
// merger indicated it should be deleted, or because an error was
// encountered.
i.iterValidityState = IterExhausted
if i.err != nil {
return
}
if i.pos != iterPosNext {
i.nextUserKey()
}
if i.closeValueCloser() != nil {
return
}
i.pos = iterPosCurForward
default:
i.err = base.CorruptionErrorf("pebble: invalid internal key kind: %d", errors.Safe(key.Kind()))
i.iterValidityState = IterExhausted
return
}
}
}
func (i *Iterator) nextPointCurrentUserKey() bool {
i.pos = iterPosCurForward
i.iterKey, i.iterValue = i.iter.Next()
i.stats.ForwardStepCount[InternalIterCall]++
if i.iterKey == nil || !i.equal(i.key, i.iterKey.UserKey) {
i.pos = iterPosNext
return false
}
key := *i.iterKey
switch key.Kind() {
case InternalKeyKindRangeKeySet:
// RangeKeySets must always be interleaved as the first internal key
// for a user key.
i.err = base.CorruptionErrorf("pebble: unexpected range key set mid-user key")
return false
case InternalKeyKindDelete, InternalKeyKindSingleDelete:
return false
case InternalKeyKindSet, InternalKeyKindSetWithDelete:
i.value = i.iterValue
return true
case InternalKeyKindMerge:
return i.mergeForward(key)
default:
i.err = base.CorruptionErrorf("pebble: invalid internal key kind: %d", errors.Safe(key.Kind()))
return false
}
}
// mergeForward resolves a MERGE key, advancing the underlying iterator forward
// to merge with subsequent keys with the same userkey. mergeForward returns a
// boolean indicating whether or not the merge yielded a valid key. A merge may
// not yield a valid key if an error occurred, in which case i.err is non-nil,
// or the user's value merger specified the key to be deleted.
//
// mergeForward does not update iterValidityState.
func (i *Iterator) mergeForward(key base.InternalKey) (valid bool) {
var valueMerger ValueMerger
valueMerger, i.err = i.merge(key.UserKey, i.iterValue)
if i.err != nil {
return false
}
i.mergeNext(key, valueMerger)
if i.err != nil {
return false
}
var needDelete bool
i.value, needDelete, i.valueCloser, i.err = finishValueMerger(
valueMerger, true /* includesBase */)
if i.err != nil {
return false
}
if needDelete {
_ = i.closeValueCloser()
return false
}
return true
}
func (i *Iterator) closeValueCloser() error {
if i.valueCloser != nil {
i.err = i.valueCloser.Close()
i.valueCloser = nil
}
return i.err
}
func (i *Iterator) nextUserKey() {
if i.iterKey == nil {
return
}
done := i.iterKey.SeqNum() == 0
if i.iterValidityState != IterValid {
i.keyBuf = append(i.keyBuf[:0], i.iterKey.UserKey...)
i.key = i.keyBuf
}
for {
i.iterKey, i.iterValue = i.iter.Next()
i.stats.ForwardStepCount[InternalIterCall]++
if done || i.iterKey == nil {
break
}
if !i.equal(i.key, i.iterKey.UserKey) {
break
}
done = i.iterKey.SeqNum() == 0
}
}
func (i *Iterator) maybeSampleRead() {
// This method is only called when a public method of Iterator is
// returning, and below we exclude the case were the iterator is paused at
// a limit. The effect of these choices is that keys that are deleted, but
// are encountered during iteration, are not accounted for in the read
// sampling and will not cause read driven compactions, even though we are
// incurring cost in iterating over them. And this issue is not limited to
// Iterator, which does not see the effect of range deletes, which may be
// causing iteration work in mergingIter. It is not clear at this time
// whether this is a deficiency worth addressing.
if i.iterValidityState != IterValid {
return
}
if i.readState == nil {
return
}
if i.readSampling.forceReadSampling {
i.sampleRead()
return
}
samplingPeriod := int32(int64(readBytesPeriod) * i.readState.db.opts.Experimental.ReadSamplingMultiplier)
if samplingPeriod <= 0 {
return
}
bytesRead := uint64(len(i.key) + len(i.value))
for i.readSampling.bytesUntilReadSampling < bytesRead {
i.readSampling.bytesUntilReadSampling += uint64(fastrand.Uint32n(2 * uint32(samplingPeriod)))
// The block below tries to adjust for the case where this is the
// first read in a newly-opened iterator. As bytesUntilReadSampling
// starts off at zero, we don't want to sample the first read of
// every newly-opened iterator, but we do want to sample some of them.
if !i.readSampling.initialSamplePassed {
i.readSampling.initialSamplePassed = true
if fastrand.Uint32n(uint32(i.readSampling.bytesUntilReadSampling)) > uint32(bytesRead) {
continue
}
}
i.sampleRead()
}
i.readSampling.bytesUntilReadSampling -= bytesRead
}
func (i *Iterator) sampleRead() {
var topFile *manifest.FileMetadata
topLevel, numOverlappingLevels := numLevels, 0
if mi, ok := i.iter.(*mergingIter); ok {
if len(mi.levels) > 1 {
mi.ForEachLevelIter(func(li *levelIter) bool {
l := manifest.LevelToInt(li.level)
if file := li.files.Current(); file != nil {
var containsKey bool
if i.pos == iterPosNext || i.pos == iterPosCurForward ||
i.pos == iterPosCurForwardPaused {
containsKey = i.cmp(file.Smallest.UserKey, i.key) <= 0
} else if i.pos == iterPosPrev || i.pos == iterPosCurReverse ||
i.pos == iterPosCurReversePaused {
containsKey = i.cmp(file.Largest.UserKey, i.key) >= 0
}
// Do nothing if the current key is not contained in file's
// bounds. We could seek the LevelIterator at this level
// to find the right file, but the performance impacts of
// doing that are significant enough to negate the benefits
// of read sampling in the first place. See the discussion
// at:
// https://github.com/cockroachdb/pebble/pull/1041#issuecomment-763226492
if containsKey {
numOverlappingLevels++
if numOverlappingLevels >= 2 {
// Terminate the loop early if at least 2 overlapping levels are found.
return true
}
topLevel = l
topFile = file
}
}
return false
})
}
}
if topFile == nil || topLevel >= numLevels {
return
}
if numOverlappingLevels >= 2 {
allowedSeeks := atomic.AddInt64(&topFile.Atomic.AllowedSeeks, -1)
if allowedSeeks == 0 {
// Since the compaction queue can handle duplicates, we can keep
// adding to the queue even once allowedSeeks hits 0.
// In fact, we NEED to keep adding to the queue, because the queue
// is small and evicts older and possibly useful compactions.
atomic.AddInt64(&topFile.Atomic.AllowedSeeks, topFile.InitAllowedSeeks)
read := readCompaction{
start: topFile.Smallest.UserKey,
end: topFile.Largest.UserKey,
level: topLevel,
fileNum: topFile.FileNum,
}
i.readSampling.pendingCompactions.add(&read, i.cmp)
}
}
}
func (i *Iterator) findPrevEntry(limit []byte) {
i.iterValidityState = IterExhausted
i.pos = iterPosCurReverse
if i.rangeKey != nil {
i.rangeKey.rangeKeyOnly = false
}
// Close the closer for the current value if one was open.
if i.valueCloser != nil {
i.err = i.valueCloser.Close()
i.valueCloser = nil
if i.err != nil {
i.iterValidityState = IterExhausted
return
}
}
var valueMerger ValueMerger
firstLoopIter := true
rangeKeyBoundary := false
// The code below compares with limit in multiple places. As documented in
// findNextEntry, this is being done to make the behavior of limit
// deterministic to allow for metamorphic testing. It is not required by
// the best-effort contract of limit.
for i.iterKey != nil {
key := *i.iterKey
// NB: We cannot pause if the current key is covered by a range key.
// Otherwise, the user might not ever learn of a range key that covers
// the key space being iterated over in which there are no point keys.
// Since limits are best effort, ignoring the limit in this case is
// allowed by the contract of limit.
if firstLoopIter && limit != nil && i.cmp(limit, i.iterKey.UserKey) > 0 && !i.iterHasRangeKey() {
i.iterValidityState = IterAtLimit
i.pos = iterPosCurReversePaused
return
}
firstLoopIter = false
if i.iterValidityState == IterValid {
if !i.equal(key.UserKey, i.key) {
// We've iterated to the previous user key.
i.pos = iterPosPrev
if valueMerger != nil {
var needDelete bool
i.value, needDelete, i.valueCloser, i.err = finishValueMerger(valueMerger, true /* includesBase */)
if i.err == nil && needDelete {
// The point key at this key is deleted. If we also have
// a range key boundary at this key, we still want to
// return. Otherwise, we need to continue looking for
// a live key.
i.value = nil
if rangeKeyBoundary {
i.rangeKey.rangeKeyOnly = true
} else {
i.iterValidityState = IterExhausted
if i.closeValueCloser() == nil {
continue
}
}
}
}
if i.err != nil {
i.iterValidityState = IterExhausted
}
return
}
}
switch key.Kind() {
case InternalKeyKindRangeKeySet:
// Range key start boundary markers are interleaved with the maximum
// sequence number, so if there's a point key also at this key, we
// must've already iterated over it.
// This is the final entry at this user key, so we may return
i.rangeKey.rangeKeyOnly = i.iterValidityState != IterValid
i.keyBuf = append(i.keyBuf[:0], key.UserKey...)
i.key = i.keyBuf
i.iterValidityState = IterValid
i.saveRangeKey()
// In all other cases, previous iteration requires advancing to
// iterPosPrev in order to determine if the key is live and
// unshadowed by another key at the same user key. In this case,
// because range key start boundary markers are always interleaved
// at the maximum sequence number, we know that there aren't any
// additional keys with the same user key in the backward direction.
//
// We Prev the underlying iterator once anyways for consistency, so
// that we can maintain the invariant during backward iteration that
// i.iterPos = iterPosPrev.
i.stats.ReverseStepCount[InternalIterCall]++
i.iterKey, i.iterValue = i.iter.Prev()
// Set rangeKeyBoundary so that on the next iteration, we know to
// return the key even if the MERGE point key is deleted.
rangeKeyBoundary = true
case InternalKeyKindDelete, InternalKeyKindSingleDelete:
i.value = nil
i.iterValidityState = IterExhausted
valueMerger = nil
i.iterKey, i.iterValue = i.iter.Prev()
i.stats.ReverseStepCount[InternalIterCall]++
// Compare with the limit. We could optimize by only checking when
// we step to the previous user key, but detecting that requires a
// comparison too. Note that this position may already passed a
// number of versions of this user key, but they are all deleted,
// so the fact that a subsequent Prev*() call will not see them is
// harmless. Also note that this is the only place in the loop,
// other than the firstLoopIter case above, where we could step
// to a different user key and start processing it for returning
// to the caller.
if limit != nil && i.iterKey != nil && i.cmp(limit, i.iterKey.UserKey) > 0 && !i.iterHasRangeKey() {
i.iterValidityState = IterAtLimit
i.pos = iterPosCurReversePaused
return
}
continue
case InternalKeyKindSet, InternalKeyKindSetWithDelete:
i.keyBuf = append(i.keyBuf[:0], key.UserKey...)
i.key = i.keyBuf
// iterValue is owned by i.iter and could change after the Prev()
// call, so use valueBuf instead. Note that valueBuf is only used
// in this one instance; everywhere else (eg. in findNextEntry),
// we just point i.value to the unsafe i.iter-owned value buffer.
i.valueBuf = append(i.valueBuf[:0], i.iterValue...)
i.value = i.valueBuf
// TODO(jackson): We may save the same range key many times. We can
// avoid that with some help from the InterleavingIter. See also the
// TODO in saveRangeKey.
i.saveRangeKey()
i.iterValidityState = IterValid
i.iterKey, i.iterValue = i.iter.Prev()
i.stats.ReverseStepCount[InternalIterCall]++
valueMerger = nil
continue
case InternalKeyKindMerge:
if i.iterValidityState == IterExhausted {
i.keyBuf = append(i.keyBuf[:0], key.UserKey...)
i.key = i.keyBuf
i.saveRangeKey()
valueMerger, i.err = i.merge(i.key, i.iterValue)
if i.err != nil {
return
}
i.iterValidityState = IterValid
} else if valueMerger == nil {
valueMerger, i.err = i.merge(i.key, i.value)
if i.err == nil {
i.err = valueMerger.MergeNewer(i.iterValue)
}
if i.err != nil {
i.iterValidityState = IterExhausted
return
}
} else {
i.err = valueMerger.MergeNewer(i.iterValue)
if i.err != nil {
i.iterValidityState = IterExhausted
return
}
}
i.iterKey, i.iterValue = i.iter.Prev()
i.stats.ReverseStepCount[InternalIterCall]++
continue
default:
i.err = base.CorruptionErrorf("pebble: invalid internal key kind: %d", errors.Safe(key.Kind()))
i.iterValidityState = IterExhausted
return
}
}
// i.iterKey == nil, so broke out of the preceding loop.
if i.iterValidityState == IterValid {
i.pos = iterPosPrev
if valueMerger != nil {
var needDelete bool
i.value, needDelete, i.valueCloser, i.err = finishValueMerger(valueMerger, true /* includesBase */)
if i.err == nil && needDelete {
i.key = nil
i.value = nil
i.iterValidityState = IterExhausted
}
}
if i.err != nil {
i.iterValidityState = IterExhausted
}
}
}
func (i *Iterator) prevUserKey() {
if i.iterKey == nil {
return
}
if i.iterValidityState != IterValid {
// If we're going to compare against the prev key, we need to save the
// current key.
i.keyBuf = append(i.keyBuf[:0], i.iterKey.UserKey...)
i.key = i.keyBuf
}
for {
i.iterKey, i.iterValue = i.iter.Prev()
i.stats.ReverseStepCount[InternalIterCall]++
if i.iterKey == nil {
break
}
if !i.equal(i.key, i.iterKey.UserKey) {
break
}
}
}
func (i *Iterator) mergeNext(key InternalKey, valueMerger ValueMerger) {
// Save the current key.
i.keyBuf = append(i.keyBuf[:0], key.UserKey...)
i.key = i.keyBuf
// Loop looking for older values for this key and merging them.
for {
i.iterKey, i.iterValue = i.iter.Next()
i.stats.ForwardStepCount[InternalIterCall]++
if i.iterKey == nil {
i.pos = iterPosNext
return
}
key = *i.iterKey
if !i.equal(i.key, key.UserKey) {
// We've advanced to the next key.
i.pos = iterPosNext
return
}
switch key.Kind() {
case InternalKeyKindDelete, InternalKeyKindSingleDelete:
// We've hit a deletion tombstone. Return everything up to this
// point.
return
case InternalKeyKindSet, InternalKeyKindSetWithDelete:
// We've hit a Set value. Merge with the existing value and return.
i.err = valueMerger.MergeOlder(i.iterValue)
return
case InternalKeyKindMerge:
// We've hit another Merge value. Merge with the existing value and
// continue looping.
i.err = valueMerger.MergeOlder(i.iterValue)
if i.err != nil {
return
}
continue
case InternalKeyKindRangeKeySet:
// The RANGEKEYSET marker must sort before a MERGE at the same user key.
i.err = base.CorruptionErrorf("pebble: out of order range key marker")
return
default:
i.err = base.CorruptionErrorf("pebble: invalid internal key kind: %d", errors.Safe(key.Kind()))
return
}
}
}
// SeekGE moves the iterator to the first key/value pair whose key is greater
// than or equal to the given key. Returns true if the iterator is pointing at
// a valid entry and false otherwise.
func (i *Iterator) SeekGE(key []byte) bool {
return i.SeekGEWithLimit(key, nil) == IterValid
}
// SeekGEWithLimit ...
func (i *Iterator) SeekGEWithLimit(key []byte, limit []byte) IterValidityState {
lastPositioningOp := i.lastPositioningOp
// Set it to unknown, since this operation may not succeed, in which case
// the SeekGE following this should not make any assumption about iterator
// position.
i.lastPositioningOp = unknownLastPositionOp
i.err = nil // clear cached iteration error
i.hasPrefix = false
i.stats.ForwardSeekCount[InterfaceCall]++
if lowerBound := i.opts.GetLowerBound(); lowerBound != nil && i.cmp(key, lowerBound) < 0 {
key = lowerBound
} else if upperBound := i.opts.GetUpperBound(); upperBound != nil && i.cmp(key, upperBound) > 0 {
key = upperBound
}
seekInternalIter := true
// The following noop optimization only applies when i.batch == nil, since
// an iterator over a batch is iterating over mutable data, that may have
// changed since the last seek.
if lastPositioningOp == seekGELastPositioningOp && i.batch == nil {
cmp := i.cmp(i.prefixOrFullSeekKey, key)
// If this seek is to the same or later key, and the iterator is
// already positioned there, this is a noop. This can be helpful for
// sparse key spaces that have many deleted keys, where one can avoid
// the overhead of iterating past them again and again.
if cmp <= 0 {
if i.iterValidityState == IterExhausted ||
(i.iterValidityState == IterValid && i.cmp(key, i.key) <= 0 &&
(limit == nil || i.cmp(i.key, limit) < 0)) {
// Noop
if !invariants.Enabled || !disableSeekOpt(key, uintptr(unsafe.Pointer(i))) {
i.lastPositioningOp = seekGELastPositioningOp
return i.iterValidityState
}
}
if i.pos == iterPosCurForwardPaused && i.cmp(key, i.iterKey.UserKey) <= 0 {
// Have some work to do, but don't need to seek, and we can
// start doing findNextEntry from i.iterKey.
seekInternalIter = false
}
}
}
if seekInternalIter {
i.iterKey, i.iterValue = i.iter.SeekGE(key)
i.stats.ForwardSeekCount[InternalIterCall]++
}
i.findNextEntry(limit)
i.maybeSampleRead()
if i.Error() == nil && i.batch == nil {
// Prepare state for a future noop optimization.
i.prefixOrFullSeekKey = append(i.prefixOrFullSeekKey[:0], key...)
i.lastPositioningOp = seekGELastPositioningOp
}
return i.iterValidityState
}
// SeekPrefixGE moves the iterator to the first key/value pair whose key is
// greater than or equal to the given key and which has the same "prefix" as
// the given key. The prefix for a key is determined by the user-defined
// Comparer.Split function. The iterator will not observe keys not matching the
// "prefix" of the search key. Calling SeekPrefixGE puts the iterator in prefix
// iteration mode. The iterator remains in prefix iteration until a subsequent
// call to another absolute positioning method (SeekGE, SeekLT, First,
// Last). Reverse iteration (Prev) is not supported when an iterator is in
// prefix iteration mode. Returns true if the iterator is pointing at a valid
// entry and false otherwise.
//
// The semantics of SeekPrefixGE are slightly unusual and designed for
// iteration to be able to take advantage of bloom filters that have been
// created on the "prefix". If you're not using bloom filters, there is no
// reason to use SeekPrefixGE.
//
// An example Split function may separate a timestamp suffix from the prefix of
// the key.
//
// Split(<key>@<timestamp>) -> <key>
//
// Consider the keys "a@1", "a@2", "aa@3", "aa@4". The prefixes for these keys
// are "a", and "aa". Note that despite "a" and "aa" sharing a prefix by the
// usual definition, those prefixes differ by the definition of the Split
// function. To see how this works, consider the following set of calls on this
// data set:
//
// SeekPrefixGE("a@0") -> "a@1"
// Next() -> "a@2"
// Next() -> EOF
//
// If you're just looking to iterate over keys with a shared prefix, as
// defined by the configured comparer, set iterator bounds instead:
//
// iter := db.NewIter(&pebble.IterOptions{
// LowerBound: []byte("prefix"),
// UpperBound: []byte("prefiy"),
// })
// for iter.First(); iter.Valid(); iter.Next() {
// // Only keys beginning with "prefix" will be visited.
// }
//
// See ExampleIterator_SeekPrefixGE for a working example.
func (i *Iterator) SeekPrefixGE(key []byte) bool {
lastPositioningOp := i.lastPositioningOp
// Set it to unknown, since this operation may not succeed, in which case
// the SeekPrefixGE following this should not make any assumption about
// iterator position.
i.lastPositioningOp = unknownLastPositionOp
i.err = nil // clear cached iteration error
i.stats.ForwardSeekCount[InterfaceCall]++
if i.split == nil {
panic("pebble: split must be provided for SeekPrefixGE")
}
prefixLen := i.split(key)
keyPrefix := key[:prefixLen]
trySeekUsingNext := false
if lastPositioningOp == seekPrefixGELastPositioningOp {
if !i.hasPrefix {
panic("lastPositioningOpsIsSeekPrefixGE is true, but hasPrefix is false")
}
// The iterator has not been repositioned after the last SeekPrefixGE.
// See if we are seeking to a larger key, since then we can optimize
// the seek by using next. Note that we could also optimize if Next
// has been called, if the iterator is not exhausted and the current
// position is <= the seek key. We are keeping this limited for now
// since such optimizations require care for correctness, and to not
// become de-optimizations (if one usually has to do all the next
// calls and then the seek). This SeekPrefixGE optimization
// specifically benefits CockroachDB.
cmp := i.cmp(i.prefixOrFullSeekKey, keyPrefix)
// cmp == 0 is not safe to optimize since
// - i.pos could be at iterPosNext, due to a merge.
// - Even if i.pos were at iterPosCurForward, we could have a DELETE,
// SET pair for a key, and the iterator would have moved past DELETE
// but stayed at iterPosCurForward. A similar situation occurs for a
// MERGE, SET pair where the MERGE is consumed and the iterator is
// at the SET.
// In general some versions of i.prefix could have been consumed by
// the iterator, so we only optimize for cmp < 0.
trySeekUsingNext = cmp < 0
if invariants.Enabled && trySeekUsingNext && disableSeekOpt(key, uintptr(unsafe.Pointer(i))) {
trySeekUsingNext = false
}
}
// Make a copy of the prefix so that modifications to the key after
// SeekPrefixGE returns does not affect the stored prefix.
if cap(i.prefixOrFullSeekKey) < prefixLen {
i.prefixOrFullSeekKey = make([]byte, prefixLen)
} else {
i.prefixOrFullSeekKey = i.prefixOrFullSeekKey[:prefixLen]
}
i.hasPrefix = true