Admin: misc improvements on admin server and workers. EC now works. (#7055)

* initial design

* added simulation as tests

* reorganized the codebase to move the simulation framework and tests into their own dedicated package

* integration test. ec worker task

* remove "enhanced" reference

* start master, volume servers, filer

Current Status
 Master: Healthy and running (port 9333)
 Filer: Healthy and running (port 8888)
 Volume Servers: All 6 servers running (ports 8080-8085)
🔄 Admin/Workers: Will start when dependencies are ready

* generate write load

* tasks are assigned

* admin start wtih grpc port. worker has its own working directory

* Update .gitignore

* working worker and admin. Task detection is not working yet.

* compiles, detection uses volumeSizeLimitMB from master

* compiles

* worker retries connecting to admin

* build and restart

* rendering pending tasks

* skip task ID column

* sticky worker id

* test canScheduleTaskNow

* worker reconnect to admin

* clean up logs

* worker register itself first

* worker can run ec work and report status

but:
1. one volume should not be repeatedly worked on.
2. ec shards needs to be distributed and source data should be deleted.

* move ec task logic

* listing ec shards

* local copy, ec. Need to distribute.

* ec is mostly working now

* distribution of ec shards needs improvement
* need configuration to enable ec

* show ec volumes

* interval field UI component

* rename

* integration test with vauuming

* garbage percentage threshold

* fix warning

* display ec shard sizes

* fix ec volumes list

* Update ui.go

* show default values

* ensure correct default value

* MaintenanceConfig use ConfigField

* use schema defined defaults

* config

* reduce duplication

* refactor to use BaseUIProvider

* each task register its schema

* checkECEncodingCandidate use ecDetector

* use vacuumDetector

* use volumeSizeLimitMB

* remove

remove

* remove unused

* refactor

* use new framework

* remove v2 reference

* refactor

* left menu can scroll now

* The maintenance manager was not being initialized when no data directory was configured for persistent storage.

* saving config

* Update task_config_schema_templ.go

* enable/disable tasks

* protobuf encoded task configurations

* fix system settings

* use ui component

* remove logs

* interface{} Reduction

* reduce interface{}

* reduce interface{}

* avoid from/to map

* reduce interface{}

* refactor

* keep it DRY

* added logging

* debug messages

* debug level

* debug

* show the log caller line

* use configured task policy

* log level

* handle admin heartbeat response

* Update worker.go

* fix EC rack and dc count

* Report task status to admin server

* fix task logging, simplify interface checking, use erasure_coding constants

* factor in empty volume server during task planning

* volume.list adds disk id

* track disk id also

* fix locking scheduled and manual scanning

* add active topology

* simplify task detector

* ec task completed, but shards are not showing up

* implement ec in ec_typed.go

* adjust log level

* dedup

* implementing ec copying shards and only ecx files

* use disk id when distributing ec shards

🎯 Planning: ActiveTopology creates DestinationPlan with specific TargetDisk
📦 Task Creation: maintenance_integration.go creates ECDestination with DiskId
🚀 Task Execution: EC task passes DiskId in VolumeEcShardsCopyRequest
💾 Volume Server: Receives disk_id and stores shards on specific disk (vs.store.Locations[req.DiskId])
📂 File System: EC shards and metadata land in the exact disk directory planned

* Delete original volume from all locations

* clean up existing shard locations

* local encoding and distributing

* Update docker/admin_integration/EC-TESTING-README.md

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

* check volume id range

* simplify

* fix tests

* fix types

* clean up logs and tests

---------

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
This commit is contained in:
Chris Lu
2025-07-30 12:38:03 -07:00
committed by GitHub
parent 64198dad83
commit 891a2fb6eb
130 changed files with 27737 additions and 4429 deletions

View File

@@ -134,7 +134,7 @@ func getValidVolumeName(basename string) string {
return ""
}
func (l *DiskLocation) loadExistingVolume(dirEntry os.DirEntry, needleMapKind NeedleMapKind, skipIfEcVolumesExists bool, ldbTimeout int64) bool {
func (l *DiskLocation) loadExistingVolume(dirEntry os.DirEntry, needleMapKind NeedleMapKind, skipIfEcVolumesExists bool, ldbTimeout int64, diskId uint32) bool {
basename := dirEntry.Name()
if dirEntry.IsDir() {
return false
@@ -184,15 +184,16 @@ func (l *DiskLocation) loadExistingVolume(dirEntry os.DirEntry, needleMapKind Ne
return false
}
v.diskId = diskId // Set the disk ID for existing volumes
l.SetVolume(vid, v)
size, _, _ := v.FileStat()
glog.V(0).Infof("data file %s, replication=%s v=%d size=%d ttl=%s",
l.Directory+"/"+volumeName+".dat", v.ReplicaPlacement, v.Version(), size, v.Ttl.String())
glog.V(0).Infof("data file %s, replication=%s v=%d size=%d ttl=%s disk_id=%d",
l.Directory+"/"+volumeName+".dat", v.ReplicaPlacement, v.Version(), size, v.Ttl.String(), diskId)
return true
}
func (l *DiskLocation) concurrentLoadingVolumes(needleMapKind NeedleMapKind, concurrency int, ldbTimeout int64) {
func (l *DiskLocation) concurrentLoadingVolumes(needleMapKind NeedleMapKind, concurrency int, ldbTimeout int64, diskId uint32) {
task_queue := make(chan os.DirEntry, 10*concurrency)
go func() {
@@ -218,7 +219,7 @@ func (l *DiskLocation) concurrentLoadingVolumes(needleMapKind NeedleMapKind, con
go func() {
defer wg.Done()
for fi := range task_queue {
_ = l.loadExistingVolume(fi, needleMapKind, true, ldbTimeout)
_ = l.loadExistingVolume(fi, needleMapKind, true, ldbTimeout, diskId)
}
}()
}
@@ -227,6 +228,10 @@ func (l *DiskLocation) concurrentLoadingVolumes(needleMapKind NeedleMapKind, con
}
func (l *DiskLocation) loadExistingVolumes(needleMapKind NeedleMapKind, ldbTimeout int64) {
l.loadExistingVolumesWithId(needleMapKind, ldbTimeout, 0) // Default disk ID for backward compatibility
}
func (l *DiskLocation) loadExistingVolumesWithId(needleMapKind NeedleMapKind, ldbTimeout int64, diskId uint32) {
workerNum := runtime.NumCPU()
val, ok := os.LookupEnv("GOMAXPROCS")
@@ -242,11 +247,11 @@ func (l *DiskLocation) loadExistingVolumes(needleMapKind NeedleMapKind, ldbTimeo
workerNum = 10
}
}
l.concurrentLoadingVolumes(needleMapKind, workerNum, ldbTimeout)
glog.V(0).Infof("Store started on dir: %s with %d volumes max %d", l.Directory, len(l.volumes), l.MaxVolumeCount)
l.concurrentLoadingVolumes(needleMapKind, workerNum, ldbTimeout, diskId)
glog.V(0).Infof("Store started on dir: %s with %d volumes max %d (disk ID: %d)", l.Directory, len(l.volumes), l.MaxVolumeCount, diskId)
l.loadAllEcShards()
glog.V(0).Infof("Store started on dir: %s with %d ec shards", l.Directory, len(l.ecVolumes))
glog.V(0).Infof("Store started on dir: %s with %d ec shards (disk ID: %d)", l.Directory, len(l.ecVolumes), diskId)
}
@@ -310,9 +315,9 @@ func (l *DiskLocation) deleteVolumeById(vid needle.VolumeId, onlyEmpty bool) (fo
return
}
func (l *DiskLocation) LoadVolume(vid needle.VolumeId, needleMapKind NeedleMapKind) bool {
func (l *DiskLocation) LoadVolume(diskId uint32, vid needle.VolumeId, needleMapKind NeedleMapKind) bool {
if fileInfo, found := l.LocateVolume(vid); found {
return l.loadExistingVolume(fileInfo, needleMapKind, false, 0)
return l.loadExistingVolume(fileInfo, needleMapKind, false, 0, diskId)
}
return false
}

View File

@@ -196,7 +196,22 @@ func (ev *EcVolume) ShardIdList() (shardIds []ShardId) {
return
}
func (ev *EcVolume) ToVolumeEcShardInformationMessage() (messages []*master_pb.VolumeEcShardInformationMessage) {
type ShardInfo struct {
ShardId ShardId
Size int64
}
func (ev *EcVolume) ShardDetails() (shards []ShardInfo) {
for _, s := range ev.Shards {
shards = append(shards, ShardInfo{
ShardId: s.ShardId,
Size: s.Size(),
})
}
return
}
func (ev *EcVolume) ToVolumeEcShardInformationMessage(diskId uint32) (messages []*master_pb.VolumeEcShardInformationMessage) {
prevVolumeId := needle.VolumeId(math.MaxUint32)
var m *master_pb.VolumeEcShardInformationMessage
for _, s := range ev.Shards {
@@ -206,6 +221,7 @@ func (ev *EcVolume) ToVolumeEcShardInformationMessage() (messages []*master_pb.V
Collection: s.Collection,
DiskType: string(ev.diskType),
ExpireAtSec: ev.ExpireAtSec,
DiskId: diskId,
}
messages = append(messages, m)
}

View File

@@ -11,15 +11,17 @@ type EcVolumeInfo struct {
Collection string
ShardBits ShardBits
DiskType string
ExpireAtSec uint64 //ec volume destroy time, calculated from the ec volume was created
DiskId uint32 // ID of the disk this EC volume is on
ExpireAtSec uint64 // ec volume destroy time, calculated from the ec volume was created
}
func NewEcVolumeInfo(diskType string, collection string, vid needle.VolumeId, shardBits ShardBits, expireAtSec uint64) *EcVolumeInfo {
func NewEcVolumeInfo(diskType string, collection string, vid needle.VolumeId, shardBits ShardBits, expireAtSec uint64, diskId uint32) *EcVolumeInfo {
return &EcVolumeInfo{
Collection: collection,
VolumeId: vid,
ShardBits: shardBits,
DiskType: diskType,
DiskId: diskId,
ExpireAtSec: expireAtSec,
}
}
@@ -62,6 +64,7 @@ func (ecInfo *EcVolumeInfo) ToVolumeEcShardInformationMessage() (ret *master_pb.
Collection: ecInfo.Collection,
DiskType: ecInfo.DiskType,
ExpireAtSec: ecInfo.ExpireAtSec,
DiskId: ecInfo.DiskId,
}
}

View File

@@ -91,11 +91,12 @@ func NewStore(grpcDialOption grpc.DialOption, ip string, port int, grpcPort int,
s.Locations = append(s.Locations, location)
stats.VolumeServerMaxVolumeCounter.Add(float64(maxVolumeCounts[i]))
diskId := uint32(i) // Track disk ID
wg.Add(1)
go func() {
go func(id uint32, diskLoc *DiskLocation) {
defer wg.Done()
location.loadExistingVolumes(needleMapKind, ldbTimeout)
}()
diskLoc.loadExistingVolumesWithId(needleMapKind, ldbTimeout, id)
}(diskId, location)
}
wg.Wait()
@@ -163,14 +164,25 @@ func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind
if s.findVolume(vid) != nil {
return fmt.Errorf("Volume Id %d already exists!", vid)
}
if location := s.FindFreeLocation(func(location *DiskLocation) bool {
return location.DiskType == diskType
}); location != nil {
glog.V(0).Infof("In dir %s adds volume:%v collection:%s replicaPlacement:%v ttl:%v",
location.Directory, vid, collection, replicaPlacement, ttl)
// Find location and its index
var location *DiskLocation
var diskId uint32
for i, loc := range s.Locations {
if loc.DiskType == diskType && s.hasFreeDiskLocation(loc) {
location = loc
diskId = uint32(i)
break
}
}
if location != nil {
glog.V(0).Infof("In dir %s (disk ID %d) adds volume:%v collection:%s replicaPlacement:%v ttl:%v",
location.Directory, diskId, vid, collection, replicaPlacement, ttl)
if volume, err := NewVolume(location.Directory, location.IdxDirectory, collection, vid, needleMapKind, replicaPlacement, ttl, preallocate, ver, memoryMapMaxSizeMb, ldbTimeout); err == nil {
volume.diskId = diskId // Set the disk ID
location.SetVolume(vid, volume)
glog.V(0).Infof("add volume %d", vid)
glog.V(0).Infof("add volume %d on disk ID %d", vid, diskId)
s.NewVolumesChan <- master_pb.VolumeShortInformationMessage{
Id: uint32(vid),
Collection: collection,
@@ -178,6 +190,7 @@ func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind
Version: uint32(volume.Version()),
Ttl: ttl.ToUint32(),
DiskType: string(diskType),
DiskId: diskId,
}
return nil
} else {
@@ -187,6 +200,11 @@ func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind
return fmt.Errorf("No more free space left")
}
// hasFreeDiskLocation checks if a disk location has free space
func (s *Store) hasFreeDiskLocation(location *DiskLocation) bool {
return int64(location.VolumesLen()) < int64(location.MaxVolumeCount)
}
func (s *Store) VolumeInfos() (allStats []*VolumeInfo) {
for _, location := range s.Locations {
stats := collectStatsForOneLocation(location)
@@ -218,21 +236,10 @@ func collectStatForOneVolume(vid needle.VolumeId, v *Volume) (s *VolumeInfo) {
Ttl: v.Ttl,
CompactRevision: uint32(v.CompactionRevision),
DiskType: v.DiskType().String(),
DiskId: v.diskId,
}
s.RemoteStorageName, s.RemoteStorageKey = v.RemoteStorageNameKey()
v.dataFileAccessLock.RLock()
defer v.dataFileAccessLock.RUnlock()
if v.nm == nil {
return
}
s.FileCount = v.nm.FileCount()
s.DeleteCount = v.nm.DeletedCount()
s.DeletedByteCount = v.nm.DeletedSize()
s.Size = v.nm.ContentSize()
s.Size, _, _ = v.FileStat()
return
}
@@ -384,7 +391,7 @@ func (s *Store) CollectHeartbeat() *master_pb.Heartbeat {
}
func (s *Store) deleteExpiredEcVolumes() (ecShards, deleted []*master_pb.VolumeEcShardInformationMessage) {
for _, location := range s.Locations {
for diskId, location := range s.Locations {
// Collect ecVolume to be deleted
var toDeleteEvs []*erasure_coding.EcVolume
location.ecVolumesLock.RLock()
@@ -392,7 +399,7 @@ func (s *Store) deleteExpiredEcVolumes() (ecShards, deleted []*master_pb.VolumeE
if ev.IsTimeToDestroy() {
toDeleteEvs = append(toDeleteEvs, ev)
} else {
messages := ev.ToVolumeEcShardInformationMessage()
messages := ev.ToVolumeEcShardInformationMessage(uint32(diskId))
ecShards = append(ecShards, messages...)
}
}
@@ -400,7 +407,7 @@ func (s *Store) deleteExpiredEcVolumes() (ecShards, deleted []*master_pb.VolumeE
// Delete expired volumes
for _, ev := range toDeleteEvs {
messages := ev.ToVolumeEcShardInformationMessage()
messages := ev.ToVolumeEcShardInformationMessage(uint32(diskId))
// deleteEcVolumeById has its own lock
err := location.deleteEcVolumeById(ev.VolumeId)
if err != nil {
@@ -515,10 +522,11 @@ func (s *Store) MarkVolumeWritable(i needle.VolumeId) error {
}
func (s *Store) MountVolume(i needle.VolumeId) error {
for _, location := range s.Locations {
if found := location.LoadVolume(i, s.NeedleMapKind); found == true {
for diskId, location := range s.Locations {
if found := location.LoadVolume(uint32(diskId), i, s.NeedleMapKind); found == true {
glog.V(0).Infof("mount volume %d", i)
v := s.findVolume(i)
v.diskId = uint32(diskId) // Set disk ID when mounting
s.NewVolumesChan <- master_pb.VolumeShortInformationMessage{
Id: uint32(v.Id),
Collection: v.Collection,
@@ -526,6 +534,7 @@ func (s *Store) MountVolume(i needle.VolumeId) error {
Version: uint32(v.Version()),
Ttl: v.Ttl.ToUint32(),
DiskType: string(v.location.DiskType),
DiskId: uint32(diskId),
}
return nil
}
@@ -546,6 +555,7 @@ func (s *Store) UnmountVolume(i needle.VolumeId) error {
Version: uint32(v.Version()),
Ttl: v.Ttl.ToUint32(),
DiskType: string(v.location.DiskType),
DiskId: v.diskId,
}
for _, location := range s.Locations {
@@ -574,6 +584,7 @@ func (s *Store) DeleteVolume(i needle.VolumeId, onlyEmpty bool) error {
Version: uint32(v.Version()),
Ttl: v.Ttl.ToUint32(),
DiskType: string(v.location.DiskType),
DiskId: v.diskId,
}
for _, location := range s.Locations {
err := location.DeleteVolume(i, onlyEmpty)

View File

@@ -25,10 +25,10 @@ import (
func (s *Store) CollectErasureCodingHeartbeat() *master_pb.Heartbeat {
var ecShardMessages []*master_pb.VolumeEcShardInformationMessage
collectionEcShardSize := make(map[string]int64)
for _, location := range s.Locations {
for diskId, location := range s.Locations {
location.ecVolumesLock.RLock()
for _, ecShards := range location.ecVolumes {
ecShardMessages = append(ecShardMessages, ecShards.ToVolumeEcShardInformationMessage()...)
ecShardMessages = append(ecShardMessages, ecShards.ToVolumeEcShardInformationMessage(uint32(diskId))...)
for _, ecShard := range ecShards.Shards {
collectionEcShardSize[ecShards.Collection] += ecShard.Size()
@@ -49,9 +49,9 @@ func (s *Store) CollectErasureCodingHeartbeat() *master_pb.Heartbeat {
}
func (s *Store) MountEcShards(collection string, vid needle.VolumeId, shardId erasure_coding.ShardId) error {
for _, location := range s.Locations {
for diskId, location := range s.Locations {
if ecVolume, err := location.LoadEcShard(collection, vid, shardId); err == nil {
glog.V(0).Infof("MountEcShards %d.%d", vid, shardId)
glog.V(0).Infof("MountEcShards %d.%d on disk ID %d", vid, shardId, diskId)
var shardBits erasure_coding.ShardBits
@@ -61,6 +61,7 @@ func (s *Store) MountEcShards(collection string, vid needle.VolumeId, shardId er
EcIndexBits: uint32(shardBits.AddShardId(shardId)),
DiskType: string(location.DiskType),
ExpireAtSec: ecVolume.ExpireAtSec,
DiskId: uint32(diskId),
}
return nil
} else if err == os.ErrNotExist {
@@ -75,7 +76,7 @@ func (s *Store) MountEcShards(collection string, vid needle.VolumeId, shardId er
func (s *Store) UnmountEcShards(vid needle.VolumeId, shardId erasure_coding.ShardId) error {
ecShard, found := s.findEcShard(vid, shardId)
diskId, ecShard, found := s.findEcShard(vid, shardId)
if !found {
return nil
}
@@ -86,26 +87,27 @@ func (s *Store) UnmountEcShards(vid needle.VolumeId, shardId erasure_coding.Shar
Collection: ecShard.Collection,
EcIndexBits: uint32(shardBits.AddShardId(shardId)),
DiskType: string(ecShard.DiskType),
DiskId: diskId,
}
for _, location := range s.Locations {
if deleted := location.UnloadEcShard(vid, shardId); deleted {
glog.V(0).Infof("UnmountEcShards %d.%d", vid, shardId)
s.DeletedEcShardsChan <- message
return nil
}
location := s.Locations[diskId]
if deleted := location.UnloadEcShard(vid, shardId); deleted {
glog.V(0).Infof("UnmountEcShards %d.%d", vid, shardId)
s.DeletedEcShardsChan <- message
return nil
}
return fmt.Errorf("UnmountEcShards %d.%d not found on disk", vid, shardId)
}
func (s *Store) findEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId) (*erasure_coding.EcVolumeShard, bool) {
for _, location := range s.Locations {
func (s *Store) findEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId) (diskId uint32, shard *erasure_coding.EcVolumeShard, found bool) {
for diskId, location := range s.Locations {
if v, found := location.FindEcShard(vid, shardId); found {
return v, found
return uint32(diskId), v, found
}
}
return nil, false
return 0, nil, false
}
func (s *Store) FindEcVolume(vid needle.VolumeId) (*erasure_coding.EcVolume, bool) {

View File

@@ -51,6 +51,7 @@ type Volume struct {
volumeInfoRWLock sync.RWMutex
volumeInfo *volume_server_pb.VolumeInfo
location *DiskLocation
diskId uint32 // ID of this volume's disk in Store.Locations array
lastIoError error
}
@@ -337,6 +338,7 @@ func (v *Volume) ToVolumeInformationMessage() (types.NeedleId, *master_pb.Volume
CompactRevision: uint32(v.SuperBlock.CompactionRevision),
ModifiedAtSecond: modTime.Unix(),
DiskType: string(v.location.DiskType),
DiskId: v.diskId,
}
volumeInfo.RemoteStorageName, volumeInfo.RemoteStorageKey = v.RemoteStorageNameKey()

View File

@@ -15,6 +15,7 @@ type VolumeInfo struct {
ReplicaPlacement *super_block.ReplicaPlacement
Ttl *needle.TTL
DiskType string
DiskId uint32
Collection string
Version needle.Version
FileCount int
@@ -42,6 +43,7 @@ func NewVolumeInfo(m *master_pb.VolumeInformationMessage) (vi VolumeInfo, err er
RemoteStorageName: m.RemoteStorageName,
RemoteStorageKey: m.RemoteStorageKey,
DiskType: m.DiskType,
DiskId: m.DiskId,
}
rp, e := super_block.NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
if e != nil {
@@ -94,6 +96,7 @@ func (vi VolumeInfo) ToVolumeInformationMessage() *master_pb.VolumeInformationMe
RemoteStorageName: vi.RemoteStorageName,
RemoteStorageKey: vi.RemoteStorageKey,
DiskType: vi.DiskType,
DiskId: vi.DiskId,
}
}