diff --git a/sstable/colblk/base.go b/sstable/colblk/base.go new file mode 100644 index 0000000000..4bdc5ca4ef --- /dev/null +++ b/sstable/colblk/base.go @@ -0,0 +1,45 @@ +// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +// Package colblk implements a columnar block format. +package colblk + +import "golang.org/x/exp/constraints" + +// align returns the next value greater than or equal to offset that's divisible +// by val. +func align[T constraints.Integer](offset, val T) T { + return (offset + val - 1) & ^(val - 1) +} + +// alignWithZeroes aligns the provided offset to val, and writing zeroes to any +// bytes in buf between the old offset and new aligned offset. This provides +// determinism when reusing memory that has not been zeroed. +func alignWithZeroes[T constraints.Integer](buf []byte, offset, val T) T { + aligned := align[T](offset, val) + for i := offset; i < aligned; i++ { + buf[i] = 0 + } + return aligned +} + +const ( + align16 = 2 + align32 = 4 + align64 = 8 +) + +// When multiplying or dividing by align{16,32,64} using signed integers, it's +// faster to shift to the left to multiply or shift to the right to divide. (The +// compiler optimization is limited to unsigned integers.) The below constants +// define the shift amounts corresponding to the above align constants. (eg, +// alignNShift = log(alignN)). +// +// TODO(jackson): Consider updating usages to use uints? They can be less +// ergonomic. +const ( + align16Shift = 1 + align32Shift = 2 + align64Shift = 3 +) diff --git a/sstable/colblk/bitmap.go b/sstable/colblk/bitmap.go new file mode 100644 index 0000000000..39a3ff8587 --- /dev/null +++ b/sstable/colblk/bitmap.go @@ -0,0 +1,317 @@ +// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package colblk + +import ( + "fmt" + "io" + "math/bits" + "slices" + "strings" + "unsafe" + + "github.com/cockroachdb/errors" + "github.com/cockroachdb/pebble/internal/binfmt" +) + +// Bitmap is a bitmap structure built on a []uint64. A bitmap utilizes ~1 +// physical bit/logical bit (~0.125 bytes/row). The bitmap is encoded into an +// 8-byte aligned array of 64-bit words which is (nRows+63)/64 words in length. +// +// A summary bitmap is stored after the primary bitmap in which each bit in the +// summary bitmap corresponds to 1 word in the primary bitmap. A bit is set in +// the summary bitmap if the corresponding word in the primary bitmap is +// non-zero. The summary bitmap accelerates predecessor and successor +// operations. +type Bitmap struct { + data UnsafeRawSlice[uint64] + bitCount int +} + +// MakeBitmap returns a Bitmap that reads from b supporting bitCount logical +// bits. No bounds checking is performed, so the caller must guarantee the +// bitmap is appropriately sized and the provided bitCount correctly identifies +// the number of bits in the bitmap. +func MakeBitmap(b []byte, bitCount int) Bitmap { + if len(b) != bitmapRequiredSize(bitCount) { + panic(errors.AssertionFailedf("bitmap of %d bits requires at %d bytes; provided with %d-byte slice", + bitCount, bitmapRequiredSize(bitCount), len(b))) + } + return Bitmap{ + data: makeUnsafeRawSlice[uint64](unsafe.Pointer(&b[0])), + bitCount: bitCount, + } +} + +// Get returns true if the bit at position i is set and false otherwise. +func (b Bitmap) Get(i int) bool { + return (b.data.At(i>>6 /* i/64 */) & (1 << uint(i%64))) != 0 +} + +// Successor returns the next bit greater than or equal to i set in the bitmap. +// The i parameter must be in [0, bitCount). Returns the number of bits +// represented by the bitmap if no next bit is set. +func (b Bitmap) Successor(i int) int { + // nextInWord returns the index of the smallest set bit with an index >= bit + // within the provided word. The returned index is an index local to the + // word. + nextInWord := func(word uint64, bit uint) int { + // We want to find the index of the next set bit. We can accomplish this + // by clearing the trailing `bit` bits from the word and counting the + // number of trailing zeros. For example, consider the word and bit=37: + // + // word: 1010101010111111111110000001110101010101011111111111000000111011 + // + // 1<> 6 // i/64 + // Fast path for common case of reasonably dense bitmaps; if the there's a + // bit > i set in the same word, return it. + if next := nextInWord(b.data.At(wordIdx), uint(i%64)); next < 64 { + return wordIdx<<6 + next + } + + // Consult summary structure to find the next word with a set bit. The word + // we just checked (wordIdx) is represented by the wordIdx%64'th bit in the + // wordIdx/64'th summary word. We want to know if any of the other later + // words that are summarized together have a set bit. We call [nextInWord] + // on the summary word to get the index of which word has a set bit, if any. + summaryTableOffset, summaryTableEnd := b.summaryTableBounds() + summaryWordIdx := summaryTableOffset + wordIdx>>6 + summaryNext := nextInWord(b.data.At(summaryWordIdx), uint(wordIdx%64)+1) + // If [summaryNext] == 64, then there are no set bits in any of the earlier + // words represented by the summary word at [summaryWordIdx]. In that case, + // we need to keep scanning the summary table forwards. + if summaryNext == 64 { + for summaryWordIdx++; ; summaryWordIdx++ { + // When we fall off the end of the summary table, we've determined + // there are no set bits after i across the entirety of the bitmap. + if summaryWordIdx >= summaryTableEnd { + return b.bitCount + } + if summaryWord := b.data.At(summaryWordIdx); summaryWord != 0 { + summaryNext = bits.TrailingZeros64(summaryWord) + break + } + } + } + // The summary word index and the summaryNext together tell us which word + // has a set bit. The number of leading zeros in the word itself tell us + // which bit is set. + wordIdx = ((summaryWordIdx - summaryTableOffset) << 6) + summaryNext + return (wordIdx << 6) + bits.TrailingZeros64(b.data.At(wordIdx)) +} + +// Predecessor returns the previous bit less than or equal to i set in the +// bitmap. The i parameter must be in [0, bitCount). Returns -1 if no previous +// bit is set. +func (b Bitmap) Predecessor(i int) int { + // prevInWord returns the index of the largest set bit ≤ bit within the + // provided word. The returned index is an index local to the word. Returns + // -1 if no set bit is found. + prevInWord := func(word uint64, bit uint) int { + // We want to find the index of the previous set bit. We can accomplish + // this by clearing the leading `bit` bits from the word and counting + // the number of leading zeros. For example, consider the word and + // bit=42: + // + // word: 1010101010111111111110000001110101010101011111111111000000111011 + // + // 1<<(bit+1): 0000000000000000000010000000000000000000000000000000000000000000 + // 1<<(bit+1)-1: 0000000000000000000001111111111111111111111111111111111111111111 + // word&1<<(bit+1)-1: 0000000000000000000000000001110101010101011111111111000000111011 + // + // Counting the leading zeroes of this last value gives us 27 leading + // zeros. 63-27 gives index 36. For visualizing, 1<<36 is: + // + // 0000000000000000000000000001000000000000000000000000000000000000 + // + return 63 - bits.LeadingZeros64(word&((1<<(bit+1))-1)) + } + + wordIdx := i >> 6 // i/64 + // Fast path for common case of reasonably dense bitmaps; if the there's a + // bit < i set in the same word, return it. + if prev := prevInWord(b.data.At(wordIdx), uint(i%64)); prev >= 0 { + return (wordIdx << 6) + prev + } + + // Consult summary structure to find the next word with a set bit. The word + // we just checked (wordIdx) is represented by the wordIdx%64'th bit in the + // wordIdx/64'th summary word. We want to know if any of other earlier words + // that are summarized together have a set bit. We call [prevInWord] on the + // summary word to get the index of which word has a set bit, if any. + summaryTableOffset, _ := b.summaryTableBounds() + summaryWordIdx := summaryTableOffset + wordIdx>>6 + summaryPrev := prevInWord(b.data.At(summaryWordIdx), uint(wordIdx%64)-1) + // If [summaryPrev] is negative, then there are no set bits in any of the + // earlier words represented by the summary word at [summaryWordIdx]. In + // that case, we need to keep scanning the summary table backwards. + if summaryPrev < 0 { + for summaryWordIdx--; ; summaryWordIdx-- { + // When we fall below the beginning of the summary table, we've + // determined there are no set bits before i across the entirety of + // the bitmap. + if summaryWordIdx < summaryTableOffset { + return -1 + } + if summaryWord := b.data.At(summaryWordIdx); summaryWord != 0 { + summaryPrev = 63 - bits.LeadingZeros64(summaryWord) + break + } + } + } + // The summary word index and the summary prev together tell us which word + // has a set bit. The number of trailing zeros in the word itself tell us + // which bit is set. + wordIdx = ((summaryWordIdx - summaryTableOffset) << 6) + summaryPrev + return (wordIdx << 6) + 63 - bits.LeadingZeros64(b.data.At(wordIdx)) +} + +func (b Bitmap) summaryTableBounds() (startOffset, endOffset int) { + startOffset = (b.bitCount + 63) >> 6 + endOffset = startOffset + startOffset>>6 + return startOffset, endOffset +} + +// String returns a string representation of the entire bitmap. +func (b Bitmap) String() string { + var sb strings.Builder + for w := 0; w < (b.bitCount+63)/64; w++ { + fmt.Fprintf(&sb, "%064b", b.data.At(w)) + } + return sb.String() +} + +// BitmapBuilder constructs a Bitmap. Bits are default false. +type BitmapBuilder struct { + words []uint64 +} + +func bitmapRequiredSize(total int) int { + nWords := (total + 63) >> 6 // divide by 64 + nSummaryWords := (nWords + 63) >> 6 // divide by 64 + return (nWords + nSummaryWords) << 3 // multiply by 8 +} + +// Set sets the bit at position i if v is true and clears the bit at position i +// otherwise. Callers need not call Set if v is false and Set(i, true) has not +// been called yet. +func (b *BitmapBuilder) Set(i int, v bool) { + w := i >> 6 // divide by 64 + for len(b.words) <= w { + b.words = append(b.words, 0) + } + if v { + b.words[w] |= 1 << uint(i%64) + } else { + b.words[w] &^= 1 << uint(i%64) + } +} + +// Reset resets the bitmap to the empty state. +func (b *BitmapBuilder) Reset() { + clear(b.words) + b.words = b.words[:0] +} + +// NumColumns implements the ColumnWriter interface. +func (b *BitmapBuilder) NumColumns() int { return 1 } + +// Size implements the ColumnWriter interface. +func (b *BitmapBuilder) Size(rows int, offset uint32) uint32 { + offset = align(offset, align64) + return offset + uint32(bitmapRequiredSize(rows)) +} + +// Invert inverts the bitmap, setting all bits that are not set and clearing all +// bits that are set. If the bitmap's tail is sparse and is not large enough to +// represent nRows rows, it's first materialized. +func (b *BitmapBuilder) Invert(nRows int) { + // If the tail of b is sparse, fill in zeroes before inverting. + nBitmapWords := (nRows + 63) >> 6 + b.words = slices.Grow(b.words, nBitmapWords-len(b.words))[:nBitmapWords] + for i := range b.words { + b.words[i] = ^b.words[i] + } +} + +// Finish finalizes the bitmap, computing the per-word summary bitmap and +// writing the resulting data to buf at offset. +func (b *BitmapBuilder) Finish(col, nRows int, offset uint32, buf []byte) (uint32, ColumnDesc) { + offset = alignWithZeroes(buf, offset, align64) + dest := makeUnsafeRawSlice[uint64](unsafe.Pointer(&buf[offset])) + + nBitmapWords := (nRows + 63) >> 6 + // Truncate the bitmap to the number of words required to represent nRows. + // The caller may have written more bits than nRows and no longer cares to + // write them out. + if len(b.words) > nBitmapWords { + b.words = b.words[:nBitmapWords] + } + // Ensure the last word of the bitmap does not contain any set bits beyond + // the last row. This is not just for determinism but also to ensure that + // the summary bitmap is correct (which is necessary for Bitmap.Successor + // correctness). + if i := nRows % 64; len(b.words) >= nBitmapWords && i != 0 { + b.words[nBitmapWords-1] &= (1 << i) - 1 + } + + // Copy all the words of the bitmap into the destination buffer. + offset += uint32(copy(dest.Slice(len(b.words)), b.words)) << align64Shift + + // The caller may have written fewer than nRows rows if the tail is all + // zeroes, relying on these bits being implicitly zero. If the tail of b is + // sparse, fill in zeroes. + for i := len(b.words); i < nBitmapWords; i++ { + dest.set(i, 0) + offset += align64 + } + + // Add the summary bitmap. + nSummaryWords := (nBitmapWords + 63) >> 6 + for i := 0; i < nSummaryWords; i++ { + wordsOff := (i << 6) // i*64 + nWords := min(64, len(b.words)-wordsOff) + var summaryWord uint64 + for j := 0; j < nWords; j++ { + if (b.words)[wordsOff+j] != 0 { + summaryWord |= 1 << j + } + } + dest.set(nBitmapWords+i, summaryWord) + } + offset += uint32(nSummaryWords) << align64Shift + return offset, ColumnDesc{DataType: DataTypeBool, Encoding: EncodingDefault} +} + +// WriteDebug implements the ColumnWriter interface. +func (b *BitmapBuilder) WriteDebug(w io.Writer, rows int) { + // TODO(jackson): Add more detailed debugging information. + fmt.Fprint(w, "bitmap") +} + +func bitmapToBinFormatter(f *binfmt.Formatter, rows int) int { + bitmapWords := (rows + 63) / 64 + for i := 0; i < bitmapWords; i++ { + f.Line(8).Append("b ").Binary(8).Done("bitmap word %d", i) + } + summaryWords := (bitmapWords + 63) / 64 + for i := 0; i < summaryWords; i++ { + f.Line(8).Append("b ").Binary(8).Done("bitmap summary word %d-%d", i*64, i*64+63) + } + return (bitmapWords + summaryWords) * align64 +} diff --git a/sstable/colblk/bitmap_test.go b/sstable/colblk/bitmap_test.go new file mode 100644 index 0000000000..82d7430f33 --- /dev/null +++ b/sstable/colblk/bitmap_test.go @@ -0,0 +1,169 @@ +// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package colblk + +import ( + "bytes" + "fmt" + "io" + "testing" + "time" + "unicode" + + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/pebble/internal/binfmt" + "golang.org/x/exp/rand" +) + +func TestBitmapFixed(t *testing.T) { + var bitmap Bitmap + var buf bytes.Buffer + datadriven.RunTest(t, "testdata/bitmap", func(t *testing.T, td *datadriven.TestData) string { + buf.Reset() + switch td.Cmd { + case "build": + var builder BitmapBuilder + var n int + for _, r := range td.Input { + if unicode.IsSpace(r) { + continue + } + if r == '1' { + builder.Set(n, r == '1') + } + n++ + } + td.MaybeScanArgs(t, "rows", &n) + + data := make([]byte, builder.Size(n, 0)) + if td.HasArg("invert") { + builder.Invert(n) + } + + _, _ = builder.Finish(0, n, 0, data) + bitmap = MakeBitmap(data, n) + dumpBitmap(&buf, bitmap) + fmt.Fprint(&buf, "\nBinary representation:\n") + f := binfmt.New(data) + bitmapToBinFormatter(f, n) + fmt.Fprint(&buf, f.String()) + return buf.String() + case "successor": + var indexes []int + td.ScanArgs(t, "indexes", &indexes) + for _, idx := range indexes { + fmt.Fprintf(&buf, "bitmap.Successor(%d) = %d\n", idx, bitmap.Successor(idx)) + } + return buf.String() + case "predecessor": + var indexes []int + td.ScanArgs(t, "indexes", &indexes) + for _, idx := range indexes { + fmt.Fprintf(&buf, "bitmap.Predecessor(%d) = %d\n", idx, bitmap.Predecessor(idx)) + } + return buf.String() + default: + panic(fmt.Sprintf("unknown command: %s", td.Cmd)) + } + }) +} + +func dumpBitmap(w io.Writer, b Bitmap) { + for i := 0; i < b.bitCount; i++ { + if i > 0 && i%64 == 0 { + w.Write([]byte{'\n'}) + } + if b.Get(i) { + w.Write([]byte{'1'}) + } else { + w.Write([]byte{'0'}) + } + } +} + +func TestBitmapRandom(t *testing.T) { + seed := uint64(time.Now().UnixNano()) + t.Logf("seed: %d", seed) + rng := rand.New(rand.NewSource(seed)) + size := rng.Intn(4096) + 1 + + testWithProbability := func(t *testing.T, p float64) { + var builder BitmapBuilder + v := make([]bool, size) + for i := 0; i < size; i++ { + v[i] = rng.Float64() < p + if v[i] { + builder.Set(i, v[i]) + } + } + data := make([]byte, builder.Size(size, 0)) + _, _ = builder.Finish(0, size, 0, data) + bitmap := MakeBitmap(data, size) + for i := 0; i < size; i++ { + if got := bitmap.Get(i); got != v[i] { + t.Fatalf("b.Get(%d) = %t; want %t", i, got, v[i]) + } + } + for i := 0; i < size; i++ { + succ := bitmap.Successor(i) + // Ensure that Successor always returns the index of a set bit. + if succ != size && !bitmap.Get(succ) { + t.Fatalf("b.Successor(%d) = %d; bit at index %d is not set", i, succ, succ) + } + pred := bitmap.Predecessor(i) + // Ensure that Predecessor always returns the index of a set bit. + if pred >= 0 && !bitmap.Get(pred) { + t.Fatalf("b.Predecessor(%d) = %d; bit at index %d is not set", i, pred, pred) + } + + // Ensure there are no set bits between i and succ. + for j := i; j < succ; j++ { + if bitmap.Get(j) { + t.Fatalf("b.Successor(%d) = %d; bit at index %d is set", i, succ, j) + } + } + // Ensure there are no set bits between pred and i. + for j := pred + 1; j < i; j++ { + if bitmap.Get(j) { + t.Fatalf("b.Predecessor(%d) = %d; bit at index %d is set", i, pred, j) + } + } + } + } + + fixedProbabilities := []float64{0.00001, 0.0001, 0.001, 0.1, 0.5, 0.9999} + for _, p := range fixedProbabilities { + t.Run(fmt.Sprintf("p=%05f", p), func(t *testing.T) { + testWithProbability(t, p) + }) + } + for i := 0; i < 10; i++ { + p := rng.ExpFloat64() * 0.1 + t.Run(fmt.Sprintf("p=%05f", p), func(t *testing.T) { + testWithProbability(t, p) + }) + } +} + +func BenchmarkBitmapBuilder(b *testing.B) { + seed := uint64(10024282523) + rng := rand.New(rand.NewSource(seed)) + size := rng.Intn(4096) + 1 + v := make([]bool, size) + for i := 0; i < size; i++ { + v[i] = rng.Intn(2) == 0 + } + data := make([]byte, bitmapRequiredSize(size)) + b.ResetTimer() + for i := 0; i < b.N; i++ { + var builder BitmapBuilder + for i := 0; i < size; i++ { + if v[i] { + builder.Set(i, v[i]) + } + } + _, _ = builder.Finish(0, size, 0, data) + } +} diff --git a/sstable/colblk/column.go b/sstable/colblk/column.go new file mode 100644 index 0000000000..b0ed8f8f7b --- /dev/null +++ b/sstable/colblk/column.go @@ -0,0 +1,159 @@ +// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package colblk + +import ( + "bytes" + "fmt" + "io" + "strings" +) + +// DataType describes the logical type of a column's values. Some data types +// have multiple possible physical representations. Encoding a column may choose +// between possible physical representations depending on the distribution of +// values and the size of the resulting physical representation. +type DataType uint8 + +const ( + // DataTypeInvalid represents an unset or invalid data type. + DataTypeInvalid DataType = 0 + // DataTypeBool is a data type encoding a bool per row. + DataTypeBool DataType = 1 + // DataTypeUint8 is a data type encoding a fixed 8 bits per row. + DataTypeUint8 DataType = 2 + // DataTypeUint16 is a data type encoding a fixed 16 bits per row. + DataTypeUint16 DataType = 3 + // DataTypeUint32 is a data type encoding a fixed 32 bits per row. + DataTypeUint32 DataType = 4 + // DataTypeUint64 is a data type encoding a fixed 64 bits per row. + DataTypeUint64 DataType = 5 + // DataTypeBytes is a data type encoding a variable-length byte string per + // row. + DataTypeBytes DataType = 6 + // DataTypePrefixBytes is a data type encoding variable-length, + // lexicographically-sorted byte strings, with prefix compression. + DataTypePrefixBytes DataType = 7 + + dataTypesCount DataType = 8 +) + +var dataTypeName [dataTypesCount]string = [dataTypesCount]string{ + DataTypeInvalid: "invalid", + DataTypeBool: "bool", + DataTypeUint8: "uint8", + DataTypeUint16: "uint16", + DataTypeUint32: "uint32", + DataTypeUint64: "uint64", + DataTypeBytes: "bytes", + DataTypePrefixBytes: "prefixbytes", +} + +// String returns a human-readable string representation of the data type. +func (t DataType) String() string { + return dataTypeName[t] +} + +// ColumnDesc describes the column's data type and its encoding. +type ColumnDesc struct { + DataType DataType + Encoding ColumnEncoding +} + +// TODO(jackson): Ensure we provide a mechanism for future extensibility: maybe +// a single byte to represent the colblk version? + +// String returns a human-readable string describing the column encoding. +func (d ColumnDesc) String() string { + var sb strings.Builder + fmt.Fprint(&sb, d.DataType.String()) + if d.Encoding != EncodingDefault { + fmt.Fprintf(&sb, "+%s", d.Encoding) + } + return sb.String() +} + +// ColumnDescs is a slice of ColumnDesc values. +type ColumnDescs []ColumnDesc + +// String returns the concatenated string representations of the column +// descriptions. +func (c ColumnDescs) String() string { + var buf bytes.Buffer + for i := range c { + if i > 0 { + buf.WriteString(" ") + } + buf.WriteString(c[i].String()) + } + return buf.String() +} + +// ColumnEncoding describes the encoding of a column. +// +// Null bitmap (bit 0): +// +// The bit at position 0 indicates whether the column is prefixed with a null +// bitmap. A null bitmap is a bitmap where each bit corresponds to a row in the +// column, and a set bit indicates that the corresponding row is NULL. +// +// TODO(jackson): Add additional column encoding types. +type ColumnEncoding uint8 + +const ( + // EncodingDefault indicates that the default encoding is in-use for a + // column, encoding n values for n rows. + EncodingDefault ColumnEncoding = 0 + // TODO(jackson): Add additional encoding types. + encodingTypeCount = 1 +) + +// String returns the string representation of the column encoding. +func (e ColumnEncoding) String() string { + return encodingName[e] +} + +var encodingName [encodingTypeCount]string = [encodingTypeCount]string{ + EncodingDefault: "default", +} + +// ColumnWriter is an interface implemented by column encoders that accumulate a +// column's values and then serialize them. +type ColumnWriter interface { + Encoder + // NumColumns returns the number of columns the ColumnWriter will encode. + NumColumns() int + // Finish serializes the column at the specified index, writing the column's + // data to buf at offset, and returning the offset at which the next column + // should be encoded. Finish also returns a column descriptor describing the + // encoding of the column, which will be serialized within the block header. + // + // The supplied buf must have enough space at the provided offset to fit the + // column. The caller may use Size() to calculate the exact size required. + // If [rows] is ≤ the highest row index at which a value has been set, + // Finish will only serialize the first [rows] values. + // + // The provided column index must be less than NumColumns(). Finish is + // called for each index < NumColumns() in order. + Finish(col int, rows int, offset uint32, buf []byte) (nextOffset uint32, desc ColumnDesc) +} + +// Encoder is an interface implemented by column encoders. +type Encoder interface { + // Reset clears the ColumnWriter's internal state, preparing it for reuse. + Reset() + // Size returns the size required to encode the column's current values. + // + // The `rows` argument must be the current number of logical rows in the + // column. Some implementations support defaults, and these implementations + // rely on the caller to inform them the current number of logical rows. The + // provided `rows` must be greater than or equal to the largest row set + 1. + // In other words, Size does not support determining the size of a column's + // earlier size before additional rows were added. + Size(rows int, offset uint32) uint32 + // WriteDebug writes a human-readable description of the current column + // state to the provided writer. + WriteDebug(w io.Writer, rows int) +} diff --git a/sstable/colblk/testdata/bitmap b/sstable/colblk/testdata/bitmap new file mode 100644 index 0000000000..36ccb5fa32 --- /dev/null +++ b/sstable/colblk/testdata/bitmap @@ -0,0 +1,254 @@ +build +10101011100011100 +---- +10101011100011100 +Binary representation: +00-08: b 1101010101110001000000000000000000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +successor indexes=(0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16) +---- +bitmap.Successor(0) = 0 +bitmap.Successor(1) = 2 +bitmap.Successor(2) = 2 +bitmap.Successor(3) = 4 +bitmap.Successor(4) = 4 +bitmap.Successor(5) = 6 +bitmap.Successor(6) = 6 +bitmap.Successor(7) = 7 +bitmap.Successor(8) = 8 +bitmap.Successor(9) = 12 +bitmap.Successor(10) = 12 +bitmap.Successor(11) = 12 +bitmap.Successor(12) = 12 +bitmap.Successor(13) = 13 +bitmap.Successor(14) = 14 +bitmap.Successor(15) = 17 +bitmap.Successor(16) = 17 + +predecessor indexes=(0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16) +---- +bitmap.Predecessor(0) = 0 +bitmap.Predecessor(1) = 0 +bitmap.Predecessor(2) = 2 +bitmap.Predecessor(3) = 2 +bitmap.Predecessor(4) = 4 +bitmap.Predecessor(5) = 4 +bitmap.Predecessor(6) = 6 +bitmap.Predecessor(7) = 7 +bitmap.Predecessor(8) = 8 +bitmap.Predecessor(9) = 8 +bitmap.Predecessor(10) = 8 +bitmap.Predecessor(11) = 8 +bitmap.Predecessor(12) = 12 +bitmap.Predecessor(13) = 13 +bitmap.Predecessor(14) = 14 +bitmap.Predecessor(15) = 14 +bitmap.Predecessor(16) = 14 + +# Test calling Invert() before finishing. + +build invert +10101011100011100 +---- +01010100011100011 +Binary representation: +00-08: b 0010101010001110000000010000000000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + + +build +1 +---- +1 +Binary representation: +00-08: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +# Test sparseness; relying on the tailing bits being implicitly zeroed. + +build rows=512 +1 +---- +1000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +Binary representation: +00-08: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 1 +16-24: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 2 +24-32: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 3 +32-40: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 4 +40-48: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 5 +48-56: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 6 +56-64: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 7 +64-72: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +build invert +1 +---- +0 +Binary representation: +00-08: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +# Test sparseness with inversion, relying on the trailing bits being implicitly +# set to one. + +build invert rows=512 +1 +---- +0111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +Binary representation: +00-08: b 1111111011111111111111111111111111111111111111111111111111111111 # bitmap word 0 +08-16: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 1 +16-24: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 2 +24-32: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 3 +32-40: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 4 +40-48: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 5 +48-56: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 6 +56-64: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 7 +64-72: b 1111111100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +# 32-bits wide + +build +1010101010111111111110000001110 +---- +1010101010111111111110000001110 +Binary representation: +00-08: b 0101010111111101000111110011100000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +# 33-bits wide + +build +10101010101111111111100000011101 +---- +10101010101111111111100000011101 +Binary representation: +00-08: b 0101010111111101000111111011100000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +# 64-bits wide + +build +1010101010111111111110000001110101010101011111111111000000111011 +---- +1010101010111111111110000001110101010101011111111111000000111011 +Binary representation: +00-08: b 0101010111111101000111111011100010101010111111100000111111011100 # bitmap word 0 +08-16: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +# 63-bits wide + +build +101010101011111111111000000111010101010101111111111100000011101 +---- +101010101011111111111000000111010101010101111111111100000011101 +Binary representation: +00-08: b 0101010111111101000111111011100010101010111111100000111101011100 # bitmap word 0 +08-16: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +# 65-bits wide + +build +1010101010111111111110000001110101010101011111111111000000111011 +1 +---- +1010101010111111111110000001110101010101011111111111000000111011 +1 +Binary representation: +00-08: b 0101010111111101000111111011100010101010111111100000111111011100 # bitmap word 0 +08-16: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap word 1 +16-24: b 0000001100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +build +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +---- +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +1111111111111111111111111111111111111111111111111111111111111111 +Binary representation: +00-08: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 0 +08-16: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 1 +16-24: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 2 +24-32: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 3 +32-40: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 4 +40-48: b 0001111100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +build +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +---- +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +Binary representation: +00-08: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 1 +16-24: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 2 +24-32: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 3 +32-40: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 4 +40-48: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +build +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000001 +---- +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000001 +Binary representation: +00-08: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 1 +16-24: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 2 +24-32: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 3 +32-40: b 0000000000000000000000000000000000000000000000000000000010000000 # bitmap word 4 +40-48: b 0001000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 + +# Write out fewer rows than we set. The bitmap summary should reflect the +# truncated view of the bitmap. + +build rows=192 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000111000000000000000000000000000000000000000000000000000 +---- +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +0000000000000000000000000000000000000000000000000000000000000000 +Binary representation: +00-08: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 +08-16: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 1 +16-24: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 2 +24-32: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 diff --git a/sstable/colblk/unsafe_slice.go b/sstable/colblk/unsafe_slice.go new file mode 100644 index 0000000000..64241ffeec --- /dev/null +++ b/sstable/colblk/unsafe_slice.go @@ -0,0 +1,41 @@ +// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package colblk + +import ( + "unsafe" + + "github.com/cockroachdb/errors" + "golang.org/x/exp/constraints" +) + +// UnsafeRawSlice maintains a pointer to a slice of elements of type T. +// UnsafeRawSlice provides no bounds checking. +type UnsafeRawSlice[T constraints.Integer] struct { + ptr unsafe.Pointer +} + +func makeUnsafeRawSlice[T constraints.Integer](ptr unsafe.Pointer) UnsafeRawSlice[T] { + if align(uintptr(ptr), unsafe.Sizeof(T(0))) != uintptr(ptr) { + panic(errors.AssertionFailedf("slice pointer %p not %d-byte aligned", ptr, unsafe.Sizeof(T(0)))) + } + return UnsafeRawSlice[T]{ptr: ptr} +} + +// At returns the `i`-th element of the slice. +func (s UnsafeRawSlice[T]) At(i int) T { + return *(*T)(unsafe.Pointer(uintptr(s.ptr) + unsafe.Sizeof(T(0))*uintptr(i))) +} + +// Slice returns a go []T slice containing the first `len` elements of the +// unsafe slice. +func (s UnsafeRawSlice[T]) Slice(len int) []T { + return unsafe.Slice((*T)(s.ptr), len) +} + +// set mutates the slice, setting the `i`-th value to `v`. +func (s UnsafeRawSlice[T]) set(i int, v T) { + *(*T)(unsafe.Pointer(uintptr(s.ptr) + unsafe.Sizeof(T(0))*uintptr(i))) = v +}