fix: fixes issue with sstables not having sequence saved so older data can be retrieved after an update
Some checks failed
Go Tests / Run Tests (1.24.2) (push) Failing after 15m7s

This commit is contained in:
Jeremy Tregunna 2025-05-17 14:58:26 -06:00
parent acc6d7ee98
commit c2c97f67a8
Signed by: jer
GPG Key ID: 1278B36BA6F5D5E4
24 changed files with 523 additions and 211 deletions

View File

@ -86,7 +86,7 @@ func (s *Server) Start() error {
kaPolicy := keepalive.EnforcementPolicy{
MinTime: 10 * time.Second, // Minimum time a client should wait between pings
PermitWithoutStream: true, // Allow pings even when there are no active streams
PermitWithoutStream: true, // Allow pings even when there are no active streams
}
serverOpts = append(serverOpts,

View File

@ -119,13 +119,13 @@ func NewClient(options ClientOptions) (*Client, error) {
}
transportOpts := transport.TransportOptions{
Timeout: options.ConnectTimeout,
MaxMessageSize: options.MaxMessageSize,
Compression: options.Compression,
TLSEnabled: options.TLSEnabled,
CertFile: options.CertFile,
KeyFile: options.KeyFile,
CAFile: options.CAFile,
Timeout: options.ConnectTimeout,
MaxMessageSize: options.MaxMessageSize,
Compression: options.Compression,
TLSEnabled: options.TLSEnabled,
CertFile: options.CertFile,
KeyFile: options.KeyFile,
CAFile: options.CAFile,
KeepaliveParams: keepaliveParams,
RetryPolicy: transport.RetryPolicy{
MaxRetries: options.MaxRetries,
@ -245,13 +245,13 @@ func (c *Client) createTransportOptions(options ClientOptions) transport.Transpo
}
return transport.TransportOptions{
Timeout: options.ConnectTimeout,
MaxMessageSize: options.MaxMessageSize,
Compression: options.Compression,
TLSEnabled: options.TLSEnabled,
CertFile: options.CertFile,
KeyFile: options.KeyFile,
CAFile: options.CAFile,
Timeout: options.ConnectTimeout,
MaxMessageSize: options.MaxMessageSize,
Compression: options.Compression,
TLSEnabled: options.TLSEnabled,
CertFile: options.CertFile,
KeyFile: options.KeyFile,
CAFile: options.CAFile,
KeepaliveParams: keepaliveParams,
RetryPolicy: transport.RetryPolicy{
MaxRetries: options.MaxRetries,

View File

@ -58,12 +58,12 @@ type Config struct {
MaxLevelWithTombstones int `json:"max_level_with_tombstones"` // Levels higher than this discard tombstones
// Transaction configuration
ReadOnlyTxTTL int64 `json:"read_only_tx_ttl"` // Time-to-live for read-only transactions in seconds (default: 180s)
ReadWriteTxTTL int64 `json:"read_write_tx_ttl"` // Time-to-live for read-write transactions in seconds (default: 60s)
IdleTxTimeout int64 `json:"idle_tx_timeout"` // Time after which an inactive transaction is considered idle in seconds (default: 30s)
TxCleanupInterval int64 `json:"tx_cleanup_interval"` // Interval for checking transaction TTLs in seconds (default: 30s)
TxWarningThreshold int `json:"tx_warning_threshold"` // Percentage of TTL after which to log warnings (default: 75)
TxCriticalThreshold int `json:"tx_critical_threshold"` // Percentage of TTL after which to log critical warnings (default: 90)
ReadOnlyTxTTL int64 `json:"read_only_tx_ttl"` // Time-to-live for read-only transactions in seconds (default: 180s)
ReadWriteTxTTL int64 `json:"read_write_tx_ttl"` // Time-to-live for read-write transactions in seconds (default: 60s)
IdleTxTimeout int64 `json:"idle_tx_timeout"` // Time after which an inactive transaction is considered idle in seconds (default: 30s)
TxCleanupInterval int64 `json:"tx_cleanup_interval"` // Interval for checking transaction TTLs in seconds (default: 30s)
TxWarningThreshold int `json:"tx_warning_threshold"` // Percentage of TTL after which to log warnings (default: 75)
TxCriticalThreshold int `json:"tx_critical_threshold"` // Percentage of TTL after which to log critical warnings (default: 90)
mu sync.RWMutex
}

View File

@ -637,34 +637,61 @@ func (m *Manager) flushMemTable(mem *memtable.MemTable) error {
count := 0
var bytesWritten uint64
// Since memtable's skiplist returns keys in sorted order,
// but possibly with duplicates (newer versions of same key first),
// we need to track all processed keys (including tombstones)
processedKeys := make(map[string]struct{})
// Structure to store information about keys as we iterate
type keyEntry struct {
key []byte
value []byte
seqNum uint64
}
// Collect keys in sorted order while tracking the newest version of each key
var entries []keyEntry
var previousKey []byte
// First iterate through memtable (already in sorted order)
for iter.SeekToFirst(); iter.Valid(); iter.Next() {
key := iter.Key()
keyStr := string(key) // Use as map key
currentKey := iter.Key()
currentValue := iter.Value()
currentSeqNum := iter.SequenceNumber()
// Skip keys we've already processed (including tombstones)
if _, seen := processedKeys[keyStr]; seen {
// If this is a tombstone (deletion marker), we skip it
if currentValue == nil {
continue
}
// Mark this key as processed regardless of whether it's a value or tombstone
processedKeys[keyStr] = struct{}{}
// Only write non-tombstone entries to the SSTable
if value := iter.Value(); value != nil {
bytesWritten += uint64(len(key) + len(value))
if err := writer.Add(key, value); err != nil {
writer.Abort()
return fmt.Errorf("failed to add entry to SSTable: %w", err)
// If this is the first key or a different key than the previous one
if previousKey == nil || !bytes.Equal(currentKey, previousKey) {
// Add this as a new entry
entries = append(entries, keyEntry{
key: append([]byte(nil), currentKey...),
value: append([]byte(nil), currentValue...),
seqNum: currentSeqNum,
})
previousKey = currentKey
} else {
// Same key as previous, check sequence number
lastIndex := len(entries) - 1
if currentSeqNum > entries[lastIndex].seqNum {
// This is a newer version of the same key, replace the previous entry
entries[lastIndex] = keyEntry{
key: append([]byte(nil), currentKey...),
value: append([]byte(nil), currentValue...),
seqNum: currentSeqNum,
}
}
count++
}
}
// Now write all collected entries to the SSTable
for _, entry := range entries {
bytesWritten += uint64(len(entry.key) + len(entry.value))
if err := writer.AddWithSequence(entry.key, entry.value, entry.seqNum); err != nil {
writer.Abort()
return fmt.Errorf("failed to add entry with sequence number to SSTable: %w", err)
}
count++
}
if count == 0 {
writer.Abort()
return nil
@ -857,3 +884,20 @@ func (m *Manager) recoverFromWAL() error {
return nil
}
// RetryOnWALRotating retries operations with ErrWALRotating
func (m *Manager) RetryOnWALRotating(operation func() error) error {
maxRetries := 3
for attempts := 0; attempts < maxRetries; attempts++ {
err := operation()
if err != wal.ErrWALRotating {
// Either success or a different error
return err
}
// Wait a bit before retrying to allow the rotation to complete
time.Sleep(10 * time.Millisecond)
}
return fmt.Errorf("operation failed after %d retries: %w", maxRetries, wal.ErrWALRotating)
}

View File

@ -23,12 +23,11 @@ func DefaultRetryConfig() *RetryConfig {
}
}
// RetryOnWALRotating retries the operation if it fails with ErrWALRotating
func (m *Manager) RetryOnWALRotating(operation func() error) error {
config := DefaultRetryConfig()
return m.RetryWithConfig(operation, config, isWALRotating)
}
// Commented out due to duplicate declaration with the one in manager.go
// func (m *Manager) RetryOnWALRotating(operation func() error) error {
// config := DefaultRetryConfig()
// return m.RetryWithConfig(operation, config, isWALRotating)
// }
// RetryWithConfig retries an operation with the given configuration
func (m *Manager) RetryWithConfig(operation func() error, config *RetryConfig, isRetryable func(error) bool) error {

View File

@ -0,0 +1,143 @@
// ABOUTME: Tests sequence number functionality in storage manager
// ABOUTME: Verifies correct version selection during recovery with sequence numbers
package storage
import (
"bytes"
"os"
"path/filepath"
"testing"
"github.com/KevoDB/kevo/pkg/config"
"github.com/KevoDB/kevo/pkg/stats"
)
// TestSequenceNumberVersioning tests that sequence numbers are properly tracked
// and used for version selection during recovery.
func TestSequenceNumberVersioning(t *testing.T) {
// Create temporary directories for the test
tempDir, err := os.MkdirTemp("", "sequence-test-*")
if err != nil {
t.Fatalf("Failed to create temp directory: %v", err)
}
defer os.RemoveAll(tempDir)
// Create subdirectories for SSTables and WAL
sstDir := filepath.Join(tempDir, "sst")
walDir := filepath.Join(tempDir, "wal")
// Create configuration
cfg := &config.Config{
Version: config.CurrentManifestVersion,
SSTDir: sstDir,
WALDir: walDir,
MemTableSize: 1024 * 1024, // 1MB
MemTablePoolCap: 2,
MaxMemTables: 2,
}
// Create a stats collector
statsCollector := stats.NewAtomicCollector()
// Create a new storage manager
manager, err := NewManager(cfg, statsCollector)
if err != nil {
t.Fatalf("Failed to create storage manager: %v", err)
}
defer manager.Close()
// Step 1: Add a key with initial value
testKey := []byte("test-key")
initialValue := []byte("initial-value")
err = manager.Put(testKey, initialValue)
if err != nil {
t.Fatalf("Failed to put initial value: %v", err)
}
// Verify the key is readable
value, err := manager.Get(testKey)
if err != nil {
t.Fatalf("Failed to get value: %v", err)
}
if !bytes.Equal(initialValue, value) {
t.Errorf("Expected initial value %s, got %s", initialValue, value)
}
// Step 2: Flush to create an SSTable
err = manager.FlushMemTables()
if err != nil {
t.Fatalf("Failed to flush memtables: %v", err)
}
// Verify data is still accessible after flush
value, err = manager.Get(testKey)
if err != nil {
t.Fatalf("Failed to get value after flush: %v", err)
}
if !bytes.Equal(initialValue, value) {
t.Errorf("Expected initial value %s after flush, got %s", initialValue, value)
}
// Step 3: Update the key with a new value
updatedValue := []byte("updated-value")
err = manager.Put(testKey, updatedValue)
if err != nil {
t.Fatalf("Failed to put updated value: %v", err)
}
// Verify the updated value is readable
value, err = manager.Get(testKey)
if err != nil {
t.Fatalf("Failed to get updated value: %v", err)
}
if !bytes.Equal(updatedValue, value) {
t.Errorf("Expected updated value %s, got %s", updatedValue, value)
}
// Step 4: Flush again to create another SSTable with the updated value
err = manager.FlushMemTables()
if err != nil {
t.Fatalf("Failed to flush memtables again: %v", err)
}
// Verify updated data is still accessible after second flush
value, err = manager.Get(testKey)
if err != nil {
t.Fatalf("Failed to get value after second flush: %v", err)
}
if !bytes.Equal(updatedValue, value) {
t.Errorf("Expected updated value %s after second flush, got %s", updatedValue, value)
}
// Get the last sequence number
lastSeqNum := manager.lastSeqNum
// Step 5: Close the manager and simulate a recovery scenario
err = manager.Close()
if err != nil {
t.Fatalf("Failed to close manager: %v", err)
}
// Create a new manager to simulate recovery
recoveredManager, err := NewManager(cfg, statsCollector)
if err != nil {
t.Fatalf("Failed to create recovered manager: %v", err)
}
defer recoveredManager.Close()
// Verify the key still has the latest value after recovery
recoveredValue, err := recoveredManager.Get(testKey)
if err != nil {
t.Fatalf("Failed to get value after recovery: %v", err)
}
if !bytes.Equal(updatedValue, recoveredValue) {
t.Errorf("Expected updated value %s after recovery, got %s", updatedValue, recoveredValue)
}
// Verify the sequence number was properly recovered
if recoveredManager.lastSeqNum < lastSeqNum {
t.Errorf("Recovered sequence number %d is less than last known sequence number %d",
recoveredManager.lastSeqNum, lastSeqNum)
}
}

View File

@ -240,7 +240,6 @@ func (s *KevoServiceServer) Scan(req *pb.ScanRequest, stream pb.KevoService_Scan
return nil
}
// BeginTransaction starts a new transaction
func (s *KevoServiceServer) BeginTransaction(ctx context.Context, req *pb.BeginTransactionRequest) (*pb.BeginTransactionResponse, error) {
// Force clean up of old transactions before creating new ones
@ -460,7 +459,7 @@ func (s *KevoServiceServer) GetStats(ctx context.Context, req *pb.GetStatsReques
// Collect basic stats that we know are available
keyCount := int64(0)
sstableCount := int32(0)
memtableCount := int32(1) // At least 1 active memtable
memtableCount := int32(0) // Will be updated from storage stats
// Create a read-only transaction to count keys
tx, err := s.engine.BeginTransaction(true)
@ -474,9 +473,18 @@ func (s *KevoServiceServer) GetStats(ctx context.Context, req *pb.GetStatsReques
// Count keys and estimate size
var totalSize int64
for iter.Next() {
keyCount++
totalSize += int64(len(iter.Key()) + len(iter.Value()))
// Position at the first key
iter.SeekToFirst()
// Iterate through all keys
for iter.Valid() {
// Only count live keys (not tombstones/deleted keys)
if !iter.IsTombstone() {
keyCount++
totalSize += int64(len(iter.Key()) + len(iter.Value()))
}
iter.Next()
}
// Get statistics from the engine
@ -484,6 +492,26 @@ func (s *KevoServiceServer) GetStats(ctx context.Context, req *pb.GetStatsReques
GetStatsProvider() interface{}
})
// Get storage stats from engine
if storageManager, ok := s.engine.(interface {
GetStorageStats() map[string]interface{}
}); ok {
storageStats := storageManager.GetStorageStats()
// Set memtable count: always 1 active memtable + any immutable memtables
memtableCount = 1 // Always have at least 1 active memtable
// Add any immutable memtables to the count
if count, ok := storageStats["immutable_memtable_count"].(int); ok {
memtableCount += int32(count)
}
// Update sstable count
if count, ok := storageStats["sstable_count"].(int); ok {
sstableCount = int32(count)
}
}
response := &pb.GetStatsResponse{
KeyCount: keyCount,
StorageSize: totalSize,

View File

@ -88,3 +88,11 @@ func (a *IteratorAdapter) Valid() bool {
func (a *IteratorAdapter) IsTombstone() bool {
return a.iter != nil && a.iter.IsTombstone()
}
// SequenceNumber returns the sequence number of the current entry
func (a *IteratorAdapter) SequenceNumber() uint64 {
if !a.Valid() || a.iter.Entry() == nil {
return 0
}
return a.iter.Entry().seqNum
}

View File

@ -317,3 +317,11 @@ func (it *Iterator) Entry() *entry {
}
return it.current.entry
}
// SequenceNumber returns the sequence number of the current entry
func (it *Iterator) SequenceNumber() uint64 {
if !it.Valid() {
return 0
}
return it.current.entry.seqNum
}

View File

@ -358,11 +358,11 @@ func (m *Manager) startPrimary() error {
opts := []grpc.ServerOption{
grpc.KeepaliveParams(keepalive.ServerParameters{
Time: 30 * time.Second, // Send pings every 30 seconds if there is no activity
Timeout: 10 * time.Second, // Wait 10 seconds for ping ack before assuming connection is dead
Timeout: 10 * time.Second, // Wait 10 seconds for ping ack before assuming connection is dead
}),
grpc.KeepaliveEnforcementPolicy(keepalive.EnforcementPolicy{
MinTime: 10 * time.Second, // Minimum time a client should wait between pings
PermitWithoutStream: true, // Allow pings even when there are no active streams
PermitWithoutStream: true, // Allow pings even when there are no active streams
}),
grpc.MaxRecvMsgSize(16 * 1024 * 1024), // 16MB max message size
grpc.MaxSendMsgSize(16 * 1024 * 1024), // 16MB max message size

View File

@ -807,7 +807,7 @@ func (p *Primary) maybeManageWALRetention() {
// Apply the retention policy using the existing WAL API
config := wal.WALRetentionConfig{
MaxAge: time.Duration(p.retentionConfig.MaxAgeHours) * time.Hour,
MaxAge: time.Duration(p.retentionConfig.MaxAgeHours) * time.Hour,
MinSequenceKeep: minAcknowledgedSeq,
}

View File

@ -32,6 +32,12 @@ func NewBuilder() *Builder {
// Add adds a key-value pair to the block
// Keys must be added in sorted order
func (b *Builder) Add(key, value []byte) error {
return b.AddWithSequence(key, value, 0) // Default sequence number to 0 for backward compatibility
}
// AddWithSequence adds a key-value pair to the block with a sequence number
// Keys must be added in sorted order
func (b *Builder) AddWithSequence(key, value []byte, seqNum uint64) 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",
@ -39,8 +45,9 @@ func (b *Builder) Add(key, value []byte) error {
}
b.entries = append(b.entries, Entry{
Key: append([]byte(nil), key...), // Make copies to avoid references
Value: append([]byte(nil), value...), // to external data
Key: append([]byte(nil), key...), // Make copies to avoid references
Value: append([]byte(nil), value...), // to external data
SequenceNum: seqNum,
})
// Add restart point if needed
@ -51,7 +58,7 @@ func (b *Builder) Add(key, value []byte) error {
b.restartIdx++
// Track the size
b.currentSize += uint32(len(key) + len(value) + 8) // 8 bytes for metadata
b.currentSize += uint32(len(key) + len(value) + 16) // 16 bytes for metadata (including sequence number)
b.lastKey = append([]byte(nil), key...)
return nil
@ -94,16 +101,26 @@ func (b *Builder) Finish(w io.Writer) (uint64, error) {
// Keys are already sorted by the Add method's requirement
// Remove any duplicate keys (keeping the last one)
// Remove any duplicate keys (keeping the one with the highest sequence number)
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])
// Group entries by key and find entry with highest sequence number
keyMap := make(map[string]Entry)
for _, entry := range b.entries {
keyStr := string(entry.Key)
if existing, exists := keyMap[keyStr]; !exists || entry.SequenceNum > existing.SequenceNum {
keyMap[keyStr] = entry
}
}
// Rebuild sorted entries from the map
uniqueEntries := make([]Entry, 0, len(keyMap))
for _, entry := range b.entries {
keyStr := string(entry.Key)
if best, exists := keyMap[keyStr]; exists {
if bytes.Equal(entry.Key, best.Key) && entry.SequenceNum == best.SequenceNum {
uniqueEntries = append(uniqueEntries, best)
delete(keyMap, keyStr) // Delete to avoid duplicates
}
}
}
b.entries = uniqueEntries
@ -176,9 +193,15 @@ func (b *Builder) Finish(w io.Writer) (uint64, error) {
}
}
// Write sequence number
err := binary.Write(buffer, binary.LittleEndian, entry.SequenceNum)
if err != nil {
return 0, fmt.Errorf("failed to write sequence number: %w", err)
}
// Write value
valueLen := uint32(len(entry.Value))
err := binary.Write(buffer, binary.LittleEndian, valueLen)
err = binary.Write(buffer, binary.LittleEndian, valueLen)
if err != nil {
return 0, fmt.Errorf("failed to write value length: %w", err)
}

View File

@ -7,13 +7,14 @@ import (
// 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)
reader *Reader
currentPos uint32
currentKey []byte
currentVal []byte
currentSeqNum uint64 // Sequence number of the current entry
restartIdx int
initialized bool
dataEnd uint32 // Position where the actual entries data ends (before restart points)
}
// SeekToFirst positions the iterator at the first entry
@ -199,6 +200,14 @@ func (it *Iterator) IsTombstone() bool {
return it.Valid() && it.currentVal == nil
}
// SequenceNumber returns the sequence number of the current entry
func (it *Iterator) SequenceNumber() uint64 {
if !it.Valid() {
return 0
}
return it.currentSeqNum
}
// decodeCurrent decodes the entry at the current position
func (it *Iterator) decodeCurrent() ([]byte, []byte, bool) {
if it.currentPos >= it.dataEnd {
@ -221,6 +230,13 @@ func (it *Iterator) decodeCurrent() ([]byte, []byte, bool) {
copy(key, data[:keyLen])
data = data[keyLen:]
// Read sequence number if format includes it (check if enough data for both seq num and value len)
seqNum := uint64(0)
if len(data) >= 12 { // 8 for seq num + 4 for value len
seqNum = binary.LittleEndian.Uint64(data)
data = data[8:]
}
// Read value
if len(data) < 4 {
return nil, nil, false
@ -238,6 +254,7 @@ func (it *Iterator) decodeCurrent() ([]byte, []byte, bool) {
it.currentKey = key
it.currentVal = value
it.currentSeqNum = seqNum
return key, value, true
}
@ -303,6 +320,14 @@ func (it *Iterator) decodeNext() ([]byte, []byte, bool) {
it.currentPos += 4 + uint32(unsharedLen)
}
// Read sequence number if format includes it (check if enough data for both seq num and value len)
seqNum := uint64(0)
if len(data) >= 12 { // 8 for seq num + 4 for value len
seqNum = binary.LittleEndian.Uint64(data)
data = data[8:]
it.currentPos += 8
}
// Read value
if len(data) < 4 {
return nil, nil, false
@ -318,6 +343,7 @@ func (it *Iterator) decodeNext() ([]byte, []byte, bool) {
value := make([]byte, valueLen)
copy(value, data[:valueLen])
it.currentSeqNum = seqNum
it.currentPos += 4 + uint32(valueLen)
return key, value, true

View File

@ -2,8 +2,9 @@ package block
// Entry represents a key-value pair within the block
type Entry struct {
Key []byte
Value []byte
Key []byte
Value []byte
SequenceNum uint64 // Sequence number for versioning
}
const (

View File

@ -209,6 +209,19 @@ func (it *Iterator) IsTombstone() bool {
return it.dataBlockIter.Value() == nil
}
// SequenceNumber returns the sequence number of the current entry
func (it *Iterator) SequenceNumber() uint64 {
it.mu.Lock()
defer it.mu.Unlock()
// Not valid means sequence number 0
if !it.initialized || it.dataBlockIter == nil || !it.dataBlockIter.Valid() {
return 0
}
return it.dataBlockIter.SequenceNumber()
}
// Error returns any error encountered during iteration
func (it *Iterator) Error() error {
it.mu.Lock()

View File

@ -57,3 +57,11 @@ func (a *IteratorAdapter) Valid() bool {
func (a *IteratorAdapter) IsTombstone() bool {
return a.Valid() && a.iter.IsTombstone()
}
// SequenceNumber returns the sequence number of the current entry
func (a *IteratorAdapter) SequenceNumber() uint64 {
if !a.Valid() {
return 0
}
return a.iter.SequenceNumber()
}

View File

@ -99,6 +99,11 @@ func (bm *BlockManager) Add(key, value []byte) error {
return bm.builder.Add(key, value)
}
// AddWithSequence adds a key-value pair with a sequence number to the current block
func (bm *BlockManager) AddWithSequence(key, value []byte, seqNum uint64) error {
return bm.builder.AddWithSequence(key, value, seqNum)
}
// EstimatedSize returns the estimated size of the current block
func (bm *BlockManager) EstimatedSize() uint32 {
return bm.builder.EstimatedSize()
@ -285,6 +290,12 @@ func NewWriterWithOptions(path string, options WriterOptions) (*Writer, error) {
// Add adds a key-value pair to the SSTable
// Keys must be added in sorted order
func (w *Writer) Add(key, value []byte) error {
return w.AddWithSequence(key, value, 0) // Default to sequence number 0 for backward compatibility
}
// AddWithSequence adds a key-value pair with a sequence number to the SSTable
// Keys must be added in sorted order
func (w *Writer) AddWithSequence(key, value []byte, seqNum uint64) error {
// Keep track of first and last keys
if w.entriesAdded == 0 {
w.firstKey = append([]byte(nil), key...)
@ -296,8 +307,8 @@ func (w *Writer) Add(key, value []byte) error {
w.currentBloomFilter.AddKey(key)
}
// Add to block
if err := w.blockManager.Add(key, value); err != nil {
// Add to block with sequence number
if err := w.blockManager.AddWithSequence(key, value, seqNum); err != nil {
return fmt.Errorf("failed to add to block: %w", err)
}

View File

@ -78,14 +78,14 @@ func (m *Manager) BeginTransaction(readOnly bool) (Transaction, error) {
}
tx := &TransactionImpl{
storage: m.storage,
mode: mode,
buffer: NewBuffer(),
rwLock: &m.txLock,
stats: m,
creationTime: now,
storage: m.storage,
mode: mode,
buffer: NewBuffer(),
rwLock: &m.txLock,
stats: m,
creationTime: now,
lastActiveTime: now,
ttl: ttl,
ttl: ttl,
}
// Set transaction as active

View File

@ -12,28 +12,28 @@ import (
// Registry manages transaction lifecycle and connections
type RegistryImpl struct {
mu sync.RWMutex
transactions map[string]Transaction
nextID uint64
cleanupTicker *time.Ticker
stopCleanup chan struct{}
connectionTxs map[string]map[string]struct{}
txTTL time.Duration
txWarningThreshold int
mu sync.RWMutex
transactions map[string]Transaction
nextID uint64
cleanupTicker *time.Ticker
stopCleanup chan struct{}
connectionTxs map[string]map[string]struct{}
txTTL time.Duration
txWarningThreshold int
txCriticalThreshold int
idleTxTTL time.Duration
idleTxTTL time.Duration
}
// NewRegistry creates a new transaction registry with default settings
func NewRegistry() Registry {
r := &RegistryImpl{
transactions: make(map[string]Transaction),
connectionTxs: make(map[string]map[string]struct{}),
stopCleanup: make(chan struct{}),
txTTL: 5 * time.Minute, // Default TTL
idleTxTTL: 30 * time.Second, // Idle timeout
txWarningThreshold: 75, // 75% of TTL
txCriticalThreshold: 90, // 90% of TTL
transactions: make(map[string]Transaction),
connectionTxs: make(map[string]map[string]struct{}),
stopCleanup: make(chan struct{}),
txTTL: 5 * time.Minute, // Default TTL
idleTxTTL: 30 * time.Second, // Idle timeout
txWarningThreshold: 75, // 75% of TTL
txCriticalThreshold: 90, // 90% of TTL
}
// Start periodic cleanup
@ -46,12 +46,12 @@ func NewRegistry() Registry {
// NewRegistryWithTTL creates a new transaction registry with a specific TTL
func NewRegistryWithTTL(ttl time.Duration, idleTimeout time.Duration, warningThreshold, criticalThreshold int) Registry {
r := &RegistryImpl{
transactions: make(map[string]Transaction),
connectionTxs: make(map[string]map[string]struct{}),
stopCleanup: make(chan struct{}),
txTTL: ttl,
idleTxTTL: idleTimeout,
txWarningThreshold: warningThreshold,
transactions: make(map[string]Transaction),
connectionTxs: make(map[string]map[string]struct{}),
stopCleanup: make(chan struct{}),
txTTL: ttl,
idleTxTTL: idleTimeout,
txWarningThreshold: warningThreshold,
txCriticalThreshold: criticalThreshold,
}
@ -127,7 +127,7 @@ func (r *RegistryImpl) CleanupStaleTransactions() {
if txImpl, ok := tx.(*TransactionImpl); ok {
fmt.Printf("WARNING: Transaction %s has been running for %s (%.1f%% of TTL)\n",
id, now.Sub(txImpl.creationTime).String(),
(float64(now.Sub(txImpl.creationTime)) / float64(txImpl.ttl)) * 100)
(float64(now.Sub(txImpl.creationTime))/float64(txImpl.ttl))*100)
}
}
}
@ -138,7 +138,7 @@ func (r *RegistryImpl) CleanupStaleTransactions() {
if txImpl, ok := tx.(*TransactionImpl); ok {
fmt.Printf("CRITICAL: Transaction %s has been running for %s (%.1f%% of TTL)\n",
id, now.Sub(txImpl.creationTime).String(),
(float64(now.Sub(txImpl.creationTime)) / float64(txImpl.ttl)) * 100)
(float64(now.Sub(txImpl.creationTime))/float64(txImpl.ttl))*100)
}
}
}

View File

@ -28,14 +28,14 @@ type RetryPolicy struct {
// TransportOptions contains common configuration across all transport types
type TransportOptions struct {
Timeout time.Duration
RetryPolicy RetryPolicy
Compression CompressionType
MaxMessageSize int
TLSEnabled bool
CertFile string
KeyFile string
CAFile string
Timeout time.Duration
RetryPolicy RetryPolicy
Compression CompressionType
MaxMessageSize int
TLSEnabled bool
CertFile string
KeyFile string
CAFile string
KeepaliveParams *keepalive.ClientParameters // Optional keepalive parameters for gRPC clients
}