The condition was inverted - it was caching lookups with errors
instead of successful lookups. This caused every replicated write
to make a gRPC call to master for volume location lookup, resulting
in ~1 second latency for writeToReplicas.
The bug particularly affected TTL volumes because:
- More unique volumes are created (separate pools per TTL)
- Volumes expire and get recreated frequently
- Each new volume requires a fresh lookup (cache miss)
- Higher volume churn = more cache misses = more master lookups
With this fix, successful lookups are cached for 10 minutes,
reducing replication latency from ~1s to ~10ms for cached volumes.
* mount: add singleflight to deduplicate concurrent EnsureVisited calls
When multiple goroutines access the same uncached directory simultaneously,
they would all make redundant network requests to the filer. This change
uses singleflight.Group to ensure only one goroutine fetches the directory
entries while others wait for the result.
This fixes a race condition where concurrent lookups or readdir operations
on the same uncached directory would:
1. Make duplicate network requests to the filer
2. Insert duplicate entries into LevelDB cache
3. Waste CPU and network bandwidth
* mount: fetch parent directories in parallel during EnsureVisited
Previously, when accessing a deep path like /a/b/c/d, the parent directories
were fetched serially from target to root. This change:
1. Collects all uncached directories from target to root first
2. Fetches them all in parallel using errgroup
3. Relies on singleflight (from previous commit) for deduplication
This reduces latency when accessing deep uncached paths, especially in
high-latency network environments where parallel requests can significantly
improve performance.
* mount: add batch inserts for LevelDB meta cache
When populating the meta cache from filer, entries were inserted one-by-one
into LevelDB. This change:
1. Adds BatchInsertEntries method to LevelDBStore that uses LevelDB's
native batch write API
2. Updates MetaCache to keep a direct reference to the LevelDB store
for batch operations
3. Modifies doEnsureVisited to collect entries and insert them in
batches of 100 entries
Batch writes are more efficient because:
- Reduces number of individual write operations
- Reduces disk syncs
- Improves throughput for large directories
* mount: fix potential nil dereference in MarkChildrenCached
Add missing check for inode existence in inode2path map before accessing
the InodeEntry. This prevents a potential nil pointer dereference if the
inode exists in path2inode but not in inode2path (which could happen due
to race conditions or bugs).
This follows the same pattern used in IsChildrenCached which properly
checks for existence before accessing the entry.
* mount: fix batch flush when last entry is hidden
The previous batch insert implementation relied on the isLast flag to flush
remaining entries. However, if the last entry is a hidden system entry
(like 'topics' or 'etc' in root), the callback returns early and the
remaining entries in the batch are never flushed.
Fix by:
1. Only flush when batch reaches threshold inside the callback
2. Flush any remaining entries after ReadDirAllEntries completes
3. Use error wrapping instead of logging+returning to avoid duplicate logs
4. Create new slice after flush to allow GC of flushed entries
5. Add documentation for batchInsertSize constant
This ensures all entries are properly inserted regardless of whether
the last entry is hidden, and prevents memory retention issues.
* mount: add context support for cancellation in EnsureVisited
Thread context.Context through the batch insert call chain to enable
proper cancellation and timeout support:
1. Use errgroup.WithContext() so if one fetch fails, others are cancelled
2. Add context parameter to BatchInsertEntries for consistency with InsertEntry
3. Pass context to ReadDirAllEntries for cancellation during network calls
4. Check context cancellation before starting work in doEnsureVisited
5. Use %w for error wrapping to preserve error types for inspection
This prevents unnecessary work when one directory fetch fails and makes
the batch operations consistent with the existing context-aware APIs.
mount: remove unused isEarlyTerminated variable
The variable was redundant because when processEachEntryFn returns false,
we immediately return fuse.OK, so the check was always false.
* fix: prevent filer.backup stall in single-filer setups (#4977)
When MetaAggregator.MetaLogBuffer is empty (which happens in single-filer
setups with no peers), ReadFromBuffer was returning nil error, causing
LoopProcessLogData to enter an infinite wait loop on ListenersCond.
This fix returns ResumeFromDiskError instead, allowing SubscribeMetadata
to loop back and read from persisted logs on disk. This ensures filer.backup
continues processing events even when the in-memory aggregator buffer is empty.
Fixes#4977
* test: add integration tests for metadata subscription
Add integration tests for metadata subscription functionality:
- TestMetadataSubscribeBasic: Tests basic subscription and event receiving
- TestMetadataSubscribeSingleFilerNoStall: Regression test for #4977,
verifies subscription doesn't stall under high load in single-filer setups
- TestMetadataSubscribeResumeFromDisk: Tests resuming subscription from disk
Related to #4977
* ci: add GitHub Actions workflow for metadata subscribe tests
Add CI workflow that runs on:
- Push/PR to master affecting filer, log_buffer, or metadata subscribe code
- Runs the integration tests for metadata subscription
- Uploads logs on failure for debugging
Related to #4977
* fix: use multipart form-data for file uploads in integration tests
The filer expects multipart/form-data for file uploads, not raw POST body.
This fixes the 'Content-Type isn't multipart/form-data' error.
* test: use -peers=none for faster master startup
* test: add -peers=none to remaining master startup in ec tests
* fix: use filer HTTP port 8888, WithFilerClient adds 10000 for gRPC
WithFilerClient calls ToGrpcAddress() which adds 10000 to the port.
Passing 18888 resulted in connecting to 28888. Use 8888 instead.
* test: add concurrent writes and million updates tests
- TestMetadataSubscribeConcurrentWrites: 50 goroutines writing 20 files each
- TestMetadataSubscribeMillionUpdates: 1 million metadata entries via gRPC
(metadata only, no actual file content for speed)
* fix: address PR review comments
- Handle os.MkdirAll errors explicitly instead of ignoring
- Handle log file creation errors with proper error messages
- Replace silent event dropping with 100ms timeout and warning log
* Update metadata_subscribe_integration_test.go
fix: skip log files with deleted volumes in filer backup (#3720)
When filer.backup or filer.meta.backup resumes after being stopped, it may
encounter persisted log files stored on volumes that have since been deleted
(via volume.deleteEmpty -force). Previously, this caused the backup to get
stuck in an infinite retry loop with 'volume X not found' errors.
This fix catches 'volume not found' errors when reading log files and skips
the problematic file instead of failing. The backup will now:
- Log a warning about the missing volume
- Skip the problematic log file
- Continue with the next log file, allowing progress
The VolumeNotFoundPattern regex was already defined but never used - this
change puts it to use.
Fixes#3720
* fix: return error on size mismatch in ReadNeedleMeta for consistency
When ReadNeedleMeta encounters a size mismatch at offset >= MaxPossibleVolumeSize,
it previously just continued without returning an error, potentially using wrong data.
This fix makes ReadNeedleMeta consistent with ReadBytes (needle_read.go), which
properly returns an error in both cases:
- ErrorSizeMismatch when offset < MaxPossibleVolumeSize (to trigger retry at offset+32GB)
- A descriptive error when offset >= MaxPossibleVolumeSize (after retry failed)
Fixes#7673
* refactor: use more accurate error message for size mismatch
* fix: prevent empty .vif files from ec.decode causing parse errors
When ec.decode copies .vif files from EC shard nodes, if a source node
doesn't have the .vif file, an empty .vif file was created on the target
node. This caused volume.configure.replication to fail with 'proto: syntax
error' when trying to parse the empty file.
This fix:
1. In writeToFile: Remove empty files when no data was written (source
file was not found) to avoid leaving corrupted empty files
2. In MaybeLoadVolumeInfo: Handle empty .vif files gracefully by treating
them as non-existent, allowing the system to create a proper one
Fixes#7666
* refactor: remove redundant dst.Close() and add error logging
Address review feedback:
- Remove redundant dst.Close() call since defer already handles it
- Add error logging for os.Remove() failure
* mount: fix weed inode nlookup do not equel kernel inode nlookup
* mount: add underflow protection for nlookup decrement in Forget
* mount: use consistent == 0 check for uint64 nlookup
* Update weed/mount/inode_to_path.go
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
* mount: snapshot data before unlock in Forget to avoid using deleted InodeEntry
---------
Co-authored-by: chrislu <chris.lu@gmail.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>
* s3api: remove redundant auth verification in getRequestDataReader
The handlers PutObjectHandler and PutObjectPartHandler are already wrapped
with s3a.iam.Auth() middleware which performs signature verification via
authRequest() before the handler is invoked.
The signature verification for authTypeSignedV2, authTypePresignedV2,
authTypePresigned, and authTypeSigned in getRequestDataReader was therefore
redundant.
The newChunkedReader() call for streaming auth types is kept as it's needed
to parse the chunked transfer encoding and extract the actual data.
Fixes#7683
* simplify switch to if statement for single condition
* s3: add s3:ExistingObjectTag condition support in policy engine
Add support for s3:ExistingObjectTag/<tag-key> condition keys in bucket
policies, allowing access control based on object tags.
Changes:
- Add ObjectEntry field to PolicyEvaluationArgs (entry.Extended metadata)
- Update EvaluateConditions to handle s3:ExistingObjectTag/<key> format
- Extract tag value from entry metadata using X-Amz-Tagging-<key> prefix
This enables policies like:
{
"Condition": {
"StringEquals": {
"s3:ExistingObjectTag/status": ["public"]
}
}
}
Fixes: https://github.com/seaweedfs/seaweedfs/issues/7447
* s3: update EvaluatePolicy to accept object entry for tag conditions
Update BucketPolicyEngine.EvaluatePolicy to accept objectEntry parameter
(entry.Extended metadata) for evaluating tag-based policy conditions.
Changes:
- Add objectEntry parameter to EvaluatePolicy method
- Update callers in auth_credentials.go and s3api_bucket_handlers.go
- Pass nil for objectEntry in auth layer (entry fetched later in handlers)
For tag-based conditions to work, handlers should call EvaluatePolicy
with the object's entry.Extended after fetching the entry from filer.
* s3: add tests for s3:ExistingObjectTag policy conditions
Add comprehensive tests for object tag-based policy conditions:
- TestExistingObjectTagCondition: Basic tag matching scenarios
- Matching/non-matching tag values
- Missing tags, no tags, empty tags
- Multiple tags with one matching
- TestExistingObjectTagConditionMultipleTags: Multiple tag conditions
- Both tags match
- Only one tag matches
- TestExistingObjectTagDenyPolicy: Deny policies with tag conditions
- Default allow without tag
- Deny when specific tag present
* s3: document s3:ExistingObjectTag support and feature status
Update policy engine documentation:
- Add s3:ExistingObjectTag/<tag-key> to supported condition keys
- Add 'Object Tag-Based Access Control' section with examples
- Add 'Feature Status' section with implemented and planned features
Planned features for future implementation:
- s3:RequestObjectTag/<key>
- s3:RequestObjectTagKeys
- s3:x-amz-server-side-encryption
- Cross-account access
* Implement tag-based policy re-check in handlers
- Add checkPolicyWithEntry helper to S3ApiServer for handlers to re-check
policy after fetching object entry (for s3:ExistingObjectTag conditions)
- Add HasPolicyForBucket method to policy engine for efficient check
- Integrate policy re-check in GetObjectHandler after entry is fetched
- Integrate policy re-check in HeadObjectHandler after entry is fetched
- Update auth_credentials.go comments to explain two-phase evaluation
- Update documentation with supported operations for tag-based conditions
This implements 'Approach 1' where handlers re-check the policy with
the object entry after fetching it, allowing tag-based conditions to
be properly evaluated.
* Add integration tests for s3:ExistingObjectTag conditions
- Add TestCheckPolicyWithEntry: tests checkPolicyWithEntry helper with various
tag scenarios (matching tags, non-matching tags, empty entry, nil entry)
- Add TestCheckPolicyWithEntryNoPolicyForBucket: tests early return when no policy
- Add TestCheckPolicyWithEntryNilPolicyEngine: tests nil engine handling
- Add TestCheckPolicyWithEntryDenyPolicy: tests deny policies with tag conditions
- Add TestHasPolicyForBucket: tests HasPolicyForBucket method
These tests cover the Phase 2 policy evaluation with object entry metadata,
ensuring tag-based conditions are properly evaluated.
* Address code review nitpicks
- Remove unused extractObjectTags placeholder function (engine.go)
- Add clarifying comment about s3:ExistingObjectTag/<key> evaluation
- Consolidate duplicate tag-based examples in README
- Factor out tagsToEntry helper to package level in tests
* Address code review feedback
- Fix unsafe type assertions in GetObjectHandler and HeadObjectHandler
when getting identity from context (properly handle type assertion failure)
- Extract getConditionContextValue helper to eliminate duplicated logic
between EvaluateConditions and EvaluateConditionsLegacy
- Ensure consistent handling of missing condition keys (always return
empty slice)
* Fix GetObjectHandler to match HeadObjectHandler pattern
Add safety check for nil objectEntryForSSE before tag-based policy
evaluation, ensuring tag-based conditions are always evaluated rather
than silently skipped if entry is unexpectedly nil.
Addresses review comment from Copilot.
* Fix HeadObject action name in docs for consistency
Change 'HeadObject' to 's3:HeadObject' to match other action names.
* Extract recheckPolicyWithObjectEntry helper to reduce duplication
Move the repeated identity extraction and policy re-check logic from
GetObjectHandler and HeadObjectHandler into a shared helper method.
* Add validation for empty tag key in s3:ExistingObjectTag condition
Prevent potential issues with malformed policies containing
s3:ExistingObjectTag/ (empty tag key after slash).
Fixes#7467
The -mserver argument line in volume-statefulset.yaml was missing a
trailing backslash, which prevented extraArgs from being passed to
the weed volume process.
Also:
- Extracted master server list generation logic into shared helper
templates in _helpers.tpl for better maintainability
- Updated all occurrences of deprecated -mserver flag to -master
across docker-compose files, test files, and documentation
* fix: prevent makeslice panic in ReadNeedleMeta with corrupted needle
When a needle's DataSize in the .dat file is corrupted to a very large
value, the calculation of metaSize can become negative, causing a panic
with 'makeslice: len out of range' when creating the metadata slice.
This fix adds validation to check if metaSize is negative before
creating the slice, returning a descriptive error instead of panicking.
Fixes#7475
* Update weed/storage/needle/needle_read_page.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>
* mount: add mutex to DirectoryHandle to fix race condition
When using Ganesha NFS on top of FUSE mount, ls operations would hang
forever on directories with hundreds of files. This was caused by a
race condition in DirectoryHandle where multiple concurrent readdir
operations could modify shared state (entryStream, entryStreamOffset,
isFinished) without synchronization.
The fix adds a mutex to DirectoryHandle and holds it for the entire
duration of doReadDirectory. This serializes concurrent readdir calls
on the same handle, which is the correct behavior for a directory
handle and fixes the race condition.
Key changes:
- Added sync.Mutex to DirectoryHandle struct
- Lock the mutex at the start of doReadDirectory
- This ensures thread-safe access to entryStream and other state
The lock is per-handle (not global), so different directories can
still be listed concurrently. Only concurrent operations on the
same directory handle are serialized.
Fixes: https://github.com/seaweedfs/seaweedfs/issues/7672
* mount: add mutex to DirectoryHandle to fix race condition
When using Ganesha NFS on top of FUSE mount, ls operations would hang
forever on directories with hundreds of files. This was caused by a
race condition in DirectoryHandle where multiple concurrent readdir
operations could modify shared state (entryStream, entryStreamOffset,
isFinished) without synchronization.
The fix adds a mutex to DirectoryHandle and holds it for the entire
duration of doReadDirectory. This serializes concurrent readdir calls
on the same handle, which is the correct behavior for a directory
handle and fixes the race condition.
Key changes:
- Added sync.Mutex to DirectoryHandle struct
- Lock the mutex at the start of doReadDirectory
- Optimized reset() to reuse slice capacity and allow GC of old entries
The lock is per-handle (not global), so different directories can
still be listed concurrently. Only concurrent operations on the
same directory handle are serialized.
Fixes: https://github.com/seaweedfs/seaweedfs/issues/7672
* sts: limit session duration to incoming token's exp claim
This fixes the issue where AssumeRoleWithWebIdentity would issue sessions
that outlive the source identity token's expiration.
For use cases like GitLab CI Jobs where the ID Token has an exp claim
limited to the CI job's timeout, the STS session should not exceed that
expiration.
Changes:
- Add TokenExpiration field to ExternalIdentity struct
- Extract exp/iat/nbf claims in OIDC provider's ValidateToken
- Pass token expiration from Authenticate to ExternalIdentity
- Modify calculateSessionDuration to cap at source token's exp
- Add comprehensive tests for the new behavior
Fixes: https://github.com/seaweedfs/seaweedfs/discussions/7653
* refactor: reduce duplication in time claim extraction
Use a loop over claim names instead of repeating the same
extraction logic three times for exp, iat, and nbf claims.
* address review: add defense-in-depth for expired tokens
- Handle already-expired tokens defensively with 1 minute minimum duration
- Enforce MaxSessionLength from config as additional cap
- Fix potential nil dereference in test mock
- Add test case for expired token scenario
* remove issue reference from test
* fix: remove early return to ensure MaxSessionLength is always checked
* fix: restore volume mount when VolumeConfigure fails
When volume.configure.replication command fails (e.g., due to corrupted
.vif file), the volume was left unmounted and the master was already
notified that the volume was deleted, causing the volume to disappear.
This fix attempts to re-mount the volume when ConfigureVolume fails,
restoring the volume state and preventing data loss.
Fixes#7666
* include mount restore error in response message
* Fix webhook duplicate deliveries and POST to GET conversion
Fixes#7667
This commit addresses two critical issues with the webhook notification system:
1. Duplicate webhook deliveries based on worker count
2. POST requests being converted to GET when following redirects
Issue 1: Multiple webhook deliveries
------------------------------------
Problem: The webhook queue was creating multiple handlers (one per worker)
that all subscribed to the same topic. With Watermill's gochannel, each
handler creates a separate subscription, and all subscriptions receive
their own copy of every message, resulting in duplicate webhook calls
equal to the worker count.
Solution: Use a single handler instead of multiple handlers to ensure
each webhook event is sent only once, regardless of worker configuration.
Issue 2: POST to GET conversion with intelligent redirect handling
------------------------------------------------------------------
Problem: When webhook endpoints returned redirects (301/302/303), Go's
default HTTP client would automatically follow them and convert POST
requests to GET requests per HTTP specification.
Solution: Implement intelligent redirect handling that:
- Prevents automatic redirects to preserve POST method
- Manually follows redirects by recreating POST requests
- Caches the final redirect destination for performance
- Invalidates cache and retries on failures (network or HTTP errors)
- Provides automatic recovery from cached endpoint failures
Benefits:
- Webhooks are now sent exactly once per event
- POST method is always preserved through redirects
- Reduced latency through redirect destination caching
- Automatic failover when cached destinations become unavailable
- Thread-safe concurrent webhook delivery
Testing:
- Added TestQueueNoDuplicateWebhooks to verify single delivery
- Added TestHttpClientFollowsRedirectAsPost for redirect handling
- Added TestHttpClientUsesCachedRedirect for caching behavior
- Added cache invalidation tests for error scenarios
- All 18 webhook tests pass successfully
* Address code review comments
- Add maxWebhookRetryDepth constant to avoid magic number
- Extract cache invalidation logic into invalidateCache() helper method
- Fix redirect handling to properly follow redirects even on retry attempts
- Remove misleading comment about nWorkers controlling handler parallelism
- Fix test assertions to match actual execution flow
- Remove trailing whitespace in test file
All tests passing.
* Refactor: use setFinalURL() instead of invalidateCache()
Replace invalidateCache() with more explicit setFinalURL() function.
This is cleaner as it makes the intent clear - we're setting the URL
(either to a value or to empty string to clear it), rather than having
a separate function just for clearing.
No functional changes, all tests passing.
* Add concurrent webhook delivery using nWorkers configuration
Webhooks were previously sent sequentially (one-by-one), which could be
a performance bottleneck for high-throughput scenarios. Now nWorkers
configuration is properly used to control concurrent webhook delivery.
Implementation:
- Added semaphore channel (buffered to nWorkers capacity)
- handleWebhook acquires semaphore slot before sending (blocks if at capacity)
- Releases slot after webhook completes
- Allows up to nWorkers concurrent webhook HTTP requests
Benefits:
- Improved throughput for slow webhook endpoints
- nWorkers config now has actual purpose (was validated but unused)
- Default 5 workers provides good balance
- Configurable from 1-100 workers based on needs
Example performance improvement:
- Before: 500ms webhook latency = ~2 webhooks/sec max
- After (5 workers): 500ms latency = ~10 webhooks/sec
- After (10 workers): 500ms latency = ~20 webhooks/sec
All tests passing.
* Replace deprecated AddNoPublisherHandler with AddConsumerHandler
AddNoPublisherHandler is deprecated in Watermill.
Use AddConsumerHandler instead, which is the current recommended API
for handlers that only consume messages without publishing.
No functional changes, all tests passing.
* Drain response bodies to enable HTTP connection reuse
Added drainBody() calls in all code paths to ensure response bodies
are consumed before returning. This is critical for HTTP keep-alive
connection reuse.
Without draining:
- Connections are closed after each request
- New TCP handshake + TLS handshake for every webhook
- Higher latency and resource usage
With draining:
- Connections are reused via HTTP keep-alive
- Significant performance improvement for repeated webhooks
- Lower latency (no handshake overhead)
- Reduced resource usage
Implementation:
- Added drainBody() helper that reads up to 1MB (prevents memory issues)
- Drain on success path (line 161)
- Drain on error responses before retry (lines 119, 152)
- Drain on redirect responses before following (line 118)
- Already had drainResponse() for network errors (line 99)
All tests passing.
* Use existing CloseResponse utility instead of custom drainBody
Replaced custom drainBody() function with the existing util_http.CloseResponse()
utility which is already used throughout the codebase. This provides:
- Consistent behavior with rest of the codebase
- Better logging (logs bytes drained via CountingReader)
- Full body drainage (not limited to 1MB)
- Cleaner code (no duplication)
CloseResponse properly drains and closes the response body to enable
HTTP keep-alive connection reuse.
All tests passing.
* Fix: Don't overwrite original error when draining response
Before: err was being overwritten by drainResponse() result
After: Use drainErr to avoid losing the original client.Do() error
This was a subtle bug where if drainResponse() succeeded (returned nil),
we would lose the original network error and potentially return a
confusing error message.
All tests passing.
* Optimize HTTP client: reuse client and remove redundant timeout
1. Reuse single http.Client instance instead of creating new one per request
- Reduces allocation overhead
- More efficient for high-volume webhooks
2. Remove redundant timeout configuration
- Before: timeout set on both context AND http.Client
- After: timeout only on context (cleaner, context fires first anyway)
Performance benefits:
- Reduced GC pressure (fewer client allocations)
- Better connection pooling (single transport instance)
- Cleaner code (no redundancy)
All tests passing.
* Nit: have `ec.encode` exit immediately if no volumes are processed.
* Update weed/shell/command_ec_encode.go
Co-authored-by: coderabbitai[bot] <136622811+coderabbitai[bot]@users.noreply.github.com>
---------
Co-authored-by: Chris Lu <chrislusf@users.noreply.github.com>
Co-authored-by: coderabbitai[bot] <136622811+coderabbitai[bot]@users.noreply.github.com>
Fixes#7643
Reordered filerHealth struct fields to ensure int64 field comes first,
guaranteeing 8-byte alignment required for atomic operations on 32-bit
ARM architectures (ARMv7, as used in OpenWRT).
Fixes#7650
This change enables the SFTP server to reload the user store configuration
(sftp_userstore.json) when a HUP signal is sent to the process, without
requiring a service restart.
Changes:
- Add Reload() method to FileStore to re-read users from disk
- Add Reload() method to SFTPService to handle reload requests
- Register reload hook with grace.OnReload() in sftp command
This allows administrators to add users or change access policies
dynamically by editing the user store file and sending a HUP signal
(e.g., 'systemctl reload seaweedfs' or 'kill -HUP <pid>').
* s3: fix ListBuckets not showing buckets created by authenticated users
Fixes#7647
## Problem
Users with proper Admin permissions could create buckets but couldn't
list them. The issue occurred because ListBucketsHandler was not wrapped
with the Auth middleware, so the authenticated identity was never set in
the request context.
## Root Cause
- PutBucketHandler uses iam.Auth() middleware which sets identity in context
- ListBucketsHandler did NOT use iam.Auth() middleware
- Without the middleware, GetIdentityNameFromContext() returned empty string
- Bucket ownership checks failed because no identity was present
## Changes
1. Wrap ListBucketsHandler with iam.Auth() middleware (s3api_server.go)
2. Update ListBucketsHandler to get identity from context (s3api_bucket_handlers.go)
3. Add lookupByIdentityName() helper method (auth_credentials.go)
4. Add comprehensive test TestListBucketsIssue7647 (s3api_bucket_handlers_test.go)
## Testing
- All existing tests pass (1348 tests in s3api package)
- New test TestListBucketsIssue7647 validates the fix
- Verified admin users can see their created buckets
- Verified admin users can see all buckets
- Verified backward compatibility maintained
* s3: fix ListBuckets for JWT/Keycloak authentication
The previous fix broke JWT/Keycloak authentication because JWT identities
are created on-the-fly and not stored in the iam.identities list.
The lookupByIdentityName() would return nil for JWT users.
Solution: Store the full Identity object in the request context, not just
the name. This allows ListBucketsHandler to retrieve the complete identity
for all authentication types (SigV2, SigV4, JWT, Anonymous).
Changes:
- Add SetIdentityInContext/GetIdentityFromContext in s3_constants/header.go
- Update Auth middleware to store full identity in context
- Update ListBucketsHandler to retrieve identity from context first,
with fallback to lookup for backward compatibility
* s3: optimize lookupByIdentityName to O(1) using map
Address code review feedback: Use a map for O(1) lookups instead of
O(N) linear scan through identities list.
Changes:
- Add nameToIdentity map to IdentityAccessManagement struct
- Populate map in loadS3ApiConfiguration (consistent with accessKeyIdent pattern)
- Update lookupByIdentityName to use map lookup instead of loop
This improves performance when many identities are configured and
aligns with the existing pattern used for accessKeyIdent lookups.
* s3: address code review feedback on nameToIdentity and logging
Address two code review points:
1. Wire nameToIdentity into env-var fallback path
- The AWS env-var fallback in NewIdentityAccessManagementWithStore now
populates nameToIdentity map along with accessKeyIdent
- Keeps all identity lookup maps in sync
- Avoids potential issues if handlers rely on lookupByIdentityName
2. Improve access key lookup logging
- Reduce log verbosity: V(1) -> V(2) for failed lookups
- Truncate access keys in logs (show first 4 chars + ***)
- Include key length for debugging
- Prevents credential exposure in production logs
- Reduces log noise from misconfigured clients
* fmt
* s3: refactor truncation logic and improve error handling
Address additional code review feedback:
1. DRY principle: Extract key truncation logic into local function
- Define truncate() helper at function start
- Reuse throughout lookupByAccessKey
- Eliminates code duplication
2. Enhanced security: Mask very short access keys
- Keys <= 4 chars now show as '***' instead of full key
- Prevents any credential exposure even for short keys
- Consistent masking across all log statements
3. Improved robustness: Add warning log for type assertion failure
- Log unexpected type when identity context object is wrong type
- Helps debug potential middleware or context issues
- Better production diagnostics
4. Documentation: Add comment about future optimization opportunity
- Note potential for lightweight identity view in context
- Suggests credential-free view for better data minimization
- Documents design decision for future maintainers
* fix: initialize missing S3 options in filer to prevent nil pointer dereference
Fixes#7644
When starting the S3 gateway from the filer, several S3Options fields
were not being initialized, which could cause nil pointer dereferences
during startup.
This commit adds initialization for:
- iamConfig: for advanced IAM configuration
- metricsHttpPort: for Prometheus metrics endpoint
- metricsHttpIp: for binding the metrics endpoint
Also ensures metricsHttpIp defaults to bindIp when not explicitly set,
matching the behavior of the standalone S3 server.
This prevents the panic that was occurring in the s3.go:226 area when
these pointer fields were accessed but never initialized.
* fix: copy value instead of pointer for metricsHttpIp default
Address review comment to avoid pointer aliasing. Copy the value
instead of the pointer to prevent unexpected side effects if the
bindIp value is modified later.
- Reduce connection_max_idle from 100 to 10 (PostgreSQL) and from 2 to 10 (MySQL)
- Reduce connection_max_open from 100 to 50 for all database stores
- Set connection_max_lifetime_seconds to 300 (5 minutes) to force connection recycling
This prevents 'cannot assign requested address' errors under high load by:
1. Limiting the number of concurrent connections to reduce ephemeral port usage
2. Forcing connection recycling to prevent stale connections and port exhaustion
3. Reducing idle connections to minimize resource consumption
Fixes#6887
The allowEmptyFolder option is no longer functional because:
1. The code that used it was already commented out
2. Empty folder cleanup is now handled asynchronously by EmptyFolderCleaner
The CLI flags are kept for backward compatibility but marked as deprecated
and ignored. This removes:
- S3ApiServerOption.AllowEmptyFolder field
- The actual usage in s3api_object_handlers_list.go
- Helm chart values and template references
- References in test Makefiles and docker-compose files
This optimization avoids an expensive filer gRPC call for every DELETE
operation on buckets that don't have Object Lock enabled.
Before this change, enforceObjectLockProtections() would always call
getObjectEntry() to fetch object metadata to check for retention/legal
hold, even for buckets that never had Object Lock configured.
Changes:
1. Add early return in enforceObjectLockProtections() if bucket has no
Object Lock config or bucket doesn't exist
2. Add isObjectLockEnabled() helper function to check if a bucket has
Object Lock configured
3. Fix validateObjectLockHeaders() to check ObjectLockConfig instead of
just versioningEnabled - this ensures object-lock headers are properly
rejected on buckets without Object Lock enabled, which aligns with
AWS S3 semantics
4. Make bucket creation with Object Lock atomic - set Object Lock config
in the same CreateEntry call as bucket creation, preventing race
conditions where bucket exists without Object Lock enabled
5. Properly handle Object Lock setup failures during bucket creation -
if StoreObjectLockConfigurationInExtended fails, roll back the bucket
creation and return an error instead of leaving a bucket without
the requested Object Lock configuration
This significantly improves DELETE latency for non-Object-Lock buckets,
which is the common case (lockCheck time reduced from 1-10ms to ~1µs).
* helm: enhance all-in-one deployment configuration
Fixes#7110
This PR addresses multiple issues with the all-in-one Helm chart configuration:
## New Features
### Configurable Replicas
- Added `allInOne.replicas` (was hardcoded to 1)
### S3 Gateway Configuration
- Added full S3 config under `allInOne.s3`:
- port, httpsPort, domainName, allowEmptyFolder
- enableAuth, existingConfigSecret, auditLogConfig
- createBuckets for declarative bucket creation
### SFTP Server Configuration
- Added full SFTP config under `allInOne.sftp`:
- port, sshPrivateKey, hostKeysFolder, authMethods
- maxAuthTries, bannerMessage, loginGraceTime
- clientAliveInterval, clientAliveCountMax, enableAuth
### Command Line Arguments
- Added `allInOne.extraArgs` for custom CLI arguments
### Update Strategy
- Added `allInOne.updateStrategy.type` (Recreate/RollingUpdate)
### Secret Environment Variables
- Added `allInOne.secretExtraEnvironmentVars` for injecting secrets
### Ingress Support
- Added `allInOne.ingress` with S3, filer, and master sub-configs
### Storage Options
- Enhanced `allInOne.data` with existingClaim support
- Added PVC template for persistentVolumeClaim type
## CI Enhancements
- Added comprehensive tests for all-in-one configurations
- Tests cover replicas, S3, SFTP, extraArgs, strategies, PVC, ingress
* helm: add real cluster deployment tests to CI
- Deploy all-in-one cluster with S3 enabled on kind cluster
- Test Master API (/cluster/status endpoint)
- Test Filer API (file upload/download)
- Test S3 API (/status endpoint)
- Test S3 operations with AWS CLI:
- Create/delete buckets
- Upload/download/delete objects
- Verify file content integrity
* helm: simplify CI and remove all-in-one ingress
Address review comments:
- Remove detailed all-in-one template rendering tests from CI
- Remove real cluster deployment tests from CI
- Remove all-in-one ingress template and values configuration
Keep the core improvements:
- allInOne.replicas configuration
- allInOne.s3.* full configuration
- allInOne.sftp.* full configuration
- allInOne.extraArgs support
- allInOne.updateStrategy configuration
- allInOne.secretExtraEnvironmentVars support
* helm: address review comments
- Fix post-install-bucket-hook.yaml: add filer.s3.enableAuth and
filer.s3.existingConfigSecret to or statements for consistency
- Fix all-in-one-deployment.yaml: use default function for s3.domainName
- Fix all-in-one-deployment.yaml: use hasKey function for s3.allowEmptyFolder
* helm: clarify updateStrategy multi-replica behavior
Expand comment to warn users that RollingUpdate with multiple replicas
requires shared storage (ReadWriteMany) to avoid data loss.
* helm: address gemini-code-assist review comments
- Make PVC accessModes configurable to support ReadWriteMany for
multi-replica deployments (defaults to ReadWriteOnce)
- Use configured readiness probe paths in post-install bucket hook
instead of hardcoded paths, respecting custom configurations
* helm: simplify allowEmptyFolder logic using coalesce
Use coalesce function for cleaner template code as suggested in review.
* helm: fix extraArgs trailing backslash issue
Remove trailing backslash after the last extraArgs argument to avoid
shell syntax error. Use counter to only add backslash between arguments.
* helm: fix fallback logic for allInOne s3/sftp configuration
Changes:
- Set allInOne.s3.* and allInOne.sftp.* override parameters to null by default
This allows proper inheritance from global s3.* and sftp.* settings
- Fix allowEmptyFolder logic to use explicit nil checking instead of coalesce
The coalesce/default functions treat 'false' as empty, causing incorrect
fallback behavior when users want to explicitly set false values
Addresses review feedback about default value conflicts with fallback logic.
* helm: fix exec in bucket creation loop causing premature termination
Remove 'exec' from the range loops that create and configure S3 buckets.
The exec command replaces the current shell process, causing the script
to terminate after the first bucket, preventing creation/configuration
of subsequent buckets.
* helm: quote extraArgs to handle arguments with spaces
Use the quote function to ensure each item in extraArgs is treated as
a single, complete argument even if it contains spaces.
* helm: make s3/filer ingress work for both normal and all-in-one modes
Modified s3-ingress.yaml and filer-ingress.yaml to dynamically select
the service name based on deployment mode:
- Normal mode: points to seaweedfs-s3 / seaweedfs-filer services
- All-in-one mode: points to seaweedfs-all-in-one service
This eliminates the need for separate all-in-one ingress templates.
Users can now use the standard s3.ingress and filer.ingress settings
for both deployment modes.
* helm: fix allInOne.data.size and storageClass to use null defaults
Change size and storageClass from empty strings to null so the template
defaults (10Gi for size, cluster default for storageClass) will apply
correctly. Empty strings prevent the Helm | default function from working.
* helm: fix S3 ingress to include standalone S3 gateway case
Add s3.enabled check to the $s3Enabled logic so the ingress works for:
1. Standalone S3 gateway (s3.enabled)
2. S3 on Filer (filer.s3.enabled) when not in all-in-one mode
3. S3 in all-in-one mode (allInOne.s3.enabled)
When completing a multipart upload, the code was listing parts with limit=0,
which relies on the server's DirListingLimit default. In 'weed server' mode,
this defaults to 1000, causing uploads with more than 1000 parts to fail
with InvalidPart error.
For a 38GB file with 8MB parts (AWS CLI default), this results in ~4564 parts,
far exceeding the 1000 limit.
Fix: Use explicit limit of MaxS3MultipartParts+1 (10001) to ensure all parts
are listed regardless of server configuration.
Fixes#7638
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.