kevo/pkg/sstable/block/block.go
Jeremy Tregunna 68283a5fed
feat: implement merged iterator across all levels with improved tombstone handling
- Remove redundant MergedIterator (was just an alias for HierarchicalIterator)
- Add IsTombstone method to all iterators to detect deletion markers
- Enhance tombstone tracking in compaction manager with preservation options
- Fix SSTable reader to properly handle tombstone entries
- Update engine tests to directly verify tombstone behavior
- Update TODO.md to mark merged iterator task as complete
2025-04-19 22:18:12 -06:00

626 lines
15 KiB
Go

package block
import (
"bytes"
"encoding/binary"
"fmt"
"io"
"github.com/cespare/xxhash/v2"
)
const (
// BlockSize is the target size for each block
BlockSize = 16 * 1024 // 16KB
// RestartInterval defines how often we store a full key
RestartInterval = 16
// MaxBlockEntries is the maximum number of entries per block
MaxBlockEntries = 1024
// BlockFooterSize is the size of the footer (checksum + restart point count)
BlockFooterSize = 8 + 4 // 8 bytes for checksum, 4 for restart count
)
// Entry represents a key-value pair within the block
type Entry struct {
Key []byte
Value []byte
}
// Builder constructs a sorted, serialized block
type Builder struct {
entries []Entry
restartPoints []uint32
restartCount uint32
currentSize uint32
lastKey []byte
restartIdx int
}
// NewBuilder creates a new block builder
func NewBuilder() *Builder {
return &Builder{
entries: make([]Entry, 0, MaxBlockEntries),
restartPoints: make([]uint32, 0, MaxBlockEntries/RestartInterval+1),
restartCount: 0,
currentSize: 0,
}
}
// Add adds a key-value pair to the block
// Keys must be added in sorted order
func (b *Builder) Add(key, value []byte) error {
// Ensure keys are added in sorted order
if len(b.entries) > 0 && bytes.Compare(key, b.lastKey) <= 0 {
return fmt.Errorf("keys must be added in strictly increasing order, got %s after %s",
string(key), string(b.lastKey))
}
b.entries = append(b.entries, Entry{
Key: append([]byte(nil), key...), // Make copies to avoid references
Value: append([]byte(nil), value...), // to external data
})
// Add restart point if needed
if b.restartIdx == 0 || b.restartIdx >= RestartInterval {
b.restartPoints = append(b.restartPoints, b.currentSize)
b.restartIdx = 0
}
b.restartIdx++
// Track the size
b.currentSize += uint32(len(key) + len(value) + 8) // 8 bytes for metadata
b.lastKey = append([]byte(nil), key...)
return nil
}
// GetEntries returns the entries in the block
func (b *Builder) GetEntries() []Entry {
return b.entries
}
// Reset clears the builder state
func (b *Builder) Reset() {
b.entries = b.entries[:0]
b.restartPoints = b.restartPoints[:0]
b.restartCount = 0
b.currentSize = 0
b.lastKey = nil
b.restartIdx = 0
}
// EstimatedSize returns the approximate size of the block when serialized
func (b *Builder) EstimatedSize() uint32 {
if len(b.entries) == 0 {
return 0
}
// Data + restart points array + footer
return b.currentSize + uint32(len(b.restartPoints)*4) + BlockFooterSize
}
// Entries returns the number of entries in the block
func (b *Builder) Entries() int {
return len(b.entries)
}
// Finish serializes the block to a writer
func (b *Builder) Finish(w io.Writer) (uint64, error) {
if len(b.entries) == 0 {
return 0, fmt.Errorf("cannot finish empty block")
}
// Keys are already sorted by the Add method's requirement
// Remove any duplicate keys (keeping the last one)
if len(b.entries) > 1 {
uniqueEntries := make([]Entry, 0, len(b.entries))
for i := 0; i < len(b.entries); i++ {
// Skip if this is a duplicate of the previous entry
if i > 0 && bytes.Equal(b.entries[i].Key, b.entries[i-1].Key) {
// Replace the previous entry with this one (to keep the latest value)
uniqueEntries[len(uniqueEntries)-1] = b.entries[i]
} else {
uniqueEntries = append(uniqueEntries, b.entries[i])
}
}
b.entries = uniqueEntries
}
// Reset restart points
b.restartPoints = b.restartPoints[:0]
b.restartPoints = append(b.restartPoints, 0) // First entry is always a restart point
// Write all entries
content := make([]byte, 0, b.EstimatedSize())
buffer := bytes.NewBuffer(content)
var prevKey []byte
restartOffset := 0
for i, entry := range b.entries {
// Start a new restart point?
isRestart := i == 0 || restartOffset >= RestartInterval
if isRestart {
restartOffset = 0
if i > 0 {
b.restartPoints = append(b.restartPoints, uint32(buffer.Len()))
}
}
// Write entry
if isRestart {
// Full key for restart points
keyLen := uint16(len(entry.Key))
err := binary.Write(buffer, binary.LittleEndian, keyLen)
if err != nil {
return 0, fmt.Errorf("failed to write key length: %w", err)
}
n, err := buffer.Write(entry.Key)
if err != nil {
return 0, fmt.Errorf("failed to write key: %w", err)
}
if n != len(entry.Key) {
return 0, fmt.Errorf("wrote incomplete key: %d of %d bytes", n, len(entry.Key))
}
} else {
// For non-restart points, delta encode the key
commonPrefix := 0
for j := 0; j < len(prevKey) && j < len(entry.Key); j++ {
if prevKey[j] != entry.Key[j] {
break
}
commonPrefix++
}
// Format: [shared prefix length][unshared length][unshared bytes]
err := binary.Write(buffer, binary.LittleEndian, uint16(commonPrefix))
if err != nil {
return 0, fmt.Errorf("failed to write common prefix length: %w", err)
}
unsharedLen := uint16(len(entry.Key) - commonPrefix)
err = binary.Write(buffer, binary.LittleEndian, unsharedLen)
if err != nil {
return 0, fmt.Errorf("failed to write unshared length: %w", err)
}
n, err := buffer.Write(entry.Key[commonPrefix:])
if err != nil {
return 0, fmt.Errorf("failed to write unshared bytes: %w", err)
}
if n != int(unsharedLen) {
return 0, fmt.Errorf("wrote incomplete unshared bytes: %d of %d bytes", n, unsharedLen)
}
}
// Write value
valueLen := uint32(len(entry.Value))
err := binary.Write(buffer, binary.LittleEndian, valueLen)
if err != nil {
return 0, fmt.Errorf("failed to write value length: %w", err)
}
n, err := buffer.Write(entry.Value)
if err != nil {
return 0, fmt.Errorf("failed to write value: %w", err)
}
if n != len(entry.Value) {
return 0, fmt.Errorf("wrote incomplete value: %d of %d bytes", n, len(entry.Value))
}
prevKey = entry.Key
restartOffset++
}
// Write restart points
for _, point := range b.restartPoints {
binary.Write(buffer, binary.LittleEndian, point)
}
// Write number of restart points
binary.Write(buffer, binary.LittleEndian, uint32(len(b.restartPoints)))
// Calculate checksum
data := buffer.Bytes()
checksum := xxhash.Sum64(data)
// Write checksum
binary.Write(buffer, binary.LittleEndian, checksum)
// Write the entire buffer to the output writer
n, err := w.Write(buffer.Bytes())
if err != nil {
return 0, fmt.Errorf("failed to write block: %w", err)
}
if n != buffer.Len() {
return 0, fmt.Errorf("wrote incomplete block: %d of %d bytes", n, buffer.Len())
}
return checksum, nil
}
// Reader provides methods to read data from a serialized block
type Reader struct {
data []byte
restartPoints []uint32
numRestarts uint32
checksum uint64
}
// NewReader creates a new block reader
func NewReader(data []byte) (*Reader, error) {
if len(data) < BlockFooterSize {
return nil, fmt.Errorf("block data too small: %d bytes", len(data))
}
// Read footer
footerOffset := len(data) - BlockFooterSize
numRestarts := binary.LittleEndian.Uint32(data[footerOffset:footerOffset+4])
checksum := binary.LittleEndian.Uint64(data[footerOffset+4:])
// Verify checksum - the checksum covers everything except the checksum itself
computedChecksum := xxhash.Sum64(data[:len(data)-8])
if computedChecksum != checksum {
return nil, fmt.Errorf("block checksum mismatch: expected %d, got %d",
checksum, computedChecksum)
}
// Read restart points
restartOffset := footerOffset - int(numRestarts)*4
if restartOffset < 0 {
return nil, fmt.Errorf("invalid restart points offset")
}
restartPoints := make([]uint32, numRestarts)
for i := uint32(0); i < numRestarts; i++ {
restartPoints[i] = binary.LittleEndian.Uint32(
data[restartOffset+int(i)*4:])
}
reader := &Reader{
data: data,
restartPoints: restartPoints,
numRestarts: numRestarts,
checksum: checksum,
}
return reader, nil
}
// Iterator returns an iterator for the block
func (r *Reader) Iterator() *Iterator {
// Calculate the data end position (everything before the restart points array)
dataEnd := len(r.data) - BlockFooterSize - 4*len(r.restartPoints)
return &Iterator{
reader: r,
currentPos: 0,
currentKey: nil,
currentVal: nil,
restartIdx: 0,
initialized: false,
dataEnd: uint32(dataEnd),
}
}
// Iterator allows iterating through key-value pairs in a block
type Iterator struct {
reader *Reader
currentPos uint32
currentKey []byte
currentVal []byte
restartIdx int
initialized bool
dataEnd uint32 // Position where the actual entries data ends (before restart points)
}
// SeekToFirst positions the iterator at the first entry
func (it *Iterator) SeekToFirst() {
if len(it.reader.restartPoints) == 0 {
it.currentKey = nil
it.currentVal = nil
it.initialized = true
return
}
it.currentPos = 0
it.restartIdx = 0
it.initialized = true
key, val, ok := it.decodeCurrent()
if ok {
it.currentKey = key
it.currentVal = val
} else {
it.currentKey = nil
it.currentVal = nil
}
}
// SeekToLast positions the iterator at the last entry
func (it *Iterator) SeekToLast() {
if len(it.reader.restartPoints) == 0 {
it.currentKey = nil
it.currentVal = nil
it.initialized = true
return
}
// Start from the last restart point
it.restartIdx = len(it.reader.restartPoints) - 1
it.currentPos = it.reader.restartPoints[it.restartIdx]
it.initialized = true
// Skip forward to the last entry
key, val, ok := it.decodeCurrent()
if !ok {
it.currentKey = nil
it.currentVal = nil
return
}
it.currentKey = key
it.currentVal = val
// Continue moving forward as long as there are more entries
for {
lastPos := it.currentPos
lastKey := it.currentKey
lastVal := it.currentVal
key, val, ok = it.decodeNext()
if !ok {
// Restore position to the last valid entry
it.currentPos = lastPos
it.currentKey = lastKey
it.currentVal = lastVal
return
}
it.currentKey = key
it.currentVal = val
}
}
// Seek positions the iterator at the first key >= target
func (it *Iterator) Seek(target []byte) bool {
if len(it.reader.restartPoints) == 0 {
return false
}
// Binary search through restart points
left, right := 0, len(it.reader.restartPoints)-1
for left < right {
mid := (left + right) / 2
it.restartIdx = mid
it.currentPos = it.reader.restartPoints[mid]
key, _, ok := it.decodeCurrent()
if !ok {
return false
}
if bytes.Compare(key, target) < 0 {
left = mid + 1
} else {
right = mid
}
}
// Position at the found restart point
it.restartIdx = left
it.currentPos = it.reader.restartPoints[left]
it.initialized = true
// First check the current position
key, val, ok := it.decodeCurrent()
if !ok {
return false
}
// If the key at this position is already >= target, we're done
if bytes.Compare(key, target) >= 0 {
it.currentKey = key
it.currentVal = val
return true
}
// Otherwise, scan forward until we find the first key >= target
for {
savePos := it.currentPos
key, val, ok = it.decodeNext()
if !ok {
// Restore position to the last valid entry
it.currentPos = savePos
key, val, ok = it.decodeCurrent()
if ok {
it.currentKey = key
it.currentVal = val
return true
}
return false
}
if bytes.Compare(key, target) >= 0 {
it.currentKey = key
it.currentVal = val
return true
}
// Update current key/value for the next iteration
it.currentKey = key
it.currentVal = val
}
}
// Next advances the iterator to the next entry
func (it *Iterator) Next() bool {
if !it.initialized {
it.SeekToFirst()
return it.Valid()
}
if it.currentKey == nil {
return false
}
key, val, ok := it.decodeNext()
if !ok {
it.currentKey = nil
it.currentVal = nil
return false
}
it.currentKey = key
it.currentVal = val
return true
}
// Key returns the current key
func (it *Iterator) Key() []byte {
return it.currentKey
}
// Value returns the current value
func (it *Iterator) Value() []byte {
return it.currentVal
}
// Valid returns true if the iterator is positioned at a valid entry
func (it *Iterator) Valid() bool {
return it.currentKey != nil && len(it.currentKey) > 0
}
// IsTombstone returns true if the current entry is a deletion marker
func (it *Iterator) IsTombstone() bool {
// For block iterators, a nil value means it's a tombstone
return it.Valid() && it.currentVal == nil
}
// decodeCurrent decodes the entry at the current position
func (it *Iterator) decodeCurrent() ([]byte, []byte, bool) {
if it.currentPos >= it.dataEnd {
return nil, nil, false
}
data := it.reader.data[it.currentPos:]
// Read key
if len(data) < 2 {
return nil, nil, false
}
keyLen := binary.LittleEndian.Uint16(data)
data = data[2:]
if uint32(len(data)) < uint32(keyLen) {
return nil, nil, false
}
key := make([]byte, keyLen)
copy(key, data[:keyLen])
data = data[keyLen:]
// Read value
if len(data) < 4 {
return nil, nil, false
}
valueLen := binary.LittleEndian.Uint32(data)
data = data[4:]
if uint32(len(data)) < valueLen {
return nil, nil, false
}
value := make([]byte, valueLen)
copy(value, data[:valueLen])
it.currentKey = key
it.currentVal = value
return key, value, true
}
// decodeNext decodes the next entry
func (it *Iterator) decodeNext() ([]byte, []byte, bool) {
if it.currentPos >= it.dataEnd {
return nil, nil, false
}
data := it.reader.data[it.currentPos:]
var key []byte
// Check if we're at a restart point
isRestart := false
for i, offset := range it.reader.restartPoints {
if offset == it.currentPos {
isRestart = true
it.restartIdx = i
break
}
}
if isRestart || it.currentKey == nil {
// Full key at restart point
if len(data) < 2 {
return nil, nil, false
}
keyLen := binary.LittleEndian.Uint16(data)
data = data[2:]
if uint32(len(data)) < uint32(keyLen) {
return nil, nil, false
}
key = make([]byte, keyLen)
copy(key, data[:keyLen])
data = data[keyLen:]
it.currentPos += 2 + uint32(keyLen)
} else {
// Delta-encoded key
if len(data) < 4 {
return nil, nil, false
}
sharedLen := binary.LittleEndian.Uint16(data)
data = data[2:]
unsharedLen := binary.LittleEndian.Uint16(data)
data = data[2:]
if sharedLen > uint16(len(it.currentKey)) ||
uint32(len(data)) < uint32(unsharedLen) {
return nil, nil, false
}
// Reconstruct key: shared prefix + unshared suffix
key = make([]byte, sharedLen+unsharedLen)
copy(key[:sharedLen], it.currentKey[:sharedLen])
copy(key[sharedLen:], data[:unsharedLen])
data = data[unsharedLen:]
it.currentPos += 4 + uint32(unsharedLen)
}
// Read value
if len(data) < 4 {
return nil, nil, false
}
valueLen := binary.LittleEndian.Uint32(data)
data = data[4:]
if uint32(len(data)) < valueLen {
return nil, nil, false
}
value := make([]byte, valueLen)
copy(value, data[:valueLen])
it.currentPos += 4 + uint32(valueLen)
return key, value, true
}