fix: normalize Windows backslash paths in file uploads
When uploading files from a Windows client to a Linux server,
file paths containing backslashes were not being properly interpreted as
directory separators. This caused files intended for subdirectories to be
created in the root directory with backslashes in their filenames.
Changes:
- Add util.CleanWindowsPath and util.CleanWindowsPathBase helper functions
in weed/util/fullpath.go for reusable path normalization
- Use path.Join/path.Clean/path.Base instead of filepath equivalents
for URL path semantics (filepath is OS-specific)
- Apply normalization in weed admin handlers and filer upload parsing
Fixes#7628
Updated Content-Disposition header generation to use mime.FormatMediaType
from the standard library, which properly handles non-ASCII characters
and special characters per RFC 6266.
Changes:
- weed/server/common.go: Updated adjustHeaderContentDisposition to use
mime.FormatMediaType instead of manual escaping with fileNameEscaper
- weed/operation/upload_content.go: Updated multipart form Content-Disposition
to use mime.FormatMediaType
- weed/server/volume_server_handlers_read.go: Removed unused fileNameEscaper
This ensures correct filename display for international users across
filer downloads and file uploads.
Fixes#7634
* fix: Admin UI file browser uses https.client TLS config for filer communication
When filer is configured with HTTPS (https.filer section in security.toml),
the Admin UI file browser was still using plain HTTP for file uploads,
downloads, and viewing. This caused TLS handshake errors:
'http: TLS handshake error: client sent an HTTP request to an HTTPS server'
This fix:
- Updates FileBrowserHandlers to use the HTTPClient from weed/util/http/client
which properly loads TLS configuration from https.client section
- The HTTPClient automatically uses HTTPS when https.client.enabled=true
- All file operations (upload, download, view) now respect TLS configuration
- Falls back to plain HTTP if TLS client creation fails
Fixes#7631
* fix: Address code review comments
- Fix fallback client Transport wiring (properly assign transport to http.Client)
- Use per-operation timeouts instead of unified 60s timeout:
- uploadFileToFiler: 60s (for large file uploads)
- ViewFile: 30s (original timeout)
- isLikelyTextFile: 10s (original timeout)
* fix: Proxy file downloads through Admin UI for mTLS support
The DownloadFile function previously used browser redirect, which would
fail when filer requires mutual TLS (client certificates) since the
browser doesn't have these certificates.
Now the Admin UI server proxies the download, using its TLS-aware HTTP
client with the configured client certificates, then streams the
response to the browser.
* fix: Ensure HTTP response body is closed on non-200 responses
In ViewFile, the response body was only closed on 200 OK paths,
which could leak connections on non-200 responses. Now the body
is always closed via defer immediately after checking err == nil,
before checking the status code.
* refactor: Extract fetchFileContent helper to reduce nesting in ViewFile
Extracted the deeply nested file fetch logic (7+ levels) into a
separate fetchFileContent helper method. This improves readability
while maintaining the same TLS-aware behavior and error handling.
* refactor: Use idiomatic Go error handling in fetchFileContent
Changed fetchFileContent to return (string, error) instead of
(content string, reason string) for idiomatic Go error handling.
This enables error wrapping and standard 'if err != nil' checks.
Also improved error messages to be more descriptive for debugging,
including the HTTP status code and response body on non-200 responses.
* refactor: Extract newClientWithTimeout helper to reduce code duplication
- Added newClientWithTimeout() helper method that creates a temporary
http.Client with the specified timeout, reusing the TLS transport
- Updated uploadFileToFiler, fetchFileContent, DownloadFile, and
isLikelyTextFile to use the new helper
- Improved error message in DownloadFile to include response body
for better debuggability (consistent with fetchFileContent)
* fix: Address CodeRabbit review comments
- Fix connection leak in isLikelyTextFile: ensure resp.Body.Close()
is called even when status code is not 200
- Use http.NewRequestWithContext in DownloadFile so the filer request
is cancelled when the client disconnects, improving resource cleanup
* fix: Escape Content-Disposition filename per RFC 2616
Filenames containing quotes, backslashes, or special characters could
break the Content-Disposition header or cause client-side parsing issues.
Now properly escapes these characters before including in the header.
* fix: Handle io.ReadAll errors when reading error response bodies
In fetchFileContent and DownloadFile, the error from io.ReadAll was
ignored when reading the filer's error response body. Now properly
handles these errors to provide complete error messages.
* fix: Fail fast when TLS client creation fails
If TLS is enabled (https.client.enabled=true) but misconfigured,
fail immediately with glog.Fatalf rather than silently falling back
to plain HTTP. This prevents confusing runtime errors when the filer
only accepts HTTPS connections.
* fix: Use mime.FormatMediaType for RFC 6266 compliant Content-Disposition
Replace manual escaping with mime.FormatMediaType which properly handles
non-ASCII characters and special characters per RFC 6266, ensuring
correct filename display for international users.
* fix: Admin UI user creation fails before filer discovery (#7624)
The credential manager's filer address function was not configured quickly
enough after admin server startup, causing 'filer address function not
configured' errors when users tried to create users immediately.
Changes:
- Use exponential backoff (200ms -> 5s) instead of fixed 5s polling for
faster filer discovery on startup
- Improve error messages to be more user-friendly and actionable
Fixes#7624
* Add more debug logging to help diagnose filer discovery issues
* fix: Use dynamic filer address function to eliminate race condition
Instead of using a goroutine to wait for filer discovery before setting
the filer address function, we now set a dynamic function immediately
that returns the current filer address whenever it's called.
This eliminates the race condition where users could create users before
the goroutine completed, and provides clearer error messages when no
filer is available.
The dynamic function is HA-aware - it automatically returns whatever
filer is currently available, adapting to filer failovers.
* 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.
* fix: S3 downloads failing after idle timeout (#7618)
The idle timeout was incorrectly terminating active downloads because
read and write deadlines were managed independently. During a download,
the server writes data but rarely reads, so the read deadline would
expire even though the connection was actively being used.
Changes:
1. Simplify to single Timeout field - since this is a 'no activity timeout'
where any activity extends the deadline, separate read/write timeouts
are unnecessary. Now uses SetDeadline() which sets both at once.
2. Implement proper 'no activity timeout' - any activity (read or write)
now extends the deadline. The connection only times out when there's
genuinely no activity in either direction.
3. Increase default S3 idleTimeout from 10s to 120s for additional safety
margin when fetching chunks from slow storage backends.
Fixes#7618
* Update weed/util/net_timeout.go
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
---------
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
* fix: check freeEcSlot before evacuating EC shards to prevent data loss
Related to #7619
The moveAwayOneEcVolume function was missing the freeEcSlot check that
exists in other EC shard placement functions. This could cause EC shards
to be moved to volume servers that have no capacity, resulting in:
1. 0-byte shard files when disk is full
2. Data loss when source shards are deleted after 'successful' copy
Changes:
- Add freeEcSlot check before attempting to move EC shards
- Sort destinations by both shard count and free slots
- Refresh topology during evacuation to get updated slot counts
- Log when nodes are skipped due to no free slots
- Update freeEcSlot count after successful moves
* fix: clarify comment wording per CodeRabbit review
The comment stated 'after each move' but the code executes before
calling moveAwayOneEcVolume. Updated to 'before moving each EC volume'
for accuracy.
* fix: collect topology once and track capacity changes locally
Remove the topology refresh within the loop as it gives a false sense
of correctness - the refreshed topology could still be stale (minutes old).
Instead, we:
1. Collect topology once at the start
2. Track capacity changes ourselves via freeEcSlot decrement after each move
This is more accurate because we know exactly what moves we've made,
rather than relying on potentially stale topology refreshes.
* fix: ensure partial EC volume moves are reported as failures
Set hasMoved=false when a shard fails to move, even if previous shards
succeeded. This prevents the caller from incorrectly assuming the entire
volume was evacuated, which could lead to data loss if the source server
is decommissioned based on this incorrect status.
* fix: also reset hasMoved on moveMountedShardToEcNode error
Same issue as the previous fix: if moveMountedShardToEcNode fails
after some shards succeeded, hasMoved would incorrectly be true.
Ensure partial moves are always reported as failures.
Fixes#7619
The writeToFile function had two critical bugs that could cause data loss
during EC shard evacuation when the destination disk is full:
Bug 1: When os.OpenFile fails (e.g., disk full), the error was silently
ignored and nil was returned. This caused the caller to think the copy
succeeded.
Bug 2: When dst.Write fails (e.g., 'no space left on device'), the error
was completely ignored because the return value was not checked.
When evacuating EC shards to a full volume server (especially on BTRFS):
1. OpenFile may succeed (creates 0-byte file inode)
2. Write fails with 'no space left on device'
3. Errors were ignored, function returned nil
4. Caller thinks copy succeeded and deletes source shard
5. Result: 0-byte shard on destination, data loss!
This fix ensures both errors are properly returned, preventing data loss.
Added unit tests to verify the fix.
* s3: support STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER for signed chunked uploads with checksums
When AWS SDK v2 clients upload with both chunked encoding and checksum
validation enabled, they use the x-amz-content-sha256 header value of
STREAMING-AWS4-HMAC-SHA256-PAYLOAD-TRAILER instead of the simpler
STREAMING-AWS4-HMAC-SHA256-PAYLOAD.
This caused the chunked reader to not be properly activated, resulting
in chunk-signature metadata being stored as part of the file content.
Changes:
- Add streamingSignedPayloadTrailer constant for the new header value
- Update isRequestSignStreamingV4() to recognize this header
- Update newChunkedReader() to handle this streaming type
- Update calculateSeedSignature() to accept this header
- Add unit test for signed streaming upload with trailer
Fixes issue where Quarkus/AWS SDK v2 uploads with checksum validation
resulted in corrupted file content containing chunk-signature data.
* address review comments: add trailer signature to test, fix constant alignment
* test: separate canonical trailer text (\n) from on-wire format (\r\n)
* test: add negative test for invalid trailer signature
* refactor: check HTTP method first in streaming auth checks (fail-fast)
* test: handle crc32 Write error return for completeness
* refactor: extract createTrailerStreamingRequest helper to reduce test duplication
* fmt
* docs: clarify test comment about trailer signature validation status
* refactor: calculate chunk data length dynamically instead of hardcoding
* Update weed/s3api/chunked_reader_v4_test.go
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
* fix: use current time for signatures instead of hardcoded past date
---------
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
AWS S3 behavior: when calling GetObject or HeadObject with the PartNumber
query parameter, the ETag header should still return the complete object's
ETag (e.g., 'abc123-4' for a 4-part multipart upload), not the individual
part's ETag.
The previous implementation incorrectly overrode the ETag with the part's
ETag, causing test_multipart_get_part to fail.
This fix removes the ETag override logic while keeping:
- x-amz-mp-parts-count header (correct)
- Content-Length adjusted to part size (correct)
- Range calculation for part boundaries (correct)
* filer: async empty folder cleanup via metadata events
Implements asynchronous empty folder cleanup when files are deleted in S3.
Key changes:
1. EmptyFolderCleaner - New component that handles folder cleanup:
- Uses consistent hashing (LockRing) to determine folder ownership
- Each filer owns specific folders, avoiding duplicate cleanup work
- Debounces delete events (10s delay) to batch multiple deletes
- Caches rough folder counts to skip unnecessary checks
- Cancels pending cleanup when new files are created
- Handles both file and subdirectory deletions
2. Integration with metadata events:
- Listens to both local and remote filer metadata events
- Processes create/delete/rename events to track folder state
- Only processes folders under /buckets/<bucket>/...
3. Removed synchronous empty folder cleanup from S3 handlers:
- DeleteObjectHandler no longer calls DoDeleteEmptyParentDirectories
- DeleteMultipleObjectsHandler no longer tracks/cleans directories
- Cleanup now happens asynchronously via metadata events
Benefits:
- Non-blocking: S3 delete requests return immediately
- Coordinated: Only one filer (the owner) cleans each folder
- Efficient: Batching and caching reduce unnecessary checks
- Event-driven: Folder deletion triggers parent folder check automatically
* filer: add CleanupQueue data structure for deduplicated folder cleanup
CleanupQueue uses a linked list for FIFO ordering and a hashmap for O(1)
deduplication. Processing is triggered when:
- Queue size reaches maxSize (default 1000), OR
- Oldest item exceeds maxAge (default 10 minutes)
Key features:
- O(1) Add, Remove, Pop, Contains operations
- Duplicate folders are ignored (keeps original position/time)
- Testable with injectable time function
- Thread-safe with mutex protection
* filer: use CleanupQueue for empty folder cleanup
Replace timer-per-folder approach with queue-based processing:
- Use CleanupQueue for deduplication and ordered processing
- Process queue when full (1000 items) or oldest item exceeds 10 minutes
- Background processor checks queue every 10 seconds
- Remove from queue on create events to cancel pending cleanup
Benefits:
- Bounded memory: queue has max size, not unlimited timers
- Efficient: O(1) add/remove/contains operations
- Batch processing: handle many folders efficiently
- Better for high-volume delete scenarios
* filer: CleanupQueue.Add moves duplicate to back with updated time
When adding a folder that already exists in the queue:
- Remove it from its current position
- Add it to the back of the queue
- Update the queue time to current time
This ensures that folders with recent delete activity are processed
later, giving more time for additional deletes to occur.
* filer: CleanupQueue uses event time and inserts in sorted order
Changes:
- Add() now takes eventTime parameter instead of using current time
- Insert items in time-sorted order (oldest at front) to handle out-of-order events
- When updating duplicate with newer time, reposition to maintain sort order
- Ignore updates with older time (keep existing later time)
This ensures proper ordering when processing events from distributed filers
where event arrival order may not match event occurrence order.
* filer: remove unused CleanupQueue functions (SetNowFunc, GetAll)
Removed test-only functions:
- SetNowFunc: tests now use real time with past event times
- GetAll: tests now use Pop() to verify order
Kept functions used in production:
- Peek: used in filer_notify_read.go
- OldestAge: used in empty_folder_cleaner.go logging
* filer: initialize cache entry on first delete/create event
Previously, roughCount was only updated if the cache entry already
existed, but entries were only created during executeCleanup. This
meant delete/create events before the first cleanup didn't track
the count.
Now create the cache entry on first event, so roughCount properly
tracks all changes from the start.
* filer: skip adding to cleanup queue if roughCount > 0
If the cached roughCount indicates there are still items in the
folder, don't bother adding it to the cleanup queue. This avoids
unnecessary queue entries and reduces wasted cleanup checks.
* filer: don't create cache entry on create event
Only update roughCount if the folder is already being tracked.
New folders don't need tracking until we see a delete event.
* filer: move empty folder cleanup to its own package
- Created weed/filer/empty_folder_cleanup package
- Defined FilerOperations interface to break circular dependency
- Added CountDirectoryEntries method to Filer
- Exported IsUnderPath and IsUnderBucketPath helper functions
* filer: make isUnderPath and isUnderBucketPath private
These helpers are only used within the empty_folder_cleanup package.
Fix the templates to read scheme from httpGet.scheme instead of the
probe level, matching the structure defined in values.yaml.
This ensures that changing *.livenessProbe.httpGet.scheme or
*.readinessProbe.httpGet.scheme in values.yaml now correctly affects
the rendered manifests.
Affected components: master, filer, volume, s3, all-in-one
Fixes#7615
* fix: SFTP HomeDir path translation for user operations
When users have a non-root HomeDir (e.g., '/sftp/user'), their SFTP
operations should be relative to that directory. Previously, when a
user uploaded to '/' via SFTP, the path was not translated to their
home directory, causing 'permission denied for / for permission write'.
This fix adds a toAbsolutePath() method that implements chroot-like
behavior where the user's HomeDir becomes their root. All file and
directory operations now translate paths through this method.
Example: User with HomeDir='/sftp/user' uploading to '/' now correctly
maps to '/sftp/user'.
Fixes: https://github.com/seaweedfs/seaweedfs/issues/7470
* test: add SFTP integration tests
Add comprehensive integration tests for the SFTP server including:
- HomeDir path translation tests (verifies fix for issue #7470)
- Basic file upload/download operations
- Directory operations (mkdir, rmdir, list)
- Large file handling (1MB test)
- File rename operations
- Stat/Lstat operations
- Path edge cases (trailing slashes, .., unicode filenames)
- Admin root access verification
The test framework starts a complete SeaweedFS cluster with:
- Master server
- Volume server
- Filer server
- SFTP server with test user credentials
Test users are configured in testdata/userstore.json:
- admin: HomeDir=/ with full access
- testuser: HomeDir=/sftp/testuser with access to home
- readonly: HomeDir=/public with read-only access
* fix: correct SFTP HomeDir path translation and add CI
Fix path.Join issue where paths starting with '/' weren't joined correctly.
path.Join('/sftp/user', '/file') returns '/file' instead of '/sftp/user/file'.
Now we strip the leading '/' before joining.
Test improvements:
- Update go.mod to Go 1.24
- Fix weed binary discovery to prefer local build over PATH
- Add stabilization delay after service startup
- All 8 SFTP integration tests pass locally
Add GitHub Actions workflow for SFTP tests:
- Runs on push/PR affecting sftpd code or tests
- Tests HomeDir path translation, file ops, directory ops
- Covers issue #7470 fix verification
* security: update golang.org/x/crypto to v0.45.0
Addresses security vulnerability in golang.org/x/crypto < 0.45.0
* security: use proper SSH host key verification in tests
Replace ssh.InsecureIgnoreHostKey() with ssh.FixedHostKey() that
verifies the server's host key matches the known test key we generated.
This addresses CodeQL warning go/insecure-hostkeycallback.
Also updates go.mod to specify go 1.24.0 explicitly.
* security: fix path traversal vulnerability in SFTP toAbsolutePath
The previous implementation had a critical security vulnerability:
- Path traversal via '../..' could escape the HomeDir chroot jail
- Absolute paths were not correctly prefixed with HomeDir
The fix:
1. Concatenate HomeDir with userPath directly, then clean
2. Add security check to ensure final path stays within HomeDir
3. If traversal detected, safely return HomeDir instead
Also adds path traversal prevention tests to verify the fix.
* fix: address PR review comments
1. Fix SkipCleanup check to use actual test config instead of default
- Added skipCleanup field to SftpTestFramework struct
- Store config.SkipCleanup during Setup()
- Use f.skipCleanup in Cleanup() instead of DefaultTestConfig()
2. Fix path prefix check false positive in mkdir
- Changed from strings.HasPrefix(absPath, fs.user.HomeDir)
- To: absPath == fs.user.HomeDir || strings.HasPrefix(absPath, fs.user.HomeDir+"/")
- Prevents matching partial directory names (e.g., /sftp/username when HomeDir is /sftp/user)
* fix: check write permission on parent dir for mkdir
Aligns makeDir's permission check with newFileWriter for consistency.
To create a directory, a user needs write permission on the parent
directory, not mkdir permission on the new directory path.
* fix: refine SFTP path traversal logic and tests
1. Refine toAbsolutePath:
- Use path.Join with strings.TrimPrefix for idiomatic path construction
- Return explicit error on path traversal attempt instead of clamping
- Updated all call sites to handle the error
2. Add Unit Tests:
- Added sftp_server_test.go to verify toAbsolutePath logic
- Covers normal paths, root path, and various traversal attempts
3. Update Integration Tests:
- Updated PathTraversalPrevention test to reflect that standard SFTP clients
sanitize paths before sending. The test now verifies successful containment
within the jail rather than blocking (since the server receives a clean path).
- The server-side blocking is verified by the new unit tests.
4. Makefile:
- Removed -v from default test target
* fix: address PR comments on tests and makefile
1. Enhanced Unit Tests:
- Added edge cases (empty path, multiple slashes, trailing slash) to sftp_server_test.go
2. Makefile Improvements:
- Added 'all' target as default entry point
3. Code Clarity:
- Added comment to mkdir permission check explaining defensive nature of HomeDir check
* fix: address PR review comments on permissions and tests
1. Security:
- Added write permission check on target directory in renameEntry
2. Logging:
- Changed dispatch log verbosity from V(0) to V(1)
3. Testing:
- Updated Makefile .PHONY targets
- Added unit test cases for empty/root HomeDir behavior in toAbsolutePath
* fix: set SFTP starting directory to virtual root
1. Critical Fix:
- Changed sftp.WithStartDirectory from fs.user.HomeDir to '/'
- Prevents double-prefixing when toAbsolutePath translates paths
- Users now correctly start at their virtual root which maps to HomeDir
2. Test Improvements:
- Use pointer for homeDir in tests for clearer nil vs empty distinction
* fix: clean HomeDir at config load time
Clean HomeDir path when loading users from JSON config.
This handles trailing slashes and other path anomalies at the source,
ensuring consistency throughout the codebase and avoiding repeated
cleaning on every toAbsolutePath call.
* test: strengthen assertions and add error checking in SFTP tests
1. Add error checking for cleanup operations in TestWalk
2. Strengthen cwd assertion to expect '/' explicitly in TestCurrentWorkingDirectory
3. Add error checking for cleanup in PathTraversalPrevention test
There's unfortunatley no way to tell whether a volume is flagged read-only
because it got full, or because it is faulty. To address this, modify the
check logic so all read-only volumes are processed; if no changes are written
(i.e. the volume is healthy) it is kept as read-only.
Volumes which are modified in this process are deemed fixed, and switched to writable.
* fix: update getVersioningState to signal non-existent buckets with ErrNotFound
This change modifies the getVersioningState function to return filer_pb.ErrNotFound when a requested bucket does not exist, allowing callers to handle the situation appropriately, such as auto-creating the bucket in PUT handlers. This improves error handling and clarity in the API's behavior regarding bucket existence.
* Update weed/s3api/s3api_bucket_config.go
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
---------
Co-authored-by: 洪晓威 <xiaoweihong@deepglint.com>
Co-authored-by: Chris Lu <chrislusf@users.noreply.github.com>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
This fixes issue #6823 where a single volume server shutdown would cause
other healthy volume servers to fail their health checks and get restarted
by Kubernetes, causing a cascading failure.
Previously, the healthz handler checked if all replicated volumes could
reach their remote replicas via GetWritableRemoteReplications(). When a
volume server went down, the master would remove it from the volume
location list. Other volume servers would then fail their healthz checks
because they couldn't find all required replicas, causing Kubernetes to
restart them.
The healthz endpoint now only checks local conditions:
1. Is the server shutting down?
2. Is the server heartbeating with the master?
This follows the principle that a health check should only verify the
health of THIS server, not the overall cluster state.
Fixes#6823
* pb: add id field to Heartbeat message for stable volume server identification
This adds an 'id' field to the Heartbeat protobuf message that allows
volume servers to identify themselves independently of their IP:port address.
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* storage: add Id field to Store struct
Add Id field to Store struct and include it in CollectHeartbeat().
The Id field provides a stable volume server identity independent of IP:port.
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* topology: support id-based DataNode identification
Update GetOrCreateDataNode to accept an id parameter for stable node
identification. When id is provided, the DataNode can maintain its identity
even when its IP address changes (e.g., in Kubernetes pod reschedules).
For backward compatibility:
- If id is provided, use it as the node ID
- If id is empty, fall back to ip:port
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* volume: add -id flag for stable volume server identity
Add -id command line flag to volume server that allows specifying a stable
identifier independent of the IP address. This is useful for Kubernetes
deployments with hostPath volumes where pods can be rescheduled to different
nodes while the persisted data remains on the original node.
Usage: weed volume -id=node-1 -ip=10.0.0.1 ...
If -id is not specified, it defaults to ip:port for backward compatibility.
Fixes https://github.com/seaweedfs/seaweedfs/issues/7487
* server: add -volume.id flag to weed server command
Support the -volume.id flag in the all-in-one 'weed server' command,
consistent with the standalone 'weed volume' command.
Usage: weed server -volume.id=node-1 ...
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* topology: add test for id-based DataNode identification
Test the key scenarios:
1. Create DataNode with explicit id
2. Same id with different IP returns same DataNode (K8s reschedule)
3. IP/PublicUrl are updated when node reconnects with new address
4. Different id creates new DataNode
5. Empty id falls back to ip:port (backward compatibility)
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* pb: add address field to DataNodeInfo for proper node addressing
Previously, DataNodeInfo.Id was used as the node address, which worked
when Id was always ip:port. Now that Id can be an explicit string,
we need a separate Address field for connection purposes.
Changes:
- Add 'address' field to DataNodeInfo protobuf message
- Update ToDataNodeInfo() to populate the address field
- Update NewServerAddressFromDataNode() to use Address (with Id fallback)
- Fix LookupEcVolume to use dn.Url() instead of dn.Id()
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* fix: trim whitespace from volume server id and fix test
- Trim whitespace from -id flag to treat ' ' as empty
- Fix store_load_balancing_test.go to include id parameter in NewStore call
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* refactor: extract GetVolumeServerId to util package
Move the volume server ID determination logic to a shared utility function
to avoid code duplication between volume.go and rack.go.
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* fix: improve transition logic for legacy nodes
- Use exact ip:port match instead of net.SplitHostPort heuristic
- Update GrpcPort and PublicUrl during transition for consistency
- Remove unused net import
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
* fix: add id normalization and address change logging
- Normalize id parameter at function boundary (trim whitespace)
- Log when DataNode IP:Port changes (helps debug K8s pod rescheduling)
Ref: https://github.com/seaweedfs/seaweedfs/issues/7487
fix: EC volume deletion issues
Fixes#7489
1. Skip cookie check for EC volume deletion when SkipCookieCheck is set
When batch deleting files from EC volumes with SkipCookieCheck=true
(e.g., orphan file cleanup), the cookie is not available. The deletion
was failing with 'unexpected cookie 0' because DeleteEcShardNeedle
always validated the cookie.
2. Optimize doDeleteNeedleFromAtLeastOneRemoteEcShards to return early
Return immediately when a deletion succeeds, instead of continuing
to try all parity shards unnecessarily.
3. Remove useless log message that always logged nil error
The log at V(1) was logging err after checking it was nil.
Regression introduced in commit 7bdae5172 (Jan 3, 2023) when EC batch
delete support was added.
* Add placement package for EC shard placement logic
- Consolidate EC shard placement algorithm for reuse across shell and worker tasks
- Support multi-pass selection: racks, then servers, then disks
- Include proper spread verification and scoring functions
- Comprehensive test coverage for various cluster topologies
* Make ec.balance disk-aware for multi-disk servers
- Add EcDisk struct to track individual disks on volume servers
- Update EcNode to maintain per-disk shard distribution
- Parse disk_id from EC shard information during topology collection
- Implement pickBestDiskOnNode() for selecting best disk per shard
- Add diskDistributionScore() for tie-breaking node selection
- Update all move operations to specify target disk in RPC calls
- Improves shard balance within multi-disk servers, not just across servers
* Use placement package in EC detection for consistent disk-level placement
- Replace custom EC disk selection logic with shared placement package
- Convert topology DiskInfo to placement.DiskCandidate format
- Use SelectDestinations() for multi-rack/server/disk spreading
- Convert placement results back to topology DiskInfo for task creation
- Ensures EC detection uses same placement logic as shell commands
* Make volume server evacuation disk-aware
- Use pickBestDiskOnNode() when selecting evacuation target disk
- Specify target disk in evacuation RPC requests
- Maintains balanced disk distribution during server evacuations
* Rename PlacementConfig to PlacementRequest for clarity
PlacementRequest better reflects that this is a request for placement
rather than a configuration object. This improves API semantics.
* Rename DefaultConfig to DefaultPlacementRequest
Aligns with the PlacementRequest type naming for consistency
* Address review comments from Gemini and CodeRabbit
Fix HIGH issues:
- Fix empty disk discovery: Now discovers all disks from VolumeInfos,
not just from EC shards. This ensures disks without EC shards are
still considered for placement.
- Fix EC shard count calculation in detection.go: Now correctly filters
by DiskId and sums actual shard counts using ShardBits.ShardIdCount()
instead of just counting EcShardInfo entries.
Fix MEDIUM issues:
- Add disk ID to evacuation log messages for consistency with other logging
- Remove unused serverToDisks variable in placement.go
- Fix comment that incorrectly said 'ascending' when sorting is 'descending'
* add ec tests
* Update ec-integration-tests.yml
* Update ec_integration_test.go
* Fix EC integration tests CI: build weed binary and update actions
- Add 'Build weed binary' step before running tests
- Update actions/setup-go from v4 to v6 (Node20 compatibility)
- Update actions/checkout from v2 to v4 (Node20 compatibility)
- Move working-directory to test step only
* Add disk-aware EC rebalancing integration tests
- Add TestDiskAwareECRebalancing test with multi-disk cluster setup
- Test EC encode with disk awareness (shows disk ID in output)
- Test EC balance with disk-level shard distribution
- Add helper functions for disk-level verification:
- startMultiDiskCluster: 3 servers x 4 disks each
- countShardsPerDisk: track shards per disk per server
- calculateDiskShardVariance: measure distribution balance
- Verify no single disk is overloaded with shards
Prevents potential screen garbling when operations are parallelized
.Also simplifies logging by automatically adding newlines on output, if necessary.
Co-authored-by: Chris Lu <chrislusf@users.noreply.github.com>
* Fix SSE-S3 copy: preserve encryption metadata and set chunk SSE type
Fixes GitHub #7562: Copying objects between encrypted buckets was failing.
Root causes:
1. processMetadataBytes was re-adding SSE headers from source entry, undoing
the encryption header filtering. Now uses dstEntry.Extended which is
already filtered.
2. SSE-S3 streaming copy returned nil metadata. Now properly generates and
returns SSE-S3 destination metadata (SeaweedFSSSES3Key, AES256 header)
via ExecuteStreamingCopyWithMetadata.
3. Chunks created during streaming copy didn't have SseType set. Now sets
SseType and per-chunk SseMetadata with chunk-specific IVs for SSE-S3,
enabling proper decryption on GetObject.
* Address review: make SSE-S3 metadata serialization failures fatal errors
- In executeEncryptCopy: return error instead of just logging if
SerializeSSES3Metadata fails
- In createChunkFromData: return error if chunk SSE-S3 metadata
serialization fails
This ensures objects/chunks are never created without proper encryption
metadata, preventing unreadable/corrupted data.
* fmt
* Refactor: reuse function names instead of creating WithMetadata variants
- Change ExecuteStreamingCopy to return (*EncryptionSpec, error) directly
- Remove ExecuteStreamingCopyWithMetadata wrapper
- Change executeStreamingReencryptCopy to return (*EncryptionSpec, error)
- Remove executeStreamingReencryptCopyWithMetadata wrapper
- Update callers to ignore encryption spec with _ where not needed
* Add TODO documenting large file SSE-S3 copy limitation
The streaming copy approach encrypts the entire stream with a single IV
but stores data in chunks with per-chunk IVs. This causes decryption
issues for large files. Small inline files work correctly.
This is a known architectural issue that needs separate work to fix.
* Use chunk-by-chunk encryption for SSE-S3 copy (consistent with SSE-C/SSE-KMS)
Instead of streaming encryption (which had IV mismatch issues for multi-chunk
files), SSE-S3 now uses the same chunk-by-chunk approach as SSE-C and SSE-KMS:
1. Extended copyMultipartCrossEncryption to handle SSE-S3:
- Added SSE-S3 source decryption in copyCrossEncryptionChunk
- Added SSE-S3 destination encryption with per-chunk IVs
- Added object-level metadata generation for SSE-S3 destinations
2. Updated routing in executeEncryptCopy/executeDecryptCopy/executeReencryptCopy
to use copyMultipartCrossEncryption for all SSE-S3 scenarios
3. Removed streaming copy functions (shouldUseStreamingCopy,
executeStreamingReencryptCopy) as they're no longer used
4. Added large file (1MB) integration test to verify chunk-by-chunk copy works
This ensures consistent behavior across all SSE types and fixes data corruption
that occurred with large files in the streaming copy approach.
* fmt
* fmt
* Address review: fail explicitly if SSE-S3 metadata is missing
Instead of silently ignoring missing SSE-S3 metadata (which could create
unreadable objects), now explicitly fail the copy operation with a clear
error message if:
- First chunk is missing
- First chunk doesn't have SSE-S3 type
- First chunk has empty SSE metadata
- Deserialization fails
* Address review: improve comment to reflect full scope of chunk creation
* Address review: fail explicitly if baseIV is empty for SSE-S3 chunk encryption
If DestinationIV is not set when encrypting SSE-S3 chunks, the chunk would
be created without SseMetadata, causing GetObject decryption to fail later.
Now fails explicitly with a clear error message.
Note: calculateIVWithOffset returns ([]byte, int) not ([]byte, error) - the
int is a skip amount for intra-block alignment, not an error code.
* Address review: handle 0-byte files in SSE-S3 copy
For 0-byte files, there are no chunks to get metadata from. Generate an IV
for the object-level metadata to ensure even empty files are properly marked
as SSE-S3 encrypted.
Also validate that we don't have a non-empty file with no chunks (which
would indicate an internal error).
* Fix issue #6847: S3 chunked encoding includes headers in stored content
- Add hasTrailer flag to s3ChunkedReader to track trailer presence
- Update state transition logic to properly handle trailers in unsigned streaming
- Enhance parseChunkChecksum to handle multiple trailer lines
- Skip checksum verification for unsigned streaming uploads
- Add test case for mixed format handling (unsigned headers with signed chunks)
- Remove redundant CRLF reading in trailer processing
This fixes the issue where chunk-signature and x-amz headers were appearing
in stored file content when using chunked encoding with newer AWS SDKs.
* Fix checksum validation for unsigned streaming uploads
- Always validate checksum for data integrity regardless of signing
- Correct checksum value in test case
- Addresses PR review feedback about checksum verification
* Add warning log when multiple checksum headers found in trailer
- Log a warning when multiple valid checksum headers appear in trailers
- Uses last checksum header as suggested by CodeRabbit reviewer
- Improves debugging for edge cases with multiple checksum algorithms
* Improve trailer parsing robustness in parseChunkChecksum
- Remove redundant trimTrailingWhitespace call since readChunkLine already trims
- Use bytes.TrimSpace for both key and value to handle whitespace around colon separator
- Follows HTTP header specifications for optional whitespace around separators
- Addresses Gemini Code Assist review feedback
- Modified test/s3/tagging/s3_tagging_test.go to use environment variables for configurable endpoint and credentials
- Added s3-tagging-tests job to .github/workflows/s3-go-tests.yml to run tagging tests in CI
- Tests will now run automatically on pull requests
- Add X-Amz-Tagging header parsing in putToFiler function for PUT object operations
- Store tags with X-Amz-Tagging- prefix in entry.Extended metadata
- Add comprehensive test suite for S3 object tagging functionality
- Tests cover upload tagging, API operations, special characters, and edge cases
* filer: add username and keyPrefix support for Redis stores
Addresses https://github.com/seaweedfs/seaweedfs/issues/7299
- Add username config option to redis2, redis_cluster2, redis_lua, and
redis_lua_cluster stores (sentinel stores already had it)
- Add keyPrefix config option to all Redis stores to prefix all keys,
useful for Envoy Redis Proxy or multi-tenant Redis setups
* refactor: reduce duplication in redis.NewClient creation
Address code review feedback by defining redis.Options once and
conditionally setting TLSConfig instead of duplicating the entire
NewClient call.
* filer.toml: add username and keyPrefix to redis2.tmp example
* Fix#7575: Correct interface check for filer address function in admin server
Problem:
User creation in object store was failing with error:
'filer_etc: filer address function not configured'
Root Cause:
In admin_server.go, the code checked for incorrect interface method
SetFilerClient(string, grpc.DialOption) instead of the actual
SetFilerAddressFunc(func() pb.ServerAddress, grpc.DialOption)
This interface mismatch prevented the filer address function from being
configured, causing user creation operations to fail.
Solution:
- Fixed interface check to use SetFilerAddressFunc
- Updated function call to properly configure filer address function
- Function now dynamically returns current active filer address
Tests Added:
- Unit tests in weed/admin/dash/user_management_test.go
- Integration tests in test/admin/user_creation_integration_test.go
- Documentation in test/admin/README.md
All tests pass successfully.
* Fix#7575: Correct interface check for filer address function in admin UI
Problem:
User creation in Admin UI was failing with error:
'filer_etc: filer address function not configured'
Root Cause:
In admin_server.go, the code checked for incorrect interface method
SetFilerClient(string, grpc.DialOption) instead of the actual
SetFilerAddressFunc(func() pb.ServerAddress, grpc.DialOption)
This interface mismatch prevented the filer address function from being
configured, causing user creation operations to fail in the Admin UI.
Note: This bug only affects the Admin UI. The S3 API and weed shell
commands (s3.configure) were unaffected as they use the correct interface
or bypass the credential manager entirely.
Solution:
- Fixed interface check in admin_server.go to use SetFilerAddressFunc
- Updated function call to properly configure filer address function
- Function now dynamically returns current active filer (HA-aware)
- Cleaned up redundant comments in the code
Tests Added:
- Unit tests in weed/admin/dash/user_management_test.go
* TestFilerAddressFunctionInterface - verifies correct interface
* TestGenerateAccessKey - tests key generation
* TestGenerateSecretKey - tests secret generation
* TestGenerateAccountId - tests account ID generation
All tests pass and will run automatically in CI.
* Fix#7575: Correct interface check for filer address function in admin UI
Problem:
User creation in Admin UI was failing with error:
'filer_etc: filer address function not configured'
Root Cause:
1. In admin_server.go, the code checked for incorrect interface method
SetFilerClient(string, grpc.DialOption) instead of the actual
SetFilerAddressFunc(func() pb.ServerAddress, grpc.DialOption)
2. The admin command was missing the filer_etc import, so the store
was never registered
This interface mismatch prevented the filer address function from being
configured, causing user creation operations to fail in the Admin UI.
Note: This bug only affects the Admin UI. The S3 API and weed shell
commands (s3.configure) were unaffected as they use the correct interface
or bypass the credential manager entirely.
Solution:
- Added filer_etc import to weed/command/admin.go to register the store
- Fixed interface check in admin_server.go to use SetFilerAddressFunc
- Updated function call to properly configure filer address function
- Function now dynamically returns current active filer (HA-aware)
- Hoisted credentialManager assignment to reduce code duplication
Tests Added:
- Unit tests in weed/admin/dash/user_management_test.go
* TestFilerAddressFunctionInterface - verifies correct interface
* TestGenerateAccessKey - tests key generation
* TestGenerateSecretKey - tests secret generation
* TestGenerateAccountId - tests account ID generation
All tests pass and will run automatically in CI.
* Enable FIPS 140-3 compliant crypto by default
Addresses #6889
- Enable GOEXPERIMENT=systemcrypto by default in all Makefiles
- Enable GOEXPERIMENT=systemcrypto by default in all Dockerfiles
- Go 1.24+ has native FIPS 140-3 support via this setting
- Users can disable by setting GOEXPERIMENT= (empty)
Algorithms used (all FIPS approved):
- AES-256-GCM for data encryption
- AES-256-CTR for SSE-C
- HMAC-SHA256 for S3 signatures
- TLS 1.2/1.3 for transport encryption
* Fix: Remove invalid GOEXPERIMENT=systemcrypto
Go 1.24 uses GODEBUG=fips140=on at runtime, not GOEXPERIMENT at build time.
- Remove GOEXPERIMENT=systemcrypto from all Makefiles
- Remove GOEXPERIMENT=systemcrypto from all Dockerfiles
FIPS 140-3 mode can be enabled at runtime:
GODEBUG=fips140=on ./weed server ...
* Add FIPS 140-3 support enabled by default
Addresses #6889
- FIPS 140-3 mode is ON by default in Docker containers
- Sets GODEBUG=fips140=on via entrypoint.sh
- To disable: docker run -e GODEBUG=fips140=off ...
Have `volume.check.disk` select a random (heathly) source volume when repairing read-only volumes.
This ensures uniform load across the topology when the command is run. Also remove a lingering
TODO about ignoring full volumes; not only there's no way to discern read-only volumes from
being full vs. being damaged, we ultimately want to check the former anyway.