Files
seaweedFS/weed/storage/disk_location.go
Chris Lu f5c35240be Add volume dir tags and EC placement priority (#8472)
* Add volume dir tags to topology

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Add preferred tag config for EC

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Prioritize EC destinations by tags

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Add EC placement planner tag tests

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Refactor EC placement tests to reuse buildActiveTopology

Remove buildActiveTopologyWithDiskTags helper function and consolidate
tag setup inline in test cases. Tests now use UpdateTopology to apply
tags after topology creation, reusing the existing buildActiveTopology
function rather than duplicating its logic.

All tag scenario tests pass:
- TestECPlacementPlannerPrefersTaggedDisks
- TestECPlacementPlannerFallsBackWhenTagsInsufficient

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Consolidate normalizeTagList into shared util package

Extract normalizeTagList from three locations (volume.go,
detection.go, erasure_coding_handler.go) into new weed/util/tag.go
as exported NormalizeTagList function. Replace all duplicate
implementations with imports and calls to util.NormalizeTagList.

This improves code reuse and maintainability by centralizing
tag normalization logic.

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Add PreferredTags to EC config persistence

Add preferred_tags field to ErasureCodingTaskConfig protobuf with field
number 5. Update GetConfigSpec to include preferred_tags field in the
UI configuration schema. Add PreferredTags to ToTaskPolicy to serialize
config to protobuf. Add PreferredTags to FromTaskPolicy to deserialize
from protobuf with defensive copy to prevent external mutation.

This allows EC preferred tags to be persisted and restored across
worker restarts.

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Add defensive copy for Tags slice in DiskLocation

Copy the incoming tags slice in NewDiskLocation instead of storing
by reference. This prevents external callers from mutating the
DiskLocation.Tags slice after construction, improving encapsulation
and preventing unexpected changes to disk metadata.

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Add doc comment to buildCandidateSets method

Document the tiered candidate selection and fallback behavior. Explain
that for a planner with preferredTags, it accumulates disks matching
each tag in order into progressively larger tiers, emits a candidate
set once a tier reaches shardsNeeded, and finally falls back to the
full candidates set if preferred-tag tiers are insufficient.

This clarifies the intended semantics for future maintainers.

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Apply final PR review fixes

1. Update parseVolumeTags to replicate single tag entry to all folders
   instead of leaving some folders with nil tags. This prevents nil
   pointer dereferences when processing folders without explicit tags.

2. Add defensive copy in ToTaskPolicy for PreferredTags slice to match
   the pattern used in FromTaskPolicy, preventing external mutation of
   the returned TaskPolicy.

3. Add clarifying comment in buildCandidateSets explaining that the
   shardsNeeded <= 0 branch is a defensive check for direct callers,
   since selectDestinations guarantees shardsNeeded > 0.

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Fix nil pointer dereference in parseVolumeTags

Ensure all folder tags are initialized to either normalized tags or
empty slices, not nil. When multiple tag entries are provided and there
are more folders than entries, remaining folders now get empty slices
instead of nil, preventing nil pointer dereference in downstream code.

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Fix NormalizeTagList to return empty slice instead of nil

Change NormalizeTagList to always return a non-nil slice. When all tags
are empty or whitespace after normalization, return an empty slice
instead of nil. This prevents nil pointer dereferences in downstream
code that expects a valid (possibly empty) slice.

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Add nil safety check for v.tags pointer

Add a safety check to handle the case where v.tags might be nil,
preventing a nil pointer dereference. If v.tags is nil, use an empty
string instead. This is defensive programming to prevent panics in
edge cases.

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

* Add volume.tags flag to weed server and weed mini commands

Add the volume.tags CLI option to both the 'weed server' and 'weed mini'
commands. This allows users to specify disk tags when running the
combined server modes, just like they can with 'weed volume'.

The flag uses the same format and description as the volume command:
comma-separated tag groups per data dir with ':' separators
(e.g. fast:ssd,archive).

Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>

---------

Co-authored-by: Copilot <copilot@github.com>
Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>
2026-03-01 10:22:00 -08:00

545 lines
14 KiB
Go

package storage
import (
"fmt"
"os"
"path/filepath"
"runtime"
"slices"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/google/uuid"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/stats"
"github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
"github.com/seaweedfs/seaweedfs/weed/storage/types"
"github.com/seaweedfs/seaweedfs/weed/util"
)
const (
UUIDFileName = "vol_dir.uuid"
UUIDFileMod = 0644
)
type DiskLocation struct {
Directory string
DirectoryUuid string
IdxDirectory string
DiskType types.DiskType
Tags []string
MaxVolumeCount int32
OriginalMaxVolumeCount int32
MinFreeSpace util.MinFreeSpace
AvailableSpace atomic.Uint64
volumes map[needle.VolumeId]*Volume
volumesLock sync.RWMutex
// erasure coding
ecVolumes map[needle.VolumeId]*erasure_coding.EcVolume
ecVolumesLock sync.RWMutex
ecShardNotifyHandler func(collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, ecVolume *erasure_coding.EcVolume)
isDiskSpaceLow bool
closeCh chan struct{}
}
func GenerateDirUuid(dir string) (dirUuidString string, err error) {
glog.V(1).Infof("Getting uuid of volume directory:%s", dir)
fileName := filepath.Join(dir, UUIDFileName)
if !util.FileExists(fileName) {
dirUuidString, err = writeNewUuid(fileName)
} else {
uuidData, readErr := os.ReadFile(fileName)
if readErr != nil {
return "", fmt.Errorf("failed to read uuid from %s : %v", fileName, readErr)
}
if len(uuidData) > 0 {
dirUuidString = string(uuidData)
} else {
dirUuidString, err = writeNewUuid(fileName)
}
}
return dirUuidString, err
}
func writeNewUuid(fileName string) (string, error) {
dirUuid, _ := uuid.NewRandom()
dirUuidString := dirUuid.String()
if err := util.WriteFile(fileName, []byte(dirUuidString), UUIDFileMod); err != nil {
return "", fmt.Errorf("failed to write uuid to %s : %v", fileName, err)
}
return dirUuidString, nil
}
func NewDiskLocation(dir string, maxVolumeCount int32, minFreeSpace util.MinFreeSpace, idxDir string, diskType types.DiskType, tags []string) *DiskLocation {
glog.V(4).Infof("Added new Disk %s: maxVolumes=%d", dir, maxVolumeCount)
dir = util.ResolvePath(dir)
if idxDir == "" {
idxDir = dir
} else {
idxDir = util.ResolvePath(idxDir)
}
dirUuid, err := GenerateDirUuid(dir)
if err != nil {
glog.Fatalf("cannot generate uuid of dir %s: %v", dir, err)
}
// Defensive copy of tags to prevent external mutation
var copiedTags []string
if len(tags) > 0 {
copiedTags = make([]string, len(tags))
copy(copiedTags, tags)
}
location := &DiskLocation{
Directory: dir,
DirectoryUuid: dirUuid,
IdxDirectory: idxDir,
DiskType: diskType,
Tags: copiedTags,
MaxVolumeCount: maxVolumeCount,
OriginalMaxVolumeCount: maxVolumeCount,
MinFreeSpace: minFreeSpace,
}
location.volumes = make(map[needle.VolumeId]*Volume)
location.ecVolumes = make(map[needle.VolumeId]*erasure_coding.EcVolume)
location.closeCh = make(chan struct{})
go func() {
location.CheckDiskSpace()
for {
select {
case <-location.closeCh:
return
case <-time.After(time.Minute):
location.CheckDiskSpace()
}
}
}()
return location
}
func volumeIdFromFileName(filename string) (needle.VolumeId, string, error) {
if isValidVolume(filename) {
base := filename[:len(filename)-4]
collection, volumeId, err := parseCollectionVolumeId(base)
return volumeId, collection, err
}
return 0, "", fmt.Errorf("file is not a volume: %s", filename)
}
func parseCollectionVolumeId(base string) (collection string, vid needle.VolumeId, err error) {
i := strings.LastIndex(base, "_")
if i > 0 {
collection, base = base[0:i], base[i+1:]
}
vol, err := needle.NewVolumeId(base)
return collection, vol, err
}
func isValidVolume(basename string) bool {
return strings.HasSuffix(basename, ".idx") || strings.HasSuffix(basename, ".vif")
}
func getValidVolumeName(basename string) string {
if isValidVolume(basename) {
return basename[:len(basename)-4]
}
return ""
}
func (l *DiskLocation) loadExistingVolume(dirEntry os.DirEntry, needleMapKind NeedleMapKind, skipIfEcVolumesExists bool, ldbTimeout int64, diskId uint32) bool {
basename := dirEntry.Name()
if dirEntry.IsDir() {
return false
}
volumeName := getValidVolumeName(basename)
if volumeName == "" {
return false
}
// parse out collection, volume id (moved up to use in EC validation)
vid, collection, err := volumeIdFromFileName(basename)
if err != nil {
glog.Warningf("get volume id failed, %s, err : %s", volumeName, err)
return false
}
// skip if ec volumes exists, but validate EC files first
if skipIfEcVolumesExists {
ecxFilePath := filepath.Join(l.IdxDirectory, volumeName+".ecx")
if util.FileExists(ecxFilePath) {
// Validate EC volume: shard count, size consistency, and expected size vs .dat file
if !l.validateEcVolume(collection, vid) {
glog.Warningf("EC volume %d validation failed, removing incomplete EC files to allow .dat file loading", vid)
l.removeEcVolumeFiles(collection, vid)
// Continue to load .dat file
} else {
// Valid EC volume exists, skip .dat file
return false
}
}
}
// check for incomplete volume
noteFile := l.Directory + "/" + volumeName + ".note"
if util.FileExists(noteFile) {
note, _ := os.ReadFile(noteFile)
glog.Warningf("volume %s was not completed: %s", volumeName, string(note))
removeVolumeFiles(l.Directory + "/" + volumeName)
removeVolumeFiles(l.IdxDirectory + "/" + volumeName)
return false
}
// avoid loading one volume more than once
l.volumesLock.RLock()
_, found := l.volumes[vid]
l.volumesLock.RUnlock()
if found {
glog.V(1).Infof("loaded volume, %v", vid)
return true
}
// load the volume
v, e := NewVolume(l.Directory, l.IdxDirectory, collection, vid, needleMapKind, nil, nil, 0, needle.GetCurrentVersion(), 0, ldbTimeout)
if e != nil {
glog.V(0).Infof("new volume %s error %s", volumeName, e)
return false
}
v.diskId = diskId // Set the disk ID for existing volumes
l.SetVolume(vid, v)
size, _, _ := v.FileStat()
glog.V(2).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, diskId uint32) {
task_queue := make(chan os.DirEntry, 10*concurrency)
go func() {
foundVolumeNames := make(map[string]bool)
if dirEntries, err := os.ReadDir(l.Directory); err == nil {
for _, entry := range dirEntries {
volumeName := getValidVolumeName(entry.Name())
if volumeName == "" {
continue
}
if _, found := foundVolumeNames[volumeName]; !found {
foundVolumeNames[volumeName] = true
task_queue <- entry
}
}
}
close(task_queue)
}()
var wg sync.WaitGroup
for workerNum := 0; workerNum < concurrency; workerNum++ {
wg.Add(1)
go func() {
defer wg.Done()
for fi := range task_queue {
_ = l.loadExistingVolume(fi, needleMapKind, true, ldbTimeout, diskId)
}
}()
}
wg.Wait()
}
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")
if ok {
num, err := strconv.Atoi(val)
if err != nil || num < 1 {
num = 10
glog.Warningf("failed to set worker number from GOMAXPROCS , set to default:10")
}
workerNum = num
} else {
if workerNum <= 10 {
workerNum = 10
}
}
l.concurrentLoadingVolumes(needleMapKind, workerNum, ldbTimeout, diskId)
glog.V(2).Infof("Store started on dir: %s with %d volumes max %d (disk ID: %d)", l.Directory, len(l.volumes), l.MaxVolumeCount, diskId)
l.loadAllEcShards(l.ecShardNotifyHandler)
glog.V(2).Infof("Store started on dir: %s with %d ec shards (disk ID: %d)", l.Directory, len(l.ecVolumes), diskId)
}
func (l *DiskLocation) DeleteCollectionFromDiskLocation(collection string) (e error) {
l.volumesLock.Lock()
delVolsMap := l.unmountVolumeByCollection(collection)
l.volumesLock.Unlock()
l.ecVolumesLock.Lock()
delEcVolsMap := l.unmountEcVolumeByCollection(collection)
l.ecVolumesLock.Unlock()
errChain := make(chan error, 2)
var wg sync.WaitGroup
wg.Add(2)
go func() {
for _, v := range delVolsMap {
if err := v.Destroy(false); err != nil {
errChain <- err
}
}
wg.Done()
}()
go func() {
for _, v := range delEcVolsMap {
v.Destroy()
}
wg.Done()
}()
go func() {
wg.Wait()
close(errChain)
}()
errBuilder := strings.Builder{}
for err := range errChain {
errBuilder.WriteString(err.Error())
errBuilder.WriteString("; ")
}
if errBuilder.Len() > 0 {
e = fmt.Errorf("%s", errBuilder.String())
}
return
}
func (l *DiskLocation) deleteVolumeById(vid needle.VolumeId, onlyEmpty bool) (found bool, e error) {
v, ok := l.volumes[vid]
if !ok {
return
}
e = v.Destroy(onlyEmpty)
if e != nil {
return
}
found = true
delete(l.volumes, vid)
return
}
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, diskId)
}
return false
}
var ErrVolumeNotFound = fmt.Errorf("volume not found")
func (l *DiskLocation) DeleteVolume(vid needle.VolumeId, onlyEmpty bool) error {
l.volumesLock.Lock()
defer l.volumesLock.Unlock()
_, ok := l.volumes[vid]
if !ok {
return ErrVolumeNotFound
}
_, err := l.deleteVolumeById(vid, onlyEmpty)
return err
}
func (l *DiskLocation) UnloadVolume(vid needle.VolumeId) error {
l.volumesLock.Lock()
defer l.volumesLock.Unlock()
v, ok := l.volumes[vid]
if !ok {
return ErrVolumeNotFound
}
v.Close()
delete(l.volumes, vid)
return nil
}
func (l *DiskLocation) unmountVolumeByCollection(collectionName string) map[needle.VolumeId]*Volume {
deltaVols := make(map[needle.VolumeId]*Volume, 0)
for k, v := range l.volumes {
if v.Collection == collectionName && !v.isCompacting && !v.isCommitCompacting {
deltaVols[k] = v
}
}
for k := range deltaVols {
delete(l.volumes, k)
}
return deltaVols
}
func (l *DiskLocation) SetVolume(vid needle.VolumeId, volume *Volume) {
l.volumesLock.Lock()
defer l.volumesLock.Unlock()
l.volumes[vid] = volume
volume.location = l
}
func (l *DiskLocation) FindVolume(vid needle.VolumeId) (*Volume, bool) {
l.volumesLock.RLock()
defer l.volumesLock.RUnlock()
v, ok := l.volumes[vid]
return v, ok
}
// Returns all regular volume IDs stored at this location.
func (l *DiskLocation) VolumeIds() []needle.VolumeId {
l.volumesLock.RLock()
defer l.volumesLock.RUnlock()
vids := make([]needle.VolumeId, len(l.volumes))
i := 0
for vid := range l.volumes {
vids[i] = vid
i++
}
slices.Sort(vids)
return vids
}
// Returns all EC volume IDs stored at this location.
func (l *DiskLocation) EcVolumeIds() []needle.VolumeId {
l.ecVolumesLock.RLock()
defer l.ecVolumesLock.RUnlock()
vids := make([]needle.VolumeId, len(l.ecVolumes))
i := 0
for vid := range l.ecVolumes {
vids[i] = vid
i++
}
slices.Sort(vids)
return vids
}
func (l *DiskLocation) VolumesLen() int {
l.volumesLock.RLock()
defer l.volumesLock.RUnlock()
return len(l.volumes)
}
func (l *DiskLocation) LocalVolumesLen() int {
l.volumesLock.RLock()
defer l.volumesLock.RUnlock()
count := 0
for _, v := range l.volumes {
if !v.HasRemoteFile() {
count++
}
}
return count
}
func (l *DiskLocation) SetStopping() {
l.volumesLock.Lock()
for _, v := range l.volumes {
v.SyncToDisk()
}
l.volumesLock.Unlock()
return
}
func (l *DiskLocation) Close() {
l.volumesLock.Lock()
for _, v := range l.volumes {
v.Close()
}
l.volumesLock.Unlock()
l.ecVolumesLock.Lock()
for _, ecVolume := range l.ecVolumes {
ecVolume.Close()
}
l.ecVolumesLock.Unlock()
close(l.closeCh)
return
}
func (l *DiskLocation) LocateVolume(vid needle.VolumeId) (os.DirEntry, bool) {
// println("LocateVolume", vid, "on", l.Directory)
if dirEntries, err := os.ReadDir(l.Directory); err == nil {
for _, entry := range dirEntries {
// println("checking", entry.Name(), "...")
volId, _, err := volumeIdFromFileName(entry.Name())
// println("volId", volId, "err", err)
if vid == volId && err == nil {
return entry, true
}
}
}
return nil, false
}
func (l *DiskLocation) UnUsedSpace(volumeSizeLimit uint64) (unUsedSpace uint64) {
l.volumesLock.RLock()
defer l.volumesLock.RUnlock()
for _, vol := range l.volumes {
if vol.IsReadOnly() {
continue
}
datSize, idxSize, _ := vol.FileStat()
unUsedSpaceVolume := int64(volumeSizeLimit) - int64(datSize+idxSize)
glog.V(4).Infof("Volume stats for %d: volumeSizeLimit=%d, datSize=%d idxSize=%d unused=%d", vol.Id, volumeSizeLimit, datSize, idxSize, unUsedSpaceVolume)
if unUsedSpaceVolume >= 0 {
unUsedSpace += uint64(unUsedSpaceVolume)
}
}
return
}
func (l *DiskLocation) CheckDiskSpace() {
if dir, e := filepath.Abs(l.Directory); e == nil {
s := stats.NewDiskStatus(dir)
available := l.MinFreeSpace.AvailableSpace(s.Free, s.All)
stats.VolumeServerResourceGauge.WithLabelValues(l.Directory, "all").Set(float64(s.All))
stats.VolumeServerResourceGauge.WithLabelValues(l.Directory, "used").Set(float64(s.Used))
stats.VolumeServerResourceGauge.WithLabelValues(l.Directory, "free").Set(float64(s.Free))
stats.VolumeServerResourceGauge.WithLabelValues(l.Directory, "avail").Set(float64(available))
l.AvailableSpace.Store(available)
isLow, desc := l.MinFreeSpace.IsLow(s.Free, s.PercentFree)
if isLow != l.isDiskSpaceLow {
l.isDiskSpaceLow = !l.isDiskSpaceLow
}
logLevel := glog.Level(4)
if l.isDiskSpaceLow {
logLevel = glog.Level(0)
}
glog.V(logLevel).Infof("dir %s %s", dir, desc)
}
}