- 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
626 lines
15 KiB
Go
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
|
|
}
|
|
|