Initial commit: SRDB - High-performance LSM-Tree database
- Core engine with MemTable, SST, WAL - B+Tree indexing for SST files - Leveled compaction strategy - Multi-table database management - Schema validation and secondary indexes - Query builder with complex conditions - Web UI with HTMX for data visualization - Command-line tools for diagnostics
This commit is contained in:
392
compaction/compaction_test.go
Normal file
392
compaction/compaction_test.go
Normal file
@@ -0,0 +1,392 @@
|
||||
package compaction
|
||||
|
||||
import (
|
||||
"code.tczkiot.com/srdb/manifest"
|
||||
"code.tczkiot.com/srdb/sst"
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestCompactionBasic(t *testing.T) {
|
||||
// 创建临时目录
|
||||
tmpDir := t.TempDir()
|
||||
sstDir := filepath.Join(tmpDir, "sst")
|
||||
manifestDir := tmpDir
|
||||
|
||||
err := os.MkdirAll(sstDir, 0755)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// 创建 VersionSet
|
||||
versionSet, err := manifest.NewVersionSet(manifestDir)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer versionSet.Close()
|
||||
|
||||
// 创建 SST Manager
|
||||
sstMgr, err := sst.NewManager(sstDir)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer sstMgr.Close()
|
||||
|
||||
// 创建测试数据
|
||||
rows1 := make([]*sst.Row, 100)
|
||||
for i := 0; i < 100; i++ {
|
||||
rows1[i] = &sst.Row{
|
||||
Seq: int64(i),
|
||||
Time: 1000,
|
||||
Data: map[string]interface{}{"value": i},
|
||||
}
|
||||
}
|
||||
|
||||
// 创建第一个 SST 文件
|
||||
reader1, err := sstMgr.CreateSST(1, rows1)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// 添加到 Version
|
||||
edit1 := manifest.NewVersionEdit()
|
||||
edit1.AddFile(&manifest.FileMetadata{
|
||||
FileNumber: 1,
|
||||
Level: 0,
|
||||
FileSize: 1024,
|
||||
MinKey: 0,
|
||||
MaxKey: 99,
|
||||
RowCount: 100,
|
||||
})
|
||||
nextFileNum := int64(2)
|
||||
edit1.SetNextFileNumber(nextFileNum)
|
||||
|
||||
err = versionSet.LogAndApply(edit1)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// 验证 Version
|
||||
version := versionSet.GetCurrent()
|
||||
if version.GetLevelFileCount(0) != 1 {
|
||||
t.Errorf("Expected 1 file in L0, got %d", version.GetLevelFileCount(0))
|
||||
}
|
||||
|
||||
// 创建 Compaction Manager
|
||||
compactionMgr := NewManager(sstDir, versionSet)
|
||||
|
||||
// 创建更多文件触发 Compaction
|
||||
for i := 1; i < 5; i++ {
|
||||
rows := make([]*sst.Row, 50)
|
||||
for j := 0; j < 50; j++ {
|
||||
rows[j] = &sst.Row{
|
||||
Seq: int64(i*100 + j),
|
||||
Time: int64(1000 + i),
|
||||
Data: map[string]interface{}{"value": i*100 + j},
|
||||
}
|
||||
}
|
||||
|
||||
_, err := sstMgr.CreateSST(int64(i+1), rows)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
edit := manifest.NewVersionEdit()
|
||||
edit.AddFile(&manifest.FileMetadata{
|
||||
FileNumber: int64(i + 1),
|
||||
Level: 0,
|
||||
FileSize: 512,
|
||||
MinKey: int64(i * 100),
|
||||
MaxKey: int64(i*100 + 49),
|
||||
RowCount: 50,
|
||||
})
|
||||
nextFileNum := int64(i + 2)
|
||||
edit.SetNextFileNumber(nextFileNum)
|
||||
|
||||
err = versionSet.LogAndApply(edit)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
// 验证 L0 有 5 个文件
|
||||
version = versionSet.GetCurrent()
|
||||
if version.GetLevelFileCount(0) != 5 {
|
||||
t.Errorf("Expected 5 files in L0, got %d", version.GetLevelFileCount(0))
|
||||
}
|
||||
|
||||
// 检查是否需要 Compaction
|
||||
picker := compactionMgr.GetPicker()
|
||||
if !picker.ShouldCompact(version) {
|
||||
t.Error("Expected compaction to be needed")
|
||||
}
|
||||
|
||||
// 获取 Compaction 任务
|
||||
tasks := picker.PickCompaction(version)
|
||||
if len(tasks) == 0 {
|
||||
t.Fatal("Expected compaction task")
|
||||
}
|
||||
|
||||
task := tasks[0] // 获取第一个任务(优先级最高)
|
||||
|
||||
if task.Level != 0 {
|
||||
t.Errorf("Expected L0 compaction, got L%d", task.Level)
|
||||
}
|
||||
|
||||
if task.OutputLevel != 1 {
|
||||
t.Errorf("Expected output to L1, got L%d", task.OutputLevel)
|
||||
}
|
||||
|
||||
t.Logf("Found %d compaction tasks", len(tasks))
|
||||
t.Logf("First task: L%d -> L%d, %d files", task.Level, task.OutputLevel, len(task.InputFiles))
|
||||
|
||||
// 清理
|
||||
reader1.Close()
|
||||
}
|
||||
|
||||
func TestPickerLevelScore(t *testing.T) {
|
||||
// 创建临时目录
|
||||
tmpDir := t.TempDir()
|
||||
manifestDir := tmpDir
|
||||
|
||||
// 创建 VersionSet
|
||||
versionSet, err := manifest.NewVersionSet(manifestDir)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer versionSet.Close()
|
||||
|
||||
// 创建 Picker
|
||||
picker := NewPicker()
|
||||
|
||||
// 添加一些文件到 L0
|
||||
edit := manifest.NewVersionEdit()
|
||||
for i := 0; i < 3; i++ {
|
||||
edit.AddFile(&manifest.FileMetadata{
|
||||
FileNumber: int64(i + 1),
|
||||
Level: 0,
|
||||
FileSize: 1024 * 1024, // 1MB
|
||||
MinKey: int64(i * 100),
|
||||
MaxKey: int64((i+1)*100 - 1),
|
||||
RowCount: 100,
|
||||
})
|
||||
}
|
||||
nextFileNum := int64(4)
|
||||
edit.SetNextFileNumber(nextFileNum)
|
||||
|
||||
err = versionSet.LogAndApply(edit)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
version := versionSet.GetCurrent()
|
||||
|
||||
// 计算 L0 的得分
|
||||
score := picker.GetLevelScore(version, 0)
|
||||
t.Logf("L0 score: %.2f (files: %d, limit: %d)", score, version.GetLevelFileCount(0), picker.levelFileLimits[0])
|
||||
|
||||
// L0 有 3 个文件,限制是 4,得分应该是 0.75
|
||||
expectedScore := 3.0 / 4.0
|
||||
if score != expectedScore {
|
||||
t.Errorf("Expected L0 score %.2f, got %.2f", expectedScore, score)
|
||||
}
|
||||
}
|
||||
|
||||
func TestCompactionMerge(t *testing.T) {
|
||||
// 创建临时目录
|
||||
tmpDir := t.TempDir()
|
||||
sstDir := filepath.Join(tmpDir, "sst")
|
||||
manifestDir := tmpDir
|
||||
|
||||
err := os.MkdirAll(sstDir, 0755)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// 创建 VersionSet
|
||||
versionSet, err := manifest.NewVersionSet(manifestDir)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer versionSet.Close()
|
||||
|
||||
// 创建 SST Manager
|
||||
sstMgr, err := sst.NewManager(sstDir)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer sstMgr.Close()
|
||||
|
||||
// 创建两个有重叠 key 的 SST 文件
|
||||
rows1 := []*sst.Row{
|
||||
{Seq: 1, Time: 1000, Data: map[string]interface{}{"value": "old"}},
|
||||
{Seq: 2, Time: 1000, Data: map[string]interface{}{"value": "old"}},
|
||||
}
|
||||
|
||||
rows2 := []*sst.Row{
|
||||
{Seq: 1, Time: 2000, Data: map[string]interface{}{"value": "new"}}, // 更新
|
||||
{Seq: 3, Time: 2000, Data: map[string]interface{}{"value": "new"}},
|
||||
}
|
||||
|
||||
reader1, err := sstMgr.CreateSST(1, rows1)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer reader1.Close()
|
||||
|
||||
reader2, err := sstMgr.CreateSST(2, rows2)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer reader2.Close()
|
||||
|
||||
// 添加到 Version
|
||||
edit := manifest.NewVersionEdit()
|
||||
edit.AddFile(&manifest.FileMetadata{
|
||||
FileNumber: 1,
|
||||
Level: 0,
|
||||
FileSize: 512,
|
||||
MinKey: 1,
|
||||
MaxKey: 2,
|
||||
RowCount: 2,
|
||||
})
|
||||
edit.AddFile(&manifest.FileMetadata{
|
||||
FileNumber: 2,
|
||||
Level: 0,
|
||||
FileSize: 512,
|
||||
MinKey: 1,
|
||||
MaxKey: 3,
|
||||
RowCount: 2,
|
||||
})
|
||||
nextFileNum := int64(3)
|
||||
edit.SetNextFileNumber(nextFileNum)
|
||||
|
||||
err = versionSet.LogAndApply(edit)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// 创建 Compactor
|
||||
compactor := NewCompactor(sstDir, versionSet)
|
||||
|
||||
// 创建 Compaction 任务
|
||||
version := versionSet.GetCurrent()
|
||||
task := &CompactionTask{
|
||||
Level: 0,
|
||||
InputFiles: version.GetLevel(0),
|
||||
OutputLevel: 1,
|
||||
}
|
||||
|
||||
// 执行 Compaction
|
||||
resultEdit, err := compactor.DoCompaction(task, version)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// 验证结果
|
||||
if len(resultEdit.DeletedFiles) != 2 {
|
||||
t.Errorf("Expected 2 deleted files, got %d", len(resultEdit.DeletedFiles))
|
||||
}
|
||||
|
||||
if len(resultEdit.AddedFiles) == 0 {
|
||||
t.Error("Expected at least 1 new file")
|
||||
}
|
||||
|
||||
t.Logf("Compaction result: deleted %d files, added %d files", len(resultEdit.DeletedFiles), len(resultEdit.AddedFiles))
|
||||
|
||||
// 验证新文件在 L1
|
||||
for _, file := range resultEdit.AddedFiles {
|
||||
if file.Level != 1 {
|
||||
t.Errorf("Expected new file in L1, got L%d", file.Level)
|
||||
}
|
||||
t.Logf("New file: %d, L%d, rows: %d, key range: [%d, %d]",
|
||||
file.FileNumber, file.Level, file.RowCount, file.MinKey, file.MaxKey)
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkCompaction(b *testing.B) {
|
||||
// 创建临时目录
|
||||
tmpDir := b.TempDir()
|
||||
sstDir := filepath.Join(tmpDir, "sst")
|
||||
manifestDir := tmpDir
|
||||
|
||||
err := os.MkdirAll(sstDir, 0755)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
|
||||
// 创建 VersionSet
|
||||
versionSet, err := manifest.NewVersionSet(manifestDir)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
defer versionSet.Close()
|
||||
|
||||
// 创建 SST Manager
|
||||
sstMgr, err := sst.NewManager(sstDir)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
defer sstMgr.Close()
|
||||
|
||||
// 创建测试数据
|
||||
const numFiles = 5
|
||||
const rowsPerFile = 1000
|
||||
|
||||
for i := 0; i < numFiles; i++ {
|
||||
rows := make([]*sst.Row, rowsPerFile)
|
||||
for j := 0; j < rowsPerFile; j++ {
|
||||
rows[j] = &sst.Row{
|
||||
Seq: int64(i*rowsPerFile + j),
|
||||
Time: int64(1000 + i),
|
||||
Data: map[string]interface{}{
|
||||
"value": fmt.Sprintf("data-%d-%d", i, j),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
reader, err := sstMgr.CreateSST(int64(i+1), rows)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
reader.Close()
|
||||
|
||||
edit := manifest.NewVersionEdit()
|
||||
edit.AddFile(&manifest.FileMetadata{
|
||||
FileNumber: int64(i + 1),
|
||||
Level: 0,
|
||||
FileSize: 10240,
|
||||
MinKey: int64(i * rowsPerFile),
|
||||
MaxKey: int64((i+1)*rowsPerFile - 1),
|
||||
RowCount: rowsPerFile,
|
||||
})
|
||||
nextFileNum := int64(i + 2)
|
||||
edit.SetNextFileNumber(nextFileNum)
|
||||
|
||||
err = versionSet.LogAndApply(edit)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
// 创建 Compactor
|
||||
compactor := NewCompactor(sstDir, versionSet)
|
||||
version := versionSet.GetCurrent()
|
||||
|
||||
task := &CompactionTask{
|
||||
Level: 0,
|
||||
InputFiles: version.GetLevel(0),
|
||||
OutputLevel: 1,
|
||||
}
|
||||
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
_, err := compactor.DoCompaction(task, version)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
370
compaction/compactor.go
Normal file
370
compaction/compactor.go
Normal file
@@ -0,0 +1,370 @@
|
||||
package compaction
|
||||
|
||||
import (
|
||||
"code.tczkiot.com/srdb/manifest"
|
||||
"code.tczkiot.com/srdb/sst"
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"sort"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// Compactor 负责执行 Compaction
|
||||
type Compactor struct {
|
||||
sstDir string
|
||||
picker *Picker
|
||||
versionSet *manifest.VersionSet
|
||||
mu sync.Mutex
|
||||
}
|
||||
|
||||
// NewCompactor 创建新的 Compactor
|
||||
func NewCompactor(sstDir string, versionSet *manifest.VersionSet) *Compactor {
|
||||
return &Compactor{
|
||||
sstDir: sstDir,
|
||||
picker: NewPicker(),
|
||||
versionSet: versionSet,
|
||||
}
|
||||
}
|
||||
|
||||
// GetPicker 获取 Picker
|
||||
func (c *Compactor) GetPicker() *Picker {
|
||||
return c.picker
|
||||
}
|
||||
|
||||
// DoCompaction 执行一次 Compaction
|
||||
// 返回: VersionEdit (记录变更), error
|
||||
func (c *Compactor) DoCompaction(task *CompactionTask, version *manifest.Version) (*manifest.VersionEdit, error) {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
if task == nil {
|
||||
return nil, fmt.Errorf("compaction task is nil")
|
||||
}
|
||||
|
||||
// 0. 验证输入文件是否存在(防止并发 compaction 导致的竞态)
|
||||
existingInputFiles := make([]*manifest.FileMetadata, 0, len(task.InputFiles))
|
||||
for _, file := range task.InputFiles {
|
||||
sstPath := filepath.Join(c.sstDir, fmt.Sprintf("%06d.sst", file.FileNumber))
|
||||
if _, err := os.Stat(sstPath); err == nil {
|
||||
existingInputFiles = append(existingInputFiles, file)
|
||||
} else {
|
||||
fmt.Printf("[Compaction] Warning: input file %06d.sst not found, skipping from task\n", file.FileNumber)
|
||||
}
|
||||
}
|
||||
|
||||
// 如果所有输入文件都不存在,直接返回(无需 compaction)
|
||||
if len(existingInputFiles) == 0 {
|
||||
fmt.Printf("[Compaction] All input files missing, compaction skipped\n")
|
||||
return nil, nil // 返回 nil 表示不需要应用任何 VersionEdit
|
||||
}
|
||||
|
||||
// 1. 读取输入文件的所有行
|
||||
inputRows, err := c.readInputFiles(existingInputFiles)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("read input files: %w", err)
|
||||
}
|
||||
|
||||
// 2. 如果输出层级有文件,需要合并重叠的文件
|
||||
outputFiles := c.getOverlappingFiles(version, task.OutputLevel, inputRows)
|
||||
var existingOutputFiles []*manifest.FileMetadata
|
||||
var missingOutputFiles []*manifest.FileMetadata
|
||||
if len(outputFiles) > 0 {
|
||||
// 验证输出文件是否存在
|
||||
existingOutputFiles = make([]*manifest.FileMetadata, 0, len(outputFiles))
|
||||
missingOutputFiles = make([]*manifest.FileMetadata, 0)
|
||||
for _, file := range outputFiles {
|
||||
sstPath := filepath.Join(c.sstDir, fmt.Sprintf("%06d.sst", file.FileNumber))
|
||||
if _, err := os.Stat(sstPath); err == nil {
|
||||
existingOutputFiles = append(existingOutputFiles, file)
|
||||
} else {
|
||||
// 输出层级的文件不存在,记录并在 VersionEdit 中删除它
|
||||
fmt.Printf("[Compaction] Warning: overlapping output file %06d.sst missing, will remove from MANIFEST\n", file.FileNumber)
|
||||
missingOutputFiles = append(missingOutputFiles, file)
|
||||
}
|
||||
}
|
||||
|
||||
outputRows, err := c.readInputFiles(existingOutputFiles)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("read output files: %w", err)
|
||||
}
|
||||
inputRows = append(inputRows, outputRows...)
|
||||
}
|
||||
|
||||
// 3. 合并和去重 (保留最新的记录)
|
||||
mergedRows := c.mergeRows(inputRows)
|
||||
|
||||
// 计算平均行大小(基于输入文件的 FileMetadata)
|
||||
avgRowSize := c.calculateAvgRowSize(existingInputFiles, existingOutputFiles)
|
||||
|
||||
// 4. 写入新的 SST 文件到输出层级
|
||||
newFiles, err := c.writeOutputFiles(mergedRows, task.OutputLevel, version, avgRowSize)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("write output files: %w", err)
|
||||
}
|
||||
|
||||
// 5. 创建 VersionEdit
|
||||
edit := manifest.NewVersionEdit()
|
||||
|
||||
// 删除实际存在且被处理的输入文件
|
||||
for _, file := range existingInputFiles {
|
||||
edit.DeleteFile(file.FileNumber)
|
||||
}
|
||||
// 删除实际存在且被处理的输出层级文件
|
||||
for _, file := range existingOutputFiles {
|
||||
edit.DeleteFile(file.FileNumber)
|
||||
}
|
||||
// 删除缺失的输出层级文件(清理 MANIFEST 中的过期引用)
|
||||
for _, file := range missingOutputFiles {
|
||||
edit.DeleteFile(file.FileNumber)
|
||||
fmt.Printf("[Compaction] Removing missing file %06d.sst from MANIFEST\n", file.FileNumber)
|
||||
}
|
||||
|
||||
// 添加新文件
|
||||
for _, file := range newFiles {
|
||||
edit.AddFile(file)
|
||||
}
|
||||
|
||||
// 持久化当前的文件编号计数器(关键修复:防止重启后文件编号重用)
|
||||
edit.SetNextFileNumber(c.versionSet.GetNextFileNumber())
|
||||
|
||||
return edit, nil
|
||||
}
|
||||
|
||||
// readInputFiles 读取输入文件的所有行
|
||||
// 注意:调用者必须确保传入的文件都存在,否则会返回错误
|
||||
func (c *Compactor) readInputFiles(files []*manifest.FileMetadata) ([]*sst.Row, error) {
|
||||
var allRows []*sst.Row
|
||||
|
||||
for _, file := range files {
|
||||
sstPath := filepath.Join(c.sstDir, fmt.Sprintf("%06d.sst", file.FileNumber))
|
||||
|
||||
reader, err := sst.NewReader(sstPath)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("open sst %d: %w", file.FileNumber, err)
|
||||
}
|
||||
|
||||
// 获取文件中实际存在的所有 key(不能用 MinKey-MaxKey 范围遍历,因为 key 可能是稀疏的)
|
||||
keys := reader.GetAllKeys()
|
||||
for _, seq := range keys {
|
||||
row, err := reader.Get(seq)
|
||||
if err != nil {
|
||||
// 这种情况理论上不应该发生(key 来自索引),但为了安全还是处理一下
|
||||
continue
|
||||
}
|
||||
allRows = append(allRows, row)
|
||||
}
|
||||
|
||||
reader.Close()
|
||||
}
|
||||
|
||||
return allRows, nil
|
||||
}
|
||||
|
||||
// getOverlappingFiles 获取输出层级中与输入行重叠的文件
|
||||
func (c *Compactor) getOverlappingFiles(version *manifest.Version, level int, rows []*sst.Row) []*manifest.FileMetadata {
|
||||
if len(rows) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
// 找到输入行的 key range
|
||||
minKey := rows[0].Seq
|
||||
maxKey := rows[0].Seq
|
||||
for _, row := range rows {
|
||||
if row.Seq < minKey {
|
||||
minKey = row.Seq
|
||||
}
|
||||
if row.Seq > maxKey {
|
||||
maxKey = row.Seq
|
||||
}
|
||||
}
|
||||
|
||||
// 找到输出层级中重叠的文件
|
||||
var overlapping []*manifest.FileMetadata
|
||||
levelFiles := version.GetLevel(level)
|
||||
for _, file := range levelFiles {
|
||||
// 检查 key range 是否重叠
|
||||
if file.MaxKey >= minKey && file.MinKey <= maxKey {
|
||||
overlapping = append(overlapping, file)
|
||||
}
|
||||
}
|
||||
|
||||
return overlapping
|
||||
}
|
||||
|
||||
// mergeRows 合并行,去重并保留最新的记录
|
||||
func (c *Compactor) mergeRows(rows []*sst.Row) []*sst.Row {
|
||||
if len(rows) == 0 {
|
||||
return rows
|
||||
}
|
||||
|
||||
// 按 Seq 排序
|
||||
sort.Slice(rows, func(i, j int) bool {
|
||||
return rows[i].Seq < rows[j].Seq
|
||||
})
|
||||
|
||||
// 去重:保留相同 Seq 的最新记录 (Timestamp 最大的)
|
||||
merged := make([]*sst.Row, 0, len(rows))
|
||||
var lastRow *sst.Row
|
||||
|
||||
for _, row := range rows {
|
||||
if lastRow == nil || lastRow.Seq != row.Seq {
|
||||
// 新的 Seq
|
||||
merged = append(merged, row)
|
||||
lastRow = row
|
||||
} else {
|
||||
// 相同 Seq,保留 Time 更大的
|
||||
if row.Time > lastRow.Time {
|
||||
merged[len(merged)-1] = row
|
||||
lastRow = row
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return merged
|
||||
}
|
||||
|
||||
// calculateAvgRowSize 基于输入文件的 FileMetadata 计算平均行大小
|
||||
func (c *Compactor) calculateAvgRowSize(inputFiles []*manifest.FileMetadata, outputFiles []*manifest.FileMetadata) int64 {
|
||||
var totalSize int64
|
||||
var totalRows int64
|
||||
|
||||
// 统计输入文件
|
||||
for _, file := range inputFiles {
|
||||
totalSize += file.FileSize
|
||||
totalRows += file.RowCount
|
||||
}
|
||||
|
||||
// 统计输出文件
|
||||
for _, file := range outputFiles {
|
||||
totalSize += file.FileSize
|
||||
totalRows += file.RowCount
|
||||
}
|
||||
|
||||
// 计算平均值
|
||||
if totalRows == 0 {
|
||||
return 1024 // 默认 1KB
|
||||
}
|
||||
return totalSize / totalRows
|
||||
}
|
||||
|
||||
// writeOutputFiles 将合并后的行写入新的 SST 文件
|
||||
func (c *Compactor) writeOutputFiles(rows []*sst.Row, level int, version *manifest.Version, avgRowSize int64) ([]*manifest.FileMetadata, error) {
|
||||
if len(rows) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// 根据层级动态调整文件大小目标
|
||||
// L0: 2MB (快速 flush,小文件)
|
||||
// L1: 10MB
|
||||
// L2: 50MB
|
||||
// L3: 100MB
|
||||
// L4+: 200MB
|
||||
targetFileSize := c.getTargetFileSize(level)
|
||||
|
||||
// 应用安全系数:由于压缩率、索引开销等因素,估算值可能不准确
|
||||
// 使用 80% 的目标大小作为分割点,避免实际文件超出目标过多
|
||||
targetFileSize = targetFileSize * 80 / 100
|
||||
|
||||
var newFiles []*manifest.FileMetadata
|
||||
var currentRows []*sst.Row
|
||||
var currentSize int64
|
||||
|
||||
for _, row := range rows {
|
||||
// 使用平均行大小估算(基于输入文件的统计信息)
|
||||
rowSize := avgRowSize
|
||||
|
||||
// 如果当前文件大小超过目标,写入文件
|
||||
if currentSize > 0 && currentSize+rowSize > targetFileSize {
|
||||
file, err := c.writeFile(currentRows, level, version)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
newFiles = append(newFiles, file)
|
||||
|
||||
// 重置
|
||||
currentRows = nil
|
||||
currentSize = 0
|
||||
}
|
||||
|
||||
currentRows = append(currentRows, row)
|
||||
currentSize += rowSize
|
||||
}
|
||||
|
||||
// 写入最后一个文件
|
||||
if len(currentRows) > 0 {
|
||||
file, err := c.writeFile(currentRows, level, version)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
newFiles = append(newFiles, file)
|
||||
}
|
||||
|
||||
return newFiles, nil
|
||||
}
|
||||
|
||||
// getTargetFileSize 根据层级返回目标文件大小
|
||||
func (c *Compactor) getTargetFileSize(level int) int64 {
|
||||
switch level {
|
||||
case 0:
|
||||
return 2 * 1024 * 1024 // 2MB
|
||||
case 1:
|
||||
return 10 * 1024 * 1024 // 10MB
|
||||
case 2:
|
||||
return 50 * 1024 * 1024 // 50MB
|
||||
case 3:
|
||||
return 100 * 1024 * 1024 // 100MB
|
||||
default: // L4+
|
||||
return 200 * 1024 * 1024 // 200MB
|
||||
}
|
||||
}
|
||||
|
||||
// writeFile 写入单个 SST 文件
|
||||
func (c *Compactor) writeFile(rows []*sst.Row, level int, version *manifest.Version) (*manifest.FileMetadata, error) {
|
||||
// 从 VersionSet 分配新的文件编号
|
||||
fileNumber := c.versionSet.AllocateFileNumber()
|
||||
sstPath := filepath.Join(c.sstDir, fmt.Sprintf("%06d.sst", fileNumber))
|
||||
|
||||
// 创建文件
|
||||
file, err := os.Create(sstPath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer file.Close()
|
||||
|
||||
writer := sst.NewWriter(file)
|
||||
|
||||
// 写入所有行
|
||||
for _, row := range rows {
|
||||
err = writer.Add(row)
|
||||
if err != nil {
|
||||
os.Remove(sstPath)
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// 完成写入
|
||||
err = writer.Finish()
|
||||
if err != nil {
|
||||
os.Remove(sstPath)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// 获取文件信息
|
||||
fileInfo, err := file.Stat()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// 创建 FileMetadata
|
||||
metadata := &manifest.FileMetadata{
|
||||
FileNumber: fileNumber,
|
||||
Level: level,
|
||||
FileSize: fileInfo.Size(),
|
||||
MinKey: rows[0].Seq,
|
||||
MaxKey: rows[len(rows)-1].Seq,
|
||||
RowCount: int64(len(rows)),
|
||||
}
|
||||
|
||||
return metadata, nil
|
||||
}
|
||||
444
compaction/manager.go
Normal file
444
compaction/manager.go
Normal file
@@ -0,0 +1,444 @@
|
||||
package compaction
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"code.tczkiot.com/srdb/manifest"
|
||||
)
|
||||
|
||||
// Manager 管理 Compaction 流程
|
||||
type Manager struct {
|
||||
compactor *Compactor
|
||||
versionSet *manifest.VersionSet
|
||||
sstDir string
|
||||
|
||||
// 控制后台 Compaction
|
||||
stopCh chan struct{}
|
||||
wg sync.WaitGroup
|
||||
|
||||
// Compaction 并发控制
|
||||
compactionMu sync.Mutex // 防止并发执行 compaction
|
||||
|
||||
// 统计信息
|
||||
mu sync.RWMutex
|
||||
totalCompactions int64
|
||||
lastCompactionTime time.Time
|
||||
lastFailedFile int64 // 最后失败的文件编号
|
||||
consecutiveFails int // 连续失败次数
|
||||
lastGCTime time.Time
|
||||
totalOrphansFound int64
|
||||
}
|
||||
|
||||
// NewManager 创建新的 Compaction Manager
|
||||
func NewManager(sstDir string, versionSet *manifest.VersionSet) *Manager {
|
||||
return &Manager{
|
||||
compactor: NewCompactor(sstDir, versionSet),
|
||||
versionSet: versionSet,
|
||||
sstDir: sstDir,
|
||||
stopCh: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
|
||||
// GetPicker 获取 Compaction Picker
|
||||
func (m *Manager) GetPicker() *Picker {
|
||||
return m.compactor.GetPicker()
|
||||
}
|
||||
|
||||
// Start 启动后台 Compaction 和垃圾回收
|
||||
func (m *Manager) Start() {
|
||||
m.wg.Add(2)
|
||||
go m.backgroundCompaction()
|
||||
go m.backgroundGarbageCollection()
|
||||
}
|
||||
|
||||
// Stop 停止后台 Compaction
|
||||
func (m *Manager) Stop() {
|
||||
close(m.stopCh)
|
||||
m.wg.Wait()
|
||||
}
|
||||
|
||||
// backgroundCompaction 后台 Compaction 循环
|
||||
func (m *Manager) backgroundCompaction() {
|
||||
defer m.wg.Done()
|
||||
|
||||
ticker := time.NewTicker(10 * time.Second) // 每 10 秒检查一次
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-m.stopCh:
|
||||
return
|
||||
case <-ticker.C:
|
||||
m.maybeCompact()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// MaybeCompact 检查是否需要 Compaction 并执行(公开方法,供外部调用)
|
||||
// 非阻塞:如果已有 compaction 在执行,直接返回
|
||||
func (m *Manager) MaybeCompact() {
|
||||
// 尝试获取锁,如果已有 compaction 在执行,直接返回
|
||||
if !m.compactionMu.TryLock() {
|
||||
return
|
||||
}
|
||||
defer m.compactionMu.Unlock()
|
||||
|
||||
m.doCompact()
|
||||
}
|
||||
|
||||
// maybeCompact 内部使用的阻塞版本(后台 goroutine 使用)
|
||||
func (m *Manager) maybeCompact() {
|
||||
m.compactionMu.Lock()
|
||||
defer m.compactionMu.Unlock()
|
||||
|
||||
m.doCompact()
|
||||
}
|
||||
|
||||
// doCompact 实际执行 compaction 的逻辑(必须在持有 compactionMu 时调用)
|
||||
// 支持并发执行多个层级的 compaction
|
||||
func (m *Manager) doCompact() {
|
||||
// 获取当前版本
|
||||
version := m.versionSet.GetCurrent()
|
||||
if version == nil {
|
||||
return
|
||||
}
|
||||
|
||||
// 获取所有需要 Compaction 的任务(已按优先级排序)
|
||||
picker := m.compactor.GetPicker()
|
||||
tasks := picker.PickCompaction(version)
|
||||
if len(tasks) == 0 {
|
||||
// 输出诊断信息
|
||||
m.printCompactionStats(version, picker)
|
||||
return
|
||||
}
|
||||
|
||||
fmt.Printf("[Compaction] Found %d tasks to execute\n", len(tasks))
|
||||
|
||||
// 并发执行所有任务
|
||||
successCount := 0
|
||||
for _, task := range tasks {
|
||||
// 检查是否是上次失败的文件(防止无限重试)
|
||||
if len(task.InputFiles) > 0 {
|
||||
firstFile := task.InputFiles[0].FileNumber
|
||||
m.mu.Lock()
|
||||
if m.lastFailedFile == firstFile && m.consecutiveFails >= 3 {
|
||||
fmt.Printf("[Compaction] Skipping L%d file %d (failed %d times)\n",
|
||||
task.Level, firstFile, m.consecutiveFails)
|
||||
m.consecutiveFails = 0
|
||||
m.lastFailedFile = 0
|
||||
m.mu.Unlock()
|
||||
continue
|
||||
}
|
||||
m.mu.Unlock()
|
||||
}
|
||||
|
||||
// 获取最新版本(每个任务执行前)
|
||||
currentVersion := m.versionSet.GetCurrent()
|
||||
if currentVersion == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// 执行 Compaction
|
||||
fmt.Printf("[Compaction] Starting: L%d -> L%d, files: %d\n",
|
||||
task.Level, task.OutputLevel, len(task.InputFiles))
|
||||
|
||||
err := m.DoCompactionWithVersion(task, currentVersion)
|
||||
if err != nil {
|
||||
fmt.Printf("[Compaction] Failed L%d -> L%d: %v\n", task.Level, task.OutputLevel, err)
|
||||
|
||||
// 记录失败信息
|
||||
if len(task.InputFiles) > 0 {
|
||||
firstFile := task.InputFiles[0].FileNumber
|
||||
m.mu.Lock()
|
||||
if m.lastFailedFile == firstFile {
|
||||
m.consecutiveFails++
|
||||
} else {
|
||||
m.lastFailedFile = firstFile
|
||||
m.consecutiveFails = 1
|
||||
}
|
||||
m.mu.Unlock()
|
||||
}
|
||||
} else {
|
||||
fmt.Printf("[Compaction] Completed: L%d -> L%d\n", task.Level, task.OutputLevel)
|
||||
successCount++
|
||||
|
||||
// 清除失败计数
|
||||
m.mu.Lock()
|
||||
m.consecutiveFails = 0
|
||||
m.lastFailedFile = 0
|
||||
m.mu.Unlock()
|
||||
}
|
||||
}
|
||||
|
||||
fmt.Printf("[Compaction] Batch completed: %d/%d tasks succeeded\n", successCount, len(tasks))
|
||||
}
|
||||
|
||||
// printCompactionStats 输出 Compaction 统计信息(每分钟一次)
|
||||
func (m *Manager) printCompactionStats(version *manifest.Version, picker *Picker) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
|
||||
// 限制输出频率:每 60 秒输出一次
|
||||
if time.Since(m.lastCompactionTime) < 60*time.Second {
|
||||
return
|
||||
}
|
||||
m.lastCompactionTime = time.Now()
|
||||
|
||||
fmt.Println("[Compaction] Status check:")
|
||||
for level := 0; level < 7; level++ {
|
||||
files := version.GetLevel(level)
|
||||
if len(files) == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
totalSize := int64(0)
|
||||
for _, f := range files {
|
||||
totalSize += f.FileSize
|
||||
}
|
||||
|
||||
score := picker.GetLevelScore(version, level)
|
||||
fmt.Printf(" L%d: %d files, %.2f MB, score: %.2f\n",
|
||||
level, len(files), float64(totalSize)/(1024*1024), score)
|
||||
}
|
||||
}
|
||||
|
||||
// DoCompactionWithVersion 使用指定的版本执行 Compaction
|
||||
func (m *Manager) DoCompactionWithVersion(task *CompactionTask, version *manifest.Version) error {
|
||||
if version == nil {
|
||||
return fmt.Errorf("version is nil")
|
||||
}
|
||||
|
||||
// 执行 Compaction(使用传入的 version,而不是重新获取)
|
||||
edit, err := m.compactor.DoCompaction(task, version)
|
||||
if err != nil {
|
||||
return fmt.Errorf("compaction failed: %w", err)
|
||||
}
|
||||
|
||||
// 如果 edit 为 nil,说明所有文件都已经不存在,无需应用变更
|
||||
if edit == nil {
|
||||
fmt.Printf("[Compaction] No changes needed (files already removed)\n")
|
||||
return nil
|
||||
}
|
||||
|
||||
// 应用 VersionEdit
|
||||
err = m.versionSet.LogAndApply(edit)
|
||||
if err != nil {
|
||||
// LogAndApply 失败,清理已写入的新 SST 文件(防止孤儿文件)
|
||||
fmt.Printf("[Compaction] LogAndApply failed, cleaning up new files: %v\n", err)
|
||||
m.cleanupNewFiles(edit)
|
||||
return fmt.Errorf("apply version edit: %w", err)
|
||||
}
|
||||
|
||||
// LogAndApply 成功后,删除废弃的 SST 文件
|
||||
m.deleteObsoleteFiles(edit)
|
||||
|
||||
// 更新统计信息
|
||||
m.mu.Lock()
|
||||
m.totalCompactions++
|
||||
m.lastCompactionTime = time.Now()
|
||||
m.mu.Unlock()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// DoCompaction 执行一次 Compaction(兼容旧接口)
|
||||
func (m *Manager) DoCompaction(task *CompactionTask) error {
|
||||
// 获取当前版本
|
||||
version := m.versionSet.GetCurrent()
|
||||
if version == nil {
|
||||
return fmt.Errorf("no current version")
|
||||
}
|
||||
|
||||
return m.DoCompactionWithVersion(task, version)
|
||||
}
|
||||
|
||||
// cleanupNewFiles 清理 LogAndApply 失败后的新文件(防止孤儿文件)
|
||||
func (m *Manager) cleanupNewFiles(edit *manifest.VersionEdit) {
|
||||
if edit == nil {
|
||||
return
|
||||
}
|
||||
|
||||
fmt.Printf("[Compaction] Cleaning up %d new files after LogAndApply failure\n", len(edit.AddedFiles))
|
||||
|
||||
// 删除新创建的文件
|
||||
for _, file := range edit.AddedFiles {
|
||||
sstPath := filepath.Join(m.sstDir, fmt.Sprintf("%06d.sst", file.FileNumber))
|
||||
err := os.Remove(sstPath)
|
||||
if err != nil {
|
||||
fmt.Printf("[Compaction] Failed to cleanup new file %06d.sst: %v\n", file.FileNumber, err)
|
||||
} else {
|
||||
fmt.Printf("[Compaction] Cleaned up new file %06d.sst\n", file.FileNumber)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// deleteObsoleteFiles 删除废弃的 SST 文件
|
||||
func (m *Manager) deleteObsoleteFiles(edit *manifest.VersionEdit) {
|
||||
if edit == nil {
|
||||
fmt.Printf("[Compaction] deleteObsoleteFiles: edit is nil\n")
|
||||
return
|
||||
}
|
||||
|
||||
fmt.Printf("[Compaction] deleteObsoleteFiles: %d files to delete\n", len(edit.DeletedFiles))
|
||||
|
||||
// 删除被标记为删除的文件
|
||||
for _, fileNum := range edit.DeletedFiles {
|
||||
sstPath := filepath.Join(m.sstDir, fmt.Sprintf("%06d.sst", fileNum))
|
||||
err := os.Remove(sstPath)
|
||||
if err != nil {
|
||||
// 删除失败只记录日志,不影响 compaction 流程
|
||||
// 后台垃圾回收器会重试
|
||||
fmt.Printf("[Compaction] Failed to delete obsolete file %06d.sst: %v\n", fileNum, err)
|
||||
} else {
|
||||
fmt.Printf("[Compaction] Deleted obsolete file %06d.sst\n", fileNum)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TriggerCompaction 手动触发一次 Compaction(所有需要的层级)
|
||||
func (m *Manager) TriggerCompaction() error {
|
||||
version := m.versionSet.GetCurrent()
|
||||
if version == nil {
|
||||
return fmt.Errorf("no current version")
|
||||
}
|
||||
|
||||
picker := m.compactor.GetPicker()
|
||||
tasks := picker.PickCompaction(version)
|
||||
if len(tasks) == 0 {
|
||||
return nil // 不需要 Compaction
|
||||
}
|
||||
|
||||
// 依次执行所有任务
|
||||
for _, task := range tasks {
|
||||
currentVersion := m.versionSet.GetCurrent()
|
||||
if err := m.DoCompactionWithVersion(task, currentVersion); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetStats 获取 Compaction 统计信息
|
||||
func (m *Manager) GetStats() map[string]interface{} {
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
|
||||
return map[string]interface{}{
|
||||
"total_compactions": m.totalCompactions,
|
||||
"last_compaction_time": m.lastCompactionTime,
|
||||
}
|
||||
}
|
||||
|
||||
// GetLevelStats 获取每层的统计信息
|
||||
func (m *Manager) GetLevelStats() []map[string]interface{} {
|
||||
version := m.versionSet.GetCurrent()
|
||||
if version == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
picker := m.compactor.GetPicker()
|
||||
stats := make([]map[string]interface{}, manifest.NumLevels)
|
||||
|
||||
for level := 0; level < manifest.NumLevels; level++ {
|
||||
files := version.GetLevel(level)
|
||||
totalSize := int64(0)
|
||||
for _, file := range files {
|
||||
totalSize += file.FileSize
|
||||
}
|
||||
|
||||
stats[level] = map[string]interface{}{
|
||||
"level": level,
|
||||
"file_count": len(files),
|
||||
"total_size": totalSize,
|
||||
"score": picker.GetLevelScore(version, level),
|
||||
}
|
||||
}
|
||||
|
||||
return stats
|
||||
}
|
||||
|
||||
// backgroundGarbageCollection 后台垃圾回收循环
|
||||
func (m *Manager) backgroundGarbageCollection() {
|
||||
defer m.wg.Done()
|
||||
|
||||
ticker := time.NewTicker(5 * time.Minute) // 每 5 分钟检查一次
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-m.stopCh:
|
||||
return
|
||||
case <-ticker.C:
|
||||
m.collectOrphanFiles()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// collectOrphanFiles 收集并删除孤儿 SST 文件
|
||||
func (m *Manager) collectOrphanFiles() {
|
||||
// 1. 获取当前版本中的所有活跃文件
|
||||
version := m.versionSet.GetCurrent()
|
||||
if version == nil {
|
||||
return
|
||||
}
|
||||
|
||||
activeFiles := make(map[int64]bool)
|
||||
for level := 0; level < manifest.NumLevels; level++ {
|
||||
files := version.GetLevel(level)
|
||||
for _, file := range files {
|
||||
activeFiles[file.FileNumber] = true
|
||||
}
|
||||
}
|
||||
|
||||
// 2. 扫描 SST 目录中的所有文件
|
||||
pattern := filepath.Join(m.sstDir, "*.sst")
|
||||
sstFiles, err := filepath.Glob(pattern)
|
||||
if err != nil {
|
||||
fmt.Printf("[GC] Failed to scan SST directory: %v\n", err)
|
||||
return
|
||||
}
|
||||
|
||||
// 3. 找出孤儿文件并删除
|
||||
orphanCount := 0
|
||||
for _, sstPath := range sstFiles {
|
||||
// 提取文件编号
|
||||
var fileNum int64
|
||||
_, err := fmt.Sscanf(filepath.Base(sstPath), "%d.sst", &fileNum)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// 检查是否是活跃文件
|
||||
if !activeFiles[fileNum] {
|
||||
// 这是孤儿文件,删除它
|
||||
err := os.Remove(sstPath)
|
||||
if err != nil {
|
||||
fmt.Printf("[GC] Failed to delete orphan file %06d.sst: %v\n", fileNum, err)
|
||||
} else {
|
||||
fmt.Printf("[GC] Deleted orphan file %06d.sst\n", fileNum)
|
||||
orphanCount++
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// 4. 更新统计信息
|
||||
m.mu.Lock()
|
||||
m.lastGCTime = time.Now()
|
||||
m.totalOrphansFound += int64(orphanCount)
|
||||
m.mu.Unlock()
|
||||
|
||||
if orphanCount > 0 {
|
||||
fmt.Printf("[GC] Completed: cleaned up %d orphan files (total: %d)\n", orphanCount, m.totalOrphansFound)
|
||||
}
|
||||
}
|
||||
|
||||
// CleanupOrphanFiles 手动触发孤儿文件清理(可在启动时调用)
|
||||
func (m *Manager) CleanupOrphanFiles() {
|
||||
fmt.Println("[GC] Manual cleanup triggered")
|
||||
m.collectOrphanFiles()
|
||||
}
|
||||
285
compaction/picker.go
Normal file
285
compaction/picker.go
Normal file
@@ -0,0 +1,285 @@
|
||||
package compaction
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"code.tczkiot.com/srdb/manifest"
|
||||
)
|
||||
|
||||
// CompactionTask 表示一个 Compaction 任务
|
||||
type CompactionTask struct {
|
||||
Level int // 源层级
|
||||
InputFiles []*manifest.FileMetadata // 需要合并的输入文件
|
||||
OutputLevel int // 输出层级
|
||||
}
|
||||
|
||||
// Picker 负责选择需要 Compaction 的文件
|
||||
type Picker struct {
|
||||
// Level 大小限制 (字节)
|
||||
levelSizeLimits [manifest.NumLevels]int64
|
||||
|
||||
// Level 文件数量限制
|
||||
levelFileLimits [manifest.NumLevels]int
|
||||
}
|
||||
|
||||
// NewPicker 创建新的 Compaction Picker
|
||||
func NewPicker() *Picker {
|
||||
p := &Picker{}
|
||||
|
||||
// 设置每层的大小限制 (指数增长)
|
||||
// L0: 10MB, L1: 100MB, L2: 1GB, L3: 10GB, L4: 100GB, L5: 1TB, L6: 无限制
|
||||
p.levelSizeLimits[0] = 10 * 1024 * 1024 // 10MB
|
||||
p.levelSizeLimits[1] = 100 * 1024 * 1024 // 100MB
|
||||
p.levelSizeLimits[2] = 1024 * 1024 * 1024 // 1GB
|
||||
p.levelSizeLimits[3] = 10 * 1024 * 1024 * 1024 // 10GB
|
||||
p.levelSizeLimits[4] = 100 * 1024 * 1024 * 1024 // 100GB
|
||||
p.levelSizeLimits[5] = 1024 * 1024 * 1024 * 1024 // 1TB
|
||||
p.levelSizeLimits[6] = 0 // 无限制
|
||||
|
||||
// 设置每层的文件数量限制
|
||||
// L0 特殊处理:文件数量限制为 4 (当有4个或更多文件时触发 compaction)
|
||||
p.levelFileLimits[0] = 4
|
||||
// L1-L6: 不限制文件数量,只限制总大小
|
||||
for i := 1; i < manifest.NumLevels; i++ {
|
||||
p.levelFileLimits[i] = 0 // 0 表示不限制
|
||||
}
|
||||
|
||||
return p
|
||||
}
|
||||
|
||||
// PickCompaction 选择需要 Compaction 的任务(支持多任务并发)
|
||||
// 返回空切片表示当前不需要 Compaction
|
||||
func (p *Picker) PickCompaction(version *manifest.Version) []*CompactionTask {
|
||||
tasks := make([]*CompactionTask, 0)
|
||||
|
||||
// 1. 检查 L0 (基于文件数量)
|
||||
if task := p.pickL0Compaction(version); task != nil {
|
||||
tasks = append(tasks, task)
|
||||
}
|
||||
|
||||
// 2. 检查 L1-L5 (基于大小)
|
||||
for level := 1; level < manifest.NumLevels-1; level++ {
|
||||
if task := p.pickLevelCompaction(version, level); task != nil {
|
||||
tasks = append(tasks, task)
|
||||
}
|
||||
}
|
||||
|
||||
// 3. 按优先级排序(score 越高越优先)
|
||||
if len(tasks) > 1 {
|
||||
p.sortTasksByPriority(tasks, version)
|
||||
}
|
||||
|
||||
return tasks
|
||||
}
|
||||
|
||||
// sortTasksByPriority 按优先级对任务排序(score 从高到低)
|
||||
func (p *Picker) sortTasksByPriority(tasks []*CompactionTask, version *manifest.Version) {
|
||||
// 简单的冒泡排序(任务数量通常很少,< 7)
|
||||
for i := 0; i < len(tasks)-1; i++ {
|
||||
for j := i + 1; j < len(tasks); j++ {
|
||||
scoreI := p.GetLevelScore(version, tasks[i].Level)
|
||||
scoreJ := p.GetLevelScore(version, tasks[j].Level)
|
||||
if scoreJ > scoreI {
|
||||
tasks[i], tasks[j] = tasks[j], tasks[i]
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// pickL0Compaction 选择 L0 的 Compaction 任务
|
||||
// L0 特殊:文件可能有重叠的 key range,需要全部合并
|
||||
func (p *Picker) pickL0Compaction(version *manifest.Version) *CompactionTask {
|
||||
l0Files := version.GetLevel(0)
|
||||
if len(l0Files) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
// 计算 L0 总大小
|
||||
totalSize := int64(0)
|
||||
for _, file := range l0Files {
|
||||
totalSize += file.FileSize
|
||||
}
|
||||
|
||||
// 检查是否需要 Compaction(同时考虑文件数量和总大小)
|
||||
// 1. 文件数量超过限制(避免读放大:每次读取需要检查太多文件)
|
||||
// 2. 总大小超过限制(避免 L0 占用过多空间)
|
||||
needCompaction := false
|
||||
if p.levelFileLimits[0] > 0 && len(l0Files) >= p.levelFileLimits[0] {
|
||||
needCompaction = true
|
||||
}
|
||||
if p.levelSizeLimits[0] > 0 && totalSize >= p.levelSizeLimits[0] {
|
||||
needCompaction = true
|
||||
}
|
||||
|
||||
if !needCompaction {
|
||||
return nil
|
||||
}
|
||||
|
||||
// L0 → L1 Compaction
|
||||
// 选择所有 L0 文件(因为 key range 可能重叠)
|
||||
return &CompactionTask{
|
||||
Level: 0,
|
||||
InputFiles: l0Files,
|
||||
OutputLevel: 1,
|
||||
}
|
||||
}
|
||||
|
||||
// pickLevelCompaction 选择 L1-L5 的 Compaction 任务
|
||||
// L1+ 的文件 key range 不重叠,可以选择多个不重叠的文件
|
||||
func (p *Picker) pickLevelCompaction(version *manifest.Version, level int) *CompactionTask {
|
||||
if level < 1 || level >= manifest.NumLevels-1 {
|
||||
return nil
|
||||
}
|
||||
|
||||
files := version.GetLevel(level)
|
||||
if len(files) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
// 计算当前层级的总大小
|
||||
totalSize := int64(0)
|
||||
for _, file := range files {
|
||||
totalSize += file.FileSize
|
||||
}
|
||||
|
||||
// 检查是否超过大小限制
|
||||
if totalSize < p.levelSizeLimits[level] {
|
||||
return nil
|
||||
}
|
||||
|
||||
// 改进策略:根据层级压力动态调整选择策略
|
||||
// 1. 计算当前层级的压力(超过限制的倍数)
|
||||
pressure := float64(totalSize) / float64(p.levelSizeLimits[level])
|
||||
|
||||
// 2. 根据压力确定目标大小和文件数量限制
|
||||
targetSize := p.getTargetCompactionSize(level + 1)
|
||||
maxFiles := 10 // 默认最多 10 个文件
|
||||
|
||||
if pressure >= 10.0 {
|
||||
// 压力极高(超过 10 倍):选择更多文件,增大目标
|
||||
maxFiles = 100
|
||||
targetSize *= 5
|
||||
fmt.Printf("[Compaction] L%d pressure: %.1fx (CRITICAL) - selecting up to %d files, target: %s\n",
|
||||
level, pressure, maxFiles, formatBytes(targetSize))
|
||||
} else if pressure >= 5.0 {
|
||||
// 压力很高(超过 5 倍)
|
||||
maxFiles = 50
|
||||
targetSize *= 3
|
||||
fmt.Printf("[Compaction] L%d pressure: %.1fx (HIGH) - selecting up to %d files, target: %s\n",
|
||||
level, pressure, maxFiles, formatBytes(targetSize))
|
||||
} else if pressure >= 2.0 {
|
||||
// 压力较高(超过 2 倍)
|
||||
maxFiles = 20
|
||||
targetSize *= 2
|
||||
fmt.Printf("[Compaction] L%d pressure: %.1fx (ELEVATED) - selecting up to %d files, target: %s\n",
|
||||
level, pressure, maxFiles, formatBytes(targetSize))
|
||||
}
|
||||
|
||||
// 选择文件,直到累计大小接近目标
|
||||
selectedFiles := make([]*manifest.FileMetadata, 0)
|
||||
currentSize := int64(0)
|
||||
|
||||
for _, file := range files {
|
||||
selectedFiles = append(selectedFiles, file)
|
||||
currentSize += file.FileSize
|
||||
|
||||
// 如果已经达到目标大小,停止选择
|
||||
if currentSize >= targetSize {
|
||||
break
|
||||
}
|
||||
|
||||
// 达到文件数量限制
|
||||
if len(selectedFiles) >= maxFiles {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return &CompactionTask{
|
||||
Level: level,
|
||||
InputFiles: selectedFiles,
|
||||
OutputLevel: level + 1,
|
||||
}
|
||||
}
|
||||
|
||||
// getTargetCompactionSize 根据层级返回建议的 compaction 大小
|
||||
func (p *Picker) getTargetCompactionSize(level int) int64 {
|
||||
switch level {
|
||||
case 0:
|
||||
return 2 * 1024 * 1024 // 2MB
|
||||
case 1:
|
||||
return 10 * 1024 * 1024 // 10MB
|
||||
case 2:
|
||||
return 50 * 1024 * 1024 // 50MB
|
||||
case 3:
|
||||
return 100 * 1024 * 1024 // 100MB
|
||||
default: // L4+
|
||||
return 200 * 1024 * 1024 // 200MB
|
||||
}
|
||||
}
|
||||
|
||||
// ShouldCompact 判断是否需要 Compaction
|
||||
func (p *Picker) ShouldCompact(version *manifest.Version) bool {
|
||||
tasks := p.PickCompaction(version)
|
||||
return len(tasks) > 0
|
||||
}
|
||||
|
||||
// GetLevelScore 获取每层的 Compaction 得分 (用于优先级排序)
|
||||
// 得分越高,越需要 Compaction
|
||||
func (p *Picker) GetLevelScore(version *manifest.Version, level int) float64 {
|
||||
if level < 0 || level >= manifest.NumLevels {
|
||||
return 0
|
||||
}
|
||||
|
||||
files := version.GetLevel(level)
|
||||
|
||||
// L0 同时考虑文件数量和总大小,取较大值作为得分
|
||||
if level == 0 {
|
||||
scoreByCount := float64(0)
|
||||
scoreBySize := float64(0)
|
||||
|
||||
if p.levelFileLimits[0] > 0 {
|
||||
scoreByCount = float64(len(files)) / float64(p.levelFileLimits[0])
|
||||
}
|
||||
|
||||
if p.levelSizeLimits[0] > 0 {
|
||||
totalSize := int64(0)
|
||||
for _, file := range files {
|
||||
totalSize += file.FileSize
|
||||
}
|
||||
scoreBySize = float64(totalSize) / float64(p.levelSizeLimits[0])
|
||||
}
|
||||
|
||||
// 返回两者中的较大值(哪个维度更紧迫)
|
||||
if scoreByCount > scoreBySize {
|
||||
return scoreByCount
|
||||
}
|
||||
return scoreBySize
|
||||
}
|
||||
|
||||
// L1+ 基于总大小
|
||||
if p.levelSizeLimits[level] == 0 {
|
||||
return 0
|
||||
}
|
||||
|
||||
totalSize := int64(0)
|
||||
for _, file := range files {
|
||||
totalSize += file.FileSize
|
||||
}
|
||||
|
||||
return float64(totalSize) / float64(p.levelSizeLimits[level])
|
||||
}
|
||||
|
||||
// formatBytes 格式化字节大小显示
|
||||
func formatBytes(bytes int64) string {
|
||||
const unit = 1024
|
||||
if bytes < unit {
|
||||
return fmt.Sprintf("%d B", bytes)
|
||||
}
|
||||
div, exp := int64(unit), 0
|
||||
for n := bytes / unit; n >= unit; n /= unit {
|
||||
div *= unit
|
||||
exp++
|
||||
}
|
||||
units := []string{"KB", "MB", "GB", "TB"}
|
||||
return fmt.Sprintf("%.2f %s", float64(bytes)/float64(div), units[exp])
|
||||
}
|
||||
Reference in New Issue
Block a user