* filer: remove lock contention during chunk download
This addresses issue #7504 where a single weed mount FUSE instance
does not fully utilize node network bandwidth when reading large files.
The SingleChunkCacher was holding a mutex during the entire HTTP download,
causing readers to block until the download completed. This serialized
chunk reads even when multiple goroutines were downloading in parallel.
Changes:
- Add sync.Cond to SingleChunkCacher for efficient waiting
- Move HTTP download outside the critical section in startCaching()
- Use condition variable in readChunkAt() to wait for download completion
- Add isComplete flag to track download state
Now multiple chunk downloads can proceed truly in parallel, and readers
wait efficiently using the condition variable instead of blocking on
a mutex held during I/O operations.
Ref: #7504
* filer: parallel chunk fetching within doReadAt
This addresses issue #7504 by enabling parallel chunk downloads within
a single read operation.
Previously, doReadAt() processed chunks sequentially in a loop, meaning
each chunk had to be fully downloaded before the next one started.
This left significant network bandwidth unused when chunks resided on
different volume servers.
Changes:
- Collect all chunk read tasks upfront
- Use errgroup to fetch multiple chunks in parallel
- Each chunk reads directly into its correct buffer position
- Limit concurrency to prefetchCount (min 4) to avoid overwhelming the system
- Handle gaps and zero-filling before parallel fetch
- Trigger prefetch after parallel reads complete
For a read spanning N chunks on different volume servers, this can
now utilize up to N times the bandwidth of a single connection.
Ref: #7504
* http: direct buffer read to reduce memory copies
This addresses issue #7504 by reducing memory copy overhead during
chunk downloads.
Previously, RetriedFetchChunkData used ReadUrlAsStream which:
1. Allocated a 64KB intermediate buffer
2. Read data in 64KB chunks
3. Called a callback to copy each chunk to the destination
For a 16MB chunk, this meant 256 copy operations plus the callback
overhead. Profiling showed significant time spent in memmove.
Changes:
- Add readUrlDirectToBuffer() that reads directly into the destination
- Add retriedFetchChunkDataDirect() for unencrypted, non-gzipped chunks
- Automatically use direct read path when possible (cipher=nil, gzip=false)
- Use http.NewRequestWithContext for proper cancellation
For unencrypted chunks (the common case), this eliminates the
intermediate buffer entirely, reading HTTP response bytes directly
into the final destination buffer.
Ref: #7504
* address review comments
- Use channel (done) instead of sync.Cond for download completion signaling
This integrates better with context cancellation patterns
- Remove redundant groupErr check in reader_at.go (errors are already captured in task.err)
- Remove buggy URL encoding logic from retriedFetchChunkDataDirect
(The existing url.PathEscape on full URL is a pre-existing bug that should be fixed separately)
* address review comments (round 2)
- Return io.ErrUnexpectedEOF when HTTP response is truncated
This prevents silent data corruption from incomplete reads
- Simplify errgroup error handling by using g.Wait() error directly
Remove redundant task.err field and manual error aggregation loop
- Define minReadConcurrency constant instead of magic number 4
Improves code readability and maintainability
Note: Context propagation to startCaching() is intentionally NOT changed.
The downloaded chunk is a shared resource that may be used by multiple
readers. Using context.Background() ensures the download completes even
if one reader cancels, preventing data loss for other waiting readers.
* http: inject request ID for observability in direct read path
Add request_id.InjectToRequest() call to readUrlDirectToBuffer() for
consistency with ReadUrlAsStream path. This ensures full-chunk reads
carry the same tracing/correlation headers for server logs and metrics.
* filer: consistent timestamp handling in sequential read path
Use max(ts, task.chunk.ModifiedTsNs) in sequential path to match
parallel path behavior. Also update ts before error check so that
on failure, the returned timestamp reflects the max of all chunks
processed so far.
* filer: document why context.Background() is used in startCaching
Add comment explaining the intentional design decision: the downloaded
chunk is a shared resource that may be used by multiple concurrent
readers. Using context.Background() ensures the download completes
even if one reader cancels, preventing errors for other waiting readers.
* filer: propagate context for reader cancellation
Address review comment: pass context through ReadChunkAt call chain so
that a reader can cancel its wait for a download. The key distinction is:
- Download uses context.Background() - shared resource, always completes
- Reader wait uses request context - can be cancelled individually
If a reader cancels, it stops waiting and returns ctx.Err(), but the
download continues to completion for other readers waiting on the same
chunk. This properly handles the shared resource semantics while still
allowing individual reader cancellation.
* filer: use defer for close(done) to guarantee signal on panic
Move close(s.done) to a defer statement at the start of startCaching()
to ensure the completion signal is always sent, even if an unexpected
panic occurs. This prevents readers from blocking indefinitely.
* filer: remove unnecessary code
- Remove close(s.cacheStartedCh) in destroy() - the channel is only used
for one-time synchronization, closing it provides no benefit
- Remove task := task loop variable capture - Go 1.22+ fixed loop variable
semantics, this capture is no longer necessary (go.mod specifies Go 1.24.0)
* filer: restore fallback to chunkCache when cacher returns no data
Fix critical issue where ReadChunkAt would return 0,nil immediately
if SingleChunkCacher couldn't provide data for the requested offset,
without trying the chunkCache fallback. Now if cacher.readChunkAt
returns n=0 and err=nil, we fall through to try chunkCache.
* filer: add comprehensive tests for ReaderCache
Tests cover:
- Context cancellation while waiting for download
- Fallback to chunkCache when cacher returns n=0, err=nil
- Multiple concurrent readers waiting for same chunk
- Partial reads at different offsets
- Downloader cleanup when exceeding cache limit
- Done channel signaling (no hangs on completion)
* filer: prioritize done channel over context cancellation
If data is already available (done channel closed), return it even if
the reader's context is also cancelled. This avoids unnecessary errors
when the download has already completed.
* filer: add lookup error test and document test limitations
Add TestSingleChunkCacherLookupError to test error handling when lookup
fails. Document that full HTTP integration tests for SingleChunkCacher
require global HTTP client initialization which is complex in unit tests.
The download path is tested via FUSE integration tests.
* filer: add tests that exercise SingleChunkCacher concurrency logic
Add tests that use blocking lookupFileIdFn to exercise the actual
SingleChunkCacher wait/cancellation logic:
- TestSingleChunkCacherContextCancellationDuringLookup: tests reader
cancellation while lookup is blocked
- TestSingleChunkCacherMultipleReadersWaitForDownload: tests multiple
readers waiting on the same download
- TestSingleChunkCacherOneReaderCancelsOthersContinue: tests that when
one reader cancels, other readers continue waiting
These tests properly exercise the done channel wait/cancel logic without
requiring HTTP calls - the blocking lookup simulates a slow download.
* mount: improve read throughput with parallel chunk fetching
This addresses issue #7504 where a single weed mount FUSE instance
does not fully utilize node network bandwidth when reading large files.
Changes:
- Add -concurrentReaders mount option (default: 16) to control the
maximum number of parallel chunk fetches during read operations
- Implement parallel section reading in ChunkGroup.ReadDataAt() using
errgroup for better throughput when reading across multiple sections
- Enhance ReaderCache with MaybeCacheMany() to prefetch multiple chunks
ahead in parallel during sequential reads (now prefetches 4 chunks)
- Increase ReaderCache limit dynamically based on concurrentReaders
to support higher read parallelism
The bottleneck was that chunks were being read sequentially even when
they reside on different volume servers. By introducing parallel chunk
fetching, a single mount instance can now better saturate available
network bandwidth.
Fixes: #7504
* fmt
* Address review comments: make prefetch configurable, improve error handling
Changes:
1. Add DefaultPrefetchCount constant (4) to reader_at.go
2. Add GetPrefetchCount() method to ChunkGroup that derives prefetch count
from concurrentReaders (1/4 ratio, min 1, max 8)
3. Pass prefetch count through NewChunkReaderAtFromClient
4. Fix error handling in readDataAtParallel to prioritize errgroup error
5. Update all callers to use DefaultPrefetchCount constant
For mount operations, prefetch scales with -concurrentReaders:
- concurrentReaders=16 (default) -> prefetch=4
- concurrentReaders=32 -> prefetch=8 (capped)
- concurrentReaders=4 -> prefetch=1
For non-mount paths (WebDAV, query engine, MQ), uses DefaultPrefetchCount.
* fmt
* Refactor: use variadic parameter instead of new function name
Use NewChunkGroup with optional concurrentReaders parameter instead of
creating a separate NewChunkGroupWithConcurrency function.
This maintains backward compatibility - existing callers without the
parameter get the default of 16 concurrent readers.
* Use explicit concurrentReaders parameter instead of variadic
* Refactor: use MaybeCache with count parameter instead of new MaybeCacheMany function
* Address nitpick review comments
- Add upper bound (128) on concurrentReaders to prevent excessive goroutine fan-out
- Cap readerCacheLimit at 256 accordingly
- Fix SetChunks: use Lock() instead of RLock() since we are writing to group.sections
* feature: we can check if a fileId is already in the cache
We using this to protect cache from adding the same needle to
the cache over and over.
* fuse mount: Do not start dowloader if needle is already in the cache
* added maxFilePartSizeInCache property to ChunkCache
If file very large only first maxFilePartSizeInCache bytes
are going to be put to the cache (subject to the needle size
constrains).
* feature: for large files put in cache no more than prescribed number of bytes
Before this patch only the first needle of a large file was intended for
caching. This patch uses maximum prescribed amount of bytes to be put in
cache. This allows to bypass default 2MB maximum for a file part stored
in the cache.
* added dummy mock methods to satisfy interfaces of ChunkCache
* Added global http client
* Added Do func for global http client
* Changed the code to use the global http client
* Fix http client in volume uploader
* Fixed pkg name
* Fixed http util funcs
* Fixed http client for bench_filer_upload
* Fixed http client for stress_filer_upload
* Fixed http client for filer_server_handlers_proxy
* Fixed http client for command_fs_merge_volumes
* Fixed http client for command_fs_merge_volumes and command_volume_fsck
* Fixed http client for s3api_server
* Added init global client for main funcs
* Rename global_client to client
* Changed:
- fixed NewHttpClient;
- added CheckIsHttpsClientEnabled func
- updated security.toml in scaffold
* Reduce the visibility of some functions in the util/http/client pkg
* Added the loadSecurityConfig function
* Use util.LoadSecurityConfiguration() in NewHttpClient func
* compare chunks by timestamp
* fix slab clearing error
* fix test compilation
* move oldest chunk to sealed, instead of by fullness
* lock on fh.entryViewCache
* remove verbose logs
* revert slat clearing
* less logs
* less logs
* track write and read by timestamp
* remove useless logic
* add entry lock on file handle release
* use mem chunk only, swap file chunk has problems
* comment out code that maybe used later
* add debug mode to compare data read and write
* more efficient readResolvedChunks with linked list
* small optimization
* fix test compilation
* minor fix on writer
* add SeparateGarbageChunks
* group chunks into sections
* turn off debug mode
* fix tests
* fix tests
* tmp enable swap file chunk
* Revert "tmp enable swap file chunk"
This reverts commit 985137ec47.
* simple refactoring
* simple refactoring
* do not re-use swap file chunk. Sealed chunks should not be re-used.
* comment out debugging facilities
* either mem chunk or swap file chunk is fine now
* remove orderedMutex as *semaphore.Weighted
not found impactful
* optimize size calculation for changing large files
* optimize performance to avoid going through the long list of chunks
* still problems with swap file chunk
* rename
* tiny optimization
* swap file chunk save only successfully read data
* fix
* enable both mem and swap file chunk
* resolve chunks with range
* rename
* fix chunk interval list
* also change file handle chunk group when adding chunks
* pick in-active chunk with time-decayed counter
* fix compilation
* avoid nil with empty fh.entry
* refactoring
* rename
* rename
* refactor visible intervals to *list.List
* refactor chunkViews to *list.List
* add IntervalList for generic interval list
* change visible interval to use IntervalList in generics
* cahnge chunkViews to *IntervalList[*ChunkView]
* use NewFileChunkSection to create
* rename variables
* refactor
* fix renaming leftover
* renaming
* renaming
* add insert interval
* interval list adds lock
* incrementally add chunks to readers
Fixes:
1. set start and stop offset for the value object
2. clone the value object
3. use pointer instead of copy-by-value when passing to interval.Value
4. use insert interval since adding chunk could be out of order
* fix tests compilation
* fix tests compilation
Sometimes when an unexpected error occurs the cacher would set an
error and return. However, it would not broadcast the condition
signal in that case, therefore leaving the goroutine that runs
readChunkAt stuck forever.
I figured that the condition is unnecessary because readChunkAt is
acquiring a lock that is still held by the cacher goroutine anyway.
Callees of startCaching have to wait for a WaitGroup which makes sure
that readChunkAt can't acquire the lock before startCaching.
This way readChunkAt can execute normally and check for the error.