-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
mvcc.go
3931 lines (3629 loc) · 139 KB
/
mvcc.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 2015 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"
"fmt"
"math"
"runtime"
"sort"
"sync"
"time"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
)
const (
// MVCCVersionTimestampSize is the size of the timestamp portion of MVCC
// version keys (used to update stats).
MVCCVersionTimestampSize int64 = 12
// RecommendedMaxOpenFiles is the recommended value for RocksDB's
// max_open_files option.
RecommendedMaxOpenFiles = 10000
// MinimumMaxOpenFiles is the minimum value that RocksDB's max_open_files
// option can be set to. While this should be set as high as possible, the
// minimum total for a single store node must be under 2048 for Windows
// compatibility.
MinimumMaxOpenFiles = 1700
// Default value for maximum number of intents reported by ExportToSST
// and Scan operations in WriteIntentError is set to half of the maximum
// lock table size.
// This value is subject to tuning in real environment as we have more
// data available.
maxIntentsPerWriteIntentErrorDefault = 5000
)
var minWALSyncInterval = settings.RegisterDurationSetting(
settings.TenantWritable,
"rocksdb.min_wal_sync_interval",
"minimum duration between syncs of the RocksDB WAL",
0*time.Millisecond,
)
// MaxIntentsPerWriteIntentError sets maximum number of intents returned in
// WriteIntentError in operations that return multiple intents per error.
// Currently it is used in Scan, ReverseScan, and ExportToSST.
var MaxIntentsPerWriteIntentError = settings.RegisterIntSetting(
settings.TenantWritable,
"storage.mvcc.max_intents_per_error",
"maximum number of intents returned in error during export of scan requests",
maxIntentsPerWriteIntentErrorDefault)
var rocksdbConcurrency = envutil.EnvOrDefaultInt(
"COCKROACH_ROCKSDB_CONCURRENCY", func() int {
// Use up to min(numCPU, 4) threads for background RocksDB compactions per
// store.
const max = 4
if n := runtime.GOMAXPROCS(0); n <= max {
return n
}
return max
}())
// MakeValue returns the inline value.
func MakeValue(meta enginepb.MVCCMetadata) roachpb.Value {
return roachpb.Value{RawBytes: meta.RawBytes}
}
func emptyKeyError() error {
return errors.Errorf("attempted access to empty key")
}
// MVCCKeyValue contains the raw bytes of the value for a key.
type MVCCKeyValue struct {
Key MVCCKey
Value []byte
}
// optionalValue represents an optional roachpb.Value. It is preferred
// over a *roachpb.Value to avoid the forced heap allocation.
type optionalValue struct {
roachpb.Value
exists bool
}
func makeOptionalValue(v roachpb.Value) optionalValue {
return optionalValue{Value: v, exists: true}
}
func (v *optionalValue) IsPresent() bool {
return v.exists && v.Value.IsPresent()
}
func (v *optionalValue) IsTombstone() bool {
return v.exists && !v.Value.IsPresent()
}
func (v *optionalValue) ToPointer() *roachpb.Value {
if !v.exists {
return nil
}
// Copy to prevent forcing receiver onto heap.
cpy := v.Value
return &cpy
}
// isSysLocal returns whether the key is system-local.
func isSysLocal(key roachpb.Key) bool {
return key.Compare(keys.LocalMax) < 0
}
// isAbortSpanKey returns whether the key is an abort span key.
func isAbortSpanKey(key roachpb.Key) bool {
if !bytes.HasPrefix(key, keys.LocalRangeIDPrefix) {
return false
}
_ /* rangeID */, infix, suffix, _ /* detail */, err := keys.DecodeRangeIDKey(key)
if err != nil {
return false
}
hasAbortSpanSuffix := infix.Equal(keys.LocalRangeIDReplicatedInfix) && suffix.Equal(keys.LocalAbortSpanSuffix)
return hasAbortSpanSuffix
}
// updateStatsForInline updates stat counters for an inline value
// (abort span entries for example). These are simpler as they don't
// involve intents or multiple versions.
func updateStatsForInline(
ms *enginepb.MVCCStats,
key roachpb.Key,
origMetaKeySize, origMetaValSize, metaKeySize, metaValSize int64,
) {
sys := isSysLocal(key)
// Remove counts for this key if the original size is non-zero.
if origMetaKeySize != 0 {
if sys {
ms.SysBytes -= (origMetaKeySize + origMetaValSize)
ms.SysCount--
// We only do this check in updateStatsForInline since
// abort span keys are always inlined - we don't associate
// timestamps with them.
if isAbortSpanKey(key) {
ms.AbortSpanBytes -= (origMetaKeySize + origMetaValSize)
}
} else {
ms.LiveBytes -= (origMetaKeySize + origMetaValSize)
ms.LiveCount--
ms.KeyBytes -= origMetaKeySize
ms.ValBytes -= origMetaValSize
ms.KeyCount--
ms.ValCount--
}
}
// Add counts for this key if the new size is non-zero.
if metaKeySize != 0 {
if sys {
ms.SysBytes += metaKeySize + metaValSize
ms.SysCount++
if isAbortSpanKey(key) {
ms.AbortSpanBytes += metaKeySize + metaValSize
}
} else {
ms.LiveBytes += metaKeySize + metaValSize
ms.LiveCount++
ms.KeyBytes += metaKeySize
ms.ValBytes += metaValSize
ms.KeyCount++
ms.ValCount++
}
}
}
// updateStatsOnMerge updates metadata stats while merging inlined
// values. Unfortunately, we're unable to keep accurate stats on merges as the
// actual details of the merge play out asynchronously during compaction. We
// actually undercount by only adding the size of the value.RawBytes byte slice
// (and eliding MVCCVersionTimestampSize, corresponding to the metadata overhead,
// even for the very "first" write). These errors are corrected during splits and
// merges.
func updateStatsOnMerge(key roachpb.Key, valSize, nowNanos int64) enginepb.MVCCStats {
var ms enginepb.MVCCStats
sys := isSysLocal(key)
ms.AgeTo(nowNanos)
ms.ContainsEstimates = 1
if sys {
ms.SysBytes += valSize
} else {
ms.LiveBytes += valSize
ms.ValBytes += valSize
}
return ms
}
// updateStatsOnPut updates stat counters for a newly put value,
// including both the metadata key & value bytes and the mvcc
// versioned value's key & value bytes. If the value is not a
// deletion tombstone, updates the live stat counters as well.
// If this value is an intent, updates the intent counters.
func updateStatsOnPut(
key roachpb.Key,
prevValSize int64,
origMetaKeySize, origMetaValSize, metaKeySize, metaValSize int64,
orig, meta *enginepb.MVCCMetadata,
) enginepb.MVCCStats {
var ms enginepb.MVCCStats
if isSysLocal(key) {
// Handling system-local keys is straightforward because
// we don't track ageable quantities for them (we
// could, but don't). Remove the contributions from the
// original, if any, and add in the new contributions.
if orig != nil {
ms.SysBytes -= origMetaKeySize + origMetaValSize
if orig.Txn != nil {
// If the original value was an intent, we're replacing the
// intent. Note that since it's a system key, it doesn't affect
// IntentByte, IntentCount, and correspondingly, IntentAge.
ms.SysBytes -= orig.KeyBytes + orig.ValBytes
}
ms.SysCount--
}
ms.SysBytes += meta.KeyBytes + meta.ValBytes + metaKeySize + metaValSize
ms.SysCount++
return ms
}
// Handle non-sys keys. This follows the same scheme: if there was a previous
// value, perhaps even an intent, subtract its contributions, and then add the
// new contributions. The complexity here is that we need to properly update
// GCBytesAge and IntentAge, which don't follow the same semantics. The difference
// between them is that an intent accrues IntentAge from its own timestamp on,
// while GCBytesAge is accrued by versions according to the following rules:
// 1. a (non-tombstone) value that is shadowed by a newer write accrues age at
// the point in time at which it is shadowed (i.e. the newer write's timestamp).
// 2. a tombstone value accrues age at its own timestamp (note that this means
// the tombstone's own contribution only -- the actual write that was deleted
// is then shadowed by this tombstone, and will thus also accrue age from
// the tombstone's value on, as per 1).
//
// This seems relatively straightforward, but only because it omits pesky
// details, which have been relegated to the comments below.
// Remove current live counts for this key.
if orig != nil {
ms.KeyCount--
// Move the (so far empty) stats to the timestamp at which the
// previous entry was created, which is where we wish to reclassify
// its contributions.
ms.AgeTo(orig.Timestamp.WallTime)
// If the original metadata for this key was an intent, subtract
// its contribution from stat counters as it's being replaced.
if orig.Txn != nil {
// Subtract counts attributable to intent we're replacing.
ms.ValCount--
ms.IntentBytes -= (orig.KeyBytes + orig.ValBytes)
ms.IntentCount--
ms.SeparatedIntentCount--
}
// If the original intent is a deletion, we're removing the intent. This
// means removing its contribution at the *old* timestamp because it has
// accrued GCBytesAge that we need to offset (rule 2).
//
// Note that there is a corresponding block for the case of a non-deletion
// (rule 1) below, at meta.Timestamp.
if orig.Deleted {
ms.KeyBytes -= origMetaKeySize
ms.ValBytes -= origMetaValSize
if orig.Txn != nil {
ms.KeyBytes -= orig.KeyBytes
ms.ValBytes -= orig.ValBytes
}
}
// Rule 1 implies that sometimes it's not only the old meta and the new meta
// that matter, but also the version below both of them. For example, take
// a version at t=1 and an intent over it at t=2 that is now being replaced
// (t=3). Then orig.Timestamp will be 2, and meta.Timestamp will be 3, but
// rule 1 tells us that for the interval [2,3) we have already accrued
// GCBytesAge for the version at t=1 that is now moot, because the intent
// at t=2 is moving to t=3; we have to emit a GCBytesAge offset to that effect.
//
// The code below achieves this by making the old version live again at
// orig.Timestamp, and then marking it as shadowed at meta.Timestamp below.
// This only happens when that version wasn't a tombstone, in which case it
// contributes from its own timestamp on anyway, and doesn't need adjustment.
//
// Note that when meta.Timestamp equals orig.Timestamp, the computation is
// moot, which is something our callers may exploit (since retrieving the
// previous version is not for free).
prevIsValue := prevValSize > 0
if prevIsValue {
// If the previous value (exists and) was not a deletion tombstone, make it
// live at orig.Timestamp. We don't have to do anything if there is a
// previous value that is a tombstone: according to rule two its age
// contributions are anchored to its own timestamp, so moving some values
// higher up doesn't affect the contributions tied to that key.
ms.LiveBytes += MVCCVersionTimestampSize + prevValSize
}
// Note that there is an interesting special case here: it's possible that
// meta.Timestamp.WallTime < orig.Timestamp.WallTime. This wouldn't happen
// outside of tests (due to our semantics of txn.ReadTimestamp, which never
// decreases) but it sure does happen in randomized testing. An earlier
// version of the code used `Forward` here, which is incorrect as it would be
// a no-op and fail to subtract out the intent bytes/GC age incurred due to
// removing the meta entry at `orig.Timestamp` (when `orig != nil`).
ms.AgeTo(meta.Timestamp.WallTime)
if prevIsValue {
// Make the previous non-deletion value non-live again, as explained in the
// sibling block above.
ms.LiveBytes -= MVCCVersionTimestampSize + prevValSize
}
// If the original version wasn't a deletion, it becomes non-live at meta.Timestamp
// as this is where it is shadowed.
if !orig.Deleted {
ms.LiveBytes -= orig.KeyBytes + orig.ValBytes
ms.LiveBytes -= origMetaKeySize + origMetaValSize
ms.LiveCount--
ms.KeyBytes -= origMetaKeySize
ms.ValBytes -= origMetaValSize
if orig.Txn != nil {
ms.KeyBytes -= orig.KeyBytes
ms.ValBytes -= orig.ValBytes
}
}
} else {
ms.AgeTo(meta.Timestamp.WallTime)
}
// If the new version isn't a deletion tombstone, add it to live counters.
if !meta.Deleted {
ms.LiveBytes += meta.KeyBytes + meta.ValBytes + metaKeySize + metaValSize
ms.LiveCount++
}
ms.KeyBytes += meta.KeyBytes + metaKeySize
ms.ValBytes += meta.ValBytes + metaValSize
ms.KeyCount++
ms.ValCount++
if meta.Txn != nil {
ms.IntentBytes += meta.KeyBytes + meta.ValBytes
ms.IntentCount++
ms.SeparatedIntentCount++
}
return ms
}
// updateStatsOnResolve updates stat counters with the difference
// between the original and new metadata sizes. The size of the
// resolved value (key & bytes) are subtracted from the intents
// counters if commit=true.
func updateStatsOnResolve(
key roachpb.Key,
prevValSize int64,
origMetaKeySize, origMetaValSize, metaKeySize, metaValSize int64,
orig, meta *enginepb.MVCCMetadata,
commit bool,
) enginepb.MVCCStats {
var ms enginepb.MVCCStats
if isSysLocal(key) {
// Straightforward: old contribution goes, new contribution comes, and we're done.
ms.SysBytes += (metaKeySize + metaValSize) - (origMetaValSize + origMetaKeySize)
return ms
}
// An intent can't turn from deleted to non-deleted and vice versa while being
// resolved.
if orig.Deleted != meta.Deleted {
log.Fatalf(context.TODO(), "on resolve, original meta was deleted=%t, but new one is deleted=%t",
orig.Deleted, meta.Deleted)
}
// In the main case, we had an old intent at orig.Timestamp, and a new intent
// or value at meta.Timestamp. We'll walk through the contributions below,
// taking special care for IntentAge and GCBytesAge.
//
// Jump into the method below for extensive commentary on their semantics
// and "rules one and two".
_ = updateStatsOnPut
ms.AgeTo(orig.Timestamp.WallTime)
// At orig.Timestamp, the original meta key disappears. Fortunately, the
// GCBytesAge computations are fairly transparent because the intent is either
// not a deletion in which case it is always live (it's the most recent value,
// so it isn't shadowed -- see rule 1), or it *is* a deletion, in which case
// its own timestamp is where it starts accruing GCBytesAge (rule 2).
ms.KeyBytes -= origMetaKeySize + orig.KeyBytes
ms.ValBytes -= origMetaValSize + orig.ValBytes
// If the old intent is a deletion, then the key already isn't tracked
// in LiveBytes any more (and the new intent/value is also a deletion).
// If we're looking at a non-deletion intent/value, update the live
// bytes to account for the difference between the previous intent and
// the new intent/value.
if !meta.Deleted {
ms.LiveBytes -= origMetaKeySize + origMetaValSize
ms.LiveBytes -= orig.KeyBytes + meta.ValBytes
}
// IntentAge is always accrued from the intent's own timestamp on.
ms.IntentBytes -= orig.KeyBytes + orig.ValBytes
ms.IntentCount--
ms.SeparatedIntentCount--
// If there was a previous value (before orig.Timestamp), and it was not a
// deletion tombstone, then we have to adjust its GCBytesAge contribution
// which was previously anchored at orig.Timestamp and now has to move to
// meta.Timestamp. Paralleling very similar code in the method below, this
// is achieved by making the previous key live between orig.Timestamp and
// meta.Timestamp. When the two are equal, this will be a zero adjustment,
// and so in that case the caller may simply pass prevValSize=0 and can
// skip computing that quantity in the first place.
_ = updateStatsOnPut
prevIsValue := prevValSize > 0
if prevIsValue {
ms.LiveBytes += MVCCVersionTimestampSize + prevValSize
}
ms.AgeTo(meta.Timestamp.WallTime)
if prevIsValue {
// The previous non-deletion value becomes non-live at meta.Timestamp.
// See the sibling code above.
ms.LiveBytes -= MVCCVersionTimestampSize + prevValSize
}
// At meta.Timestamp, the new meta key appears.
ms.KeyBytes += metaKeySize + meta.KeyBytes
ms.ValBytes += metaValSize + meta.ValBytes
// The new meta key appears.
if !meta.Deleted {
ms.LiveBytes += (metaKeySize + metaValSize) + (meta.KeyBytes + meta.ValBytes)
}
if !commit {
// If not committing, the intent reappears (but at meta.Timestamp).
//
// This is the case in which an intent is pushed (a similar case
// happens when an intent is overwritten, but that's handled in
// updateStatsOnPut, not this method).
ms.IntentBytes += meta.KeyBytes + meta.ValBytes
ms.IntentCount++
ms.SeparatedIntentCount++
}
return ms
}
// updateStatsOnClear updates stat counters by subtracting a
// cleared value's key and value byte sizes. If an earlier version
// was restored, the restored values are added to live bytes and
// count if the restored value isn't a deletion tombstone.
func updateStatsOnClear(
key roachpb.Key,
origMetaKeySize, origMetaValSize, restoredMetaKeySize, restoredMetaValSize int64,
orig, restored *enginepb.MVCCMetadata,
restoredNanos int64,
) enginepb.MVCCStats {
var ms enginepb.MVCCStats
if isSysLocal(key) {
if restored != nil {
ms.SysBytes += restoredMetaKeySize + restoredMetaValSize
ms.SysCount++
}
ms.SysBytes -= (orig.KeyBytes + orig.ValBytes) + (origMetaKeySize + origMetaValSize)
ms.SysCount--
return ms
}
// If we're restoring a previous value (which is thus not an intent), there are
// two main cases:
//
// 1. the previous value is a tombstone, so according to rule 2 it accrues
// GCBytesAge from its own timestamp on (we need to adjust only for the
// implicit meta key that "pops up" at that timestamp), -- or --
// 2. it is not, and it has been shadowed by the key we are clearing,
// in which case we need to offset its GCBytesAge contribution from
// restoredNanos to orig.Timestamp (rule 1).
if restored != nil {
if restored.Txn != nil {
panic("restored version should never be an intent")
}
ms.AgeTo(restoredNanos)
if restored.Deleted {
// The new meta key will be implicit and at restoredNanos. It needs to
// catch up on the GCBytesAge from that point on until orig.Timestamp
// (rule 2).
ms.KeyBytes += restoredMetaKeySize
ms.ValBytes += restoredMetaValSize
}
ms.AgeTo(orig.Timestamp.WallTime)
ms.KeyCount++
if !restored.Deleted {
// At orig.Timestamp, make the non-deletion version live again.
// Note that there's no need to explicitly age to the "present time"
// after.
ms.KeyBytes += restoredMetaKeySize
ms.ValBytes += restoredMetaValSize
ms.LiveBytes += restored.KeyBytes + restored.ValBytes
ms.LiveCount++
ms.LiveBytes += restoredMetaKeySize + restoredMetaValSize
}
} else {
ms.AgeTo(orig.Timestamp.WallTime)
}
if !orig.Deleted {
ms.LiveBytes -= (orig.KeyBytes + orig.ValBytes) + (origMetaKeySize + origMetaValSize)
ms.LiveCount--
}
ms.KeyBytes -= (orig.KeyBytes + origMetaKeySize)
ms.ValBytes -= (orig.ValBytes + origMetaValSize)
ms.KeyCount--
ms.ValCount--
if orig.Txn != nil {
ms.IntentBytes -= (orig.KeyBytes + orig.ValBytes)
ms.IntentCount--
ms.SeparatedIntentCount--
}
return ms
}
// updateStatsOnGC updates stat counters after garbage collection
// by subtracting key and value byte counts, updating key and
// value counts, and updating the GC'able bytes age. If meta is
// not nil, then the value being GC'd is the mvcc metadata and we
// decrement the key count.
//
// nonLiveMS is the timestamp at which the value became non-live.
// For a deletion tombstone this will be its own timestamp (rule two
// in updateStatsOnPut) and for a regular version it will be the closest
// newer version's (rule one).
func updateStatsOnGC(
key roachpb.Key, keySize, valSize int64, meta *enginepb.MVCCMetadata, nonLiveMS int64,
) enginepb.MVCCStats {
var ms enginepb.MVCCStats
if isSysLocal(key) {
ms.SysBytes -= (keySize + valSize)
if meta != nil {
ms.SysCount--
}
return ms
}
ms.AgeTo(nonLiveMS)
ms.KeyBytes -= keySize
ms.ValBytes -= valSize
if meta != nil {
ms.KeyCount--
} else {
ms.ValCount--
}
return ms
}
// MVCCGetProto fetches the value at the specified key and unmarshals it into
// msg if msg is non-nil. Returns true on success or false if the key was not
// found.
//
// See the documentation for MVCCGet for the semantics of the MVCCGetOptions.
func MVCCGetProto(
ctx context.Context,
reader Reader,
key roachpb.Key,
timestamp hlc.Timestamp,
msg protoutil.Message,
opts MVCCGetOptions,
) (bool, error) {
// TODO(tschottdorf): Consider returning skipped intents to the caller.
value, _, mvccGetErr := MVCCGet(ctx, reader, key, timestamp, opts)
found := value != nil
// If we found a result, parse it regardless of the error returned by MVCCGet.
if found && msg != nil {
// If the unmarshal failed, return its result. Otherwise, pass
// through the underlying error (which may be a WriteIntentError
// to be handled specially alongside the returned value).
if err := value.GetProto(msg); err != nil {
return found, err
}
}
return found, mvccGetErr
}
// MVCCPutProto sets the given key to the protobuf-serialized byte
// string of msg and the provided timestamp.
func MVCCPutProto(
ctx context.Context,
rw ReadWriter,
ms *enginepb.MVCCStats,
key roachpb.Key,
timestamp hlc.Timestamp,
txn *roachpb.Transaction,
msg protoutil.Message,
) error {
value := roachpb.Value{}
if err := value.SetProto(msg); err != nil {
return err
}
value.InitChecksum(key)
return MVCCPut(ctx, rw, ms, key, timestamp, value, txn)
}
// MVCCBlindPutProto sets the given key to the protobuf-serialized byte string
// of msg and the provided timestamp. See MVCCBlindPut for a discussion on this
// fast-path and when it is appropriate to use.
func MVCCBlindPutProto(
ctx context.Context,
writer Writer,
ms *enginepb.MVCCStats,
key roachpb.Key,
timestamp hlc.Timestamp,
msg protoutil.Message,
txn *roachpb.Transaction,
) error {
value := roachpb.Value{}
if err := value.SetProto(msg); err != nil {
return err
}
value.InitChecksum(key)
return MVCCBlindPut(ctx, writer, ms, key, timestamp, value, txn)
}
// MVCCGetOptions bundles options for the MVCCGet family of functions.
type MVCCGetOptions struct {
// See the documentation for MVCCGet for information on these parameters.
Inconsistent bool
Tombstones bool
FailOnMoreRecent bool
Txn *roachpb.Transaction
Uncertainty uncertainty.Interval
// MemoryAccount is used for tracking memory allocations.
MemoryAccount *mon.BoundAccount
}
func (opts *MVCCGetOptions) validate() error {
if opts.Inconsistent && opts.Txn != nil {
return errors.Errorf("cannot allow inconsistent reads within a transaction")
}
if opts.Inconsistent && opts.FailOnMoreRecent {
return errors.Errorf("cannot allow inconsistent reads with fail on more recent option")
}
return nil
}
func newMVCCIterator(reader Reader, inlineMeta bool, opts IterOptions) MVCCIterator {
iterKind := MVCCKeyAndIntentsIterKind
if inlineMeta {
iterKind = MVCCKeyIterKind
}
return reader.NewMVCCIterator(iterKind, opts)
}
// MVCCGet returns the most recent value for the specified key whose timestamp
// is less than or equal to the supplied timestamp. If no such value exists, nil
// is returned instead.
//
// In tombstones mode, if the most recent value is a deletion tombstone, the
// result will be a non-nil roachpb.Value whose RawBytes field is nil.
// Otherwise, a deletion tombstone results in a nil roachpb.Value.
//
// In inconsistent mode, if an intent is encountered, it will be placed in the
// dedicated return parameter. By contrast, in consistent mode, an intent will
// generate a WriteIntentError with the intent embedded within, and the intent
// result parameter will be nil.
//
// Note that transactional gets must be consistent. Put another way, only
// non-transactional gets may be inconsistent.
//
// If the timestamp is specified as hlc.Timestamp{}, the value is expected to be
// "inlined". See MVCCPut().
//
// When reading in "fail on more recent" mode, a WriteTooOldError will be
// returned if the read observes a version with a timestamp above the read
// timestamp. Similarly, a WriteIntentError will be returned if the read
// observes another transaction's intent, even if it has a timestamp above
// the read timestamp.
func MVCCGet(
ctx context.Context, reader Reader, key roachpb.Key, timestamp hlc.Timestamp, opts MVCCGetOptions,
) (*roachpb.Value, *roachpb.Intent, error) {
iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{Prefix: true})
defer iter.Close()
value, intent, err := mvccGet(ctx, iter, key, timestamp, opts)
return value.ToPointer(), intent, err
}
func mvccGet(
ctx context.Context,
iter MVCCIterator,
key roachpb.Key,
timestamp hlc.Timestamp,
opts MVCCGetOptions,
) (value optionalValue, intent *roachpb.Intent, err error) {
if len(key) == 0 {
return optionalValue{}, nil, emptyKeyError()
}
if timestamp.WallTime < 0 {
return optionalValue{}, nil, errors.Errorf("cannot write to %q at timestamp %s", key, timestamp)
}
if err := opts.validate(); err != nil {
return optionalValue{}, nil, err
}
mvccScanner := pebbleMVCCScannerPool.Get().(*pebbleMVCCScanner)
defer mvccScanner.release()
// MVCCGet is implemented as an MVCCScan where we retrieve a single key. We
// specify an empty key for the end key which will ensure we don't retrieve a
// key different than the start key. This is a bit of a hack.
*mvccScanner = pebbleMVCCScanner{
parent: iter,
memAccount: opts.MemoryAccount,
start: key,
ts: timestamp,
maxKeys: 1,
inconsistent: opts.Inconsistent,
tombstones: opts.Tombstones,
failOnMoreRecent: opts.FailOnMoreRecent,
keyBuf: mvccScanner.keyBuf,
}
mvccScanner.init(opts.Txn, opts.Uncertainty, 0)
mvccScanner.get(ctx)
// If we have a trace, emit the scan stats that we produced.
traceSpan := tracing.SpanFromContext(ctx)
recordIteratorStats(traceSpan, mvccScanner.stats())
if mvccScanner.err != nil {
return optionalValue{}, nil, mvccScanner.err
}
intents, err := buildScanIntents(mvccScanner.intentsRepr())
if err != nil {
return optionalValue{}, nil, err
}
if !opts.Inconsistent && len(intents) > 0 {
return optionalValue{}, nil, &roachpb.WriteIntentError{Intents: intents}
}
if len(intents) > 1 {
return optionalValue{}, nil, errors.Errorf("expected 0 or 1 intents, got %d", len(intents))
} else if len(intents) == 1 {
intent = &intents[0]
}
if len(mvccScanner.results.repr) == 0 {
return optionalValue{}, intent, nil
}
mvccKey, rawValue, _, err := MVCCScanDecodeKeyValue(mvccScanner.results.repr)
if err != nil {
return optionalValue{}, nil, err
}
value = makeOptionalValue(roachpb.Value{
RawBytes: rawValue,
Timestamp: mvccKey.Timestamp,
})
return value, intent, nil
}
// MVCCGetAsTxn constructs a temporary transaction from the given transaction
// metadata and calls MVCCGet as that transaction. This method is required
// only for reading intents of a transaction when only its metadata is known
// and should rarely be used.
//
// The read is carried out without the chance of uncertainty restarts.
func MVCCGetAsTxn(
ctx context.Context,
reader Reader,
key roachpb.Key,
timestamp hlc.Timestamp,
txnMeta enginepb.TxnMeta,
) (*roachpb.Value, *roachpb.Intent, error) {
return MVCCGet(ctx, reader, key, timestamp, MVCCGetOptions{
Txn: &roachpb.Transaction{
TxnMeta: txnMeta,
Status: roachpb.PENDING,
ReadTimestamp: txnMeta.WriteTimestamp,
GlobalUncertaintyLimit: txnMeta.WriteTimestamp,
}})
}
// mvccGetMetadata returns or reconstructs the meta key for the given key.
// A prefix scan using the iterator is performed, resulting in one of the
// following successful outcomes:
// 1) iterator finds nothing; returns (false, 0, 0, nil).
// 2) iterator finds an explicit meta key; unmarshals and returns its size.
// ok is set to true.
// 3) iterator finds a value, i.e. the meta key is implicit.
// In this case, it accounts for the size of the key with the portion
// of the user key found which is not the MVCC timestamp suffix (since
// that is the usual contribution of the meta key). The value size returned
// will be zero, as there is no stored MVCCMetadata.
// ok is set to true.
// The passed in MVCCMetadata must not be nil.
//
// If the supplied iterator is nil, no seek operation is performed. This is
// used by the Blind{Put,ConditionalPut} operations to avoid seeking when the
// metadata is known not to exist. If iterAlreadyPositioned is true, the
// iterator has already been seeked to metaKey, so a wasteful seek can be
// avoided.
func mvccGetMetadata(
iter MVCCIterator, metaKey MVCCKey, iterAlreadyPositioned bool, meta *enginepb.MVCCMetadata,
) (ok bool, keyBytes, valBytes int64, err error) {
if iter == nil {
return false, 0, 0, nil
}
if !iterAlreadyPositioned {
iter.SeekGE(metaKey)
}
if ok, err := iter.Valid(); !ok {
return false, 0, 0, err
}
unsafeKey := iter.UnsafeKey()
if !unsafeKey.Key.Equal(metaKey.Key) {
return false, 0, 0, nil
}
if !unsafeKey.IsValue() {
if err := iter.ValueProto(meta); err != nil {
return false, 0, 0, err
}
return true, int64(unsafeKey.EncodedSize()),
int64(len(iter.UnsafeValue())), nil
}
meta.Reset()
// For values, the size of keys is always accounted for as
// MVCCVersionTimestampSize. The size of the metadata key is
// accounted for separately.
meta.KeyBytes = MVCCVersionTimestampSize
meta.ValBytes = int64(len(iter.UnsafeValue()))
meta.Deleted = meta.ValBytes == 0
meta.Timestamp = unsafeKey.Timestamp.ToLegacyTimestamp()
return true, int64(unsafeKey.EncodedSize()) - meta.KeyBytes, 0, nil
}
// putBuffer holds pointer data needed by mvccPutInternal. Bundling
// this data into a single structure reduces memory
// allocations. Managing this temporary buffer using a sync.Pool
// completely eliminates allocation from the put common path.
type putBuffer struct {
meta enginepb.MVCCMetadata
newMeta enginepb.MVCCMetadata
ts hlc.LegacyTimestamp
tmpbuf []byte
}
var putBufferPool = sync.Pool{
New: func() interface{} {
return &putBuffer{}
},
}
func newPutBuffer() *putBuffer {
return putBufferPool.Get().(*putBuffer)
}
func (b *putBuffer) release() {
*b = putBuffer{tmpbuf: b.tmpbuf[:0]}
putBufferPool.Put(b)
}
func (b *putBuffer) marshalMeta(meta *enginepb.MVCCMetadata) (_ []byte, err error) {
size := meta.Size()
data := b.tmpbuf
if cap(data) < size {
data = make([]byte, size)
} else {
data = data[:size]
}
n, err := protoutil.MarshalTo(meta, data)
if err != nil {
return nil, err
}
b.tmpbuf = data
return data[:n], nil
}
func (b *putBuffer) putInlineMeta(
writer Writer, key MVCCKey, meta *enginepb.MVCCMetadata,
) (keyBytes, valBytes int64, err error) {
bytes, err := b.marshalMeta(meta)
if err != nil {
return 0, 0, err
}
if err := writer.PutUnversioned(key.Key, bytes); err != nil {
return 0, 0, err
}
return int64(key.EncodedSize()), int64(len(bytes)), nil
}
var trueValue = true
func (b *putBuffer) putIntentMeta(
ctx context.Context, writer Writer, key MVCCKey, meta *enginepb.MVCCMetadata, alreadyExists bool,
) (keyBytes, valBytes int64, err error) {
if meta.Txn != nil && meta.Timestamp.ToTimestamp() != meta.Txn.WriteTimestamp {
// The timestamps are supposed to be in sync. If they weren't, it wouldn't
// be clear for readers which one to use for what.
return 0, 0, errors.AssertionFailedf(
"meta.Timestamp != meta.Txn.WriteTimestamp: %s != %s", meta.Timestamp, meta.Txn.WriteTimestamp)
}
if alreadyExists {
// Absence represents false.
meta.TxnDidNotUpdateMeta = nil
} else {
meta.TxnDidNotUpdateMeta = &trueValue
}
bytes, err := b.marshalMeta(meta)
if err != nil {
return 0, 0, err
}
if err = writer.PutIntent(ctx, key.Key, bytes, meta.Txn.ID); err != nil {
return 0, 0, err
}
return int64(key.EncodedSize()), int64(len(bytes)), nil
}
// MVCCPut sets the value for a specified key. It will save the value
// with different versions according to its timestamp and update the
// key metadata. The timestamp must be passed as a parameter; using
// the Timestamp field on the value results in an error.
//
// Note that, when writing transactionally, the txn's timestamps
// dictate the timestamp of the operation, and the timestamp parameter is
// confusing and redundant. See the comment on mvccPutInternal for details.
//
// If the timestamp is specified as hlc.Timestamp{}, the value is
// inlined instead of being written as a timestamp-versioned value. A
// zero timestamp write to a key precludes a subsequent write using a
// non-zero timestamp and vice versa. Inlined values require only a
// single row and never accumulate more than a single value. Successive
// zero timestamp writes to a key replace the value and deletes clear
// the value. In addition, zero timestamp values may be merged.
func MVCCPut(
ctx context.Context,
rw ReadWriter,
ms *enginepb.MVCCStats,
key roachpb.Key,
timestamp hlc.Timestamp,
value roachpb.Value,
txn *roachpb.Transaction,
) error {
// If we're not tracking stats for the key and we're writing a non-versioned
// key we can utilize a blind put to avoid reading any existing value.
var iter MVCCIterator
blind := ms == nil && timestamp.IsEmpty()
if !blind {
iter = rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{Prefix: true})
defer iter.Close()
}