-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
pebble_mvcc_scanner.go
1477 lines (1356 loc) · 51.6 KB
/
pebble_mvcc_scanner.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 2019 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package storage
import (
"bytes"
"context"
"encoding/binary"
"sort"
"sync"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
)
// Key value lengths take up 8 bytes (2 x Uint32).
const kvLenSize = 8
var maxItersBeforeSeek = util.ConstantWithMetamorphicTestRange(
"mvcc-max-iters-before-seek",
10, /* defaultValue */
0, /* min */
3, /* max */
)
// Struct to store MVCCScan / MVCCGet in the same binary format as that
// expected by MVCCScanDecodeKeyValue.
type pebbleResults struct {
count int64
bytes int64
repr []byte
bufs [][]byte
// lastOffsets is a ring buffer that keeps track of byte offsets for the last
// N KV pairs. It is used to discard a partial SQL row at the end of the
// result via maybeTrimPartialLastRows() -- such rows can span multiple KV
// pairs. The length of lastOffsets is interpreted as the maximum expected SQL
// row size (i.e. number of column families).
//
// lastOffsets is initialized with a fixed length giving the N number of last
// KV pair offsets to track. lastOffsetIdx contains the index in lastOffsets
// where the next KV byte offset will be written, wrapping around to 0 when it
// reaches the end of lastOffsets.
//
// The lastOffsets values are byte offsets in p.repr and p.bufs. The latest
// lastOffset (i.e. the one at lastOffsetIdx-1) will be an offset in p.repr.
// When iterating backwards through the ring buffer and crossing a byte offset
// of 0, the next iterated byte offset in the ring buffer (at i-1) will then
// point to the previous buffer in p.bufs.
//
// Actual and default 0 values in the slice are disambiguated when iterating
// backwards through p.repr and p.bufs. If we iterate to the start of all byte
// buffers without iterating through all of lastOffsets (i.e. when there are
// fewer KV pairs than the length of lastOffsets), then we must be at the start
// of lastOffsets, and any 0 values at the end are of no interest.
lastOffsetsEnabled bool // NB: significantly faster than checking lastOffsets != nil
lastOffsets []int
lastOffsetIdx int
}
func (p *pebbleResults) clear() {
*p = pebbleResults{}
}
// The repr that MVCCScan / MVCCGet expects to provide as output goes:
// <valueLen:Uint32><keyLen:Uint32><Key><Value>
// This function adds to repr in that format.
// - maxNewSize, if positive, indicates the maximum capacity for a new repr that
// can be allocated. It is assumed that maxNewSize (when positive) is sufficient
// for the new key-value pair.
func (p *pebbleResults) put(
ctx context.Context, key []byte, value []byte, memAccount *mon.BoundAccount, maxNewSize int,
) error {
const minSize = 16
const maxSize = 128 << 20 // 128 MB
// We maintain a list of buffers, always encoding into the last one (a.k.a.
// pebbleResults.repr). The size of the buffers is exponentially increasing,
// capped at maxSize. The exponential increase allows us to amortize the
// cost of the allocation over multiple put calls. If this (key, value) pair
// needs capacity greater than maxSize, we allocate exactly the size needed.
lenKey := len(key)
lenValue := len(value)
lenToAdd := p.sizeOf(lenKey, lenValue)
if len(p.repr)+lenToAdd > cap(p.repr) {
// Exponential increase by default, while ensuring that we respect
// - a hard lower bound of lenToAdd
// - a soft upper bound of maxSize
// - a hard upper bound of maxNewSize (if set).
if maxNewSize > 0 && maxNewSize < lenToAdd {
// Hard upper bound is greater than hard lower bound - this is a
// violation of our assumptions.
return errors.AssertionFailedf("maxNewSize %dB is not sufficient, %dB required", maxNewSize, lenToAdd)
}
// Exponential growth to ensure newSize >= lenToAdd.
newSize := 2 * cap(p.repr)
if newSize == 0 || newSize > maxSize {
// If the previous buffer exceeded maxSize, we don't double its
// capacity for next allocation, and instead reset the exponential
// increase, in case we had a stray huge key-value.
newSize = minSize
}
for newSize < lenToAdd {
newSize *= 2
}
// Respect soft upper-bound before hard lower-bound, since it could be
// lower than hard lower-bound.
if newSize > maxSize {
newSize = maxSize
}
// Respect hard upper-bound.
if maxNewSize > 0 && newSize > maxNewSize {
newSize = maxNewSize
}
// Now respect hard lower-bound.
if newSize < lenToAdd {
newSize = lenToAdd
}
if len(p.repr) > 0 {
p.bufs = append(p.bufs, p.repr)
}
if err := memAccount.Grow(ctx, int64(newSize)); err != nil {
return err
}
p.repr = nonZeroingMakeByteSlice(newSize)[:0]
}
startIdx := len(p.repr)
p.repr = p.repr[:startIdx+lenToAdd]
binary.LittleEndian.PutUint32(p.repr[startIdx:], uint32(lenValue))
binary.LittleEndian.PutUint32(p.repr[startIdx+4:], uint32(lenKey))
copy(p.repr[startIdx+kvLenSize:], key)
copy(p.repr[startIdx+kvLenSize+lenKey:], value)
p.count++
p.bytes += int64(lenToAdd)
// If we're tracking KV offsets, update the ring buffer.
if p.lastOffsetsEnabled {
p.lastOffsets[p.lastOffsetIdx] = startIdx
p.lastOffsetIdx++
// NB: Branching is significantly faster than modulo in benchmarks, likely
// because of a high branch prediction hit rate.
if p.lastOffsetIdx == len(p.lastOffsets) {
p.lastOffsetIdx = 0
}
}
return nil
}
func (p *pebbleResults) sizeOf(lenKey, lenValue int) int {
return kvLenSize + lenKey + lenValue
}
// continuesFirstRow returns true if the given key belongs to the same SQL row
// as the first KV pair in the result (or if the result is empty). If either
// key is not a valid SQL row key, returns false.
func (p *pebbleResults) continuesFirstRow(key roachpb.Key) bool {
repr := p.repr
if len(p.bufs) > 0 {
repr = p.bufs[0]
}
if len(repr) == 0 {
return true // no rows in the result
}
rowPrefix := getRowPrefix(key)
if rowPrefix == nil {
return false
}
return bytes.Equal(rowPrefix, getRowPrefix(extractResultKey(repr)))
}
// lastRowHasFinalColumnFamily returns true if the last key in the result is the
// maximum column family ID of the row (i.e. when it equals len(lastOffsets)-1).
// If so, we know that the row is complete. However, the inverse is not true:
// the final column families of the row may be omitted, in which case the caller
// has to scan to the next key to find out whether the row is complete.
func (p *pebbleResults) lastRowHasFinalColumnFamily() bool {
if !p.lastOffsetsEnabled || p.count == 0 {
return false
}
lastOffsetIdx := p.lastOffsetIdx - 1 // p.lastOffsetIdx is where next offset would be stored
if lastOffsetIdx < 0 {
lastOffsetIdx = len(p.lastOffsets) - 1
}
lastOffset := p.lastOffsets[lastOffsetIdx]
key := extractResultKey(p.repr[lastOffset:])
colFamilyID, err := keys.DecodeFamilyKey(key)
if err != nil {
return false
}
return int(colFamilyID) == len(p.lastOffsets)-1
}
// maybeTrimPartialLastRow removes the last KV pairs from the result that are part
// of the same SQL row as the given key, returning the earliest key removed. The
// row cannot be made up of more KV pairs than given by len(lastOffsets),
// otherwise an error is returned. Must be called before finish().
func (p *pebbleResults) maybeTrimPartialLastRow(nextKey roachpb.Key) (roachpb.Key, error) {
if !p.lastOffsetsEnabled || len(p.repr) == 0 {
return nil, nil
}
trimRowPrefix := getRowPrefix(nextKey)
if trimRowPrefix == nil {
return nil, nil
}
var firstTrimmedKey roachpb.Key
// We're iterating backwards through the p.lastOffsets ring buffer, starting
// at p.lastOffsetIdx-1 (which is where the last KV was stored). The loop
// condition simply makes sure we limit the number of iterations to the size
// of the ring buffer, to prevent wrapping around.
for i := 0; i < len(p.lastOffsets); i++ {
lastOffsetIdx := p.lastOffsetIdx - 1 // p.lastOffsetIdx is where next offset would be stored
if lastOffsetIdx < 0 {
lastOffsetIdx = len(p.lastOffsets) - 1
}
lastOffset := p.lastOffsets[lastOffsetIdx]
// The remainder of repr from the offset is now a single KV.
repr := p.repr[lastOffset:]
key := extractResultKey(repr)
rowPrefix := getRowPrefix(key)
// If the prefix belongs to a different row, we're done trimming.
if !bytes.Equal(rowPrefix, trimRowPrefix) {
return firstTrimmedKey, nil
}
// Remove this KV pair.
p.repr = p.repr[:lastOffset]
p.count--
p.bytes -= int64(len(repr))
firstTrimmedKey = key
p.lastOffsetIdx = lastOffsetIdx
p.lastOffsets[lastOffsetIdx] = 0
if len(p.repr) == 0 {
if len(p.bufs) == 0 {
// The entire result set was trimmed, so we're done.
return firstTrimmedKey, nil
}
// Pop the last buf back into repr.
p.repr = p.bufs[len(p.bufs)-1]
p.bufs = p.bufs[:len(p.bufs)-1]
}
}
return nil, errors.Errorf("row exceeds expected max size (%d): %s", len(p.lastOffsets), nextKey)
}
func (p *pebbleResults) finish() [][]byte {
if len(p.repr) > 0 {
p.bufs = append(p.bufs, p.repr)
p.repr = nil
}
return p.bufs
}
// getRowPrefix decodes a SQL row prefix from the given key. Returns nil if the
// key is not a valid SQL row, or if the prefix is the entire key.
func getRowPrefix(key roachpb.Key) []byte {
if len(key) == 0 {
return nil
}
n, err := keys.GetRowPrefixLength(key)
if err != nil || n <= 0 || n >= len(key) {
return nil
}
return key[:n]
}
// extractResultKey takes in a binary KV result representation, finds the raw
// key, decodes it as an MVCC key, and returns the key (without timestamp).
// Returns nil if the key could not be decoded. repr must be a valid, non-empty
// KV representation, otherwise this may panic.
func extractResultKey(repr []byte) roachpb.Key {
keyLen := binary.LittleEndian.Uint32(repr[4:8])
key, ok := DecodeEngineKey(repr[8 : 8+keyLen])
if !ok {
return nil
}
return key.Key
}
// pebbleMVCCScanner handles MVCCScan / MVCCGet using a Pebble iterator. If any
// MVCC range tombstones are encountered, it synthesizes MVCC point tombstones
// by switching to a pointSynthesizingIter.
type pebbleMVCCScanner struct {
parent MVCCIterator
// parentReverse is true if the previous parent positioning operation was a
// reverse operation (SeekLT or Prev). This is needed to correctly initialize
// pointIter when encountering an MVCC range tombstone in reverse.
// TODO(erikgrinaker): Consider adding MVCCIterator.IsReverse() instead.
parentReverse bool
// pointIter is a point synthesizing iterator that wraps and replaces parent
// when an MVCC range tombstone is encountered. A separate reference to it is
// kept in order to release it back to its pool when the scanner is done.
pointIter *PointSynthesizingIter
// memAccount is used to account for the size of the scan results.
memAccount *mon.BoundAccount
// lockTable is used to determine whether keys are locked in the in-memory
// lock table when scanning with the skipLocked option.
lockTable LockTableView
reverse bool
peeked bool
// Iteration bounds. Does not contain MVCC timestamp.
start, end roachpb.Key
// Timestamp with which MVCCScan/MVCCGet was called.
ts hlc.Timestamp
// Max number of keys to return.
maxKeys int64
// Stop adding keys once p.result.bytes matches or exceeds this threshold,
// if nonzero.
targetBytes int64
// If true, return an empty result if the first result exceeds targetBytes.
allowEmpty bool
// If set, don't return partial SQL rows (spanning multiple KV pairs) when
// hitting a limit. Partial rows at the end of the result will be trimmed. If
// allowEmpty is false, and the partial row is the first row in the result,
// the row will instead be completed by fetching additional KV pairs.
//
// Requires init() to have been called with trackLastOffsets set to the
// maximum number of KV pairs in a row.
wholeRows bool
// Stop adding intents and abort scan once maxIntents threshold is reached.
// This limit is only applicable to consistent scans since they return
// intents as an error.
// Not used in inconsistent scans.
// Ignored if zero.
maxIntents int64
// Resume fields describe the resume span to return. resumeReason must be set
// to a non-zero value to return a resume span, the others are optional.
resumeReason roachpb.ResumeReason
resumeKey roachpb.Key // if unset, falls back to p.advanceKey()
resumeNextBytes int64 // set when targetBytes is exceeded
// Transaction epoch and sequence number.
txn *roachpb.Transaction
txnEpoch enginepb.TxnEpoch
txnSequence enginepb.TxnSeq
txnIgnoredSeqNums []enginepb.IgnoredSeqNumRange
// Uncertainty related fields.
uncertainty uncertainty.Interval
checkUncertainty bool
// Metadata object for unmarshalling intents.
meta enginepb.MVCCMetadata
// Bools copied over from MVCC{Scan,Get}Options. See the comment on the
// package level MVCCScan for what these mean.
inconsistent bool
skipLocked bool
tombstones bool
failOnMoreRecent bool
isGet bool
keyBuf []byte
savedBuf []byte
// cur* variables store the "current" record we're pointing to. Updated in
// updateCurrent. Note that the timestamp can be clobbered in the case of
// adding an intent from the intent history but is otherwise meaningful.
curUnsafeKey MVCCKey
curRawKey []byte
curUnsafeValue MVCCValue
curRawValue []byte
results pebbleResults
intents pebble.Batch
// mostRecentTS stores the largest timestamp observed that is equal to or
// above the scan timestamp. Only applicable if failOnMoreRecent is true. If
// set and no other error is hit, a WriteToOld error will be returned from
// the scan. mostRecentKey is one of the keys (not necessarily at
// mostRecentTS) that was more recent than the scan.
mostRecentTS hlc.Timestamp
mostRecentKey roachpb.Key
// Stores any error returned. If non-nil, iteration short circuits.
err error
// Number of iterations to try before we do a Seek/SeekReverse. Stays within
// [0, maxItersBeforeSeek] and defaults to maxItersBeforeSeek/2 .
itersBeforeSeek int
}
// Pool for allocating pebble MVCC Scanners.
var pebbleMVCCScannerPool = sync.Pool{
New: func() interface{} {
return &pebbleMVCCScanner{}
},
}
func (p *pebbleMVCCScanner) release() {
if p.pointIter != nil {
p.pointIter.release()
}
// Discard most memory references before placing in pool.
*p = pebbleMVCCScanner{
keyBuf: p.keyBuf,
}
pebbleMVCCScannerPool.Put(p)
}
// init sets bounds on the underlying pebble iterator, and initializes other
// fields not set by the calling method.
func (p *pebbleMVCCScanner) init(
txn *roachpb.Transaction, ui uncertainty.Interval, trackLastOffsets int,
) {
p.itersBeforeSeek = maxItersBeforeSeek / 2
if trackLastOffsets > 0 {
p.results.lastOffsetsEnabled = true
p.results.lastOffsets = make([]int, trackLastOffsets)
}
if txn != nil {
p.txn = txn
p.txnEpoch = txn.Epoch
p.txnSequence = txn.Sequence
p.txnIgnoredSeqNums = txn.IgnoredSeqNums
}
p.uncertainty = ui
// We must check uncertainty even if p.ts >= local_uncertainty_limit
// because the local uncertainty limit cannot be applied to values with
// synthetic timestamps. We are only able to skip uncertainty checks if
// p.ts >= global_uncertainty_limit.
p.checkUncertainty = p.ts.Less(p.uncertainty.GlobalLimit)
}
// get iterates exactly once and adds one KV to the result set.
func (p *pebbleMVCCScanner) get(ctx context.Context) {
p.isGet = true
// The iterator may already be positioned on a range key that SeekGE hits, in
// which case RangeKeyChanged() wouldn't fire, so we enable point synthesis
// here if needed. We check this before SeekGE, because in the typical case
// this will be a new, unpositioned iterator, which allows omitting the
// HasPointAndRange() call.
if ok, _ := p.parent.Valid(); ok {
if _, hasRange := p.parent.HasPointAndRange(); hasRange {
if !p.enablePointSynthesis() {
return
}
}
}
p.parentReverse = false
p.parent.SeekGE(MVCCKey{Key: p.start})
if !p.updateCurrent() {
return
}
p.getAndAdvance(ctx)
p.maybeFailOnMoreRecent()
}
// scan iterates until a limit is exceeded, the underlying iterator is
// exhausted, or an error is encountered. If a limit was exceeded, it returns a
// resume span, resume reason, and for targetBytes the size of the next result.
func (p *pebbleMVCCScanner) scan(
ctx context.Context,
) (*roachpb.Span, roachpb.ResumeReason, int64, error) {
if p.wholeRows && !p.results.lastOffsetsEnabled {
return nil, 0, 0, errors.AssertionFailedf("cannot use wholeRows without trackLastOffsets")
}
p.isGet = false
// The iterator may already be positioned on a range key that the seek hits,
// in which case RangeKeyChanged() wouldn't fire, so we enable point synthesis
// here if needed. We check this before seeking, because in the typical case
// this will be a new, unpositioned iterator, which allows omitting the
// HasPointAndRange() call.
if ok, _ := p.parent.Valid(); ok {
if _, hasRange := p.parent.HasPointAndRange(); hasRange {
if !p.enablePointSynthesis() {
return nil, 0, 0, p.err
}
}
}
if p.reverse {
if !p.iterSeekReverse(MVCCKey{Key: p.end}) {
return nil, 0, 0, p.err
}
} else {
if !p.iterSeek(MVCCKey{Key: p.start}) {
return nil, 0, 0, p.err
}
}
for p.getAndAdvance(ctx) {
}
p.maybeFailOnMoreRecent()
if p.err != nil {
return nil, 0, 0, p.err
}
if p.resumeReason != 0 {
resumeKey := p.resumeKey
if len(resumeKey) == 0 {
if !p.advanceKey() {
return nil, 0, 0, nil // nothing to resume
}
resumeKey = p.curUnsafeKey.Key
}
var resumeSpan *roachpb.Span
if p.reverse {
// NB: this is equivalent to:
// append(roachpb.Key(nil), resumeKey...).Next()
// but with half the allocations.
resumeKeyCopy := make(roachpb.Key, len(resumeKey), len(resumeKey)+1)
copy(resumeKeyCopy, resumeKey)
resumeSpan = &roachpb.Span{
Key: p.start,
EndKey: resumeKeyCopy.Next(),
}
} else {
resumeSpan = &roachpb.Span{
Key: append(roachpb.Key(nil), resumeKey...),
EndKey: p.end,
}
}
return resumeSpan, p.resumeReason, p.resumeNextBytes, nil
}
return nil, 0, 0, nil
}
// Increments itersBeforeSeek while ensuring it stays <= maxItersBeforeSeek
func (p *pebbleMVCCScanner) incrementItersBeforeSeek() {
p.itersBeforeSeek++
if p.itersBeforeSeek > maxItersBeforeSeek {
p.itersBeforeSeek = maxItersBeforeSeek
}
}
// Decrements itersBeforeSeek while ensuring it stays positive.
func (p *pebbleMVCCScanner) decrementItersBeforeSeek() {
p.itersBeforeSeek--
if p.itersBeforeSeek < 1 {
if maxItersBeforeSeek > 0 {
p.itersBeforeSeek = 1
} else if p.itersBeforeSeek < 0 {
// INVARIANT: maxItersBeforeSeek == 0 && p.itersBeforeSeek < 0.
p.itersBeforeSeek = 0
}
}
}
// Try to read from the current value's intent history. Assumes p.meta has been
// unmarshalled already. Returns found = true if a value was found and returned.
func (p *pebbleMVCCScanner) getFromIntentHistory() (value []byte, found bool) {
intentHistory := p.meta.IntentHistory
// upIdx is the index of the first intent in intentHistory with a sequence
// number greater than our transaction's sequence number. Subtract 1 from it
// to get the index of the intent with the highest sequence number that is
// still less than or equal to p.txnSeq.
upIdx := sort.Search(len(intentHistory), func(i int) bool {
return intentHistory[i].Sequence > p.txnSequence
})
// If the candidate intent has a sequence number that is ignored by this txn,
// iterate backward along the sorted intent history until we come across an
// intent which isn't ignored.
//
// TODO(itsbilal): Explore if this iteration can be improved through binary
// search.
for upIdx > 0 && enginepb.TxnSeqIsIgnored(p.meta.IntentHistory[upIdx-1].Sequence, p.txnIgnoredSeqNums) {
upIdx--
}
if upIdx == 0 {
// It is possible that no intent exists such that the sequence is less
// than the read sequence, and is not ignored by this transaction.
// In this case, we cannot read a value from the intent history.
return nil, false
}
intent := &p.meta.IntentHistory[upIdx-1]
return intent.Value, true
}
// Returns a write too old error if an error is not already set on the scanner
// and a more recent value was found during the scan.
func (p *pebbleMVCCScanner) maybeFailOnMoreRecent() {
if p.err != nil || p.mostRecentTS.IsEmpty() {
return
}
// The txn can't write at the existing timestamp, so we provide the error
// with the timestamp immediately after it.
p.err = roachpb.NewWriteTooOldError(p.ts, p.mostRecentTS.Next(), p.mostRecentKey)
p.results.clear()
p.intents.Reset()
}
// Returns an uncertainty error with the specified timestamp and p.txn.
func (p *pebbleMVCCScanner) uncertaintyError(ts hlc.Timestamp) bool {
p.err = roachpb.NewReadWithinUncertaintyIntervalError(
p.ts, ts, p.uncertainty.LocalLimit.ToTimestamp(), p.txn)
p.results.clear()
p.intents.Reset()
return false
}
// Emit a tuple and return true if we have reason to believe iteration can
// continue.
func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool {
if !p.curUnsafeKey.Timestamp.IsEmpty() {
if extended, valid := p.tryDecodeCurrentValueSimple(); !valid {
return false
} else if extended {
if !p.decodeCurrentValueExtended() {
return false
}
}
// ts < read_ts
if p.curUnsafeKey.Timestamp.Less(p.ts) {
// 1. Fast path: there is no intent and our read timestamp is newer
// than the most recent version's timestamp.
return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes)
}
// ts == read_ts
if p.curUnsafeKey.Timestamp.EqOrdering(p.ts) {
if p.failOnMoreRecent {
// 2. Our txn's read timestamp is equal to the most recent
// version's timestamp and the scanner has been configured to
// throw a write too old error on equal or more recent versions.
if p.skipLocked {
if locked, ok := p.isKeyLockedByConflictingTxn(ctx, p.curRawKey); !ok {
return false
} else if locked {
// 2a. the scanner was configured to skip locked keys, and
// this key was locked, so we can advance past it without
// raising the write too old error.
return p.advanceKey()
}
}
// 2b. We need to raise a write too old error. Merge the current
// timestamp with the maximum timestamp we've seen so we know to
// return an error, but then keep scanning so that we can return
// the largest possible time.
p.mostRecentTS.Forward(p.curUnsafeKey.Timestamp)
if len(p.mostRecentKey) == 0 {
p.mostRecentKey = append(p.mostRecentKey, p.curUnsafeKey.Key...)
}
return p.advanceKey()
}
// 3. There is no intent and our read timestamp is equal to the most
// recent version's timestamp.
return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes)
}
// ts > read_ts
if p.failOnMoreRecent {
// 4. Our txn's read timestamp is less than the most recent
// version's timestamp and the scanner has been configured to
// throw a write too old error on equal or more recent versions.
if p.skipLocked {
if locked, ok := p.isKeyLockedByConflictingTxn(ctx, p.curRawKey); !ok {
return false
} else if locked {
// 4a. the scanner was configured to skip locked keys, and
// this key was locked, so we can advance past it without
// raising the write too old error.
return p.advanceKey()
}
}
// 4b. We need to raise a write too old error. Merge the current
// timestamp with the maximum timestamp we've seen so we know to
// return an error, but then keep scanning so that we can return
// the largest possible time.
p.mostRecentTS.Forward(p.curUnsafeKey.Timestamp)
if len(p.mostRecentKey) == 0 {
p.mostRecentKey = append(p.mostRecentKey, p.curUnsafeKey.Key...)
}
return p.advanceKey()
}
if p.checkUncertainty {
// 5. Our txn's read timestamp is less than the max timestamp
// seen by the txn. We need to check for clock uncertainty
// errors.
localTS := p.curUnsafeValue.GetLocalTimestamp(p.curUnsafeKey.Timestamp)
if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, localTS) {
return p.uncertaintyError(p.curUnsafeKey.Timestamp)
}
// This value is not within the reader's uncertainty window, but
// there could be other uncertain committed values, so seek and
// check uncertainty using the uncertainty interval's GlobalLimit.
return p.seekVersion(ctx, p.uncertainty.GlobalLimit, true)
}
// 6. Our txn's read timestamp is greater than or equal to the
// max timestamp seen by the txn so clock uncertainty checks are
// unnecessary. We need to seek to the desired version of the
// value (i.e. one with a timestamp earlier than our read
// timestamp).
return p.seekVersion(ctx, p.ts, false)
}
if !p.decodeCurrentMetadata() {
return false
}
if len(p.meta.RawBytes) != 0 {
// 7. Emit immediately if the value is inline.
return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.meta.RawBytes)
}
if p.meta.Txn == nil {
p.err = errors.Errorf("intent without transaction")
return false
}
metaTS := p.meta.Timestamp.ToTimestamp()
// metaTS is the timestamp of an intent value, which we may or may
// not end up ignoring, depending on factors codified below. If we do ignore
// the intent then we want to read at a lower timestamp that's strictly
// below the intent timestamp (to skip the intent), but also does not exceed
// our read timestamp (to avoid erroneously picking up future committed
// values); this timestamp is prevTS.
prevTS := p.ts
if metaTS.LessEq(p.ts) {
prevTS = metaTS.Prev()
}
ownIntent := p.txn != nil && p.meta.Txn.ID.Equal(p.txn.ID)
if !ownIntent {
conflictingIntent := metaTS.LessEq(p.ts) || p.failOnMoreRecent
if !conflictingIntent {
// 8. The key contains an intent, but we're reading below the intent.
// Seek to the desired version, checking for uncertainty if necessary.
//
// Note that if we own the intent (i.e. we're reading transactionally)
// we want to read the intent regardless of our read timestamp and fall
// into case 11 below.
if p.checkUncertainty {
// The intent's provisional value may be within the uncertainty window.
// Or there could be a different, uncertain committed value in the
// window. To detect either case, seek to and past the uncertainty
// interval's global limit and check uncertainty as we scan.
return p.seekVersion(ctx, p.uncertainty.GlobalLimit, true)
}
return p.seekVersion(ctx, p.ts, false)
}
if p.inconsistent {
// 9. The key contains an intent and we're doing an inconsistent
// read at a timestamp newer than the intent. We ignore the
// intent by insisting that the timestamp we're reading at is a
// historical timestamp < the intent timestamp. However, we
// return the intent separately; the caller may want to resolve
// it.
//
// p.intents is a pebble.Batch which grows its byte slice capacity in
// chunks to amortize allocations. The memMonitor is under-counting here
// by only accounting for the key and value bytes.
if !p.addCurIntent(ctx) {
return false
}
return p.seekVersion(ctx, prevTS, false)
}
if p.skipLocked {
// 10. The scanner has been configured with the skipLocked option. Ignore
// intents written by other transactions and seek to the next key.
// However, we return the intent separately if we have room; the caller
// may want to resolve it. Unlike below, this intent will not result in
// a WriteIntentError because MVCC{Scan,Get}Options.errOnIntents returns
// false when skipLocked in enabled.
if p.maxIntents == 0 || int64(p.intents.Count()) < p.maxIntents {
if !p.addCurIntent(ctx) {
return false
}
}
return p.advanceKey()
}
// 11. The key contains an intent which was not written by our
// transaction and either:
// - our read timestamp is equal to or newer than that of the
// intent
// - our read timestamp is older than that of the intent but
// the intent is in our transaction's uncertainty interval
// - our read timestamp is older than that of the intent but
// we want to fail on more recent writes
// Note that this will trigger an error higher up the stack. We
// continue scanning so that we can return all of the intents
// in the scan range.
if !p.addCurIntent(ctx) {
return false
}
// Limit number of intents returned in write intent error.
if p.maxIntents > 0 && int64(p.intents.Count()) >= p.maxIntents {
p.resumeReason = roachpb.RESUME_INTENT_LIMIT
return false
}
return p.advanceKey()
}
if p.txnEpoch == p.meta.Txn.Epoch {
if p.txnSequence >= p.meta.Txn.Sequence && !enginepb.TxnSeqIsIgnored(p.meta.Txn.Sequence, p.txnIgnoredSeqNums) {
// 12. We're reading our own txn's intent at an equal or higher sequence.
// Note that we read at the intent timestamp, not at our read timestamp
// as the intent timestamp may have been pushed forward by another
// transaction. Txn's always need to read their own writes.
return p.seekVersion(ctx, metaTS, false)
}
// 13. We're reading our own txn's intent at a lower sequence than is
// currently present in the intent. This means the intent we're seeing
// was written at a higher sequence than the read and that there may or
// may not be earlier versions of the intent (with lower sequence
// numbers) that we should read. If there exists a value in the intent
// history that has a sequence number equal to or less than the read
// sequence, read that value.
if intentValueRaw, found := p.getFromIntentHistory(); found {
// If we're adding a value due to a previous intent, we want to populate
// the timestamp as of current metaTimestamp. Note that this may be
// controversial as this maybe be neither the write timestamp when this
// intent was written. However, this was the only case in which a value
// could have been returned from a read without an MVCC timestamp.
//
// Note: this assumes that it is safe to corrupt curKey here because we're
// about to advance. If this proves to be a problem later, we can extend
// addAndAdvance to take an MVCCKey explicitly.
p.curUnsafeKey.Timestamp = metaTS
p.keyBuf = EncodeMVCCKeyToBuf(p.keyBuf[:0], p.curUnsafeKey)
p.curUnsafeValue, p.err = DecodeMVCCValue(intentValueRaw)
if p.err != nil {
return false
}
return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.keyBuf, p.curUnsafeValue.Value.RawBytes)
}
// 14. If no value in the intent history has a sequence number equal to
// or less than the read, we must ignore the intents laid down by the
// transaction all together. We ignore the intent by insisting that the
// timestamp we're reading at is a historical timestamp < the intent
// timestamp.
return p.seekVersion(ctx, prevTS, false)
}
if p.txnEpoch < p.meta.Txn.Epoch {
// 15. We're reading our own txn's intent but the current txn has
// an earlier epoch than the intent. Return an error so that the
// earlier incarnation of our transaction aborts (presumably
// this is some operation that was retried).
p.err = errors.Errorf("failed to read with epoch %d due to a write intent with epoch %d",
p.txnEpoch, p.meta.Txn.Epoch)
return false
}
// 16. We're reading our own txn's intent but the current txn has a
// later epoch than the intent. This can happen if the txn was
// restarted and an earlier iteration wrote the value we're now
// reading. In this case, we ignore the intent and read the
// previous value as if the transaction were starting fresh.
return p.seekVersion(ctx, prevTS, false)
}
// nextKey advances to the next user key.
func (p *pebbleMVCCScanner) nextKey() bool {
p.keyBuf = append(p.keyBuf[:0], p.curUnsafeKey.Key...)
for i := 0; i < p.itersBeforeSeek; i++ {
if !p.iterNext() {
return false
}
if !bytes.Equal(p.curUnsafeKey.Key, p.keyBuf) {
p.incrementItersBeforeSeek()
return true
}
}
p.decrementItersBeforeSeek()
// We're pointed at a different version of the same key. Fall back to
// seeking to the next key. We append a NUL to account for the "next-key".
// Note that we cannot rely on curUnsafeKey.Key being unchanged even though
// we are at a different version of the same key -- the underlying
// MVCCIterator is free to mutate the backing for p.curUnsafeKey.Key
// arbitrarily. Therefore we use p.keyBuf here which we have handy.
p.keyBuf = append(p.keyBuf, 0)
return p.iterSeek(MVCCKey{Key: p.keyBuf})
}
// backwardLatestVersion backs up the iterator to the latest version for the
// specified key. The parameter i is used to maintain iteration count between
// the loop here and the caller (usually prevKey). Returns false if the
// iterator was exhausted. Assumes that the iterator is currently positioned at
// the oldest version of key.
func (p *pebbleMVCCScanner) backwardLatestVersion(key []byte, i int) bool {
p.keyBuf = append(p.keyBuf[:0], key...)
for ; i < p.itersBeforeSeek; i++ {
peekedKey, ok := p.iterPeekPrev()
if !ok {
// No previous entry exists, so we're at the latest version of key.
return true
}
if !bytes.Equal(peekedKey, p.keyBuf) {
p.incrementItersBeforeSeek()
return true
}
if !p.iterPrev() {
return false
}
}
// We're still not pointed to the latest version of the key. Fall back to
// seeking to the latest version. Note that we cannot rely on key being
// unchanged even though we are at a different version of the same key --
// the underlying MVCCIterator is free to mutate the backing for key
// arbitrarily. Therefore we use p.keyBuf here which we have handy.
p.decrementItersBeforeSeek()
return p.iterSeek(MVCCKey{Key: p.keyBuf})
}
// prevKey advances to the newest version of the user key preceding the
// specified key. Assumes that the iterator is currently positioned at
// key or 1 record after key.
func (p *pebbleMVCCScanner) prevKey(key []byte) bool {
p.keyBuf = append(p.keyBuf[:0], key...)
for i := 0; i < p.itersBeforeSeek; i++ {
peekedKey, ok := p.iterPeekPrev()
if !ok {
return false
}
if !bytes.Equal(peekedKey, p.keyBuf) {
return p.backwardLatestVersion(peekedKey, i+1)
}
if !p.iterPrev() {
return false
}
}
p.decrementItersBeforeSeek()
return p.iterSeekReverse(MVCCKey{Key: p.keyBuf})
}
// advanceKey advances to the next key in the iterator's direction.
func (p *pebbleMVCCScanner) advanceKey() bool {
if p.isGet {
return false
}
if p.reverse {
return p.prevKey(p.curUnsafeKey.Key)
}
return p.nextKey()
}
// advanceKeyAtEnd advances to the next key when the iterator's end has been
// reached.
func (p *pebbleMVCCScanner) advanceKeyAtEnd() bool {
if p.reverse {
// Iterating to the next key might have caused the iterator to reach the
// end of the key space. If that happens, back up to the very last key.
p.peeked = false
p.parentReverse = true
p.parent.SeekLT(MVCCKey{Key: p.end})
if !p.updateCurrent() {
return false
}
return p.advanceKey()
}
// We've reached the end of the iterator and there is nothing left to do.
return false
}
// advanceKeyAtNewKey advances to the key after the specified key, assuming we
// have just reached the specified key.
func (p *pebbleMVCCScanner) advanceKeyAtNewKey(key []byte) bool {
if p.reverse {
// We've advanced to the next key but need to move back to the previous
// key.
return p.prevKey(key)
}
// We're already at the new key so there is nothing to do.
return true
}
// Adds the specified key and value to the result set, excluding tombstones unless