Files
seaweedFS/weed/storage/disk_location.go
Chris Lu 4c88fbfd5e Fix nil pointer crash during concurrent vacuum compaction (#8592)
* check for nil needle map before compaction sync

When CommitCompact runs concurrently, it sets v.nm = nil under
dataFileAccessLock. CompactByIndex does not hold that lock, so
v.nm.Sync() can hit a nil pointer. Add an early nil check to
return an error instead of crashing.

Fixes #8591

* guard copyDataBasedOnIndexFile size check against nil needle map

The post-compaction size validation at line 538 accesses
v.nm.ContentSize() and v.nm.DeletedSize(). If CommitCompact has
concurrently set v.nm to nil, this causes a SIGSEGV. Skip the
validation when v.nm is nil since the actual data copy uses local
needle maps (oldNm/newNm) and is unaffected.

Fixes #8591

* use atomic.Bool for compaction flags to prevent concurrent vacuum races

The isCompacting and isCommitCompacting flags were plain bools
read and written from multiple goroutines without synchronization.
This allowed concurrent vacuums on the same volume to pass the
guard checks and run simultaneously, leading to the nil pointer
crash. Using atomic.Bool with CompareAndSwap ensures only one
compaction or commit can run per volume at a time.

Fixes #8591

* use go-version-file in CI workflows instead of hardcoded versions

Use go-version-file: 'go.mod' so CI automatically picks up the Go
version from go.mod, avoiding future version drift. Reordered
checkout before setup-go in go.yml and e2e.yml so go.mod is
available. Removed the now-unused GO_VERSION env vars.

* capture v.nm locally in CompactByIndex to close TOCTOU race

A bare nil check on v.nm followed by v.nm.Sync() has a race window
where CommitCompact can set v.nm = nil between the two. Snapshot
the pointer into a local variable so the nil check and Sync operate
on the same reference.

* add dynamic timeouts to plugin worker vacuum gRPC calls

All vacuum gRPC calls used context.Background() with no deadline,
so the plugin scheduler's execution timeout could kill a job while
a large volume compact was still in progress. Use volume-size-scaled
timeouts matching the topology vacuum approach: 3 min/GB for compact,
1 min/GB for check, commit, and cleanup.

Fixes #8591

* Revert "add dynamic timeouts to plugin worker vacuum gRPC calls"

This reverts commit 80951934c37416bc4f6c1472a5d3f8d204a637d9.

* unify compaction lifecycle into single atomic flag

Replace separate isCompacting and isCommitCompacting flags with a
single isCompactionInProgress atomic.Bool. This ensures CompactBy*,
CommitCompact, Close, and Destroy are mutually exclusive — only one
can run at a time per volume.

Key changes:
- All entry points use CompareAndSwap(false, true) to claim exclusive
  access. CompactByVolumeData and CompactByIndex now also guard v.nm
  and v.DataBackend with local captures.
- Close() waits for the flag outside dataFileAccessLock to avoid
  deadlocking with CommitCompact (which holds the flag while waiting
  for the lock). It claims the flag before acquiring the lock so no
  new compaction can start.
- Destroy() uses CAS instead of a racy Load check, preventing
  concurrent compaction from racing with volume teardown.
- unmountVolumeByCollection no longer deletes from the map;
  DeleteCollectionFromDiskLocation removes entries only after
  successful Destroy, preventing orphaned volumes on failure.

Fixes #8591
2026-03-10 13:31:45 -07:00

549 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) && l.IdxDirectory != l.Directory {
// .ecx may have been created before -dir.idx was configured
ecxFilePath = filepath.Join(l.Directory, 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 k, v := range delVolsMap {
if err := v.Destroy(false); err != nil {
errChain <- err
} else {
l.volumesLock.Lock()
delete(l.volumes, k)
l.volumesLock.Unlock()
}
}
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.isCompactionInProgress.Load() {
deltaVols[k] = v
}
}
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)
}
}