diff --git a/internal/crdbtest/crdbtest.go b/internal/crdbtest/crdbtest.go index 026a87d431..e6f388d70d 100644 --- a/internal/crdbtest/crdbtest.go +++ b/internal/crdbtest/crdbtest.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/crlib/crstrings" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/invariants" "golang.org/x/exp/rand" ) @@ -154,29 +155,60 @@ func EncodeTimestamp(key []byte, walltime uint64, logical uint32) []byte { return key } -// DecodeTimestamp decodes a MVCC timestamp from a serialized MVCC key. -func DecodeTimestamp( - mvccKey []byte, -) (prefix []byte, untypedSuffix []byte, wallTime uint64, logicalTime uint32) { - tsLen := int(mvccKey[len(mvccKey)-1]) - keyPartEnd := len(mvccKey) - tsLen - if keyPartEnd < 0 { +// DecodeEngineKey decodes an Engine key (the key that gets stored in Pebble). +// Returns: +// +// - roachKey: key prefix without the separator byte; corresponds to +// MVCCKey.Key / EngineKey.Key (roachpb.Key type). +// +// - untypedVersion: when the suffix does not correspond to a timestamp, +// untypedVersion is set to the suffix without the length +// byte; corresponds to EngineKey.Version. +// +// - wallTime, logicalTime: timestamp for the key, or 0 if the key does not +// correspond to a timestamp. +func DecodeEngineKey( + engineKey []byte, +) (roachKey []byte, untypedVersion []byte, wallTime uint64, logicalTime uint32) { + tsLen := int(engineKey[len(engineKey)-1]) + tsStart := len(engineKey) - tsLen + if tsStart < 0 { + if invariants.Enabled { + panic("invalid length byte") + } return nil, nil, 0, 0 } + roachKeyEnd := tsStart - 1 + if roachKeyEnd < 0 { + roachKeyEnd = 0 + } else if invariants.Enabled && engineKey[roachKeyEnd] != 0 { + panic("invalid separator byte") + } - key := mvccKey[:keyPartEnd] - if tsLen > 0 { - ts := mvccKey[keyPartEnd : len(mvccKey)-1] - switch len(ts) { - case 8: - return key, nil, binary.BigEndian.Uint64(ts[:8]), 0 - case 12, 13: - return key, nil, binary.BigEndian.Uint64(ts[:8]), binary.BigEndian.Uint32(ts[8:12]) - default: - return key, ts, 0, 0 - } + roachKey = engineKey[:roachKeyEnd] + untypedVersion, wallTime, logicalTime = DecodeSuffix(engineKey[tsStart:]) + return roachKey, untypedVersion, wallTime, logicalTime +} + +// DecodeSuffix decodes the suffix of a key. If the suffix is a timestamp, +// wallTime and logicalTime are returned; otherwise untypedVersion contains the +// version (which is the suffix without the terminator length byte). +// +//gcassert:inline +func DecodeSuffix(suffix []byte) (untypedVersion []byte, wallTime uint64, logicalTime uint32) { + if invariants.Enabled && len(suffix) > 0 && len(suffix) != int(suffix[len(suffix)-1]) { + panic("invalid length byte") + } + switch len(suffix) { + case 0: + return nil, 0, 0 + case 9: + return nil, binary.BigEndian.Uint64(suffix[:8]), 0 + case 13, 14: + return nil, binary.BigEndian.Uint64(suffix[:8]), binary.BigEndian.Uint32(suffix[8:12]) + default: + return suffix[:len(suffix)-1], 0, 0 } - return key, nil, 0, 0 } // Split implements base.Split for CockroachDB keys. diff --git a/sstable/colblk/cockroach_test.go b/sstable/colblk/cockroach_test.go index 17927af1e0..8a88daadd5 100644 --- a/sstable/colblk/cockroach_test.go +++ b/sstable/colblk/cockroach_test.go @@ -27,26 +27,26 @@ import ( ) const ( - cockroachColPrefix int = iota + cockroachColRoachKey int = iota cockroachColMVCCWallTime cockroachColMVCCLogical - cockroachColUntypedSuffix + cockroachColUntypedVersion cockroachColCount ) var cockroachKeySchema = KeySchema{ ColumnTypes: []DataType{ - cockroachColPrefix: DataTypePrefixBytes, - cockroachColMVCCWallTime: DataTypeUint, - cockroachColMVCCLogical: DataTypeUint, - cockroachColUntypedSuffix: DataTypeBytes, + cockroachColRoachKey: DataTypePrefixBytes, + cockroachColMVCCWallTime: DataTypeUint, + cockroachColMVCCLogical: DataTypeUint, + cockroachColUntypedVersion: DataTypeBytes, }, NewKeyWriter: func() KeyWriter { kw := &cockroachKeyWriter{} - kw.prefixes.Init(16) + kw.roachKeys.Init(16) kw.wallTimes.Init() kw.logicalTimes.InitWithDefault() - kw.untypedSuffixes.Init() + kw.untypedVersions.Init() return kw }, NewKeySeeker: func() KeySeeker { @@ -55,29 +55,31 @@ var cockroachKeySchema = KeySchema{ } type cockroachKeyWriter struct { - prefixes PrefixBytesBuilder + roachKeys PrefixBytesBuilder wallTimes UintBuilder logicalTimes UintBuilder - untypedSuffixes RawBytesBuilder + untypedVersions RawBytesBuilder prevSuffix []byte } func (kw *cockroachKeyWriter) ComparePrev(key []byte) KeyComparison { var cmpv KeyComparison cmpv.PrefixLen = int32(crdbtest.Split(key)) // TODO(jackson): Inline - if kw.prefixes.Rows() == 0 { + if kw.roachKeys.Rows() == 0 { cmpv.UserKeyComparison = 1 return cmpv } - lp := kw.prefixes.UnsafeGet(kw.prefixes.Rows() - 1) - cmpv.CommonPrefixLen = int32(crbytes.CommonPrefix(lp, key[:cmpv.PrefixLen])) - if cmpv.CommonPrefixLen == cmpv.PrefixLen { + lp := kw.roachKeys.UnsafeGet(kw.roachKeys.Rows() - 1) + cmpv.CommonPrefixLen = int32(crbytes.CommonPrefix(lp, key[:cmpv.PrefixLen-1])) + if cmpv.CommonPrefixLen == cmpv.PrefixLen-1 { + // Adjust CommonPrefixLen to include the sentinel byte, + cmpv.CommonPrefixLen = cmpv.PrefixLen cmpv.UserKeyComparison = int32(crdbtest.CompareSuffixes(key[cmpv.PrefixLen:], kw.prevSuffix)) return cmpv } // The keys have different MVCC prefixes. We haven't determined which is // greater, but we know the index at which they diverge. The base.Comparer - // contract dictates that prefixes must be lexicograrphically ordered. + // contract dictates that prefixes must be lexicographically ordered. if len(lp) == int(cmpv.CommonPrefixLen) { // cmpv.PrefixLen > cmpv.PrefixLenShared; key is greater. cmpv.UserKeyComparison = +1 @@ -94,9 +96,11 @@ func (kw *cockroachKeyWriter) WriteKey( ) { // TODO(jackson): Avoid copying the previous suffix. // TODO(jackson): Use keyPrefixLen to speed up decoding. - prefix, untypedSuffix, wallTime, logicalTime := crdbtest.DecodeTimestamp(key) + roachKey, untypedVersion, wallTime, logicalTime := crdbtest.DecodeEngineKey(key) kw.prevSuffix = append(kw.prevSuffix[:0], key[keyPrefixLen:]...) - kw.prefixes.Put(prefix, int(keyPrefixLenSharedWithPrev)) + // When the roach key is the same, keyPrefixLenSharedWithPrev includes the + // separator byte. + kw.roachKeys.Put(roachKey, min(int(keyPrefixLenSharedWithPrev), len(roachKey))) kw.wallTimes.Set(row, wallTime) // The w.logicalTimes builder was initialized with InitWithDefault, so if we // don't set a value, the column value is implicitly zero. We only need to @@ -104,27 +108,31 @@ func (kw *cockroachKeyWriter) WriteKey( if logicalTime > 0 { kw.logicalTimes.Set(row, uint64(logicalTime)) } - kw.untypedSuffixes.Put(untypedSuffix) + kw.untypedVersions.Put(untypedVersion) } func (kw *cockroachKeyWriter) MaterializeKey(dst []byte, i int) []byte { - dst = append(dst, kw.prefixes.UnsafeGet(i)...) - if untypedSuffixed := kw.untypedSuffixes.UnsafeGet(i); len(untypedSuffixed) > 0 { - return append(dst, untypedSuffixed...) + dst = append(dst, kw.roachKeys.UnsafeGet(i)...) + // Append separator byte. + dst = append(dst, 0) + if untypedVersion := kw.untypedVersions.UnsafeGet(i); len(untypedVersion) > 0 { + dst = append(dst, untypedVersion...) + dst = append(dst, byte(len(untypedVersion))) + return dst } return crdbtest.AppendTimestamp(dst, kw.wallTimes.Get(i), uint32(kw.logicalTimes.Get(i))) } func (kw *cockroachKeyWriter) Reset() { - kw.prefixes.Reset() + kw.roachKeys.Reset() kw.wallTimes.Reset() kw.logicalTimes.Reset() - kw.untypedSuffixes.Reset() + kw.untypedVersions.Reset() } func (kw *cockroachKeyWriter) WriteDebug(dst io.Writer, rows int) { fmt.Fprint(dst, "prefixes: ") - kw.prefixes.WriteDebug(dst, rows) + kw.roachKeys.WriteDebug(dst, rows) fmt.Fprintln(dst) fmt.Fprint(dst, "wall times: ") kw.wallTimes.WriteDebug(dst, rows) @@ -133,7 +141,7 @@ func (kw *cockroachKeyWriter) WriteDebug(dst io.Writer, rows int) { kw.logicalTimes.WriteDebug(dst, rows) fmt.Fprintln(dst) fmt.Fprint(dst, "untyped suffixes: ") - kw.untypedSuffixes.WriteDebug(dst, rows) + kw.untypedVersions.WriteDebug(dst, rows) fmt.Fprintln(dst) } @@ -146,10 +154,10 @@ func (kw *cockroachKeyWriter) DataType(col int) DataType { } func (kw *cockroachKeyWriter) Size(rows int, offset uint32) uint32 { - offset = kw.prefixes.Size(rows, offset) + offset = kw.roachKeys.Size(rows, offset) offset = kw.wallTimes.Size(rows, offset) offset = kw.logicalTimes.Size(rows, offset) - offset = kw.untypedSuffixes.Size(rows, offset) + offset = kw.untypedVersions.Size(rows, offset) return offset } @@ -157,14 +165,14 @@ func (kw *cockroachKeyWriter) Finish( col int, rows int, offset uint32, buf []byte, ) (endOffset uint32) { switch col { - case cockroachColPrefix: - return kw.prefixes.Finish(0, rows, offset, buf) + case cockroachColRoachKey: + return kw.roachKeys.Finish(0, rows, offset, buf) case cockroachColMVCCWallTime: return kw.wallTimes.Finish(0, rows, offset, buf) case cockroachColMVCCLogical: return kw.logicalTimes.Finish(0, rows, offset, buf) - case cockroachColUntypedSuffix: - return kw.untypedSuffixes.Finish(0, rows, offset, buf) + case cockroachColUntypedVersion: + return kw.untypedVersions.Finish(0, rows, offset, buf) default: panic(fmt.Sprintf("unknown default key column: %d", col)) } @@ -176,10 +184,10 @@ var cockroachKeySeekerPool = sync.Pool{ type cockroachKeySeeker struct { reader *DataBlockReader - prefixes PrefixBytes + roachKeys PrefixBytes mvccWallTimes UnsafeUints mvccLogical UnsafeUints - untypedSuffixes RawBytes + untypedVersions RawBytes sharedPrefix []byte } @@ -188,11 +196,11 @@ var _ KeySeeker = (*cockroachKeySeeker)(nil) // Init is part of the KeySeeker interface. func (ks *cockroachKeySeeker) Init(r *DataBlockReader) error { ks.reader = r - ks.prefixes = r.r.PrefixBytes(cockroachColPrefix) + ks.roachKeys = r.r.PrefixBytes(cockroachColRoachKey) ks.mvccWallTimes = r.r.Uints(cockroachColMVCCWallTime) ks.mvccLogical = r.r.Uints(cockroachColMVCCLogical) - ks.untypedSuffixes = r.r.RawBytes(cockroachColUntypedSuffix) - ks.sharedPrefix = ks.prefixes.SharedPrefix() + ks.untypedVersions = r.r.RawBytes(cockroachColUntypedVersion) + ks.sharedPrefix = ks.roachKeys.SharedPrefix() return nil } @@ -201,18 +209,18 @@ func (ks *cockroachKeySeeker) Init(r *DataBlockReader) error { // // Compare(firstUserKey, k) func (ks *cockroachKeySeeker) IsLowerBound(k []byte, syntheticSuffix []byte) bool { - prefix, untypedSuffix, wallTime, logicalTime := crdbtest.DecodeTimestamp(k) - if v := crdbtest.Compare(ks.prefixes.UnsafeFirstSlice(), prefix); v != 0 { + roachKey, untypedVersion, wallTime, logicalTime := crdbtest.DecodeEngineKey(k) + if v := crdbtest.Compare(ks.roachKeys.UnsafeFirstSlice(), roachKey); v != 0 { return v > 0 } if len(syntheticSuffix) > 0 { - return crdbtest.Compare(syntheticSuffix, k[len(prefix):]) >= 0 + return crdbtest.Compare(syntheticSuffix, k[len(roachKey)+1:]) >= 0 } - if len(untypedSuffix) > 0 { + if len(untypedVersion) > 0 { if invariants.Enabled && ks.mvccWallTimes.At(0) != 0 { panic("comparing timestamp with untyped suffix") } - return crdbtest.Compare(ks.untypedSuffixes.At(0), untypedSuffix) >= 0 + return crdbtest.Compare(ks.untypedVersions.At(0), untypedVersion) >= 0 } if v := cmp.Compare(ks.mvccWallTimes.At(0), wallTime); v != 0 { return v > 0 @@ -226,7 +234,7 @@ func (ks *cockroachKeySeeker) SeekGE( ) (row int, equalPrefix bool) { // TODO(jackson): Inline crdbtest.Split. si := crdbtest.Split(key) - row, eq := ks.prefixes.Search(key[:si-1]) + row, eq := ks.roachKeys.Search(key[:si-1]) if eq { return ks.seekGEOnSuffix(row, key[si:]), true } @@ -252,22 +260,22 @@ func (ks *cockroachKeySeeker) seekGEOnSuffix(index int, seekSuffix []byte) (row // the prefix. return index case withLogical, withSynthetic: - seekWallTime = binary.LittleEndian.Uint64(seekSuffix) - seekLogicalTime = binary.LittleEndian.Uint32(seekSuffix[8:]) + seekWallTime = binary.BigEndian.Uint64(seekSuffix) + seekLogicalTime = binary.BigEndian.Uint32(seekSuffix[8:]) case withWall: - seekWallTime = binary.LittleEndian.Uint64(seekSuffix) + seekWallTime = binary.BigEndian.Uint64(seekSuffix) default: // The suffix is untyped. Compare the untyped suffixes. // Binary search between [index, prefixChanged.SeekSetBitGE(index+1)]. // - // Define f(l-1) == false and f(u) == true. + // Define f(i) = true iff key at i is >= seek key. // Invariant: f(l-1) == false, f(u) == true. l := index u := ks.reader.prefixChanged.SeekSetBitGE(index + 1) for l < u { h := int(uint(l+u) >> 1) // avoid overflow when computing h // l ≤ h < u - if bytes.Compare(ks.untypedSuffixes.At(h), seekSuffix) >= 0 { + if bytes.Compare(ks.untypedVersions.At(h), seekSuffix) >= 0 { u = h // preserves f(u) == true } else { l = h + 1 // preserves f(l-1) == false @@ -279,25 +287,21 @@ func (ks *cockroachKeySeeker) seekGEOnSuffix(index int, seekSuffix []byte) (row // TODO(jackson): What if the row has an untyped suffix? - // Binary search between [index, prefixChanged.SeekSetBitGE(index)]. + // Binary search between [index, prefixChanged.SeekSetBitGE(index+1)]. // - // Define f(l-1) == false and f(u) == true. + // Define f(i) = true iff key at i is >= seek key. // Invariant: f(l-1) == false, f(u) == true. l := index - u := ks.reader.prefixChanged.SeekSetBitGE(index) + u := ks.reader.prefixChanged.SeekSetBitGE(index + 1) for l < u { h := int(uint(l+u) >> 1) // avoid overflow when computing h // l ≤ h < u - switch cmp.Compare(ks.mvccWallTimes.At(h), seekWallTime) { - case -1: - l = h + 1 // preserves f(l-1) == false - case +1: - u = h // preserves f(u) == true - } - if cmp.Compare(uint32(ks.mvccLogical.At(h)), seekLogicalTime) >= 0 { - u = h // preserves f(u) == true + hWallTime := ks.mvccWallTimes.At(h) + if hWallTime < seekWallTime || + (hWallTime == seekWallTime && uint32(ks.mvccLogical.At(h)) <= seekLogicalTime) { + u = h // preserves f(u) = true } else { - l = h + 1 // preserves f(l-1) == false + l = h + 1 // preserves f(l-1) = false } } return l @@ -306,39 +310,51 @@ func (ks *cockroachKeySeeker) seekGEOnSuffix(index int, seekSuffix []byte) (row // MaterializeUserKey is part of the KeySeeker interface. func (ks *cockroachKeySeeker) MaterializeUserKey(ki *PrefixBytesIter, prevRow, row int) []byte { if prevRow+1 == row && prevRow >= 0 { - ks.prefixes.SetNext(ki) + ks.roachKeys.SetNext(ki) } else { - ks.prefixes.SetAt(ki, row) + ks.roachKeys.SetAt(ki, row) } - ptr := unsafe.Pointer(uintptr(unsafe.Pointer(unsafe.SliceData(ki.buf))) + uintptr(len(ki.buf))) + roachKeyLen := len(ki.buf) + ptr := unsafe.Pointer(uintptr(unsafe.Pointer(unsafe.SliceData(ki.buf))) + uintptr(roachKeyLen)) mvccWall := ks.mvccWallTimes.At(row) mvccLogical := uint32(ks.mvccLogical.At(row)) if mvccWall == 0 && mvccLogical == 0 { // This is not an MVCC key. Use the untyped suffix. - untypedSuffixed := ks.untypedSuffixes.At(row) + untypedVersion := ks.untypedVersions.At(row) + if len(untypedVersion) == 0 { + res := ki.buf[:roachKeyLen+1] + res[roachKeyLen] = 0 + return res + } // Slice first, to check that the capacity is sufficient. - res := ki.buf[:len(ki.buf)+len(untypedSuffixed)] - memmove(ptr, unsafe.Pointer(unsafe.SliceData(untypedSuffixed)), uintptr(len(untypedSuffixed))) + res := ki.buf[:roachKeyLen+1+len(untypedVersion)] + *(*byte)(ptr) = 0 + memmove( + unsafe.Pointer(uintptr(ptr)+1), + unsafe.Pointer(unsafe.SliceData(untypedVersion)), + uintptr(len(untypedVersion)), + ) return res } // Inline binary.BigEndian.PutUint64. Note that this code is converted into // word-size instructions by the compiler. - *(*byte)(ptr) = byte(mvccWall >> 56) - *(*byte)(unsafe.Pointer(uintptr(ptr) + 1)) = byte(mvccWall >> 48) - *(*byte)(unsafe.Pointer(uintptr(ptr) + 2)) = byte(mvccWall >> 40) - *(*byte)(unsafe.Pointer(uintptr(ptr) + 3)) = byte(mvccWall >> 32) - *(*byte)(unsafe.Pointer(uintptr(ptr) + 4)) = byte(mvccWall >> 24) - *(*byte)(unsafe.Pointer(uintptr(ptr) + 5)) = byte(mvccWall >> 16) - *(*byte)(unsafe.Pointer(uintptr(ptr) + 6)) = byte(mvccWall >> 8) - *(*byte)(unsafe.Pointer(uintptr(ptr) + 7)) = byte(mvccWall) - - ptr = unsafe.Pointer(uintptr(ptr) + 8) + *(*byte)(ptr) = 0 + *(*byte)(unsafe.Pointer(uintptr(ptr) + 1)) = byte(mvccWall >> 56) + *(*byte)(unsafe.Pointer(uintptr(ptr) + 2)) = byte(mvccWall >> 48) + *(*byte)(unsafe.Pointer(uintptr(ptr) + 3)) = byte(mvccWall >> 40) + *(*byte)(unsafe.Pointer(uintptr(ptr) + 4)) = byte(mvccWall >> 32) + *(*byte)(unsafe.Pointer(uintptr(ptr) + 5)) = byte(mvccWall >> 24) + *(*byte)(unsafe.Pointer(uintptr(ptr) + 6)) = byte(mvccWall >> 16) + *(*byte)(unsafe.Pointer(uintptr(ptr) + 7)) = byte(mvccWall >> 8) + *(*byte)(unsafe.Pointer(uintptr(ptr) + 8)) = byte(mvccWall) + + ptr = unsafe.Pointer(uintptr(ptr) + 9) // This is an MVCC key. if mvccLogical == 0 { *(*byte)(ptr) = 9 - return ki.buf[:len(ki.buf)+9] + return ki.buf[:len(ki.buf)+10] } // Inline binary.BigEndian.PutUint32. @@ -347,7 +363,7 @@ func (ks *cockroachKeySeeker) MaterializeUserKey(ki *PrefixBytesIter, prevRow, r *(*byte)(unsafe.Pointer(uintptr(ptr) + 2)) = byte(mvccWall >> 8) *(*byte)(unsafe.Pointer(uintptr(ptr) + 3)) = byte(mvccWall) *(*byte)(unsafe.Pointer(uintptr(ptr) + 4)) = 13 - return ki.buf[:len(ki.buf)+13] + return ki.buf[:len(ki.buf)+14] } // MaterializeUserKeyWithSyntheticSuffix is part of the KeySeeker interface. @@ -355,15 +371,16 @@ func (ks *cockroachKeySeeker) MaterializeUserKeyWithSyntheticSuffix( ki *PrefixBytesIter, suffix []byte, prevRow, row int, ) []byte { if prevRow+1 == row && prevRow >= 0 { - ks.prefixes.SetNext(ki) + ks.roachKeys.SetNext(ki) } else { - ks.prefixes.SetAt(ki, row) + ks.roachKeys.SetAt(ki, row) } // Slice first, to check that the capacity is sufficient. - res := ki.buf[:len(ki.buf)+len(suffix)] + res := ki.buf[:len(ki.buf)+1+len(suffix)] ptr := unsafe.Pointer(uintptr(unsafe.Pointer(unsafe.SliceData(ki.buf))) + uintptr(len(ki.buf))) - memmove(ptr, unsafe.Pointer(unsafe.SliceData(suffix)), uintptr(len(suffix))) + *(*byte)(ptr) = 0 + memmove(unsafe.Pointer(uintptr(ptr)+1), unsafe.Pointer(unsafe.SliceData(suffix)), uintptr(len(suffix))) return res } @@ -375,25 +392,17 @@ func (ks *cockroachKeySeeker) Release() { func TestCockroachDataBlock(t *testing.T) { const targetBlockSize = 32 << 10 + const valueLen = 100 seed := uint64(time.Now().UnixNano()) t.Logf("seed: %d", seed) rng := rand.New(rand.NewSource(seed)) - keys, values := crdbtest.RandomKVs(rng, targetBlockSize/100, crdbtest.KeyConfig{ + serializedBlock, keys, values := generateDataBlock(rng, targetBlockSize, crdbtest.KeyConfig{ PrefixAlphabetLen: 26, PrefixLen: 12, + AvgKeysPerPrefix: 2, BaseWallTime: seed, - }, 100) + }, valueLen) - var w DataBlockWriter - w.Init(cockroachKeySchema) - var count int - for w.Size() < targetBlockSize { - ik := base.MakeInternalKey(keys[count], base.SeqNum(rng.Uint64n(uint64(base.SeqNumMax))), base.InternalKeyKindSet) - kcmp := w.KeyWriter.ComparePrev(ik.UserKey) - w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, false /* isObsolete */) - count++ - } - serializedBlock, _ := w.Finish(w.Rows(), w.Size()) var reader DataBlockReader var it DataBlockIter it.InitOnce(cockroachKeySchema, crdbtest.Compare, crdbtest.Split, getLazyValuer(func([]byte) base.LazyValue { @@ -415,22 +424,45 @@ func TestCockroachDataBlock(t *testing.T) { t.Fatalf("expected %x, but found %x", values[i], kv.V.InPlaceValue()) } } - require.Equal(t, count, i) + require.Equal(t, len(keys), i) }) t.Run("SeekGE", func(t *testing.T) { rng := rand.New(rand.NewSource(seed)) - for _, i := range rng.Perm(count) { + for _, i := range rng.Perm(len(keys)) { kv := it.SeekGE(keys[i], base.SeekGEFlagsNone) if kv == nil { t.Fatalf("%q not found", keys[i]) } if !bytes.Equal(kv.V.InPlaceValue(), values[i]) { - t.Fatalf("expected %x, but found %x", values[i], kv.V.InPlaceValue()) + t.Fatalf( + "expected:\n %x\nfound:\n %x\nquery key:\n %x\nreturned key:\n %x", + values[i], kv.V.InPlaceValue(), keys[i], kv.K.UserKey) } } }) } +// generateDataBlock writes out a random cockroach data block using the given +// parameters. Returns the serialized block data and the keys and values +// written. +func generateDataBlock( + rng *rand.Rand, targetBlockSize int, cfg crdbtest.KeyConfig, valueLen int, +) (data []byte, keys [][]byte, values [][]byte) { + keys, values = crdbtest.RandomKVs(rng, targetBlockSize/valueLen, cfg, valueLen) + + var w DataBlockWriter + w.Init(cockroachKeySchema) + count := 0 + for w.Size() < targetBlockSize { + ik := base.MakeInternalKey(keys[count], base.SeqNum(rng.Uint64n(uint64(base.SeqNumMax))), base.InternalKeyKindSet) + kcmp := w.KeyWriter.ComparePrev(ik.UserKey) + w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, false /* isObsolete */) + count++ + } + data, _ = w.Finish(w.Rows(), w.Size()) + return data, keys[:count], values[:count] +} + func BenchmarkCockroachDataBlockWriter(b *testing.B) { for _, alphaLen := range []int{4, 8, 26} { for _, lenSharedPct := range []float64{0.25, 0.5} { @@ -442,6 +474,7 @@ func BenchmarkCockroachDataBlockWriter(b *testing.B) { PrefixLen: prefixLen, PrefixLenShared: lenShared, PercentLogical: 0, + AvgKeysPerPrefix: 2, BaseWallTime: uint64(time.Now().UnixNano()), } b.Run(fmt.Sprintf("%s,valueLen=%d", keyConfig, valueLen), func(b *testing.B) { @@ -457,7 +490,7 @@ func benchmarkCockroachDataBlockWriter(b *testing.B, keyConfig crdbtest.KeyConfi const targetBlockSize = 32 << 10 seed := uint64(time.Now().UnixNano()) rng := rand.New(rand.NewSource(seed)) - keys, values := crdbtest.RandomKVs(rng, targetBlockSize/valueLen, keyConfig, valueLen) + _, keys, values := generateDataBlock(rng, targetBlockSize, keyConfig, valueLen) var w DataBlockWriter w.Init(cockroachKeySchema) @@ -593,20 +626,9 @@ func benchmarkCockroachDataBlockIter( seed := uint64(time.Now().UnixNano()) rng := rand.New(rand.NewSource(seed)) cfg.BaseWallTime = seed - keys, values := crdbtest.RandomKVs(rng, targetBlockSize/cfg.ValueLen, cfg.KeyConfig, cfg.ValueLen) - var w DataBlockWriter - w.Init(cockroachKeySchema) - var count int - for w.Size() < targetBlockSize { - ik := base.MakeInternalKey(keys[count], base.SeqNum(rng.Uint64n(uint64(base.SeqNumMax))), base.InternalKeyKindSet) - kcmp := w.KeyWriter.ComparePrev(ik.UserKey) - isObsolete := rng.Intn(20) == 0 - w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, isObsolete) - count++ - } - keys = keys[:count] - serializedBlock, _ := w.Finish(w.Rows(), w.Size()) + serializedBlock, keys, _ := generateDataBlock(rng, targetBlockSize, cfg.KeyConfig, cfg.ValueLen) + var reader DataBlockReader var it DataBlockIter it.InitOnce(cockroachKeySchema, crdbtest.Compare, crdbtest.Split, getLazyValuer(func([]byte) base.LazyValue { @@ -616,7 +638,7 @@ func benchmarkCockroachDataBlockIter( if err := it.Init(&reader, transforms); err != nil { b.Fatal(err) } - avgRowSize := float64(len(serializedBlock)) / float64(count) + avgRowSize := float64(len(serializedBlock)) / float64(len(keys)) if transforms.SyntheticPrefix.IsSet() { for i := range keys {