diff --git a/internal/binfmt/binfmt.go b/internal/binfmt/binfmt.go index dc251247a4..2eded91789 100644 --- a/internal/binfmt/binfmt.go +++ b/internal/binfmt/binfmt.go @@ -18,7 +18,7 @@ import ( // New constructs a new binary formatter. func New(data []byte) *Formatter { - offsetWidth := strconv.Itoa(int(math.Log10(float64(len(data)-1))) + 1) + offsetWidth := strconv.Itoa(max(int(math.Log10(float64(len(data)-1)))+1, 1)) return &Formatter{ data: data, lineWidth: 40, diff --git a/sstable/colblk/block.go b/sstable/colblk/block.go index 4fa056d7c6..a70e4d6fc1 100644 --- a/sstable/colblk/block.go +++ b/sstable/colblk/block.go @@ -126,10 +126,10 @@ // [s,e) have the corresponding bounds. // // Both range deletions and range keys are encoded with the same schema. Range -// deletion keyspan.Keys never contain suffixes or values. When encoded, the -// RawBytes encoding uses the UintDeltaEncodingConstant encoding to avoid -// materializing encoding N offsets. Each of these empty columns is encoded in -// just ~5 bytes of column data. +// deletion keyspan.Keys never contain suffixes or values. When one of these +// columns is encoded, the RawBytes encoding uses uintEncodingAllZero to avoid +// encoding N offsets. Each of these empty columns is encoded in just 1 byte of +// column data. package colblk import ( @@ -357,28 +357,10 @@ func (r *BlockReader) PrefixBytes(col int) PrefixBytes { return DecodeColumn(r, col, int(r.header.Rows), DataTypePrefixBytes, DecodePrefixBytes) } -// Uint8s retrieves the col'th column as a column of uint8s. The column must be -// of type DataTypeUint8. -func (r *BlockReader) Uint8s(col int) UnsafeUint8s { - return DecodeColumn(r, col, int(r.header.Rows), DataTypeUint8, DecodeUnsafeIntegerSlice[uint8]) -} - -// Uint16s retrieves the col'th column as a column of uint8s. The column must be -// of type DataTypeUint16. -func (r *BlockReader) Uint16s(col int) UnsafeUint16s { - return DecodeColumn(r, col, int(r.header.Rows), DataTypeUint16, DecodeUnsafeIntegerSlice[uint16]) -} - -// Uint32s retrieves the col'th column as a column of uint32s. The column must be -// of type DataTypeUint32. -func (r *BlockReader) Uint32s(col int) UnsafeUint32s { - return DecodeColumn(r, col, int(r.header.Rows), DataTypeUint32, DecodeUnsafeIntegerSlice[uint32]) -} - -// Uint64s retrieves the col'th column as a column of uint64s. The column must be -// of type DataTypeUint64. -func (r *BlockReader) Uint64s(col int) UnsafeUint64s { - return DecodeColumn(r, col, int(r.header.Rows), DataTypeUint64, DecodeUnsafeIntegerSlice[uint64]) +// Uints retrieves the col'th column as a column of uints. The column must be +// of type DataTypeUint. +func (r *BlockReader) Uints(col int) UnsafeUints { + return DecodeColumn(r, col, int(r.header.Rows), DataTypeUint, DecodeUnsafeUints) } func (r *BlockReader) pageStart(col int) uint32 { @@ -387,7 +369,7 @@ func (r *BlockReader) pageStart(col int) uint32 { return uint32(len(r.data) - 1) } return binary.LittleEndian.Uint32( - unsafe.Slice((*byte)(unsafe.Pointer(r.pointer(r.customHeaderSize+uint32(blockHeaderBaseSize+columnHeaderSize*col+1)))), 4)) + unsafe.Slice((*byte)(r.pointer(r.customHeaderSize+uint32(blockHeaderBaseSize+columnHeaderSize*col+1))), 4)) } func (r *BlockReader) pointer(offset uint32) unsafe.Pointer { @@ -443,8 +425,8 @@ func (r *BlockReader) columnToBinFormatter(f *binfmt.Formatter, col, rows int) { switch dataType { case DataTypeBool: bitmapToBinFormatter(f, rows) - case DataTypeUint8, DataTypeUint16, DataTypeUint32, DataTypeUint64: - uintsToBinFormatter(f, rows, dataType, nil) + case DataTypeUint: + uintsToBinFormatter(f, rows, nil) case DataTypePrefixBytes: prefixBytesToBinFormatter(f, rows, nil) case DataTypeBytes: diff --git a/sstable/colblk/block_test.go b/sstable/colblk/block_test.go index 24fd8b0bfa..03ec588d9c 100644 --- a/sstable/colblk/block_test.go +++ b/sstable/colblk/block_test.go @@ -7,6 +7,7 @@ package colblk import ( "bytes" "fmt" + "math" "reflect" "slices" "strconv" @@ -18,9 +19,58 @@ import ( "golang.org/x/exp/rand" ) -type ColumnSpec struct { +type testColumnSpec struct { DataType - BundleSize int // Only used for DataTypePrefixBytes + IntRange intRange // Only used for DataTypeUint + BundleSize int // Only used for DataTypePrefixBytes +} + +type intRange struct { + Min, Max uint64 + ExpectedEncoding UintEncoding +} + +func (ir intRange) Rand(rng *rand.Rand) uint64 { + v := rng.Uint64() + if ir.Min == 0 && ir.Max == math.MaxUint64 { + return v + } + return ir.Min + v%(ir.Max-ir.Min+1) +} + +var interestingIntRanges = []intRange{ + // zero + {Min: 0, Max: 0, ExpectedEncoding: makeUintEncoding(0, false)}, + // const + {Min: 1, Max: 1, ExpectedEncoding: makeUintEncoding(0, true)}, + {Min: math.MaxUint32, Max: math.MaxUint32, ExpectedEncoding: makeUintEncoding(0, true)}, + {Min: math.MaxUint64, Max: math.MaxUint64, ExpectedEncoding: makeUintEncoding(0, true)}, + // 1b + {Min: 10, Max: 200, ExpectedEncoding: makeUintEncoding(1, false)}, + {Min: 0, Max: math.MaxUint8, ExpectedEncoding: makeUintEncoding(1, false)}, + // 1b,delta + {Min: 100, Max: 300, ExpectedEncoding: makeUintEncoding(1, true)}, + {Min: math.MaxUint32 + 100, Max: math.MaxUint32 + 300, ExpectedEncoding: makeUintEncoding(1, true)}, + {Min: math.MaxUint64 - 1, Max: math.MaxUint64, ExpectedEncoding: makeUintEncoding(1, true)}, + // 2b + {Min: 10, Max: 20_000, ExpectedEncoding: makeUintEncoding(2, false)}, + {Min: 0, Max: math.MaxUint8 + 1, ExpectedEncoding: makeUintEncoding(2, false)}, + {Min: 0, Max: math.MaxUint16, ExpectedEncoding: makeUintEncoding(2, false)}, + // 2b,delta + {Min: 20_000, Max: 80_000, ExpectedEncoding: makeUintEncoding(2, true)}, + {Min: math.MaxUint32, Max: math.MaxUint32 + 50_000, ExpectedEncoding: makeUintEncoding(2, true)}, + // 4b + {Min: 10, Max: 20_000_000, ExpectedEncoding: makeUintEncoding(4, false)}, + {Min: 0, Max: math.MaxUint16 + 1, ExpectedEncoding: makeUintEncoding(4, false)}, + {Min: 0, Max: math.MaxUint32, ExpectedEncoding: makeUintEncoding(4, false)}, + // 4b,delta + {Min: 100_000, Max: math.MaxUint32 + 10, ExpectedEncoding: makeUintEncoding(4, true)}, + {Min: math.MaxUint32, Max: math.MaxUint32 + 20_000_000, ExpectedEncoding: makeUintEncoding(4, true)}, + // 8b + {Min: 10, Max: math.MaxUint32 + 100, ExpectedEncoding: makeUintEncoding(8, false)}, + {Min: 0, Max: math.MaxUint32 + 1, ExpectedEncoding: makeUintEncoding(8, false)}, + {Min: 0, Max: math.MaxUint64, ExpectedEncoding: makeUintEncoding(8, false)}, + {Min: math.MaxUint64 - math.MaxUint32 - 1, Max: math.MaxUint64, ExpectedEncoding: makeUintEncoding(8, false)}, } func TestBlockWriter(t *testing.T) { @@ -48,20 +98,8 @@ func TestBlockWriter(t *testing.T) { switch colDataTypes[i] { case DataTypeBool: colWriters[i] = &BitmapBuilder{} - case DataTypeUint8: - b := &UintBuilder[uint8]{} - b.Init() - colWriters[i] = b - case DataTypeUint16: - b := &UintBuilder[uint16]{} - b.Init() - colWriters[i] = b - case DataTypeUint32: - b := &UintBuilder[uint32]{} - b.Init() - colWriters[i] = b - case DataTypeUint64: - b := &UintBuilder[uint64]{} + case DataTypeUint: + b := &UintBuilder{} b.Init() colWriters[i] = b case DataTypeBytes: @@ -93,29 +131,8 @@ func TestBlockWriter(t *testing.T) { panicIfErr(dataType, lineFields[r][c], err) bb.Set(r, v) } - case DataTypeUint8: - b := colWriters[c].(*UintBuilder[uint8]) - for r := range lineFields { - v, err := strconv.ParseUint(lineFields[r][c], 10, 8) - panicIfErr(dataType, lineFields[r][c], err) - b.Set(r, uint8(v)) - } - case DataTypeUint16: - b := colWriters[c].(*UintBuilder[uint16]) - for r := range lineFields { - v, err := strconv.ParseUint(lineFields[r][c], 10, 16) - panicIfErr(dataType, lineFields[r][c], err) - b.Set(r, uint16(v)) - } - case DataTypeUint32: - b := colWriters[c].(*UintBuilder[uint32]) - for r := range lineFields { - v, err := strconv.ParseUint(lineFields[r][c], 10, 32) - panicIfErr(dataType, lineFields[r][c], err) - b.Set(r, uint32(v)) - } - case DataTypeUint64: - b := colWriters[c].(*UintBuilder[uint64]) + case DataTypeUint: + b := colWriters[c].(*UintBuilder) for r := range lineFields { v, err := strconv.ParseUint(lineFields[r][c], 10, 64) panicIfErr(dataType, lineFields[r][c], err) @@ -156,7 +173,7 @@ func dataTypeFromName(name string) DataType { // returns the serialized raw block and a []interface{} slice containing the // generated data. The type of each element of the slice is dependent on the // corresponding column's type. -func randBlock(rng *rand.Rand, rows int, schema []ColumnSpec) ([]byte, []interface{}) { +func randBlock(rng *rand.Rand, rows int, schema []testColumnSpec) ([]byte, []interface{}) { data := make([]interface{}, len(schema)) for col := range data { switch schema[col].DataType { @@ -166,28 +183,10 @@ func randBlock(rng *rand.Rand, rows int, schema []ColumnSpec) ([]byte, []interfa v[row] = (rng.Int31() % 2) == 0 } data[col] = v - case DataTypeUint8: - v := make([]uint8, rows) - for row := 0; row < rows; row++ { - v[row] = uint8(rng.Uint32()) - } - data[col] = v - case DataTypeUint16: - v := make([]uint16, rows) - for row := 0; row < rows; row++ { - v[row] = uint16(rng.Uint32()) - } - data[col] = v - case DataTypeUint32: - v := make([]uint32, rows) - for row := 0; row < rows; row++ { - v[row] = rng.Uint32() - } - data[col] = v - case DataTypeUint64: + case DataTypeUint: v := make([]uint64, rows) for row := 0; row < rows; row++ { - v[row] = rng.Uint64() + v[row] = schema[col].IntRange.Rand(rng) } data[col] = v case DataTypeBytes: @@ -213,7 +212,7 @@ func randBlock(rng *rand.Rand, rows int, schema []ColumnSpec) ([]byte, []interfa return buf, data } -func buildBlock(schema []ColumnSpec, rows int, data []interface{}) []byte { +func buildBlock(schema []testColumnSpec, rows int, data []interface{}) []byte { cw := make([]ColumnWriter, len(schema)) for col := range schema { switch schema[col].DataType { @@ -224,29 +223,8 @@ func buildBlock(schema []ColumnSpec, rows int, data []interface{}) []byte { bb.Set(row, v) } cw[col] = &bb - case DataTypeUint8: - var b UintBuilder[uint8] - b.Init() - for row, v := range data[col].([]uint8) { - b.Set(row, v) - } - cw[col] = &b - case DataTypeUint16: - var b UintBuilder[uint16] - b.Init() - for row, v := range data[col].([]uint16) { - b.Set(row, v) - } - cw[col] = &b - case DataTypeUint32: - var b UintBuilder[uint32] - b.Init() - for row, v := range data[col].([]uint32) { - b.Set(row, v) - } - cw[col] = &b - case DataTypeUint64: - var b UintBuilder[uint64] + case DataTypeUint: + var b UintBuilder b.Init() for row, v := range data[col].([]uint64) { b.Set(row, v) @@ -276,7 +254,7 @@ func buildBlock(schema []ColumnSpec, rows int, data []interface{}) []byte { return FinishBlock(rows, cw) } -func testRandomBlock(t *testing.T, rng *rand.Rand, rows int, schema []ColumnSpec) { +func testRandomBlock(t *testing.T, rng *rand.Rand, rows int, schema []testColumnSpec) { var sb strings.Builder for i := range schema { if i > 0 { @@ -306,14 +284,8 @@ func testRandomBlock(t *testing.T, rng *rand.Rand, rows int, schema []ColumnSpec switch spec.DataType { case DataTypeBool: got = Clone(r.Bitmap(col), rows) - case DataTypeUint8: - got = Clone(r.Uint8s(col), rows) - case DataTypeUint16: - got = Clone(r.Uint16s(col), rows) - case DataTypeUint32: - got = Clone(r.Uint32s(col), rows) - case DataTypeUint64: - got = Clone(r.Uint64s(col), rows) + case DataTypeUint: + got = Clone(r.Uints(col), rows) case DataTypeBytes: got = Clone(r.RawBytes(col), rows) case DataTypePrefixBytes: @@ -334,16 +306,15 @@ func TestBlockWriterRandomized(t *testing.T) { randInt := func(lo, hi int) int { return lo + rng.Intn(hi-lo) } - testRandomBlock(t, rng, randInt(1, 100), []ColumnSpec{{DataType: DataTypeBool}}) - testRandomBlock(t, rng, randInt(1, 100), []ColumnSpec{{DataType: DataTypeUint8}}) - testRandomBlock(t, rng, randInt(1, 100), []ColumnSpec{{DataType: DataTypeUint16}}) - testRandomBlock(t, rng, randInt(1, 100), []ColumnSpec{{DataType: DataTypeUint32}}) - testRandomBlock(t, rng, randInt(1, 100), []ColumnSpec{{DataType: DataTypeUint64}}) - testRandomBlock(t, rng, randInt(1, 100), []ColumnSpec{{DataType: DataTypeBytes}}) - testRandomBlock(t, rng, randInt(1, 100), []ColumnSpec{{DataType: DataTypePrefixBytes, BundleSize: 1 << randInt(0, 6)}}) + testRandomBlock(t, rng, randInt(1, 100), []testColumnSpec{{DataType: DataTypeBool}}) + for _, r := range interestingIntRanges { + testRandomBlock(t, rng, randInt(1, 100), []testColumnSpec{{DataType: DataTypeUint, IntRange: r}}) + } + testRandomBlock(t, rng, randInt(1, 100), []testColumnSpec{{DataType: DataTypeBytes}}) + testRandomBlock(t, rng, randInt(1, 100), []testColumnSpec{{DataType: DataTypePrefixBytes, BundleSize: 1 << randInt(0, 6)}}) for i := 0; i < 100; i++ { - schema := make([]ColumnSpec, 2+rng.Intn(8)) + schema := make([]testColumnSpec, 2+rng.Intn(8)) for j := range schema { schema[j].DataType = DataType(randInt(1, int(dataTypesCount))) if schema[j].DataType == DataTypePrefixBytes { diff --git a/sstable/colblk/cockroach_test.go b/sstable/colblk/cockroach_test.go index b0bfb7b51a..1ef27a003d 100644 --- a/sstable/colblk/cockroach_test.go +++ b/sstable/colblk/cockroach_test.go @@ -33,8 +33,8 @@ const ( var cockroachKeySchema = KeySchema{ ColumnTypes: []DataType{ cockroachColPrefix: DataTypePrefixBytes, - cockroachColMVCCWallTime: DataTypeUint64, - cockroachColMVCCLogical: DataTypeUint32, + cockroachColMVCCWallTime: DataTypeUint, + cockroachColMVCCLogical: DataTypeUint, cockroachColUntypedSuffix: DataTypeBytes, }, NewKeyWriter: func() KeyWriter { @@ -52,8 +52,8 @@ var cockroachKeySchema = KeySchema{ type cockroachKeyWriter struct { prefixes PrefixBytesBuilder - wallTimes UintBuilder[uint64] - logicalTimes UintBuilder[uint32] + wallTimes UintBuilder + logicalTimes UintBuilder untypedSuffixes RawBytesBuilder prevSuffix []byte } @@ -94,7 +94,7 @@ func (kw *cockroachKeyWriter) WriteKey( // don't set a value, the column value is implicitly zero. We only need to // Set anything for non-zero values. if logicalTime > 0 { - kw.logicalTimes.Set(row, logicalTime) + kw.logicalTimes.Set(row, uint64(logicalTime)) } kw.untypedSuffixes.Put(untypedSuffix) } @@ -161,8 +161,8 @@ var cockroachKeySeekerPool = sync.Pool{ type cockroachKeySeeker struct { reader *DataBlockReader prefixes PrefixBytes - mvccWallTimes UnsafeUint64s - mvccLogical UnsafeUint32s + mvccWallTimes UnsafeUints + mvccLogical UnsafeUints untypedSuffixes RawBytes sharedPrefix []byte } @@ -170,8 +170,8 @@ type cockroachKeySeeker struct { func (ks *cockroachKeySeeker) Init(r *DataBlockReader) error { ks.reader = r ks.prefixes = r.r.PrefixBytes(cockroachColPrefix) - ks.mvccWallTimes = r.r.Uint64s(cockroachColMVCCWallTime) - ks.mvccLogical = r.r.Uint32s(cockroachColMVCCLogical) + ks.mvccWallTimes = r.r.Uints(cockroachColMVCCWallTime) + ks.mvccLogical = r.r.Uints(cockroachColMVCCLogical) ks.untypedSuffixes = r.r.RawBytes(cockroachColUntypedSuffix) ks.sharedPrefix = ks.prefixes.SharedPrefix() return nil @@ -248,7 +248,7 @@ func (ks *cockroachKeySeeker) seekGEOnSuffix(index int, seekSuffix []byte) (row case +1: u = h // preserves f(u) == true } - if cmp.Compare(ks.mvccLogical.At(h), seekLogicalTime) >= 0 { + if cmp.Compare(uint32(ks.mvccLogical.At(h)), seekLogicalTime) >= 0 { u = h // preserves f(u) == true } else { l = h + 1 // preserves f(l-1) == false @@ -265,7 +265,7 @@ func (ks *cockroachKeySeeker) MaterializeUserKey(ki *PrefixBytesIter, prevRow, r } mvccWall := ks.mvccWallTimes.At(row) - mvccLogical := ks.mvccLogical.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) diff --git a/sstable/colblk/column.go b/sstable/colblk/column.go index 9f28ac2c78..9267e18eb8 100644 --- a/sstable/colblk/column.go +++ b/sstable/colblk/column.go @@ -17,31 +17,22 @@ const ( 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 + // DataTypeUint is a data type encoding a fixed 8 bits per row. + DataTypeUint DataType = 2 // DataTypeBytes is a data type encoding a variable-length byte string per // row. - DataTypeBytes DataType = 6 + DataTypeBytes DataType = 3 // DataTypePrefixBytes is a data type encoding variable-length, // lexicographically-sorted byte strings, with prefix compression. - DataTypePrefixBytes DataType = 7 + DataTypePrefixBytes DataType = 4 - dataTypesCount DataType = 8 + dataTypesCount DataType = 5 ) var dataTypeName [dataTypesCount]string = [dataTypesCount]string{ DataTypeInvalid: "invalid", DataTypeBool: "bool", - DataTypeUint8: "uint8", - DataTypeUint16: "uint16", - DataTypeUint32: "uint32", - DataTypeUint64: "uint64", + DataTypeUint: "uint", DataTypeBytes: "bytes", DataTypePrefixBytes: "prefixbytes", } @@ -51,17 +42,6 @@ func (t DataType) String() string { return dataTypeName[t] } -func (t DataType) uintWidth() uint32 { - if t >= DataTypeUint8 && t <= DataTypeUint64 { - rv := 1 << (t - DataTypeUint8) - if rv > 8 { - panic("width greater than 8 bytes") - } - return uint32(rv) - } - panic("not a unit") -} - // ColumnWriter is an interface implemented by column encoders that accumulate a // column's values and then serialize them. type ColumnWriter interface { diff --git a/sstable/colblk/data_block.go b/sstable/colblk/data_block.go index 882fcb7a53..49d1a90447 100644 --- a/sstable/colblk/data_block.go +++ b/sstable/colblk/data_block.go @@ -338,7 +338,7 @@ type DataBlockWriter struct { Schema KeySchema KeyWriter KeyWriter // trailers is the column writer for InternalKey uint64 trailers. - trailers UintBuilder[uint64] + trailers UintBuilder // prefixSame is the column writer for the prefix-changed bitmap that // indicates when a new key prefix begins. During block building, the bitmap // represents when the prefix stays the same, which is expected to be a @@ -514,7 +514,7 @@ type DataBlockReader struct { r BlockReader // trailers holds an array of the InternalKey trailers, encoding the key // kind and sequence number of each key. - trailers UnsafeUint64s + trailers UnsafeUints // prefixChanged is a bitmap indicating when the prefix (as defined by // Split) of a key changes, relative to the preceding key. This is used to // bound seeks within a prefix, and to optimize NextPrefix. @@ -542,7 +542,7 @@ func (r *DataBlockReader) BlockReader() *BlockReader { // Init initializes the data block reader with the given serialized data block. func (r *DataBlockReader) Init(schema KeySchema, data []byte) { r.r.Init(data, dataBlockCustomHeaderSize) - r.trailers = r.r.Uint64s(len(schema.ColumnTypes) + dataBlockColumnTrailer) + r.trailers = r.r.Uints(len(schema.ColumnTypes) + dataBlockColumnTrailer) r.prefixChanged = r.r.Bitmap(len(schema.ColumnTypes) + dataBlockColumnPrefixChanged) r.values = r.r.RawBytes(len(schema.ColumnTypes) + dataBlockColumnValue) r.isValueExternal = r.r.Bitmap(len(schema.ColumnTypes) + dataBlockColumnIsValueExternal) diff --git a/sstable/colblk/keyspan.go b/sstable/colblk/keyspan.go index c21bad7765..b8ebfdd48d 100644 --- a/sstable/colblk/keyspan.go +++ b/sstable/colblk/keyspan.go @@ -43,10 +43,10 @@ type KeyspanBlockWriter struct { // boundary columns boundaryUserKeys RawBytesBuilder - boundaryKeyIndexes UintBuilder[uint32] + boundaryKeyIndexes UintBuilder // keyspan.Key columns - trailers UintBuilder[uint64] + trailers UintBuilder suffixes RawBytesBuilder values RawBytesBuilder @@ -87,13 +87,13 @@ func (w *KeyspanBlockWriter) AddSpan(s *keyspan.Span) { // end key is the next span's start key. Check if the previous user key // equals this span's start key, and avoid encoding it again if so. if w.unsafeLastUserKey == nil || !w.equal(w.unsafeLastUserKey, s.Start) { - w.boundaryKeyIndexes.Set(w.boundaryUserKeys.rows, uint32(w.keyCount)) + w.boundaryKeyIndexes.Set(w.boundaryUserKeys.rows, uint64(w.keyCount)) w.boundaryUserKeys.Put(s.Start) } // The end key must be strictly greater than the start key and spans are // already sorted, so the end key is guaranteed to not be present in the // column yet. We need to encode it. - w.boundaryKeyIndexes.Set(w.boundaryUserKeys.rows, uint32(w.keyCount+len(s.Keys))) + w.boundaryKeyIndexes.Set(w.boundaryUserKeys.rows, uint64(w.keyCount+len(s.Keys))) w.boundaryUserKeys.Put(s.End) // Hold on to a slice of the copy of s.End we just added to the bytes @@ -182,10 +182,10 @@ type KeyspanReader struct { // Span boundary columns with boundaryKeysCount elements. boundaryKeysCount uint32 boundaryKeys RawBytes - boundaryKeyIndices UnsafeUint32s + boundaryKeyIndices UnsafeUints // keyspan.Key columns with blockReader.header.Rows elements. - trailers UnsafeUint64s + trailers UnsafeUints suffixes RawBytes values RawBytes } @@ -200,9 +200,9 @@ func (r *KeyspanReader) Init(data []byte) { r.boundaryKeys = DecodeColumn(&r.blockReader, keyspanColBoundaryUserKeys, int(r.boundaryKeysCount), DataTypeBytes, DecodeRawBytes) r.boundaryKeyIndices = DecodeColumn(&r.blockReader, keyspanColBoundaryKeyIndices, - int(r.boundaryKeysCount), DataTypeUint32, DecodeUnsafeIntegerSlice[uint32]) + int(r.boundaryKeysCount), DataTypeUint, DecodeUnsafeUints) - r.trailers = r.blockReader.Uint64s(keyspanColTrailers) + r.trailers = r.blockReader.Uints(keyspanColTrailers) r.suffixes = r.blockReader.RawBytes(keyspanColSuffixes) r.values = r.blockReader.RawBytes(keyspanColValues) } diff --git a/sstable/colblk/prefix_bytes.go b/sstable/colblk/prefix_bytes.go index e9bc44470d..cae878b31b 100644 --- a/sstable/colblk/prefix_bytes.go +++ b/sstable/colblk/prefix_bytes.go @@ -220,10 +220,6 @@ func DecodePrefixBytes( rows: count, rawBytes: rb, } - // We always set the base to zero. - if pb.rawBytes.offsets.base != 0 { - panic(errors.AssertionFailedf("unexpected non-zero base in offsets")) - } pb.sharedPrefixLen = int(pb.rawBytes.offsets.At(0)) return pb, endOffset } @@ -583,7 +579,7 @@ func prefixBytesToBinFormatter(f *binfmt.Formatter, count int, sliceFormatter fu f.HexBytesln(1, "bundleSize: %d", 1<= blockPrefixLen - currentBundlePrefixLen int // the length of the "current" bundle's prefix - currentBundlePrefixOffset int // the index of the offset of the "current" bundle's prefix - completedBundleLen int // the encoded size of completed bundles - compressedDataLen int // the compressed, encoded size of data - offsetDeltaWidth uint32 // the offset width necessary to encode the max offset + currentBundlePrefixLen int // the length of the "current" bundle's prefix + currentBundlePrefixOffset int // the index of the offset of the "current" bundle's prefix + completedBundleLen int // the encoded size of completed bundles + compressedDataLen int // the compressed, encoded size of data + offsetEncoding UintEncoding // the encoding necessary to encode the offsets } func (sz *prefixBytesSizing) String() string { return fmt.Sprintf("lastKeyLen:%d offsetCount:%d blockPrefixLen:%d\n"+ "currentBundleDistinct{Len,Keys}: (%d,%d)\n"+ "currentBundlePrefix{Len,Offset}: (%d,%d)\n"+ - "completedBundleLen:%d compressedDataLen:%d offsetDeltaWidth:%d", + "completedBundleLen:%d compressedDataLen:%d offsetEncoding:%s", sz.lastKeyLen, sz.offsetCount, sz.blockPrefixLen, sz.currentBundleDistinctLen, sz.currentBundleDistinctKeys, sz.currentBundlePrefixLen, sz.currentBundlePrefixOffset, - sz.completedBundleLen, sz.compressedDataLen, sz.offsetDeltaWidth) + sz.completedBundleLen, sz.compressedDataLen, sz.offsetEncoding) } // Put adds the provided key to the column. The provided key must be @@ -788,7 +784,7 @@ func (b *PrefixBytesBuilder) Put(key []byte, bytesSharedWithPrev int) { currentBundlePrefixOffset: 1, completedBundleLen: 0, compressedDataLen: len(key), - offsetDeltaWidth: max(1, deltaWidth(uint64(len(key)))), + offsetEncoding: DetermineUintEncoding(0, uint64(len(key))), } case b.nKeys&(b.bundleSize-1) == 0: // We're starting a new bundle. @@ -821,7 +817,7 @@ func (b *PrefixBytesBuilder) Put(key []byte, bytesSharedWithPrev int) { currentBundleDistinctKeys: 1, compressedDataLen: completedBundleSize + len(key) - (b.bundleCount(b.nKeys)-1)*blockPrefixLen, } - curr.offsetDeltaWidth = max(1, deltaWidth(uint64(curr.compressedDataLen))) + curr.offsetEncoding = DetermineUintEncoding(0, uint64(curr.compressedDataLen)) b.data = append(b.data, key...) b.addOffset(0) // Placeholder for bundle prefix. b.addOffset(uint32(len(b.data))) @@ -862,8 +858,8 @@ func (b *PrefixBytesBuilder) Put(key []byte, bytesSharedWithPrev int) { // the block prefix being stripped from every bundle except the first one. curr.compressedDataLen -= (b.bundleCount(b.nKeys) - 1) * curr.blockPrefixLen // The compressedDataLen is the largest offset we'll need to encode in the - // offset table, so we can use it to compute the width of the offset deltas. - curr.offsetDeltaWidth = max(1, deltaWidth(uint64(curr.compressedDataLen))) + // offset table. + curr.offsetEncoding = DetermineUintEncoding(0, uint64(curr.compressedDataLen)) b.data = append(b.data, key...) b.addOffset(uint32(len(b.data))) } @@ -1001,35 +997,23 @@ func (b *PrefixBytesBuilder) Finish( offset++ sz := &b.sizings[(rows+1)%2] - stringDataOffset := uintColumnSize[uint32](uint32(sz.offsetCount), offset, sz.offsetDeltaWidth) - switch sz.offsetDeltaWidth { - case 1: - buf[offset] = byte(UintDeltaEncoding8) - offset++ - // The uint32 delta-encoding requires a uint32 constant representing the - // base value. This is always zero for PrefixBytes, but we include it - // anyways for format consistency. - buf[offset], buf[offset+1], buf[offset+2], buf[offset+3] = 0, 0, 0, 0 - offset += align32 + stringDataOffset := uintColumnSize(uint32(sz.offsetCount), offset, sz.offsetEncoding) + if sz.offsetEncoding.IsDelta() { + panic(errors.AssertionFailedf("offsets never need delta encoding")) + } + width := uint32(sz.offsetEncoding.Width()) + buf[offset] = byte(sz.offsetEncoding) + offset++ + offset = alignWithZeroes(buf, offset, width) + switch width { + case 1: offsetDest := makeUnsafeRawSlice[uint8](unsafe.Pointer(&buf[offset])) writePrefixCompressed[uint8](b, rows, sz, offsetDest, buf[stringDataOffset:]) case align16: - buf[offset] = byte(UintDeltaEncoding16) - offset++ - // The uint32 delta-encoding requires a uint32 constant representing the - // base value. This is always zero for PrefixBytes, but we include it - // anyways for format consistency. - buf[offset], buf[offset+1], buf[offset+2], buf[offset+3] = 0, 0, 0, 0 - offset += align32 - - offset = alignWithZeroes(buf, offset, align16) offsetDest := makeUnsafeRawSlice[uint16](unsafe.Pointer(&buf[offset])) writePrefixCompressed[uint16](b, rows, sz, offsetDest, buf[stringDataOffset:]) case align32: - buf[offset] = byte(UintDeltaEncodingNone) - offset++ - offset = alignWithZeroes(buf, offset, align32) offsetDest := makeUnsafeRawSlice[uint32](unsafe.Pointer(&buf[offset])) writePrefixCompressed[uint32](b, rows, sz, offsetDest, buf[stringDataOffset:]) default: @@ -1053,7 +1037,7 @@ func (b *PrefixBytesBuilder) Size(rows int, offset uint32) uint32 { // The 1-byte bundleSize. offset++ // Compute the size of the offsets table. - offset = uintColumnSize[uint32](uint32(sz.offsetCount), offset, sz.offsetDeltaWidth) + offset = uintColumnSize(uint32(sz.offsetCount), offset, sz.offsetEncoding) return offset + uint32(sz.compressedDataLen) } diff --git a/sstable/colblk/raw_bytes.go b/sstable/colblk/raw_bytes.go index a7c9e19725..a8500e7c00 100644 --- a/sstable/colblk/raw_bytes.go +++ b/sstable/colblk/raw_bytes.go @@ -22,30 +22,31 @@ import ( // // An array of N byte slices encodes N+1 offsets. The beginning of the data // representation holds an offsets table, in the same encoding as a -// DataTypeUint32 column. The Uint32 offsets may be delta-encoded to save space -// if all offsets fit within an 8-bit or 16-bit uint. Each offset is relative to -// the beginning of the string data section (after the offset table). +// DataTypeUint32 column. The integer offsets may be encoded using smaller width +// integers to save space if all offsets fit within an 8-bit or 16-bit uint. +// Each offset is relative to the beginning of the string data section (after +// the offset table). // -// The use of delta encoding conserves space in the common case. In the context -// of CockroachDB, the vast majority of offsets will fit in 16-bits when using -// 32 KiB blocks (the size in use by CockroachDB). However a single value larger +// The use of UintEncoding conserves space in the common case. In the context of +// CockroachDB, the vast majority of offsets will fit in 16-bits when using 32 +// KiB blocks (the size in use by CockroachDB). However, a single value larger // than 65535 bytes requires an offset too large to fit within 16 bits, in which // case offsets will be encoded as 32-bit integers. // // +-------------------------------------------------------------------+ -// | a uint32 offsets table, possibly delta encoded, | -// | possibly padded for 32-bit alignment | -// | (see DeltaEncoding) | +// | a uint offsets table, usually encoded with 16-bits, | +// | possibly padded for alignment | +// | (see UintEncoding) | // +-------------------------------------------------------------------+ // | String Data | // | abcabcada.... | // +-------------------------------------------------------------------+ // -// The DeltaEncoding bits of the ColumnEncoding for a RawBytes column describes -// the delta encoding of the offset table. +// The UintEncoding bits of the ColumnEncoding for a RawBytes column describes +// the encoding of the offset table. type RawBytes struct { slices int - offsets UnsafeUint32s + offsets UnsafeOffsets start unsafe.Pointer data unsafe.Pointer } @@ -60,7 +61,7 @@ func DecodeRawBytes(b []byte, offset uint32, count int) (rawBytes RawBytes, endO if count == 0 { return RawBytes{}, 0 } - offsets, dataOff := DecodeUnsafeIntegerSlice[uint32](b, offset, count+1 /* +1 offset */) + offsets, dataOff := DecodeUnsafeOffsets(b, offset, count+1 /* +1 offset */) return RawBytes{ slices: count, offsets: offsets, @@ -88,7 +89,7 @@ func rawBytesToBinFormatter(f *binfmt.Formatter, count int, sliceFormatter func( dataOffset := uint64(f.Offset()) + uint64(uintptr(rb.data)-uintptr(rb.start)) f.CommentLine("rawbytes") f.CommentLine("offsets table") - uintsToBinFormatter(f, count+1, DataTypeUint32, func(offset, base uint64) string { + uintsToBinFormatter(f, count+1, func(offset, base uint64) string { // NB: base is always zero for RawBytes columns. return fmt.Sprintf("%d [%d overall]", offset+base, offset+base+dataOffset) }) @@ -121,7 +122,7 @@ func (b *RawBytes) Slices() int { type RawBytesBuilder struct { rows int data []byte - offsets UintBuilder[uint32] + offsets UintBuilder } // Assert that *RawBytesBuilder implements ColumnWriter. @@ -153,7 +154,7 @@ func (b *RawBytesBuilder) DataType(int) DataType { return DataTypeBytes } func (b *RawBytesBuilder) Put(s []byte) { b.data = append(b.data, s...) b.rows++ - b.offsets.Set(b.rows, uint32(len(b.data))) + b.offsets.Set(b.rows, uint64(len(b.data))) } // PutConcat appends a single byte slice formed by the concatenation of the two @@ -161,7 +162,7 @@ func (b *RawBytesBuilder) Put(s []byte) { func (b *RawBytesBuilder) PutConcat(s1, s2 []byte) { b.data = append(append(b.data, s1...), s2...) b.rows++ - b.offsets.Set(b.rows, uint32(len(b.data))) + b.offsets.Set(b.rows, uint64(len(b.data))) } // LastSlice returns the last slice added to the builder. The returned slice is @@ -199,7 +200,7 @@ func (b *RawBytesBuilder) Size(rows int, offset uint32) uint32 { offset = b.offsets.Size(rows+1, offset) // Add the value of offset[rows] since that is the accumulated size of the // first [rows] slices. - return offset + b.offsets.Get(rows) + return offset + uint32(b.offsets.Get(rows)) } // WriteDebug implements Encoder. diff --git a/sstable/colblk/testdata/block_writer b/sstable/colblk/testdata/block_writer index 07026b0cac..f86c5e7933 100644 --- a/sstable/colblk/testdata/block_writer +++ b/sstable/colblk/testdata/block_writer @@ -1,4 +1,4 @@ -init schema=(uint64) +init schema=(uint) ---- write @@ -17,20 +17,19 @@ write finish ---- # columnar block header -00-01: x 01 # version 1 -01-03: x 0100 # 1 columns -03-07: x 0a000000 # 10 rows +00-01: x 01 # version 1 +01-03: x 0100 # 1 columns +03-07: x 0a000000 # 10 rows # column 0 -07-08: b 00000101 # uint64 -08-12: x 0c000000 # page start 12 +07-08: b 00000010 # uint +08-12: x 0c000000 # page start 12 # data for column 0 -12-13: x 01 # delta encoding: const -13-21: x 0000000000000000 # 64-bit constant: 0 -21-22: x 00 # block trailer padding +12-13: x 00 # encoding: zero +13-14: x 00 # block trailer padding -# Test a uint64 column with all values equal but non-zero. +# Test a uint column with all values equal but non-zero. -init schema=(uint64) +init schema=(uint) ---- write @@ -55,17 +54,17 @@ finish 01-03: x 0100 # 1 columns 03-07: x 0c000000 # 12 rows # column 0 -07-08: b 00000101 # uint64 +07-08: b 00000010 # uint 08-12: x 0c000000 # page start 12 # data for column 0 -12-13: x 01 # delta encoding: const +12-13: x 80 # encoding: const 13-21: x ffffffffffffff7f # 64-bit constant: 9223372036854775807 21-22: x 00 # block trailer padding -# Test a uint64 column with a mix of values, but all values less than 256 +# Test a uint column with a mix of values, but all values less than 256 # greater than 4149660732785475243. It should use the delta8 encoding. -init schema=(uint64) +init schema=(uint) ---- write @@ -83,10 +82,10 @@ finish 01-03: x 0100 # 1 columns 03-07: x 05000000 # 5 rows # column 0 -07-08: b 00000101 # uint64 +07-08: b 00000010 # uint 08-12: x 0c000000 # page start 12 # data for column 0 -12-13: x 02 # delta encoding: delta8 +12-13: x 81 # encoding: 1b,delta 13-21: x abbe105c738e9639 # 64-bit constant: 4149660732785475243 21-22: x 01 # data[0] = 1 + 4149660732785475243 = 4149660732785475244 22-23: x 00 # data[1] = 0 + 4149660732785475243 = 4149660732785475243 @@ -98,7 +97,7 @@ finish # Test the same case, but this time with a value that is exactly 256 greater # than the lowest value. The column should use the delta16 encoding. -init schema=(uint64) +init schema=(uint) ---- write @@ -117,13 +116,12 @@ finish 01-03: x 0100 # 1 columns 03-07: x 06000000 # 6 rows # column 0 -07-08: b 00000101 # uint64 +07-08: b 00000010 # uint 08-12: x 0c000000 # page start 12 # data for column 0 -12-13: x 03 # delta encoding: delta16 +12-13: x 82 # encoding: 2b,delta 13-21: x abbe105c738e9639 # 64-bit constant: 4149660732785475243 -# padding -21-22: x 00 # aligning to 16-bit boundary +21-22: x 00 # padding (aligning to 16-bit boundary) 22-24: x 0100 # data[0] = 1 + 4149660732785475243 = 4149660732785475244 24-26: x 0000 # data[1] = 0 + 4149660732785475243 = 4149660732785475243 26-28: x 3300 # data[2] = 51 + 4149660732785475243 = 4149660732785475294 @@ -132,7 +130,7 @@ finish 32-34: x 9300 # data[5] = 147 + 4149660732785475243 = 4149660732785475390 34-35: x 00 # block trailer padding -init schema=(uint64) +init schema=(uint) ---- write @@ -150,31 +148,29 @@ write finish ---- # columnar block header -00-01: x 01 # version 1 -01-03: x 0100 # 1 columns -03-07: x 09000000 # 9 rows +00-01: x 01 # version 1 +01-03: x 0100 # 1 columns +03-07: x 09000000 # 9 rows # column 0 -07-08: b 00000101 # uint64 -08-12: x 0c000000 # page start 12 +07-08: b 00000010 # uint +08-12: x 0c000000 # page start 12 # data for column 0 -12-13: x 04 # delta encoding: delta32 -13-21: x 0000000000000000 # 64-bit constant: 0 -# padding -21-24: x 000000 # aligning to 32-bit boundary -24-28: x 00000000 # data[0] = 0 -28-32: x 01000000 # data[1] = 1 -32-36: x 02000000 # data[2] = 2 -36-40: x 03000000 # data[3] = 3 -40-44: x 04000000 # data[4] = 4 -44-48: x 05000000 # data[5] = 5 -48-52: x 06000000 # data[6] = 6 -52-56: x ffffff7f # data[7] = 2147483647 -56-60: x 00000100 # data[8] = 65536 -60-61: x 00 # block trailer padding +12-13: x 04 # encoding: 4b +13-16: x 000000 # padding (aligning to 32-bit boundary) +16-20: x 00000000 # data[0] = 0 +20-24: x 01000000 # data[1] = 1 +24-28: x 02000000 # data[2] = 2 +28-32: x 03000000 # data[3] = 3 +32-36: x 04000000 # data[4] = 4 +36-40: x 05000000 # data[5] = 5 +40-44: x 06000000 # data[6] = 6 +44-48: x ffffff7f # data[7] = 2147483647 +48-52: x 00000100 # data[8] = 65536 +52-53: x 00 # block trailer padding # Test two columns: a uint32 and a uint64. -init schema=(uint32,uint64) +init schema=(uint) ---- write @@ -193,44 +189,27 @@ write finish ---- # columnar block header -00-01: x 01 # version 1 -01-03: x 0200 # 2 columns -03-07: x 0a000000 # 10 rows +00-01: x 01 # version 1 +01-03: x 0100 # 1 columns +03-07: x 0a000000 # 10 rows # column 0 -07-08: b 00000100 # uint32 -08-12: x 11000000 # page start 17 -# column 1 -12-13: b 00000101 # uint64 -13-17: x 20000000 # page start 32 +07-08: b 00000010 # uint +08-12: x 0c000000 # page start 12 # data for column 0 -17-18: x 02 # delta encoding: delta8 -18-22: x 00000000 # 32-bit constant: 0 -22-23: x 00 # data[0] = 0 -23-24: x 01 # data[1] = 1 -24-25: x 02 # data[2] = 2 -25-26: x 03 # data[3] = 3 -26-27: x 04 # data[4] = 4 -27-28: x 05 # data[5] = 5 -28-29: x 06 # data[6] = 6 -29-30: x 07 # data[7] = 7 -30-31: x 08 # data[8] = 8 -31-32: x 09 # data[9] = 9 -# data for column 1 -32-33: x 02 # delta encoding: delta8 -33-41: x 0000000000000000 # 64-bit constant: 0 -41-42: x 00 # data[0] = 0 -42-43: x 01 # data[1] = 1 -43-44: x 02 # data[2] = 2 -44-45: x 03 # data[3] = 3 -45-46: x 04 # data[4] = 4 -46-47: x 05 # data[5] = 5 -47-48: x 06 # data[6] = 6 -48-49: x 07 # data[7] = 7 -49-50: x 08 # data[8] = 8 -50-51: x 09 # data[9] = 9 -51-52: x 00 # block trailer padding +12-13: x 01 # encoding: 1b +13-14: x 00 # data[0] = 0 +14-15: x 01 # data[1] = 1 +15-16: x 02 # data[2] = 2 +16-17: x 03 # data[3] = 3 +17-18: x 04 # data[4] = 4 +18-19: x 05 # data[5] = 5 +19-20: x 06 # data[6] = 6 +20-21: x 07 # data[7] = 7 +21-22: x 08 # data[8] = 8 +22-23: x 09 # data[9] = 9 +23-24: x 00 # block trailer padding -init schema=(uint32,uint64) +init schema=(uint) ---- write @@ -241,27 +220,18 @@ write finish ---- # columnar block header -00-01: x 01 # version 1 -01-03: x 0200 # 2 columns -03-07: x 02000000 # 2 rows +00-01: x 01 # version 1 +01-03: x 0100 # 1 columns +03-07: x 02000000 # 2 rows # column 0 -07-08: b 00000100 # uint32 -08-12: x 11000000 # page start 17 -# column 1 -12-13: b 00000101 # uint64 -13-17: x 1a000000 # page start 26 +07-08: b 00000010 # uint +08-12: x 0c000000 # page start 12 # data for column 0 -17-18: x 03 # delta encoding: delta16 -18-22: x 01000000 # 32-bit constant: 1 -22-24: x 0630 # data[0] = 12294 + 1 = 12295 -24-26: x 0000 # data[1] = 0 + 1 = 1 -# data for column 1 -26-27: x 00 # delta encoding: none -# padding -27-32: x 0000000000 # aligning to 64-bit boundary -32-40: x b2b4949e5b020000 # data[0] = 2592525825202 -40-48: x 0100000000000000 # data[1] = 1 -48-49: x 00 # block trailer padding +12-13: x 02 # encoding: 2b +13-14: x 00 # padding (aligning to 16-bit boundary) +14-16: x 0730 # data[0] = 12295 +16-18: x 0100 # data[1] = 1 +18-19: x 00 # block trailer padding init schema=(bool) ---- @@ -308,7 +278,7 @@ finish 24-32: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 32-33: x 00 # block trailer padding -init schema=(bytes,uint64) +init schema=(bytes,uint) ---- write @@ -330,46 +300,44 @@ finish 001-003: x 0200 # 2 columns 003-007: x 09000000 # 9 rows # column 0 -007-008: b 00000110 # bytes +007-008: b 00000011 # bytes 008-012: x 11000000 # page start 17 # column 1 -012-013: b 00000101 # uint64 -013-017: x 5e000000 # page start 94 +012-013: b 00000010 # uint +013-017: x 5a000000 # page start 90 # data for column 0 # rawbytes # offsets table -017-018: x 02 # delta encoding: delta8 -018-022: x 00000000 # 32-bit constant: 0 -022-023: x 00 # data[0] = 0 [32 overall] -023-024: x 05 # data[1] = 5 [37 overall] -024-025: x 0b # data[2] = 11 [43 overall] -025-026: x 12 # data[3] = 18 [50 overall] -026-027: x 1d # data[4] = 29 [61 overall] -027-028: x 27 # data[5] = 39 [71 overall] -028-029: x 2d # data[6] = 45 [77 overall] -029-030: x 31 # data[7] = 49 [81 overall] -030-031: x 3a # data[8] = 58 [90 overall] -031-032: x 3e # data[9] = 62 [94 overall] +017-018: x 01 # encoding: 1b +018-019: x 00 # data[0] = 0 [28 overall] +019-020: x 05 # data[1] = 5 [33 overall] +020-021: x 0b # data[2] = 11 [39 overall] +021-022: x 12 # data[3] = 18 [46 overall] +022-023: x 1d # data[4] = 29 [57 overall] +023-024: x 27 # data[5] = 39 [67 overall] +024-025: x 2d # data[6] = 45 [73 overall] +025-026: x 31 # data[7] = 49 [77 overall] +026-027: x 3a # data[8] = 58 [86 overall] +027-028: x 3e # data[9] = 62 [90 overall] # data -032-037: x 6170706c65 # data[0]: apple -037-043: x 62616e616e61 # data[1]: banana -043-050: x 636f636f6e7574 # data[2]: coconut -050-061: x 647261676f6e6672756974 # data[3]: dragonfruit -061-071: x 656c6465726265727279 # data[4]: elderberry -071-077: x 667261697365 # data[5]: fraise -077-081: x 676f6a69 # data[6]: goji -081-090: x 6a61636b6672756974 # data[7]: jackfruit -090-094: x 6b697769 # data[8]: kiwi +028-033: x 6170706c65 # data[0]: apple +033-039: x 62616e616e61 # data[1]: banana +039-046: x 636f636f6e7574 # data[2]: coconut +046-057: x 647261676f6e6672756974 # data[3]: dragonfruit +057-067: x 656c6465726265727279 # data[4]: elderberry +067-073: x 667261697365 # data[5]: fraise +073-077: x 676f6a69 # data[6]: goji +077-086: x 6a61636b6672756974 # data[7]: jackfruit +086-090: x 6b697769 # data[8]: kiwi # data for column 1 -094-095: x 02 # delta encoding: delta8 -095-103: x 0a00000000000000 # 64-bit constant: 10 -103-104: x 0a # data[0] = 10 + 10 = 20 -104-105: x 14 # data[1] = 20 + 10 = 30 -105-106: x 00 # data[2] = 0 + 10 = 10 -106-107: x 3c # data[3] = 60 + 10 = 70 -107-108: x 28 # data[4] = 40 + 10 = 50 -108-109: x 32 # data[5] = 50 + 10 = 60 -109-110: x 46 # data[6] = 70 + 10 = 80 -110-111: x 50 # data[7] = 80 + 10 = 90 -111-112: x 5a # data[8] = 90 + 10 = 100 -112-113: x 00 # block trailer padding +090-091: x 01 # encoding: 1b +091-092: x 14 # data[0] = 20 +092-093: x 1e # data[1] = 30 +093-094: x 0a # data[2] = 10 +094-095: x 46 # data[3] = 70 +095-096: x 32 # data[4] = 50 +096-097: x 3c # data[5] = 60 +097-098: x 50 # data[6] = 80 +098-099: x 5a # data[7] = 90 +099-100: x 64 # data[8] = 100 +100-101: x 00 # block trailer padding diff --git a/sstable/colblk/testdata/data_block/bundle_search b/sstable/colblk/testdata/data_block/bundle_search index addcdaa6e9..ecf16a8cb7 100644 --- a/sstable/colblk/testdata/data_block/bundle_search +++ b/sstable/colblk/testdata/data_block/bundle_search @@ -13,7 +13,7 @@ init bundle-size=4 size=1: 0: prefixes: prefixbytes(4): 0 keys 1: suffixes: bytes: 0 rows set; 0 bytes in data -2: trailers: uint64: 0 rows +2: trailers: uint: 0 rows 3: prefix changed: bitmap 4: values: bytes: 0 rows set; 0 bytes in data 5: is-value-ext: bitmap @@ -86,10 +86,10 @@ backwash#1,SET:v backwoods#1,SET:v bacteria#1,SET:v ---- -size=761: +size=745: 0: prefixes: prefixbytes(4): 66 keys 1: suffixes: bytes: 66 rows set; 0 bytes in data -2: trailers: uint64: 66 rows +2: trailers: uint: 66 rows 3: prefix changed: bitmap 4: values: bytes: 66 rows set; 66 bytes in data 5: is-value-ext: bitmap @@ -103,425 +103,421 @@ finish 005-007: x 0600 # 6 columns 007-011: x 42000000 # 66 rows # column 0 -011-012: b 00000111 # prefixbytes +011-012: b 00000100 # prefixbytes 012-016: x 29000000 # page start 41 # column 1 -016-017: b 00000110 # bytes -017-021: x 29020000 # page start 553 +016-017: b 00000011 # bytes +017-021: x 25020000 # page start 549 # column 2 -021-022: b 00000101 # uint64 -022-026: x 2e020000 # page start 558 +021-022: b 00000010 # uint +022-026: x 26020000 # page start 550 # column 3 026-027: b 00000001 # bool -027-031: x 37020000 # page start 567 +027-031: x 2f020000 # page start 559 # column 4 -031-032: b 00000110 # bytes -032-036: x 50020000 # page start 592 +031-032: b 00000011 # bytes +032-036: x 48020000 # page start 584 # column 5 036-037: b 00000001 # bool -037-041: x da020000 # page start 730 +037-041: x ce020000 # page start 718 # data for column 0 # PrefixBytes 041-042: x 02 # bundleSize: 4 # Offsets table -042-043: x 03 # delta encoding: delta16 -043-047: x 00000000 # 32-bit constant: 0 -# padding -047-048: x 00 # aligning to 16-bit boundary -048-050: x 0200 # data[0] = 2 [218 overall] -050-052: x 0300 # data[1] = 3 [219 overall] -052-054: x 0400 # data[2] = 4 [220 overall] -054-056: x 0700 # data[3] = 7 [223 overall] -056-058: x 0e00 # data[4] = 14 [230 overall] -058-060: x 1200 # data[5] = 18 [234 overall] -060-062: x 1500 # data[6] = 21 [237 overall] -062-064: x 1a00 # data[7] = 26 [242 overall] -064-066: x 1d00 # data[8] = 29 [245 overall] -066-068: x 2200 # data[9] = 34 [250 overall] -068-070: x 2500 # data[10] = 37 [253 overall] -070-072: x 2600 # data[11] = 38 [254 overall] -072-074: x 2d00 # data[12] = 45 [261 overall] -074-076: x 3000 # data[13] = 48 [264 overall] -076-078: x 3100 # data[14] = 49 [265 overall] -078-080: x 3400 # data[15] = 52 [268 overall] -080-082: x 3500 # data[16] = 53 [269 overall] -082-084: x 3b00 # data[17] = 59 [275 overall] -084-086: x 4100 # data[18] = 65 [281 overall] -086-088: x 4600 # data[19] = 70 [286 overall] -088-090: x 4700 # data[20] = 71 [287 overall] -090-092: x 4900 # data[21] = 73 [289 overall] -092-094: x 4c00 # data[22] = 76 [292 overall] -094-096: x 5000 # data[23] = 80 [296 overall] -096-098: x 5500 # data[24] = 85 [301 overall] -098-100: x 5800 # data[25] = 88 [304 overall] -100-102: x 5a00 # data[26] = 90 [306 overall] -102-104: x 5f00 # data[27] = 95 [311 overall] -104-106: x 6600 # data[28] = 102 [318 overall] -106-108: x 6900 # data[29] = 105 [321 overall] -108-110: x 6d00 # data[30] = 109 [325 overall] -110-112: x 7400 # data[31] = 116 [332 overall] -112-114: x 7400 # data[32] = 116 [332 overall] -114-116: x 7700 # data[33] = 119 [335 overall] -116-118: x 7d00 # data[34] = 125 [341 overall] -118-120: x 8200 # data[35] = 130 [346 overall] -120-122: x 8300 # data[36] = 131 [347 overall] -122-124: x 8c00 # data[37] = 140 [356 overall] -124-126: x 9900 # data[38] = 153 [369 overall] -126-128: x a200 # data[39] = 162 [378 overall] -128-130: x a300 # data[40] = 163 [379 overall] -130-132: x a500 # data[41] = 165 [381 overall] -132-134: x a900 # data[42] = 169 [385 overall] -134-136: x ad00 # data[43] = 173 [389 overall] -136-138: x b200 # data[44] = 178 [394 overall] -138-140: x ba00 # data[45] = 186 [402 overall] -140-142: x bc00 # data[46] = 188 [404 overall] -142-144: x c100 # data[47] = 193 [409 overall] -144-146: x c600 # data[48] = 198 [414 overall] -146-148: x ca00 # data[49] = 202 [418 overall] -148-150: x ce00 # data[50] = 206 [422 overall] -150-152: x d000 # data[51] = 208 [424 overall] -152-154: x d400 # data[52] = 212 [428 overall] -154-156: x d600 # data[53] = 214 [430 overall] -156-158: x db00 # data[54] = 219 [435 overall] -158-160: x df00 # data[55] = 223 [439 overall] -160-162: x e100 # data[56] = 225 [441 overall] -162-164: x e500 # data[57] = 229 [445 overall] -164-166: x eb00 # data[58] = 235 [451 overall] -166-168: x f100 # data[59] = 241 [457 overall] -168-170: x f700 # data[60] = 247 [463 overall] -170-172: x fd00 # data[61] = 253 [469 overall] -172-174: x fd00 # data[62] = 253 [469 overall] -174-176: x ff00 # data[63] = 255 [471 overall] -176-178: x 0301 # data[64] = 259 [475 overall] -178-180: x 0901 # data[65] = 265 [481 overall] -180-182: x 0b01 # data[66] = 267 [483 overall] -182-184: x 1001 # data[67] = 272 [488 overall] -184-186: x 1401 # data[68] = 276 [492 overall] -186-188: x 1901 # data[69] = 281 [497 overall] -188-190: x 1e01 # data[70] = 286 [502 overall] -190-192: x 2001 # data[71] = 288 [504 overall] -192-194: x 2501 # data[72] = 293 [509 overall] -194-196: x 2901 # data[73] = 297 [513 overall] -196-198: x 2f01 # data[74] = 303 [519 overall] -198-200: x 3401 # data[75] = 308 [524 overall] -200-202: x 3601 # data[76] = 310 [526 overall] -202-204: x 3801 # data[77] = 312 [528 overall] -204-206: x 3c01 # data[78] = 316 [532 overall] -206-208: x 4101 # data[79] = 321 [537 overall] -208-210: x 4501 # data[80] = 325 [541 overall] -210-212: x 4601 # data[81] = 326 [542 overall] -212-214: x 4c01 # data[82] = 332 [548 overall] -214-216: x 5101 # data[83] = 337 [553 overall] +042-043: x 02 # encoding: 2b +043-044: x 00 # padding (aligning to 16-bit boundary) +044-046: x 0200 # data[0] = 2 [214 overall] +046-048: x 0300 # data[1] = 3 [215 overall] +048-050: x 0400 # data[2] = 4 [216 overall] +050-052: x 0700 # data[3] = 7 [219 overall] +052-054: x 0e00 # data[4] = 14 [226 overall] +054-056: x 1200 # data[5] = 18 [230 overall] +056-058: x 1500 # data[6] = 21 [233 overall] +058-060: x 1a00 # data[7] = 26 [238 overall] +060-062: x 1d00 # data[8] = 29 [241 overall] +062-064: x 2200 # data[9] = 34 [246 overall] +064-066: x 2500 # data[10] = 37 [249 overall] +066-068: x 2600 # data[11] = 38 [250 overall] +068-070: x 2d00 # data[12] = 45 [257 overall] +070-072: x 3000 # data[13] = 48 [260 overall] +072-074: x 3100 # data[14] = 49 [261 overall] +074-076: x 3400 # data[15] = 52 [264 overall] +076-078: x 3500 # data[16] = 53 [265 overall] +078-080: x 3b00 # data[17] = 59 [271 overall] +080-082: x 4100 # data[18] = 65 [277 overall] +082-084: x 4600 # data[19] = 70 [282 overall] +084-086: x 4700 # data[20] = 71 [283 overall] +086-088: x 4900 # data[21] = 73 [285 overall] +088-090: x 4c00 # data[22] = 76 [288 overall] +090-092: x 5000 # data[23] = 80 [292 overall] +092-094: x 5500 # data[24] = 85 [297 overall] +094-096: x 5800 # data[25] = 88 [300 overall] +096-098: x 5a00 # data[26] = 90 [302 overall] +098-100: x 5f00 # data[27] = 95 [307 overall] +100-102: x 6600 # data[28] = 102 [314 overall] +102-104: x 6900 # data[29] = 105 [317 overall] +104-106: x 6d00 # data[30] = 109 [321 overall] +106-108: x 7400 # data[31] = 116 [328 overall] +108-110: x 7400 # data[32] = 116 [328 overall] +110-112: x 7700 # data[33] = 119 [331 overall] +112-114: x 7d00 # data[34] = 125 [337 overall] +114-116: x 8200 # data[35] = 130 [342 overall] +116-118: x 8300 # data[36] = 131 [343 overall] +118-120: x 8c00 # data[37] = 140 [352 overall] +120-122: x 9900 # data[38] = 153 [365 overall] +122-124: x a200 # data[39] = 162 [374 overall] +124-126: x a300 # data[40] = 163 [375 overall] +126-128: x a500 # data[41] = 165 [377 overall] +128-130: x a900 # data[42] = 169 [381 overall] +130-132: x ad00 # data[43] = 173 [385 overall] +132-134: x b200 # data[44] = 178 [390 overall] +134-136: x ba00 # data[45] = 186 [398 overall] +136-138: x bc00 # data[46] = 188 [400 overall] +138-140: x c100 # data[47] = 193 [405 overall] +140-142: x c600 # data[48] = 198 [410 overall] +142-144: x ca00 # data[49] = 202 [414 overall] +144-146: x ce00 # data[50] = 206 [418 overall] +146-148: x d000 # data[51] = 208 [420 overall] +148-150: x d400 # data[52] = 212 [424 overall] +150-152: x d600 # data[53] = 214 [426 overall] +152-154: x db00 # data[54] = 219 [431 overall] +154-156: x df00 # data[55] = 223 [435 overall] +156-158: x e100 # data[56] = 225 [437 overall] +158-160: x e500 # data[57] = 229 [441 overall] +160-162: x eb00 # data[58] = 235 [447 overall] +162-164: x f100 # data[59] = 241 [453 overall] +164-166: x f700 # data[60] = 247 [459 overall] +166-168: x fd00 # data[61] = 253 [465 overall] +168-170: x fd00 # data[62] = 253 [465 overall] +170-172: x ff00 # data[63] = 255 [467 overall] +172-174: x 0301 # data[64] = 259 [471 overall] +174-176: x 0901 # data[65] = 265 [477 overall] +176-178: x 0b01 # data[66] = 267 [479 overall] +178-180: x 1001 # data[67] = 272 [484 overall] +180-182: x 1401 # data[68] = 276 [488 overall] +182-184: x 1901 # data[69] = 281 [493 overall] +184-186: x 1e01 # data[70] = 286 [498 overall] +186-188: x 2001 # data[71] = 288 [500 overall] +188-190: x 2501 # data[72] = 293 [505 overall] +190-192: x 2901 # data[73] = 297 [509 overall] +192-194: x 2f01 # data[74] = 303 [515 overall] +194-196: x 3401 # data[75] = 308 [520 overall] +196-198: x 3601 # data[76] = 310 [522 overall] +198-200: x 3801 # data[77] = 312 [524 overall] +200-202: x 3c01 # data[78] = 316 [528 overall] +202-204: x 4101 # data[79] = 321 [533 overall] +204-206: x 4501 # data[80] = 325 [537 overall] +206-208: x 4601 # data[81] = 326 [538 overall] +208-210: x 4c01 # data[82] = 332 [544 overall] +210-212: x 5101 # data[83] = 337 [549 overall] # Data -216-218: x 6261 # data[00]: ba (block prefix) -218-219: x 62 # data[01]: ..b (bundle prefix) -219-220: x 61 # data[02]: ...a -220-223: x 626c65 # data[03]: ...ble -223-230: x 626c656d656e74 # data[04]: ...blement -230-234: x 626c6572 # data[05]: ...bler -234-237: x 62626c # data[06]: ..bbl (bundle prefix) -237-242: x 65736f6d65 # data[07]: .....esome -242-245: x 696e67 # data[08]: .....ing -245-250: x 696e676c79 # data[09]: .....ingly -250-253: x 697368 # data[10]: .....ish -253-254: x 62 # data[11]: ..b (bundle prefix) -254-261: x 626c6973686c79 # data[12]: ...blishly -261-264: x 626c79 # data[13]: ...bly -264-265: x 65 # data[14]: ...e -265-268: x 6f6f6e # data[15]: ...oon -268-269: x 62 # data[16]: ..b (bundle prefix) -269-275: x 6f6f6e657279 # data[17]: ...oonery -275-281: x 6f6f6e697368 # data[18]: ...oonish -281-286: x 7573686b61 # data[19]: ...ushka -286-287: x 79 # data[20]: ...y -287-289: x 6279 # data[21]: ..by (bundle prefix) -289-292: x 646f6d # data[22]: ....dom -292-296: x 686f6f64 # data[23]: ....hood -296-301: x 686f757365 # data[24]: ....house -301-304: x 697368 # data[25]: ....ish -304-306: x 6279 # data[26]: ..by (bundle prefix) -306-311: x 6973686c79 # data[27]: ....ishly -311-318: x 6973686e657373 # data[28]: ....ishness -318-321: x 69736d # data[29]: ....ism -321-325: x 6c696b65 # data[30]: ....like -325-332: x 636368616e616c # data[31]: ..cchanal (bundle prefix) -332-332: x # data[32]: ......... -332-335: x 69616e # data[33]: .........ian -335-341: x 69616e69736d # data[34]: .........ianism -341-346: x 69616e6c79 # data[35]: .........ianly -346-347: x 63 # data[36]: ..c (bundle prefix) -347-356: x 6368616e616c69736d # data[37]: ...chanalism -356-366: x 6368616e616c697a6174 # data[38]: ...chanalization -366-369: x 696f6e # (continued...) -369-378: x 6368616e616c697a65 # data[39]: ...chanalize -378-379: x 6b # data[40]: ...k -379-381: x 636b # data[41]: ..ck (bundle prefix) -381-385: x 61636865 # data[42]: ....ache -385-389: x 626f6e65 # data[43]: ....bone -389-394: x 626f6e6564 # data[44]: ....boned -394-402: x 627265616b696e67 # data[45]: ....breaking -402-404: x 636b # data[46]: ..ck (bundle prefix) -404-409: x 636f757274 # data[47]: ....court -409-414: x 63726f7373 # data[48]: ....cross -414-418: x 646f6f72 # data[49]: ....door -418-422: x 646f776e # data[50]: ....down -422-424: x 636b # data[51]: ..ck (bundle prefix) -424-428: x 64726f70 # data[52]: ....drop -428-430: x 6564 # data[53]: ....ed -430-435: x 6669656c64 # data[54]: ....field -435-439: x 66696c6c # data[55]: ....fill -439-441: x 636b # data[56]: ..ck (bundle prefix) -441-445: x 66697265 # data[57]: ....fire -445-451: x 666972696e67 # data[58]: ....firing -451-457: x 67616d6d6f6e # data[59]: ....gammon -457-463: x 67726f756e64 # data[60]: ....ground -463-469: x 636b68616e64 # data[61]: ..ckhand (bundle prefix) -469-469: x # data[62]: ........ -469-471: x 6564 # data[63]: ........ed -471-475: x 65646c79 # data[64]: ........edly -475-481: x 65646e657373 # data[65]: ........edness -481-483: x 636b # data[66]: ..ck (bundle prefix) -483-488: x 706564616c # data[67]: ....pedal -488-492: x 736c6170 # data[68]: ....slap -492-497: x 736c696465 # data[69]: ....slide -497-502: x 7370616365 # data[70]: ....space -502-504: x 636b # data[71]: ..ck (bundle prefix) -504-509: x 7374616765 # data[72]: ....stage -509-513: x 73746f70 # data[73]: ....stop -513-519: x 7374726f6b65 # data[74]: ....stroke -519-524: x 747261636b # data[75]: ....track -524-526: x 636b # data[76]: ..ck (bundle prefix) -526-528: x 7570 # data[77]: ....up -528-532: x 77617264 # data[78]: ....ward -532-537: x 7761726473 # data[79]: ....wards -537-541: x 77617368 # data[80]: ....wash -541-542: x 63 # data[81]: ..c (bundle prefix) -542-548: x 6b776f6f6473 # data[82]: ...kwoods -548-553: x 7465726961 # data[83]: ...teria +212-214: x 6261 # data[00]: ba (block prefix) +214-215: x 62 # data[01]: ..b (bundle prefix) +215-216: x 61 # data[02]: ...a +216-219: x 626c65 # data[03]: ...ble +219-226: x 626c656d656e74 # data[04]: ...blement +226-230: x 626c6572 # data[05]: ...bler +230-233: x 62626c # data[06]: ..bbl (bundle prefix) +233-238: x 65736f6d65 # data[07]: .....esome +238-241: x 696e67 # data[08]: .....ing +241-246: x 696e676c79 # data[09]: .....ingly +246-249: x 697368 # data[10]: .....ish +249-250: x 62 # data[11]: ..b (bundle prefix) +250-257: x 626c6973686c79 # data[12]: ...blishly +257-260: x 626c79 # data[13]: ...bly +260-261: x 65 # data[14]: ...e +261-264: x 6f6f6e # data[15]: ...oon +264-265: x 62 # data[16]: ..b (bundle prefix) +265-271: x 6f6f6e657279 # data[17]: ...oonery +271-277: x 6f6f6e697368 # data[18]: ...oonish +277-282: x 7573686b61 # data[19]: ...ushka +282-283: x 79 # data[20]: ...y +283-285: x 6279 # data[21]: ..by (bundle prefix) +285-288: x 646f6d # data[22]: ....dom +288-292: x 686f6f64 # data[23]: ....hood +292-297: x 686f757365 # data[24]: ....house +297-300: x 697368 # data[25]: ....ish +300-302: x 6279 # data[26]: ..by (bundle prefix) +302-307: x 6973686c79 # data[27]: ....ishly +307-314: x 6973686e657373 # data[28]: ....ishness +314-317: x 69736d # data[29]: ....ism +317-321: x 6c696b65 # data[30]: ....like +321-328: x 636368616e616c # data[31]: ..cchanal (bundle prefix) +328-328: x # data[32]: ......... +328-331: x 69616e # data[33]: .........ian +331-337: x 69616e69736d # data[34]: .........ianism +337-342: x 69616e6c79 # data[35]: .........ianly +342-343: x 63 # data[36]: ..c (bundle prefix) +343-352: x 6368616e616c69736d # data[37]: ...chanalism +352-362: x 6368616e616c697a6174 # data[38]: ...chanalization +362-365: x 696f6e # (continued...) +365-374: x 6368616e616c697a65 # data[39]: ...chanalize +374-375: x 6b # data[40]: ...k +375-377: x 636b # data[41]: ..ck (bundle prefix) +377-381: x 61636865 # data[42]: ....ache +381-385: x 626f6e65 # data[43]: ....bone +385-390: x 626f6e6564 # data[44]: ....boned +390-398: x 627265616b696e67 # data[45]: ....breaking +398-400: x 636b # data[46]: ..ck (bundle prefix) +400-405: x 636f757274 # data[47]: ....court +405-410: x 63726f7373 # data[48]: ....cross +410-414: x 646f6f72 # data[49]: ....door +414-418: x 646f776e # data[50]: ....down +418-420: x 636b # data[51]: ..ck (bundle prefix) +420-424: x 64726f70 # data[52]: ....drop +424-426: x 6564 # data[53]: ....ed +426-431: x 6669656c64 # data[54]: ....field +431-435: x 66696c6c # data[55]: ....fill +435-437: x 636b # data[56]: ..ck (bundle prefix) +437-441: x 66697265 # data[57]: ....fire +441-447: x 666972696e67 # data[58]: ....firing +447-453: x 67616d6d6f6e # data[59]: ....gammon +453-459: x 67726f756e64 # data[60]: ....ground +459-465: x 636b68616e64 # data[61]: ..ckhand (bundle prefix) +465-465: x # data[62]: ........ +465-467: x 6564 # data[63]: ........ed +467-471: x 65646c79 # data[64]: ........edly +471-477: x 65646e657373 # data[65]: ........edness +477-479: x 636b # data[66]: ..ck (bundle prefix) +479-484: x 706564616c # data[67]: ....pedal +484-488: x 736c6170 # data[68]: ....slap +488-493: x 736c696465 # data[69]: ....slide +493-498: x 7370616365 # data[70]: ....space +498-500: x 636b # data[71]: ..ck (bundle prefix) +500-505: x 7374616765 # data[72]: ....stage +505-509: x 73746f70 # data[73]: ....stop +509-515: x 7374726f6b65 # data[74]: ....stroke +515-520: x 747261636b # data[75]: ....track +520-522: x 636b # data[76]: ..ck (bundle prefix) +522-524: x 7570 # data[77]: ....up +524-528: x 77617264 # data[78]: ....ward +528-533: x 7761726473 # data[79]: ....wards +533-537: x 77617368 # data[80]: ....wash +537-538: x 63 # data[81]: ..c (bundle prefix) +538-544: x 6b776f6f6473 # data[82]: ...kwoods +544-549: x 7465726961 # data[83]: ...teria # data for column 1 # rawbytes # offsets table -553-554: x 01 # delta encoding: const -554-558: x 00000000 # 32-bit constant: 0 +549-550: x 00 # encoding: zero # data -558-558: x # data[0]: -558-558: x # data[1]: -558-558: x # data[2]: -558-558: x # data[3]: -558-558: x # data[4]: -558-558: x # data[5]: -558-558: x # data[6]: -558-558: x # data[7]: -558-558: x # data[8]: -558-558: x # data[9]: -558-558: x # data[10]: -558-558: x # data[11]: -558-558: x # data[12]: -558-558: x # data[13]: -558-558: x # data[14]: -558-558: x # data[15]: -558-558: x # data[16]: -558-558: x # data[17]: -558-558: x # data[18]: -558-558: x # data[19]: -558-558: x # data[20]: -558-558: x # data[21]: -558-558: x # data[22]: -558-558: x # data[23]: -558-558: x # data[24]: -558-558: x # data[25]: -558-558: x # data[26]: -558-558: x # data[27]: -558-558: x # data[28]: -558-558: x # data[29]: -558-558: x # data[30]: -558-558: x # data[31]: -558-558: x # data[32]: -558-558: x # data[33]: -558-558: x # data[34]: -558-558: x # data[35]: -558-558: x # data[36]: -558-558: x # data[37]: -558-558: x # data[38]: -558-558: x # data[39]: -558-558: x # data[40]: -558-558: x # data[41]: -558-558: x # data[42]: -558-558: x # data[43]: -558-558: x # data[44]: -558-558: x # data[45]: -558-558: x # data[46]: -558-558: x # data[47]: -558-558: x # data[48]: -558-558: x # data[49]: -558-558: x # data[50]: -558-558: x # data[51]: -558-558: x # data[52]: -558-558: x # data[53]: -558-558: x # data[54]: -558-558: x # data[55]: -558-558: x # data[56]: -558-558: x # data[57]: -558-558: x # data[58]: -558-558: x # data[59]: -558-558: x # data[60]: -558-558: x # data[61]: -558-558: x # data[62]: -558-558: x # data[63]: -558-558: x # data[64]: -558-558: x # data[65]: +550-550: x # data[0]: +550-550: x # data[1]: +550-550: x # data[2]: +550-550: x # data[3]: +550-550: x # data[4]: +550-550: x # data[5]: +550-550: x # data[6]: +550-550: x # data[7]: +550-550: x # data[8]: +550-550: x # data[9]: +550-550: x # data[10]: +550-550: x # data[11]: +550-550: x # data[12]: +550-550: x # data[13]: +550-550: x # data[14]: +550-550: x # data[15]: +550-550: x # data[16]: +550-550: x # data[17]: +550-550: x # data[18]: +550-550: x # data[19]: +550-550: x # data[20]: +550-550: x # data[21]: +550-550: x # data[22]: +550-550: x # data[23]: +550-550: x # data[24]: +550-550: x # data[25]: +550-550: x # data[26]: +550-550: x # data[27]: +550-550: x # data[28]: +550-550: x # data[29]: +550-550: x # data[30]: +550-550: x # data[31]: +550-550: x # data[32]: +550-550: x # data[33]: +550-550: x # data[34]: +550-550: x # data[35]: +550-550: x # data[36]: +550-550: x # data[37]: +550-550: x # data[38]: +550-550: x # data[39]: +550-550: x # data[40]: +550-550: x # data[41]: +550-550: x # data[42]: +550-550: x # data[43]: +550-550: x # data[44]: +550-550: x # data[45]: +550-550: x # data[46]: +550-550: x # data[47]: +550-550: x # data[48]: +550-550: x # data[49]: +550-550: x # data[50]: +550-550: x # data[51]: +550-550: x # data[52]: +550-550: x # data[53]: +550-550: x # data[54]: +550-550: x # data[55]: +550-550: x # data[56]: +550-550: x # data[57]: +550-550: x # data[58]: +550-550: x # data[59]: +550-550: x # data[60]: +550-550: x # data[61]: +550-550: x # data[62]: +550-550: x # data[63]: +550-550: x # data[64]: +550-550: x # data[65]: # data for column 2 -558-559: x 01 # delta encoding: const -559-567: x 0101000000000000 # 64-bit constant: 257 +550-551: x 80 # encoding: const +551-559: x 0101000000000000 # 64-bit constant: 257 # data for column 3 -567-568: x 00 # padding to align to 64-bit boundary -568-576: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 0 -576-584: b 0000001100000000000000000000000000000000000000000000000000000000 # bitmap word 1 -584-592: b 0000001100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +559-560: x 00 # padding to align to 64-bit boundary +560-568: b 1111111111111111111111111111111111111111111111111111111111111111 # bitmap word 0 +568-576: b 0000001100000000000000000000000000000000000000000000000000000000 # bitmap word 1 +576-584: b 0000001100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 # data for column 4 # rawbytes # offsets table -592-593: x 02 # delta encoding: delta8 -593-597: x 00000000 # 32-bit constant: 0 -597-598: x 00 # data[0] = 0 [664 overall] -598-599: x 01 # data[1] = 1 [665 overall] -599-600: x 02 # data[2] = 2 [666 overall] -600-601: x 03 # data[3] = 3 [667 overall] -601-602: x 04 # data[4] = 4 [668 overall] -602-603: x 05 # data[5] = 5 [669 overall] -603-604: x 06 # data[6] = 6 [670 overall] -604-605: x 07 # data[7] = 7 [671 overall] -605-606: x 08 # data[8] = 8 [672 overall] -606-607: x 09 # data[9] = 9 [673 overall] -607-608: x 0a # data[10] = 10 [674 overall] -608-609: x 0b # data[11] = 11 [675 overall] -609-610: x 0c # data[12] = 12 [676 overall] -610-611: x 0d # data[13] = 13 [677 overall] -611-612: x 0e # data[14] = 14 [678 overall] -612-613: x 0f # data[15] = 15 [679 overall] -613-614: x 10 # data[16] = 16 [680 overall] -614-615: x 11 # data[17] = 17 [681 overall] -615-616: x 12 # data[18] = 18 [682 overall] -616-617: x 13 # data[19] = 19 [683 overall] -617-618: x 14 # data[20] = 20 [684 overall] -618-619: x 15 # data[21] = 21 [685 overall] -619-620: x 16 # data[22] = 22 [686 overall] -620-621: x 17 # data[23] = 23 [687 overall] -621-622: x 18 # data[24] = 24 [688 overall] -622-623: x 19 # data[25] = 25 [689 overall] -623-624: x 1a # data[26] = 26 [690 overall] -624-625: x 1b # data[27] = 27 [691 overall] -625-626: x 1c # data[28] = 28 [692 overall] -626-627: x 1d # data[29] = 29 [693 overall] -627-628: x 1e # data[30] = 30 [694 overall] -628-629: x 1f # data[31] = 31 [695 overall] -629-630: x 20 # data[32] = 32 [696 overall] -630-631: x 21 # data[33] = 33 [697 overall] -631-632: x 22 # data[34] = 34 [698 overall] -632-633: x 23 # data[35] = 35 [699 overall] -633-634: x 24 # data[36] = 36 [700 overall] -634-635: x 25 # data[37] = 37 [701 overall] -635-636: x 26 # data[38] = 38 [702 overall] -636-637: x 27 # data[39] = 39 [703 overall] -637-638: x 28 # data[40] = 40 [704 overall] -638-639: x 29 # data[41] = 41 [705 overall] -639-640: x 2a # data[42] = 42 [706 overall] -640-641: x 2b # data[43] = 43 [707 overall] -641-642: x 2c # data[44] = 44 [708 overall] -642-643: x 2d # data[45] = 45 [709 overall] -643-644: x 2e # data[46] = 46 [710 overall] -644-645: x 2f # data[47] = 47 [711 overall] -645-646: x 30 # data[48] = 48 [712 overall] -646-647: x 31 # data[49] = 49 [713 overall] -647-648: x 32 # data[50] = 50 [714 overall] -648-649: x 33 # data[51] = 51 [715 overall] -649-650: x 34 # data[52] = 52 [716 overall] -650-651: x 35 # data[53] = 53 [717 overall] -651-652: x 36 # data[54] = 54 [718 overall] -652-653: x 37 # data[55] = 55 [719 overall] -653-654: x 38 # data[56] = 56 [720 overall] -654-655: x 39 # data[57] = 57 [721 overall] -655-656: x 3a # data[58] = 58 [722 overall] -656-657: x 3b # data[59] = 59 [723 overall] -657-658: x 3c # data[60] = 60 [724 overall] -658-659: x 3d # data[61] = 61 [725 overall] -659-660: x 3e # data[62] = 62 [726 overall] -660-661: x 3f # data[63] = 63 [727 overall] -661-662: x 40 # data[64] = 64 [728 overall] -662-663: x 41 # data[65] = 65 [729 overall] -663-664: x 42 # data[66] = 66 [730 overall] +584-585: x 01 # encoding: 1b +585-586: x 00 # data[0] = 0 [652 overall] +586-587: x 01 # data[1] = 1 [653 overall] +587-588: x 02 # data[2] = 2 [654 overall] +588-589: x 03 # data[3] = 3 [655 overall] +589-590: x 04 # data[4] = 4 [656 overall] +590-591: x 05 # data[5] = 5 [657 overall] +591-592: x 06 # data[6] = 6 [658 overall] +592-593: x 07 # data[7] = 7 [659 overall] +593-594: x 08 # data[8] = 8 [660 overall] +594-595: x 09 # data[9] = 9 [661 overall] +595-596: x 0a # data[10] = 10 [662 overall] +596-597: x 0b # data[11] = 11 [663 overall] +597-598: x 0c # data[12] = 12 [664 overall] +598-599: x 0d # data[13] = 13 [665 overall] +599-600: x 0e # data[14] = 14 [666 overall] +600-601: x 0f # data[15] = 15 [667 overall] +601-602: x 10 # data[16] = 16 [668 overall] +602-603: x 11 # data[17] = 17 [669 overall] +603-604: x 12 # data[18] = 18 [670 overall] +604-605: x 13 # data[19] = 19 [671 overall] +605-606: x 14 # data[20] = 20 [672 overall] +606-607: x 15 # data[21] = 21 [673 overall] +607-608: x 16 # data[22] = 22 [674 overall] +608-609: x 17 # data[23] = 23 [675 overall] +609-610: x 18 # data[24] = 24 [676 overall] +610-611: x 19 # data[25] = 25 [677 overall] +611-612: x 1a # data[26] = 26 [678 overall] +612-613: x 1b # data[27] = 27 [679 overall] +613-614: x 1c # data[28] = 28 [680 overall] +614-615: x 1d # data[29] = 29 [681 overall] +615-616: x 1e # data[30] = 30 [682 overall] +616-617: x 1f # data[31] = 31 [683 overall] +617-618: x 20 # data[32] = 32 [684 overall] +618-619: x 21 # data[33] = 33 [685 overall] +619-620: x 22 # data[34] = 34 [686 overall] +620-621: x 23 # data[35] = 35 [687 overall] +621-622: x 24 # data[36] = 36 [688 overall] +622-623: x 25 # data[37] = 37 [689 overall] +623-624: x 26 # data[38] = 38 [690 overall] +624-625: x 27 # data[39] = 39 [691 overall] +625-626: x 28 # data[40] = 40 [692 overall] +626-627: x 29 # data[41] = 41 [693 overall] +627-628: x 2a # data[42] = 42 [694 overall] +628-629: x 2b # data[43] = 43 [695 overall] +629-630: x 2c # data[44] = 44 [696 overall] +630-631: x 2d # data[45] = 45 [697 overall] +631-632: x 2e # data[46] = 46 [698 overall] +632-633: x 2f # data[47] = 47 [699 overall] +633-634: x 30 # data[48] = 48 [700 overall] +634-635: x 31 # data[49] = 49 [701 overall] +635-636: x 32 # data[50] = 50 [702 overall] +636-637: x 33 # data[51] = 51 [703 overall] +637-638: x 34 # data[52] = 52 [704 overall] +638-639: x 35 # data[53] = 53 [705 overall] +639-640: x 36 # data[54] = 54 [706 overall] +640-641: x 37 # data[55] = 55 [707 overall] +641-642: x 38 # data[56] = 56 [708 overall] +642-643: x 39 # data[57] = 57 [709 overall] +643-644: x 3a # data[58] = 58 [710 overall] +644-645: x 3b # data[59] = 59 [711 overall] +645-646: x 3c # data[60] = 60 [712 overall] +646-647: x 3d # data[61] = 61 [713 overall] +647-648: x 3e # data[62] = 62 [714 overall] +648-649: x 3f # data[63] = 63 [715 overall] +649-650: x 40 # data[64] = 64 [716 overall] +650-651: x 41 # data[65] = 65 [717 overall] +651-652: x 42 # data[66] = 66 [718 overall] # data -664-665: x 76 # data[0]: v -665-666: x 76 # data[1]: v -666-667: x 76 # data[2]: v -667-668: x 76 # data[3]: v -668-669: x 76 # data[4]: v -669-670: x 76 # data[5]: v -670-671: x 76 # data[6]: v -671-672: x 76 # data[7]: v -672-673: x 76 # data[8]: v -673-674: x 76 # data[9]: v -674-675: x 76 # data[10]: v -675-676: x 76 # data[11]: v -676-677: x 76 # data[12]: v -677-678: x 76 # data[13]: v -678-679: x 76 # data[14]: v -679-680: x 76 # data[15]: v -680-681: x 76 # data[16]: v -681-682: x 76 # data[17]: v -682-683: x 76 # data[18]: v -683-684: x 76 # data[19]: v -684-685: x 76 # data[20]: v -685-686: x 76 # data[21]: v -686-687: x 76 # data[22]: v -687-688: x 76 # data[23]: v -688-689: x 76 # data[24]: v -689-690: x 76 # data[25]: v -690-691: x 76 # data[26]: v -691-692: x 76 # data[27]: v -692-693: x 76 # data[28]: v -693-694: x 76 # data[29]: v -694-695: x 76 # data[30]: v -695-696: x 76 # data[31]: v -696-697: x 76 # data[32]: v -697-698: x 76 # data[33]: v -698-699: x 76 # data[34]: v -699-700: x 76 # data[35]: v -700-701: x 76 # data[36]: v -701-702: x 76 # data[37]: v -702-703: x 76 # data[38]: v -703-704: x 76 # data[39]: v -704-705: x 76 # data[40]: v -705-706: x 76 # data[41]: v -706-707: x 76 # data[42]: v -707-708: x 76 # data[43]: v -708-709: x 76 # data[44]: v -709-710: x 76 # data[45]: v -710-711: x 76 # data[46]: v -711-712: x 76 # data[47]: v -712-713: x 76 # data[48]: v -713-714: x 76 # data[49]: v -714-715: x 76 # data[50]: v -715-716: x 76 # data[51]: v -716-717: x 76 # data[52]: v -717-718: x 76 # data[53]: v -718-719: x 76 # data[54]: v -719-720: x 76 # data[55]: v -720-721: x 76 # data[56]: v -721-722: x 76 # data[57]: v -722-723: x 76 # data[58]: v -723-724: x 76 # data[59]: v -724-725: x 76 # data[60]: v -725-726: x 76 # data[61]: v -726-727: x 76 # data[62]: v -727-728: x 76 # data[63]: v -728-729: x 76 # data[64]: v -729-730: x 76 # data[65]: v +652-653: x 76 # data[0]: v +653-654: x 76 # data[1]: v +654-655: x 76 # data[2]: v +655-656: x 76 # data[3]: v +656-657: x 76 # data[4]: v +657-658: x 76 # data[5]: v +658-659: x 76 # data[6]: v +659-660: x 76 # data[7]: v +660-661: x 76 # data[8]: v +661-662: x 76 # data[9]: v +662-663: x 76 # data[10]: v +663-664: x 76 # data[11]: v +664-665: x 76 # data[12]: v +665-666: x 76 # data[13]: v +666-667: x 76 # data[14]: v +667-668: x 76 # data[15]: v +668-669: x 76 # data[16]: v +669-670: x 76 # data[17]: v +670-671: x 76 # data[18]: v +671-672: x 76 # data[19]: v +672-673: x 76 # data[20]: v +673-674: x 76 # data[21]: v +674-675: x 76 # data[22]: v +675-676: x 76 # data[23]: v +676-677: x 76 # data[24]: v +677-678: x 76 # data[25]: v +678-679: x 76 # data[26]: v +679-680: x 76 # data[27]: v +680-681: x 76 # data[28]: v +681-682: x 76 # data[29]: v +682-683: x 76 # data[30]: v +683-684: x 76 # data[31]: v +684-685: x 76 # data[32]: v +685-686: x 76 # data[33]: v +686-687: x 76 # data[34]: v +687-688: x 76 # data[35]: v +688-689: x 76 # data[36]: v +689-690: x 76 # data[37]: v +690-691: x 76 # data[38]: v +691-692: x 76 # data[39]: v +692-693: x 76 # data[40]: v +693-694: x 76 # data[41]: v +694-695: x 76 # data[42]: v +695-696: x 76 # data[43]: v +696-697: x 76 # data[44]: v +697-698: x 76 # data[45]: v +698-699: x 76 # data[46]: v +699-700: x 76 # data[47]: v +700-701: x 76 # data[48]: v +701-702: x 76 # data[49]: v +702-703: x 76 # data[50]: v +703-704: x 76 # data[51]: v +704-705: x 76 # data[52]: v +705-706: x 76 # data[53]: v +706-707: x 76 # data[54]: v +707-708: x 76 # data[55]: v +708-709: x 76 # data[56]: v +709-710: x 76 # data[57]: v +710-711: x 76 # data[58]: v +711-712: x 76 # data[59]: v +712-713: x 76 # data[60]: v +713-714: x 76 # data[61]: v +714-715: x 76 # data[62]: v +715-716: x 76 # data[63]: v +716-717: x 76 # data[64]: v +717-718: x 76 # data[65]: v # data for column 5 -730-736: x 000000000000 # padding to align to 64-bit boundary -736-744: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 -744-752: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 1 -752-760: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +718-720: x 0000 # padding to align to 64-bit boundary +720-728: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 +728-736: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 1 +736-744: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 iter seek-ge backache diff --git a/sstable/colblk/testdata/data_block/external_value b/sstable/colblk/testdata/data_block/external_value index fe9b689e1c..6fe407e0ef 100644 --- a/sstable/colblk/testdata/data_block/external_value +++ b/sstable/colblk/testdata/data_block/external_value @@ -3,7 +3,7 @@ init size=1: 0: prefixes: prefixbytes(16): 0 keys 1: suffixes: bytes: 0 rows set; 0 bytes in data -2: trailers: uint64: 0 rows +2: trailers: uint: 0 rows 3: prefix changed: bitmap 4: values: bytes: 0 rows set; 0 bytes in data 5: is-value-ext: bitmap @@ -30,10 +30,10 @@ blockprefix_kiwi@99#0,SET:valueHandle-kiwi99 blockprefix_kiwi@98#0,SET:valueHandle-kiwi98 blockprefix_lemon@92#0,DEL: ---- -size=665: +size=641: 0: prefixes: prefixbytes(16): 20 keys 1: suffixes: bytes: 20 rows set; 54 bytes in data -2: trailers: uint64: 20 rows +2: trailers: uint: 20 rows 3: prefix changed: bitmap 4: values: bytes: 20 rows set; 331 bytes in data 5: is-value-ext: bitmap @@ -47,228 +47,222 @@ finish 005-007: x 0600 # 6 columns 007-011: x 14000000 # 20 rows # column 0 -011-012: b 00000111 # prefixbytes +011-012: b 00000100 # prefixbytes 012-016: x 29000000 # page start 41 # column 1 -016-017: b 00000110 # bytes -017-021: x 72000000 # page start 114 +016-017: b 00000011 # bytes +017-021: x 6e000000 # page start 110 # column 2 -021-022: b 00000101 # uint64 -022-026: x c2000000 # page start 194 +021-022: b 00000010 # uint +022-026: x ba000000 # page start 186 # column 3 026-027: b 00000001 # bool -027-031: x f4000000 # page start 244 +027-031: x e4000000 # page start 228 # column 4 -031-032: b 00000110 # bytes -032-036: x 08010000 # page start 264 +031-032: b 00000011 # bytes +032-036: x f8000000 # page start 248 # column 5 036-037: b 00000001 # bool -037-041: x 83020000 # page start 643 +037-041: x 6f020000 # page start 623 # data for column 0 # PrefixBytes 041-042: x 04 # bundleSize: 16 # Offsets table -042-043: x 02 # delta encoding: delta8 -043-047: x 00000000 # 32-bit constant: 0 -047-048: x 0c # data[0] = 12 [82 overall] -048-049: x 0c # data[1] = 12 [82 overall] -049-050: x 11 # data[2] = 17 [87 overall] -050-051: x 11 # data[3] = 17 [87 overall] -051-052: x 11 # data[4] = 17 [87 overall] -052-053: x 11 # data[5] = 17 [87 overall] -053-054: x 17 # data[6] = 23 [93 overall] -054-055: x 17 # data[7] = 23 [93 overall] -055-056: x 17 # data[8] = 23 [93 overall] -056-057: x 17 # data[9] = 23 [93 overall] -057-058: x 17 # data[10] = 23 [93 overall] -058-059: x 17 # data[11] = 23 [93 overall] -059-060: x 1e # data[12] = 30 [100 overall] -060-061: x 1e # data[13] = 30 [100 overall] -061-062: x 1e # data[14] = 30 [100 overall] -062-063: x 1e # data[15] = 30 [100 overall] -063-064: x 1e # data[16] = 30 [100 overall] -064-065: x 1e # data[17] = 30 [100 overall] -065-066: x 1e # data[18] = 30 [100 overall] -066-067: x 23 # data[19] = 35 [105 overall] -067-068: x 27 # data[20] = 39 [109 overall] -068-069: x 27 # data[21] = 39 [109 overall] -069-070: x 2c # data[22] = 44 [114 overall] +042-043: x 01 # encoding: 1b +043-044: x 0c # data[0] = 12 [78 overall] +044-045: x 0c # data[1] = 12 [78 overall] +045-046: x 11 # data[2] = 17 [83 overall] +046-047: x 11 # data[3] = 17 [83 overall] +047-048: x 11 # data[4] = 17 [83 overall] +048-049: x 11 # data[5] = 17 [83 overall] +049-050: x 17 # data[6] = 23 [89 overall] +050-051: x 17 # data[7] = 23 [89 overall] +051-052: x 17 # data[8] = 23 [89 overall] +052-053: x 17 # data[9] = 23 [89 overall] +053-054: x 17 # data[10] = 23 [89 overall] +054-055: x 17 # data[11] = 23 [89 overall] +055-056: x 1e # data[12] = 30 [96 overall] +056-057: x 1e # data[13] = 30 [96 overall] +057-058: x 1e # data[14] = 30 [96 overall] +058-059: x 1e # data[15] = 30 [96 overall] +059-060: x 1e # data[16] = 30 [96 overall] +060-061: x 1e # data[17] = 30 [96 overall] +061-062: x 1e # data[18] = 30 [96 overall] +062-063: x 23 # data[19] = 35 [101 overall] +063-064: x 27 # data[20] = 39 [105 overall] +064-065: x 27 # data[21] = 39 [105 overall] +065-066: x 2c # data[22] = 44 [110 overall] # Data -070-080: x 626c6f636b7072656669 # data[00]: blockprefix_ (block prefix) -080-082: x 785f # (continued...) -082-082: x # data[01]: ............ (bundle prefix) -082-087: x 6170706c65 # data[02]: ............apple -087-087: x # data[03]: ................. -087-087: x # data[04]: ................. -087-087: x # data[05]: ................. -087-093: x 62616e616e61 # data[06]: ............banana -093-093: x # data[07]: .................. -093-093: x # data[08]: .................. -093-093: x # data[09]: .................. -093-093: x # data[10]: .................. -093-093: x # data[11]: .................. -093-100: x 636f636f6e7574 # data[12]: ............coconut -100-100: x # data[13]: ................... -100-100: x # data[14]: ................... -100-100: x # data[15]: ................... -100-100: x # data[16]: ................... -100-100: x # data[17]: ................... -100-100: x # data[18]: ............ (bundle prefix) -100-105: x 6775617661 # data[19]: ............guava -105-109: x 6b697769 # data[20]: ............kiwi -109-109: x # data[21]: ................ -109-114: x 6c656d6f6e # data[22]: ............lemon +066-076: x 626c6f636b7072656669 # data[00]: blockprefix_ (block prefix) +076-078: x 785f # (continued...) +078-078: x # data[01]: ............ (bundle prefix) +078-083: x 6170706c65 # data[02]: ............apple +083-083: x # data[03]: ................. +083-083: x # data[04]: ................. +083-083: x # data[05]: ................. +083-089: x 62616e616e61 # data[06]: ............banana +089-089: x # data[07]: .................. +089-089: x # data[08]: .................. +089-089: x # data[09]: .................. +089-089: x # data[10]: .................. +089-089: x # data[11]: .................. +089-096: x 636f636f6e7574 # data[12]: ............coconut +096-096: x # data[13]: ................... +096-096: x # data[14]: ................... +096-096: x # data[15]: ................... +096-096: x # data[16]: ................... +096-096: x # data[17]: ................... +096-096: x # data[18]: ............ (bundle prefix) +096-101: x 6775617661 # data[19]: ............guava +101-105: x 6b697769 # data[20]: ............kiwi +105-105: x # data[21]: ................ +105-110: x 6c656d6f6e # data[22]: ............lemon # data for column 1 # rawbytes # offsets table -114-115: x 02 # delta encoding: delta8 -115-119: x 00000000 # 32-bit constant: 0 -119-120: x 00 # data[0] = 0 [140 overall] -120-121: x 03 # data[1] = 3 [143 overall] -121-122: x 06 # data[2] = 6 [146 overall] -122-123: x 09 # data[3] = 9 [149 overall] -123-124: x 0c # data[4] = 12 [152 overall] -124-125: x 0f # data[5] = 15 [155 overall] -125-126: x 12 # data[6] = 18 [158 overall] -126-127: x 15 # data[7] = 21 [161 overall] -127-128: x 18 # data[8] = 24 [164 overall] -128-129: x 1a # data[9] = 26 [166 overall] -129-130: x 1c # data[10] = 28 [168 overall] -130-131: x 1c # data[11] = 28 [168 overall] -131-132: x 1f # data[12] = 31 [171 overall] -132-133: x 22 # data[13] = 34 [174 overall] -133-134: x 25 # data[14] = 37 [177 overall] -134-135: x 28 # data[15] = 40 [180 overall] -135-136: x 2a # data[16] = 42 [182 overall] -136-137: x 2d # data[17] = 45 [185 overall] -137-138: x 30 # data[18] = 48 [188 overall] -138-139: x 33 # data[19] = 51 [191 overall] -139-140: x 36 # data[20] = 54 [194 overall] +110-111: x 01 # encoding: 1b +111-112: x 00 # data[0] = 0 [132 overall] +112-113: x 03 # data[1] = 3 [135 overall] +113-114: x 06 # data[2] = 6 [138 overall] +114-115: x 09 # data[3] = 9 [141 overall] +115-116: x 0c # data[4] = 12 [144 overall] +116-117: x 0f # data[5] = 15 [147 overall] +117-118: x 12 # data[6] = 18 [150 overall] +118-119: x 15 # data[7] = 21 [153 overall] +119-120: x 18 # data[8] = 24 [156 overall] +120-121: x 1a # data[9] = 26 [158 overall] +121-122: x 1c # data[10] = 28 [160 overall] +122-123: x 1c # data[11] = 28 [160 overall] +123-124: x 1f # data[12] = 31 [163 overall] +124-125: x 22 # data[13] = 34 [166 overall] +125-126: x 25 # data[14] = 37 [169 overall] +126-127: x 28 # data[15] = 40 [172 overall] +127-128: x 2a # data[16] = 42 [174 overall] +128-129: x 2d # data[17] = 45 [177 overall] +129-130: x 30 # data[18] = 48 [180 overall] +130-131: x 33 # data[19] = 51 [183 overall] +131-132: x 36 # data[20] = 54 [186 overall] # data -140-143: x 403938 # data[0]: @98 -143-146: x 403532 # data[1]: @52 -146-149: x 403233 # data[2]: @23 -149-152: x 403131 # data[3]: @11 -152-155: x 403934 # data[4]: @94 -155-158: x 403933 # data[5]: @93 -158-161: x 403933 # data[6]: @93 -161-164: x 403732 # data[7]: @72 -164-166: x 4039 # data[8]: @9 -166-168: x 4031 # data[9]: @1 -168-168: x # data[10]: -168-171: x 403932 # data[11]: @92 -171-174: x 403335 # data[12]: @35 -174-177: x 403232 # data[13]: @22 -177-180: x 403231 # data[14]: @21 -180-182: x 4031 # data[15]: @1 -182-185: x 403939 # data[16]: @99 -185-188: x 403939 # data[17]: @99 -188-191: x 403938 # data[18]: @98 -191-194: x 403932 # data[19]: @92 +132-135: x 403938 # data[0]: @98 +135-138: x 403532 # data[1]: @52 +138-141: x 403233 # data[2]: @23 +141-144: x 403131 # data[3]: @11 +144-147: x 403934 # data[4]: @94 +147-150: x 403933 # data[5]: @93 +150-153: x 403933 # data[6]: @93 +153-156: x 403732 # data[7]: @72 +156-158: x 4039 # data[8]: @9 +158-160: x 4031 # data[9]: @1 +160-160: x # data[10]: +160-163: x 403932 # data[11]: @92 +163-166: x 403335 # data[12]: @35 +166-169: x 403232 # data[13]: @22 +169-172: x 403231 # data[14]: @21 +172-174: x 4031 # data[15]: @1 +174-177: x 403939 # data[16]: @99 +177-180: x 403939 # data[17]: @99 +180-183: x 403938 # data[18]: @98 +183-186: x 403932 # data[19]: @92 # data for column 2 -194-195: x 03 # delta encoding: delta16 -195-203: x 0000000000000000 # 64-bit constant: 0 -# padding -203-204: x 00 # aligning to 16-bit boundary -204-206: x 0100 # data[0] = 1 -206-208: x 0100 # data[1] = 1 -208-210: x 0100 # data[2] = 1 -210-212: x 1200 # data[3] = 18 -212-214: x 12f5 # data[4] = 62738 -214-216: x 00f4 # data[5] = 62464 -216-218: x 12dd # data[6] = 56594 -218-220: x 1200 # data[7] = 18 -220-222: x 0100 # data[8] = 1 -222-224: x 0100 # data[9] = 1 -224-226: x 0100 # data[10] = 1 -226-228: x 0100 # data[11] = 1 -228-230: x 0100 # data[12] = 1 -230-232: x 0100 # data[13] = 1 -232-234: x 0100 # data[14] = 1 -234-236: x 0100 # data[15] = 1 -236-238: x 0100 # data[16] = 1 -238-240: x 0100 # data[17] = 1 -240-242: x 0100 # data[18] = 1 -242-244: x 0000 # data[19] = 0 +186-187: x 02 # encoding: 2b +187-188: x 00 # padding (aligning to 16-bit boundary) +188-190: x 0100 # data[0] = 1 +190-192: x 0100 # data[1] = 1 +192-194: x 0100 # data[2] = 1 +194-196: x 1200 # data[3] = 18 +196-198: x 12f5 # data[4] = 62738 +198-200: x 00f4 # data[5] = 62464 +200-202: x 12dd # data[6] = 56594 +202-204: x 1200 # data[7] = 18 +204-206: x 0100 # data[8] = 1 +206-208: x 0100 # data[9] = 1 +208-210: x 0100 # data[10] = 1 +210-212: x 0100 # data[11] = 1 +212-214: x 0100 # data[12] = 1 +214-216: x 0100 # data[13] = 1 +216-218: x 0100 # data[14] = 1 +218-220: x 0100 # data[15] = 1 +220-222: x 0100 # data[16] = 1 +222-224: x 0100 # data[17] = 1 +224-226: x 0100 # data[18] = 1 +226-228: x 0000 # data[19] = 0 # data for column 3 -244-248: x 00000000 # padding to align to 64-bit boundary -248-256: b 0001000100000100000010110000000000000000000000000000000000000000 # bitmap word 0 -256-264: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +228-232: x 00000000 # padding to align to 64-bit boundary +232-240: b 0001000100000100000010110000000000000000000000000000000000000000 # bitmap word 0 +240-248: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 # data for column 4 # rawbytes # offsets table -264-265: x 03 # delta encoding: delta16 -265-269: x 00000000 # 32-bit constant: 0 -# padding -269-270: x 00 # aligning to 16-bit boundary -270-272: x 0000 # data[0] = 0 [312 overall] -272-274: x 0700 # data[1] = 7 [319 overall] -274-276: x 1b00 # data[2] = 27 [339 overall] -276-278: x 2f00 # data[3] = 47 [359 overall] -278-280: x 4300 # data[4] = 67 [379 overall] -280-282: x 4b00 # data[5] = 75 [387 overall] -282-284: x 4b00 # data[6] = 75 [387 overall] -284-286: x 6000 # data[7] = 96 [408 overall] -286-288: x 7500 # data[8] = 117 [429 overall] -288-290: x 8900 # data[9] = 137 [449 overall] -290-292: x 9d00 # data[10] = 157 [469 overall] -292-294: x a400 # data[11] = 164 [476 overall] -294-296: x ba00 # data[12] = 186 [498 overall] -296-298: x d000 # data[13] = 208 [520 overall] -298-300: x e600 # data[14] = 230 [542 overall] -300-302: x fc00 # data[15] = 252 [564 overall] -302-304: x 1101 # data[16] = 273 [585 overall] -304-306: x 2501 # data[17] = 293 [605 overall] -306-308: x 3801 # data[18] = 312 [624 overall] -308-310: x 4b01 # data[19] = 331 [643 overall] -310-312: x 4b01 # data[20] = 331 [643 overall] +248-249: x 02 # encoding: 2b +249-250: x 00 # padding (aligning to 16-bit boundary) +250-252: x 0000 # data[0] = 0 [292 overall] +252-254: x 0700 # data[1] = 7 [299 overall] +254-256: x 1b00 # data[2] = 27 [319 overall] +256-258: x 2f00 # data[3] = 47 [339 overall] +258-260: x 4300 # data[4] = 67 [359 overall] +260-262: x 4b00 # data[5] = 75 [367 overall] +262-264: x 4b00 # data[6] = 75 [367 overall] +264-266: x 6000 # data[7] = 96 [388 overall] +266-268: x 7500 # data[8] = 117 [409 overall] +268-270: x 8900 # data[9] = 137 [429 overall] +270-272: x 9d00 # data[10] = 157 [449 overall] +272-274: x a400 # data[11] = 164 [456 overall] +274-276: x ba00 # data[12] = 186 [478 overall] +276-278: x d000 # data[13] = 208 [500 overall] +278-280: x e600 # data[14] = 230 [522 overall] +280-282: x fc00 # data[15] = 252 [544 overall] +282-284: x 1101 # data[16] = 273 [565 overall] +284-286: x 2501 # data[17] = 293 [585 overall] +286-288: x 3801 # data[18] = 312 [604 overall] +288-290: x 4b01 # data[19] = 331 [623 overall] +290-292: x 4b01 # data[20] = 331 [623 overall] # data -312-319: x 6170706c653938 # data[0]: apple98 -319-329: x a076616c756548616e64 # data[1]: "\xa0valueHandle-apple52" -329-339: x 6c652d6170706c653532 # (continued...) -339-349: x a076616c756548616e64 # data[2]: "\xa0valueHandle-apple23" -349-359: x 6c652d6170706c653233 # (continued...) -359-369: x a076616c756548616e64 # data[3]: "\xa0valueHandle-apple11" -369-379: x 6c652d6170706c653131 # (continued...) -379-387: x 62616e616e613934 # data[4]: banana94 -387-387: x # data[5]: -387-397: x a076616c756548616e64 # data[6]: "\xa0valueHandle-banana93" -397-407: x 6c652d62616e616e6139 # (continued...) -407-408: x 33 # (continued...) -408-418: x a076616c756548616e64 # data[7]: "\xa0valueHandle-banana72" -418-428: x 6c652d62616e616e6137 # (continued...) -428-429: x 32 # (continued...) -429-439: x a076616c756548616e64 # data[8]: "\xa0valueHandle-banana9" -439-449: x 6c652d62616e616e6139 # (continued...) -449-459: x a076616c756548616e64 # data[9]: "\xa0valueHandle-banana1" -459-469: x 6c652d62616e616e6131 # (continued...) -469-476: x 636f636f6e7574 # data[10]: coconut -476-486: x a076616c756548616e64 # data[11]: "\xa0valueHandle-coconut92" -486-496: x 6c652d636f636f6e7574 # (continued...) -496-498: x 3932 # (continued...) -498-508: x a076616c756548616e64 # data[12]: "\xa0valueHandle-coconut35" -508-518: x 6c652d636f636f6e7574 # (continued...) -518-520: x 3335 # (continued...) -520-530: x a076616c756548616e64 # data[13]: "\xa0valueHandle-coconut22" -530-540: x 6c652d636f636f6e7574 # (continued...) -540-542: x 3232 # (continued...) -542-552: x a076616c756548616e64 # data[14]: "\xa0valueHandle-coconut21" -552-562: x 6c652d636f636f6e7574 # (continued...) -562-564: x 3231 # (continued...) -564-574: x a076616c756548616e64 # data[15]: "\xa0valueHandle-coconut1" -574-584: x 6c652d636f636f6e7574 # (continued...) -584-585: x 31 # (continued...) -585-595: x 8076616c756548616e64 # data[16]: "\x80valueHandle-guava99" -595-605: x 6c652d67756176613939 # (continued...) -605-615: x 8076616c756548616e64 # data[17]: "\x80valueHandle-kiwi99" -615-624: x 6c652d6b6977693939 # (continued...) -624-634: x a076616c756548616e64 # data[18]: "\xa0valueHandle-kiwi98" -634-643: x 6c652d6b6977693938 # (continued...) -643-643: x # data[19]: +292-299: x 6170706c653938 # data[0]: apple98 +299-309: x a076616c756548616e64 # data[1]: "\xa0valueHandle-apple52" +309-319: x 6c652d6170706c653532 # (continued...) +319-329: x a076616c756548616e64 # data[2]: "\xa0valueHandle-apple23" +329-339: x 6c652d6170706c653233 # (continued...) +339-349: x a076616c756548616e64 # data[3]: "\xa0valueHandle-apple11" +349-359: x 6c652d6170706c653131 # (continued...) +359-367: x 62616e616e613934 # data[4]: banana94 +367-367: x # data[5]: +367-377: x a076616c756548616e64 # data[6]: "\xa0valueHandle-banana93" +377-387: x 6c652d62616e616e6139 # (continued...) +387-388: x 33 # (continued...) +388-398: x a076616c756548616e64 # data[7]: "\xa0valueHandle-banana72" +398-408: x 6c652d62616e616e6137 # (continued...) +408-409: x 32 # (continued...) +409-419: x a076616c756548616e64 # data[8]: "\xa0valueHandle-banana9" +419-429: x 6c652d62616e616e6139 # (continued...) +429-439: x a076616c756548616e64 # data[9]: "\xa0valueHandle-banana1" +439-449: x 6c652d62616e616e6131 # (continued...) +449-456: x 636f636f6e7574 # data[10]: coconut +456-466: x a076616c756548616e64 # data[11]: "\xa0valueHandle-coconut92" +466-476: x 6c652d636f636f6e7574 # (continued...) +476-478: x 3932 # (continued...) +478-488: x a076616c756548616e64 # data[12]: "\xa0valueHandle-coconut35" +488-498: x 6c652d636f636f6e7574 # (continued...) +498-500: x 3335 # (continued...) +500-510: x a076616c756548616e64 # data[13]: "\xa0valueHandle-coconut22" +510-520: x 6c652d636f636f6e7574 # (continued...) +520-522: x 3232 # (continued...) +522-532: x a076616c756548616e64 # data[14]: "\xa0valueHandle-coconut21" +532-542: x 6c652d636f636f6e7574 # (continued...) +542-544: x 3231 # (continued...) +544-554: x a076616c756548616e64 # data[15]: "\xa0valueHandle-coconut1" +554-564: x 6c652d636f636f6e7574 # (continued...) +564-565: x 31 # (continued...) +565-575: x 8076616c756548616e64 # data[16]: "\x80valueHandle-guava99" +575-585: x 6c652d67756176613939 # (continued...) +585-595: x 8076616c756548616e64 # data[17]: "\x80valueHandle-kiwi99" +595-604: x 6c652d6b6977693939 # (continued...) +604-614: x a076616c756548616e64 # data[18]: "\xa0valueHandle-kiwi98" +614-623: x 6c652d6b6977693938 # (continued...) +623-623: x # data[19]: # data for column 5 -643-648: x 0000000000 # padding to align to 64-bit boundary -648-656: b 1100111011111011000001110000000000000000000000000000000000000000 # bitmap word 0 -656-664: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +623-624: x 00 # padding to align to 64-bit boundary +624-632: b 1100111011111011000001110000000000000000000000000000000000000000 # bitmap word 0 +632-640: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 # Scan across the block using next. iter diff --git a/sstable/colblk/testdata/data_block/next_prefix b/sstable/colblk/testdata/data_block/next_prefix index 006ebf910b..1b2c42466d 100644 --- a/sstable/colblk/testdata/data_block/next_prefix +++ b/sstable/colblk/testdata/data_block/next_prefix @@ -3,7 +3,7 @@ init size=1: 0: prefixes: prefixbytes(16): 0 keys 1: suffixes: bytes: 0 rows set; 0 bytes in data -2: trailers: uint64: 0 rows +2: trailers: uint: 0 rows 3: prefix changed: bitmap 4: values: bytes: 0 rows set; 0 bytes in data 5: is-value-ext: bitmap @@ -30,10 +30,10 @@ blockprefix_kiwi@99#0,SET:kiwi99 blockprefix_kiwi@98#0,SET:kiwi98 blockprefix_lemon@92#0,DEL: ---- -size=449: +size=425: 0: prefixes: prefixbytes(16): 20 keys 1: suffixes: bytes: 20 rows set; 54 bytes in data -2: trailers: uint64: 20 rows +2: trailers: uint: 20 rows 3: prefix changed: bitmap 4: values: bytes: 20 rows set; 136 bytes in data 5: is-value-ext: bitmap @@ -47,204 +47,199 @@ finish 005-007: x 0600 # 6 columns 007-011: x 14000000 # 20 rows # column 0 -011-012: b 00000111 # prefixbytes +011-012: b 00000100 # prefixbytes 012-016: x 29000000 # page start 41 # column 1 -016-017: b 00000110 # bytes -017-021: x 72000000 # page start 114 +016-017: b 00000011 # bytes +017-021: x 6e000000 # page start 110 # column 2 -021-022: b 00000101 # uint64 -022-026: x c2000000 # page start 194 +021-022: b 00000010 # uint +022-026: x ba000000 # page start 186 # column 3 026-027: b 00000001 # bool -027-031: x f4000000 # page start 244 +027-031: x e4000000 # page start 228 # column 4 -031-032: b 00000110 # bytes -032-036: x 08010000 # page start 264 +031-032: b 00000011 # bytes +032-036: x f8000000 # page start 248 # column 5 036-037: b 00000001 # bool -037-041: x aa010000 # page start 426 +037-041: x 96010000 # page start 406 # data for column 0 # PrefixBytes 041-042: x 04 # bundleSize: 16 # Offsets table -042-043: x 02 # delta encoding: delta8 -043-047: x 00000000 # 32-bit constant: 0 -047-048: x 0c # data[0] = 12 [82 overall] -048-049: x 0c # data[1] = 12 [82 overall] -049-050: x 11 # data[2] = 17 [87 overall] -050-051: x 11 # data[3] = 17 [87 overall] -051-052: x 11 # data[4] = 17 [87 overall] -052-053: x 11 # data[5] = 17 [87 overall] -053-054: x 17 # data[6] = 23 [93 overall] -054-055: x 17 # data[7] = 23 [93 overall] -055-056: x 17 # data[8] = 23 [93 overall] -056-057: x 17 # data[9] = 23 [93 overall] -057-058: x 17 # data[10] = 23 [93 overall] -058-059: x 17 # data[11] = 23 [93 overall] -059-060: x 1e # data[12] = 30 [100 overall] -060-061: x 1e # data[13] = 30 [100 overall] -061-062: x 1e # data[14] = 30 [100 overall] -062-063: x 1e # data[15] = 30 [100 overall] -063-064: x 1e # data[16] = 30 [100 overall] -064-065: x 1e # data[17] = 30 [100 overall] -065-066: x 1e # data[18] = 30 [100 overall] -066-067: x 23 # data[19] = 35 [105 overall] -067-068: x 27 # data[20] = 39 [109 overall] -068-069: x 27 # data[21] = 39 [109 overall] -069-070: x 2c # data[22] = 44 [114 overall] +042-043: x 01 # encoding: 1b +043-044: x 0c # data[0] = 12 [78 overall] +044-045: x 0c # data[1] = 12 [78 overall] +045-046: x 11 # data[2] = 17 [83 overall] +046-047: x 11 # data[3] = 17 [83 overall] +047-048: x 11 # data[4] = 17 [83 overall] +048-049: x 11 # data[5] = 17 [83 overall] +049-050: x 17 # data[6] = 23 [89 overall] +050-051: x 17 # data[7] = 23 [89 overall] +051-052: x 17 # data[8] = 23 [89 overall] +052-053: x 17 # data[9] = 23 [89 overall] +053-054: x 17 # data[10] = 23 [89 overall] +054-055: x 17 # data[11] = 23 [89 overall] +055-056: x 1e # data[12] = 30 [96 overall] +056-057: x 1e # data[13] = 30 [96 overall] +057-058: x 1e # data[14] = 30 [96 overall] +058-059: x 1e # data[15] = 30 [96 overall] +059-060: x 1e # data[16] = 30 [96 overall] +060-061: x 1e # data[17] = 30 [96 overall] +061-062: x 1e # data[18] = 30 [96 overall] +062-063: x 23 # data[19] = 35 [101 overall] +063-064: x 27 # data[20] = 39 [105 overall] +064-065: x 27 # data[21] = 39 [105 overall] +065-066: x 2c # data[22] = 44 [110 overall] # Data -070-080: x 626c6f636b7072656669 # data[00]: blockprefix_ (block prefix) -080-082: x 785f # (continued...) -082-082: x # data[01]: ............ (bundle prefix) -082-087: x 6170706c65 # data[02]: ............apple -087-087: x # data[03]: ................. -087-087: x # data[04]: ................. -087-087: x # data[05]: ................. -087-093: x 62616e616e61 # data[06]: ............banana -093-093: x # data[07]: .................. -093-093: x # data[08]: .................. -093-093: x # data[09]: .................. -093-093: x # data[10]: .................. -093-093: x # data[11]: .................. -093-100: x 636f636f6e7574 # data[12]: ............coconut -100-100: x # data[13]: ................... -100-100: x # data[14]: ................... -100-100: x # data[15]: ................... -100-100: x # data[16]: ................... -100-100: x # data[17]: ................... -100-100: x # data[18]: ............ (bundle prefix) -100-105: x 6775617661 # data[19]: ............guava -105-109: x 6b697769 # data[20]: ............kiwi -109-109: x # data[21]: ................ -109-114: x 6c656d6f6e # data[22]: ............lemon +066-076: x 626c6f636b7072656669 # data[00]: blockprefix_ (block prefix) +076-078: x 785f # (continued...) +078-078: x # data[01]: ............ (bundle prefix) +078-083: x 6170706c65 # data[02]: ............apple +083-083: x # data[03]: ................. +083-083: x # data[04]: ................. +083-083: x # data[05]: ................. +083-089: x 62616e616e61 # data[06]: ............banana +089-089: x # data[07]: .................. +089-089: x # data[08]: .................. +089-089: x # data[09]: .................. +089-089: x # data[10]: .................. +089-089: x # data[11]: .................. +089-096: x 636f636f6e7574 # data[12]: ............coconut +096-096: x # data[13]: ................... +096-096: x # data[14]: ................... +096-096: x # data[15]: ................... +096-096: x # data[16]: ................... +096-096: x # data[17]: ................... +096-096: x # data[18]: ............ (bundle prefix) +096-101: x 6775617661 # data[19]: ............guava +101-105: x 6b697769 # data[20]: ............kiwi +105-105: x # data[21]: ................ +105-110: x 6c656d6f6e # data[22]: ............lemon # data for column 1 # rawbytes # offsets table -114-115: x 02 # delta encoding: delta8 -115-119: x 00000000 # 32-bit constant: 0 -119-120: x 00 # data[0] = 0 [140 overall] -120-121: x 03 # data[1] = 3 [143 overall] -121-122: x 06 # data[2] = 6 [146 overall] -122-123: x 09 # data[3] = 9 [149 overall] -123-124: x 0c # data[4] = 12 [152 overall] -124-125: x 0f # data[5] = 15 [155 overall] -125-126: x 12 # data[6] = 18 [158 overall] -126-127: x 15 # data[7] = 21 [161 overall] -127-128: x 18 # data[8] = 24 [164 overall] -128-129: x 1a # data[9] = 26 [166 overall] -129-130: x 1c # data[10] = 28 [168 overall] -130-131: x 1c # data[11] = 28 [168 overall] -131-132: x 1f # data[12] = 31 [171 overall] -132-133: x 22 # data[13] = 34 [174 overall] -133-134: x 25 # data[14] = 37 [177 overall] -134-135: x 28 # data[15] = 40 [180 overall] -135-136: x 2a # data[16] = 42 [182 overall] -136-137: x 2d # data[17] = 45 [185 overall] -137-138: x 30 # data[18] = 48 [188 overall] -138-139: x 33 # data[19] = 51 [191 overall] -139-140: x 36 # data[20] = 54 [194 overall] +110-111: x 01 # encoding: 1b +111-112: x 00 # data[0] = 0 [132 overall] +112-113: x 03 # data[1] = 3 [135 overall] +113-114: x 06 # data[2] = 6 [138 overall] +114-115: x 09 # data[3] = 9 [141 overall] +115-116: x 0c # data[4] = 12 [144 overall] +116-117: x 0f # data[5] = 15 [147 overall] +117-118: x 12 # data[6] = 18 [150 overall] +118-119: x 15 # data[7] = 21 [153 overall] +119-120: x 18 # data[8] = 24 [156 overall] +120-121: x 1a # data[9] = 26 [158 overall] +121-122: x 1c # data[10] = 28 [160 overall] +122-123: x 1c # data[11] = 28 [160 overall] +123-124: x 1f # data[12] = 31 [163 overall] +124-125: x 22 # data[13] = 34 [166 overall] +125-126: x 25 # data[14] = 37 [169 overall] +126-127: x 28 # data[15] = 40 [172 overall] +127-128: x 2a # data[16] = 42 [174 overall] +128-129: x 2d # data[17] = 45 [177 overall] +129-130: x 30 # data[18] = 48 [180 overall] +130-131: x 33 # data[19] = 51 [183 overall] +131-132: x 36 # data[20] = 54 [186 overall] # data -140-143: x 403938 # data[0]: @98 -143-146: x 403532 # data[1]: @52 -146-149: x 403233 # data[2]: @23 -149-152: x 403131 # data[3]: @11 -152-155: x 403934 # data[4]: @94 -155-158: x 403933 # data[5]: @93 -158-161: x 403933 # data[6]: @93 -161-164: x 403732 # data[7]: @72 -164-166: x 4039 # data[8]: @9 -166-168: x 4031 # data[9]: @1 -168-168: x # data[10]: -168-171: x 403932 # data[11]: @92 -171-174: x 403335 # data[12]: @35 -174-177: x 403232 # data[13]: @22 -177-180: x 403231 # data[14]: @21 -180-182: x 4031 # data[15]: @1 -182-185: x 403939 # data[16]: @99 -185-188: x 403939 # data[17]: @99 -188-191: x 403938 # data[18]: @98 -191-194: x 403932 # data[19]: @92 +132-135: x 403938 # data[0]: @98 +135-138: x 403532 # data[1]: @52 +138-141: x 403233 # data[2]: @23 +141-144: x 403131 # data[3]: @11 +144-147: x 403934 # data[4]: @94 +147-150: x 403933 # data[5]: @93 +150-153: x 403933 # data[6]: @93 +153-156: x 403732 # data[7]: @72 +156-158: x 4039 # data[8]: @9 +158-160: x 4031 # data[9]: @1 +160-160: x # data[10]: +160-163: x 403932 # data[11]: @92 +163-166: x 403335 # data[12]: @35 +166-169: x 403232 # data[13]: @22 +169-172: x 403231 # data[14]: @21 +172-174: x 4031 # data[15]: @1 +174-177: x 403939 # data[16]: @99 +177-180: x 403939 # data[17]: @99 +180-183: x 403938 # data[18]: @98 +183-186: x 403932 # data[19]: @92 # data for column 2 -194-195: x 03 # delta encoding: delta16 -195-203: x 0000000000000000 # 64-bit constant: 0 -# padding -203-204: x 00 # aligning to 16-bit boundary -204-206: x 0100 # data[0] = 1 -206-208: x 0100 # data[1] = 1 -208-210: x 0100 # data[2] = 1 -210-212: x 1200 # data[3] = 18 -212-214: x 12f5 # data[4] = 62738 -214-216: x 00f4 # data[5] = 62464 -216-218: x 12dd # data[6] = 56594 -218-220: x 1200 # data[7] = 18 -220-222: x 0100 # data[8] = 1 -222-224: x 0100 # data[9] = 1 -224-226: x 0100 # data[10] = 1 -226-228: x 0100 # data[11] = 1 -228-230: x 0100 # data[12] = 1 -230-232: x 0100 # data[13] = 1 -232-234: x 0100 # data[14] = 1 -234-236: x 0100 # data[15] = 1 -236-238: x 0100 # data[16] = 1 -238-240: x 0100 # data[17] = 1 -240-242: x 0100 # data[18] = 1 -242-244: x 0000 # data[19] = 0 +186-187: x 02 # encoding: 2b +187-188: x 00 # padding (aligning to 16-bit boundary) +188-190: x 0100 # data[0] = 1 +190-192: x 0100 # data[1] = 1 +192-194: x 0100 # data[2] = 1 +194-196: x 1200 # data[3] = 18 +196-198: x 12f5 # data[4] = 62738 +198-200: x 00f4 # data[5] = 62464 +200-202: x 12dd # data[6] = 56594 +202-204: x 1200 # data[7] = 18 +204-206: x 0100 # data[8] = 1 +206-208: x 0100 # data[9] = 1 +208-210: x 0100 # data[10] = 1 +210-212: x 0100 # data[11] = 1 +212-214: x 0100 # data[12] = 1 +214-216: x 0100 # data[13] = 1 +216-218: x 0100 # data[14] = 1 +218-220: x 0100 # data[15] = 1 +220-222: x 0100 # data[16] = 1 +222-224: x 0100 # data[17] = 1 +224-226: x 0100 # data[18] = 1 +226-228: x 0000 # data[19] = 0 # data for column 3 -244-248: x 00000000 # padding to align to 64-bit boundary -248-256: b 0001000100000100000010110000000000000000000000000000000000000000 # bitmap word 0 -256-264: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +228-232: x 00000000 # padding to align to 64-bit boundary +232-240: b 0001000100000100000010110000000000000000000000000000000000000000 # bitmap word 0 +240-248: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 # data for column 4 # rawbytes # offsets table -264-265: x 02 # delta encoding: delta8 -265-269: x 00000000 # 32-bit constant: 0 -269-270: x 00 # data[0] = 0 [290 overall] -270-271: x 07 # data[1] = 7 [297 overall] -271-272: x 0e # data[2] = 14 [304 overall] -272-273: x 15 # data[3] = 21 [311 overall] -273-274: x 1c # data[4] = 28 [318 overall] -274-275: x 24 # data[5] = 36 [326 overall] -275-276: x 24 # data[6] = 36 [326 overall] -276-277: x 2c # data[7] = 44 [334 overall] -277-278: x 34 # data[8] = 52 [342 overall] -278-279: x 3b # data[9] = 59 [349 overall] -279-280: x 42 # data[10] = 66 [356 overall] -280-281: x 49 # data[11] = 73 [363 overall] -281-282: x 52 # data[12] = 82 [372 overall] -282-283: x 5b # data[13] = 91 [381 overall] -283-284: x 64 # data[14] = 100 [390 overall] -284-285: x 6d # data[15] = 109 [399 overall] -285-286: x 75 # data[16] = 117 [407 overall] -286-287: x 7c # data[17] = 124 [414 overall] -287-288: x 82 # data[18] = 130 [420 overall] -288-289: x 88 # data[19] = 136 [426 overall] -289-290: x 88 # data[20] = 136 [426 overall] +248-249: x 01 # encoding: 1b +249-250: x 00 # data[0] = 0 [270 overall] +250-251: x 07 # data[1] = 7 [277 overall] +251-252: x 0e # data[2] = 14 [284 overall] +252-253: x 15 # data[3] = 21 [291 overall] +253-254: x 1c # data[4] = 28 [298 overall] +254-255: x 24 # data[5] = 36 [306 overall] +255-256: x 24 # data[6] = 36 [306 overall] +256-257: x 2c # data[7] = 44 [314 overall] +257-258: x 34 # data[8] = 52 [322 overall] +258-259: x 3b # data[9] = 59 [329 overall] +259-260: x 42 # data[10] = 66 [336 overall] +260-261: x 49 # data[11] = 73 [343 overall] +261-262: x 52 # data[12] = 82 [352 overall] +262-263: x 5b # data[13] = 91 [361 overall] +263-264: x 64 # data[14] = 100 [370 overall] +264-265: x 6d # data[15] = 109 [379 overall] +265-266: x 75 # data[16] = 117 [387 overall] +266-267: x 7c # data[17] = 124 [394 overall] +267-268: x 82 # data[18] = 130 [400 overall] +268-269: x 88 # data[19] = 136 [406 overall] +269-270: x 88 # data[20] = 136 [406 overall] # data -290-297: x 6170706c653938 # data[0]: apple98 -297-304: x 6170706c653532 # data[1]: apple52 -304-311: x 6170706c653233 # data[2]: apple23 -311-318: x 6170706c653131 # data[3]: apple11 -318-326: x 62616e616e613934 # data[4]: banana94 -326-326: x # data[5]: -326-334: x 62616e616e613933 # data[6]: banana93 -334-342: x 62616e616e613732 # data[7]: banana72 -342-349: x 62616e616e6139 # data[8]: banana9 -349-356: x 62616e616e6131 # data[9]: banana1 -356-363: x 636f636f6e7574 # data[10]: coconut -363-372: x 636f636f6e75743932 # data[11]: coconut92 -372-381: x 636f636f6e75743335 # data[12]: coconut35 -381-390: x 636f636f6e75743232 # data[13]: coconut22 -390-399: x 636f636f6e75743231 # data[14]: coconut21 -399-407: x 636f636f6e757431 # data[15]: coconut1 -407-414: x 67756176613939 # data[16]: guava99 -414-420: x 6b6977693939 # data[17]: kiwi99 -420-426: x 6b6977693938 # data[18]: kiwi98 -426-426: x # data[19]: +270-277: x 6170706c653938 # data[0]: apple98 +277-284: x 6170706c653532 # data[1]: apple52 +284-291: x 6170706c653233 # data[2]: apple23 +291-298: x 6170706c653131 # data[3]: apple11 +298-306: x 62616e616e613934 # data[4]: banana94 +306-306: x # data[5]: +306-314: x 62616e616e613933 # data[6]: banana93 +314-322: x 62616e616e613732 # data[7]: banana72 +322-329: x 62616e616e6139 # data[8]: banana9 +329-336: x 62616e616e6131 # data[9]: banana1 +336-343: x 636f636f6e7574 # data[10]: coconut +343-352: x 636f636f6e75743932 # data[11]: coconut92 +352-361: x 636f636f6e75743335 # data[12]: coconut35 +361-370: x 636f636f6e75743232 # data[13]: coconut22 +370-379: x 636f636f6e75743231 # data[14]: coconut21 +379-387: x 636f636f6e757431 # data[15]: coconut1 +387-394: x 67756176613939 # data[16]: guava99 +394-400: x 6b6977693939 # data[17]: kiwi99 +400-406: x 6b6977693938 # data[18]: kiwi98 +406-406: x # data[19]: # data for column 5 -426-432: x 000000000000 # padding to align to 64-bit boundary -432-440: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 -440-448: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +406-408: x 0000 # padding to align to 64-bit boundary +408-416: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 +416-424: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 # Scan across the block using next prefix. diff --git a/sstable/colblk/testdata/data_block/simple b/sstable/colblk/testdata/data_block/simple index 52e8d71f9c..25bd2e8d5c 100644 --- a/sstable/colblk/testdata/data_block/simple +++ b/sstable/colblk/testdata/data_block/simple @@ -3,7 +3,7 @@ init size=1: 0: prefixes: prefixbytes(16): 0 keys 1: suffixes: bytes: 0 rows set; 0 bytes in data -2: trailers: uint64: 0 rows +2: trailers: uint: 0 rows 3: prefix changed: bitmap 4: values: bytes: 0 rows set; 0 bytes in data 5: is-value-ext: bitmap @@ -16,10 +16,10 @@ c@9#0,SETWITHDEL:coconut c@6#0,SET:cantelope c@1#0,SET:clementine ---- -size=201: +size=177: 0: prefixes: prefixbytes(16): 6 keys 1: suffixes: bytes: 6 rows set; 13 bytes in data -2: trailers: uint64: 6 rows +2: trailers: uint: 6 rows 3: prefix changed: bitmap 4: values: bytes: 6 rows set; 46 bytes in data 5: is-value-ext: bitmap @@ -27,10 +27,10 @@ size=201: write d@11#0,DEL: ---- -size=209: +size=185: 0: prefixes: prefixbytes(16): 7 keys 1: suffixes: bytes: 7 rows set; 16 bytes in data -2: trailers: uint64: 7 rows +2: trailers: uint: 7 rows 3: prefix changed: bitmap 4: values: bytes: 7 rows set; 46 bytes in data 5: is-value-ext: bitmap @@ -44,108 +44,104 @@ finish 005-007: x 0600 # 6 columns 007-011: x 07000000 # 7 rows # column 0 -011-012: b 00000111 # prefixbytes +011-012: b 00000100 # prefixbytes 012-016: x 29000000 # page start 41 # column 1 -016-017: b 00000110 # bytes -017-021: x 3c000000 # page start 60 +016-017: b 00000011 # bytes +017-021: x 38000000 # page start 56 # column 2 -021-022: b 00000101 # uint64 -022-026: x 59000000 # page start 89 +021-022: b 00000010 # uint +022-026: x 51000000 # page start 81 # column 3 026-027: b 00000001 # bool -027-031: x 69000000 # page start 105 +027-031: x 59000000 # page start 89 # column 4 -031-032: b 00000110 # bytes -032-036: x 80000000 # page start 128 +031-032: b 00000011 # bytes +032-036: x 70000000 # page start 112 # column 5 036-037: b 00000001 # bool -037-041: x bb000000 # page start 187 +037-041: x a7000000 # page start 167 # data for column 0 # PrefixBytes 041-042: x 04 # bundleSize: 16 # Offsets table -042-043: x 02 # delta encoding: delta8 -043-047: x 00000000 # 32-bit constant: 0 -047-048: x 00 # data[0] = 0 [56 overall] -048-049: x 00 # data[1] = 0 [56 overall] -049-050: x 01 # data[2] = 1 [57 overall] -050-051: x 02 # data[3] = 2 [58 overall] -051-052: x 02 # data[4] = 2 [58 overall] -052-053: x 03 # data[5] = 3 [59 overall] -053-054: x 03 # data[6] = 3 [59 overall] -054-055: x 03 # data[7] = 3 [59 overall] -055-056: x 04 # data[8] = 4 [60 overall] +042-043: x 01 # encoding: 1b +043-044: x 00 # data[0] = 0 [52 overall] +044-045: x 00 # data[1] = 0 [52 overall] +045-046: x 01 # data[2] = 1 [53 overall] +046-047: x 02 # data[3] = 2 [54 overall] +047-048: x 02 # data[4] = 2 [54 overall] +048-049: x 03 # data[5] = 3 [55 overall] +049-050: x 03 # data[6] = 3 [55 overall] +050-051: x 03 # data[7] = 3 [55 overall] +051-052: x 04 # data[8] = 4 [56 overall] # Data -056-056: x # data[00]: (block prefix) -056-056: x # data[01]: (bundle prefix) -056-057: x 61 # data[02]: a -057-058: x 62 # data[03]: b -058-058: x # data[04]: . -058-059: x 63 # data[05]: c -059-059: x # data[06]: . -059-059: x # data[07]: . -059-060: x 64 # data[08]: d +052-052: x # data[00]: (block prefix) +052-052: x # data[01]: (bundle prefix) +052-053: x 61 # data[02]: a +053-054: x 62 # data[03]: b +054-054: x # data[04]: . +054-055: x 63 # data[05]: c +055-055: x # data[06]: . +055-055: x # data[07]: . +055-056: x 64 # data[08]: d # data for column 1 # rawbytes # offsets table -060-061: x 02 # delta encoding: delta8 -061-065: x 00000000 # 32-bit constant: 0 -065-066: x 00 # data[0] = 0 [73 overall] -066-067: x 03 # data[1] = 3 [76 overall] -067-068: x 05 # data[2] = 5 [78 overall] -068-069: x 07 # data[3] = 7 [80 overall] -069-070: x 09 # data[4] = 9 [82 overall] -070-071: x 0b # data[5] = 11 [84 overall] -071-072: x 0d # data[6] = 13 [86 overall] -072-073: x 10 # data[7] = 16 [89 overall] +056-057: x 01 # encoding: 1b +057-058: x 00 # data[0] = 0 [65 overall] +058-059: x 03 # data[1] = 3 [68 overall] +059-060: x 05 # data[2] = 5 [70 overall] +060-061: x 07 # data[3] = 7 [72 overall] +061-062: x 09 # data[4] = 9 [74 overall] +062-063: x 0b # data[5] = 11 [76 overall] +063-064: x 0d # data[6] = 13 [78 overall] +064-065: x 10 # data[7] = 16 [81 overall] # data -073-076: x 403130 # data[0]: @10 -076-078: x 4035 # data[1]: @5 -078-080: x 4032 # data[2]: @2 -080-082: x 4039 # data[3]: @9 -082-084: x 4036 # data[4]: @6 -084-086: x 4031 # data[5]: @1 -086-089: x 403131 # data[6]: @11 +065-068: x 403130 # data[0]: @10 +068-070: x 4035 # data[1]: @5 +070-072: x 4032 # data[2]: @2 +072-074: x 4039 # data[3]: @9 +074-076: x 4036 # data[4]: @6 +076-078: x 4031 # data[5]: @1 +078-081: x 403131 # data[6]: @11 # data for column 2 -089-090: x 02 # delta encoding: delta8 -090-098: x 0000000000000000 # 64-bit constant: 0 -098-099: x 01 # data[0] = 1 -099-100: x 01 # data[1] = 1 -100-101: x 12 # data[2] = 18 -101-102: x 12 # data[3] = 18 -102-103: x 01 # data[4] = 1 -103-104: x 01 # data[5] = 1 -104-105: x 00 # data[6] = 0 +081-082: x 01 # encoding: 1b +082-083: x 01 # data[0] = 1 +083-084: x 01 # data[1] = 1 +084-085: x 12 # data[2] = 18 +085-086: x 12 # data[3] = 18 +086-087: x 01 # data[4] = 1 +087-088: x 01 # data[5] = 1 +088-089: x 00 # data[6] = 0 # data for column 3 -105-112: x 00000000000000 # padding to align to 64-bit boundary -112-120: b 0100101100000000000000000000000000000000000000000000000000000000 # bitmap word 0 -120-128: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +089-096: x 00000000000000 # padding to align to 64-bit boundary +096-104: b 0100101100000000000000000000000000000000000000000000000000000000 # bitmap word 0 +104-112: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 # data for column 4 # rawbytes # offsets table -128-129: x 02 # delta encoding: delta8 -129-133: x 00000000 # 32-bit constant: 0 -133-134: x 00 # data[0] = 0 [141 overall] -134-135: x 05 # data[1] = 5 [146 overall] -135-136: x 0b # data[2] = 11 [152 overall] -136-137: x 14 # data[3] = 20 [161 overall] -137-138: x 1b # data[4] = 27 [168 overall] -138-139: x 24 # data[5] = 36 [177 overall] -139-140: x 2e # data[6] = 46 [187 overall] -140-141: x 2e # data[7] = 46 [187 overall] +112-113: x 01 # encoding: 1b +113-114: x 00 # data[0] = 0 [121 overall] +114-115: x 05 # data[1] = 5 [126 overall] +115-116: x 0b # data[2] = 11 [132 overall] +116-117: x 14 # data[3] = 20 [141 overall] +117-118: x 1b # data[4] = 27 [148 overall] +118-119: x 24 # data[5] = 36 [157 overall] +119-120: x 2e # data[6] = 46 [167 overall] +120-121: x 2e # data[7] = 46 [167 overall] # data -141-146: x 6170706c65 # data[0]: apple -146-152: x 62616e616e61 # data[1]: banana -152-161: x 626c75656265727279 # data[2]: blueberry -161-168: x 636f636f6e7574 # data[3]: coconut -168-177: x 63616e74656c6f7065 # data[4]: cantelope -177-187: x 636c656d656e74696e65 # data[5]: clementine -187-187: x # data[6]: +121-126: x 6170706c65 # data[0]: apple +126-132: x 62616e616e61 # data[1]: banana +132-141: x 626c75656265727279 # data[2]: blueberry +141-148: x 636f636f6e7574 # data[3]: coconut +148-157: x 63616e74656c6f7065 # data[4]: cantelope +157-167: x 636c656d656e74696e65 # data[5]: clementine +167-167: x # data[6]: # data for column 5 -187-192: x 0000000000 # padding to align to 64-bit boundary -192-200: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 -200-208: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +167-168: x 00 # padding to align to 64-bit boundary +168-176: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 +176-184: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 iter first @@ -306,7 +302,7 @@ init size=1: 0: prefixes: prefixbytes(16): 0 keys 1: suffixes: bytes: 0 rows set; 0 bytes in data -2: trailers: uint64: 0 rows +2: trailers: uint: 0 rows 3: prefix changed: bitmap 4: values: bytes: 0 rows set; 0 bytes in data 5: is-value-ext: bitmap @@ -330,10 +326,10 @@ aaaaaaaaaaaaaaapproves@10#0,SET:a aaaaaaaaaaaaaaarresting@10#0,SET:a aaaaaaaaaaaaaaarrived@10#0,SET:a ---- -size=361: +size=353: 0: prefixes: prefixbytes(16): 17 keys 1: suffixes: bytes: 17 rows set; 51 bytes in data -2: trailers: uint64: 17 rows +2: trailers: uint: 17 rows 3: prefix changed: bitmap 4: values: bytes: 17 rows set; 17 bytes in data 5: is-value-ext: bitmap @@ -347,163 +343,161 @@ finish 005-007: x 0600 # 6 columns 007-011: x 11000000 # 17 rows # column 0 -011-012: b 00000111 # prefixbytes +011-012: b 00000100 # prefixbytes 012-016: x 29000000 # page start 41 # column 1 -016-017: b 00000110 # bytes -017-021: x c7000000 # page start 199 +016-017: b 00000011 # bytes +017-021: x c3000000 # page start 195 # column 2 -021-022: b 00000101 # uint64 -022-026: x 11010000 # page start 273 +021-022: b 00000010 # uint +022-026: x 09010000 # page start 265 # column 3 026-027: b 00000001 # bool -027-031: x 1a010000 # page start 282 +027-031: x 12010000 # page start 274 # column 4 -031-032: b 00000110 # bytes -032-036: x 30010000 # page start 304 +031-032: b 00000011 # bytes +032-036: x 28010000 # page start 296 # column 5 036-037: b 00000001 # bool -037-041: x 58010000 # page start 344 +037-041: x 4c010000 # page start 332 # data for column 0 # PrefixBytes 041-042: x 04 # bundleSize: 16 # Offsets table -042-043: x 02 # delta encoding: delta8 -043-047: x 00000000 # 32-bit constant: 0 -047-048: x 0f # data[0] = 15 [82 overall] -048-049: x 0f # data[1] = 15 [82 overall] -049-050: x 17 # data[2] = 23 [90 overall] -050-051: x 1d # data[3] = 29 [96 overall] -051-052: x 26 # data[4] = 38 [105 overall] -052-053: x 2e # data[5] = 46 [113 overall] -053-054: x 33 # data[6] = 51 [118 overall] -054-055: x 3b # data[7] = 59 [126 overall] -055-056: x 42 # data[8] = 66 [133 overall] -056-057: x 48 # data[9] = 72 [139 overall] -057-058: x 4c # data[10] = 76 [143 overall] -058-059: x 56 # data[11] = 86 [153 overall] -059-060: x 5c # data[12] = 92 [159 overall] -060-061: x 63 # data[13] = 99 [166 overall] -061-062: x 69 # data[14] = 105 [172 overall] -062-063: x 6f # data[15] = 111 [178 overall] -063-064: x 76 # data[16] = 118 [185 overall] -064-065: x 7e # data[17] = 126 [193 overall] -065-066: x 84 # data[18] = 132 [199 overall] -066-067: x 84 # data[19] = 132 [199 overall] +042-043: x 01 # encoding: 1b +043-044: x 0f # data[0] = 15 [78 overall] +044-045: x 0f # data[1] = 15 [78 overall] +045-046: x 17 # data[2] = 23 [86 overall] +046-047: x 1d # data[3] = 29 [92 overall] +047-048: x 26 # data[4] = 38 [101 overall] +048-049: x 2e # data[5] = 46 [109 overall] +049-050: x 33 # data[6] = 51 [114 overall] +050-051: x 3b # data[7] = 59 [122 overall] +051-052: x 42 # data[8] = 66 [129 overall] +052-053: x 48 # data[9] = 72 [135 overall] +053-054: x 4c # data[10] = 76 [139 overall] +054-055: x 56 # data[11] = 86 [149 overall] +055-056: x 5c # data[12] = 92 [155 overall] +056-057: x 63 # data[13] = 99 [162 overall] +057-058: x 69 # data[14] = 105 [168 overall] +058-059: x 6f # data[15] = 111 [174 overall] +059-060: x 76 # data[16] = 118 [181 overall] +060-061: x 7e # data[17] = 126 [189 overall] +061-062: x 84 # data[18] = 132 [195 overall] +062-063: x 84 # data[19] = 132 [195 overall] # Data -067-077: x 61616161616161616161 # data[00]: aaaaaaaaaaaaaaa (block prefix) -077-082: x 6161616161 # (continued...) -082-082: x # data[01]: ............... (bundle prefix) -082-090: x 7070616c6c696e67 # data[02]: ...............ppalling -090-096: x 70706172656c # data[03]: ...............pparel -096-105: x 707061726974696f6e # data[04]: ...............pparition -105-113: x 7070656172696e67 # data[05]: ...............ppearing -113-118: x 7070656e64 # data[06]: ...............ppend -118-126: x 7070656e64616765 # data[07]: ...............ppendage -126-133: x 7070656e646978 # data[08]: ...............ppendix -133-139: x 70706c617564 # data[09]: ...............pplaud -139-143: x 70706c65 # data[10]: ...............pple -143-153: x 70706c69636174696f6e # data[11]: ...............pplication -153-159: x 70706c696564 # data[12]: ...............pplied -159-166: x 70706c79696e67 # data[13]: ...............pplying -166-172: x 70706f696e74 # data[14]: ...............ppoint -172-178: x 70706f736573 # data[15]: ...............pposes -178-185: x 7070726f766573 # data[16]: ...............pproves -185-193: x 7272657374696e67 # data[17]: ...............rresting -193-199: x 727269766564 # data[18]: ...............rrived (bundle prefix) -199-199: x # data[19]: ..................... +063-073: x 61616161616161616161 # data[00]: aaaaaaaaaaaaaaa (block prefix) +073-078: x 6161616161 # (continued...) +078-078: x # data[01]: ............... (bundle prefix) +078-086: x 7070616c6c696e67 # data[02]: ...............ppalling +086-092: x 70706172656c # data[03]: ...............pparel +092-101: x 707061726974696f6e # data[04]: ...............pparition +101-109: x 7070656172696e67 # data[05]: ...............ppearing +109-114: x 7070656e64 # data[06]: ...............ppend +114-122: x 7070656e64616765 # data[07]: ...............ppendage +122-129: x 7070656e646978 # data[08]: ...............ppendix +129-135: x 70706c617564 # data[09]: ...............pplaud +135-139: x 70706c65 # data[10]: ...............pple +139-149: x 70706c69636174696f6e # data[11]: ...............pplication +149-155: x 70706c696564 # data[12]: ...............pplied +155-162: x 70706c79696e67 # data[13]: ...............pplying +162-168: x 70706f696e74 # data[14]: ...............ppoint +168-174: x 70706f736573 # data[15]: ...............pposes +174-181: x 7070726f766573 # data[16]: ...............pproves +181-189: x 7272657374696e67 # data[17]: ...............rresting +189-195: x 727269766564 # data[18]: ...............rrived (bundle prefix) +195-195: x # data[19]: ..................... # data for column 1 # rawbytes # offsets table -199-200: x 02 # delta encoding: delta8 -200-204: x 00000000 # 32-bit constant: 0 -204-205: x 00 # data[0] = 0 [222 overall] -205-206: x 03 # data[1] = 3 [225 overall] -206-207: x 06 # data[2] = 6 [228 overall] -207-208: x 09 # data[3] = 9 [231 overall] -208-209: x 0c # data[4] = 12 [234 overall] -209-210: x 0f # data[5] = 15 [237 overall] -210-211: x 12 # data[6] = 18 [240 overall] -211-212: x 15 # data[7] = 21 [243 overall] -212-213: x 18 # data[8] = 24 [246 overall] -213-214: x 1b # data[9] = 27 [249 overall] -214-215: x 1e # data[10] = 30 [252 overall] -215-216: x 21 # data[11] = 33 [255 overall] -216-217: x 24 # data[12] = 36 [258 overall] -217-218: x 27 # data[13] = 39 [261 overall] -218-219: x 2a # data[14] = 42 [264 overall] -219-220: x 2d # data[15] = 45 [267 overall] -220-221: x 30 # data[16] = 48 [270 overall] -221-222: x 33 # data[17] = 51 [273 overall] +195-196: x 01 # encoding: 1b +196-197: x 00 # data[0] = 0 [214 overall] +197-198: x 03 # data[1] = 3 [217 overall] +198-199: x 06 # data[2] = 6 [220 overall] +199-200: x 09 # data[3] = 9 [223 overall] +200-201: x 0c # data[4] = 12 [226 overall] +201-202: x 0f # data[5] = 15 [229 overall] +202-203: x 12 # data[6] = 18 [232 overall] +203-204: x 15 # data[7] = 21 [235 overall] +204-205: x 18 # data[8] = 24 [238 overall] +205-206: x 1b # data[9] = 27 [241 overall] +206-207: x 1e # data[10] = 30 [244 overall] +207-208: x 21 # data[11] = 33 [247 overall] +208-209: x 24 # data[12] = 36 [250 overall] +209-210: x 27 # data[13] = 39 [253 overall] +210-211: x 2a # data[14] = 42 [256 overall] +211-212: x 2d # data[15] = 45 [259 overall] +212-213: x 30 # data[16] = 48 [262 overall] +213-214: x 33 # data[17] = 51 [265 overall] # data -222-225: x 403130 # data[0]: @10 -225-228: x 403130 # data[1]: @10 -228-231: x 403130 # data[2]: @10 -231-234: x 403130 # data[3]: @10 -234-237: x 403130 # data[4]: @10 -237-240: x 403130 # data[5]: @10 -240-243: x 403130 # data[6]: @10 -243-246: x 403130 # data[7]: @10 -246-249: x 403130 # data[8]: @10 -249-252: x 403130 # data[9]: @10 -252-255: x 403130 # data[10]: @10 -255-258: x 403130 # data[11]: @10 -258-261: x 403130 # data[12]: @10 -261-264: x 403130 # data[13]: @10 -264-267: x 403130 # data[14]: @10 -267-270: x 403130 # data[15]: @10 -270-273: x 403130 # data[16]: @10 +214-217: x 403130 # data[0]: @10 +217-220: x 403130 # data[1]: @10 +220-223: x 403130 # data[2]: @10 +223-226: x 403130 # data[3]: @10 +226-229: x 403130 # data[4]: @10 +229-232: x 403130 # data[5]: @10 +232-235: x 403130 # data[6]: @10 +235-238: x 403130 # data[7]: @10 +238-241: x 403130 # data[8]: @10 +241-244: x 403130 # data[9]: @10 +244-247: x 403130 # data[10]: @10 +247-250: x 403130 # data[11]: @10 +250-253: x 403130 # data[12]: @10 +253-256: x 403130 # data[13]: @10 +256-259: x 403130 # data[14]: @10 +259-262: x 403130 # data[15]: @10 +262-265: x 403130 # data[16]: @10 # data for column 2 -273-274: x 01 # delta encoding: const -274-282: x 0100000000000000 # 64-bit constant: 1 +265-266: x 80 # encoding: const +266-274: x 0100000000000000 # 64-bit constant: 1 # data for column 3 -282-288: x 000000000000 # padding to align to 64-bit boundary -288-296: b 1111111111111111000000010000000000000000000000000000000000000000 # bitmap word 0 -296-304: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +274-280: x 000000000000 # padding to align to 64-bit boundary +280-288: b 1111111111111111000000010000000000000000000000000000000000000000 # bitmap word 0 +288-296: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 # data for column 4 # rawbytes # offsets table -304-305: x 02 # delta encoding: delta8 -305-309: x 00000000 # 32-bit constant: 0 -309-310: x 00 # data[0] = 0 [327 overall] -310-311: x 01 # data[1] = 1 [328 overall] -311-312: x 02 # data[2] = 2 [329 overall] -312-313: x 03 # data[3] = 3 [330 overall] -313-314: x 04 # data[4] = 4 [331 overall] -314-315: x 05 # data[5] = 5 [332 overall] -315-316: x 06 # data[6] = 6 [333 overall] -316-317: x 07 # data[7] = 7 [334 overall] -317-318: x 08 # data[8] = 8 [335 overall] -318-319: x 09 # data[9] = 9 [336 overall] -319-320: x 0a # data[10] = 10 [337 overall] -320-321: x 0b # data[11] = 11 [338 overall] -321-322: x 0c # data[12] = 12 [339 overall] -322-323: x 0d # data[13] = 13 [340 overall] -323-324: x 0e # data[14] = 14 [341 overall] -324-325: x 0f # data[15] = 15 [342 overall] -325-326: x 10 # data[16] = 16 [343 overall] -326-327: x 11 # data[17] = 17 [344 overall] +296-297: x 01 # encoding: 1b +297-298: x 00 # data[0] = 0 [315 overall] +298-299: x 01 # data[1] = 1 [316 overall] +299-300: x 02 # data[2] = 2 [317 overall] +300-301: x 03 # data[3] = 3 [318 overall] +301-302: x 04 # data[4] = 4 [319 overall] +302-303: x 05 # data[5] = 5 [320 overall] +303-304: x 06 # data[6] = 6 [321 overall] +304-305: x 07 # data[7] = 7 [322 overall] +305-306: x 08 # data[8] = 8 [323 overall] +306-307: x 09 # data[9] = 9 [324 overall] +307-308: x 0a # data[10] = 10 [325 overall] +308-309: x 0b # data[11] = 11 [326 overall] +309-310: x 0c # data[12] = 12 [327 overall] +310-311: x 0d # data[13] = 13 [328 overall] +311-312: x 0e # data[14] = 14 [329 overall] +312-313: x 0f # data[15] = 15 [330 overall] +313-314: x 10 # data[16] = 16 [331 overall] +314-315: x 11 # data[17] = 17 [332 overall] # data -327-328: x 61 # data[0]: a -328-329: x 61 # data[1]: a -329-330: x 61 # data[2]: a -330-331: x 61 # data[3]: a -331-332: x 61 # data[4]: a -332-333: x 61 # data[5]: a -333-334: x 61 # data[6]: a -334-335: x 61 # data[7]: a -335-336: x 61 # data[8]: a -336-337: x 61 # data[9]: a -337-338: x 61 # data[10]: a -338-339: x 61 # data[11]: a -339-340: x 61 # data[12]: a -340-341: x 61 # data[13]: a -341-342: x 61 # data[14]: a -342-343: x 61 # data[15]: a -343-344: x 61 # data[16]: a +315-316: x 61 # data[0]: a +316-317: x 61 # data[1]: a +317-318: x 61 # data[2]: a +318-319: x 61 # data[3]: a +319-320: x 61 # data[4]: a +320-321: x 61 # data[5]: a +321-322: x 61 # data[6]: a +322-323: x 61 # data[7]: a +323-324: x 61 # data[8]: a +324-325: x 61 # data[9]: a +325-326: x 61 # data[10]: a +326-327: x 61 # data[11]: a +327-328: x 61 # data[12]: a +328-329: x 61 # data[13]: a +329-330: x 61 # data[14]: a +330-331: x 61 # data[15]: a +331-332: x 61 # data[16]: a # data for column 5 -344-352: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 -352-360: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 +332-336: x 00000000 # padding to align to 64-bit boundary +336-344: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap word 0 +344-352: b 0000000000000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 iter seek-ge aaa diff --git a/sstable/colblk/testdata/keyspan_block b/sstable/colblk/testdata/keyspan_block index afabc695de..47d44c3b29 100644 --- a/sstable/colblk/testdata/keyspan_block +++ b/sstable/colblk/testdata/keyspan_block @@ -2,137 +2,131 @@ init ---- size=1: 0: user keys: bytes: 0 rows set; 0 bytes in data -1: start indices: uint32: 0 rows -2: trailers: uint64: 0 rows +1: start indices: uint: 0 rows +2: trailers: uint: 0 rows 3: suffixes: bytes: 0 rows set; 0 bytes in data 4: values: bytes: 0 rows set; 0 bytes in data add a-b:{(#0,RANGEDEL)} ---- -size=73: +size=57: 0: user keys: bytes: 2 rows set; 2 bytes in data -1: start indices: uint32: 2 rows -2: trailers: uint64: 1 rows +1: start indices: uint: 2 rows +2: trailers: uint: 1 rows 3: suffixes: bytes: 1 rows set; 0 bytes in data 4: values: bytes: 1 rows set; 0 bytes in data add b-c:{(#100,RANGEDEL) (#20,RANGEDEL) (#0,RANGEDEL)} ---- -size=85: +size=61: 0: user keys: bytes: 3 rows set; 3 bytes in data -1: start indices: uint32: 3 rows -2: trailers: uint64: 4 rows +1: start indices: uint: 3 rows +2: trailers: uint: 4 rows 3: suffixes: bytes: 4 rows set; 0 bytes in data 4: values: bytes: 4 rows set; 0 bytes in data add c-d:{(#100,RANGEDEL) (#0,RANGEDEL)} ---- -size=91: +size=67: 0: user keys: bytes: 4 rows set; 4 bytes in data -1: start indices: uint32: 4 rows -2: trailers: uint64: 6 rows +1: start indices: uint: 4 rows +2: trailers: uint: 6 rows 3: suffixes: bytes: 6 rows set; 0 bytes in data 4: values: bytes: 6 rows set; 0 bytes in data add d-e:{(#0,RANGEDEL)} ---- -size=97: +size=73: 0: user keys: bytes: 5 rows set; 5 bytes in data -1: start indices: uint32: 5 rows -2: trailers: uint64: 7 rows +1: start indices: uint: 5 rows +2: trailers: uint: 7 rows 3: suffixes: bytes: 7 rows set; 0 bytes in data 4: values: bytes: 7 rows set; 0 bytes in data finish ---- # keyspan block header -00-04: x 05000000 # user key count: 5 +00-04: x 05000000 # user key count: 5 # columnar block header -04-05: x 01 # version 1 -05-07: x 0500 # 5 columns -07-11: x 07000000 # 7 rows +04-05: x 01 # version 1 +05-07: x 0500 # 5 columns +07-11: x 07000000 # 7 rows # column 0 -11-12: b 00000110 # bytes -12-16: x 24000000 # page start 36 +11-12: b 00000011 # bytes +12-16: x 24000000 # page start 36 # column 1 -16-17: b 00000100 # uint32 -17-21: x 34000000 # page start 52 +16-17: b 00000010 # uint +17-21: x 30000000 # page start 48 # column 2 -21-22: b 00000101 # uint64 -22-26: x 3e000000 # page start 62 +21-22: b 00000010 # uint +22-26: x 36000000 # page start 54 # column 3 -26-27: b 00000110 # bytes -27-31: x 56000000 # page start 86 +26-27: b 00000011 # bytes +27-31: x 46000000 # page start 70 # column 4 -31-32: b 00000110 # bytes -32-36: x 5b000000 # page start 91 +31-32: b 00000011 # bytes +32-36: x 47000000 # page start 71 # data for column 0 # rawbytes # offsets table -36-37: x 02 # delta encoding: delta8 -37-41: x 00000000 # 32-bit constant: 0 -41-42: x 00 # data[0] = 0 [47 overall] -42-43: x 01 # data[1] = 1 [48 overall] -43-44: x 02 # data[2] = 2 [49 overall] -44-45: x 03 # data[3] = 3 [50 overall] -45-46: x 04 # data[4] = 4 [51 overall] -46-47: x 05 # data[5] = 5 [52 overall] +36-37: x 01 # encoding: 1b +37-38: x 00 # data[0] = 0 [43 overall] +38-39: x 01 # data[1] = 1 [44 overall] +39-40: x 02 # data[2] = 2 [45 overall] +40-41: x 03 # data[3] = 3 [46 overall] +41-42: x 04 # data[4] = 4 [47 overall] +42-43: x 05 # data[5] = 5 [48 overall] # data -47-48: x 61 # data[0]: a -48-49: x 62 # data[1]: b -49-50: x 63 # data[2]: c -50-51: x 64 # data[3]: d -51-52: x 65 # data[4]: e +43-44: x 61 # data[0]: a +44-45: x 62 # data[1]: b +45-46: x 63 # data[2]: c +46-47: x 64 # data[3]: d +47-48: x 65 # data[4]: e # data for column 1 -52-53: x 02 # delta encoding: delta8 -53-57: x 00000000 # 32-bit constant: 0 -57-58: x 00 # data[0] = 0 -58-59: x 01 # data[1] = 1 -59-60: x 04 # data[2] = 4 -60-61: x 06 # data[3] = 6 -61-62: x 07 # data[4] = 7 +48-49: x 01 # encoding: 1b +49-50: x 00 # data[0] = 0 +50-51: x 01 # data[1] = 1 +51-52: x 04 # data[2] = 4 +52-53: x 06 # data[3] = 6 +53-54: x 07 # data[4] = 7 # data for column 2 -62-63: x 03 # delta encoding: delta16 -63-71: x 0f00000000000000 # 64-bit constant: 15 -# padding -71-72: x 00 # aligning to 16-bit boundary -72-74: x 0000 # data[0] = 0 + 15 = 15 -74-76: x 0064 # data[1] = 25600 + 15 = 25615 -76-78: x 0014 # data[2] = 5120 + 15 = 5135 -78-80: x 0000 # data[3] = 0 + 15 = 15 -80-82: x 0064 # data[4] = 25600 + 15 = 25615 -82-84: x 0000 # data[5] = 0 + 15 = 15 -84-86: x 0000 # data[6] = 0 + 15 = 15 +54-55: x 02 # encoding: 2b +55-56: x 00 # padding (aligning to 16-bit boundary) +56-58: x 0f00 # data[0] = 15 +58-60: x 0f64 # data[1] = 25615 +60-62: x 0f14 # data[2] = 5135 +62-64: x 0f00 # data[3] = 15 +64-66: x 0f64 # data[4] = 25615 +66-68: x 0f00 # data[5] = 15 +68-70: x 0f00 # data[6] = 15 # data for column 3 # rawbytes # offsets table -86-87: x 01 # delta encoding: const -87-91: x 00000000 # 32-bit constant: 0 +70-71: x 00 # encoding: zero # data -91-91: x # data[0]: -91-91: x # data[1]: -91-91: x # data[2]: -91-91: x # data[3]: -91-91: x # data[4]: -91-91: x # data[5]: -91-91: x # data[6]: +71-71: x # data[0]: +71-71: x # data[1]: +71-71: x # data[2]: +71-71: x # data[3]: +71-71: x # data[4]: +71-71: x # data[5]: +71-71: x # data[6]: # data for column 4 # rawbytes # offsets table -91-92: x 01 # delta encoding: const -92-96: x 00000000 # 32-bit constant: 0 +71-72: x 00 # encoding: zero # data -96-96: x # data[0]: -96-96: x # data[1]: -96-96: x # data[2]: -96-96: x # data[3]: -96-96: x # data[4]: -96-96: x # data[5]: -96-96: x # data[6]: +72-72: x # data[0]: +72-72: x # data[1]: +72-72: x # data[2]: +72-72: x # data[3]: +72-72: x # data[4]: +72-72: x # data[5]: +72-72: x # data[6]: # Test iterating over the block's spans. @@ -220,18 +214,18 @@ init ---- size=1: 0: user keys: bytes: 0 rows set; 0 bytes in data -1: start indices: uint32: 0 rows -2: trailers: uint64: 0 rows +1: start indices: uint: 0 rows +2: trailers: uint: 0 rows 3: suffixes: bytes: 0 rows set; 0 bytes in data 4: values: bytes: 0 rows set; 0 bytes in data add b-d:{(#4,RANGEKEYSET,@3,coconut)} ---- -size=86: +size=70: 0: user keys: bytes: 2 rows set; 2 bytes in data -1: start indices: uint32: 2 rows -2: trailers: uint64: 1 rows +1: start indices: uint: 2 rows +2: trailers: uint: 1 rows 3: suffixes: bytes: 1 rows set; 2 bytes in data 4: values: bytes: 1 rows set; 7 bytes in data @@ -244,57 +238,53 @@ finish 05-07: x 0500 # 5 columns 07-11: x 01000000 # 1 rows # column 0 -11-12: b 00000110 # bytes +11-12: b 00000011 # bytes 12-16: x 24000000 # page start 36 # column 1 -16-17: b 00000100 # uint32 -17-21: x 2e000000 # page start 46 +16-17: b 00000010 # uint +17-21: x 2a000000 # page start 42 # column 2 -21-22: b 00000101 # uint64 -22-26: x 35000000 # page start 53 +21-22: b 00000010 # uint +22-26: x 2d000000 # page start 45 # column 3 -26-27: b 00000110 # bytes -27-31: x 3e000000 # page start 62 +26-27: b 00000011 # bytes +27-31: x 36000000 # page start 54 # column 4 -31-32: b 00000110 # bytes -32-36: x 47000000 # page start 71 +31-32: b 00000011 # bytes +32-36: x 3b000000 # page start 59 # data for column 0 # rawbytes # offsets table -36-37: x 02 # delta encoding: delta8 -37-41: x 00000000 # 32-bit constant: 0 -41-42: x 00 # data[0] = 0 [44 overall] -42-43: x 01 # data[1] = 1 [45 overall] -43-44: x 02 # data[2] = 2 [46 overall] +36-37: x 01 # encoding: 1b +37-38: x 00 # data[0] = 0 [40 overall] +38-39: x 01 # data[1] = 1 [41 overall] +39-40: x 02 # data[2] = 2 [42 overall] # data -44-45: x 62 # data[0]: b -45-46: x 64 # data[1]: d +40-41: x 62 # data[0]: b +41-42: x 64 # data[1]: d # data for column 1 -46-47: x 02 # delta encoding: delta8 -47-51: x 00000000 # 32-bit constant: 0 -51-52: x 00 # data[0] = 0 -52-53: x 01 # data[1] = 1 +42-43: x 01 # encoding: 1b +43-44: x 00 # data[0] = 0 +44-45: x 01 # data[1] = 1 # data for column 2 -53-54: x 01 # delta encoding: const -54-62: x 1504000000000000 # 64-bit constant: 1045 +45-46: x 80 # encoding: const +46-54: x 1504000000000000 # 64-bit constant: 1045 # data for column 3 # rawbytes # offsets table -62-63: x 02 # delta encoding: delta8 -63-67: x 00000000 # 32-bit constant: 0 -67-68: x 00 # data[0] = 0 [69 overall] -68-69: x 02 # data[1] = 2 [71 overall] +54-55: x 01 # encoding: 1b +55-56: x 00 # data[0] = 0 [57 overall] +56-57: x 02 # data[1] = 2 [59 overall] # data -69-71: x 4033 # data[0]: @3 +57-59: x 4033 # data[0]: @3 # data for column 4 # rawbytes # offsets table -71-72: x 02 # delta encoding: delta8 -72-76: x 00000000 # 32-bit constant: 0 -76-77: x 00 # data[0] = 0 [78 overall] -77-78: x 07 # data[1] = 7 [85 overall] +59-60: x 01 # encoding: 1b +60-61: x 00 # data[0] = 0 [62 overall] +61-62: x 07 # data[1] = 7 [69 overall] # data -78-85: x 636f636f6e7574 # data[0]: coconut +62-69: x 636f636f6e7574 # data[0]: coconut iter seek-ge a @@ -313,8 +303,8 @@ reset ---- size=1: 0: user keys: bytes: 0 rows set; 0 bytes in data -1: start indices: uint32: 0 rows -2: trailers: uint64: 0 rows +1: start indices: uint: 0 rows +2: trailers: uint: 0 rows 3: suffixes: bytes: 0 rows set; 0 bytes in data 4: values: bytes: 0 rows set; 0 bytes in data @@ -322,85 +312,80 @@ add b-d:{(#4,RANGEKEYSET,@3,coconut)} e-g:{(#5,RANGEKEYSET,@1,tree)} ---- -size=104: +size=80: 0: user keys: bytes: 4 rows set; 4 bytes in data -1: start indices: uint32: 4 rows -2: trailers: uint64: 2 rows +1: start indices: uint: 4 rows +2: trailers: uint: 2 rows 3: suffixes: bytes: 2 rows set; 4 bytes in data 4: values: bytes: 2 rows set; 11 bytes in data finish ---- # keyspan block header -000-004: x 04000000 # user key count: 4 +00-04: x 04000000 # user key count: 4 # columnar block header -004-005: x 01 # version 1 -005-007: x 0500 # 5 columns -007-011: x 02000000 # 2 rows +04-05: x 01 # version 1 +05-07: x 0500 # 5 columns +07-11: x 02000000 # 2 rows # column 0 -011-012: b 00000110 # bytes -012-016: x 24000000 # page start 36 +11-12: b 00000011 # bytes +12-16: x 24000000 # page start 36 # column 1 -016-017: b 00000100 # uint32 -017-021: x 32000000 # page start 50 +16-17: b 00000010 # uint +17-21: x 2e000000 # page start 46 # column 2 -021-022: b 00000101 # uint64 -022-026: x 3b000000 # page start 59 +21-22: b 00000010 # uint +22-26: x 33000000 # page start 51 # column 3 -026-027: b 00000110 # bytes -027-031: x 48000000 # page start 72 +26-27: b 00000011 # bytes +27-31: x 38000000 # page start 56 # column 4 -031-032: b 00000110 # bytes -032-036: x 54000000 # page start 84 +31-32: b 00000011 # bytes +32-36: x 40000000 # page start 64 # data for column 0 # rawbytes # offsets table -036-037: x 02 # delta encoding: delta8 -037-041: x 00000000 # 32-bit constant: 0 -041-042: x 00 # data[0] = 0 [46 overall] -042-043: x 01 # data[1] = 1 [47 overall] -043-044: x 02 # data[2] = 2 [48 overall] -044-045: x 03 # data[3] = 3 [49 overall] -045-046: x 04 # data[4] = 4 [50 overall] +36-37: x 01 # encoding: 1b +37-38: x 00 # data[0] = 0 [42 overall] +38-39: x 01 # data[1] = 1 [43 overall] +39-40: x 02 # data[2] = 2 [44 overall] +40-41: x 03 # data[3] = 3 [45 overall] +41-42: x 04 # data[4] = 4 [46 overall] # data -046-047: x 62 # data[0]: b -047-048: x 64 # data[1]: d -048-049: x 65 # data[2]: e -049-050: x 67 # data[3]: g +42-43: x 62 # data[0]: b +43-44: x 64 # data[1]: d +44-45: x 65 # data[2]: e +45-46: x 67 # data[3]: g # data for column 1 -050-051: x 02 # delta encoding: delta8 -051-055: x 00000000 # 32-bit constant: 0 -055-056: x 00 # data[0] = 0 -056-057: x 01 # data[1] = 1 -057-058: x 01 # data[2] = 1 -058-059: x 02 # data[3] = 2 +46-47: x 01 # encoding: 1b +47-48: x 00 # data[0] = 0 +48-49: x 01 # data[1] = 1 +49-50: x 01 # data[2] = 1 +50-51: x 02 # data[3] = 2 # data for column 2 -059-060: x 03 # delta encoding: delta16 -060-068: x 1504000000000000 # 64-bit constant: 1045 -068-070: x 0000 # data[0] = 0 + 1045 = 1045 -070-072: x 0001 # data[1] = 256 + 1045 = 1301 +51-52: x 02 # encoding: 2b +52-54: x 1504 # data[0] = 1045 +54-56: x 1505 # data[1] = 1301 # data for column 3 # rawbytes # offsets table -072-073: x 02 # delta encoding: delta8 -073-077: x 00000000 # 32-bit constant: 0 -077-078: x 00 # data[0] = 0 [80 overall] -078-079: x 02 # data[1] = 2 [82 overall] -079-080: x 04 # data[2] = 4 [84 overall] +56-57: x 01 # encoding: 1b +57-58: x 00 # data[0] = 0 [60 overall] +58-59: x 02 # data[1] = 2 [62 overall] +59-60: x 04 # data[2] = 4 [64 overall] # data -080-082: x 4033 # data[0]: @3 -082-084: x 4031 # data[1]: @1 +60-62: x 4033 # data[0]: @3 +62-64: x 4031 # data[1]: @1 # data for column 4 # rawbytes # offsets table -084-085: x 02 # delta encoding: delta8 -085-089: x 00000000 # 32-bit constant: 0 -089-090: x 00 # data[0] = 0 [92 overall] -090-091: x 07 # data[1] = 7 [99 overall] -091-092: x 0b # data[2] = 11 [103 overall] +64-65: x 01 # encoding: 1b +65-66: x 00 # data[0] = 0 [68 overall] +66-67: x 07 # data[1] = 7 [75 overall] +67-68: x 0b # data[2] = 11 [79 overall] # data -092-099: x 636f636f6e7574 # data[0]: coconut -099-103: x 74726565 # data[1]: tree +68-75: x 636f636f6e7574 # data[0]: coconut +75-79: x 74726565 # data[1]: tree iter seek-ge dog diff --git a/sstable/colblk/testdata/prefix_bytes b/sstable/colblk/testdata/prefix_bytes index 15fe28b7d6..dac317d26e 100644 --- a/sstable/colblk/testdata/prefix_bytes +++ b/sstable/colblk/testdata/prefix_bytes @@ -5,7 +5,7 @@ Size: 0 put abc ---- -Size: 12 +Size: 8 nKeys=1; bundleSize=4 blockPrefixLen=3; currentBundleLen=3; currentBundleKeys=1 Offsets: @@ -16,17 +16,16 @@ abc finish rows=1 ---- # PrefixBytes -00-01: x 02 # bundleSize: 4 +0-1: x 02 # bundleSize: 4 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 03 # data[0] = 3 [12 overall] -07-08: x 03 # data[1] = 3 [12 overall] -08-09: x 03 # data[2] = 3 [12 overall] +1-2: x 01 # encoding: 1b +2-3: x 03 # data[0] = 3 [8 overall] +3-4: x 03 # data[1] = 3 [8 overall] +4-5: x 03 # data[2] = 3 [8 overall] # Data -09-12: x 616263 # data[00]: abc (block prefix) -12-12: x # data[01]: ... (bundle prefix) -12-12: x # data[02]: ... +5-8: x 616263 # data[00]: abc (block prefix) +8-8: x # data[01]: ... (bundle prefix) +8-8: x # data[02]: ... init bundle-size=4 ---- @@ -35,7 +34,7 @@ Size: 0 put abc ---- -Size: 12 +Size: 8 nKeys=1; bundleSize=4 blockPrefixLen=3; currentBundleLen=3; currentBundleKeys=1 Offsets: @@ -46,7 +45,7 @@ abc put abcd ---- -Size: 14 +Size: 10 nKeys=2; bundleSize=4 blockPrefixLen=3; currentBundleLen=7; currentBundleKeys=2 Offsets: @@ -57,7 +56,7 @@ abcabcd put abce ---- -Size: 16 +Size: 12 nKeys=3; bundleSize=4 blockPrefixLen=3; currentBundleLen=11; currentBundleKeys=3 Offsets: @@ -68,7 +67,7 @@ abcabcdabce put abdd ---- -Size: 21 +Size: 17 nKeys=4; bundleSize=4 blockPrefixLen=2; currentBundleLen=15; currentBundleKeys=4 Offsets: @@ -79,7 +78,7 @@ abcabcdabceabdd put abde ---- -Size: 25 +Size: 21 nKeys=5; bundleSize=4 blockPrefixLen=2; currentBundleLen=4; currentBundleKeys=1 Offsets: @@ -92,50 +91,48 @@ abcabcdabceabddabde finish rows=4 ---- # PrefixBytes -00-01: x 02 # bundleSize: 4 +00-01: x 02 # bundleSize: 4 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 02 # data[0] = 2 [14 overall] -07-08: x 02 # data[1] = 2 [14 overall] -08-09: x 03 # data[2] = 3 [15 overall] -09-10: x 05 # data[3] = 5 [17 overall] -10-11: x 07 # data[4] = 7 [19 overall] -11-12: x 09 # data[5] = 9 [21 overall] +01-02: x 01 # encoding: 1b +02-03: x 02 # data[0] = 2 [10 overall] +03-04: x 02 # data[1] = 2 [10 overall] +04-05: x 03 # data[2] = 3 [11 overall] +05-06: x 05 # data[3] = 5 [13 overall] +06-07: x 07 # data[4] = 7 [15 overall] +07-08: x 09 # data[5] = 9 [17 overall] # Data -12-14: x 6162 # data[00]: ab (block prefix) -14-14: x # data[01]: .. (bundle prefix) -14-15: x 63 # data[02]: ..c -15-17: x 6364 # data[03]: ..cd -17-19: x 6365 # data[04]: ..ce -19-21: x 6464 # data[05]: ..dd +08-10: x 6162 # data[00]: ab (block prefix) +10-10: x # data[01]: .. (bundle prefix) +10-11: x 63 # data[02]: ..c +11-13: x 6364 # data[03]: ..cd +13-15: x 6365 # data[04]: ..ce +15-17: x 6464 # data[05]: ..dd # Finish the entirety of all put rows. finish rows=5 ---- # PrefixBytes -00-01: x 02 # bundleSize: 4 +00-01: x 02 # bundleSize: 4 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 02 # data[0] = 2 [16 overall] -07-08: x 02 # data[1] = 2 [16 overall] -08-09: x 03 # data[2] = 3 [17 overall] -09-10: x 05 # data[3] = 5 [19 overall] -10-11: x 07 # data[4] = 7 [21 overall] -11-12: x 09 # data[5] = 9 [23 overall] -12-13: x 0b # data[6] = 11 [25 overall] -13-14: x 0b # data[7] = 11 [25 overall] +01-02: x 01 # encoding: 1b +02-03: x 02 # data[0] = 2 [12 overall] +03-04: x 02 # data[1] = 2 [12 overall] +04-05: x 03 # data[2] = 3 [13 overall] +05-06: x 05 # data[3] = 5 [15 overall] +06-07: x 07 # data[4] = 7 [17 overall] +07-08: x 09 # data[5] = 9 [19 overall] +08-09: x 0b # data[6] = 11 [21 overall] +09-10: x 0b # data[7] = 11 [21 overall] # Data -14-16: x 6162 # data[00]: ab (block prefix) -16-16: x # data[01]: .. (bundle prefix) -16-17: x 63 # data[02]: ..c -17-19: x 6364 # data[03]: ..cd -19-21: x 6365 # data[04]: ..ce -21-23: x 6464 # data[05]: ..dd -23-25: x 6465 # data[06]: ..de (bundle prefix) -25-25: x # data[07]: .... +10-12: x 6162 # data[00]: ab (block prefix) +12-12: x # data[01]: .. (bundle prefix) +12-13: x 63 # data[02]: ..c +13-15: x 6364 # data[03]: ..cd +15-17: x 6365 # data[04]: ..ce +17-19: x 6464 # data[05]: ..dd +19-21: x 6465 # data[06]: ..de (bundle prefix) +21-21: x # data[07]: .... get indices=(0, 1, 2, 3, 4) ---- @@ -184,7 +181,7 @@ Size: 0 put aaabbbc ---- -Size: 16 +Size: 12 nKeys=1; bundleSize=4 blockPrefixLen=7; currentBundleLen=7; currentBundleKeys=1 Offsets: @@ -195,7 +192,7 @@ aaabbbc put aaabbbcc ---- -Size: 18 +Size: 14 nKeys=2; bundleSize=4 blockPrefixLen=7; currentBundleLen=15; currentBundleKeys=2 Offsets: @@ -206,7 +203,7 @@ aaabbbcaaabbbcc put aaabbbcde ---- -Size: 21 +Size: 17 nKeys=3; bundleSize=4 blockPrefixLen=7; currentBundleLen=24; currentBundleKeys=3 Offsets: @@ -217,7 +214,7 @@ aaabbbcaaabbbccaaabbbcde put aaabbbce ---- -Size: 23 +Size: 19 nKeys=4; bundleSize=4 blockPrefixLen=7; currentBundleLen=32; currentBundleKeys=4 Offsets: @@ -228,7 +225,7 @@ aaabbbcaaabbbccaaabbbcdeaaabbbce put aaabbbdee* ---- -Size: 29 +Size: 25 nKeys=5; bundleSize=4 blockPrefixLen=6; currentBundleLen=10; currentBundleKeys=1 Offsets: @@ -239,7 +236,7 @@ aaabbbcaaabbbccaaabbbcdeaaabbbceaaabbbdee* put aaabbbdee* ---- -Size: 30 +Size: 26 nKeys=6; bundleSize=4 blockPrefixLen=6; currentBundleLen=10; currentBundleKeys=1 Offsets: @@ -250,7 +247,7 @@ aaabbbcaaabbbccaaabbbcdeaaabbbceaaabbbdee* put aaabbbdee* ---- -Size: 31 +Size: 27 nKeys=7; bundleSize=4 blockPrefixLen=6; currentBundleLen=10; currentBundleKeys=1 Offsets: @@ -261,7 +258,7 @@ aaabbbcaaabbbccaaabbbcdeaaabbbceaaabbbdee* put aaabbbeff ---- -Size: 35 +Size: 31 nKeys=8; bundleSize=4 blockPrefixLen=6; currentBundleLen=19; currentBundleKeys=2 Offsets: @@ -273,7 +270,7 @@ aaabbbcaaabbbccaaabbbcdeaaabbbceaaabbbdee*aaabbbeff put aaabbe ---- -Size: 39 +Size: 35 nKeys=9; bundleSize=4 blockPrefixLen=5; currentBundleLen=6; currentBundleKeys=1 Offsets: @@ -285,7 +282,7 @@ aaabbbcaaabbbccaaabbbcdeaaabbbceaaabbbdee*aaabbbeffaaabbe put aaabbeef* ---- -Size: 43 +Size: 39 nKeys=10; bundleSize=4 blockPrefixLen=5; currentBundleLen=15; currentBundleKeys=2 Offsets: @@ -298,7 +295,7 @@ bbeef* put aaabbeef* ---- -Size: 44 +Size: 40 nKeys=11; bundleSize=4 blockPrefixLen=5; currentBundleLen=15; currentBundleKeys=2 Offsets: @@ -311,7 +308,7 @@ bbeef* put aaabc ---- -Size: 50 +Size: 46 nKeys=12; bundleSize=4 blockPrefixLen=4; currentBundleLen=20; currentBundleKeys=3 Offsets: @@ -325,7 +322,7 @@ put aabcceef* aabcceef* ---- -Size: 64 +Size: 60 nKeys=14; bundleSize=4 blockPrefixLen=2; currentBundleLen=9; currentBundleKeys=1 Offsets: @@ -340,7 +337,7 @@ bbeef*aaabcaabcceef* put aabcceegggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggg ---- -Size: 339 +Size: 335 nKeys=15; bundleSize=4 blockPrefixLen=2; currentBundleLen=270; currentBundleKeys=2 Offsets: @@ -361,47 +358,46 @@ finish rows=14 # PrefixBytes 00-01: x 02 # bundleSize: 4 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 02 # data[0] = 2 [27 overall] -07-08: x 07 # data[1] = 7 [32 overall] -08-09: x 07 # data[2] = 7 [32 overall] -09-10: x 08 # data[3] = 8 [33 overall] -10-11: x 0a # data[4] = 10 [35 overall] -11-12: x 0b # data[5] = 11 [36 overall] -12-13: x 0f # data[6] = 15 [40 overall] -13-14: x 13 # data[7] = 19 [44 overall] -14-15: x 13 # data[8] = 19 [44 overall] -15-16: x 13 # data[9] = 19 [44 overall] -16-17: x 16 # data[10] = 22 [47 overall] -17-18: x 18 # data[11] = 24 [49 overall] -18-19: x 1a # data[12] = 26 [51 overall] -19-20: x 1f # data[13] = 31 [56 overall] -20-21: x 1f # data[14] = 31 [56 overall] -21-22: x 20 # data[15] = 32 [57 overall] -22-23: x 27 # data[16] = 39 [64 overall] -23-24: x 27 # data[17] = 39 [64 overall] -24-25: x 27 # data[18] = 39 [64 overall] +01-02: x 01 # encoding: 1b +02-03: x 02 # data[0] = 2 [23 overall] +03-04: x 07 # data[1] = 7 [28 overall] +04-05: x 07 # data[2] = 7 [28 overall] +05-06: x 08 # data[3] = 8 [29 overall] +06-07: x 0a # data[4] = 10 [31 overall] +07-08: x 0b # data[5] = 11 [32 overall] +08-09: x 0f # data[6] = 15 [36 overall] +09-10: x 13 # data[7] = 19 [40 overall] +10-11: x 13 # data[8] = 19 [40 overall] +11-12: x 13 # data[9] = 19 [40 overall] +12-13: x 16 # data[10] = 22 [43 overall] +13-14: x 18 # data[11] = 24 [45 overall] +14-15: x 1a # data[12] = 26 [47 overall] +15-16: x 1f # data[13] = 31 [52 overall] +16-17: x 1f # data[14] = 31 [52 overall] +17-18: x 20 # data[15] = 32 [53 overall] +18-19: x 27 # data[16] = 39 [60 overall] +19-20: x 27 # data[17] = 39 [60 overall] +20-21: x 27 # data[18] = 39 [60 overall] # Data -25-27: x 6161 # data[00]: aa (block prefix) -27-32: x 6162626263 # data[01]: ..abbbc (bundle prefix) -32-32: x # data[02]: ....... -32-33: x 63 # data[03]: .......c -33-35: x 6465 # data[04]: .......de -35-36: x 65 # data[05]: .......e -36-40: x 61626262 # data[06]: ..abbb (bundle prefix) -40-44: x 6465652a # data[07]: ......dee* -44-44: x # data[08]: .......... -44-44: x # data[09]: .......... -44-47: x 656666 # data[10]: ......eff -47-49: x 6162 # data[11]: ..ab (bundle prefix) -49-51: x 6265 # data[12]: ....be -51-56: x 626565662a # data[13]: ....beef* -56-56: x # data[14]: ......... -56-57: x 63 # data[15]: ....c -57-64: x 6263636565662a # data[16]: ..bcceef* (bundle prefix) -64-64: x # data[17]: ......... -64-64: x # data[18]: ......... +21-23: x 6161 # data[00]: aa (block prefix) +23-28: x 6162626263 # data[01]: ..abbbc (bundle prefix) +28-28: x # data[02]: ....... +28-29: x 63 # data[03]: .......c +29-31: x 6465 # data[04]: .......de +31-32: x 65 # data[05]: .......e +32-36: x 61626262 # data[06]: ..abbb (bundle prefix) +36-40: x 6465652a # data[07]: ......dee* +40-40: x # data[08]: .......... +40-40: x # data[09]: .......... +40-43: x 656666 # data[10]: ......eff +43-45: x 6162 # data[11]: ..ab (bundle prefix) +45-47: x 6265 # data[12]: ....be +47-52: x 626565662a # data[13]: ....beef* +52-52: x # data[14]: ......... +52-53: x 63 # data[15]: ....c +53-60: x 6263636565662a # data[16]: ..bcceef* (bundle prefix) +60-60: x # data[17]: ......... +60-60: x # data[18]: ......... get indices=(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13) @@ -469,61 +465,60 @@ finish rows=15 # PrefixBytes 000-001: x 02 # bundleSize: 4 # Offsets table -001-002: x 03 # delta encoding: delta16 -002-006: x 00000000 # 32-bit constant: 0 -006-008: x 0200 # data[0] = 2 [48 overall] -008-010: x 0700 # data[1] = 7 [53 overall] -010-012: x 0700 # data[2] = 7 [53 overall] -012-014: x 0800 # data[3] = 8 [54 overall] -014-016: x 0a00 # data[4] = 10 [56 overall] -016-018: x 0b00 # data[5] = 11 [57 overall] -018-020: x 0f00 # data[6] = 15 [61 overall] -020-022: x 1300 # data[7] = 19 [65 overall] -022-024: x 1300 # data[8] = 19 [65 overall] -024-026: x 1300 # data[9] = 19 [65 overall] -026-028: x 1600 # data[10] = 22 [68 overall] -028-030: x 1800 # data[11] = 24 [70 overall] -030-032: x 1a00 # data[12] = 26 [72 overall] -032-034: x 1f00 # data[13] = 31 [77 overall] -034-036: x 1f00 # data[14] = 31 [77 overall] -036-038: x 2000 # data[15] = 32 [78 overall] -038-040: x 2500 # data[16] = 37 [83 overall] -040-042: x 2700 # data[17] = 39 [85 overall] -042-044: x 2700 # data[18] = 39 [85 overall] -044-046: x 2501 # data[19] = 293 [339 overall] +001-002: x 02 # encoding: 2b +002-004: x 0200 # data[0] = 2 [44 overall] +004-006: x 0700 # data[1] = 7 [49 overall] +006-008: x 0700 # data[2] = 7 [49 overall] +008-010: x 0800 # data[3] = 8 [50 overall] +010-012: x 0a00 # data[4] = 10 [52 overall] +012-014: x 0b00 # data[5] = 11 [53 overall] +014-016: x 0f00 # data[6] = 15 [57 overall] +016-018: x 1300 # data[7] = 19 [61 overall] +018-020: x 1300 # data[8] = 19 [61 overall] +020-022: x 1300 # data[9] = 19 [61 overall] +022-024: x 1600 # data[10] = 22 [64 overall] +024-026: x 1800 # data[11] = 24 [66 overall] +026-028: x 1a00 # data[12] = 26 [68 overall] +028-030: x 1f00 # data[13] = 31 [73 overall] +030-032: x 1f00 # data[14] = 31 [73 overall] +032-034: x 2000 # data[15] = 32 [74 overall] +034-036: x 2500 # data[16] = 37 [79 overall] +036-038: x 2700 # data[17] = 39 [81 overall] +038-040: x 2700 # data[18] = 39 [81 overall] +040-042: x 2501 # data[19] = 293 [335 overall] # Data -046-048: x 6161 # data[00]: aa (block prefix) -048-053: x 6162626263 # data[01]: ..abbbc (bundle prefix) -053-053: x # data[02]: ....... -053-054: x 63 # data[03]: .......c -054-056: x 6465 # data[04]: .......de -056-057: x 65 # data[05]: .......e -057-061: x 61626262 # data[06]: ..abbb (bundle prefix) -061-065: x 6465652a # data[07]: ......dee* -065-065: x # data[08]: .......... -065-065: x # data[09]: .......... -065-068: x 656666 # data[10]: ......eff -068-070: x 6162 # data[11]: ..ab (bundle prefix) -070-072: x 6265 # data[12]: ....be -072-077: x 626565662a # data[13]: ....beef* -077-077: x # data[14]: ......... -077-078: x 63 # data[15]: ....c -078-083: x 6263636565 # data[16]: ..bccee (bundle prefix) -083-085: x 662a # data[17]: .......f* -085-085: x # data[18]: ......... -085-105: x 6767676767676767676767676767676767676767 # data[19]: .......gggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggg -105-125: x 6767676767676767676767676767676767676767 # (continued...) -125-145: x 6767676767676767676767676767676767676767 # (continued...) -145-165: x 6767676767676767676767676767676767676767 # (continued...) -165-185: x 6767676767676767676767676767676767676767 # (continued...) -185-205: x 6767676767676767676767676767676767676767 # (continued...) -205-225: x 6767676767676767676767676767676767676767 # (continued...) -225-245: x 6767676767676767676767676767676767676767 # (continued...) -245-265: x 6767676767676767676767676767676767676767 # (continued...) -265-285: x 6767676767676767676767676767676767676767 # (continued...) -285-305: x 6767676767676767676767676767676767676767 # (continued...) -305-325: x 6767676767676767676767676767676767676767 # (continued...) -325-339: x 6767676767676767676767676767 # (continued...) +042-044: x 6161 # data[00]: aa (block prefix) +044-049: x 6162626263 # data[01]: ..abbbc (bundle prefix) +049-049: x # data[02]: ....... +049-050: x 63 # data[03]: .......c +050-052: x 6465 # data[04]: .......de +052-053: x 65 # data[05]: .......e +053-057: x 61626262 # data[06]: ..abbb (bundle prefix) +057-061: x 6465652a # data[07]: ......dee* +061-061: x # data[08]: .......... +061-061: x # data[09]: .......... +061-064: x 656666 # data[10]: ......eff +064-066: x 6162 # data[11]: ..ab (bundle prefix) +066-068: x 6265 # data[12]: ....be +068-073: x 626565662a # data[13]: ....beef* +073-073: x # data[14]: ......... +073-074: x 63 # data[15]: ....c +074-079: x 6263636565 # data[16]: ..bccee (bundle prefix) +079-081: x 662a # data[17]: .......f* +081-081: x # data[18]: ......... +081-101: x 6767676767676767676767676767676767676767 # data[19]: .......gggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggggg +101-121: x 6767676767676767676767676767676767676767 # (continued...) +121-141: x 6767676767676767676767676767676767676767 # (continued...) +141-161: x 6767676767676767676767676767676767676767 # (continued...) +161-181: x 6767676767676767676767676767676767676767 # (continued...) +181-201: x 6767676767676767676767676767676767676767 # (continued...) +201-221: x 6767676767676767676767676767676767676767 # (continued...) +221-241: x 6767676767676767676767676767676767676767 # (continued...) +241-261: x 6767676767676767676767676767676767676767 # (continued...) +261-281: x 6767676767676767676767676767676767676767 # (continued...) +281-301: x 6767676767676767676767676767676767676767 # (continued...) +301-321: x 6767676767676767676767676767676767676767 # (continued...) +321-335: x 6767676767676767676767676767 # (continued...) init bundle-size=2 ---- @@ -546,7 +541,7 @@ aabcceef* aabcceef* aabcceef* ---- -Size: 94 +Size: 90 nKeys=15; bundleSize=2 blockPrefixLen=2; currentBundleLen=9; currentBundleKeys=1 Offsets: @@ -562,110 +557,108 @@ finish rows=14 # PrefixBytes 00-01: x 01 # bundleSize: 2 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 02 # data[0] = 2 [30 overall] -07-08: x 07 # data[1] = 7 [35 overall] -08-09: x 07 # data[2] = 7 [35 overall] -09-10: x 08 # data[3] = 8 [36 overall] -10-11: x 0d # data[4] = 13 [41 overall] -11-12: x 0f # data[5] = 15 [43 overall] -12-13: x 10 # data[6] = 16 [44 overall] -13-14: x 18 # data[7] = 24 [52 overall] -14-15: x 18 # data[8] = 24 [52 overall] -15-16: x 18 # data[9] = 24 [52 overall] -16-17: x 1c # data[10] = 28 [56 overall] -17-18: x 20 # data[11] = 32 [60 overall] -18-19: x 23 # data[12] = 35 [63 overall] -19-20: x 27 # data[13] = 39 [67 overall] -20-21: x 27 # data[14] = 39 [67 overall] -21-22: x 2a # data[15] = 42 [70 overall] -22-23: x 2c # data[16] = 44 [72 overall] -23-24: x 31 # data[17] = 49 [77 overall] -24-25: x 32 # data[18] = 50 [78 overall] -25-26: x 39 # data[19] = 57 [85 overall] -26-27: x 39 # data[20] = 57 [85 overall] -27-28: x 39 # data[21] = 57 [85 overall] +01-02: x 01 # encoding: 1b +02-03: x 02 # data[0] = 2 [26 overall] +03-04: x 07 # data[1] = 7 [31 overall] +04-05: x 07 # data[2] = 7 [31 overall] +05-06: x 08 # data[3] = 8 [32 overall] +06-07: x 0d # data[4] = 13 [37 overall] +07-08: x 0f # data[5] = 15 [39 overall] +08-09: x 10 # data[6] = 16 [40 overall] +09-10: x 18 # data[7] = 24 [48 overall] +10-11: x 18 # data[8] = 24 [48 overall] +11-12: x 18 # data[9] = 24 [48 overall] +12-13: x 1c # data[10] = 28 [52 overall] +13-14: x 20 # data[11] = 32 [56 overall] +14-15: x 23 # data[12] = 35 [59 overall] +15-16: x 27 # data[13] = 39 [63 overall] +16-17: x 27 # data[14] = 39 [63 overall] +17-18: x 2a # data[15] = 42 [66 overall] +18-19: x 2c # data[16] = 44 [68 overall] +19-20: x 31 # data[17] = 49 [73 overall] +20-21: x 32 # data[18] = 50 [74 overall] +21-22: x 39 # data[19] = 57 [81 overall] +22-23: x 39 # data[20] = 57 [81 overall] +23-24: x 39 # data[21] = 57 [81 overall] # Data -28-30: x 6161 # data[00]: aa (block prefix) -30-35: x 6162626263 # data[01]: ..abbbc (bundle prefix) -35-35: x # data[02]: ....... -35-36: x 63 # data[03]: .......c -36-41: x 6162626263 # data[04]: ..abbbc (bundle prefix) -41-43: x 6465 # data[05]: .......de -43-44: x 65 # data[06]: .......e -44-52: x 616262626465652a # data[07]: ..abbbdee* (bundle prefix) -52-52: x # data[08]: .......... -52-52: x # data[09]: .......... -52-56: x 61626262 # data[10]: ..abbb (bundle prefix) -56-60: x 6465652a # data[11]: ......dee* -60-63: x 656666 # data[12]: ......eff -63-67: x 61626265 # data[13]: ..abbe (bundle prefix) -67-67: x # data[14]: ...... -67-70: x 65662a # data[15]: ......ef* -70-72: x 6162 # data[16]: ..ab (bundle prefix) -72-77: x 626565662a # data[17]: ....beef* -77-78: x 63 # data[18]: ....c -78-85: x 6263636565662a # data[19]: ..bcceef* (bundle prefix) -85-85: x # data[20]: ......... -85-85: x # data[21]: ......... +24-26: x 6161 # data[00]: aa (block prefix) +26-31: x 6162626263 # data[01]: ..abbbc (bundle prefix) +31-31: x # data[02]: ....... +31-32: x 63 # data[03]: .......c +32-37: x 6162626263 # data[04]: ..abbbc (bundle prefix) +37-39: x 6465 # data[05]: .......de +39-40: x 65 # data[06]: .......e +40-48: x 616262626465652a # data[07]: ..abbbdee* (bundle prefix) +48-48: x # data[08]: .......... +48-48: x # data[09]: .......... +48-52: x 61626262 # data[10]: ..abbb (bundle prefix) +52-56: x 6465652a # data[11]: ......dee* +56-59: x 656666 # data[12]: ......eff +59-63: x 61626265 # data[13]: ..abbe (bundle prefix) +63-63: x # data[14]: ...... +63-66: x 65662a # data[15]: ......ef* +66-68: x 6162 # data[16]: ..ab (bundle prefix) +68-73: x 626565662a # data[17]: ....beef* +73-74: x 63 # data[18]: ....c +74-81: x 6263636565662a # data[19]: ..bcceef* (bundle prefix) +81-81: x # data[20]: ......... +81-81: x # data[21]: ......... finish rows=15 ---- # PrefixBytes 00-01: x 01 # bundleSize: 2 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 02 # data[0] = 2 [32 overall] -07-08: x 07 # data[1] = 7 [37 overall] -08-09: x 07 # data[2] = 7 [37 overall] -09-10: x 08 # data[3] = 8 [38 overall] -10-11: x 0d # data[4] = 13 [43 overall] -11-12: x 0f # data[5] = 15 [45 overall] -12-13: x 10 # data[6] = 16 [46 overall] -13-14: x 18 # data[7] = 24 [54 overall] -14-15: x 18 # data[8] = 24 [54 overall] -15-16: x 18 # data[9] = 24 [54 overall] -16-17: x 1c # data[10] = 28 [58 overall] -17-18: x 20 # data[11] = 32 [62 overall] -18-19: x 23 # data[12] = 35 [65 overall] -19-20: x 27 # data[13] = 39 [69 overall] -20-21: x 27 # data[14] = 39 [69 overall] -21-22: x 2a # data[15] = 42 [72 overall] -22-23: x 2c # data[16] = 44 [74 overall] -23-24: x 31 # data[17] = 49 [79 overall] -24-25: x 32 # data[18] = 50 [80 overall] -25-26: x 39 # data[19] = 57 [87 overall] -26-27: x 39 # data[20] = 57 [87 overall] -27-28: x 39 # data[21] = 57 [87 overall] -28-29: x 40 # data[22] = 64 [94 overall] -29-30: x 40 # data[23] = 64 [94 overall] +01-02: x 01 # encoding: 1b +02-03: x 02 # data[0] = 2 [28 overall] +03-04: x 07 # data[1] = 7 [33 overall] +04-05: x 07 # data[2] = 7 [33 overall] +05-06: x 08 # data[3] = 8 [34 overall] +06-07: x 0d # data[4] = 13 [39 overall] +07-08: x 0f # data[5] = 15 [41 overall] +08-09: x 10 # data[6] = 16 [42 overall] +09-10: x 18 # data[7] = 24 [50 overall] +10-11: x 18 # data[8] = 24 [50 overall] +11-12: x 18 # data[9] = 24 [50 overall] +12-13: x 1c # data[10] = 28 [54 overall] +13-14: x 20 # data[11] = 32 [58 overall] +14-15: x 23 # data[12] = 35 [61 overall] +15-16: x 27 # data[13] = 39 [65 overall] +16-17: x 27 # data[14] = 39 [65 overall] +17-18: x 2a # data[15] = 42 [68 overall] +18-19: x 2c # data[16] = 44 [70 overall] +19-20: x 31 # data[17] = 49 [75 overall] +20-21: x 32 # data[18] = 50 [76 overall] +21-22: x 39 # data[19] = 57 [83 overall] +22-23: x 39 # data[20] = 57 [83 overall] +23-24: x 39 # data[21] = 57 [83 overall] +24-25: x 40 # data[22] = 64 [90 overall] +25-26: x 40 # data[23] = 64 [90 overall] # Data -30-32: x 6161 # data[00]: aa (block prefix) -32-37: x 6162626263 # data[01]: ..abbbc (bundle prefix) -37-37: x # data[02]: ....... -37-38: x 63 # data[03]: .......c -38-43: x 6162626263 # data[04]: ..abbbc (bundle prefix) -43-45: x 6465 # data[05]: .......de -45-46: x 65 # data[06]: .......e -46-54: x 616262626465652a # data[07]: ..abbbdee* (bundle prefix) -54-54: x # data[08]: .......... -54-54: x # data[09]: .......... -54-58: x 61626262 # data[10]: ..abbb (bundle prefix) -58-62: x 6465652a # data[11]: ......dee* -62-65: x 656666 # data[12]: ......eff -65-69: x 61626265 # data[13]: ..abbe (bundle prefix) -69-69: x # data[14]: ...... -69-72: x 65662a # data[15]: ......ef* -72-74: x 6162 # data[16]: ..ab (bundle prefix) -74-79: x 626565662a # data[17]: ....beef* -79-80: x 63 # data[18]: ....c -80-87: x 6263636565662a # data[19]: ..bcceef* (bundle prefix) -87-87: x # data[20]: ......... -87-87: x # data[21]: ......... -87-94: x 6263636565662a # data[22]: ..bcceef* (bundle prefix) -94-94: x # data[23]: ......... +26-28: x 6161 # data[00]: aa (block prefix) +28-33: x 6162626263 # data[01]: ..abbbc (bundle prefix) +33-33: x # data[02]: ....... +33-34: x 63 # data[03]: .......c +34-39: x 6162626263 # data[04]: ..abbbc (bundle prefix) +39-41: x 6465 # data[05]: .......de +41-42: x 65 # data[06]: .......e +42-50: x 616262626465652a # data[07]: ..abbbdee* (bundle prefix) +50-50: x # data[08]: .......... +50-50: x # data[09]: .......... +50-54: x 61626262 # data[10]: ..abbb (bundle prefix) +54-58: x 6465652a # data[11]: ......dee* +58-61: x 656666 # data[12]: ......eff +61-65: x 61626265 # data[13]: ..abbe (bundle prefix) +65-65: x # data[14]: ...... +65-68: x 65662a # data[15]: ......ef* +68-70: x 6162 # data[16]: ..ab (bundle prefix) +70-75: x 626565662a # data[17]: ....beef* +75-76: x 63 # data[18]: ....c +76-83: x 6263636565662a # data[19]: ..bcceef* (bundle prefix) +83-83: x # data[20]: ......... +83-83: x # data[21]: ......... +83-90: x 6263636565662a # data[22]: ..bcceef* (bundle prefix) +90-90: x # data[23]: ......... get indices=(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14) ---- @@ -697,7 +690,7 @@ abce abcf abg ---- -Size: 21 +Size: 17 nKeys=4; bundleSize=4 blockPrefixLen=2; currentBundleLen=15; currentBundleKeys=4 Offsets: @@ -708,21 +701,20 @@ abcdabceabcfabg finish rows=3 ---- # PrefixBytes -00-01: x 02 # bundleSize: 4 +00-01: x 02 # bundleSize: 4 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 03 # data[0] = 3 [14 overall] -07-08: x 03 # data[1] = 3 [14 overall] -08-09: x 04 # data[2] = 4 [15 overall] -09-10: x 05 # data[3] = 5 [16 overall] -10-11: x 06 # data[4] = 6 [17 overall] +01-02: x 01 # encoding: 1b +02-03: x 03 # data[0] = 3 [10 overall] +03-04: x 03 # data[1] = 3 [10 overall] +04-05: x 04 # data[2] = 4 [11 overall] +05-06: x 05 # data[3] = 5 [12 overall] +06-07: x 06 # data[4] = 6 [13 overall] # Data -11-14: x 616263 # data[00]: abc (block prefix) -14-14: x # data[01]: ... (bundle prefix) -14-15: x 64 # data[02]: ...d -15-16: x 65 # data[03]: ...e -16-17: x 66 # data[04]: ...f +07-10: x 616263 # data[00]: abc (block prefix) +10-10: x # data[01]: ... (bundle prefix) +10-11: x 64 # data[02]: ...d +11-12: x 65 # data[03]: ...e +12-13: x 66 # data[04]: ...f # Try finishing without the last key which forces a shorter bundle prefix only. @@ -736,7 +728,7 @@ abae abbf abc ---- -Size: 21 +Size: 17 nKeys=4; bundleSize=2 blockPrefixLen=2; currentBundleLen=7; currentBundleKeys=2 Offsets: @@ -747,23 +739,22 @@ abadabaeabbfabc finish rows=3 ---- # PrefixBytes -00-01: x 01 # bundleSize: 2 +00-01: x 01 # bundleSize: 2 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 02 # data[0] = 2 [14 overall] -07-08: x 03 # data[1] = 3 [15 overall] -08-09: x 04 # data[2] = 4 [16 overall] -09-10: x 05 # data[3] = 5 [17 overall] -10-11: x 07 # data[4] = 7 [19 overall] -11-12: x 07 # data[5] = 7 [19 overall] +01-02: x 01 # encoding: 1b +02-03: x 02 # data[0] = 2 [10 overall] +03-04: x 03 # data[1] = 3 [11 overall] +04-05: x 04 # data[2] = 4 [12 overall] +05-06: x 05 # data[3] = 5 [13 overall] +06-07: x 07 # data[4] = 7 [15 overall] +07-08: x 07 # data[5] = 7 [15 overall] # Data -12-14: x 6162 # data[00]: ab (block prefix) -14-15: x 61 # data[01]: ..a (bundle prefix) -15-16: x 64 # data[02]: ...d -16-17: x 65 # data[03]: ...e -17-19: x 6266 # data[04]: ..bf (bundle prefix) -19-19: x # data[05]: .... +08-10: x 6162 # data[00]: ab (block prefix) +10-11: x 61 # data[01]: ..a (bundle prefix) +11-12: x 64 # data[02]: ...d +12-13: x 65 # data[03]: ...e +13-15: x 6266 # data[04]: ..bf (bundle prefix) +15-15: x # data[05]: .... # Test strings long enough to force 16-bit offsets, and have zero-length block # and bundle prefixes. @@ -780,7 +771,7 @@ dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff ---- -Size: 686 +Size: 682 nKeys=6; bundleSize=2 blockPrefixLen=0; currentBundleLen=220; currentBundleKeys=2 Offsets: @@ -803,59 +794,58 @@ finish rows=6 # PrefixBytes 000-001: x 01 # bundleSize: 2 # Offsets table -001-002: x 03 # delta encoding: delta16 -002-006: x 00000000 # 32-bit constant: 0 -006-008: x 0000 # data[0] = 0 [26 overall] -008-010: x 0000 # data[1] = 0 [26 overall] -010-012: x 6e00 # data[2] = 110 [136 overall] -012-014: x dc00 # data[3] = 220 [246 overall] -014-016: x dc00 # data[4] = 220 [246 overall] -016-018: x 4a01 # data[5] = 330 [356 overall] -018-020: x b801 # data[6] = 440 [466 overall] -020-022: x b801 # data[7] = 440 [466 overall] -022-024: x 2602 # data[8] = 550 [576 overall] -024-026: x 9402 # data[9] = 660 [686 overall] +001-002: x 02 # encoding: 2b +002-004: x 0000 # data[0] = 0 [22 overall] +004-006: x 0000 # data[1] = 0 [22 overall] +006-008: x 6e00 # data[2] = 110 [132 overall] +008-010: x dc00 # data[3] = 220 [242 overall] +010-012: x dc00 # data[4] = 220 [242 overall] +012-014: x 4a01 # data[5] = 330 [352 overall] +014-016: x b801 # data[6] = 440 [462 overall] +016-018: x b801 # data[7] = 440 [462 overall] +018-020: x 2602 # data[8] = 550 [572 overall] +020-022: x 9402 # data[9] = 660 [682 overall] # Data -026-026: x # data[00]: (block prefix) -026-026: x # data[01]: (bundle prefix) -026-046: x 6161616161616161616161616161616161616161 # data[02]: aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -046-066: x 6161616161616161616161616161616161616161 # (continued...) -066-086: x 6161616161616161616161616161616161616161 # (continued...) -086-106: x 6161616161616161616161616161616161616161 # (continued...) -106-126: x 6161616161616161616161616161616161616161 # (continued...) -126-136: x 61616161616161616161 # (continued...) -136-156: x 6262626262626262626262626262626262626262 # data[03]: bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -156-176: x 6262626262626262626262626262626262626262 # (continued...) -176-196: x 6262626262626262626262626262626262626262 # (continued...) -196-216: x 6262626262626262626262626262626262626262 # (continued...) -216-236: x 6262626262626262626262626262626262626262 # (continued...) -236-246: x 62626262626262626262 # (continued...) -246-246: x # data[04]: (bundle prefix) -246-266: x 6363636363636363636363636363636363636363 # data[05]: cccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc -266-286: x 6363636363636363636363636363636363636363 # (continued...) -286-306: x 6363636363636363636363636363636363636363 # (continued...) -306-326: x 6363636363636363636363636363636363636363 # (continued...) -326-346: x 6363636363636363636363636363636363636363 # (continued...) -346-356: x 63636363636363636363 # (continued...) -356-376: x 6464646464646464646464646464646464646464 # data[06]: dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd -376-396: x 6464646464646464646464646464646464646464 # (continued...) -396-416: x 6464646464646464646464646464646464646464 # (continued...) -416-436: x 6464646464646464646464646464646464646464 # (continued...) -436-456: x 6464646464646464646464646464646464646464 # (continued...) -456-466: x 64646464646464646464 # (continued...) -466-466: x # data[07]: (bundle prefix) -466-486: x 6565656565656565656565656565656565656565 # data[08]: eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -486-506: x 6565656565656565656565656565656565656565 # (continued...) -506-526: x 6565656565656565656565656565656565656565 # (continued...) -526-546: x 6565656565656565656565656565656565656565 # (continued...) -546-566: x 6565656565656565656565656565656565656565 # (continued...) -566-576: x 65656565656565656565 # (continued...) -576-596: x 6666666666666666666666666666666666666666 # data[09]: ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -596-616: x 6666666666666666666666666666666666666666 # (continued...) -616-636: x 6666666666666666666666666666666666666666 # (continued...) -636-656: x 6666666666666666666666666666666666666666 # (continued...) -656-676: x 6666666666666666666666666666666666666666 # (continued...) -676-686: x 66666666666666666666 # (continued...) +022-022: x # data[00]: (block prefix) +022-022: x # data[01]: (bundle prefix) +022-042: x 6161616161616161616161616161616161616161 # data[02]: aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +042-062: x 6161616161616161616161616161616161616161 # (continued...) +062-082: x 6161616161616161616161616161616161616161 # (continued...) +082-102: x 6161616161616161616161616161616161616161 # (continued...) +102-122: x 6161616161616161616161616161616161616161 # (continued...) +122-132: x 61616161616161616161 # (continued...) +132-152: x 6262626262626262626262626262626262626262 # data[03]: bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +152-172: x 6262626262626262626262626262626262626262 # (continued...) +172-192: x 6262626262626262626262626262626262626262 # (continued...) +192-212: x 6262626262626262626262626262626262626262 # (continued...) +212-232: x 6262626262626262626262626262626262626262 # (continued...) +232-242: x 62626262626262626262 # (continued...) +242-242: x # data[04]: (bundle prefix) +242-262: x 6363636363636363636363636363636363636363 # data[05]: cccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc +262-282: x 6363636363636363636363636363636363636363 # (continued...) +282-302: x 6363636363636363636363636363636363636363 # (continued...) +302-322: x 6363636363636363636363636363636363636363 # (continued...) +322-342: x 6363636363636363636363636363636363636363 # (continued...) +342-352: x 63636363636363636363 # (continued...) +352-372: x 6464646464646464646464646464646464646464 # data[06]: dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd +372-392: x 6464646464646464646464646464646464646464 # (continued...) +392-412: x 6464646464646464646464646464646464646464 # (continued...) +412-432: x 6464646464646464646464646464646464646464 # (continued...) +432-452: x 6464646464646464646464646464646464646464 # (continued...) +452-462: x 64646464646464646464 # (continued...) +462-462: x # data[07]: (bundle prefix) +462-482: x 6565656565656565656565656565656565656565 # data[08]: eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +482-502: x 6565656565656565656565656565656565656565 # (continued...) +502-522: x 6565656565656565656565656565656565656565 # (continued...) +522-542: x 6565656565656565656565656565656565656565 # (continued...) +542-562: x 6565656565656565656565656565656565656565 # (continued...) +562-572: x 65656565656565656565 # (continued...) +572-592: x 6666666666666666666666666666666666666666 # data[09]: ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff +592-612: x 6666666666666666666666666666666666666666 # (continued...) +612-632: x 6666666666666666666666666666666666666666 # (continued...) +632-652: x 6666666666666666666666666666666666666666 # (continued...) +652-672: x 6666666666666666666666666666666666666666 # (continued...) +672-682: x 66666666666666666666 # (continued...) get indices=(0, 1, 2, 3, 4, 5) ---- @@ -878,7 +868,7 @@ aaaab aaaac aaaad ---- -Size: 20 +Size: 16 nKeys=4; bundleSize=4 blockPrefixLen=4; currentBundleLen=20; currentBundleKeys=4 Offsets: @@ -890,7 +880,7 @@ put aaaae aab ---- -Size: 27 +Size: 23 nKeys=6; bundleSize=4 blockPrefixLen=2; currentBundleLen=8; currentBundleKeys=2 Offsets: @@ -903,22 +893,21 @@ finish rows=5 # PrefixBytes 00-01: x 02 # bundleSize: 4 # Offsets table -01-02: x 02 # delta encoding: delta8 -02-06: x 00000000 # 32-bit constant: 0 -06-07: x 04 # data[0] = 4 [18 overall] -07-08: x 04 # data[1] = 4 [18 overall] -08-09: x 05 # data[2] = 5 [19 overall] -09-10: x 06 # data[3] = 6 [20 overall] -10-11: x 07 # data[4] = 7 [21 overall] -11-12: x 08 # data[5] = 8 [22 overall] -12-13: x 09 # data[6] = 9 [23 overall] -13-14: x 09 # data[7] = 9 [23 overall] +01-02: x 01 # encoding: 1b +02-03: x 04 # data[0] = 4 [14 overall] +03-04: x 04 # data[1] = 4 [14 overall] +04-05: x 05 # data[2] = 5 [15 overall] +05-06: x 06 # data[3] = 6 [16 overall] +06-07: x 07 # data[4] = 7 [17 overall] +07-08: x 08 # data[5] = 8 [18 overall] +08-09: x 09 # data[6] = 9 [19 overall] +09-10: x 09 # data[7] = 9 [19 overall] # Data -14-18: x 61616161 # data[00]: aaaa (block prefix) -18-18: x # data[01]: .... (bundle prefix) -18-19: x 61 # data[02]: ....a -19-20: x 62 # data[03]: ....b -20-21: x 63 # data[04]: ....c -21-22: x 64 # data[05]: ....d -22-23: x 65 # data[06]: ....e (bundle prefix) -23-23: x # data[07]: ..... +10-14: x 61616161 # data[00]: aaaa (block prefix) +14-14: x # data[01]: .... (bundle prefix) +14-15: x 61 # data[02]: ....a +15-16: x 62 # data[03]: ....b +16-17: x 63 # data[04]: ....c +17-18: x 64 # data[05]: ....d +18-19: x 65 # data[06]: ....e (bundle prefix) +19-19: x # data[07]: ..... diff --git a/sstable/colblk/testdata/raw_bytes b/sstable/colblk/testdata/raw_bytes index 9ae4cfa9bb..22f70ea299 100644 --- a/sstable/colblk/testdata/raw_bytes +++ b/sstable/colblk/testdata/raw_bytes @@ -5,53 +5,49 @@ a ---- # rawbytes # offsets table -0-1: x 02 # delta encoding: delta8 -1-5: x 00000000 # 32-bit constant: 0 -5-6: x 00 # data[0] = 0 [7 overall] -6-7: x 01 # data[1] = 1 [8 overall] +0-1: x 01 # encoding: 1b +1-2: x 00 # data[0] = 0 [3 overall] +2-3: x 01 # data[1] = 1 [4 overall] # data -7-8: x 61 # data[0]: a +3-4: x 61 # data[0]: a # Try the same thing, but with offsets 1, 2, 3, 4. build offset=1 a ---- -0-1: x 00 # start offset +0-1: x 00 # start offset # rawbytes # offsets table -1-2: x 02 # delta encoding: delta8 -2-6: x 00000000 # 32-bit constant: 0 -6-7: x 00 # data[0] = 0 [8 overall] -7-8: x 01 # data[1] = 1 [9 overall] +1-2: x 01 # encoding: 1b +2-3: x 00 # data[0] = 0 [4 overall] +3-4: x 01 # data[1] = 1 [5 overall] # data -8-9: x 61 # data[0]: a +4-5: x 61 # data[0]: a build offset=2 a ---- -00-02: x 0000 # start offset +0-2: x 0000 # start offset # rawbytes # offsets table -02-03: x 02 # delta encoding: delta8 -03-07: x 00000000 # 32-bit constant: 0 -07-08: x 00 # data[0] = 0 [9 overall] -08-09: x 01 # data[1] = 1 [10 overall] +2-3: x 01 # encoding: 1b +3-4: x 00 # data[0] = 0 [5 overall] +4-5: x 01 # data[1] = 1 [6 overall] # data -09-10: x 61 # data[0]: a +5-6: x 61 # data[0]: a build offset=3 a ---- -00-03: x 000000 # start offset +0-3: x 000000 # start offset # rawbytes # offsets table -03-04: x 02 # delta encoding: delta8 -04-08: x 00000000 # 32-bit constant: 0 -08-09: x 00 # data[0] = 0 [10 overall] -09-10: x 01 # data[1] = 1 [11 overall] +3-4: x 01 # encoding: 1b +4-5: x 00 # data[0] = 0 [6 overall] +5-6: x 01 # data[1] = 1 [7 overall] # data -10-11: x 61 # data[0]: a +6-7: x 61 # data[0]: a build offset=4 a @@ -59,12 +55,11 @@ a 00-04: x 00000000 # start offset # rawbytes # offsets table -04-05: x 02 # delta encoding: delta8 -05-09: x 00000000 # 32-bit constant: 0 -09-10: x 00 # data[0] = 0 [11 overall] -10-11: x 01 # data[1] = 1 [12 overall] +04-05: x 01 # encoding: 1b +05-06: x 00 # data[0] = 0 [7 overall] +06-07: x 01 # data[1] = 1 [8 overall] # data -11-12: x 61 # data[0]: a +07-08: x 61 # data[0]: a # Create a RawBytes with two byte slices: 'a' and 'b'. @@ -74,14 +69,13 @@ b ---- # rawbytes # offsets table -0-1: x 02 # delta encoding: delta8 -1-5: x 00000000 # 32-bit constant: 0 -5-6: x 00 # data[0] = 0 [8 overall] -6-7: x 01 # data[1] = 1 [9 overall] -7-8: x 02 # data[2] = 2 [10 overall] +0-1: x 01 # encoding: 1b +1-2: x 00 # data[0] = 0 [4 overall] +2-3: x 01 # data[1] = 1 [5 overall] +3-4: x 02 # data[2] = 2 [6 overall] # data -8-9: x 61 # data[0]: a -9-10: x 62 # data[1]: b +4-5: x 61 # data[0]: a +5-6: x 62 # data[1]: b build offset=0 a @@ -90,28 +84,26 @@ abc ---- # rawbytes # offsets table -00-01: x 02 # delta encoding: delta8 -01-05: x 00000000 # 32-bit constant: 0 -05-06: x 00 # data[0] = 0 [9 overall] -06-07: x 01 # data[1] = 1 [10 overall] -07-08: x 03 # data[2] = 3 [12 overall] -08-09: x 06 # data[3] = 6 [15 overall] +00-01: x 01 # encoding: 1b +01-02: x 00 # data[0] = 0 [5 overall] +02-03: x 01 # data[1] = 1 [6 overall] +03-04: x 03 # data[2] = 3 [8 overall] +04-05: x 06 # data[3] = 6 [11 overall] # data -09-10: x 61 # data[0]: a -10-12: x 6162 # data[1]: ab -12-15: x 616263 # data[2]: abc +05-06: x 61 # data[0]: a +06-08: x 6162 # data[1]: ab +08-11: x 616263 # data[2]: abc build offset=0 aaabbbc ---- # rawbytes # offsets table -00-01: x 02 # delta encoding: delta8 -01-05: x 00000000 # 32-bit constant: 0 -05-06: x 00 # data[0] = 0 [7 overall] -06-07: x 07 # data[1] = 7 [14 overall] +0-1: x 01 # encoding: 1b +1-2: x 00 # data[0] = 0 [3 overall] +2-3: x 07 # data[1] = 7 [10 overall] # data -07-14: x 61616162626263 # data[0]: aaabbbc +3-10: x 61616162626263 # data[0]: aaabbbc build offset=0 a @@ -120,16 +112,15 @@ abc ---- # rawbytes # offsets table -00-01: x 02 # delta encoding: delta8 -01-05: x 00000000 # 32-bit constant: 0 -05-06: x 00 # data[0] = 0 [9 overall] -06-07: x 01 # data[1] = 1 [10 overall] -07-08: x 03 # data[2] = 3 [12 overall] -08-09: x 06 # data[3] = 6 [15 overall] +00-01: x 01 # encoding: 1b +01-02: x 00 # data[0] = 0 [5 overall] +02-03: x 01 # data[1] = 1 [6 overall] +03-04: x 03 # data[2] = 3 [8 overall] +04-05: x 06 # data[3] = 6 [11 overall] # data -09-10: x 61 # data[0]: a -10-12: x 6162 # data[1]: ab -12-15: x 616263 # data[2]: abc +05-06: x 61 # data[0]: a +06-08: x 6162 # data[1]: ab +08-11: x 616263 # data[2]: abc at indices=(0, 1, 2) ---- @@ -150,28 +141,27 @@ kale ---- # rawbytes # offsets table -00-01: x 02 # delta encoding: delta8 -01-05: x 00000000 # 32-bit constant: 0 -05-06: x 00 # data[0] = 0 [15 overall] -06-07: x 03 # data[1] = 3 [18 overall] -07-08: x 09 # data[2] = 9 [24 overall] -08-09: x 11 # data[3] = 17 [32 overall] -09-10: x 16 # data[4] = 22 [37 overall] -10-11: x 1b # data[5] = 27 [42 overall] -11-12: x 21 # data[6] = 33 [48 overall] -12-13: x 2a # data[7] = 42 [57 overall] -13-14: x 31 # data[8] = 49 [64 overall] -14-15: x 35 # data[9] = 53 [68 overall] +00-01: x 01 # encoding: 1b +01-02: x 00 # data[0] = 0 [11 overall] +02-03: x 03 # data[1] = 3 [14 overall] +03-04: x 09 # data[2] = 9 [20 overall] +04-05: x 11 # data[3] = 17 [28 overall] +05-06: x 16 # data[4] = 22 [33 overall] +06-07: x 1b # data[5] = 27 [38 overall] +07-08: x 21 # data[6] = 33 [44 overall] +08-09: x 2a # data[7] = 42 [53 overall] +09-10: x 31 # data[8] = 49 [60 overall] +10-11: x 35 # data[9] = 53 [64 overall] # data -15-18: x 636174 # data[0]: cat -18-24: x 6f72616e6765 # data[1]: orange -24-32: x 7a75636368696e69 # data[2]: zucchini -32-37: x 6c656d6f6e # data[3]: lemon -37-42: x 6170706c65 # data[4]: apple -42-48: x 62616e616e61 # data[5]: banana -48-57: x 63616e74656c6f7065 # data[6]: cantelope -57-64: x 6c657474756365 # data[7]: lettuce -64-68: x 6b616c65 # data[8]: kale +11-14: x 636174 # data[0]: cat +14-20: x 6f72616e6765 # data[1]: orange +20-28: x 7a75636368696e69 # data[2]: zucchini +28-33: x 6c656d6f6e # data[3]: lemon +33-38: x 6170706c65 # data[4]: apple +38-44: x 62616e616e61 # data[5]: banana +44-53: x 63616e74656c6f7065 # data[6]: cantelope +53-60: x 6c657474756365 # data[7]: lettuce +60-64: x 6b616c65 # data[8]: kale at indices=(0, 1, 2, 3, 4, 5, 6, 7, 8) ---- @@ -200,18 +190,17 @@ kale ---- # rawbytes # offsets table -00-01: x 02 # delta encoding: delta8 -01-05: x 00000000 # 32-bit constant: 0 -05-06: x 00 # data[0] = 0 [10 overall] -06-07: x 03 # data[1] = 3 [13 overall] -07-08: x 09 # data[2] = 9 [19 overall] -08-09: x 11 # data[3] = 17 [27 overall] -09-10: x 16 # data[4] = 22 [32 overall] +00-01: x 01 # encoding: 1b +01-02: x 00 # data[0] = 0 [6 overall] +02-03: x 03 # data[1] = 3 [9 overall] +03-04: x 09 # data[2] = 9 [15 overall] +04-05: x 11 # data[3] = 17 [23 overall] +05-06: x 16 # data[4] = 22 [28 overall] # data -10-13: x 636174 # data[0]: cat -13-19: x 6f72616e6765 # data[1]: orange -19-27: x 7a75636368696e69 # data[2]: zucchini -27-32: x 6c656d6f6e # data[3]: lemon +06-09: x 636174 # data[0]: cat +09-15: x 6f72616e6765 # data[1]: orange +15-23: x 7a75636368696e69 # data[2]: zucchini +23-28: x 6c656d6f6e # data[3]: lemon at indices=(0, 1, 2, 3) ---- diff --git a/sstable/colblk/testdata/uints b/sstable/colblk/testdata/uints index d5e3fee116..84ab8de960 100644 --- a/sstable/colblk/testdata/uints +++ b/sstable/colblk/testdata/uints @@ -6,29 +6,23 @@ # Test a default-zero builder that only contains zeros. -init widths=(32) default-zero +init default-zero ---- -b32 size rows=(100) ---- -b32: - 32: *colblk.UintBuilder[uint32].Size(100, 0) = 5 +Size(100, 0) = 1 -finish widths=(32) rows=100 +finish rows=100 ---- -b32: *colblk.UintBuilder[uint32]: -0-1: x 01 # delta encoding: const -1-5: x 00000000 # 32-bit constant: 0 +0-1: x 00 # encoding: zero -# Initialize all four writers. +finish rows=10 +---- +0-1: x 00 # encoding: zero -init widths=(8, 16, 32, 64) +init ---- -b8 -b16 -b32 -b64 # Write a few zero values at index [0,4]. @@ -41,39 +35,21 @@ write size rows=(5, 4, 3, 2, 1, 0) ---- -b8: - 8: *colblk.UintBuilder[uint8].Size(5, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(4, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(3, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(2, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(1, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(0, 0) = 0 -b16: - 16: *colblk.UintBuilder[uint16].Size(5, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(4, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(3, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(2, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(1, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(0, 0) = 0 -b32: - 32: *colblk.UintBuilder[uint32].Size(5, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(4, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(3, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(2, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(1, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(0, 0) = 0 -b64: - 64: *colblk.UintBuilder[uint64].Size(5, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(4, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(3, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(2, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(1, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(0, 0) = 0 +Size(5, 0) = 1 +Size(4, 0) = 1 +Size(3, 0) = 1 +Size(2, 0) = 1 +Size(1, 0) = 1 +Size(0, 0) = 0 # Add a nonzero value. Size calls that include the new row count should # increase, but the size calls that don't include the new row count should not. # The increased sizes should reflect use of a uint8 delta encoding. +finish rows=8 +---- +0-1: x 00 # encoding: zero + write 5:10 6:0 @@ -82,52 +58,20 @@ write size rows=(8, 7, 6, 5, 4, 3, 2, 1, 0) ---- -b8: - 8: *colblk.UintBuilder[uint8].Size(8, 0) = 9 - 8: *colblk.UintBuilder[uint8].Size(7, 0) = 8 - 8: *colblk.UintBuilder[uint8].Size(6, 0) = 7 - 8: *colblk.UintBuilder[uint8].Size(5, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(4, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(3, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(2, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(1, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(0, 0) = 0 -b16: - 16: *colblk.UintBuilder[uint16].Size(8, 0) = 11 - 16: *colblk.UintBuilder[uint16].Size(7, 0) = 10 - 16: *colblk.UintBuilder[uint16].Size(6, 0) = 9 - 16: *colblk.UintBuilder[uint16].Size(5, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(4, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(3, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(2, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(1, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(0, 0) = 0 -b32: - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 - 32: *colblk.UintBuilder[uint32].Size(7, 0) = 12 - 32: *colblk.UintBuilder[uint32].Size(6, 0) = 11 - 32: *colblk.UintBuilder[uint32].Size(5, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(4, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(3, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(2, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(1, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(0, 0) = 0 -b64: - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 - 64: *colblk.UintBuilder[uint64].Size(7, 0) = 16 - 64: *colblk.UintBuilder[uint64].Size(6, 0) = 15 - 64: *colblk.UintBuilder[uint64].Size(5, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(4, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(3, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(2, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(1, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(0, 0) = 0 +Size(8, 0) = 9 +Size(7, 0) = 8 +Size(6, 0) = 7 +Size(5, 0) = 1 +Size(4, 0) = 1 +Size(3, 0) = 1 +Size(2, 0) = 1 +Size(1, 0) = 1 +Size(0, 0) = 0 -# Finish the b8 so we can test 16-bit encoding. -finish widths=(8) rows=8 +# Check width=8 encoding. +finish rows=8 ---- -b8: *colblk.UintBuilder[uint8]: -0-1: x 00 # delta encoding: none +0-1: x 01 # encoding: 1b 1-2: x 00 # data[0] = 0 2-3: x 00 # data[1] = 0 3-4: x 00 # data[2] = 0 @@ -136,11 +80,8 @@ b8: *colblk.UintBuilder[uint8]: 6-7: x 0a # data[5] = 10 7-8: x 00 # data[6] = 0 8-9: x 0a # data[7] = 10 -Keeping b16 open -Keeping b32 open -Keeping b64 open -# Add 1000 which should force a 16-bit delta encoding. +# Add 1000 which should force a 16-bit encoding. write 8:1000 @@ -148,52 +89,24 @@ write size rows=(9, 8) ---- -b16: - 16: *colblk.UintBuilder[uint16].Size(9, 0) = 20 - 16: *colblk.UintBuilder[uint16].Size(8, 0) = 11 -b32: - 32: *colblk.UintBuilder[uint32].Size(9, 0) = 24 - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 -b64: - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 +Size(9, 0) = 20 +Size(8, 0) = 9 size rows=(9, 8) offset=1 ---- -b16: - 16: *colblk.UintBuilder[uint16].Size(9, 1) = 20 [19 w/o offset] - 16: *colblk.UintBuilder[uint16].Size(8, 1) = 12 [11 w/o offset] -b32: - 32: *colblk.UintBuilder[uint32].Size(9, 1) = 24 [23 w/o offset] - 32: *colblk.UintBuilder[uint32].Size(8, 1) = 14 [13 w/o offset] -b64: - 64: *colblk.UintBuilder[uint64].Size(9, 1) = 28 [27 w/o offset] - 64: *colblk.UintBuilder[uint64].Size(8, 1) = 18 [17 w/o offset] +Size(9, 1) = 20 [19 w/o offset] +Size(8, 1) = 10 [9 w/o offset] size rows=(9, 8) offset=2 ---- -b16: - 16: *colblk.UintBuilder[uint16].Size(9, 2) = 22 [20 w/o offset] - 16: *colblk.UintBuilder[uint16].Size(8, 2) = 13 [11 w/o offset] -b32: - 32: *colblk.UintBuilder[uint32].Size(9, 2) = 26 [24 w/o offset] - 32: *colblk.UintBuilder[uint32].Size(8, 2) = 15 [13 w/o offset] -b64: - 64: *colblk.UintBuilder[uint64].Size(9, 2) = 30 [28 w/o offset] - 64: *colblk.UintBuilder[uint64].Size(8, 2) = 19 [17 w/o offset] +Size(9, 2) = 22 [20 w/o offset] +Size(8, 2) = 11 [9 w/o offset] size rows=(9, 8) offset=5 ---- -b16: - 16: *colblk.UintBuilder[uint16].Size(9, 5) = 24 [19 w/o offset] - 16: *colblk.UintBuilder[uint16].Size(8, 5) = 16 [11 w/o offset] -b32: - 32: *colblk.UintBuilder[uint32].Size(9, 5) = 28 [23 w/o offset] - 32: *colblk.UintBuilder[uint32].Size(8, 5) = 18 [13 w/o offset] -b64: - 64: *colblk.UintBuilder[uint64].Size(9, 5) = 32 [27 w/o offset] - 64: *colblk.UintBuilder[uint64].Size(8, 5) = 22 [17 w/o offset] +Size(9, 5) = 24 [19 w/o offset] +Size(8, 5) = 14 [9 w/o offset] # We should be able to write up to 2^16-1 without triggering a 32-bit encoding. @@ -203,28 +116,14 @@ write size rows=(10, 9, 8) ---- -b16: - 16: *colblk.UintBuilder[uint16].Size(10, 0) = 22 - 16: *colblk.UintBuilder[uint16].Size(9, 0) = 20 - 16: *colblk.UintBuilder[uint16].Size(8, 0) = 11 -b32: - 32: *colblk.UintBuilder[uint32].Size(10, 0) = 26 - 32: *colblk.UintBuilder[uint32].Size(9, 0) = 24 - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 -b64: - 64: *colblk.UintBuilder[uint64].Size(10, 0) = 30 - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 - -# But 2^16 should trigger a 32-bit encoding. (Finish b16 so we can test 32-bit -# encoding.) - -finish widths=(16) rows=10 ----- -b16: *colblk.UintBuilder[uint16]: -00-01: x 00 # delta encoding: none -# padding -01-02: x 00 # aligning to 16-bit boundary +Size(10, 0) = 22 +Size(9, 0) = 20 +Size(8, 0) = 9 + +finish rows=10 +---- +00-01: x 02 # encoding: 2b +01-02: x 00 # padding (aligning to 16-bit boundary) 02-04: x 0000 # data[0] = 0 04-06: x 0000 # data[1] = 0 06-08: x 0000 # data[2] = 0 @@ -235,25 +134,19 @@ b16: *colblk.UintBuilder[uint16]: 16-18: x 0a00 # data[7] = 10 18-20: x e803 # data[8] = 1000 20-22: x ffff # data[9] = 65535 -Keeping b32 open -Keeping b64 open +# 2^16 should trigger a 32-bit encoding. +# write 10:65536 ---- size rows=(11, 10, 9, 8) ---- -b32: - 32: *colblk.UintBuilder[uint32].Size(11, 0) = 48 - 32: *colblk.UintBuilder[uint32].Size(10, 0) = 26 - 32: *colblk.UintBuilder[uint32].Size(9, 0) = 24 - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 -b64: - 64: *colblk.UintBuilder[uint64].Size(11, 0) = 56 - 64: *colblk.UintBuilder[uint64].Size(10, 0) = 30 - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 +Size(11, 0) = 48 +Size(10, 0) = 22 +Size(9, 0) = 20 +Size(8, 0) = 9 # We should be able to write up to 2^32-1 without triggering a 64-bit encoding. @@ -263,27 +156,16 @@ write size rows=(12, 11, 10, 9, 8) ---- -b32: - 32: *colblk.UintBuilder[uint32].Size(12, 0) = 52 - 32: *colblk.UintBuilder[uint32].Size(11, 0) = 48 - 32: *colblk.UintBuilder[uint32].Size(10, 0) = 26 - 32: *colblk.UintBuilder[uint32].Size(9, 0) = 24 - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 -b64: - 64: *colblk.UintBuilder[uint64].Size(12, 0) = 60 - 64: *colblk.UintBuilder[uint64].Size(11, 0) = 56 - 64: *colblk.UintBuilder[uint64].Size(10, 0) = 30 - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 - -# But 2^32 should trigger a 64-bit encoding. - -finish widths=(32) rows=12 ----- -b32: *colblk.UintBuilder[uint32]: -00-01: x 00 # delta encoding: none -# padding -01-04: x 000000 # aligning to 32-bit boundary +Size(12, 0) = 52 +Size(11, 0) = 48 +Size(10, 0) = 22 +Size(9, 0) = 20 +Size(8, 0) = 9 + +finish rows=12 +---- +00-01: x 04 # encoding: 4b +01-04: x 000000 # padding (aligning to 32-bit boundary) 04-08: x 00000000 # data[0] = 0 08-12: x 00000000 # data[1] = 0 12-16: x 00000000 # data[2] = 0 @@ -296,28 +178,26 @@ b32: *colblk.UintBuilder[uint32]: 40-44: x ffff0000 # data[9] = 65535 44-48: x 00000100 # data[10] = 65536 48-52: x ffffffff # data[11] = 4294967295 -Keeping b64 open +# 2^32 should trigger a 64-bit encoding. +# write 12:4294967296 ---- size rows=(13, 12, 11, 10, 9, 8) ---- -b64: - 64: *colblk.UintBuilder[uint64].Size(13, 0) = 112 - 64: *colblk.UintBuilder[uint64].Size(12, 0) = 60 - 64: *colblk.UintBuilder[uint64].Size(11, 0) = 56 - 64: *colblk.UintBuilder[uint64].Size(10, 0) = 30 - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 +Size(13, 0) = 112 +Size(12, 0) = 52 +Size(11, 0) = 48 +Size(10, 0) = 22 +Size(9, 0) = 20 +Size(8, 0) = 9 -finish widths=(64) rows=13 +finish rows=13 ---- -b64: *colblk.UintBuilder[uint64]: -000-001: x 00 # delta encoding: none -# padding -001-008: x 00000000000000 # aligning to 64-bit boundary +000-001: x 08 # encoding: 8b +001-008: x 00000000000000 # padding (aligning to 64-bit boundary) 008-016: x 0000000000000000 # data[0] = 0 016-024: x 0000000000000000 # data[1] = 0 024-032: x 0000000000000000 # data[2] = 0 @@ -335,46 +215,20 @@ b64: *colblk.UintBuilder[uint64]: # Repeat the above tests but with a zero default value, and without explicitly # setting any of the zero values. -init widths=(8, 16, 32, 64) default-zero +init default-zero ---- -b8 -b16 -b32 -b64 # At all row counts, the column should be encoded as a constant using the column # type width. size rows=(5, 4, 3, 2, 1, 0) ---- -b8: - 8: *colblk.UintBuilder[uint8].Size(5, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(4, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(3, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(2, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(1, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(0, 0) = 0 -b16: - 16: *colblk.UintBuilder[uint16].Size(5, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(4, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(3, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(2, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(1, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(0, 0) = 0 -b32: - 32: *colblk.UintBuilder[uint32].Size(5, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(4, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(3, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(2, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(1, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(0, 0) = 0 -b64: - 64: *colblk.UintBuilder[uint64].Size(5, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(4, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(3, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(2, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(1, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(0, 0) = 0 +Size(5, 0) = 1 +Size(4, 0) = 1 +Size(3, 0) = 1 +Size(2, 0) = 1 +Size(1, 0) = 1 +Size(0, 0) = 0 # Add a couple nonzero values. Size calls that include the new row count should # increase, but the size calls that don't include the new row count should not. @@ -386,52 +240,20 @@ write size rows=(8, 7, 6, 5, 4, 3, 2, 1, 0) ---- -b8: - 8: *colblk.UintBuilder[uint8].Size(8, 0) = 9 - 8: *colblk.UintBuilder[uint8].Size(7, 0) = 8 - 8: *colblk.UintBuilder[uint8].Size(6, 0) = 7 - 8: *colblk.UintBuilder[uint8].Size(5, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(4, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(3, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(2, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(1, 0) = 2 - 8: *colblk.UintBuilder[uint8].Size(0, 0) = 0 -b16: - 16: *colblk.UintBuilder[uint16].Size(8, 0) = 11 - 16: *colblk.UintBuilder[uint16].Size(7, 0) = 10 - 16: *colblk.UintBuilder[uint16].Size(6, 0) = 9 - 16: *colblk.UintBuilder[uint16].Size(5, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(4, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(3, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(2, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(1, 0) = 3 - 16: *colblk.UintBuilder[uint16].Size(0, 0) = 0 -b32: - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 - 32: *colblk.UintBuilder[uint32].Size(7, 0) = 12 - 32: *colblk.UintBuilder[uint32].Size(6, 0) = 11 - 32: *colblk.UintBuilder[uint32].Size(5, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(4, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(3, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(2, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(1, 0) = 5 - 32: *colblk.UintBuilder[uint32].Size(0, 0) = 0 -b64: - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 - 64: *colblk.UintBuilder[uint64].Size(7, 0) = 16 - 64: *colblk.UintBuilder[uint64].Size(6, 0) = 15 - 64: *colblk.UintBuilder[uint64].Size(5, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(4, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(3, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(2, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(1, 0) = 9 - 64: *colblk.UintBuilder[uint64].Size(0, 0) = 0 +Size(8, 0) = 9 +Size(7, 0) = 8 +Size(6, 0) = 7 +Size(5, 0) = 1 +Size(4, 0) = 1 +Size(3, 0) = 1 +Size(2, 0) = 1 +Size(1, 0) = 1 +Size(0, 0) = 0 # Finish the b8 so we can test 16-bit encoding. -finish widths=(8) rows=8 +finish rows=8 ---- -b8: *colblk.UintBuilder[uint8]: -0-1: x 00 # delta encoding: none +0-1: x 01 # encoding: 1b 1-2: x 00 # data[0] = 0 2-3: x 00 # data[1] = 0 3-4: x 00 # data[2] = 0 @@ -440,9 +262,6 @@ b8: *colblk.UintBuilder[uint8]: 6-7: x 0a # data[5] = 10 7-8: x 00 # data[6] = 0 8-9: x 0a # data[7] = 10 -Keeping b16 open -Keeping b32 open -Keeping b64 open # Add 1000 which should force a 16-bit delta encoding. @@ -452,15 +271,8 @@ write size rows=(9, 8) ---- -b16: - 16: *colblk.UintBuilder[uint16].Size(9, 0) = 20 - 16: *colblk.UintBuilder[uint16].Size(8, 0) = 11 -b32: - 32: *colblk.UintBuilder[uint32].Size(9, 0) = 24 - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 -b64: - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 +Size(9, 0) = 20 +Size(8, 0) = 9 # We should be able to write up to 2^16-1 without triggering a 32-bit encoding. @@ -470,28 +282,14 @@ write size rows=(10, 9, 8) ---- -b16: - 16: *colblk.UintBuilder[uint16].Size(10, 0) = 22 - 16: *colblk.UintBuilder[uint16].Size(9, 0) = 20 - 16: *colblk.UintBuilder[uint16].Size(8, 0) = 11 -b32: - 32: *colblk.UintBuilder[uint32].Size(10, 0) = 26 - 32: *colblk.UintBuilder[uint32].Size(9, 0) = 24 - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 -b64: - 64: *colblk.UintBuilder[uint64].Size(10, 0) = 30 - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 - -# But 2^16 should trigger a 32-bit encoding. (Finish b16 so we can test 32-bit -# encoding.) - -finish widths=(16) rows=10 ----- -b16: *colblk.UintBuilder[uint16]: -00-01: x 00 # delta encoding: none -# padding -01-02: x 00 # aligning to 16-bit boundary +Size(10, 0) = 22 +Size(9, 0) = 20 +Size(8, 0) = 9 + +finish rows=10 +---- +00-01: x 02 # encoding: 2b +01-02: x 00 # padding (aligning to 16-bit boundary) 02-04: x 0000 # data[0] = 0 04-06: x 0000 # data[1] = 0 06-08: x 0000 # data[2] = 0 @@ -502,8 +300,8 @@ b16: *colblk.UintBuilder[uint16]: 16-18: x 0a00 # data[7] = 10 18-20: x e803 # data[8] = 1000 20-22: x ffff # data[9] = 65535 -Keeping b32 open -Keeping b64 open + +# 2^16 should trigger a 32-bit encoding. write 10:65536 @@ -511,16 +309,10 @@ write size rows=(11, 10, 9, 8) ---- -b32: - 32: *colblk.UintBuilder[uint32].Size(11, 0) = 48 - 32: *colblk.UintBuilder[uint32].Size(10, 0) = 26 - 32: *colblk.UintBuilder[uint32].Size(9, 0) = 24 - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 -b64: - 64: *colblk.UintBuilder[uint64].Size(11, 0) = 56 - 64: *colblk.UintBuilder[uint64].Size(10, 0) = 30 - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 +Size(11, 0) = 48 +Size(10, 0) = 22 +Size(9, 0) = 20 +Size(8, 0) = 9 # We should be able to write up to 2^32-1 without triggering a 64-bit encoding. @@ -530,27 +322,16 @@ write size rows=(12, 11, 10, 9, 8) ---- -b32: - 32: *colblk.UintBuilder[uint32].Size(12, 0) = 52 - 32: *colblk.UintBuilder[uint32].Size(11, 0) = 48 - 32: *colblk.UintBuilder[uint32].Size(10, 0) = 26 - 32: *colblk.UintBuilder[uint32].Size(9, 0) = 24 - 32: *colblk.UintBuilder[uint32].Size(8, 0) = 13 -b64: - 64: *colblk.UintBuilder[uint64].Size(12, 0) = 60 - 64: *colblk.UintBuilder[uint64].Size(11, 0) = 56 - 64: *colblk.UintBuilder[uint64].Size(10, 0) = 30 - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 - -# But 2^32 should trigger a 64-bit encoding. - -finish widths=(32) rows=12 ----- -b32: *colblk.UintBuilder[uint32]: -00-01: x 00 # delta encoding: none -# padding -01-04: x 000000 # aligning to 32-bit boundary +Size(12, 0) = 52 +Size(11, 0) = 48 +Size(10, 0) = 22 +Size(9, 0) = 20 +Size(8, 0) = 9 + +finish rows=12 +---- +00-01: x 04 # encoding: 4b +01-04: x 000000 # padding (aligning to 32-bit boundary) 04-08: x 00000000 # data[0] = 0 08-12: x 00000000 # data[1] = 0 12-16: x 00000000 # data[2] = 0 @@ -563,7 +344,8 @@ b32: *colblk.UintBuilder[uint32]: 40-44: x ffff0000 # data[9] = 65535 44-48: x 00000100 # data[10] = 65536 48-52: x ffffffff # data[11] = 4294967295 -Keeping b64 open + +# 2^32 should trigger a 64-bit encoding. write 12:4294967296 @@ -571,20 +353,17 @@ write size rows=(13, 12, 11, 10, 9, 8) ---- -b64: - 64: *colblk.UintBuilder[uint64].Size(13, 0) = 112 - 64: *colblk.UintBuilder[uint64].Size(12, 0) = 60 - 64: *colblk.UintBuilder[uint64].Size(11, 0) = 56 - 64: *colblk.UintBuilder[uint64].Size(10, 0) = 30 - 64: *colblk.UintBuilder[uint64].Size(9, 0) = 28 - 64: *colblk.UintBuilder[uint64].Size(8, 0) = 17 +Size(13, 0) = 112 +Size(12, 0) = 52 +Size(11, 0) = 48 +Size(10, 0) = 22 +Size(9, 0) = 20 +Size(8, 0) = 9 -finish widths=(64) rows=13 +finish rows=13 ---- -b64: *colblk.UintBuilder[uint64]: -000-001: x 00 # delta encoding: none -# padding -001-008: x 00000000000000 # aligning to 64-bit boundary +000-001: x 08 # encoding: 8b +001-008: x 00000000000000 # padding (aligning to 64-bit boundary) 008-016: x 0000000000000000 # data[0] = 0 016-024: x 0000000000000000 # data[1] = 0 024-032: x 0000000000000000 # data[2] = 0 @@ -601,12 +380,8 @@ b64: *colblk.UintBuilder[uint64]: # Test serializing a few columns using delta encoding. -init widths=(8, 16, 32, 64) default-zero +init default-zero ---- -b8 -b16 -b32 -b64 write 0:1 2:92 3:1 7:86 20:221 @@ -614,56 +389,23 @@ write size rows=5 ---- -b8: - 8: *colblk.UintBuilder[uint8].Size(5, 0) = 6 -b16: - 16: *colblk.UintBuilder[uint16].Size(5, 0) = 8 -b32: - 32: *colblk.UintBuilder[uint32].Size(5, 0) = 10 -b64: - 64: *colblk.UintBuilder[uint64].Size(5, 0) = 14 +Size(5, 0) = 6 -finish widths=(8,16,32,64) rows=5 +finish rows=5 ---- -b8: *colblk.UintBuilder[uint8]: -0-1: x 00 # delta encoding: none +0-1: x 01 # encoding: 1b 1-2: x 01 # data[0] = 1 2-3: x 00 # data[1] = 0 3-4: x 5c # data[2] = 92 4-5: x 01 # data[3] = 1 5-6: x 00 # data[4] = 0 -b16: *colblk.UintBuilder[uint16]: -0-1: x 02 # delta encoding: delta8 -1-3: x 0000 # 16-bit constant: 0 -3-4: x 01 # data[0] = 1 -4-5: x 00 # data[1] = 0 -5-6: x 5c # data[2] = 92 -6-7: x 01 # data[3] = 1 -7-8: x 00 # data[4] = 0 -b32: *colblk.UintBuilder[uint32]: -0-1: x 02 # delta encoding: delta8 -1-5: x 00000000 # 32-bit constant: 0 -5-6: x 01 # data[0] = 1 -6-7: x 00 # data[1] = 0 -7-8: x 5c # data[2] = 92 -8-9: x 01 # data[3] = 1 -9-10: x 00 # data[4] = 0 -b64: *colblk.UintBuilder[uint64]: -00-01: x 02 # delta encoding: delta8 -01-09: x 0000000000000000 # 64-bit constant: 0 -09-10: x 01 # data[0] = 1 -10-11: x 00 # data[1] = 0 -11-12: x 5c # data[2] = 92 -12-13: x 01 # data[3] = 1 -13-14: x 00 # data[4] = 0 # Test a situation where the most recently written value requirs a wider delta # encoding, but we Finish with few enough rows that we should serialize using # the smaller encoding. -init widths=(64) +init ---- -b64 write 0:0 1:29 2:595 3:2 4:2 5:9 @@ -671,8 +413,7 @@ write size rows=(6) ---- -b64: - 64: *colblk.UintBuilder[uint64].Size(6, 0) = 22 +Size(6, 0) = 14 write 6:70395 @@ -680,28 +421,35 @@ write size rows=(7) ---- -b64: - 64: *colblk.UintBuilder[uint64].Size(7, 0) = 40 +Size(7, 0) = 32 -finish widths=(64) rows=6 +finish rows=6 ---- -b64: *colblk.UintBuilder[uint64]: -00-01: x 03 # delta encoding: delta16 -01-09: x 0000000000000000 # 64-bit constant: 0 -# padding -09-10: x 00 # aligning to 16-bit boundary -10-12: x 0000 # data[0] = 0 -12-14: x 1d00 # data[1] = 29 -14-16: x 5302 # data[2] = 595 -16-18: x 0200 # data[3] = 2 -18-20: x 0200 # data[4] = 2 -20-22: x 0900 # data[5] = 9 +00-01: x 02 # encoding: 2b +01-02: x 00 # padding (aligning to 16-bit boundary) +02-04: x 0000 # data[0] = 0 +04-06: x 1d00 # data[1] = 29 +06-08: x 5302 # data[2] = 595 +08-10: x 0200 # data[3] = 2 +10-12: x 0200 # data[4] = 2 +12-14: x 0900 # data[5] = 9 + +finish rows=7 +---- +00-01: x 04 # encoding: 4b +01-04: x 000000 # padding (aligning to 32-bit boundary) +04-08: x 00000000 # data[0] = 0 +08-12: x 1d000000 # data[1] = 29 +12-16: x 53020000 # data[2] = 595 +16-20: x 02000000 # data[3] = 2 +20-24: x 02000000 # data[4] = 2 +24-28: x 09000000 # data[5] = 9 +28-32: x fb120100 # data[6] = 70395 -# Test the constant encoded. +# Test the constant encoding. -init widths=(64) +init ---- -b64 write 0:1 1:1 2:1 3:1 4:1 5:1 @@ -709,38 +457,32 @@ write size rows=(6) ---- -b64: - 64: *colblk.UintBuilder[uint64].Size(6, 0) = 9 +Size(6, 0) = 9 -finish widths=(64) rows=6 +finish rows=6 ---- -b64: *colblk.UintBuilder[uint64]: -0-1: x 01 # delta encoding: const +0-1: x 80 # encoding: const 1-9: x 0100000000000000 # 64-bit constant: 1 # Test 32-bit delta encoding. -init widths=(64) +init ---- -b64 write -0:1 1:63936 2:2957252 +0:1 1:63936 2:4294967296 ---- size rows=(3) offset=1 ---- -b64: - 64: *colblk.UintBuilder[uint64].Size(3, 1) = 24 [23 w/o offset] +Size(3, 1) = 24 [23 w/o offset] -finish widths=(64) rows=3 offset=1 +finish rows=3 offset=1 ---- -b64: *colblk.UintBuilder[uint64]: 00-01: x 00 # artificial start offset -01-02: x 04 # delta encoding: delta32 +01-02: x 84 # encoding: 4b,delta 02-10: x 0100000000000000 # 64-bit constant: 1 -# padding -10-12: x 0000 # aligning to 32-bit boundary +10-12: x 0000 # padding (aligning to 32-bit boundary) 12-16: x 00000000 # data[0] = 0 + 1 = 1 16-20: x bff90000 # data[1] = 63935 + 1 = 63936 -20-24: x c31f2d00 # data[2] = 2957251 + 1 = 2957252 +20-24: x ffffffff # data[2] = 4294967295 + 1 = 4294967296 diff --git a/sstable/colblk/uints.go b/sstable/colblk/uints.go index 54d250afbb..c68911e7d4 100644 --- a/sstable/colblk/uints.go +++ b/sstable/colblk/uints.go @@ -8,6 +8,8 @@ import ( "encoding/binary" "fmt" "io" + "math" + "math/bits" "unsafe" "github.com/cockroachdb/pebble/internal/binfmt" @@ -21,71 +23,94 @@ type Uint interface { ~uint8 | ~uint16 | ~uint32 | ~uint64 } -// UintDeltaEncoding indicates what delta encoding, if any is in use by a -// uint{8,16,32,64} column to reduce the per-row storage size. +// UintEncoding indicates how unsigned integers (of at most 64 bits) are +// encoded. It has two components: +// - the low bits indicate how many bytes per integer are used, with +// allowed values 0, 1, 2, 4, or 8. +// - whether we are using a delta encoding, meaning that a base (64-bit) value +// is encoded separately and each encoded value is a delta from that base. +// Delta encoding is never necessary when we use 8 bytes per integer. // -// A uint delta encoding represents every non-NULL element in an array of uints -// as a delta relative to the column's constant. The logical value of each row -// is computed as C + D[i] where C is the column constant and D[i] is the delta. +// Note that 0-byte encodings imply that all values are equal (either to the +// base value if we are using a delta encoding, otherwise to 0). // -// The UintDeltaEncoding byte is serialized to the uint column before the column +// The UintEncoding byte is serialized to the uint column before the column // data. -type UintDeltaEncoding uint8 - -const ( - // UintDeltaEncodingNone indicates no delta encoding is in use. N rows are - // represented using N values of the column's logical data type. - UintDeltaEncodingNone UintDeltaEncoding = 0 - // UintDeltaEncodingConstant indicates that all rows of the column share the - // same value. The column data encodes the constant value and no deltas. - UintDeltaEncodingConstant UintDeltaEncoding = 1 - // UintDeltaEncoding8 indicates each delta is represented as a 1-byte uint8. - UintDeltaEncoding8 UintDeltaEncoding = 2 - // UintDeltaEncoding16 indicates each delta is represented as a 2-byte uint16. - UintDeltaEncoding16 UintDeltaEncoding = 3 - // UintDeltaEncoding32 indicates each delta is represented as a 4-byte uint32. - UintDeltaEncoding32 UintDeltaEncoding = 4 -) +type UintEncoding uint8 -// String implements fmt.Stringer. -func (d UintDeltaEncoding) String() string { - switch d { - case UintDeltaEncodingNone: - return "none" - case UintDeltaEncodingConstant: - return "const" - case UintDeltaEncoding8: - return "delta8" - case UintDeltaEncoding16: - return "delta16" - case UintDeltaEncoding32: - return "delta32" +const uintEncodingDeltaBit UintEncoding = 1 << 7 +const uintEncodingAllZero UintEncoding = 0 + +// IsDelta returns true if it is a delta encoding. +func (e UintEncoding) IsDelta() bool { + return e&uintEncodingDeltaBit != 0 +} + +// Width returns the number of bytes used per integer. It can be 0, 1, 2, 4, or 8. +func (e UintEncoding) Width() int { + return int(e &^ uintEncodingDeltaBit) +} + +// IsValid returns true if the encoding is valid. +func (e UintEncoding) IsValid() bool { + switch e.Width() { + case 0, 1, 2, 4: + return true + case 8: + // We should never need to do delta encoding if we store all 64 bits. + return !e.IsDelta() default: - panic("unreachable") + return false } } -func (d UintDeltaEncoding) width() int { - switch d { - case UintDeltaEncodingConstant: - return 0 - case UintDeltaEncoding8: - return 1 - case UintDeltaEncoding16: - return 2 - case UintDeltaEncoding32: - return 4 - default: - panic("unreachable") +// String implements fmt.Stringer. +func (e UintEncoding) String() string { + if e.Width() == 0 { + if e.IsDelta() { + return "const" + } + return "zero" + } + deltaString := "" + if e.IsDelta() { + deltaString = ",delta" + } + return fmt.Sprintf("%db%s", e.Width(), deltaString) +} + +// DetermineUintEncoding returns the best valid encoding that can be used to +// represent integers in the range [minValue, maxValue]. +func DetermineUintEncoding(minValue, maxValue uint64) UintEncoding { + // Find the number of bytes-per-value necessary for a delta encoding. + b := (bits.Len64(maxValue-minValue) + 7) >> 3 + // Round up to the nearest allowed value (0, 1, 2, 4, or 8). + if b > 4 { + return UintEncoding(8) + } + if b == 3 { + b = 4 + } + // Check if we can use the same number of bytes without a delta encoding. + isDelta := maxValue >= (1 << (b << 3)) + return makeUintEncoding(b, isDelta) +} + +func makeUintEncoding(width int, isDelta bool) UintEncoding { + e := UintEncoding(width) + if isDelta { + e |= uintEncodingDeltaBit } + if invariants.Enabled && !e.IsValid() { + panic(e) + } + return e } -// UintBuilder builds a column of unsigned integers of the same width. -// UintBuilder uses a delta encoding when possible to store values using -// lower-width integers. See DeltaEncoding. -type UintBuilder[T Uint] struct { +// UintBuilder builds a column of unsigned integers. It uses the smallest +// possible UintEncoding, depending on the values. +type UintBuilder struct { // configuration fixed on Init; preserved across Reset - dt DataType useDefault bool // array holds the underlying heap-allocated array in which values are @@ -96,17 +121,16 @@ type UintBuilder[T Uint] struct { n int // elems provides access to elements without bounds checking. elems is // grown automatically in Set. - elems UnsafeRawSlice[T] + elems UnsafeRawSlice[uint64] } - // delta holds state for the purpose of tracking which DeltaEncoding would + // stats holds state for the purpose of tracking which UintEncoding would // be used if the caller Finished the column including all elements Set so - // far. The delta state is used by Size (and Finish) to cheaply determine + // far. The stats state is used by Size (and Finish) to cheaply determine // which encoding may most concisely encode the array. // // Every Set(i, v) call updates minimum and maximum if necessary. If a call - // updates minimum, maximum or both, it sets the width to the number of - // bytes necessary to represent the new difference between maximum and - // minimum. It also sets widthRow=i, indicating which row last updated the + // updates minimum, maximum or both, it recalculates the encoding and if it + // changed sets sets encodingRow=i, indicating which row last updated the // width. // // Any call to Size or Finish that supplies [rows] that's inclusive of the @@ -122,16 +146,16 @@ type UintBuilder[T Uint] struct { // cases. Or, if we limit the API to only allow Finish to be called with one // less than the last set row, we could maintain the width of only the last // two rows. - delta struct { - minimum T - maximum T - width uint32 // 0, 1, 2, 4, or 8 - widthRow int // index of last update to width + stats struct { + minimum uint64 + maximum uint64 + encoding UintEncoding + encodingRow int // index of last update to encoding } } // Init initializes the UintBuilder. -func (b *UintBuilder[T]) Init() { +func (b *UintBuilder) Init() { b.init(false) } @@ -142,95 +166,79 @@ func (b *UintBuilder[T]) Init() { // // InitWithDefault may be preferrable when a nonzero value is uncommon, and the // caller can avoid explicitly Set-ing every zero value. -func (b *UintBuilder[T]) InitWithDefault() { +func (b *UintBuilder) InitWithDefault() { b.init(true) } -func (b *UintBuilder[T]) init(useDefault bool) { +func (b *UintBuilder) init(useDefault bool) { b.useDefault = useDefault - switch unsafe.Sizeof(T(0)) { - case 1: - b.dt = DataTypeUint8 - case 2: - b.dt = DataTypeUint16 - case 4: - b.dt = DataTypeUint32 - case 8: - b.dt = DataTypeUint64 - default: - panic("unreachable") - } b.Reset() } // NumColumns implements ColumnWriter. -func (b *UintBuilder[T]) NumColumns() int { return 1 } +func (b *UintBuilder) NumColumns() int { return 1 } // DataType implements ColumnWriter. -func (b *UintBuilder[T]) DataType(int) DataType { return b.dt } +func (b *UintBuilder) DataType(int) DataType { return DataTypeUint } // Reset implements ColumnWriter and resets the builder, reusing existing // allocated memory. -func (b *UintBuilder[T]) Reset() { +func (b *UintBuilder) Reset() { if b.useDefault { // If the caller configured a default zero, we assume that the array // will include at least one default value. - b.delta.minimum = 0 - b.delta.maximum = 0 + b.stats.minimum = 0 + b.stats.maximum = 0 clear(b.array.elems.Slice(b.array.n)) } else { - // Initialize the minimum to the max value that a T can represent. We - // subtract from zero, relying on the fact that T is unsigned and will - // wrap around to the maximal value. - b.delta.minimum = T(0) - 1 - b.delta.maximum = 0 + b.stats.minimum = math.MaxUint64 + b.stats.maximum = 0 // We could reset all values as a precaution, but it has a visible cost // in benchmarks. if invariants.Sometimes(50) { for i := 0; i < b.array.n; i++ { - b.array.elems.set(i, T(0)-1) + b.array.elems.set(i, math.MaxUint64) } } } - b.delta.widthRow = 0 - b.delta.width = 0 + b.stats.encoding = uintEncodingAllZero + b.stats.encodingRow = 0 } // Get gets the value of the provided row index. The provided row must have been // Set. -func (b *UintBuilder[T]) Get(row int) T { +func (b *UintBuilder) Get(row int) uint64 { return b.array.elems.At(row) } // Set sets the value of the provided row index to v. -func (b *UintBuilder[T]) Set(row int, v T) { +func (b *UintBuilder) Set(row int, v uint64) { if b.array.n <= row { - // Double the size of the allocated array, or initialize it to at least - // 256 bytes if this is the first allocation. Then double until there's - // sufficient space for n bytes. - n2 := max(b.array.n<<1, 256/int(unsafe.Sizeof(T(0)))) + // Double the size of the allocated array, or initialize it to at least 32 + // values (256 bytes) if this is the first allocation. Then double until + // there's sufficient space. + n2 := max(b.array.n<<1, 32) for n2 <= row { n2 <<= 1 /* double the size */ } - // NB: Go guarantees the allocated array will be T-aligned. - newDataTyped := make([]T, n2) + // NB: Go guarantees the allocated array will be 64-bit aligned. + newDataTyped := make([]uint64, n2) copy(newDataTyped, b.array.elems.Slice(b.array.n)) - newElems := makeUnsafeRawSlice[T](unsafe.Pointer(&newDataTyped[0])) + newElems := makeUnsafeRawSlice[uint64](unsafe.Pointer(&newDataTyped[0])) b.array.n = n2 b.array.elems = newElems } // Maintain the running minimum and maximum for the purpose of maintaining // knowledge of the delta encoding that would be used. - if b.delta.minimum > v || b.delta.maximum < v { - b.delta.minimum = min(v, b.delta.minimum) - b.delta.maximum = max(v, b.delta.maximum) - // If updating the minimum and maximum means that we now much use a - // wider width integer, update the width and the index of the update to - // it. - if w := deltaWidth(uint64(b.delta.maximum - b.delta.minimum)); w != b.delta.width { - b.delta.width = w - b.delta.widthRow = row + if b.stats.minimum > v || b.stats.maximum < v { + b.stats.minimum = min(v, b.stats.minimum) + b.stats.maximum = max(v, b.stats.maximum) + // If updating the minimum and maximum means that we now much use a wider + // width integer, update the encoding and the index of the update to it. + if e := DetermineUintEncoding(b.stats.minimum, b.stats.maximum); e != b.stats.encoding { + b.stats.encoding = e + b.stats.encodingRow = row } } b.array.elems.set(row, v) @@ -238,36 +246,40 @@ func (b *UintBuilder[T]) Set(row int, v T) { // Size implements ColumnWriter and returns the size of the column if its first // [rows] rows were serialized, serializing the column into offset [offset]. -func (b *UintBuilder[T]) Size(rows int, offset uint32) uint32 { +func (b *UintBuilder) Size(rows int, offset uint32) uint32 { if rows == 0 { return 0 } - // Determine the width of each element with delta-encoding applied. - // b.delta.width is the precomputed width for all rows. It's the best - // encoding we can use as long as b.delta.widthRow is included. If - // b.delta.widthRow is not included (b.delta.widthRow > rows-1), we need to - // scan the [rows] elements of the array to recalculate the appropriate - // delta. - w := b.delta.width - if b.delta.widthRow > rows-1 { - minimum, maximum := computeMinMax(b.array.elems.Slice(rows)) - w = deltaWidth(uint64(maximum - minimum)) + e, _ := b.determineEncoding(rows) + return uintColumnSize(uint32(rows), offset, e) +} + +// determineEncoding determines the best encoding for a column containing the +// first [rows], along with the minimum value (used as the "base" value when we +// use a stats encoding). +func (b *UintBuilder) determineEncoding(rows int) (_ UintEncoding, minimum uint64) { + if b.stats.encodingRow < rows { + // b.delta.encoding became the current value within the first [rows], so we + // can use it. + return b.stats.encoding, b.stats.minimum } - return uintColumnSize[T](uint32(rows), offset, w) + + // We have to recalculate the minimum and maximum. + minimum, maximum := computeMinMax(b.array.elems.Slice(rows)) + return DetermineUintEncoding(minimum, maximum), minimum } -// uintColumnSize returns the size of a column of unsigned integers of type T, -// encoded at the provided offset using the provided width. If width < -// sizeof(T), then a delta encoding is assumed. -func uintColumnSize[T Uint](rows, offset, width uint32) uint32 { +// uintColumnSize returns the size of a column of unsigned integers, encoded at +// the provided offset using the provided width. If width < sizeof(T), then a +// delta encoding is assumed. +func uintColumnSize(rows, offset uint32, e UintEncoding) uint32 { offset++ // DeltaEncoding byte - logicalWidth := uint32(unsafe.Sizeof(T(0))) - if width != logicalWidth { + if e.IsDelta() { // A delta encoding will be used. We need to first account for the constant - // that encodes the minimum. This constant is the full width of the column's - // logical data type. - offset += logicalWidth + // that encodes the base value. + offset += 8 } + width := uint32(e.Width()) // Include alignment bytes necessary to align offset appropriately for // elements of the delta width. if width > 0 { @@ -279,24 +291,12 @@ func uintColumnSize[T Uint](rows, offset, width uint32) uint32 { // Finish implements ColumnWriter, serializing the column into offset [offset] of // [buf]. -func (b *UintBuilder[T]) Finish(col, rows int, offset uint32, buf []byte) uint32 { +func (b *UintBuilder) Finish(col, rows int, offset uint32, buf []byte) uint32 { if rows == 0 { return offset } - // Determine the width of each element with delta-encoding applied. - // b.delta.width is the precomputed width for all rows. It's the best - // encoding we can use as long as b.delta.widthRow is included. If - // b.delta.widthRow is not included (b.delta.widthRow > rows-1), we need to - // scan the [rows] elements of the array to recalculate the appropriate - // delta. - minimum := b.delta.minimum - w := b.delta.width - if b.delta.widthRow > rows-1 { - var maximum T - minimum, maximum = computeMinMax(b.array.elems.Slice(rows)) - w = deltaWidth(uint64(maximum - minimum)) - } + e, minimum := b.determineEncoding(rows) // NB: In some circumstances, it's possible for b.array.elems.ptr to be nil. // Specifically, if the builder is initialized using InitWithDefault and no @@ -304,108 +304,64 @@ func (b *UintBuilder[T]) Finish(col, rows int, offset uint32, buf []byte) uint32 // allocate b.array.elems.ptr). It's illegal to try to construct an unsafe // slice from a nil ptr with non-zero rows. Only attempt to construct the // values slice if there's actually a non-nil ptr. - var valuesSlice []T + var valuesSlice []uint64 if b.array.elems.ptr != nil { valuesSlice = b.array.elems.Slice(rows) } - return uintColumnFinish[T](minimum, valuesSlice, w, offset, buf) + return uintColumnFinish(minimum, valuesSlice, e, offset, buf) } -// uintColumnFinish finishes the column of unsigned integers of type T, encoding -// per-row deltas of size width if width < sizeof(T). -func uintColumnFinish[T Uint](minimum T, values []T, width, offset uint32, buf []byte) uint32 { - // Compare the computed delta width to see if we're able to use an array of - // lower-width deltas to encode the column. - if uintptr(width) < unsafe.Sizeof(T(0)) { - switch width { - case 0: - // All the column values are the same and we can elide any deltas at - // all. - buf[offset] = byte(UintDeltaEncodingConstant) - offset++ - offset += uint32(writeLittleEndianNonaligned(buf, offset, minimum)) - return offset - case 1: - buf[offset] = byte(UintDeltaEncoding8) - offset++ - offset += uint32(writeLittleEndianNonaligned(buf, offset, minimum)) - dest := makeUnsafeRawSlice[uint8](unsafe.Pointer(&buf[offset])) - reduceUints[T, uint8](minimum, values, dest.Slice(len(values))) - offset += uint32(len(values)) - return offset - case align16: - buf[offset] = byte(UintDeltaEncoding16) - offset++ - offset += uint32(writeLittleEndianNonaligned(buf, offset, minimum)) - // Align the offset appropriately for uint16s. - offset = alignWithZeroes(buf, offset, align16) - dest := makeUnsafeRawSlice[uint16](unsafe.Pointer(&buf[offset])) - reduceUints[T, uint16](minimum, values, dest.Slice(len(values))) - offset += uint32(len(values)) * align16 - return offset - case align32: - buf[offset] = byte(UintDeltaEncoding32) - offset++ - offset += uint32(writeLittleEndianNonaligned(buf, offset, minimum)) - // Align the offset appropriately for uint32s. - offset = alignWithZeroes(buf, offset, align32) - dest := makeUnsafeRawSlice[uint32](unsafe.Pointer(&buf[offset])) - reduceUints[T, uint32](minimum, values, dest.Slice(len(values))) - offset += uint32(len(values)) * align32 - return offset - default: - panic("unreachable") - } - } - buf[offset] = byte(UintDeltaEncodingNone) +// uintColumnFinish finishes the column of unsigned integers of type T, applying +// the given encoding. +func uintColumnFinish( + minimum uint64, values []uint64, e UintEncoding, offset uint32, buf []byte, +) uint32 { + buf[offset] = byte(e) offset++ - offset = align(offset, uint32(unsafe.Sizeof(T(0)))) - dest := makeUnsafeRawSlice[T](unsafe.Pointer(&buf[offset])).Slice(len(values)) - offset += uint32(copy(dest, values)) * uint32(unsafe.Sizeof(T(0))) - return offset -} -// writeLittleEndianNonaligned writes v to buf at the provided offset in -// little-endian, without assuming that the offset is aligned to the size of T. -func writeLittleEndianNonaligned[T Uint](buf []byte, offset uint32, v T) int { - sz := unsafe.Sizeof(v) - switch sz { - case 1: - buf[offset] = byte(v) - case 2: - binary.LittleEndian.PutUint16(buf[offset:], uint16(v)) - case 4: - binary.LittleEndian.PutUint32(buf[offset:], uint32(v)) - case 8: - binary.LittleEndian.PutUint64(buf[offset:], uint64(v)) - default: - panic("unreachable") + deltaBase := uint64(0) + if e.IsDelta() { + deltaBase = minimum + binary.LittleEndian.PutUint64(buf[offset:], minimum) + offset += 8 } - return int(sz) -} + width := uint32(e.Width()) + if width == 0 { + // All the column values are the same. + return offset + } + // Align the offset appropriately. + offset = alignWithZeroes(buf, offset, width) -// readLittleEndianNonaligned reads a value of type T from buf at the provided -// offset in little-endian, without assuming that the offset is aligned to the -// size of T. -func readLittleEndianNonaligned[T constraints.Integer](buf []byte, offset uint32) T { - sz := unsafe.Sizeof(T(0)) - switch sz { + switch e.Width() { case 1: - return T(buf[offset]) + dest := makeUnsafeRawSlice[uint8](unsafe.Pointer(&buf[offset])).Slice(len(values)) + reduceUints(deltaBase, values, dest) + case 2: - return T(binary.LittleEndian.Uint16(buf[offset:])) + dest := makeUnsafeRawSlice[uint16](unsafe.Pointer(&buf[offset])).Slice(len(values)) + reduceUints(deltaBase, values, dest) + case 4: - return T(binary.LittleEndian.Uint32(buf[offset:])) + dest := makeUnsafeRawSlice[uint32](unsafe.Pointer(&buf[offset])).Slice(len(values)) + reduceUints(deltaBase, values, dest) + case 8: - return T(binary.LittleEndian.Uint64(buf[offset:])) + if deltaBase != 0 { + panic("unreachable") + } + dest := makeUnsafeRawSlice[uint64](unsafe.Pointer(&buf[offset])).Slice(len(values)) + copy(dest, values) + default: panic("unreachable") } + return offset + uint32(len(values))*width } // WriteDebug implements Encoder. -func (b *UintBuilder[T]) WriteDebug(w io.Writer, rows int) { - fmt.Fprintf(w, "%s: %d rows", b.dt, rows) +func (b *UintBuilder) WriteDebug(w io.Writer, rows int) { + fmt.Fprintf(w, "%s: %d rows", DataTypeUint, rows) } // reduceUints reduces the bit-width of a slice of unsigned by subtracting a @@ -432,28 +388,8 @@ func computeMinMax[I constraints.Unsigned](values []I) (I, I) { return minimum, maximum } -// deltaWidth returns the width in bytes of the integer type that can represent -// the provided value. -func deltaWidth(delta uint64) uint32 { - // TODO(jackson): Consider making this generic; We could compare against - // unsafe.Sizeof(T(0)) to ensure that we don't overflow T and that the - // higher width cases get elided at compile time for the smaller width Ts. - switch { - case delta == 0: - return 0 - case delta < (1 << 8): - return 1 - case delta < (1 << 16): - return align16 - case delta < (1 << 32): - return align32 - default: - return align64 - } -} - func uintsToBinFormatter( - f *binfmt.Formatter, rows int, dataType DataType, uintFormatter func(el, base uint64) string, + f *binfmt.Formatter, rows int, uintFormatter func(el, base uint64) string, ) { if uintFormatter == nil { uintFormatter = func(v, base uint64) string { @@ -464,36 +400,27 @@ func uintsToBinFormatter( } } - deltaEncoding := UintDeltaEncoding(f.PeekUint(1)) // DeltaEncoding byte - f.HexBytesln(1, "delta encoding: %s", deltaEncoding) - - logicalWidth := dataType.uintWidth() + e := UintEncoding(f.PeekUint(1)) // UintEncoding byte + if !e.IsValid() { + panic(fmt.Sprintf("%d", e)) + } + f.HexBytesln(1, "encoding: %s", e) var base uint64 - elementWidth := int(logicalWidth) - if deltaEncoding != UintDeltaEncodingNone { - base = f.PeekUint(int(logicalWidth)) - f.HexBytesln(int(logicalWidth), "%d-bit constant: %d", logicalWidth*8, base) - - switch deltaEncoding { - case UintDeltaEncodingConstant: - // This is just a constant (that was already read/formatted). - return - case UintDeltaEncoding8: - elementWidth = 1 - case UintDeltaEncoding16: - elementWidth = align16 - case UintDeltaEncoding32: - elementWidth = align32 - default: - panic("unreachable") - } + if e.IsDelta() { + base = f.PeekUint(8) + f.HexBytesln(8, "64-bit constant: %d", base) } - if off := align(f.Offset(), int(elementWidth)); off != f.Offset() { - f.CommentLine("padding") - f.HexBytesln(off-f.Offset(), "aligning to %d-bit boundary", elementWidth*8) + width := e.Width() + if width == 0 { + // The column is zero or constant. + return + } + + if off := align(f.Offset(), width); off != f.Offset() { + f.HexBytesln(off-f.Offset(), "padding (aligning to %d-bit boundary)", width*8) } for i := 0; i < rows; i++ { - f.HexBytesln(elementWidth, "data[%d] = %s", i, uintFormatter(f.PeekUint(elementWidth), base)) + f.HexBytesln(width, "data[%d] = %s", i, uintFormatter(f.PeekUint(width), base)) } } diff --git a/sstable/colblk/uints_test.go b/sstable/colblk/uints_test.go index f6e6fc2218..8b35917525 100644 --- a/sstable/colblk/uints_test.go +++ b/sstable/colblk/uints_test.go @@ -16,43 +16,24 @@ import ( "github.com/cockroachdb/pebble/internal/binfmt" ) +func TestUintEncoding(t *testing.T) { + for _, r := range interestingIntRanges { + actual := DetermineUintEncoding(r.Min, r.Max) + if actual != r.ExpectedEncoding { + t.Errorf("%d/%d expected %s, but got %s", r.Min, r.Max, r.ExpectedEncoding, actual) + } + } +} + func TestUints(t *testing.T) { - var b8 UintBuilder[uint8] - var b16 UintBuilder[uint16] - var b32 UintBuilder[uint32] - var b64 UintBuilder[uint64] + var b UintBuilder - var out bytes.Buffer - var widths []int - var writers []ColumnWriter datadriven.RunTest(t, "testdata/uints", func(t *testing.T, td *datadriven.TestData) string { - out.Reset() switch td.Cmd { case "init": - widths = widths[:0] - writers = writers[:0] - td.ScanArgs(t, "widths", &widths) defaultZero := td.HasArg("default-zero") - for _, w := range widths { - switch w { - case 8: - b8.init(defaultZero) - writers = append(writers, &b8) - case 16: - b16.init(defaultZero) - writers = append(writers, &b16) - case 32: - b32.init(defaultZero) - writers = append(writers, &b32) - case 64: - b64.init(defaultZero) - writers = append(writers, &b64) - default: - panic(fmt.Sprintf("unknown width: %d", w)) - } - fmt.Fprintf(&out, "b%d\n", w) - } - return out.String() + b.init(defaultZero) + return "" case "write": for _, f := range strings.Fields(td.Input) { delim := strings.IndexByte(f, ':') @@ -60,77 +41,43 @@ func TestUints(t *testing.T) { if err != nil { return err.Error() } - for _, width := range widths { - v, err := strconv.ParseUint(f[delim+1:], 10, width) - if err != nil { - return err.Error() - } - switch width { - case 8: - b8.Set(i, uint8(v)) - case 16: - b16.Set(i, uint16(v)) - case 32: - b32.Set(i, uint32(v)) - case 64: - b64.Set(i, v) - default: - panic(fmt.Sprintf("unknown width: %d", width)) - } + v, err := strconv.ParseUint(f[delim+1:], 10, 64) + if err != nil { + return err.Error() } + b.Set(i, v) } - return out.String() + return "" case "size": var offset uint32 var rowCounts []int td.ScanArgs(t, "rows", &rowCounts) td.MaybeScanArgs(t, "offset", &offset) - for wIdx, w := range writers { - fmt.Fprintf(&out, "b%d:\n", widths[wIdx]) - for _, rows := range rowCounts { - sz := w.Size(rows, offset) - if offset > 0 { - fmt.Fprintf(&out, " %d: %T.Size(%d, %d) = %d [%d w/o offset]\n", widths[wIdx], w, rows, offset, sz, sz-offset) - } else { - fmt.Fprintf(&out, " %d: %T.Size(%d, %d) = %d\n", widths[wIdx], w, rows, offset, sz) - } + var out bytes.Buffer + for _, rows := range rowCounts { + sz := b.Size(rows, offset) + if offset > 0 { + fmt.Fprintf(&out, "Size(%d, %d) = %d [%d w/o offset]\n", rows, offset, sz, sz-offset) + } else { + fmt.Fprintf(&out, "Size(%d, %d) = %d\n", rows, offset, sz) } } return out.String() case "finish": var rows int var offset uint32 - var finishWidths []int td.ScanArgs(t, "rows", &rows) - td.ScanArgs(t, "widths", &finishWidths) td.MaybeScanArgs(t, "offset", &offset) - var newWriters []ColumnWriter - var newWidths []int - for wIdx, width := range widths { - var shouldFinish bool - for _, fw := range finishWidths { - shouldFinish = shouldFinish || width == fw - } - if shouldFinish { - sz := writers[wIdx].Size(rows, offset) - buf := aligned.ByteSlice(int(sz)) - _ = writers[wIdx].Finish(0, rows, offset, buf) - fmt.Fprintf(&out, "b%d: %T:\n", width, writers[wIdx]) - f := binfmt.New(buf).LineWidth(20) - if offset > 0 { - f.HexBytesln(int(offset), "artificial start offset") - } - uintsToBinFormatter(f, rows, writers[wIdx].DataType(0), nil) - fmt.Fprintf(&out, "%s", f.String()) - } else { - fmt.Fprintf(&out, "Keeping b%d open\n", width) - newWidths = append(newWidths, width) - newWriters = append(newWriters, writers[wIdx]) - } + + sz := b.Size(rows, offset) + buf := aligned.ByteSlice(int(sz)) + _ = b.Finish(0, rows, offset, buf) + f := binfmt.New(buf).LineWidth(20) + if offset > 0 { + f.HexBytesln(int(offset), "artificial start offset") } - writers = newWriters - widths = newWidths - return out.String() + uintsToBinFormatter(f, rows, nil) + return f.String() default: panic(fmt.Sprintf("unknown command: %s", td.Cmd)) } diff --git a/sstable/colblk/unsafe_slice.go b/sstable/colblk/unsafe_slice.go index a4a6f2f9ba..bc3b3966dd 100644 --- a/sstable/colblk/unsafe_slice.go +++ b/sstable/colblk/unsafe_slice.go @@ -5,6 +5,7 @@ package colblk import ( + "encoding/binary" "unsafe" "github.com/cockroachdb/errors" @@ -40,99 +41,111 @@ func (s UnsafeRawSlice[T]) set(i int, v T) { *(*T)(unsafe.Pointer(uintptr(s.ptr) + unsafe.Sizeof(T(0))*uintptr(i))) = v } -// UnsafeUint8s is an UnsafeIntegerSlice of uint8s, possibly using delta -// encoding internally. -type UnsafeUint8s = UnsafeIntegerSlice[uint8] - -// UnsafeUint16s is an UnsafeIntegerSlice of uint16s, possibly using delta -// encoding internally. -type UnsafeUint16s = UnsafeIntegerSlice[uint16] - -// UnsafeUint32s is an UnsafeIntegerSlice of uint32s, possibly using delta -// encoding internally. -type UnsafeUint32s = UnsafeIntegerSlice[uint32] - -// UnsafeUint64s is an UnsafeIntegerSlice of uint64s, possibly using delta -// encoding internally. -type UnsafeUint64s = UnsafeIntegerSlice[uint64] - -// UnsafeIntegerSlice exposes a read-only slice of integers from a column. If -// the column's values are delta-encoded, UnsafeIntegerSlice transparently -// applies deltas. +// UnsafeUints exposes a read-only view of integers from a column, transparently +// decoding data based on the UintEncoding. // -// See DeltaEncoding and UintBuilder. -type UnsafeIntegerSlice[T constraints.Integer] struct { - base T - deltaPtr unsafe.Pointer - deltaWidth uintptr +// See UintEncoding and UintBuilder. +type UnsafeUints struct { + base uint64 + ptr unsafe.Pointer + width uint8 } // Assert that UnsafeIntegerSlice implements Array. -var _ Array[uint8] = UnsafeIntegerSlice[uint8]{} +var _ Array[uint64] = UnsafeUints{} -// DecodeUnsafeIntegerSlice decodes the structure of a slice of uints from a +// DecodeUnsafeUints decodes the structure of a slice of uints from a // byte slice. -func DecodeUnsafeIntegerSlice[T constraints.Integer]( - b []byte, off uint32, rows int, -) (slice UnsafeIntegerSlice[T], endOffset uint32) { - delta := UintDeltaEncoding(b[off]) +func DecodeUnsafeUints(b []byte, off uint32, rows int) (_ UnsafeUints, endOffset uint32) { + encoding := UintEncoding(b[off]) + if !encoding.IsValid() { + panic(errors.AssertionFailedf("invalid encoding 0x%x", b)) + } off++ - switch delta { - case UintDeltaEncodingNone: - off = align(off, uint32(unsafe.Sizeof(T(0)))) - slice = makeUnsafeIntegerSlice[T](0, unsafe.Pointer(&b[off]), int(unsafe.Sizeof(T(0)))) - off += uint32(unsafe.Sizeof(T(0))) * uint32(rows) - case UintDeltaEncodingConstant: - base := readLittleEndianNonaligned[T](b, off) - off += uint32(unsafe.Sizeof(T(0))) - slice = makeUnsafeIntegerSlice[T](base, unsafe.Pointer(&b[off]), 0) - case UintDeltaEncoding8, UintDeltaEncoding16, UintDeltaEncoding32: - w := delta.width() - base := readLittleEndianNonaligned[T](b, off) - off += uint32(unsafe.Sizeof(T(0))) + var base uint64 + if encoding.IsDelta() { + base = binary.LittleEndian.Uint64(b[off:]) + off += 8 + } + w := encoding.Width() + if w > 0 { off = align(off, uint32(w)) - slice = makeUnsafeIntegerSlice[T](base, unsafe.Pointer(&b[off]), w) - off += uint32(rows) * uint32(w) - default: - panic("unreachable") } - return slice, off + return makeUnsafeUints(base, unsafe.Pointer(&b[off]), w), off + uint32(rows*w) } // Assert that DecodeUnsafeIntegerSlice implements DecodeFunc. -var _ DecodeFunc[UnsafeUint8s] = DecodeUnsafeIntegerSlice[uint8] - -func makeUnsafeIntegerSlice[T constraints.Integer]( - base T, deltaPtr unsafe.Pointer, deltaWidth int, -) UnsafeIntegerSlice[T] { - return UnsafeIntegerSlice[T]{ - base: base, - deltaPtr: deltaPtr, - deltaWidth: uintptr(deltaWidth), +var _ DecodeFunc[UnsafeUints] = DecodeUnsafeUints + +func makeUnsafeUints(base uint64, ptr unsafe.Pointer, width int) UnsafeUints { + switch width { + case 0, 1, 2, 4, 8: + default: + panic("invalid width") + } + return UnsafeUints{ + base: base, + ptr: ptr, + width: uint8(width), } } -// At returns the `i`-th element of the slice. -func (s UnsafeIntegerSlice[T]) At(i int) T { +// 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.deltaWidth { + switch s.width { case 0: return s.base case 1: - return s.base + T(*(*uint8)(unsafe.Pointer(uintptr(s.deltaPtr) + uintptr(i)))) + return s.base + uint64(*(*uint8)(unsafe.Pointer(uintptr(s.ptr) + uintptr(i)))) case 2: - return s.base + T(*(*uint16)(unsafe.Pointer(uintptr(s.deltaPtr) + uintptr(i)<