Skip to content

Commit

Permalink
colblk: add UnsafeOffsets.At2 and improve UnsafeUints.At
Browse files Browse the repository at this point in the history
We add a function that returns two consecutive offsets (to cut down on
the branches and size of the inlined code). We also reorganize
`UnsafeUints.At` to check the most common cases first.

```
CockroachDataBlockIter/AlphaLen=4,Shared=8,PrefixLen=32,Logical=0,value=8/Next    11.1ns ± 0%    10.3ns ± 3%  -7.17%  (p=0.016 n=4+5)
```
  • Loading branch information
RaduBerinde committed Aug 22, 2024
1 parent 55fcc6e commit 99abcf7
Show file tree
Hide file tree
Showing 4 changed files with 104 additions and 35 deletions.
4 changes: 2 additions & 2 deletions sstable/colblk/data_block.go
Original file line number Diff line number Diff line change
Expand Up @@ -662,8 +662,8 @@ func (i *DataBlockIter) Next() *base.InternalKV {
Trailer: base.InternalKeyTrailer(i.r.trailers.At(i.row)),
}
// Inline i.r.values.At(row).
startOffset := i.r.values.offsets.At(i.row)
v := unsafe.Slice((*byte)(i.r.values.ptr(startOffset)), i.r.values.offsets.At(i.row+1)-startOffset)
startOffset, endOffset := i.r.values.offsets.At2(i.row)
v := unsafe.Slice((*byte)(i.r.values.ptr(startOffset)), endOffset-startOffset)
if i.r.isValueExternal.At(i.row) {
i.kv.V = i.getLazyValue(v)
} else {
Expand Down
33 changes: 15 additions & 18 deletions sstable/colblk/prefix_bytes.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,8 +255,8 @@ type PrefixBytesIter struct {
func (b *PrefixBytes) SetAt(it *PrefixBytesIter, i int) {
// Determine the offset and length of the bundle prefix.
bundleOffsetIndex := b.bundleOffsetIndexForRow(i)
bundleOffsetStart := b.rawBytes.offsets.At(bundleOffsetIndex)
it.bundlePrefixLen = b.rawBytes.offsets.At(bundleOffsetIndex+1) - bundleOffsetStart
bundleOffsetStart, bundleOffsetEnd := b.rawBytes.offsets.At2(bundleOffsetIndex)
it.bundlePrefixLen = bundleOffsetEnd - bundleOffsetStart

// Determine the offset and length of the row's individual suffix.
it.offsetIndex = b.rowSuffixIndex(i)
Expand All @@ -265,9 +265,9 @@ func (b *PrefixBytes) SetAt(it *PrefixBytesIter, i int) {
// this recomputation? The expected case is non-duplicate keys, so it may
// not be worthwhile.
rowSuffixStart, rowSuffixEnd := b.rowSuffixOffsets(i, it.offsetIndex)
rowSuffixLen := rowSuffixEnd - rowSuffixStart

// Grow the size of the iterator's buffer if necessary.
it.buf = it.buf[:b.sharedPrefixLen+int(it.bundlePrefixLen)+int(rowSuffixEnd-rowSuffixStart)]
it.buf = it.buf[:b.sharedPrefixLen+int(it.bundlePrefixLen+rowSuffixLen)]

ptr := unsafe.Pointer(unsafe.SliceData(it.buf))
// Copy the shared key prefix.
Expand All @@ -281,7 +281,7 @@ func (b *PrefixBytes) SetAt(it *PrefixBytesIter, i int) {
memmove(
unsafe.Pointer(uintptr(ptr)+uintptr(b.sharedPrefixLen)+uintptr(it.bundlePrefixLen)),
unsafe.Pointer(uintptr(b.rawBytes.data)+uintptr(rowSuffixStart)),
uintptr(rowSuffixEnd-rowSuffixStart))
uintptr(rowSuffixLen))
// Set nextBundleOffsetIndex so that a call to SetNext can cheaply determine
// whether the next row is in the same bundle.
it.nextBundleOffsetIndex = bundleOffsetIndex + (1 << b.bundleShift) + 1
Expand All @@ -297,40 +297,38 @@ func (b *PrefixBytes) SetNext(it *PrefixBytesIter) {
// If the next row is in the same bundle, we can take a fast path of only
// updating the per-row suffix.
if it.offsetIndex < it.nextBundleOffsetIndex {
rowSuffixStart := b.rawBytes.offsets.At(it.offsetIndex)
rowSuffixEnd := b.rawBytes.offsets.At(it.offsetIndex + 1)
if rowSuffixStart == rowSuffixEnd {
rowSuffixStart, rowSuffixEnd := b.rawBytes.offsets.At2(it.offsetIndex)
rowSuffixLen := rowSuffixEnd - rowSuffixStart
if rowSuffixLen == 0 {
// The start and end offsets are equal, indicating that the key is a
// duplicate. Since it's identical to the previous key, there's
// nothing left to do, we can leave buf as-is.
return
}
// Grow the buffer if necessary.
it.buf = it.buf[:b.sharedPrefixLen+int(it.bundlePrefixLen)+int(rowSuffixEnd-rowSuffixStart)]
it.buf = it.buf[:b.sharedPrefixLen+int(it.bundlePrefixLen+rowSuffixLen)]
// Copy in the per-row suffix.
ptr := unsafe.Pointer(unsafe.SliceData(it.buf))
memmove(
unsafe.Pointer(uintptr(ptr)+uintptr(b.sharedPrefixLen)+uintptr(it.bundlePrefixLen)),
unsafe.Pointer(uintptr(b.rawBytes.data)+uintptr(rowSuffixStart)),
uintptr(rowSuffixEnd-rowSuffixStart))
uintptr(rowSuffixLen))
return
}

// We've reached the end of the bundle. We need to update the bundle prefix.
// The offsetIndex is currently pointing to the start of the new bundle
// prefix. Increment it to point at the start of the new row suffix.
it.offsetIndex++
rowSuffixStart := b.rawBytes.offsets.At(it.offsetIndex)
rowSuffixEnd := b.rawBytes.offsets.At(it.offsetIndex + 1)
rowSuffixStart, rowSuffixEnd := b.rawBytes.offsets.At2(it.offsetIndex)
rowSuffixLen := rowSuffixEnd - rowSuffixStart

// Read the offsets of the new bundle prefix and update the index of the
// next bundle.
bundlePrefixStart := b.rawBytes.offsets.At(it.nextBundleOffsetIndex)
it.bundlePrefixLen = rowSuffixStart - bundlePrefixStart
it.nextBundleOffsetIndex = it.offsetIndex + (1 << b.bundleShift)

// Grow the buffer if necessary.
it.buf = it.buf[:b.sharedPrefixLen+int(it.bundlePrefixLen)+int(rowSuffixEnd-rowSuffixStart)]
it.buf = it.buf[:b.sharedPrefixLen+int(it.bundlePrefixLen+rowSuffixLen)]
// Copy in the new bundle suffix.
ptr := unsafe.Pointer(unsafe.SliceData(it.buf))
memmove(
Expand All @@ -341,7 +339,7 @@ func (b *PrefixBytes) SetNext(it *PrefixBytesIter) {
memmove(
unsafe.Pointer(uintptr(ptr)+uintptr(b.sharedPrefixLen)+uintptr(it.bundlePrefixLen)),
unsafe.Pointer(uintptr(b.rawBytes.data)+uintptr(rowSuffixStart)),
uintptr(rowSuffixEnd-rowSuffixStart))
uintptr(rowSuffixLen))
}

// SharedPrefix return a []byte of the shared prefix that was extracted from
Expand Down Expand Up @@ -383,8 +381,7 @@ func (b *PrefixBytes) RowSuffix(row int) []byte {
func (b *PrefixBytes) rowSuffixOffsets(row, i int) (low uint32, high uint32) {
// Retrieve the low and high offsets indicating the start and end of the
// row's suffix slice.
low = b.rawBytes.offsets.At(i)
high = b.rawBytes.offsets.At(i + 1)
low, high = b.rawBytes.offsets.At2(i)
// If there's a non-empty slice for the row, this row is different than its
// predecessor.
if low != high {
Expand Down
50 changes: 38 additions & 12 deletions sstable/colblk/unsafe_slice.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,23 +92,27 @@ func makeUnsafeUints(base uint64, ptr unsafe.Pointer, width int) UnsafeUints {

// At returns the `i`-th element.
func (s UnsafeUints) At(i int) uint64 {
// TODO(jackson): Experiment with other alternatives that might be faster
// and avoid switching on the width.
switch s.width {
case 0:
return s.base
case 1:
return s.base + uint64(*(*uint8)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i))))
case 2:
return s.base + uint64(*(*uint16)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align16Shift)))
case 4:
return s.base + uint64(*(*uint32)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align32Shift)))
default:
// TODO(radu): this implementation assumes little-endian architecture.

// One of the most common case is decoding timestamps, which require the full
// 8 bytes (2^32 nanoseconds is only ~4 seconds).
if s.width == 8 {
// NB: The slice encodes 64-bit integers, there is no base (it doesn't save
// any bits to compute a delta). We cast directly into a *uint64 pointer and
// don't add the base.
return *(*uint64)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align64Shift))
}
// Another common case is 0 width, when all keys have zero logical timestamps.
if s.width == 0 {
return s.base
}
if s.width == 4 {
return s.base + uint64(*(*uint32)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align32Shift)))
}
if s.width == 2 {
return s.base + uint64(*(*uint16)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align16Shift)))
}
return s.base + uint64(*(*uint8)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i))))
}

// UnsafeOffsets is a specialization of UnsafeInts (providing the same
Expand All @@ -135,6 +139,8 @@ func DecodeUnsafeOffsets(b []byte, off uint32, rows int) (_ UnsafeOffsets, endOf

// At returns the `i`-th offset.
func (s UnsafeOffsets) At(i int) uint32 {
// TODO(radu): this implementation assumes little-endian architecture.

// We expect offsets to be encoded as 16-bit integers in most cases.
if s.width == 2 {
return uint32(*(*uint16)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align16Shift)))
Expand All @@ -148,6 +154,26 @@ func (s UnsafeOffsets) At(i int) uint32 {
return *(*uint32)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align32Shift))
}

// At2 returns the `i`-th and `i+1`-th offsets.
func (s UnsafeOffsets) At2(i int) (uint32, uint32) {
// TODO(radu): this implementation assumes little-endian architecture.

// We expect offsets to be encoded as 16-bit integers in most cases.
if s.width == 2 {
v := *(*uint32)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align16Shift))
return v & 0xFFFF, v >> 16
}
if s.width <= 1 {
if s.width == 0 {
return 0, 0
}
v := *(*uint16)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)))
return uint32(v & 0xFF), uint32(v >> 8)
}
v := *(*uint64)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)<<align32Shift))
return uint32(v), uint32(v >> 32)
}

// UnsafeBuf provides a buffer without bounds checking. Every buf has a len and
// capacity.
type UnsafeBuf struct {
Expand Down
52 changes: 49 additions & 3 deletions sstable/colblk/unsafe_slice_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,56 @@ import (
"golang.org/x/exp/rand"
)

func TestUnsafeUints(t *testing.T) {
rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano())))
for _, r := range interestingIntRanges {
t.Run(r.ExpectedEncoding.String(), func(t *testing.T) {
for _, rows := range []int{1, 10, 100, 10000} {
t.Run(fmt.Sprint(rows), func(t *testing.T) {
vals := make([]uint64, rows)
for i := range vals {
vals[i] = r.Rand(rng)
}
var ub UintBuilder
ub.Init()
for i := 0; i < rows; i++ {
ub.Set(i, vals[i])
}
sz := ub.Size(rows, 0)
buf := aligned.ByteSlice(int(sz) + 1 /* trailing padding byte */)
_ = ub.Finish(0, rows, 0, buf)

uints, _ := DecodeUnsafeUints(buf, 0, rows)
for i := range rows {
if uints.At(i) != vals[i] {
t.Fatalf("mismatch at row %d: got %d, expected %d", i, uints.At(i), vals[i])
}
}
if encoding := UintEncoding(buf[0]); encoding.Width() <= 4 && !encoding.IsDelta() {
offsets, _ := DecodeUnsafeOffsets(buf, 0, rows)
for i := range rows {
if uint64(offsets.At(i)) != vals[i] {
t.Fatalf("mismatch at row %d: got %d, expected %d", i, uints.At(i), vals[i])
}
}
for i := 0; i < rows-1; i++ {
a, b := offsets.At2(i)
if uint64(a) != vals[i] || uint64(b) != vals[i+1] {
t.Fatalf("mismatch at row %d: got %d,%d, expected %d,%d", i, a, b, vals[i], vals[i+1])
}
}
}
})
}
})
}
}

func BenchmarkUnsafeUints(b *testing.B) {
rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano())))
intRanges := []intRange{
// const
{Min: 1, Max: 1, ExpectedEncoding: makeUintEncoding(0, true)},
// zero
{Min: 0, Max: 0, ExpectedEncoding: makeUintEncoding(0, false)},
// 1b
{Min: 10, Max: 200, ExpectedEncoding: makeUintEncoding(1, false)},
// 1b,delta
Expand All @@ -31,10 +76,11 @@ func BenchmarkUnsafeUints(b *testing.B) {
// 4b
{Min: 0, Max: math.MaxUint32, ExpectedEncoding: makeUintEncoding(4, false)},
// 4b,delta
{Min: 100_000, Max: math.MaxUint32 + 10, ExpectedEncoding: makeUintEncoding(4, true)},
{Min: 1_000_000_100, Max: math.MaxUint32 + 1_000_000_000, ExpectedEncoding: makeUintEncoding(4, true)},
// 8b
{Min: 0, Max: math.MaxUint64, ExpectedEncoding: makeUintEncoding(8, false)},
}

for _, r := range intRanges {
benchmarkUnsafeUints(b, rng, 1000, r)
}
Expand Down

0 comments on commit 99abcf7

Please sign in to comment.