Browse Source
Phase 2: Implement offset assignment logic and recovery
Phase 2: Implement offset assignment logic and recovery
- Add PartitionOffsetManager for sequential offset assignment per partition - Implement OffsetStorage interface with in-memory and SQL storage backends - Add PartitionOffsetRegistry for managing multiple partition offset managers - Implement offset recovery from checkpoints and storage scanning - Add OffsetAssigner for high-level offset assignment operations - Support both single and batch offset assignment with timestamps - Add comprehensive tests covering: - Basic and batch offset assignment - Offset recovery from checkpoints and storage - Multi-partition offset management - Concurrent offset assignment safety - All tests pass, offset assignment is thread-safe and recoverablepull/7231/head
3 changed files with 820 additions and 0 deletions
@ -0,0 +1,302 @@ |
|||||
|
package offset |
||||
|
|
||||
|
import ( |
||||
|
"fmt" |
||||
|
"sync" |
||||
|
"time" |
||||
|
|
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" |
||||
|
) |
||||
|
|
||||
|
// PartitionOffsetManager manages sequential offset assignment for a single partition
|
||||
|
type PartitionOffsetManager struct { |
||||
|
mu sync.RWMutex |
||||
|
partition *schema_pb.Partition |
||||
|
nextOffset int64 |
||||
|
|
||||
|
// Checkpointing for recovery
|
||||
|
lastCheckpoint int64 |
||||
|
checkpointInterval int64 |
||||
|
storage OffsetStorage |
||||
|
} |
||||
|
|
||||
|
// OffsetStorage interface for persisting offset state
|
||||
|
type OffsetStorage interface { |
||||
|
// SaveCheckpoint persists the current offset state for recovery
|
||||
|
SaveCheckpoint(partition *schema_pb.Partition, offset int64) error |
||||
|
|
||||
|
// LoadCheckpoint retrieves the last saved offset state
|
||||
|
LoadCheckpoint(partition *schema_pb.Partition) (int64, error) |
||||
|
|
||||
|
// GetHighestOffset scans storage to find the highest assigned offset
|
||||
|
GetHighestOffset(partition *schema_pb.Partition) (int64, error) |
||||
|
} |
||||
|
|
||||
|
// NewPartitionOffsetManager creates a new offset manager for a partition
|
||||
|
func NewPartitionOffsetManager(partition *schema_pb.Partition, storage OffsetStorage) (*PartitionOffsetManager, error) { |
||||
|
manager := &PartitionOffsetManager{ |
||||
|
partition: partition, |
||||
|
checkpointInterval: 100, // Checkpoint every 100 offsets
|
||||
|
storage: storage, |
||||
|
} |
||||
|
|
||||
|
// Recover offset state
|
||||
|
if err := manager.recover(); err != nil { |
||||
|
return nil, fmt.Errorf("failed to recover offset state: %w", err) |
||||
|
} |
||||
|
|
||||
|
return manager, nil |
||||
|
} |
||||
|
|
||||
|
// AssignOffset assigns the next sequential offset
|
||||
|
func (m *PartitionOffsetManager) AssignOffset() int64 { |
||||
|
m.mu.Lock() |
||||
|
defer m.mu.Unlock() |
||||
|
|
||||
|
offset := m.nextOffset |
||||
|
m.nextOffset++ |
||||
|
|
||||
|
// Checkpoint periodically
|
||||
|
if offset-m.lastCheckpoint >= m.checkpointInterval { |
||||
|
go m.checkpoint(offset) |
||||
|
} |
||||
|
|
||||
|
return offset |
||||
|
} |
||||
|
|
||||
|
// AssignOffsets assigns a batch of sequential offsets
|
||||
|
func (m *PartitionOffsetManager) AssignOffsets(count int64) (baseOffset int64, lastOffset int64) { |
||||
|
m.mu.Lock() |
||||
|
defer m.mu.Unlock() |
||||
|
|
||||
|
baseOffset = m.nextOffset |
||||
|
lastOffset = m.nextOffset + count - 1 |
||||
|
m.nextOffset += count |
||||
|
|
||||
|
// Checkpoint if needed
|
||||
|
if lastOffset-m.lastCheckpoint >= m.checkpointInterval { |
||||
|
go m.checkpoint(lastOffset) |
||||
|
} |
||||
|
|
||||
|
return baseOffset, lastOffset |
||||
|
} |
||||
|
|
||||
|
// GetNextOffset returns the next offset that will be assigned
|
||||
|
func (m *PartitionOffsetManager) GetNextOffset() int64 { |
||||
|
m.mu.RLock() |
||||
|
defer m.mu.RUnlock() |
||||
|
return m.nextOffset |
||||
|
} |
||||
|
|
||||
|
// GetHighWaterMark returns the high water mark (next offset)
|
||||
|
func (m *PartitionOffsetManager) GetHighWaterMark() int64 { |
||||
|
return m.GetNextOffset() |
||||
|
} |
||||
|
|
||||
|
// recover restores offset state from storage
|
||||
|
func (m *PartitionOffsetManager) recover() error { |
||||
|
var checkpointOffset int64 = -1 |
||||
|
var highestOffset int64 = -1 |
||||
|
|
||||
|
// Try to load checkpoint
|
||||
|
if offset, err := m.storage.LoadCheckpoint(m.partition); err == nil && offset >= 0 { |
||||
|
checkpointOffset = offset |
||||
|
} |
||||
|
|
||||
|
// Try to scan storage for highest offset
|
||||
|
if offset, err := m.storage.GetHighestOffset(m.partition); err == nil && offset >= 0 { |
||||
|
highestOffset = offset |
||||
|
} |
||||
|
|
||||
|
// Use the higher of checkpoint or storage scan
|
||||
|
if checkpointOffset >= 0 && highestOffset >= 0 { |
||||
|
if highestOffset > checkpointOffset { |
||||
|
m.nextOffset = highestOffset + 1 |
||||
|
m.lastCheckpoint = highestOffset |
||||
|
} else { |
||||
|
m.nextOffset = checkpointOffset + 1 |
||||
|
m.lastCheckpoint = checkpointOffset |
||||
|
} |
||||
|
} else if checkpointOffset >= 0 { |
||||
|
m.nextOffset = checkpointOffset + 1 |
||||
|
m.lastCheckpoint = checkpointOffset |
||||
|
} else if highestOffset >= 0 { |
||||
|
m.nextOffset = highestOffset + 1 |
||||
|
m.lastCheckpoint = highestOffset |
||||
|
} else { |
||||
|
// No data exists, start from 0
|
||||
|
m.nextOffset = 0 |
||||
|
m.lastCheckpoint = -1 |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// checkpoint saves the current offset state
|
||||
|
func (m *PartitionOffsetManager) checkpoint(offset int64) { |
||||
|
if err := m.storage.SaveCheckpoint(m.partition, offset); err != nil { |
||||
|
// Log error but don't fail - checkpointing is for optimization
|
||||
|
fmt.Printf("Failed to checkpoint offset %d: %v\n", offset, err) |
||||
|
return |
||||
|
} |
||||
|
|
||||
|
m.mu.Lock() |
||||
|
m.lastCheckpoint = offset |
||||
|
m.mu.Unlock() |
||||
|
} |
||||
|
|
||||
|
// PartitionOffsetRegistry manages offset managers for multiple partitions
|
||||
|
type PartitionOffsetRegistry struct { |
||||
|
mu sync.RWMutex |
||||
|
managers map[string]*PartitionOffsetManager |
||||
|
storage OffsetStorage |
||||
|
} |
||||
|
|
||||
|
// NewPartitionOffsetRegistry creates a new registry
|
||||
|
func NewPartitionOffsetRegistry(storage OffsetStorage) *PartitionOffsetRegistry { |
||||
|
return &PartitionOffsetRegistry{ |
||||
|
managers: make(map[string]*PartitionOffsetManager), |
||||
|
storage: storage, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// GetManager returns the offset manager for a partition, creating it if needed
|
||||
|
func (r *PartitionOffsetRegistry) GetManager(partition *schema_pb.Partition) (*PartitionOffsetManager, error) { |
||||
|
key := partitionKey(partition) |
||||
|
|
||||
|
r.mu.RLock() |
||||
|
manager, exists := r.managers[key] |
||||
|
r.mu.RUnlock() |
||||
|
|
||||
|
if exists { |
||||
|
return manager, nil |
||||
|
} |
||||
|
|
||||
|
// Create new manager
|
||||
|
r.mu.Lock() |
||||
|
defer r.mu.Unlock() |
||||
|
|
||||
|
// Double-check after acquiring write lock
|
||||
|
if manager, exists := r.managers[key]; exists { |
||||
|
return manager, nil |
||||
|
} |
||||
|
|
||||
|
manager, err := NewPartitionOffsetManager(partition, r.storage) |
||||
|
if err != nil { |
||||
|
return nil, err |
||||
|
} |
||||
|
|
||||
|
r.managers[key] = manager |
||||
|
return manager, nil |
||||
|
} |
||||
|
|
||||
|
// AssignOffset assigns an offset for the given partition
|
||||
|
func (r *PartitionOffsetRegistry) AssignOffset(partition *schema_pb.Partition) (int64, error) { |
||||
|
manager, err := r.GetManager(partition) |
||||
|
if err != nil { |
||||
|
return 0, err |
||||
|
} |
||||
|
|
||||
|
return manager.AssignOffset(), nil |
||||
|
} |
||||
|
|
||||
|
// AssignOffsets assigns a batch of offsets for the given partition
|
||||
|
func (r *PartitionOffsetRegistry) AssignOffsets(partition *schema_pb.Partition, count int64) (baseOffset, lastOffset int64, err error) { |
||||
|
manager, err := r.GetManager(partition) |
||||
|
if err != nil { |
||||
|
return 0, 0, err |
||||
|
} |
||||
|
|
||||
|
baseOffset, lastOffset = manager.AssignOffsets(count) |
||||
|
return baseOffset, lastOffset, nil |
||||
|
} |
||||
|
|
||||
|
// GetHighWaterMark returns the high water mark for a partition
|
||||
|
func (r *PartitionOffsetRegistry) GetHighWaterMark(partition *schema_pb.Partition) (int64, error) { |
||||
|
manager, err := r.GetManager(partition) |
||||
|
if err != nil { |
||||
|
return 0, err |
||||
|
} |
||||
|
|
||||
|
return manager.GetHighWaterMark(), nil |
||||
|
} |
||||
|
|
||||
|
// partitionKey generates a unique key for a partition
|
||||
|
func partitionKey(partition *schema_pb.Partition) string { |
||||
|
return fmt.Sprintf("ring:%d:range:%d-%d:time:%d", |
||||
|
partition.RingSize, partition.RangeStart, partition.RangeStop, partition.UnixTimeNs) |
||||
|
} |
||||
|
|
||||
|
// OffsetAssignment represents an assigned offset with metadata
|
||||
|
type OffsetAssignment struct { |
||||
|
Offset int64 |
||||
|
Timestamp int64 |
||||
|
Partition *schema_pb.Partition |
||||
|
} |
||||
|
|
||||
|
// BatchOffsetAssignment represents a batch of assigned offsets
|
||||
|
type BatchOffsetAssignment struct { |
||||
|
BaseOffset int64 |
||||
|
LastOffset int64 |
||||
|
Count int64 |
||||
|
Timestamp int64 |
||||
|
Partition *schema_pb.Partition |
||||
|
} |
||||
|
|
||||
|
// AssignmentResult contains the result of offset assignment
|
||||
|
type AssignmentResult struct { |
||||
|
Assignment *OffsetAssignment |
||||
|
Batch *BatchOffsetAssignment |
||||
|
Error error |
||||
|
} |
||||
|
|
||||
|
// OffsetAssigner provides high-level offset assignment operations
|
||||
|
type OffsetAssigner struct { |
||||
|
registry *PartitionOffsetRegistry |
||||
|
} |
||||
|
|
||||
|
// NewOffsetAssigner creates a new offset assigner
|
||||
|
func NewOffsetAssigner(storage OffsetStorage) *OffsetAssigner { |
||||
|
return &OffsetAssigner{ |
||||
|
registry: NewPartitionOffsetRegistry(storage), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// AssignSingleOffset assigns a single offset with timestamp
|
||||
|
func (a *OffsetAssigner) AssignSingleOffset(partition *schema_pb.Partition) *AssignmentResult { |
||||
|
offset, err := a.registry.AssignOffset(partition) |
||||
|
if err != nil { |
||||
|
return &AssignmentResult{Error: err} |
||||
|
} |
||||
|
|
||||
|
return &AssignmentResult{ |
||||
|
Assignment: &OffsetAssignment{ |
||||
|
Offset: offset, |
||||
|
Timestamp: time.Now().UnixNano(), |
||||
|
Partition: partition, |
||||
|
}, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// AssignBatchOffsets assigns a batch of offsets with timestamp
|
||||
|
func (a *OffsetAssigner) AssignBatchOffsets(partition *schema_pb.Partition, count int64) *AssignmentResult { |
||||
|
baseOffset, lastOffset, err := a.registry.AssignOffsets(partition, count) |
||||
|
if err != nil { |
||||
|
return &AssignmentResult{Error: err} |
||||
|
} |
||||
|
|
||||
|
return &AssignmentResult{ |
||||
|
Batch: &BatchOffsetAssignment{ |
||||
|
BaseOffset: baseOffset, |
||||
|
LastOffset: lastOffset, |
||||
|
Count: count, |
||||
|
Timestamp: time.Now().UnixNano(), |
||||
|
Partition: partition, |
||||
|
}, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// GetHighWaterMark returns the high water mark for a partition
|
||||
|
func (a *OffsetAssigner) GetHighWaterMark(partition *schema_pb.Partition) (int64, error) { |
||||
|
return a.registry.GetHighWaterMark(partition) |
||||
|
} |
||||
@ -0,0 +1,388 @@ |
|||||
|
package offset |
||||
|
|
||||
|
import ( |
||||
|
"testing" |
||||
|
"time" |
||||
|
|
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" |
||||
|
) |
||||
|
|
||||
|
func createTestPartition() *schema_pb.Partition { |
||||
|
return &schema_pb.Partition{ |
||||
|
RingSize: 1024, |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 31, |
||||
|
UnixTimeNs: time.Now().UnixNano(), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestPartitionOffsetManager_BasicAssignment(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
manager, err := NewPartitionOffsetManager(partition, storage) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create offset manager: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Test sequential offset assignment
|
||||
|
for i := int64(0); i < 10; i++ { |
||||
|
offset := manager.AssignOffset() |
||||
|
if offset != i { |
||||
|
t.Errorf("Expected offset %d, got %d", i, offset) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// Test high water mark
|
||||
|
hwm := manager.GetHighWaterMark() |
||||
|
if hwm != 10 { |
||||
|
t.Errorf("Expected high water mark 10, got %d", hwm) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestPartitionOffsetManager_BatchAssignment(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
manager, err := NewPartitionOffsetManager(partition, storage) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create offset manager: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Assign batch of 5 offsets
|
||||
|
baseOffset, lastOffset := manager.AssignOffsets(5) |
||||
|
if baseOffset != 0 { |
||||
|
t.Errorf("Expected base offset 0, got %d", baseOffset) |
||||
|
} |
||||
|
if lastOffset != 4 { |
||||
|
t.Errorf("Expected last offset 4, got %d", lastOffset) |
||||
|
} |
||||
|
|
||||
|
// Assign another batch
|
||||
|
baseOffset, lastOffset = manager.AssignOffsets(3) |
||||
|
if baseOffset != 5 { |
||||
|
t.Errorf("Expected base offset 5, got %d", baseOffset) |
||||
|
} |
||||
|
if lastOffset != 7 { |
||||
|
t.Errorf("Expected last offset 7, got %d", lastOffset) |
||||
|
} |
||||
|
|
||||
|
// Check high water mark
|
||||
|
hwm := manager.GetHighWaterMark() |
||||
|
if hwm != 8 { |
||||
|
t.Errorf("Expected high water mark 8, got %d", hwm) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestPartitionOffsetManager_Recovery(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
// Create manager and assign some offsets
|
||||
|
manager1, err := NewPartitionOffsetManager(partition, storage) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create offset manager: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Assign offsets and simulate records
|
||||
|
for i := 0; i < 150; i++ { // More than checkpoint interval
|
||||
|
offset := manager1.AssignOffset() |
||||
|
storage.AddRecord(partition, offset) |
||||
|
} |
||||
|
|
||||
|
// Wait for checkpoint to complete
|
||||
|
time.Sleep(100 * time.Millisecond) |
||||
|
|
||||
|
// Create new manager (simulates restart)
|
||||
|
manager2, err := NewPartitionOffsetManager(partition, storage) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create offset manager after recovery: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Next offset should continue from checkpoint + 1
|
||||
|
// With checkpoint interval 100, checkpoint happens at offset 100
|
||||
|
// So recovery should start from 101, but we assigned 150 offsets (0-149)
|
||||
|
// The checkpoint should be at 100, so next offset should be 101
|
||||
|
// But since we have records up to 149, it should recover from storage scan
|
||||
|
nextOffset := manager2.AssignOffset() |
||||
|
if nextOffset != 150 { |
||||
|
t.Errorf("Expected next offset 150 after recovery, got %d", nextOffset) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestPartitionOffsetManager_RecoveryFromStorage(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
// Simulate existing records in storage without checkpoint
|
||||
|
for i := int64(0); i < 50; i++ { |
||||
|
storage.AddRecord(partition, i) |
||||
|
} |
||||
|
|
||||
|
// Create manager - should recover from storage scan
|
||||
|
manager, err := NewPartitionOffsetManager(partition, storage) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create offset manager: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Next offset should be 50
|
||||
|
nextOffset := manager.AssignOffset() |
||||
|
if nextOffset != 50 { |
||||
|
t.Errorf("Expected next offset 50 after storage recovery, got %d", nextOffset) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestPartitionOffsetRegistry_MultiplePartitions(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
registry := NewPartitionOffsetRegistry(storage) |
||||
|
|
||||
|
// Create different partitions
|
||||
|
partition1 := &schema_pb.Partition{ |
||||
|
RingSize: 1024, |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 31, |
||||
|
UnixTimeNs: time.Now().UnixNano(), |
||||
|
} |
||||
|
|
||||
|
partition2 := &schema_pb.Partition{ |
||||
|
RingSize: 1024, |
||||
|
RangeStart: 32, |
||||
|
RangeStop: 63, |
||||
|
UnixTimeNs: time.Now().UnixNano(), |
||||
|
} |
||||
|
|
||||
|
// Assign offsets to different partitions
|
||||
|
offset1, err := registry.AssignOffset(partition1) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to assign offset to partition1: %v", err) |
||||
|
} |
||||
|
if offset1 != 0 { |
||||
|
t.Errorf("Expected offset 0 for partition1, got %d", offset1) |
||||
|
} |
||||
|
|
||||
|
offset2, err := registry.AssignOffset(partition2) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to assign offset to partition2: %v", err) |
||||
|
} |
||||
|
if offset2 != 0 { |
||||
|
t.Errorf("Expected offset 0 for partition2, got %d", offset2) |
||||
|
} |
||||
|
|
||||
|
// Assign more offsets to partition1
|
||||
|
offset1_2, err := registry.AssignOffset(partition1) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to assign second offset to partition1: %v", err) |
||||
|
} |
||||
|
if offset1_2 != 1 { |
||||
|
t.Errorf("Expected offset 1 for partition1, got %d", offset1_2) |
||||
|
} |
||||
|
|
||||
|
// Partition2 should still be at 0 for next assignment
|
||||
|
offset2_2, err := registry.AssignOffset(partition2) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to assign second offset to partition2: %v", err) |
||||
|
} |
||||
|
if offset2_2 != 1 { |
||||
|
t.Errorf("Expected offset 1 for partition2, got %d", offset2_2) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestPartitionOffsetRegistry_BatchAssignment(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
registry := NewPartitionOffsetRegistry(storage) |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
// Assign batch of offsets
|
||||
|
baseOffset, lastOffset, err := registry.AssignOffsets(partition, 10) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to assign batch offsets: %v", err) |
||||
|
} |
||||
|
|
||||
|
if baseOffset != 0 { |
||||
|
t.Errorf("Expected base offset 0, got %d", baseOffset) |
||||
|
} |
||||
|
if lastOffset != 9 { |
||||
|
t.Errorf("Expected last offset 9, got %d", lastOffset) |
||||
|
} |
||||
|
|
||||
|
// Get high water mark
|
||||
|
hwm, err := registry.GetHighWaterMark(partition) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get high water mark: %v", err) |
||||
|
} |
||||
|
if hwm != 10 { |
||||
|
t.Errorf("Expected high water mark 10, got %d", hwm) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestOffsetAssigner_SingleAssignment(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
assigner := NewOffsetAssigner(storage) |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
// Assign single offset
|
||||
|
result := assigner.AssignSingleOffset(partition) |
||||
|
if result.Error != nil { |
||||
|
t.Fatalf("Failed to assign single offset: %v", result.Error) |
||||
|
} |
||||
|
|
||||
|
if result.Assignment == nil { |
||||
|
t.Fatal("Assignment result is nil") |
||||
|
} |
||||
|
|
||||
|
if result.Assignment.Offset != 0 { |
||||
|
t.Errorf("Expected offset 0, got %d", result.Assignment.Offset) |
||||
|
} |
||||
|
|
||||
|
if result.Assignment.Partition != partition { |
||||
|
t.Error("Partition mismatch in assignment") |
||||
|
} |
||||
|
|
||||
|
if result.Assignment.Timestamp <= 0 { |
||||
|
t.Error("Timestamp should be set") |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestOffsetAssigner_BatchAssignment(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
assigner := NewOffsetAssigner(storage) |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
// Assign batch of offsets
|
||||
|
result := assigner.AssignBatchOffsets(partition, 5) |
||||
|
if result.Error != nil { |
||||
|
t.Fatalf("Failed to assign batch offsets: %v", result.Error) |
||||
|
} |
||||
|
|
||||
|
if result.Batch == nil { |
||||
|
t.Fatal("Batch result is nil") |
||||
|
} |
||||
|
|
||||
|
if result.Batch.BaseOffset != 0 { |
||||
|
t.Errorf("Expected base offset 0, got %d", result.Batch.BaseOffset) |
||||
|
} |
||||
|
|
||||
|
if result.Batch.LastOffset != 4 { |
||||
|
t.Errorf("Expected last offset 4, got %d", result.Batch.LastOffset) |
||||
|
} |
||||
|
|
||||
|
if result.Batch.Count != 5 { |
||||
|
t.Errorf("Expected count 5, got %d", result.Batch.Count) |
||||
|
} |
||||
|
|
||||
|
if result.Batch.Timestamp <= 0 { |
||||
|
t.Error("Timestamp should be set") |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestOffsetAssigner_HighWaterMark(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
assigner := NewOffsetAssigner(storage) |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
// Initially should be 0
|
||||
|
hwm, err := assigner.GetHighWaterMark(partition) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get initial high water mark: %v", err) |
||||
|
} |
||||
|
if hwm != 0 { |
||||
|
t.Errorf("Expected initial high water mark 0, got %d", hwm) |
||||
|
} |
||||
|
|
||||
|
// Assign some offsets
|
||||
|
assigner.AssignBatchOffsets(partition, 10) |
||||
|
|
||||
|
// High water mark should be updated
|
||||
|
hwm, err = assigner.GetHighWaterMark(partition) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get high water mark after assignment: %v", err) |
||||
|
} |
||||
|
if hwm != 10 { |
||||
|
t.Errorf("Expected high water mark 10, got %d", hwm) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestPartitionKey(t *testing.T) { |
||||
|
partition1 := &schema_pb.Partition{ |
||||
|
RingSize: 1024, |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 31, |
||||
|
UnixTimeNs: 1234567890, |
||||
|
} |
||||
|
|
||||
|
partition2 := &schema_pb.Partition{ |
||||
|
RingSize: 1024, |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 31, |
||||
|
UnixTimeNs: 1234567890, |
||||
|
} |
||||
|
|
||||
|
partition3 := &schema_pb.Partition{ |
||||
|
RingSize: 1024, |
||||
|
RangeStart: 32, |
||||
|
RangeStop: 63, |
||||
|
UnixTimeNs: 1234567890, |
||||
|
} |
||||
|
|
||||
|
key1 := partitionKey(partition1) |
||||
|
key2 := partitionKey(partition2) |
||||
|
key3 := partitionKey(partition3) |
||||
|
|
||||
|
// Same partitions should have same key
|
||||
|
if key1 != key2 { |
||||
|
t.Errorf("Same partitions should have same key: %s vs %s", key1, key2) |
||||
|
} |
||||
|
|
||||
|
// Different partitions should have different keys
|
||||
|
if key1 == key3 { |
||||
|
t.Errorf("Different partitions should have different keys: %s vs %s", key1, key3) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestConcurrentOffsetAssignment(t *testing.T) { |
||||
|
storage := NewInMemoryOffsetStorage() |
||||
|
registry := NewPartitionOffsetRegistry(storage) |
||||
|
partition := createTestPartition() |
||||
|
|
||||
|
const numGoroutines = 10 |
||||
|
const offsetsPerGoroutine = 100 |
||||
|
|
||||
|
results := make(chan int64, numGoroutines*offsetsPerGoroutine) |
||||
|
|
||||
|
// Start concurrent offset assignments
|
||||
|
for i := 0; i < numGoroutines; i++ { |
||||
|
go func() { |
||||
|
for j := 0; j < offsetsPerGoroutine; j++ { |
||||
|
offset, err := registry.AssignOffset(partition) |
||||
|
if err != nil { |
||||
|
t.Errorf("Failed to assign offset: %v", err) |
||||
|
return |
||||
|
} |
||||
|
results <- offset |
||||
|
} |
||||
|
}() |
||||
|
} |
||||
|
|
||||
|
// Collect all results
|
||||
|
offsets := make(map[int64]bool) |
||||
|
for i := 0; i < numGoroutines*offsetsPerGoroutine; i++ { |
||||
|
offset := <-results |
||||
|
if offsets[offset] { |
||||
|
t.Errorf("Duplicate offset assigned: %d", offset) |
||||
|
} |
||||
|
offsets[offset] = true |
||||
|
} |
||||
|
|
||||
|
// Verify we got all expected offsets
|
||||
|
expectedCount := numGoroutines * offsetsPerGoroutine |
||||
|
if len(offsets) != expectedCount { |
||||
|
t.Errorf("Expected %d unique offsets, got %d", expectedCount, len(offsets)) |
||||
|
} |
||||
|
|
||||
|
// Verify offsets are in expected range
|
||||
|
for offset := range offsets { |
||||
|
if offset < 0 || offset >= int64(expectedCount) { |
||||
|
t.Errorf("Offset %d is out of expected range [0, %d)", offset, expectedCount) |
||||
|
} |
||||
|
} |
||||
|
} |
||||
@ -0,0 +1,130 @@ |
|||||
|
package offset |
||||
|
|
||||
|
import ( |
||||
|
"fmt" |
||||
|
"sync" |
||||
|
|
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" |
||||
|
) |
||||
|
|
||||
|
// InMemoryOffsetStorage provides an in-memory implementation of OffsetStorage for testing
|
||||
|
type InMemoryOffsetStorage struct { |
||||
|
mu sync.RWMutex |
||||
|
checkpoints map[string]int64 // partition key -> offset
|
||||
|
records map[string]map[int64]bool // partition key -> offset -> exists
|
||||
|
} |
||||
|
|
||||
|
// NewInMemoryOffsetStorage creates a new in-memory storage
|
||||
|
func NewInMemoryOffsetStorage() *InMemoryOffsetStorage { |
||||
|
return &InMemoryOffsetStorage{ |
||||
|
checkpoints: make(map[string]int64), |
||||
|
records: make(map[string]map[int64]bool), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// SaveCheckpoint saves the checkpoint for a partition
|
||||
|
func (s *InMemoryOffsetStorage) SaveCheckpoint(partition *schema_pb.Partition, offset int64) error { |
||||
|
s.mu.Lock() |
||||
|
defer s.mu.Unlock() |
||||
|
|
||||
|
key := partitionKey(partition) |
||||
|
s.checkpoints[key] = offset |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// LoadCheckpoint loads the checkpoint for a partition
|
||||
|
func (s *InMemoryOffsetStorage) LoadCheckpoint(partition *schema_pb.Partition) (int64, error) { |
||||
|
s.mu.RLock() |
||||
|
defer s.mu.RUnlock() |
||||
|
|
||||
|
key := partitionKey(partition) |
||||
|
offset, exists := s.checkpoints[key] |
||||
|
if !exists { |
||||
|
return -1, fmt.Errorf("no checkpoint found") |
||||
|
} |
||||
|
|
||||
|
return offset, nil |
||||
|
} |
||||
|
|
||||
|
// GetHighestOffset finds the highest offset in storage for a partition
|
||||
|
func (s *InMemoryOffsetStorage) GetHighestOffset(partition *schema_pb.Partition) (int64, error) { |
||||
|
s.mu.RLock() |
||||
|
defer s.mu.RUnlock() |
||||
|
|
||||
|
key := partitionKey(partition) |
||||
|
offsets, exists := s.records[key] |
||||
|
if !exists || len(offsets) == 0 { |
||||
|
return -1, fmt.Errorf("no records found") |
||||
|
} |
||||
|
|
||||
|
var highest int64 = -1 |
||||
|
for offset := range offsets { |
||||
|
if offset > highest { |
||||
|
highest = offset |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
return highest, nil |
||||
|
} |
||||
|
|
||||
|
// AddRecord simulates storing a record with an offset (for testing)
|
||||
|
func (s *InMemoryOffsetStorage) AddRecord(partition *schema_pb.Partition, offset int64) { |
||||
|
s.mu.Lock() |
||||
|
defer s.mu.Unlock() |
||||
|
|
||||
|
key := partitionKey(partition) |
||||
|
if s.records[key] == nil { |
||||
|
s.records[key] = make(map[int64]bool) |
||||
|
} |
||||
|
s.records[key][offset] = true |
||||
|
} |
||||
|
|
||||
|
// GetRecordCount returns the number of records for a partition (for testing)
|
||||
|
func (s *InMemoryOffsetStorage) GetRecordCount(partition *schema_pb.Partition) int { |
||||
|
s.mu.RLock() |
||||
|
defer s.mu.RUnlock() |
||||
|
|
||||
|
key := partitionKey(partition) |
||||
|
if offsets, exists := s.records[key]; exists { |
||||
|
return len(offsets) |
||||
|
} |
||||
|
return 0 |
||||
|
} |
||||
|
|
||||
|
// Clear removes all data (for testing)
|
||||
|
func (s *InMemoryOffsetStorage) Clear() { |
||||
|
s.mu.Lock() |
||||
|
defer s.mu.Unlock() |
||||
|
|
||||
|
s.checkpoints = make(map[string]int64) |
||||
|
s.records = make(map[string]map[int64]bool) |
||||
|
} |
||||
|
|
||||
|
// SQLOffsetStorage provides a SQL-based implementation of OffsetStorage
|
||||
|
type SQLOffsetStorage struct { |
||||
|
// TODO: Implement SQL-based storage with _index column
|
||||
|
// This will be implemented in a later phase
|
||||
|
} |
||||
|
|
||||
|
// NewSQLOffsetStorage creates a new SQL-based storage
|
||||
|
func NewSQLOffsetStorage() *SQLOffsetStorage { |
||||
|
return &SQLOffsetStorage{} |
||||
|
} |
||||
|
|
||||
|
// SaveCheckpoint saves the checkpoint for a partition
|
||||
|
func (s *SQLOffsetStorage) SaveCheckpoint(partition *schema_pb.Partition, offset int64) error { |
||||
|
// TODO: Implement SQL checkpoint storage
|
||||
|
return fmt.Errorf("SQL storage not implemented yet") |
||||
|
} |
||||
|
|
||||
|
// LoadCheckpoint loads the checkpoint for a partition
|
||||
|
func (s *SQLOffsetStorage) LoadCheckpoint(partition *schema_pb.Partition) (int64, error) { |
||||
|
// TODO: Implement SQL checkpoint loading
|
||||
|
return -1, fmt.Errorf("SQL storage not implemented yet") |
||||
|
} |
||||
|
|
||||
|
// GetHighestOffset finds the highest offset in storage for a partition
|
||||
|
func (s *SQLOffsetStorage) GetHighestOffset(partition *schema_pb.Partition) (int64, error) { |
||||
|
// TODO: Implement SQL query to find highest _index value
|
||||
|
return -1, fmt.Errorf("SQL storage not implemented yet") |
||||
|
} |
||||
Write
Preview
Loading…
Cancel
Save
Reference in new issue