mount: async flush on close() when writebackCache is enabled (#8727)
* mount: async flush on close() when writebackCache is enabled When -writebackCache is enabled, defer data upload and metadata flush from Flush() (triggered by close()) to a background goroutine in Release(). This allows processes like rsync that write many small files to proceed to the next file immediately instead of blocking on two network round-trips (volume upload + filer metadata) per file. Fixes #8718 * mount: add retry with backoff for async metadata flush The metadata flush in completeAsyncFlush now retries up to 3 times with exponential backoff (1s, 2s, 4s) on transient gRPC errors. Since the chunk data is already safely on volume servers at this point, only the filer metadata reference needs persisting — retrying is both safe and effective. Data flush (FlushData) is not retried externally because UploadWithRetry already handles transient HTTP/gRPC errors internally; if it still fails, the chunk memory has been freed. * test: add integration tests for writebackCache async flush Add comprehensive FUSE integration tests for the writebackCache async flush feature (issue #8718): - Basic operations: write/read, sequential files, large files, empty files, overwrites - Fsync correctness: fsync forces synchronous flush even in writeback mode, immediate read-after-fsync - Concurrent small files: multi-worker parallel writes (rsync-like workload), multi-directory, rapid create/close - Data integrity: append after close, partial writes, file size correctness, binary data preservation - Performance comparison: writeback vs synchronous flush throughput - Stress test: 16 workers x 100 files with content verification - Mixed concurrent operations: reads, writes, creates running together Also fix pre-existing test infrastructure issues: - Rename framework.go to framework_test.go (fixes Go package conflict) - Fix undefined totalSize variable in concurrent_operations_test.go * ci: update fuse-integration workflow to run full test suite The workflow previously only ran placeholder tests (simple_test.go, working_demo_test.go) in a temp directory due to a Go module conflict. Now that framework.go is renamed to framework_test.go, the full test suite compiles and runs correctly from test/fuse_integration/. Changes: - Run go test directly in test/fuse_integration/ (no temp dir copy) - Install weed binary to /usr/local/bin for test framework discovery - Configure /etc/fuse.conf with user_allow_other for FUSE mounts - Install fuse3 for modern FUSE support - Stream test output to log file for artifact upload * mount: fix three P1 races in async flush P1-1: Reopen overwrites data still flushing in background ReleaseByHandle removes the old handle from fhMap before the deferred flush finishes. A reopen of the same inode during that window would build from stale filer metadata, overwriting the async flush. Fix: Track in-flight async flushes per inode via pendingAsyncFlush map. AcquireHandle now calls waitForPendingAsyncFlush(inode) to block until any pending flush completes before reading filer metadata. P1-2: Deferred flush races rename and unlink after close completeAsyncFlush captured the path once at entry, but rename or unlink after close() could cause metadata to be written under the wrong name or recreate a deleted file. Fix: Re-resolve path from inode via GetPath right before metadata flush. GetPath returns the current path (reflecting renames) or ENOENT (if unlinked), in which case we skip the metadata flush. P1-3: SIGINT/SIGTERM bypasses the async-flush drain grace.OnInterrupt runs hooks then calls os.Exit(0), so WaitForAsyncFlush after server.Serve() never executes on signal. Fix: Add WaitForAsyncFlush (with 10s timeout) to the WFS interrupt handler, before cache cleanup. The timeout prevents hanging on Ctrl-C when the filer is unreachable. * mount: fix P1 races — draining handle stays in fhMap P1-1: Reopen TOCTOU The gap between ReleaseByHandle removing from fhMap and submitAsyncFlush registering in pendingAsyncFlush allowed a concurrent AcquireHandle to slip through with stale metadata. Fix: Hold pendingAsyncFlushMu across both the counter decrement (ReleaseByHandle) and the pending registration. The handle is registered as pending before the lock is released, so waitForPendingAsyncFlush always sees it. P1-2: Rename/unlink can't find draining handle ReleaseByHandle deleted from fhMap immediately. Rename's FindFileHandle(inode) at line 251 could not find the handle to update entry.Name. Unlink could not coordinate either. Fix: When asyncFlushPending is true, ReleaseByHandle/ReleaseByInode leave the handle in fhMap (counter=0 but maps intact). The handle stays visible to FindFileHandle so rename can update entry.Name. completeAsyncFlush re-resolves the path from the inode (GetPath) right before metadata flush for correctness after rename/unlink. After drain, RemoveFileHandle cleans up the maps. Double-return prevention: ReleaseByHandle/ReleaseByInode return nil if counter is already <= 0, so Forget after Release doesn't start a second drain goroutine. P1-3: SIGINT deletes swap files under running goroutines After the 10s timeout, os.RemoveAll deleted the write cache dir (containing swap files) while FlushData goroutines were still reading from them. Fix: Increase timeout to 30s. If timeout expires, skip write cache dir removal so in-flight goroutines can finish reading swap files. The OS (or next mount) cleans them up. Read cache is always removed. * mount: never skip metadata flush when Forget drops inode mapping Forget removes the inode→path mapping when the kernel's lookup count reaches zero, but this does NOT mean the file was unlinked — it only means the kernel evicted its cache entry. completeAsyncFlush was treating GetPath failure as "file unlinked" and skipping the metadata flush, which orphaned the just-uploaded chunks for live files. Fix: Save dir and name at doFlush defer time. In completeAsyncFlush, try GetPath first to pick up renames; if the mapping is gone, fall back to the saved dir/name. Always attempt the metadata flush — the filer is the authority on whether the file exists, not the local inode cache. * mount: distinguish Forget from Unlink in async flush path fallback The saved-path fallback (from the previous fix) always flushed metadata when GetPath failed, which recreated files that were explicitly unlinked after close(). The same stale fallback could recreate the pre-rename path if Forget dropped the inode mapping after a rename. Root cause: GetPath failure has two meanings: 1. Forget — kernel evicted the cache entry (file still exists) 2. Unlink — file was explicitly deleted (should not recreate) Fix (three coordinated changes): Unlink (weedfs_file_mkrm.go): Before RemovePath, look up the inode and find any draining handle via FindFileHandle. Set fh.isDeleted = true so the async flush knows the file was explicitly removed. Rename (weedfs_rename.go): When renaming a file with a draining handle, update asyncFlushDir/asyncFlushName to the post-rename location. This keeps the saved-path fallback current so Forget after rename doesn't flush to the old (pre-rename) path. completeAsyncFlush (weedfs_async_flush.go): Check fh.isDeleted first — if true, skip metadata flush (file was unlinked, chunks become orphans for volume.fsck). Otherwise, try GetPath for the current path (renames); fall back to saved path if Forget dropped the mapping (file is live, just evicted from kernel cache). * test/ci: address PR review nitpicks concurrent_operations_test.go: - Restore precise totalSize assertion instead of info.Size() > 0 writeback_cache_test.go: - Check rand.Read errors in all 3 locations (lines 310, 512, 757) - Check os.MkdirAll error in stress test (line 752) - Remove dead verifyErrors variable (line 332) - Replace both time.Sleep(5s) with polling via waitForFileContent to avoid flaky tests under CI load (lines 638, 700) fuse-integration.yml: - Add set -o pipefail so go test failures propagate through tee * ci: fix fuse3/fuse package conflict on ubuntu-22.04 runner fuse3 is pre-installed on ubuntu-22.04 runners and conflicts with the legacy fuse package. Only install libfuse3-dev for the headers. * mount/page_writer: remove debug println statements Remove leftover debug println("read new data1/2") from ReadDataAt in MemChunk and SwapFileChunk. * test: fix findWeedBinary matching source directory instead of binary findWeedBinary() matched ../../weed (the source directory) via os.Stat before checking PATH, then tried to exec a directory which fails with "permission denied" on the CI runner. Fix: Check PATH first (reliable in CI where the binary is installed to /usr/local/bin). For relative paths, verify the candidate is a regular file (!info.IsDir()). Add ../../weed/weed as a candidate for in-tree builds. * test: fix framework — dynamic ports, output capture, data dirs The integration test framework was failing in CI because: 1. All tests used hardcoded ports (19333/18080/18888), so sequential tests could conflict when prior processes hadn't fully released their ports yet. 2. Data subdirectories (data/master, data/volume) were not created before starting processes. 3. Master was started with -peers=none which is not a valid address. 4. Process stdout/stderr was not captured, making failures opaque ("service not ready within timeout" with no diagnostics). 5. The unmount fallback used 'umount' instead of 'fusermount -u'. 6. The mount used -cacheSizeMB (nonexistent) instead of -cacheCapacityMB and was missing -allowOthers=false for unprivileged CI runners. Fixes: - Dynamic port allocation via freePort() (net.Listen ":0") - Explicit gRPC ports via -port.grpc to avoid default port conflicts - Create data/master and data/volume directories in Setup() - Remove invalid -peers=none and -raftBootstrap flags - Capture process output to logDir/*.log via startProcess() helper - dumpLog() prints tail of log file on service startup failure - Use fusermount3/fusermount -u for unmount - Fix mount flag names (-cacheCapacityMB, -allowOthers=false) * test: remove explicit -port.grpc flags from test framework SeaweedFS convention: gRPC port = HTTP port + 10000. Volume and filer discover the master gRPC port by this convention. Setting explicit -port.grpc on master/volume/filer broke inter-service communication because the volume server computed master gRPC as HTTP+10000 but the actual gRPC was on a different port. Remove all -port.grpc flags and let the default convention work. Dynamic HTTP ports already ensure uniqueness; the derived gRPC ports (HTTP+10000) will also be unique. --------- Co-authored-by: Copilot <copilot@github.com>
This commit is contained in:
@@ -349,6 +349,7 @@ func RunMount(option *MountOptions, umask os.FileMode) bool {
|
||||
RdmaMaxConcurrent: *option.rdmaMaxConcurrent,
|
||||
RdmaTimeoutMs: *option.rdmaTimeoutMs,
|
||||
DirIdleEvictSec: *option.dirIdleEvictSec,
|
||||
WritebackCache: option.writebackCache != nil && *option.writebackCache,
|
||||
})
|
||||
|
||||
// create mount root
|
||||
@@ -396,6 +397,10 @@ func RunMount(option *MountOptions, umask os.FileMode) bool {
|
||||
|
||||
server.Serve()
|
||||
|
||||
// Wait for any pending background flushes (writebackCache async mode)
|
||||
// before clearing caches, to prevent data loss during clean unmount.
|
||||
seaweedFileSystem.WaitForAsyncFlush()
|
||||
|
||||
seaweedFileSystem.ClearCacheDir()
|
||||
|
||||
return true
|
||||
|
||||
@@ -24,10 +24,15 @@ type FileHandle struct {
|
||||
wfs *WFS
|
||||
|
||||
// cache file has been written to
|
||||
dirtyMetadata bool
|
||||
dirtyPages *PageWriter
|
||||
reader *filer.ChunkReadAt
|
||||
contentType string
|
||||
dirtyMetadata bool
|
||||
dirtyPages *PageWriter
|
||||
reader *filer.ChunkReadAt
|
||||
contentType string
|
||||
asyncFlushPending bool // set in writebackCache mode to defer flush to Release
|
||||
asyncFlushUid uint32 // saved uid for deferred metadata flush
|
||||
asyncFlushGid uint32 // saved gid for deferred metadata flush
|
||||
asyncFlushDir string // saved directory at defer time (fallback if inode forgotten)
|
||||
asyncFlushName string // saved file name at defer time (fallback if inode forgotten)
|
||||
|
||||
isDeleted bool
|
||||
|
||||
|
||||
@@ -55,39 +55,72 @@ func (i *FileHandleToInode) AcquireFileHandle(wfs *WFS, inode uint64, entry *fil
|
||||
return fh
|
||||
}
|
||||
|
||||
func (i *FileHandleToInode) ReleaseByInode(inode uint64) {
|
||||
func (i *FileHandleToInode) ReleaseByInode(inode uint64) *FileHandle {
|
||||
i.Lock()
|
||||
defer i.Unlock()
|
||||
fh, found := i.inode2fh[inode]
|
||||
if found {
|
||||
fh.counter--
|
||||
if fh.counter <= 0 {
|
||||
delete(i.inode2fh, inode)
|
||||
delete(i.fh2inode, fh.fh)
|
||||
fh.ReleaseHandle()
|
||||
}
|
||||
if !found {
|
||||
return nil
|
||||
}
|
||||
// If the counter is already <= 0, a prior Release already started the
|
||||
// drain. Return nil to prevent double-processing (e.g. Forget after Release).
|
||||
if fh.counter <= 0 {
|
||||
return nil
|
||||
}
|
||||
fh.counter--
|
||||
if fh.counter <= 0 {
|
||||
if fh.asyncFlushPending {
|
||||
// Handle stays in fhMap so rename/unlink can find it during drain.
|
||||
return fh
|
||||
}
|
||||
delete(i.inode2fh, inode)
|
||||
delete(i.fh2inode, fh.fh)
|
||||
return fh
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *FileHandleToInode) ReleaseByHandle(fh FileHandleId) {
|
||||
func (i *FileHandleToInode) ReleaseByHandle(fh FileHandleId) *FileHandle {
|
||||
i.Lock()
|
||||
defer i.Unlock()
|
||||
|
||||
inode, found := i.fh2inode[fh]
|
||||
if !found {
|
||||
return // Handle already released or invalid
|
||||
return nil
|
||||
}
|
||||
|
||||
fhHandle, fhFound := i.inode2fh[inode]
|
||||
if !fhFound {
|
||||
delete(i.fh2inode, fh)
|
||||
return
|
||||
return nil
|
||||
}
|
||||
|
||||
// If the counter is already <= 0, a prior Release already started the
|
||||
// drain. Return nil to prevent double-processing.
|
||||
if fhHandle.counter <= 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
fhHandle.counter--
|
||||
if fhHandle.counter <= 0 {
|
||||
if fhHandle.asyncFlushPending {
|
||||
// Handle stays in fhMap so rename/unlink can still find it
|
||||
// via FindFileHandle during the background drain.
|
||||
return fhHandle
|
||||
}
|
||||
delete(i.inode2fh, inode)
|
||||
delete(i.fh2inode, fhHandle.fh)
|
||||
fhHandle.ReleaseHandle()
|
||||
return fhHandle
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// RemoveFileHandle removes a handle from both maps. Called after an async
|
||||
// drain completes to clean up the handle that was intentionally kept in the
|
||||
// maps during the flush.
|
||||
func (i *FileHandleToInode) RemoveFileHandle(fh FileHandleId, inode uint64) {
|
||||
i.Lock()
|
||||
defer i.Unlock()
|
||||
delete(i.inode2fh, inode)
|
||||
delete(i.fh2inode, fh)
|
||||
}
|
||||
|
||||
@@ -65,10 +65,6 @@ func (mc *MemChunk) ReadDataAt(p []byte, off int64, tsNs int64) (maxStop int64)
|
||||
if logicStart < logicStop {
|
||||
copy(p[logicStart-off:logicStop-off], mc.buf[logicStart-memChunkBaseOffset:logicStop-memChunkBaseOffset])
|
||||
maxStop = max(maxStop, logicStop)
|
||||
|
||||
if t.TsNs >= tsNs {
|
||||
println("read new data1", t.TsNs-tsNs, "ns")
|
||||
}
|
||||
}
|
||||
}
|
||||
mc.activityScore.MarkRead()
|
||||
|
||||
@@ -151,10 +151,6 @@ func (sc *SwapFileChunk) ReadDataAt(p []byte, off int64, tsNs int64) (maxStop in
|
||||
break
|
||||
}
|
||||
maxStop = max(maxStop, logicStop)
|
||||
|
||||
if t.TsNs > tsNs {
|
||||
println("read new data2", t.TsNs-tsNs, "ns")
|
||||
}
|
||||
}
|
||||
}
|
||||
//sc.memChunk.ReadDataAt(memCopy, off, tsNs)
|
||||
|
||||
@@ -78,6 +78,10 @@ type Option struct {
|
||||
// Directory cache refresh/eviction controls
|
||||
DirIdleEvictSec int
|
||||
|
||||
// WritebackCache enables async flush on close for improved small file write performance.
|
||||
// When true, Flush() returns immediately and data upload + metadata flush happen in background.
|
||||
WritebackCache bool
|
||||
|
||||
uniqueCacheDirForRead string
|
||||
uniqueCacheDirForWrite string
|
||||
}
|
||||
@@ -110,6 +114,16 @@ type WFS struct {
|
||||
dirHotWindow time.Duration
|
||||
dirHotThreshold int
|
||||
dirIdleEvict time.Duration
|
||||
|
||||
// asyncFlushWg tracks pending background flush goroutines for writebackCache mode.
|
||||
// Must be waited on before unmount cleanup to prevent data loss.
|
||||
asyncFlushWg sync.WaitGroup
|
||||
|
||||
// pendingAsyncFlush tracks in-flight async flush goroutines by inode.
|
||||
// AcquireHandle checks this to wait for a pending flush before reopening
|
||||
// the same inode, preventing stale metadata from overwriting the async flush.
|
||||
pendingAsyncFlushMu sync.Mutex
|
||||
pendingAsyncFlush map[uint64]chan struct{}
|
||||
}
|
||||
|
||||
const (
|
||||
@@ -151,13 +165,14 @@ func NewSeaweedFileSystem(option *Option) *WFS {
|
||||
}
|
||||
|
||||
wfs := &WFS{
|
||||
RawFileSystem: fuse.NewDefaultRawFileSystem(),
|
||||
option: option,
|
||||
signature: util.RandomInt32(),
|
||||
inodeToPath: NewInodeToPath(util.FullPath(option.FilerMountRootPath), option.CacheMetaTTlSec),
|
||||
fhMap: NewFileHandleToInode(),
|
||||
dhMap: NewDirectoryHandleToInode(),
|
||||
filerClient: filerClient, // nil for proxy mode, initialized for direct access
|
||||
RawFileSystem: fuse.NewDefaultRawFileSystem(),
|
||||
option: option,
|
||||
signature: util.RandomInt32(),
|
||||
inodeToPath: NewInodeToPath(util.FullPath(option.FilerMountRootPath), option.CacheMetaTTlSec),
|
||||
fhMap: NewFileHandleToInode(),
|
||||
dhMap: NewDirectoryHandleToInode(),
|
||||
filerClient: filerClient, // nil for proxy mode, initialized for direct access
|
||||
pendingAsyncFlush: make(map[uint64]chan struct{}),
|
||||
fhLockTable: util.NewLockTable[FileHandleId](),
|
||||
refreshingDirs: make(map[util.FullPath]struct{}),
|
||||
dirHotWindow: dirHotWindow,
|
||||
@@ -204,8 +219,32 @@ func NewSeaweedFileSystem(option *Option) *WFS {
|
||||
}
|
||||
})
|
||||
grace.OnInterrupt(func() {
|
||||
// grace calls os.Exit(0) after all hooks, so WaitForAsyncFlush
|
||||
// after server.Serve() would never execute. Drain here first.
|
||||
//
|
||||
// Use a timeout to avoid hanging on Ctrl-C if the filer is
|
||||
// unreachable (metadata retry can take up to 7 seconds).
|
||||
// If the timeout expires, skip the write-cache removal so that
|
||||
// still-running goroutines can finish reading swap files.
|
||||
asyncDrained := true
|
||||
if wfs.option.WritebackCache {
|
||||
done := make(chan struct{})
|
||||
go func() {
|
||||
wfs.asyncFlushWg.Wait()
|
||||
close(done)
|
||||
}()
|
||||
select {
|
||||
case <-done:
|
||||
glog.V(0).Infof("all async flushes completed before shutdown")
|
||||
case <-time.After(30 * time.Second):
|
||||
glog.Warningf("timed out waiting for async flushes — swap files preserved for in-flight uploads")
|
||||
asyncDrained = false
|
||||
}
|
||||
}
|
||||
wfs.metaCache.Shutdown()
|
||||
os.RemoveAll(option.getUniqueCacheDirForWrite())
|
||||
if asyncDrained {
|
||||
os.RemoveAll(option.getUniqueCacheDirForWrite())
|
||||
}
|
||||
os.RemoveAll(option.getUniqueCacheDirForRead())
|
||||
if wfs.rdmaClient != nil {
|
||||
wfs.rdmaClient.Close()
|
||||
@@ -240,6 +279,10 @@ func NewSeaweedFileSystem(option *Option) *WFS {
|
||||
}
|
||||
|
||||
func (wfs *WFS) StartBackgroundTasks() error {
|
||||
if wfs.option.WritebackCache {
|
||||
glog.V(0).Infof("writebackCache enabled: async flush on close() for improved small file performance")
|
||||
}
|
||||
|
||||
follower, err := wfs.subscribeFilerConfEvents()
|
||||
if err != nil {
|
||||
return err
|
||||
|
||||
96
weed/mount/weedfs_async_flush.go
Normal file
96
weed/mount/weedfs_async_flush.go
Normal file
@@ -0,0 +1,96 @@
|
||||
package mount
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/go-fuse/v2/fuse"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
)
|
||||
|
||||
const asyncFlushMetadataRetries = 3
|
||||
|
||||
// completeAsyncFlush is called in a background goroutine when a file handle
|
||||
// with pending async flush work is released. It performs the deferred data
|
||||
// upload and metadata flush that was skipped in doFlush() for writebackCache mode.
|
||||
//
|
||||
// This enables close() to return immediately for small file workloads (e.g., rsync),
|
||||
// while the actual I/O happens concurrently in the background.
|
||||
//
|
||||
// The caller (submitAsyncFlush) owns asyncFlushWg and the per-inode done channel.
|
||||
func (wfs *WFS) completeAsyncFlush(fh *FileHandle) {
|
||||
// Phase 1: Flush dirty pages — seals writable chunks, uploads to volume servers, and waits.
|
||||
// The underlying UploadWithRetry already retries transient HTTP/gRPC errors internally,
|
||||
// so a failure here indicates a persistent issue; the chunk data has been freed.
|
||||
if err := fh.dirtyPages.FlushData(); err != nil {
|
||||
glog.Errorf("completeAsyncFlush inode %d: data flush failed: %v", fh.inode, err)
|
||||
// Data is lost at this point (chunks freed after internal retry exhaustion).
|
||||
// Proceed to cleanup to avoid resource leaks and unmount hangs.
|
||||
} else if fh.dirtyMetadata {
|
||||
// Phase 2: Flush metadata unless the file was explicitly unlinked.
|
||||
//
|
||||
// isDeleted is set by the Unlink handler when it finds a draining
|
||||
// handle. In that case the filer entry is already gone and
|
||||
// flushing would recreate it. The uploaded chunks become orphans
|
||||
// and are cleaned up by volume.fsck.
|
||||
if fh.isDeleted {
|
||||
glog.V(3).Infof("completeAsyncFlush inode %d: file was unlinked, skipping metadata flush", fh.inode)
|
||||
} else {
|
||||
// Resolve the current path for metadata flush.
|
||||
//
|
||||
// Try GetPath first — it reflects any rename that happened
|
||||
// after close(). If the inode mapping is gone (Forget
|
||||
// dropped it after the kernel's lookup count hit zero), fall
|
||||
// back to the dir/name saved at doFlush time. Rename also
|
||||
// updates the saved path, so the fallback is always current.
|
||||
//
|
||||
// Forget does NOT mean the file was deleted — it only means
|
||||
// the kernel evicted its cache entry.
|
||||
dir, name := fh.asyncFlushDir, fh.asyncFlushName
|
||||
fileFullPath := util.FullPath(dir).Child(name)
|
||||
|
||||
if resolvedPath, status := wfs.inodeToPath.GetPath(fh.inode); status == fuse.OK {
|
||||
dir, name = resolvedPath.DirAndName()
|
||||
fileFullPath = resolvedPath
|
||||
}
|
||||
|
||||
wfs.flushMetadataWithRetry(fh, dir, name, fileFullPath)
|
||||
}
|
||||
}
|
||||
|
||||
glog.V(3).Infof("completeAsyncFlush done inode %d fh %d", fh.inode, fh.fh)
|
||||
|
||||
// Phase 3: Destroy the upload pipeline and free resources.
|
||||
fh.ReleaseHandle()
|
||||
}
|
||||
|
||||
// flushMetadataWithRetry attempts to flush file metadata to the filer, retrying
|
||||
// with exponential backoff on transient errors. The chunk data is already on the
|
||||
// volume servers at this point; only the filer metadata reference needs persisting.
|
||||
func (wfs *WFS) flushMetadataWithRetry(fh *FileHandle, dir, name string, fileFullPath util.FullPath) {
|
||||
for attempt := 0; attempt <= asyncFlushMetadataRetries; attempt++ {
|
||||
if attempt > 0 {
|
||||
backoff := time.Duration(1<<uint(attempt-1)) * time.Second
|
||||
glog.Warningf("completeAsyncFlush %s: retrying metadata flush (attempt %d/%d) after %v",
|
||||
fileFullPath, attempt+1, asyncFlushMetadataRetries+1, backoff)
|
||||
time.Sleep(backoff)
|
||||
}
|
||||
|
||||
if err := wfs.flushMetadataToFiler(fh, dir, name, fh.asyncFlushUid, fh.asyncFlushGid); err != nil {
|
||||
if attempt == asyncFlushMetadataRetries {
|
||||
glog.Errorf("completeAsyncFlush %s: metadata flush failed after %d attempts: %v — "+
|
||||
"chunks are uploaded but NOT referenced in filer metadata; "+
|
||||
"they will appear as orphans in volume.fsck",
|
||||
fileFullPath, asyncFlushMetadataRetries+1, err)
|
||||
}
|
||||
continue
|
||||
}
|
||||
return // success
|
||||
}
|
||||
}
|
||||
|
||||
// WaitForAsyncFlush waits for all pending background flush goroutines to complete.
|
||||
// Called before unmount cleanup to ensure no data is lost.
|
||||
func (wfs *WFS) WaitForAsyncFlush() {
|
||||
wfs.asyncFlushWg.Wait()
|
||||
}
|
||||
@@ -161,6 +161,15 @@ func (wfs *WFS) Unlink(cancel <-chan struct{}, header *fuse.InHeader, name strin
|
||||
}
|
||||
wfs.inodeToPath.TouchDirectory(dirFullPath)
|
||||
|
||||
// If there is an async-draining handle for this file, mark it as deleted
|
||||
// so the background flush skips the metadata write instead of recreating
|
||||
// the just-unlinked entry. The handle is still in fhMap during drain.
|
||||
if inode, found := wfs.inodeToPath.GetInode(entryFullPath); found {
|
||||
if fh, fhFound := wfs.fhMap.FindFileHandle(inode); fhFound {
|
||||
fh.isDeleted = true
|
||||
}
|
||||
}
|
||||
|
||||
wfs.inodeToPath.RemovePath(entryFullPath)
|
||||
|
||||
return fuse.OK
|
||||
|
||||
@@ -59,7 +59,7 @@ func (wfs *WFS) Flush(cancel <-chan struct{}, in *fuse.FlushIn) fuse.Status {
|
||||
return fuse.OK
|
||||
}
|
||||
|
||||
return wfs.doFlush(fh, in.Uid, in.Gid)
|
||||
return wfs.doFlush(fh, in.Uid, in.Gid, true)
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -88,11 +88,12 @@ func (wfs *WFS) Fsync(cancel <-chan struct{}, in *fuse.FsyncIn) (code fuse.Statu
|
||||
return fuse.ENOENT
|
||||
}
|
||||
|
||||
return wfs.doFlush(fh, in.Uid, in.Gid)
|
||||
// Fsync is an explicit sync request — always flush synchronously
|
||||
return wfs.doFlush(fh, in.Uid, in.Gid, false)
|
||||
|
||||
}
|
||||
|
||||
func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
|
||||
func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32, allowAsync bool) fuse.Status {
|
||||
|
||||
// flush works at fh level
|
||||
fileFullPath := fh.FullPath()
|
||||
@@ -100,6 +101,26 @@ func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
|
||||
// send the data to the OS
|
||||
glog.V(4).Infof("doFlush %s fh %d", fileFullPath, fh.fh)
|
||||
|
||||
// When writebackCache is enabled and this is a close()-triggered Flush (not fsync),
|
||||
// defer the expensive data upload + metadata flush to a background goroutine.
|
||||
// This allows the calling process (e.g., rsync) to proceed to the next file immediately.
|
||||
// POSIX does not require close() to wait for delayed I/O to complete.
|
||||
if allowAsync && wfs.option.WritebackCache && fh.dirtyMetadata {
|
||||
if wfs.IsOverQuotaWithUncommitted() {
|
||||
return fuse.Status(syscall.ENOSPC)
|
||||
}
|
||||
fh.asyncFlushPending = true
|
||||
fh.asyncFlushUid = uid
|
||||
fh.asyncFlushGid = gid
|
||||
fh.asyncFlushDir = dir
|
||||
fh.asyncFlushName = name
|
||||
glog.V(3).Infof("doFlush async deferred %s fh %d", fileFullPath, fh.fh)
|
||||
return fuse.OK
|
||||
}
|
||||
|
||||
// Synchronous flush path (normal mode, fsync, or no dirty data)
|
||||
fh.asyncFlushPending = false
|
||||
|
||||
// Check quota including uncommitted writes for real-time enforcement
|
||||
isOverQuota := wfs.IsOverQuotaWithUncommitted()
|
||||
if !isOverQuota {
|
||||
@@ -117,6 +138,23 @@ func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
|
||||
return fuse.Status(syscall.ENOSPC)
|
||||
}
|
||||
|
||||
if err := wfs.flushMetadataToFiler(fh, dir, name, uid, gid); err != nil {
|
||||
glog.Errorf("%v fh %d flush: %v", fileFullPath, fh.fh, err)
|
||||
return fuse.EIO
|
||||
}
|
||||
|
||||
if IsDebugFileReadWrite {
|
||||
fh.mirrorFile.Sync()
|
||||
}
|
||||
|
||||
return fuse.OK
|
||||
}
|
||||
|
||||
// flushMetadataToFiler sends the file's chunk references and attributes to the filer.
|
||||
// This is shared between the synchronous doFlush path and the async flush completion.
|
||||
func (wfs *WFS) flushMetadataToFiler(fh *FileHandle, dir, name string, uid, gid uint32) error {
|
||||
fileFullPath := fh.FullPath()
|
||||
|
||||
fhActiveLock := fh.wfs.fhLockTable.AcquireLock("doFlush", fh.fh, util.ExclusiveLock)
|
||||
defer fh.wfs.fhLockTable.ReleaseLock(fh.fh, fhActiveLock)
|
||||
|
||||
@@ -144,9 +182,6 @@ func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
|
||||
}
|
||||
|
||||
glog.V(4).Infof("%s set chunks: %v", fileFullPath, len(entry.GetChunks()))
|
||||
//for i, chunk := range entry.GetChunks() {
|
||||
// glog.V(4).Infof("%s chunks %d: %v [%d,%d)", fileFullPath, i, chunk.GetFileIdString(), chunk.Offset, chunk.Offset+int64(chunk.Size))
|
||||
//}
|
||||
|
||||
manifestChunks, nonManifestChunks := filer.SeparateManifestChunks(entry.GetChunks())
|
||||
|
||||
@@ -183,14 +218,5 @@ func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
|
||||
fh.dirtyMetadata = false
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
glog.Errorf("%v fh %d flush: %v", fileFullPath, fh.fh, err)
|
||||
return fuse.EIO
|
||||
}
|
||||
|
||||
if IsDebugFileReadWrite {
|
||||
fh.mirrorFile.Sync()
|
||||
}
|
||||
|
||||
return fuse.OK
|
||||
return err
|
||||
}
|
||||
|
||||
@@ -2,11 +2,18 @@ package mount
|
||||
|
||||
import (
|
||||
"github.com/seaweedfs/go-fuse/v2/fuse"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
)
|
||||
|
||||
func (wfs *WFS) AcquireHandle(inode uint64, flags, uid, gid uint32) (fileHandle *FileHandle, status fuse.Status) {
|
||||
// If there is an in-flight async flush for this inode, wait for it to
|
||||
// complete before reopening. Otherwise the new handle would be built
|
||||
// from pre-close filer metadata and its next flush could overwrite the
|
||||
// data that was just written asynchronously.
|
||||
wfs.waitForPendingAsyncFlush(inode)
|
||||
|
||||
var entry *filer_pb.Entry
|
||||
var path util.FullPath
|
||||
path, _, entry, status = wfs.maybeReadEntry(inode)
|
||||
@@ -20,8 +27,59 @@ func (wfs *WFS) AcquireHandle(inode uint64, flags, uid, gid uint32) (fileHandle
|
||||
return
|
||||
}
|
||||
|
||||
// ReleaseHandle is called from FUSE Release. For handles with a pending
|
||||
// async flush, the map removal and the pendingAsyncFlush registration are
|
||||
// done under a single lock hold so that a concurrent AcquireHandle cannot
|
||||
// slip through the gap between the two (P1-1 TOCTOU fix).
|
||||
//
|
||||
// The handle intentionally stays in fhMap during the drain so that rename
|
||||
// and unlink can still find it via FindFileHandle (P1-2 fix). It is
|
||||
// removed from fhMap only after the drain completes (RemoveFileHandle).
|
||||
func (wfs *WFS) ReleaseHandle(handleId FileHandleId) {
|
||||
wfs.fhMap.ReleaseByHandle(handleId)
|
||||
// Hold pendingAsyncFlushMu across the counter decrement and the
|
||||
// pending-flush registration. Lock ordering: pendingAsyncFlushMu → fhMap.
|
||||
wfs.pendingAsyncFlushMu.Lock()
|
||||
fhToRelease := wfs.fhMap.ReleaseByHandle(handleId)
|
||||
if fhToRelease != nil && fhToRelease.asyncFlushPending {
|
||||
done := make(chan struct{})
|
||||
wfs.pendingAsyncFlush[fhToRelease.inode] = done
|
||||
wfs.pendingAsyncFlushMu.Unlock()
|
||||
|
||||
wfs.asyncFlushWg.Add(1)
|
||||
go func() {
|
||||
defer wfs.asyncFlushWg.Done()
|
||||
defer func() {
|
||||
// Remove from fhMap first (so AcquireFileHandle creates a fresh handle).
|
||||
wfs.fhMap.RemoveFileHandle(fhToRelease.fh, fhToRelease.inode)
|
||||
// Then signal completion (unblocks waitForPendingAsyncFlush).
|
||||
close(done)
|
||||
wfs.pendingAsyncFlushMu.Lock()
|
||||
delete(wfs.pendingAsyncFlush, fhToRelease.inode)
|
||||
wfs.pendingAsyncFlushMu.Unlock()
|
||||
}()
|
||||
wfs.completeAsyncFlush(fhToRelease)
|
||||
}()
|
||||
return
|
||||
}
|
||||
wfs.pendingAsyncFlushMu.Unlock()
|
||||
|
||||
if fhToRelease != nil {
|
||||
fhToRelease.ReleaseHandle()
|
||||
}
|
||||
}
|
||||
|
||||
// waitForPendingAsyncFlush blocks until any in-flight async flush for
|
||||
// the given inode completes. Called from AcquireHandle before building
|
||||
// new handle state, so the filer metadata reflects the flushed data.
|
||||
func (wfs *WFS) waitForPendingAsyncFlush(inode uint64) {
|
||||
wfs.pendingAsyncFlushMu.Lock()
|
||||
done, found := wfs.pendingAsyncFlush[inode]
|
||||
wfs.pendingAsyncFlushMu.Unlock()
|
||||
|
||||
if found {
|
||||
glog.V(3).Infof("waitForPendingAsyncFlush: waiting for inode %d", inode)
|
||||
<-done
|
||||
}
|
||||
}
|
||||
|
||||
func (wfs *WFS) GetHandle(handleId FileHandleId) *FileHandle {
|
||||
|
||||
@@ -66,5 +66,10 @@ func (wfs *WFS) Forget(nodeid, nlookup uint64) {
|
||||
wfs.inodeToPath.Forget(nodeid, nlookup, func(dir util.FullPath) {
|
||||
wfs.metaCache.DeleteFolderChildren(context.Background(), dir)
|
||||
})
|
||||
wfs.fhMap.ReleaseByInode(nodeid)
|
||||
// ReleaseByInode returns nil if the handle is already draining (counter
|
||||
// was already <= 0 from a prior Release). Only non-async handles that
|
||||
// reach counter 0 here need cleanup.
|
||||
if fhToRelease := wfs.fhMap.ReleaseByInode(nodeid); fhToRelease != nil {
|
||||
fhToRelease.ReleaseHandle()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -253,6 +253,12 @@ func (wfs *WFS) handleRenameResponse(ctx context.Context, resp *filer_pb.StreamR
|
||||
if entry := fh.GetEntry(); entry != nil {
|
||||
entry.Name = newName
|
||||
}
|
||||
// Keep the saved async-flush path current so the fallback
|
||||
// after Forget uses the post-rename location, not the old one.
|
||||
if fh.asyncFlushPending {
|
||||
fh.asyncFlushDir = string(newParent)
|
||||
fh.asyncFlushName = newName
|
||||
}
|
||||
}
|
||||
// invalidate attr and data
|
||||
// wfs.fuseServer.InodeNotify(sourceInode, 0, -1)
|
||||
|
||||
Reference in New Issue
Block a user