Browse Source
Phase 5: Implement SQL storage backend for offset persistence
Phase 5: Implement SQL storage backend for offset persistence
- Design comprehensive SQL schema for offset storage with future _index column support - Implement SQLOffsetStorage with full database operations: - Partition offset checkpoints with UPSERT functionality - Detailed offset mappings with range queries and statistics - Database migration system with version tracking - Performance optimizations with proper indexing - Add database migration manager with automatic schema updates - Create comprehensive test suite with 11 test cases covering: - Schema initialization and table creation - Checkpoint save/load operations with error handling - Offset mapping storage and retrieval with sorting - Range queries and highest offset detection - Partition statistics with NULL value handling - Cleanup operations for old data retention - Concurrent access safety and database vacuum - Extend BrokerOffsetManager with SQL storage integration: - NewBrokerOffsetManagerWithSQL for database-backed storage - Configurable storage backends (in-memory fallback, SQL preferred) - Database connection management and error handling - Add SQLite driver dependency and configure for optimal performance - Support for future database types (PostgreSQL, MySQL) with abstraction layer Key TODOs and Assumptions: - TODO: Add _index as computed column when database supports it - TODO: Implement database backup and restore functionality - TODO: Add configuration for database path and connection parameters - ASSUMPTION: Using SQLite for now, extensible to other databases - ASSUMPTION: WAL mode and performance pragmas for production use - ASSUMPTION: Migration system handles schema evolution gracefully All 11 SQL storage tests pass, providing robust persistent offset management.pull/7231/head
7 changed files with 1244 additions and 31 deletions
-
1go.mod
-
2go.sum
-
35weed/mq/broker/broker_offset_manager.go
-
301weed/mq/offset/migration.go
-
391weed/mq/offset/sql_storage.go
-
516weed/mq/offset/sql_storage_test.go
-
29weed/mq/offset/storage.go
@ -0,0 +1,301 @@ |
|||||
|
package offset |
||||
|
|
||||
|
import ( |
||||
|
"database/sql" |
||||
|
"fmt" |
||||
|
"time" |
||||
|
) |
||||
|
|
||||
|
// MigrationVersion represents a database migration version
|
||||
|
type MigrationVersion struct { |
||||
|
Version int |
||||
|
Description string |
||||
|
SQL string |
||||
|
} |
||||
|
|
||||
|
// GetMigrations returns all available migrations for offset storage
|
||||
|
func GetMigrations() []MigrationVersion { |
||||
|
return []MigrationVersion{ |
||||
|
{ |
||||
|
Version: 1, |
||||
|
Description: "Create initial offset storage tables", |
||||
|
SQL: ` |
||||
|
-- Partition offset checkpoints table |
||||
|
-- TODO: Add _index as computed column when supported by database |
||||
|
CREATE TABLE IF NOT EXISTS partition_offset_checkpoints ( |
||||
|
partition_key TEXT PRIMARY KEY, |
||||
|
ring_size INTEGER NOT NULL, |
||||
|
range_start INTEGER NOT NULL, |
||||
|
range_stop INTEGER NOT NULL, |
||||
|
unix_time_ns INTEGER NOT NULL, |
||||
|
checkpoint_offset INTEGER NOT NULL, |
||||
|
updated_at INTEGER NOT NULL |
||||
|
); |
||||
|
|
||||
|
-- Offset mappings table for detailed tracking |
||||
|
-- TODO: Add _index as computed column when supported by database |
||||
|
CREATE TABLE IF NOT EXISTS offset_mappings ( |
||||
|
id INTEGER PRIMARY KEY AUTOINCREMENT, |
||||
|
partition_key TEXT NOT NULL, |
||||
|
kafka_offset INTEGER NOT NULL, |
||||
|
smq_timestamp INTEGER NOT NULL, |
||||
|
message_size INTEGER NOT NULL, |
||||
|
created_at INTEGER NOT NULL, |
||||
|
UNIQUE(partition_key, kafka_offset) |
||||
|
); |
||||
|
|
||||
|
-- Schema migrations tracking table |
||||
|
CREATE TABLE IF NOT EXISTS schema_migrations ( |
||||
|
version INTEGER PRIMARY KEY, |
||||
|
description TEXT NOT NULL, |
||||
|
applied_at INTEGER NOT NULL |
||||
|
); |
||||
|
`, |
||||
|
}, |
||||
|
{ |
||||
|
Version: 2, |
||||
|
Description: "Add indexes for performance optimization", |
||||
|
SQL: ` |
||||
|
-- Indexes for performance |
||||
|
CREATE INDEX IF NOT EXISTS idx_partition_offset_checkpoints_partition |
||||
|
ON partition_offset_checkpoints(partition_key); |
||||
|
|
||||
|
CREATE INDEX IF NOT EXISTS idx_offset_mappings_partition_offset |
||||
|
ON offset_mappings(partition_key, kafka_offset); |
||||
|
|
||||
|
CREATE INDEX IF NOT EXISTS idx_offset_mappings_timestamp |
||||
|
ON offset_mappings(partition_key, smq_timestamp); |
||||
|
|
||||
|
CREATE INDEX IF NOT EXISTS idx_offset_mappings_created_at |
||||
|
ON offset_mappings(created_at); |
||||
|
`, |
||||
|
}, |
||||
|
{ |
||||
|
Version: 3, |
||||
|
Description: "Add partition metadata table for enhanced tracking", |
||||
|
SQL: ` |
||||
|
-- Partition metadata table |
||||
|
CREATE TABLE IF NOT EXISTS partition_metadata ( |
||||
|
partition_key TEXT PRIMARY KEY, |
||||
|
ring_size INTEGER NOT NULL, |
||||
|
range_start INTEGER NOT NULL, |
||||
|
range_stop INTEGER NOT NULL, |
||||
|
unix_time_ns INTEGER NOT NULL, |
||||
|
created_at INTEGER NOT NULL, |
||||
|
last_activity_at INTEGER NOT NULL, |
||||
|
record_count INTEGER DEFAULT 0, |
||||
|
total_size INTEGER DEFAULT 0 |
||||
|
); |
||||
|
|
||||
|
-- Index for partition metadata |
||||
|
CREATE INDEX IF NOT EXISTS idx_partition_metadata_activity |
||||
|
ON partition_metadata(last_activity_at); |
||||
|
`, |
||||
|
}, |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// MigrationManager handles database schema migrations
|
||||
|
type MigrationManager struct { |
||||
|
db *sql.DB |
||||
|
} |
||||
|
|
||||
|
// NewMigrationManager creates a new migration manager
|
||||
|
func NewMigrationManager(db *sql.DB) *MigrationManager { |
||||
|
return &MigrationManager{db: db} |
||||
|
} |
||||
|
|
||||
|
// GetCurrentVersion returns the current schema version
|
||||
|
func (m *MigrationManager) GetCurrentVersion() (int, error) { |
||||
|
// First, ensure the migrations table exists
|
||||
|
_, err := m.db.Exec(` |
||||
|
CREATE TABLE IF NOT EXISTS schema_migrations ( |
||||
|
version INTEGER PRIMARY KEY, |
||||
|
description TEXT NOT NULL, |
||||
|
applied_at INTEGER NOT NULL |
||||
|
) |
||||
|
`) |
||||
|
if err != nil { |
||||
|
return 0, fmt.Errorf("failed to create migrations table: %w", err) |
||||
|
} |
||||
|
|
||||
|
var version int |
||||
|
err = m.db.QueryRow("SELECT MAX(version) FROM schema_migrations").Scan(&version) |
||||
|
if err == sql.ErrNoRows { |
||||
|
return 0, nil // No migrations applied yet
|
||||
|
} |
||||
|
if err != nil { |
||||
|
return 0, fmt.Errorf("failed to get current version: %w", err) |
||||
|
} |
||||
|
|
||||
|
return version, nil |
||||
|
} |
||||
|
|
||||
|
// ApplyMigrations applies all pending migrations
|
||||
|
func (m *MigrationManager) ApplyMigrations() error { |
||||
|
currentVersion, err := m.GetCurrentVersion() |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("failed to get current version: %w", err) |
||||
|
} |
||||
|
|
||||
|
migrations := GetMigrations() |
||||
|
|
||||
|
for _, migration := range migrations { |
||||
|
if migration.Version <= currentVersion { |
||||
|
continue // Already applied
|
||||
|
} |
||||
|
|
||||
|
fmt.Printf("Applying migration %d: %s\n", migration.Version, migration.Description) |
||||
|
|
||||
|
// Begin transaction
|
||||
|
tx, err := m.db.Begin() |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("failed to begin transaction for migration %d: %w", migration.Version, err) |
||||
|
} |
||||
|
|
||||
|
// Execute migration SQL
|
||||
|
_, err = tx.Exec(migration.SQL) |
||||
|
if err != nil { |
||||
|
tx.Rollback() |
||||
|
return fmt.Errorf("failed to execute migration %d: %w", migration.Version, err) |
||||
|
} |
||||
|
|
||||
|
// Record migration as applied
|
||||
|
_, err = tx.Exec( |
||||
|
"INSERT INTO schema_migrations (version, description, applied_at) VALUES (?, ?, ?)", |
||||
|
migration.Version, |
||||
|
migration.Description, |
||||
|
getCurrentTimestamp(), |
||||
|
) |
||||
|
if err != nil { |
||||
|
tx.Rollback() |
||||
|
return fmt.Errorf("failed to record migration %d: %w", migration.Version, err) |
||||
|
} |
||||
|
|
||||
|
// Commit transaction
|
||||
|
err = tx.Commit() |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("failed to commit migration %d: %w", migration.Version, err) |
||||
|
} |
||||
|
|
||||
|
fmt.Printf("Successfully applied migration %d\n", migration.Version) |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// RollbackMigration rolls back a specific migration (if supported)
|
||||
|
func (m *MigrationManager) RollbackMigration(version int) error { |
||||
|
// TODO: Implement rollback functionality
|
||||
|
// ASSUMPTION: For now, rollbacks are not supported as they require careful planning
|
||||
|
return fmt.Errorf("migration rollbacks not implemented - manual intervention required") |
||||
|
} |
||||
|
|
||||
|
// GetAppliedMigrations returns a list of all applied migrations
|
||||
|
func (m *MigrationManager) GetAppliedMigrations() ([]AppliedMigration, error) { |
||||
|
rows, err := m.db.Query(` |
||||
|
SELECT version, description, applied_at |
||||
|
FROM schema_migrations |
||||
|
ORDER BY version |
||||
|
`) |
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to query applied migrations: %w", err) |
||||
|
} |
||||
|
defer rows.Close() |
||||
|
|
||||
|
var migrations []AppliedMigration |
||||
|
for rows.Next() { |
||||
|
var migration AppliedMigration |
||||
|
err := rows.Scan(&migration.Version, &migration.Description, &migration.AppliedAt) |
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to scan migration: %w", err) |
||||
|
} |
||||
|
migrations = append(migrations, migration) |
||||
|
} |
||||
|
|
||||
|
return migrations, nil |
||||
|
} |
||||
|
|
||||
|
// ValidateSchema validates that the database schema is up to date
|
||||
|
func (m *MigrationManager) ValidateSchema() error { |
||||
|
currentVersion, err := m.GetCurrentVersion() |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("failed to get current version: %w", err) |
||||
|
} |
||||
|
|
||||
|
migrations := GetMigrations() |
||||
|
if len(migrations) == 0 { |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
latestVersion := migrations[len(migrations)-1].Version |
||||
|
if currentVersion < latestVersion { |
||||
|
return fmt.Errorf("schema is outdated: current version %d, latest version %d", currentVersion, latestVersion) |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// AppliedMigration represents a migration that has been applied
|
||||
|
type AppliedMigration struct { |
||||
|
Version int |
||||
|
Description string |
||||
|
AppliedAt int64 |
||||
|
} |
||||
|
|
||||
|
// getCurrentTimestamp returns the current timestamp in nanoseconds
|
||||
|
func getCurrentTimestamp() int64 { |
||||
|
return time.Now().UnixNano() |
||||
|
} |
||||
|
|
||||
|
// CreateDatabase creates and initializes a new offset storage database
|
||||
|
func CreateDatabase(dbPath string) (*sql.DB, error) { |
||||
|
// TODO: Support different database types (PostgreSQL, MySQL, etc.)
|
||||
|
// ASSUMPTION: Using SQLite for now, can be extended for other databases
|
||||
|
|
||||
|
db, err := sql.Open("sqlite3", dbPath) |
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to open database: %w", err) |
||||
|
} |
||||
|
|
||||
|
// Configure SQLite for better performance
|
||||
|
pragmas := []string{ |
||||
|
"PRAGMA journal_mode=WAL", // Write-Ahead Logging for better concurrency
|
||||
|
"PRAGMA synchronous=NORMAL", // Balance between safety and performance
|
||||
|
"PRAGMA cache_size=10000", // Increase cache size
|
||||
|
"PRAGMA foreign_keys=ON", // Enable foreign key constraints
|
||||
|
"PRAGMA temp_store=MEMORY", // Store temporary tables in memory
|
||||
|
} |
||||
|
|
||||
|
for _, pragma := range pragmas { |
||||
|
_, err := db.Exec(pragma) |
||||
|
if err != nil { |
||||
|
db.Close() |
||||
|
return nil, fmt.Errorf("failed to set pragma %s: %w", pragma, err) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// Apply migrations
|
||||
|
migrationManager := NewMigrationManager(db) |
||||
|
err = migrationManager.ApplyMigrations() |
||||
|
if err != nil { |
||||
|
db.Close() |
||||
|
return nil, fmt.Errorf("failed to apply migrations: %w", err) |
||||
|
} |
||||
|
|
||||
|
return db, nil |
||||
|
} |
||||
|
|
||||
|
// BackupDatabase creates a backup of the offset storage database
|
||||
|
func BackupDatabase(sourceDB *sql.DB, backupPath string) error { |
||||
|
// TODO: Implement database backup functionality
|
||||
|
// ASSUMPTION: This would use database-specific backup mechanisms
|
||||
|
return fmt.Errorf("database backup not implemented yet") |
||||
|
} |
||||
|
|
||||
|
// RestoreDatabase restores a database from a backup
|
||||
|
func RestoreDatabase(backupPath, targetPath string) error { |
||||
|
// TODO: Implement database restore functionality
|
||||
|
// ASSUMPTION: This would use database-specific restore mechanisms
|
||||
|
return fmt.Errorf("database restore not implemented yet") |
||||
|
} |
||||
@ -0,0 +1,391 @@ |
|||||
|
package offset |
||||
|
|
||||
|
import ( |
||||
|
"database/sql" |
||||
|
"fmt" |
||||
|
"time" |
||||
|
|
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" |
||||
|
) |
||||
|
|
||||
|
// OffsetEntry represents a mapping between Kafka offset and SMQ timestamp
|
||||
|
type OffsetEntry struct { |
||||
|
KafkaOffset int64 |
||||
|
SMQTimestamp int64 |
||||
|
MessageSize int32 |
||||
|
} |
||||
|
|
||||
|
// SQLOffsetStorage implements OffsetStorage using SQL database with _index column
|
||||
|
type SQLOffsetStorage struct { |
||||
|
db *sql.DB |
||||
|
} |
||||
|
|
||||
|
// NewSQLOffsetStorage creates a new SQL-based offset storage
|
||||
|
func NewSQLOffsetStorage(db *sql.DB) (*SQLOffsetStorage, error) { |
||||
|
storage := &SQLOffsetStorage{db: db} |
||||
|
|
||||
|
// Initialize database schema
|
||||
|
if err := storage.initializeSchema(); err != nil { |
||||
|
return nil, fmt.Errorf("failed to initialize schema: %w", err) |
||||
|
} |
||||
|
|
||||
|
return storage, nil |
||||
|
} |
||||
|
|
||||
|
// initializeSchema creates the necessary tables for offset storage
|
||||
|
func (s *SQLOffsetStorage) initializeSchema() error { |
||||
|
// TODO: Create offset storage tables with _index as hidden column
|
||||
|
// ASSUMPTION: Using SQLite-compatible syntax, may need adaptation for other databases
|
||||
|
|
||||
|
queries := []string{ |
||||
|
// Partition offset checkpoints table
|
||||
|
// TODO: Add _index as computed column when supported by database
|
||||
|
// ASSUMPTION: Using regular columns for now, _index concept preserved for future enhancement
|
||||
|
`CREATE TABLE IF NOT EXISTS partition_offset_checkpoints ( |
||||
|
partition_key TEXT PRIMARY KEY, |
||||
|
ring_size INTEGER NOT NULL, |
||||
|
range_start INTEGER NOT NULL, |
||||
|
range_stop INTEGER NOT NULL, |
||||
|
unix_time_ns INTEGER NOT NULL, |
||||
|
checkpoint_offset INTEGER NOT NULL, |
||||
|
updated_at INTEGER NOT NULL |
||||
|
)`, |
||||
|
|
||||
|
// Offset mappings table for detailed tracking
|
||||
|
// TODO: Add _index as computed column when supported by database
|
||||
|
`CREATE TABLE IF NOT EXISTS offset_mappings ( |
||||
|
id INTEGER PRIMARY KEY AUTOINCREMENT, |
||||
|
partition_key TEXT NOT NULL, |
||||
|
kafka_offset INTEGER NOT NULL, |
||||
|
smq_timestamp INTEGER NOT NULL, |
||||
|
message_size INTEGER NOT NULL, |
||||
|
created_at INTEGER NOT NULL, |
||||
|
UNIQUE(partition_key, kafka_offset) |
||||
|
)`, |
||||
|
|
||||
|
// Indexes for performance
|
||||
|
`CREATE INDEX IF NOT EXISTS idx_partition_offset_checkpoints_partition |
||||
|
ON partition_offset_checkpoints(partition_key)`, |
||||
|
|
||||
|
`CREATE INDEX IF NOT EXISTS idx_offset_mappings_partition_offset |
||||
|
ON offset_mappings(partition_key, kafka_offset)`, |
||||
|
|
||||
|
`CREATE INDEX IF NOT EXISTS idx_offset_mappings_timestamp |
||||
|
ON offset_mappings(partition_key, smq_timestamp)`, |
||||
|
} |
||||
|
|
||||
|
for _, query := range queries { |
||||
|
if _, err := s.db.Exec(query); err != nil { |
||||
|
return fmt.Errorf("failed to execute schema query: %w", err) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// SaveCheckpoint saves the checkpoint for a partition
|
||||
|
func (s *SQLOffsetStorage) SaveCheckpoint(partition *schema_pb.Partition, offset int64) error { |
||||
|
partitionKey := partitionKey(partition) |
||||
|
now := time.Now().UnixNano() |
||||
|
|
||||
|
// TODO: Use UPSERT for better performance
|
||||
|
// ASSUMPTION: SQLite REPLACE syntax, may need adaptation for other databases
|
||||
|
query := ` |
||||
|
REPLACE INTO partition_offset_checkpoints |
||||
|
(partition_key, ring_size, range_start, range_stop, unix_time_ns, checkpoint_offset, updated_at) |
||||
|
VALUES (?, ?, ?, ?, ?, ?, ?) |
||||
|
` |
||||
|
|
||||
|
_, err := s.db.Exec(query, |
||||
|
partitionKey, |
||||
|
partition.RingSize, |
||||
|
partition.RangeStart, |
||||
|
partition.RangeStop, |
||||
|
partition.UnixTimeNs, |
||||
|
offset, |
||||
|
now, |
||||
|
) |
||||
|
|
||||
|
if err != nil { |
||||
|
return fmt.Errorf("failed to save checkpoint: %w", err) |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// LoadCheckpoint loads the checkpoint for a partition
|
||||
|
func (s *SQLOffsetStorage) LoadCheckpoint(partition *schema_pb.Partition) (int64, error) { |
||||
|
partitionKey := partitionKey(partition) |
||||
|
|
||||
|
query := ` |
||||
|
SELECT checkpoint_offset |
||||
|
FROM partition_offset_checkpoints |
||||
|
WHERE partition_key = ? |
||||
|
` |
||||
|
|
||||
|
var checkpointOffset int64 |
||||
|
err := s.db.QueryRow(query, partitionKey).Scan(&checkpointOffset) |
||||
|
|
||||
|
if err == sql.ErrNoRows { |
||||
|
return -1, fmt.Errorf("no checkpoint found") |
||||
|
} |
||||
|
|
||||
|
if err != nil { |
||||
|
return -1, fmt.Errorf("failed to load checkpoint: %w", err) |
||||
|
} |
||||
|
|
||||
|
return checkpointOffset, nil |
||||
|
} |
||||
|
|
||||
|
// GetHighestOffset finds the highest offset in storage for a partition
|
||||
|
func (s *SQLOffsetStorage) GetHighestOffset(partition *schema_pb.Partition) (int64, error) { |
||||
|
partitionKey := partitionKey(partition) |
||||
|
|
||||
|
// TODO: Use _index column for efficient querying
|
||||
|
// ASSUMPTION: kafka_offset represents the sequential offset we're tracking
|
||||
|
query := ` |
||||
|
SELECT MAX(kafka_offset) |
||||
|
FROM offset_mappings |
||||
|
WHERE partition_key = ? |
||||
|
` |
||||
|
|
||||
|
var highestOffset sql.NullInt64 |
||||
|
err := s.db.QueryRow(query, partitionKey).Scan(&highestOffset) |
||||
|
|
||||
|
if err != nil { |
||||
|
return -1, fmt.Errorf("failed to get highest offset: %w", err) |
||||
|
} |
||||
|
|
||||
|
if !highestOffset.Valid { |
||||
|
return -1, fmt.Errorf("no records found") |
||||
|
} |
||||
|
|
||||
|
return highestOffset.Int64, nil |
||||
|
} |
||||
|
|
||||
|
// SaveOffsetMapping stores an offset mapping (extends OffsetStorage interface)
|
||||
|
func (s *SQLOffsetStorage) SaveOffsetMapping(partitionKey string, kafkaOffset, smqTimestamp int64, size int32) error { |
||||
|
now := time.Now().UnixNano() |
||||
|
|
||||
|
// TODO: Handle duplicate key conflicts gracefully
|
||||
|
// ASSUMPTION: Using INSERT OR REPLACE for conflict resolution
|
||||
|
query := ` |
||||
|
INSERT OR REPLACE INTO offset_mappings |
||||
|
(partition_key, kafka_offset, smq_timestamp, message_size, created_at) |
||||
|
VALUES (?, ?, ?, ?, ?) |
||||
|
` |
||||
|
|
||||
|
_, err := s.db.Exec(query, partitionKey, kafkaOffset, smqTimestamp, size, now) |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("failed to save offset mapping: %w", err) |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// LoadOffsetMappings retrieves all offset mappings for a partition
|
||||
|
func (s *SQLOffsetStorage) LoadOffsetMappings(partitionKey string) ([]OffsetEntry, error) { |
||||
|
// TODO: Add pagination for large result sets
|
||||
|
// ASSUMPTION: Loading all mappings for now, should be paginated in production
|
||||
|
query := ` |
||||
|
SELECT kafka_offset, smq_timestamp, message_size |
||||
|
FROM offset_mappings |
||||
|
WHERE partition_key = ? |
||||
|
ORDER BY kafka_offset ASC |
||||
|
` |
||||
|
|
||||
|
rows, err := s.db.Query(query, partitionKey) |
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to query offset mappings: %w", err) |
||||
|
} |
||||
|
defer rows.Close() |
||||
|
|
||||
|
var entries []OffsetEntry |
||||
|
for rows.Next() { |
||||
|
var entry OffsetEntry |
||||
|
err := rows.Scan(&entry.KafkaOffset, &entry.SMQTimestamp, &entry.MessageSize) |
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to scan offset entry: %w", err) |
||||
|
} |
||||
|
entries = append(entries, entry) |
||||
|
} |
||||
|
|
||||
|
if err := rows.Err(); err != nil { |
||||
|
return nil, fmt.Errorf("error iterating offset mappings: %w", err) |
||||
|
} |
||||
|
|
||||
|
return entries, nil |
||||
|
} |
||||
|
|
||||
|
// GetOffsetMappingsByRange retrieves offset mappings within a specific range
|
||||
|
func (s *SQLOffsetStorage) GetOffsetMappingsByRange(partitionKey string, startOffset, endOffset int64) ([]OffsetEntry, error) { |
||||
|
// TODO: Use _index column for efficient range queries
|
||||
|
query := ` |
||||
|
SELECT kafka_offset, smq_timestamp, message_size |
||||
|
FROM offset_mappings |
||||
|
WHERE partition_key = ? AND kafka_offset >= ? AND kafka_offset <= ? |
||||
|
ORDER BY kafka_offset ASC |
||||
|
` |
||||
|
|
||||
|
rows, err := s.db.Query(query, partitionKey, startOffset, endOffset) |
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to query offset range: %w", err) |
||||
|
} |
||||
|
defer rows.Close() |
||||
|
|
||||
|
var entries []OffsetEntry |
||||
|
for rows.Next() { |
||||
|
var entry OffsetEntry |
||||
|
err := rows.Scan(&entry.KafkaOffset, &entry.SMQTimestamp, &entry.MessageSize) |
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to scan offset entry: %w", err) |
||||
|
} |
||||
|
entries = append(entries, entry) |
||||
|
} |
||||
|
|
||||
|
return entries, nil |
||||
|
} |
||||
|
|
||||
|
// GetPartitionStats returns statistics about a partition's offset usage
|
||||
|
func (s *SQLOffsetStorage) GetPartitionStats(partitionKey string) (*PartitionStats, error) { |
||||
|
query := ` |
||||
|
SELECT |
||||
|
COUNT(*) as record_count, |
||||
|
MIN(kafka_offset) as earliest_offset, |
||||
|
MAX(kafka_offset) as latest_offset, |
||||
|
SUM(message_size) as total_size, |
||||
|
MIN(created_at) as first_record_time, |
||||
|
MAX(created_at) as last_record_time |
||||
|
FROM offset_mappings |
||||
|
WHERE partition_key = ? |
||||
|
` |
||||
|
|
||||
|
var stats PartitionStats |
||||
|
var earliestOffset, latestOffset sql.NullInt64 |
||||
|
var totalSize sql.NullInt64 |
||||
|
var firstRecordTime, lastRecordTime sql.NullInt64 |
||||
|
|
||||
|
err := s.db.QueryRow(query, partitionKey).Scan( |
||||
|
&stats.RecordCount, |
||||
|
&earliestOffset, |
||||
|
&latestOffset, |
||||
|
&totalSize, |
||||
|
&firstRecordTime, |
||||
|
&lastRecordTime, |
||||
|
) |
||||
|
|
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to get partition stats: %w", err) |
||||
|
} |
||||
|
|
||||
|
stats.PartitionKey = partitionKey |
||||
|
|
||||
|
if earliestOffset.Valid { |
||||
|
stats.EarliestOffset = earliestOffset.Int64 |
||||
|
} else { |
||||
|
stats.EarliestOffset = -1 |
||||
|
} |
||||
|
|
||||
|
if latestOffset.Valid { |
||||
|
stats.LatestOffset = latestOffset.Int64 |
||||
|
stats.HighWaterMark = latestOffset.Int64 + 1 |
||||
|
} else { |
||||
|
stats.LatestOffset = -1 |
||||
|
stats.HighWaterMark = 0 |
||||
|
} |
||||
|
|
||||
|
if firstRecordTime.Valid { |
||||
|
stats.FirstRecordTime = firstRecordTime.Int64 |
||||
|
} |
||||
|
|
||||
|
if lastRecordTime.Valid { |
||||
|
stats.LastRecordTime = lastRecordTime.Int64 |
||||
|
} |
||||
|
|
||||
|
if totalSize.Valid { |
||||
|
stats.TotalSize = totalSize.Int64 |
||||
|
} |
||||
|
|
||||
|
return &stats, nil |
||||
|
} |
||||
|
|
||||
|
// CleanupOldMappings removes offset mappings older than the specified time
|
||||
|
func (s *SQLOffsetStorage) CleanupOldMappings(olderThanNs int64) error { |
||||
|
// TODO: Add configurable cleanup policies
|
||||
|
// ASSUMPTION: Simple time-based cleanup, could be enhanced with retention policies
|
||||
|
query := ` |
||||
|
DELETE FROM offset_mappings |
||||
|
WHERE created_at < ? |
||||
|
` |
||||
|
|
||||
|
result, err := s.db.Exec(query, olderThanNs) |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("failed to cleanup old mappings: %w", err) |
||||
|
} |
||||
|
|
||||
|
rowsAffected, _ := result.RowsAffected() |
||||
|
if rowsAffected > 0 { |
||||
|
// Log cleanup activity
|
||||
|
fmt.Printf("Cleaned up %d old offset mappings\n", rowsAffected) |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// Close closes the database connection
|
||||
|
func (s *SQLOffsetStorage) Close() error { |
||||
|
if s.db != nil { |
||||
|
return s.db.Close() |
||||
|
} |
||||
|
return nil |
||||
|
} |
||||
|
|
||||
|
// PartitionStats provides statistics about a partition's offset usage
|
||||
|
type PartitionStats struct { |
||||
|
PartitionKey string |
||||
|
RecordCount int64 |
||||
|
EarliestOffset int64 |
||||
|
LatestOffset int64 |
||||
|
HighWaterMark int64 |
||||
|
TotalSize int64 |
||||
|
FirstRecordTime int64 |
||||
|
LastRecordTime int64 |
||||
|
} |
||||
|
|
||||
|
// GetAllPartitions returns a list of all partitions with offset data
|
||||
|
func (s *SQLOffsetStorage) GetAllPartitions() ([]string, error) { |
||||
|
query := ` |
||||
|
SELECT DISTINCT partition_key |
||||
|
FROM offset_mappings |
||||
|
ORDER BY partition_key |
||||
|
` |
||||
|
|
||||
|
rows, err := s.db.Query(query) |
||||
|
if err != nil { |
||||
|
return nil, fmt.Errorf("failed to get all partitions: %w", err) |
||||
|
} |
||||
|
defer rows.Close() |
||||
|
|
||||
|
var partitions []string |
||||
|
for rows.Next() { |
||||
|
var partitionKey string |
||||
|
if err := rows.Scan(&partitionKey); err != nil { |
||||
|
return nil, fmt.Errorf("failed to scan partition key: %w", err) |
||||
|
} |
||||
|
partitions = append(partitions, partitionKey) |
||||
|
} |
||||
|
|
||||
|
return partitions, nil |
||||
|
} |
||||
|
|
||||
|
// Vacuum performs database maintenance operations
|
||||
|
func (s *SQLOffsetStorage) Vacuum() error { |
||||
|
// TODO: Add database-specific optimization commands
|
||||
|
// ASSUMPTION: SQLite VACUUM command, may need adaptation for other databases
|
||||
|
_, err := s.db.Exec("VACUUM") |
||||
|
if err != nil { |
||||
|
return fmt.Errorf("failed to vacuum database: %w", err) |
||||
|
} |
||||
|
|
||||
|
return nil |
||||
|
} |
||||
@ -0,0 +1,516 @@ |
|||||
|
package offset |
||||
|
|
||||
|
import ( |
||||
|
"database/sql" |
||||
|
"os" |
||||
|
"testing" |
||||
|
"time" |
||||
|
|
||||
|
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb" |
||||
|
_ "github.com/mattn/go-sqlite3" // SQLite driver
|
||||
|
) |
||||
|
|
||||
|
func createTestDB(t *testing.T) *sql.DB { |
||||
|
// Create temporary database file
|
||||
|
tmpFile, err := os.CreateTemp("", "offset_test_*.db") |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create temp database file: %v", err) |
||||
|
} |
||||
|
tmpFile.Close() |
||||
|
|
||||
|
// Clean up the file when test completes
|
||||
|
t.Cleanup(func() { |
||||
|
os.Remove(tmpFile.Name()) |
||||
|
}) |
||||
|
|
||||
|
db, err := sql.Open("sqlite3", tmpFile.Name()) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to open database: %v", err) |
||||
|
} |
||||
|
|
||||
|
t.Cleanup(func() { |
||||
|
db.Close() |
||||
|
}) |
||||
|
|
||||
|
return db |
||||
|
} |
||||
|
|
||||
|
func createTestPartitionForSQL() *schema_pb.Partition { |
||||
|
return &schema_pb.Partition{ |
||||
|
RingSize: 1024, |
||||
|
RangeStart: 0, |
||||
|
RangeStop: 31, |
||||
|
UnixTimeNs: time.Now().UnixNano(), |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_InitializeSchema(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
|
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
// Verify tables were created
|
||||
|
tables := []string{ |
||||
|
"partition_offset_checkpoints", |
||||
|
"offset_mappings", |
||||
|
} |
||||
|
|
||||
|
for _, table := range tables { |
||||
|
var count int |
||||
|
err := db.QueryRow("SELECT COUNT(*) FROM sqlite_master WHERE type='table' AND name=?", table).Scan(&count) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to check table %s: %v", table, err) |
||||
|
} |
||||
|
|
||||
|
if count != 1 { |
||||
|
t.Errorf("Table %s was not created", table) |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_SaveLoadCheckpoint(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
|
||||
|
// Test saving checkpoint
|
||||
|
err = storage.SaveCheckpoint(partition, 100) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to save checkpoint: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Test loading checkpoint
|
||||
|
checkpoint, err := storage.LoadCheckpoint(partition) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to load checkpoint: %v", err) |
||||
|
} |
||||
|
|
||||
|
if checkpoint != 100 { |
||||
|
t.Errorf("Expected checkpoint 100, got %d", checkpoint) |
||||
|
} |
||||
|
|
||||
|
// Test updating checkpoint
|
||||
|
err = storage.SaveCheckpoint(partition, 200) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to update checkpoint: %v", err) |
||||
|
} |
||||
|
|
||||
|
checkpoint, err = storage.LoadCheckpoint(partition) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to load updated checkpoint: %v", err) |
||||
|
} |
||||
|
|
||||
|
if checkpoint != 200 { |
||||
|
t.Errorf("Expected updated checkpoint 200, got %d", checkpoint) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_LoadCheckpointNotFound(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
|
||||
|
// Test loading non-existent checkpoint
|
||||
|
_, err = storage.LoadCheckpoint(partition) |
||||
|
if err == nil { |
||||
|
t.Error("Expected error for non-existent checkpoint") |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_SaveLoadOffsetMappings(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
partitionKey := partitionKey(partition) |
||||
|
|
||||
|
// Save multiple offset mappings
|
||||
|
mappings := []struct { |
||||
|
offset int64 |
||||
|
timestamp int64 |
||||
|
size int32 |
||||
|
}{ |
||||
|
{0, 1000, 100}, |
||||
|
{1, 2000, 150}, |
||||
|
{2, 3000, 200}, |
||||
|
} |
||||
|
|
||||
|
for _, mapping := range mappings { |
||||
|
err := storage.SaveOffsetMapping(partitionKey, mapping.offset, mapping.timestamp, mapping.size) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to save offset mapping: %v", err) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// Load offset mappings
|
||||
|
entries, err := storage.LoadOffsetMappings(partitionKey) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to load offset mappings: %v", err) |
||||
|
} |
||||
|
|
||||
|
if len(entries) != len(mappings) { |
||||
|
t.Errorf("Expected %d entries, got %d", len(mappings), len(entries)) |
||||
|
} |
||||
|
|
||||
|
// Verify entries are sorted by offset
|
||||
|
for i, entry := range entries { |
||||
|
expected := mappings[i] |
||||
|
if entry.KafkaOffset != expected.offset { |
||||
|
t.Errorf("Entry %d: expected offset %d, got %d", i, expected.offset, entry.KafkaOffset) |
||||
|
} |
||||
|
if entry.SMQTimestamp != expected.timestamp { |
||||
|
t.Errorf("Entry %d: expected timestamp %d, got %d", i, expected.timestamp, entry.SMQTimestamp) |
||||
|
} |
||||
|
if entry.MessageSize != expected.size { |
||||
|
t.Errorf("Entry %d: expected size %d, got %d", i, expected.size, entry.MessageSize) |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_GetHighestOffset(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
partitionKey := partitionKey(partition) |
||||
|
|
||||
|
// Test empty partition
|
||||
|
_, err = storage.GetHighestOffset(partition) |
||||
|
if err == nil { |
||||
|
t.Error("Expected error for empty partition") |
||||
|
} |
||||
|
|
||||
|
// Add some offset mappings
|
||||
|
offsets := []int64{5, 1, 3, 2, 4} |
||||
|
for _, offset := range offsets { |
||||
|
err := storage.SaveOffsetMapping(partitionKey, offset, offset*1000, 100) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to save offset mapping: %v", err) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// Get highest offset
|
||||
|
highest, err := storage.GetHighestOffset(partition) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get highest offset: %v", err) |
||||
|
} |
||||
|
|
||||
|
if highest != 5 { |
||||
|
t.Errorf("Expected highest offset 5, got %d", highest) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_GetOffsetMappingsByRange(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
partitionKey := partitionKey(partition) |
||||
|
|
||||
|
// Add offset mappings
|
||||
|
for i := int64(0); i < 10; i++ { |
||||
|
err := storage.SaveOffsetMapping(partitionKey, i, i*1000, 100) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to save offset mapping: %v", err) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// Get range of offsets
|
||||
|
entries, err := storage.GetOffsetMappingsByRange(partitionKey, 3, 7) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get offset range: %v", err) |
||||
|
} |
||||
|
|
||||
|
expectedCount := 5 // offsets 3, 4, 5, 6, 7
|
||||
|
if len(entries) != expectedCount { |
||||
|
t.Errorf("Expected %d entries, got %d", expectedCount, len(entries)) |
||||
|
} |
||||
|
|
||||
|
// Verify range
|
||||
|
for i, entry := range entries { |
||||
|
expectedOffset := int64(3 + i) |
||||
|
if entry.KafkaOffset != expectedOffset { |
||||
|
t.Errorf("Entry %d: expected offset %d, got %d", i, expectedOffset, entry.KafkaOffset) |
||||
|
} |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_GetPartitionStats(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
partitionKey := partitionKey(partition) |
||||
|
|
||||
|
// Test empty partition stats
|
||||
|
stats, err := storage.GetPartitionStats(partitionKey) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get empty partition stats: %v", err) |
||||
|
} |
||||
|
|
||||
|
if stats.RecordCount != 0 { |
||||
|
t.Errorf("Expected record count 0, got %d", stats.RecordCount) |
||||
|
} |
||||
|
|
||||
|
if stats.EarliestOffset != -1 { |
||||
|
t.Errorf("Expected earliest offset -1, got %d", stats.EarliestOffset) |
||||
|
} |
||||
|
|
||||
|
// Add some data
|
||||
|
sizes := []int32{100, 150, 200} |
||||
|
for i, size := range sizes { |
||||
|
err := storage.SaveOffsetMapping(partitionKey, int64(i), int64(i*1000), size) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to save offset mapping: %v", err) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
// Get stats with data
|
||||
|
stats, err = storage.GetPartitionStats(partitionKey) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get partition stats: %v", err) |
||||
|
} |
||||
|
|
||||
|
if stats.RecordCount != 3 { |
||||
|
t.Errorf("Expected record count 3, got %d", stats.RecordCount) |
||||
|
} |
||||
|
|
||||
|
if stats.EarliestOffset != 0 { |
||||
|
t.Errorf("Expected earliest offset 0, got %d", stats.EarliestOffset) |
||||
|
} |
||||
|
|
||||
|
if stats.LatestOffset != 2 { |
||||
|
t.Errorf("Expected latest offset 2, got %d", stats.LatestOffset) |
||||
|
} |
||||
|
|
||||
|
if stats.HighWaterMark != 3 { |
||||
|
t.Errorf("Expected high water mark 3, got %d", stats.HighWaterMark) |
||||
|
} |
||||
|
|
||||
|
expectedTotalSize := int64(100 + 150 + 200) |
||||
|
if stats.TotalSize != expectedTotalSize { |
||||
|
t.Errorf("Expected total size %d, got %d", expectedTotalSize, stats.TotalSize) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_GetAllPartitions(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
// Test empty database
|
||||
|
partitions, err := storage.GetAllPartitions() |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get all partitions: %v", err) |
||||
|
} |
||||
|
|
||||
|
if len(partitions) != 0 { |
||||
|
t.Errorf("Expected 0 partitions, got %d", len(partitions)) |
||||
|
} |
||||
|
|
||||
|
// Add data for multiple partitions
|
||||
|
partition1 := createTestPartitionForSQL() |
||||
|
partition2 := &schema_pb.Partition{ |
||||
|
RingSize: 1024, |
||||
|
RangeStart: 32, |
||||
|
RangeStop: 63, |
||||
|
UnixTimeNs: time.Now().UnixNano(), |
||||
|
} |
||||
|
|
||||
|
partitionKey1 := partitionKey(partition1) |
||||
|
partitionKey2 := partitionKey(partition2) |
||||
|
|
||||
|
storage.SaveOffsetMapping(partitionKey1, 0, 1000, 100) |
||||
|
storage.SaveOffsetMapping(partitionKey2, 0, 2000, 150) |
||||
|
|
||||
|
// Get all partitions
|
||||
|
partitions, err = storage.GetAllPartitions() |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to get all partitions: %v", err) |
||||
|
} |
||||
|
|
||||
|
if len(partitions) != 2 { |
||||
|
t.Errorf("Expected 2 partitions, got %d", len(partitions)) |
||||
|
} |
||||
|
|
||||
|
// Verify partition keys are present
|
||||
|
partitionMap := make(map[string]bool) |
||||
|
for _, p := range partitions { |
||||
|
partitionMap[p] = true |
||||
|
} |
||||
|
|
||||
|
if !partitionMap[partitionKey1] { |
||||
|
t.Errorf("Partition key %s not found", partitionKey1) |
||||
|
} |
||||
|
|
||||
|
if !partitionMap[partitionKey2] { |
||||
|
t.Errorf("Partition key %s not found", partitionKey2) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_CleanupOldMappings(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
partitionKey := partitionKey(partition) |
||||
|
|
||||
|
// Add mappings with different timestamps
|
||||
|
now := time.Now().UnixNano() |
||||
|
|
||||
|
// Add old mapping by directly inserting with old timestamp
|
||||
|
oldTime := now - (24 * time.Hour).Nanoseconds() // 24 hours ago
|
||||
|
_, err = db.Exec(` |
||||
|
INSERT INTO offset_mappings |
||||
|
(partition_key, kafka_offset, smq_timestamp, message_size, created_at) |
||||
|
VALUES (?, ?, ?, ?, ?) |
||||
|
`, partitionKey, 0, oldTime, 100, oldTime) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to insert old mapping: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Add recent mapping
|
||||
|
storage.SaveOffsetMapping(partitionKey, 1, now, 150) |
||||
|
|
||||
|
// Verify both mappings exist
|
||||
|
entries, err := storage.LoadOffsetMappings(partitionKey) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to load mappings: %v", err) |
||||
|
} |
||||
|
|
||||
|
if len(entries) != 2 { |
||||
|
t.Errorf("Expected 2 mappings before cleanup, got %d", len(entries)) |
||||
|
} |
||||
|
|
||||
|
// Cleanup old mappings (older than 12 hours)
|
||||
|
cutoffTime := now - (12 * time.Hour).Nanoseconds() |
||||
|
err = storage.CleanupOldMappings(cutoffTime) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to cleanup old mappings: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Verify only recent mapping remains
|
||||
|
entries, err = storage.LoadOffsetMappings(partitionKey) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to load mappings after cleanup: %v", err) |
||||
|
} |
||||
|
|
||||
|
if len(entries) != 1 { |
||||
|
t.Errorf("Expected 1 mapping after cleanup, got %d", len(entries)) |
||||
|
} |
||||
|
|
||||
|
if entries[0].KafkaOffset != 1 { |
||||
|
t.Errorf("Expected remaining mapping offset 1, got %d", entries[0].KafkaOffset) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_Vacuum(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
// Vacuum should not fail on empty database
|
||||
|
err = storage.Vacuum() |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to vacuum database: %v", err) |
||||
|
} |
||||
|
|
||||
|
// Add some data and vacuum again
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
partitionKey := partitionKey(partition) |
||||
|
storage.SaveOffsetMapping(partitionKey, 0, 1000, 100) |
||||
|
|
||||
|
err = storage.Vacuum() |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to vacuum database with data: %v", err) |
||||
|
} |
||||
|
} |
||||
|
|
||||
|
func TestSQLOffsetStorage_ConcurrentAccess(t *testing.T) { |
||||
|
db := createTestDB(t) |
||||
|
storage, err := NewSQLOffsetStorage(db) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to create SQL storage: %v", err) |
||||
|
} |
||||
|
defer storage.Close() |
||||
|
|
||||
|
partition := createTestPartitionForSQL() |
||||
|
partitionKey := partitionKey(partition) |
||||
|
|
||||
|
// Test concurrent writes
|
||||
|
const numGoroutines = 10 |
||||
|
const offsetsPerGoroutine = 10 |
||||
|
|
||||
|
done := make(chan bool, numGoroutines) |
||||
|
|
||||
|
for i := 0; i < numGoroutines; i++ { |
||||
|
go func(goroutineID int) { |
||||
|
defer func() { done <- true }() |
||||
|
|
||||
|
for j := 0; j < offsetsPerGoroutine; j++ { |
||||
|
offset := int64(goroutineID*offsetsPerGoroutine + j) |
||||
|
err := storage.SaveOffsetMapping(partitionKey, offset, offset*1000, 100) |
||||
|
if err != nil { |
||||
|
t.Errorf("Failed to save offset mapping %d: %v", offset, err) |
||||
|
return |
||||
|
} |
||||
|
} |
||||
|
}(i) |
||||
|
} |
||||
|
|
||||
|
// Wait for all goroutines to complete
|
||||
|
for i := 0; i < numGoroutines; i++ { |
||||
|
<-done |
||||
|
} |
||||
|
|
||||
|
// Verify all mappings were saved
|
||||
|
entries, err := storage.LoadOffsetMappings(partitionKey) |
||||
|
if err != nil { |
||||
|
t.Fatalf("Failed to load mappings: %v", err) |
||||
|
} |
||||
|
|
||||
|
expectedCount := numGoroutines * offsetsPerGoroutine |
||||
|
if len(entries) != expectedCount { |
||||
|
t.Errorf("Expected %d mappings, got %d", expectedCount, len(entries)) |
||||
|
} |
||||
|
} |
||||
Write
Preview
Loading…
Cancel
Save
Reference in new issue