Skip to content

Commit 39e006e

Browse files
committed
storage: improve documentation for MVCCValue encoding
We also clean up some related code a bit. Epic: none Release note: None
1 parent 1bedd25 commit 39e006e

File tree

4 files changed

+41
-36
lines changed

4 files changed

+41
-36
lines changed

pkg/storage/mvcc.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2742,7 +2742,7 @@ func mvccPutInternal(
27422742
}
27432743
newMeta.Timestamp = versionKey.Timestamp.ToLegacyTimestamp()
27442744
newMeta.KeyBytes = MVCCVersionTimestampSize
2745-
newMeta.ValBytes = int64(encodedMVCCValueSize(versionValue))
2745+
newMeta.ValBytes = int64(versionValue.encodedSize())
27462746
newMeta.Deleted = versionValue.IsTombstone()
27472747
newMeta.IntentHistory = newIntentHistory
27482748

@@ -5497,7 +5497,7 @@ func mvccResolveWriteIntent(
54975497
newMeta.Txn.WriteTimestamp = newTimestamp
54985498
newMeta.Timestamp = newTimestamp.ToLegacyTimestamp()
54995499
newMeta.KeyBytes = MVCCVersionTimestampSize
5500-
newMeta.ValBytes = int64(encodedMVCCValueSize(newValue))
5500+
newMeta.ValBytes = int64(newValue.encodedSize())
55015501
newMeta.Deleted = newValue.IsTombstone()
55025502

55035503
if err = writer.PutMVCC(newKey, newValue); err != nil {

pkg/storage/mvcc_value.go

Lines changed: 25 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -36,27 +36,29 @@ const (
3636
//
3737
// The mvcc value has a "simple" and an "extended" encoding scheme, depending on
3838
// whether the value's header is empty or not. If the value's header is empty,
39-
// it is omitted in the encoding and the mvcc value's encoding is identical to
40-
// that of roachpb.Value. This provided backwards compatibility and ensures that
41-
// the MVCCValue optimizes away in the common case. If the value's header is not
42-
// empty, it is prepended to the roachpb.Value encoding. The encoding scheme's
43-
// variants are:
39+
// it is omitted in the encoding and the mvcc value's encoding is just that of
40+
// roachpb.Value. This provides backwards compatibility and ensures that the
41+
// MVCCValueHeader optimizes away in the common case. If the value's header is
42+
// not empty, it is prepended to the roachpb.Value encoding. The encoding
43+
// scheme's variants are:
4444
//
45-
// Simple (identical to the roachpb.Value encoding):
45+
// Simple (just the roachpb.Value encoding):
4646
//
4747
// <4-byte-checksum><1-byte-tag><encoded-data>
4848
//
49-
// Extended (header prepended to roachpb.Value encoding):
49+
// Extended (header prepended to the roachpb.Value encoding):
5050
//
51-
// <4-byte-header-len><1-byte-sentinel><mvcc-header><4-byte-checksum><1-byte-tag><encoded-data>
51+
// <4-byte-header-len><1-byte-sentinel-tag><mvcc-header><4-byte-checksum><1-byte-tag><encoded-data>
52+
// ^ ^
53+
// \-------- roachpb.Value encoding ---------/
5254
//
5355
// The two encoding scheme variants are distinguished using the 5th byte, which
5456
// is either the roachpb.Value tag (which has many values) or a sentinel tag not
5557
// used by the roachpb.Value encoding which indicates the extended encoding
5658
// scheme.
5759
//
58-
// For a deletion tombstone, the encoding of roachpb.Value is special cased to
59-
// be empty, i.e., no checksum, tag, or encoded-data. In that case the extended
60+
// For a deletion tombstone, the encoding of roachpb.Value is special-cased to
61+
// be empty, i.e. no checksum, tag, or encoded-data. In that case the extended
6062
// encoding above is simply:
6163
//
6264
// <4-byte-header-len><1-byte-sentinel><mvcc-header>
@@ -150,9 +152,9 @@ func EncodeMVCCValueForExport(mvccValue MVCCValue, b []byte) ([]byte, bool, erro
150152
return EncodeMVCCValueToBuf(mvccValue, b)
151153
}
152154

153-
// When running a metamorphic build, disable the simple MVCC value encoding to
154-
// prevent code from assuming that the MVCCValue encoding is identical to the
155-
// roachpb.Value encoding.
155+
// disableSimpleValueEncoding forces encoding of the MVCCValueHeader even when
156+
// it is empty (see MVCCValue). It is set metamorphically to extend testing
157+
// coverage.
156158
var disableSimpleValueEncoding = metamorphic.ConstantWithTestBool(
157159
"mvcc-value-disable-simple-encoding", false)
158160

@@ -171,9 +173,14 @@ func DisableMetamorphicSimpleValueEncoding(t interface {
171173
}
172174
}
173175

174-
// encodedMVCCValueSize returns the size of the MVCCValue when encoded.
175-
func encodedMVCCValueSize(v MVCCValue) int {
176-
if v.MVCCValueHeader.IsEmpty() && !disableSimpleValueEncoding {
176+
//gcassert:inline
177+
func (v *MVCCValue) useSimpleEncoding() bool {
178+
return v.MVCCValueHeader.IsEmpty() && !disableSimpleValueEncoding
179+
}
180+
181+
// encodedSize returns the size of the MVCCValue when encoded.
182+
func (v *MVCCValue) encodedSize() int {
183+
if v.useSimpleEncoding() {
177184
return len(v.Value.RawBytes)
178185
}
179186
return extendedPreludeSize + v.MVCCValueHeader.Size() + len(v.Value.RawBytes)
@@ -204,7 +211,7 @@ func EncodeMVCCValue(v MVCCValue) ([]byte, error) {
204211
// negates the inlining gain. Reconsider this with Go 1.20. See:
205212
// https://github.com/cockroachdb/cockroach/issues/88818
206213
func EncodeMVCCValueToBuf(v MVCCValue, buf []byte) ([]byte, bool, error) {
207-
if v.MVCCValueHeader.IsEmpty() && !disableSimpleValueEncoding {
214+
if v.useSimpleEncoding() {
208215
// Simple encoding. Use the roachpb.Value encoding directly with no
209216
// modification. No need to re-allocate or copy.
210217
return v.Value.RawBytes, false, nil
@@ -243,21 +250,14 @@ func EncodeMVCCValueToBuf(v MVCCValue, buf []byte) ([]byte, bool, error) {
243250
return buf, true, nil
244251
}
245252

246-
func mvccValueSize(v MVCCValue) (size int, extendedEncoding bool) {
247-
if v.MVCCValueHeader.IsEmpty() && !disableSimpleValueEncoding {
248-
return len(v.Value.RawBytes), false
249-
}
250-
return extendedPreludeSize + v.MVCCValueHeader.Size() + len(v.Value.RawBytes), true
251-
}
252-
253253
// encodeExtendedMVCCValueToSizedBuf encodes an MVCCValue into its encoded form
254254
// in the provided buffer. The provided buf must be exactly sized, matching the
255255
// value returned by MVCCValue.encodedMVCCValueSize.
256256
//
257257
// See EncodeMVCCValueToBuf for detailed comments on the encoding scheme.
258258
func encodeExtendedMVCCValueToSizedBuf(v MVCCValue, buf []byte) error {
259259
if buildutil.CrdbTestBuild {
260-
if sz := encodedMVCCValueSize(v); sz != len(buf) {
260+
if sz := v.encodedSize(); sz != len(buf) {
261261
panic(errors.AssertionFailedf("provided buf (len=%d) is not sized correctly; expected %d", len(buf), sz))
262262
}
263263
}

pkg/storage/mvcc_value_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -219,7 +219,7 @@ func TestEncodeDecodeMVCCValue(t *testing.T) {
219219
enc, err := EncodeMVCCValue(tc.val)
220220
require.NoError(t, err)
221221
fmt.Fprintf(&buf, "encoded: %x", enc)
222-
assert.Equal(t, encodedMVCCValueSize(tc.val), len(enc))
222+
assert.Equal(t, tc.val.encodedSize(), len(enc))
223223

224224
dec, err := DecodeMVCCValue(enc)
225225
require.NoError(t, err)
@@ -239,7 +239,7 @@ func TestEncodeDecodeMVCCValue(t *testing.T) {
239239
t.Run("DeocdeValueFromMVCCValue/"+name, func(t *testing.T) {
240240
enc, err := EncodeMVCCValue(tc.val)
241241
require.NoError(t, err)
242-
assert.Equal(t, encodedMVCCValueSize(tc.val), len(enc))
242+
assert.Equal(t, tc.val.encodedSize(), len(enc))
243243

244244
dec, err := DecodeValueFromMVCCValue(enc)
245245
require.NoError(t, err)

pkg/storage/pebble_batch.go

Lines changed: 12 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ import (
1515
"github.com/cockroachdb/cockroach/pkg/storage/fs"
1616
"github.com/cockroachdb/cockroach/pkg/storage/mvccencoding"
1717
"github.com/cockroachdb/cockroach/pkg/storage/pebbleiter"
18+
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
1819
"github.com/cockroachdb/errors"
1920
"github.com/cockroachdb/pebble"
2021
"github.com/cockroachdb/pebble/rangekey"
@@ -307,21 +308,25 @@ func (wb *writeBatch) putMVCC(key MVCCKey, value MVCCValue) error {
307308
// - encode the MVCC key and MVCC value directly into the Batch
308309
// - call Finish on the deferred operation (which will index the key if
309310
// wb.batch is indexed)
310-
valueLen, isExtended := mvccValueSize(value)
311311
keyLen := mvccencoding.EncodedMVCCKeyLength(key.Key, key.Timestamp)
312-
o := wb.batch.SetDeferred(keyLen, valueLen)
313-
mvccencoding.EncodeMVCCKeyToBufSized(o.Key, key.Key, key.Timestamp, keyLen)
314-
if !isExtended {
312+
var o *pebble.DeferredBatchOp
313+
if value.useSimpleEncoding() {
315314
// Fast path; we don't need to use the extended encoding and can copy
316315
// RawBytes in verbatim.
316+
o = wb.batch.SetDeferred(keyLen, len(value.Value.RawBytes))
317317
copy(o.Value, value.Value.RawBytes)
318318
} else {
319-
// Slow path; we need the MVCC value header.
320-
err := encodeExtendedMVCCValueToSizedBuf(value, o.Value)
321-
if err != nil {
319+
// Slow path; we need the MVCC value header. Inline the relevant part of encodedSize().
320+
valueLen := extendedPreludeSize + value.MVCCValueHeader.Size() + len(value.Value.RawBytes)
321+
if buildutil.CrdbTestBuild && valueLen != value.encodedSize() {
322+
panic("incorrect valueLen calculation")
323+
}
324+
o = wb.batch.SetDeferred(keyLen, valueLen)
325+
if err := encodeExtendedMVCCValueToSizedBuf(value, o.Value); err != nil {
322326
return err
323327
}
324328
}
329+
mvccencoding.EncodeMVCCKeyToBufSized(o.Key, key.Key, key.Timestamp, keyLen)
325330
return o.Finish()
326331
}
327332

0 commit comments

Comments
 (0)