Browse Source

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 80951934c3.

* 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
pull/8594/head
Chris Lu 4 days ago
committed by GitHub
parent
commit
4c88fbfd5e
No known key found for this signature in database GPG Key ID: B5690EEEBB952194
  1. 13
      .github/workflows/e2e.yml
  2. 33
      .github/workflows/go.yml
  3. 5
      .github/workflows/metadata-subscribe-tests.yml
  4. 5
      .github/workflows/sftp-tests.yml
  5. 5
      .github/workflows/volume-server-integration-tests.yml
  6. 12
      weed/storage/disk_location.go
  7. 19
      weed/storage/volume.go
  8. 40
      weed/storage/volume_vacuum.go
  9. 2
      weed/storage/volume_write.go

13
.github/workflows/e2e.yml

@ -23,14 +23,13 @@ jobs:
runs-on: ubuntu-22.04
timeout-minutes: 30
steps:
- name: Set up Go 1.x
uses: actions/setup-go@a5f9b05d2d216f63e13859e0d847461041025775 # v2
with:
go-version: ^1.13
id: go
- name: Check out code into the Go module directory
uses: actions/checkout@8e8c483db84b4bee98b60c0593521ed34d9990e8 # v2
uses: actions/checkout@v6
- name: Set up Go
uses: actions/setup-go@v6
with:
go-version-file: 'go.mod'
- name: Set up Docker Buildx
uses: docker/setup-buildx-action@v4

33
.github/workflows/go.yml

@ -19,13 +19,12 @@ jobs:
name: Go Vet
runs-on: ubuntu-latest
steps:
- name: Set up Go 1.x
uses: actions/setup-go@a5f9b05d2d216f63e13859e0d847461041025775 # v2
with:
go-version: ^1.13
id: go
- name: Check out code into the Go module directory
uses: actions/checkout@8e8c483db84b4bee98b60c0593521ed34d9990e8 # v2
uses: actions/checkout@v6
- name: Set up Go
uses: actions/setup-go@v6
with:
go-version-file: 'go.mod'
- name: Get dependencies
run: |
cd weed; go get -v -t -d ./...
@ -42,13 +41,12 @@ jobs:
name: Build
runs-on: ubuntu-latest
steps:
- name: Set up Go 1.x
uses: actions/setup-go@a5f9b05d2d216f63e13859e0d847461041025775 # v2
with:
go-version: ^1.13
id: go
- name: Check out code into the Go module directory
uses: actions/checkout@8e8c483db84b4bee98b60c0593521ed34d9990e8 # v2
uses: actions/checkout@v6
- name: Set up Go
uses: actions/setup-go@v6
with:
go-version-file: 'go.mod'
- name: Build
run: cd weed; go build -tags "elastic gocdk sqlite ydb tarantool tikv rclone" -v .
@ -56,12 +54,11 @@ jobs:
name: Test
runs-on: ubuntu-latest
steps:
- name: Set up Go 1.x
uses: actions/setup-go@a5f9b05d2d216f63e13859e0d847461041025775 # v2
with:
go-version: ^1.13
id: go
- name: Check out code into the Go module directory
uses: actions/checkout@8e8c483db84b4bee98b60c0593521ed34d9990e8 # v2
uses: actions/checkout@v6
- name: Set up Go
uses: actions/setup-go@v6
with:
go-version-file: 'go.mod'
- name: Test
run: cd weed; go test -tags "elastic gocdk sqlite ydb tarantool tikv rclone" -v ./...

5
.github/workflows/metadata-subscribe-tests.yml

@ -30,7 +30,6 @@ permissions:
contents: read
env:
GO_VERSION: '1.24'
TEST_TIMEOUT: '10m'
jobs:
@ -43,10 +42,10 @@ jobs:
- name: Checkout code
uses: actions/checkout@v6
- name: Set up Go ${{ env.GO_VERSION }}
- name: Set up Go
uses: actions/setup-go@v6
with:
go-version: ${{ env.GO_VERSION }}
go-version-file: 'go.mod'
- name: Build SeaweedFS
run: |

5
.github/workflows/sftp-tests.yml

@ -24,7 +24,6 @@ permissions:
contents: read
env:
GO_VERSION: '1.24'
TEST_TIMEOUT: '15m'
jobs:
@ -37,10 +36,10 @@ jobs:
- name: Checkout code
uses: actions/checkout@v6
- name: Set up Go ${{ env.GO_VERSION }}
- name: Set up Go
uses: actions/setup-go@v6
with:
go-version: ${{ env.GO_VERSION }}
go-version-file: 'go.mod'
- name: Install dependencies
run: |

5
.github/workflows/volume-server-integration-tests.yml

@ -28,7 +28,6 @@ permissions:
contents: read
env:
GO_VERSION: '1.24'
TEST_TIMEOUT: '30m'
jobs:
@ -46,10 +45,10 @@ jobs:
- name: Checkout code
uses: actions/checkout@v6
- name: Set up Go ${{ env.GO_VERSION }}
- name: Set up Go
uses: actions/setup-go@v6
with:
go-version: ${{ env.GO_VERSION }}
go-version-file: 'go.mod'
- name: Build SeaweedFS binary
run: |

12
weed/storage/disk_location.go

@ -300,9 +300,13 @@ func (l *DiskLocation) DeleteCollectionFromDiskLocation(collection string) (e er
var wg sync.WaitGroup
wg.Add(2)
go func() {
for _, v := range delVolsMap {
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()
@ -383,14 +387,10 @@ func (l *DiskLocation) UnloadVolume(vid needle.VolumeId) error {
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 {
if v.Collection == collectionName && !v.isCompactionInProgress.Load() {
deltaVols[k] = v
}
}
for k := range deltaVols {
delete(l.volumes, k)
}
return deltaVols
}

19
weed/storage/volume.go

@ -5,6 +5,7 @@ import (
"path"
"strconv"
"sync"
"sync/atomic"
"time"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
@ -45,8 +46,7 @@ type Volume struct {
lastCompactRevision uint16
ldbTimeout int64
isCompacting bool
isCommitCompacting bool
isCompactionInProgress atomic.Bool
volumeInfoRWLock sync.RWMutex
volumeInfo *volume_server_pb.VolumeInfo
@ -224,6 +224,16 @@ func (v *Volume) SyncToDisk() {
// Close cleanly shuts down this volume
func (v *Volume) Close() {
// Wait for any in-progress compaction to finish and claim the flag so no
// new compaction can start. This must happen BEFORE acquiring
// dataFileAccessLock to avoid deadlocking with CommitCompact which holds
// the flag while waiting for the lock.
for !v.isCompactionInProgress.CompareAndSwap(false, true) {
time.Sleep(521 * time.Millisecond)
glog.Warningf("Volume Close wait for compaction %d", v.Id)
}
defer v.isCompactionInProgress.Store(false)
v.dataFileAccessLock.Lock()
defer v.dataFileAccessLock.Unlock()
@ -231,11 +241,6 @@ func (v *Volume) Close() {
}
func (v *Volume) doClose() {
for v.isCommitCompacting {
time.Sleep(521 * time.Millisecond)
glog.Warningf("Volume Close wait for compaction %d", v.Id)
}
if v.nm != nil {
if err := v.nm.Sync(); err != nil {
glog.Warningf("Volume Close fail to sync volume idx %d", v.Id)

40
weed/storage/volume_vacuum.go

@ -64,22 +64,26 @@ func (v *Volume) CompactByVolumeData(opts *CompactOptions) error {
//v.accessLock.Lock()
//defer v.accessLock.Unlock()
//glog.V(3).Infof("Got Compaction lock...")
if v.isCompacting || v.isCommitCompacting {
if !v.isCompactionInProgress.CompareAndSwap(false, true) {
glog.V(0).Infof("Volume %d is already compacting...", v.Id)
return nil
}
v.isCompacting = true
defer func() {
v.isCompacting = false
}()
defer v.isCompactionInProgress.Store(false)
v.lastCompactIndexOffset = v.IndexFileSize()
v.lastCompactRevision = v.SuperBlock.CompactionRevision
glog.V(3).Infof("creating copies for volume %d ,last offset %d...", v.Id, v.lastCompactIndexOffset)
if v.DataBackend == nil {
return fmt.Errorf("volume %d backend is empty remote:%v", v.Id, v.HasRemoteFile())
}
nm := v.nm
if nm == nil {
return fmt.Errorf("volume %d needle map is nil", v.Id)
}
if err := v.DataBackend.Sync(); err != nil {
glog.V(0).Infof("compact failed to sync volume %d", v.Id)
}
if err := v.nm.Sync(); err != nil {
if err := nm.Sync(); err != nil {
glog.V(0).Infof("compact failed to sync volume idx %d", v.Id)
}
@ -102,14 +106,11 @@ func (v *Volume) CompactByIndex(opts *CompactOptions) error {
}
glog.V(3).Infof("Compact2 volume %d ...", v.Id)
if v.isCompacting || v.isCommitCompacting {
if !v.isCompactionInProgress.CompareAndSwap(false, true) {
glog.V(0).Infof("Volume %d is already compacting2 ...", v.Id)
return nil
}
v.isCompacting = true
defer func() {
v.isCompacting = false
}()
defer v.isCompactionInProgress.Store(false)
v.lastCompactIndexOffset = v.IndexFileSize()
v.lastCompactRevision = v.SuperBlock.CompactionRevision
@ -117,10 +118,14 @@ func (v *Volume) CompactByIndex(opts *CompactOptions) error {
if v.DataBackend == nil {
return fmt.Errorf("volume %d backend is empty remote:%v", v.Id, v.HasRemoteFile())
}
nm := v.nm
if nm == nil {
return fmt.Errorf("volume %d needle map is nil", v.Id)
}
if err := v.DataBackend.Sync(); err != nil {
glog.V(0).Infof("compact2 failed to sync volume dat %d: %v", v.Id, err)
}
if err := v.nm.Sync(); err != nil {
if err := nm.Sync(); err != nil {
glog.V(0).Infof("compact2 failed to sync volume idx %d: %v", v.Id, err)
}
@ -139,14 +144,11 @@ func (v *Volume) CommitCompact() error {
}
glog.V(0).Infof("Committing volume %d vacuuming...", v.Id)
if v.isCommitCompacting {
glog.V(0).Infof("Volume %d is already commit compacting ...", v.Id)
if !v.isCompactionInProgress.CompareAndSwap(false, true) {
glog.V(0).Infof("Volume %d is already compacting ...", v.Id)
return nil
}
v.isCommitCompacting = true
defer func() {
v.isCommitCompacting = false
}()
defer v.isCompactionInProgress.Store(false)
v.dataFileAccessLock.Lock()
defer v.dataFileAccessLock.Unlock()
@ -530,7 +532,7 @@ func (v *Volume) copyDataBasedOnIndexFile(opts *CompactOptions) (err error) {
if err != nil {
return err
}
if v.Ttl.String() == "" {
if v.Ttl.String() == "" && v.nm != nil {
dstDatSize, _, err := dstDatBackend.GetStat()
if err != nil {
return err

2
weed/storage/volume_write.go

@ -70,7 +70,7 @@ func (v *Volume) Destroy(onlyEmpty bool) (err error) {
return
}
}
if v.isCompacting || v.isCommitCompacting {
if !v.isCompactionInProgress.CompareAndSwap(false, true) {
err = fmt.Errorf("volume %d is compacting", v.Id)
return
}

Loading…
Cancel
Save