Commit Graph

978 Commits

Author SHA1 Message Date
Chris Lu
3b05efbdbc shell: fix potential deadlock in fs.meta.save BFS traversal
Refactored doTraverseBfsAndSaving to use context cancellation.
If the saving process fails, the traversal is stopped immediately
to prevent workers from blocking on the output channel.
2026-01-29 14:42:10 -08:00
Chris Lu
550a4ff761 Fix inconsistent TTL reporting in volume.list #8158 (#8164)
* fix inconsistent TTL reporting in volume.list #8158

* simplify volume.list output using vi.String()
2026-01-29 14:16:42 -08:00
Andrei Kvapil
b1d63d0943 feat(shell): add Object Lock management commands (#8141)
* feat(shell): add s3.bucket.lock command for Object Lock management

Add new weed shell command to view and enable S3 Object Lock on existing
buckets. This allows administrators to enable Object Lock without
recreating buckets, which is useful when buckets already contain data.

The command:
- Shows current Object Lock and Versioning status
- Enables Object Lock with -enable flag (irreversible, per AWS S3 spec)
- Automatically enables Versioning if not already enabled (required for Object Lock)

Usage:
  s3.bucket.lock -name <bucket>          # view status
  s3.bucket.lock -name <bucket> -enable  # enable Object Lock

Co-Authored-By: Claude <noreply@anthropic.com>
Signed-off-by: Andrei Kvapil <kvapss@gmail.com>

* feat(shell): add -withLock flag to s3.bucket.create command

Add support for creating buckets with Object Lock enabled directly from
weed shell. The flag automatically enables versioning as required by
Object Lock.

Usage:
  s3.bucket.create -name mybucket -withLock

Co-Authored-By: Claude <noreply@anthropic.com>
Signed-off-by: Andrei Kvapil <kvapss@gmail.com>

* Apply suggestion from @gemini-code-assist[bot]

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

---------

Signed-off-by: Andrei Kvapil <kvapss@gmail.com>
Co-authored-by: Claude <noreply@anthropic.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>
2026-01-27 10:50:16 -08:00
Chris Lu
6bf088cec9 IAM Policy Management via gRPC (#8109)
* Add IAM gRPC service definition

- Add GetConfiguration/PutConfiguration for config management
- Add CreateUser/GetUser/UpdateUser/DeleteUser/ListUsers for user management
- Add CreateAccessKey/DeleteAccessKey/GetUserByAccessKey for access key management
- Methods mirror existing IAM HTTP API functionality

* Add IAM gRPC handlers on filer server

- Implement IamGrpcServer with CredentialManager integration
- Handle configuration get/put operations
- Handle user CRUD operations
- Handle access key create/delete operations
- All methods delegate to CredentialManager for actual storage

* Wire IAM gRPC service to filer server

- Add CredentialManager field to FilerOption and FilerServer
- Import credential store implementations in filer command
- Initialize CredentialManager from credential.toml if available
- Register IAM gRPC service on filer gRPC server
- Enable credential management via gRPC alongside existing filer services

* Regenerate IAM protobuf with gRPC service methods

* iam_pb: add Policy Management to protobuf definitions

* credential: implement PolicyManager in credential stores

* filer: implement IAM Policy Management RPCs

* shell: add s3.policy command

* test: add integration test for s3.policy

* test: fix compilation errors in policy_test

* pb

* fmt

* test

* weed shell: add -policies flag to s3.configure

This allows linking/unlinking IAM policies to/from identities
directly from the s3.configure command.

* test: verify s3.configure policy linking and fix port allocation

- Added test case for linking policies to users via s3.configure
- Implemented findAvailablePortPair to ensure HTTP and gRPC ports
  are both available, avoiding conflicts with randomized port assignments.
- Updated assertion to match jsonpb output (policyNames)

* credential: add StoreTypeGrpc constant

* credential: add IAM gRPC store boilerplate

* credential: implement identity methods in gRPC store

* credential: implement policy methods in gRPC store

* admin: use gRPC credential store for AdminServer

This ensures that all IAM and policy changes made through the Admin UI
are persisted via the Filer's IAM gRPC service instead of direct file manipulation.

* shell: s3.configure use granular IAM gRPC APIs instead of full config patching

* shell: s3.configure use granular IAM gRPC APIs

* shell: replace deprecated ioutil with os in s3.policy

* filer: use gRPC FailedPrecondition for unconfigured credential manager

* test: improve s3.policy integration tests and fix error checks

* ci: add s3 policy shell integration tests to github workflow

* filer: fix LoadCredentialConfiguration error handling

* credential/grpc: propagate unmarshal errors in GetPolicies

* filer/grpc: improve error handling and validation

* shell: use gRPC status codes in s3.configure

* credential: document PutPolicy as create-or-replace

* credential/postgres: reuse CreatePolicy in PutPolicy to deduplicate logic

* shell: add timeout context and strictly enforce flags in s3.policy

* iam: standardize policy content field naming in gRPC and proto

* shell: extract slice helper functions in s3.configure

* filer: map credential store errors to gRPC status codes

* filer: add input validation for UpdateUser and CreateAccessKey

* iam: improve validation in policy and config handlers

* filer: ensure IAM service registration by defaulting credential manager

* credential: add GetStoreName method to manager

* test: verify policy deletion in integration test
2026-01-25 13:39:30 -08:00
Chris Lu
0a46577700 Fix #8040: Support '_default' keyword in collectionPattern to match default collection (#8046)
* Fix #8040: Support 'default' keyword in collectionPattern to match default collection

The default collection in SeaweedFS is represented as an empty string internally.
Previously, it was impossible to specifically target only the default collection
because:
- Empty collectionPattern matched ALL collections (filter was skipped)
- Using collectionPattern="default" tried to match the literal string "default"

This commit adds special handling for the keyword "default" in collectionPattern
across multiple shell commands:
- volume.tier.move
- volume.list
- volume.fix.replication
- volume.configure.replication

Now users can use -collectionPattern="default" to specifically target volumes
in the default collection (empty collection name), while maintaining backward
compatibility where empty pattern matches all collections.

Updated help text to document this feature.

* Update compileCollectionPattern to support 'default' keyword

This extends the fix to all commands that use regex-based collection
pattern matching:
- ec.encode
- ec.decode
- volume.tier.download
- volume.balance

The compileCollectionPattern function now treats "default" as a special
keyword that compiles to the regex "^$" (matching empty strings), making
it consistent with the other commands that use filepath.Match.

* Use CollectionDefault constant instead of hardcoded "default" string

Refactored the collection pattern matching logic to use a central constant
CollectionDefault defined in weed/shell/common.go. This improves maintainability
and ensures consistency across all shell commands.

* Address PR review feedback: simplify logic and use '_default' keyword

Changes:
1. Changed CollectionDefault from "default" to "_default" to avoid collision
   with literal collection names
2. Simplified pattern matching logic to reduce code duplication across all
   affected commands
3. Fixed error handling in command_volume_tier_move.go to properly propagate
   filepath.Match errors instead of swallowing them
4. Updated documentation to clarify how to match a literal "default"
   collection using regex patterns like "^default$"

This addresses all feedback from PR review comments.

* Remove unnecessary documentation about matching literal 'default'

Since we changed the keyword to '_default', users can now simply use
'default' to match a literal collection named "default". The previous
documentation about using regex patterns was confusing and no longer needed.

* Fix error propagation and empty pattern handling

1. command_volume_tier_move.go: Added early termination check after
   eachDataNode callback to stop processing remaining nodes if a pattern
   matching error occurred, improving efficiency

2. command_volume_configure_replication.go: Fixed empty pattern handling
   to match all collections (collectionMatched = true when pattern is empty),
   mirroring the behavior in other commands

These changes address the remaining PR review feedback.
2026-01-16 12:31:48 -08:00
Chris Lu
7eb90fdfd7 Enhance EC balancing to separate parity and data shards (#8038)
* Enhance EC balancing to separate parity and data shards across racks

* Rename avoidRacks to antiAffinityRacks for clarity

* Implement server-level EC separation for parity/data shards

* Optimize EC balancing: consolidate helpers and extract two-pass selection logic

* Add comprehensive edge case tests for EC balancing logic

* Apply code review feedback: rename select_(), add divide-by-zero guard, fix comment

* Remove unused parameters from doBalanceEcShardsWithinOneRack and add explicit anti-affinity check

* Add disk-level anti-affinity for data/parity shard separation

- Modified pickBestDiskOnNode to accept shardId and dataShardCount
- Implemented explicit anti-affinity: 1000-point penalty for placing data shards on disks with parity (and vice versa)
- Updated all call sites including balancing and evacuation
- For evacuation, disabled anti-affinity by passing dataShardCount=0
2026-01-15 12:43:44 -08:00
Chris Lu
905e7e72d9 Add remote.copy.local command to copy local files to remote storage (#8033)
* Add remote.copy.local command to copy local files to remote storage

This new command solves the issue described in GitHub Discussion #8031 where
files exist locally but are not synced to remote storage due to missing filer logs.

Features:
- Copies local-only files to remote storage
- Supports file filtering (include/exclude patterns)
- Dry run mode to preview actions
- Configurable concurrency for performance
- Force update option for existing remote files
- Comprehensive error handling with retry logic

Usage:
  remote.copy.local -dir=/path/to/mount/dir [options]

This addresses the need to manually sync files when filer logs were
deleted or when local files were never synced to remote storage.

* shell: rename commandRemoteLocalSync to commandRemoteCopyLocal

* test: add comprehensive remote cache integration tests

* shell: fix forceUpdate logic in remote.copy.local

The previous logic only allowed force updates when localEntry.RemoteEntry
was not nil, which defeated the purpose of using -forceUpdate to fix
inconsistencies where local metadata might be missing.

Now -forceUpdate will overwrite remote files whenever they exist,
regardless of local metadata state.

* shell: fix code review issues in remote.copy.local

- Return actual error from flag parsing instead of swallowing it
- Use sync.Once to safely capture first error in concurrent operations
- Add atomic counter to track actual successful copies
- Protect concurrent writes to output with mutex to prevent interleaving
- Fix path matching to prevent false positives with sibling directories
  (e.g., /mnt/remote2 no longer matches /mnt/remote)

* test: address code review nitpicks in integration tests

- Improve create_bucket error handling to fail on real errors
- Fix test assertions to properly verify expected failures
- Use case-insensitive string matching for error detection
- Replace weak logging-only tests with proper assertions
- Remove extra blank line in Makefile

* test: remove redundant edge case tests

Removed 5 tests that were either duplicates or didn't assert meaningful behavior:
- TestEdgeCaseEmptyDirectory (duplicate of TestRemoteCopyLocalEmptyDirectory)
- TestEdgeCaseRapidCacheUncache (no meaningful assertions)
- TestEdgeCaseConcurrentCommands (only logs errors, no assertions)
- TestEdgeCaseInvalidPaths (no security assertions)
- TestEdgeCaseFileNamePatterns (duplicate of pattern tests in cache tests)

Kept valuable stress tests: nested directories, special characters,
very large files (100MB), many small files (100), and zero-byte files.

* test: fix CI failures by forcing localhost IP advertising

Added -ip=127.0.0.1 flag to both primary and remote weed mini commands
to prevent IP auto-detection issues in CI environments. Without this flag,
the master would advertise itself using the actual IP (e.g., 10.1.0.17)
while binding to 127.0.0.1, causing connection refused errors when other
services tried to connect to the gRPC port.

* test: address final code review issues

- Add proper error assertions for concurrent commands test
- Require errors for invalid path tests instead of just logging
- Remove unused 'match' field from pattern test struct
- Add dry-run output assertion to verify expected behavior
- Simplify redundant condition in remote.copy.local (remove entry.RemoteEntry check)

* test: fix remote.configure tests to match actual validation rules

- Use only letters in remote names (no numbers) to match validation
- Relax missing parameter test expectations since validation may not be strict
- Generate unique names using letter suffix instead of numbers

* shell: rename pathToCopyCopy to localPath for clarity

Improved variable naming in concurrent copy loop to make the code
more readable and less repetitive.

* test: fix remaining test failures

- Remove strict error requirement for invalid paths (commands handle gracefully)
- Fix TestRemoteUncacheBasic to actually test uncache instead of cache
- Use simple numeric names for remote.configure tests (testcfg1234 format)
  to avoid validation issues with letter-only or complex name generation

* test: use only letters in remote.configure test names

The validation regex ^[A-Za-z][A-Za-z0-9]*$ requires names to start with
a letter, but using static letter-only names avoids any potential issues
with the validation.

* test: remove quotes from -name parameter in remote.configure tests

Single quotes were being included as part of the name value, causing
validation failures. Changed from -name='testremote' to -name=testremote.

* test: fix remote.configure assertion to be flexible about JSON formatting

Changed from checking exact JSON format with specific spacing to just
checking if the name appears in the output, since JSON formatting
may vary (e.g., "name":  "value" vs "name": "value").
2026-01-15 00:52:57 -08:00
Jaehoon Kim
f2e7af257d Fix volume.fsck -forcePurging -reallyDeleteFromVolume to fail fast on filer traversal errors (#8015)
* Add TraverseBfsWithContext and fix race conditions in error handling

- Add TraverseBfsWithContext function to support context cancellation
- Fix race condition in doTraverseBfsAndSaving using atomic.Bool and sync.Once
- Improve error handling with fail-fast behavior and proper error propagation
- Update command_volume_fsck to use error-returning saveFn callback
- Enhance error messages in readFilerFileIdFile with detailed context

* refactoring

* fix error format

* atomic

* filer_pb: make enqueue return void

* shell: simplify fs.meta.save error handling

* filer_pb: handle enqueue return value

* Revert "atomic"

This reverts commit 712648bc354b186d6654fdb8a46fd4848fdc4e00.

* shell: refine fs.meta.save logic

---------

Co-authored-by: Chris Lu <chris.lu@gmail.com>
2026-01-14 21:37:50 -08:00
Chris Lu
f47bc8c539 Fix remote.meta.sync TTL issue (#8021) (#8030)
* Fix remote.meta.sync TTL issue (#8021)

Remote entries should not have TTL applied because they represent files
in remote storage, not local SeaweedFS files. When TTL was configured on
a prefix, remote.meta.sync would create entries that immediately expired,
causing them to be deleted and recreated on each sync.

Changes:
- Set TtlSec=0 explicitly when creating remote entries in remote.meta.sync
- Skip TTL application in CreateEntry handler for entries with Remote field set

Fixes #8021

* Add TTL protection for remote entries in update path

- Set TtlSec=0 in doSaveRemoteEntry before calling UpdateEntry
- Add server-side TTL protection in UpdateEntry handler for remote entries
- Ensures remote entries don't inherit or preserve TTL when updated
2026-01-14 14:45:52 -08:00
promalert
9012069bd7 chore: execute goimports to format the code (#7983)
* chore: execute goimports to format the code

Signed-off-by: promalert <promalert@outlook.com>

* goimports -w .

---------

Signed-off-by: promalert <promalert@outlook.com>
Co-authored-by: Chris Lu <chris.lu@gmail.com>
2026-01-07 13:06:08 -08:00
Chris Lu
e10f11b480 opt: reduce ShardsInfo memory usage with bitmap and sorted slice (#7974)
* opt: reduce ShardsInfo memory usage with bitmap and sorted slice

- Replace map[ShardId]*ShardInfo with sorted []ShardInfo slice
- Add ShardBits (uint32) bitmap for O(1) existence checks
- Use binary search for O(log n) lookups by shard ID
- Maintain sorted order for efficient iteration
- Add comprehensive unit tests and benchmarks

Memory savings:
- Map overhead: ~48 bytes per entry eliminated
- Pointers: 8 bytes per entry eliminated
- Total: ~56 bytes per shard saved

Performance improvements:
- Has(): O(1) using bitmap
- Size(): O(log n) using binary search (was O(1), acceptable tradeoff)
- Count(): O(1) using popcount on bitmap
- Iteration: Faster due to cache locality

* refactor: add methods to ShardBits type

- Add Has(), Set(), Clear(), and Count() methods to ShardBits
- Simplify ShardsInfo methods by using ShardBits methods
- Improves code readability and encapsulation

* opt: use ShardBits directly in ShardsCountFromVolumeEcShardInformationMessage

Avoid creating a full ShardsInfo object just to count shards.
Directly cast vi.EcIndexBits to ShardBits and use Count() method.

* opt: use strings.Builder in ShardsInfo.String() for efficiency

* refactor: change AsSlice to return []ShardInfo (values instead of pointers)

This completes the memory optimization by avoiding unnecessary pointer slices and potential allocations.

* refactor: rename ShardsCountFromVolumeEcShardInformationMessage to GetShardCount

* fix: prevent deadlock in Add and Subtract methods

Copy shards data from 'other' before releasing its lock to avoid
potential deadlock when a.Add(b) and b.Add(a) are called concurrently.

The previous implementation held other's lock while calling si.Set/Delete,
which acquires si's lock. This could deadlock if two goroutines tried to
add/subtract each other concurrently.

* opt: avoid unnecessary locking in constructor functions

ShardsInfoFromVolume and ShardsInfoFromVolumeEcShardInformationMessage
now build shards slice and bitmap directly without calling Set(), which
acquires a lock on every call. Since the object is local and not yet
shared, locking is unnecessary and adds overhead.

This improves performance during object construction.

* fix: rename 'copy' variable to avoid shadowing built-in function

The variable name 'copy' in TestShardsInfo_Copy shadowed the built-in
copy() function, which is confusing and bad practice. Renamed to 'siCopy'.

* opt: use math/bits.OnesCount32 and reorganize types

1. Replace manual popcount loop with math/bits.OnesCount32 for better
   performance and idiomatic Go code
2. Move ShardSize type definition to ec_shards_info.go for better code
   organization since it's primarily used there

* refactor: Set() now accepts ShardInfo for future extensibility

Changed Set(id ShardId, size ShardSize) to Set(shard ShardInfo) to
support future additions to ShardInfo without changing the API.

This makes the code more extensible as new fields can be added to
ShardInfo (e.g., checksum, location, etc.) without breaking the Set API.

* refactor: move ShardInfo and ShardSize to separate file

Created ec_shard_info.go to hold the basic shard types (ShardInfo and
ShardSize) for better code organization and separation of concerns.

* refactor: add ShardInfo constructor and helper functions

Added NewShardInfo() constructor and IsValid() method to better
encapsulate ShardInfo creation and validation. Updated code to use
the constructor for cleaner, more maintainable code.

* fix: update remaining Set() calls to use NewShardInfo constructor

Fixed compilation errors in storage and shell packages where Set() calls
were not updated to use the new NewShardInfo() constructor.

* fix: remove unreachable code in filer backup commands

Removed unreachable return statements after infinite loops in
filer_backup.go and filer_meta_backup.go to fix compilation errors.

* fix: rename 'new' variable to avoid shadowing built-in

Renamed 'new' to 'result' in MinusParityShards, Plus, and Minus methods
to avoid shadowing Go's built-in new() function.

* fix: update remaining test files to use NewShardInfo constructor

Fixed Set() calls in command_volume_list_test.go and
ec_rebalance_slots_test.go to use NewShardInfo() constructor.
2026-01-06 00:09:52 -08:00
Chris Lu
0b5a65e00b avoid extra missing shard warning
fix https://github.com/seaweedfs/seaweedfs/issues/7956
2026-01-04 00:38:53 -08:00
Lisandro Pin
91fcc60898 Have volume.list account for EC shards when computing disk usage. (#7909) 2025-12-30 17:27:11 -08:00
Lisandro Pin
6b98b52acc Fix reporting of EC shard sizes from nodes to masters. (#7835)
SeaweedFS tracks EC shard sizes on topology data stuctures, but this information is never
relayed to master servers :( The end result is that commands reporting disk usage, such
as `volume.list` and `cluster.status`, yield incorrect figures when EC shards are present.

As an example for a simple 5-node test cluster, before...

```
> volume.list
Topology volumeSizeLimit:30000 MB hdd(volume:6/40 active:6 free:33 remote:0)
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9001 hdd(volume:1/8 active:1 free:7 remote:0)
        Disk hdd(volume:1/8 active:1 free:7 remote:0) id:0
          volume id:3  size:88967096  file_count:172  replica_placement:2  version:3  modified_at_second:1766349617
          ec volume id:1 collection: shards:[1 5]
        Disk hdd total size:88967096 file_count:172
      DataNode 192.168.10.111:9001 total size:88967096 file_count:172
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9002 hdd(volume:2/8 active:2 free:6 remote:0)
        Disk hdd(volume:2/8 active:2 free:6 remote:0) id:0
          volume id:2  size:77267536  file_count:166  replica_placement:2  version:3  modified_at_second:1766349617
          volume id:3  size:88967096  file_count:172  replica_placement:2  version:3  modified_at_second:1766349617
          ec volume id:1 collection: shards:[0 4]
        Disk hdd total size:166234632 file_count:338
      DataNode 192.168.10.111:9002 total size:166234632 file_count:338
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9003 hdd(volume:1/8 active:1 free:7 remote:0)
        Disk hdd(volume:1/8 active:1 free:7 remote:0) id:0
          volume id:2  size:77267536  file_count:166  replica_placement:2  version:3  modified_at_second:1766349617
          ec volume id:1 collection: shards:[2 6]
        Disk hdd total size:77267536 file_count:166
      DataNode 192.168.10.111:9003 total size:77267536 file_count:166
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9004 hdd(volume:2/8 active:2 free:6 remote:0)
        Disk hdd(volume:2/8 active:2 free:6 remote:0) id:0
          volume id:2  size:77267536  file_count:166  replica_placement:2  version:3  modified_at_second:1766349617
          volume id:3  size:88967096  file_count:172  replica_placement:2  version:3  modified_at_second:1766349617
          ec volume id:1 collection: shards:[3 7]
        Disk hdd total size:166234632 file_count:338
      DataNode 192.168.10.111:9004 total size:166234632 file_count:338
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9005 hdd(volume:0/8 active:0 free:8 remote:0)
        Disk hdd(volume:0/8 active:0 free:8 remote:0) id:0
          ec volume id:1 collection: shards:[8 9 10 11 12 13]
        Disk hdd total size:0 file_count:0
    Rack DefaultRack total size:498703896 file_count:1014
  DataCenter DefaultDataCenter total size:498703896 file_count:1014
total size:498703896 file_count:1014
```

...and after:

```
> volume.list
Topology volumeSizeLimit:30000 MB hdd(volume:6/40 active:6 free:33 remote:0)
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9001 hdd(volume:1/8 active:1 free:7 remote:0)
        Disk hdd(volume:1/8 active:1 free:7 remote:0) id:0
          volume id:2  size:81761800  file_count:161  replica_placement:2  version:3  modified_at_second:1766349495
          ec volume id:1 collection: shards:[1 5 9] sizes:[1:8.00 MiB 5:8.00 MiB 9:8.00 MiB] total:24.00 MiB
        Disk hdd total size:81761800 file_count:161
      DataNode 192.168.10.111:9001 total size:81761800 file_count:161
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9002 hdd(volume:1/8 active:1 free:7 remote:0)
        Disk hdd(volume:1/8 active:1 free:7 remote:0) id:0
          volume id:3  size:88678712  file_count:170  replica_placement:2  version:3  modified_at_second:1766349495
          ec volume id:1 collection: shards:[11 12 13] sizes:[11:8.00 MiB 12:8.00 MiB 13:8.00 MiB] total:24.00 MiB
        Disk hdd total size:88678712 file_count:170
      DataNode 192.168.10.111:9002 total size:88678712 file_count:170
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9003 hdd(volume:2/8 active:2 free:6 remote:0)
        Disk hdd(volume:2/8 active:2 free:6 remote:0) id:0
          volume id:2  size:81761800  file_count:161  replica_placement:2  version:3  modified_at_second:1766349495
          volume id:3  size:88678712  file_count:170  replica_placement:2  version:3  modified_at_second:1766349495
          ec volume id:1 collection: shards:[0 4 8] sizes:[0:8.00 MiB 4:8.00 MiB 8:8.00 MiB] total:24.00 MiB
        Disk hdd total size:170440512 file_count:331
      DataNode 192.168.10.111:9003 total size:170440512 file_count:331
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9004 hdd(volume:2/8 active:2 free:6 remote:0)
        Disk hdd(volume:2/8 active:2 free:6 remote:0) id:0
          volume id:2  size:81761800  file_count:161  replica_placement:2  version:3  modified_at_second:1766349495
          volume id:3  size:88678712  file_count:170  replica_placement:2  version:3  modified_at_second:1766349495
          ec volume id:1 collection: shards:[2 6 10] sizes:[2:8.00 MiB 6:8.00 MiB 10:8.00 MiB] total:24.00 MiB
        Disk hdd total size:170440512 file_count:331
      DataNode 192.168.10.111:9004 total size:170440512 file_count:331
  DataCenter DefaultDataCenter hdd(volume:6/40 active:6 free:33 remote:0)
    Rack DefaultRack hdd(volume:6/40 active:6 free:33 remote:0)
      DataNode 192.168.10.111:9005 hdd(volume:0/8 active:0 free:8 remote:0)
        Disk hdd(volume:0/8 active:0 free:8 remote:0) id:0
          ec volume id:1 collection: shards:[3 7] sizes:[3:8.00 MiB 7:8.00 MiB] total:16.00 MiB
        Disk hdd total size:0 file_count:0
    Rack DefaultRack total size:511321536 file_count:993
  DataCenter DefaultDataCenter total size:511321536 file_count:993
total size:511321536 file_count:993
```
2025-12-28 19:30:42 -08:00
Chris Lu
f88b9a643d add one example 2025-12-28 11:39:06 -08:00
Chris Lu
2b3ff3cd05 verbose mode 2025-12-26 12:42:00 -08:00
Copilot
b866907461 fs.meta.save: fix directory entry parent path in FullEntry construction (#7886)
* Checkpoint from VS Code for coding agent session

* Fix fs.meta.save to correctly save directory's own metadata

Co-authored-by: chrislusf <1543151+chrislusf@users.noreply.github.com>

* address error

---------

Co-authored-by: Chris Lu <chris.lu@gmail.com>
Co-authored-by: copilot-swe-agent[bot] <198982749+Copilot@users.noreply.github.com>
Co-authored-by: chrislusf <1543151+chrislusf@users.noreply.github.com>
2025-12-26 12:30:30 -08:00
Chris Lu
4aa50bfa6a fix: EC rebalance fails with replica placement 000 (#7812)
* fix: EC rebalance fails with replica placement 000

This PR fixes several issues with EC shard distribution:

1. Pre-flight check before EC encoding
   - Verify target disk type has capacity before encoding starts
   - Prevents encoding shards only to fail during rebalance
   - Shows helpful error when wrong diskType is specified (e.g., ssd when volumes are on hdd)

2. Fix EC rebalance with replica placement 000
   - When DiffRackCount=0, shards should be distributed freely across racks
   - The '000' placement means 'no volume replication needed' because EC provides redundancy
   - Previously all racks were skipped with error 'shards X > replica placement limit (0)'

3. Add unit tests for EC rebalance slot calculation
   - TestECRebalanceWithLimitedSlots: documents the limited slots scenario
   - TestECRebalanceZeroFreeSlots: reproduces the 0 free slots error

4. Add Makefile for manual EC testing
   - make setup: start cluster and populate data
   - make shell: open weed shell for EC commands
   - make clean: stop cluster and cleanup

* fix: default -rebalance to true for ec.encode

The -rebalance flag was defaulting to false, which meant ec.encode would
only print shard moves but not actually execute them. This is a poor
default since the whole point of EC encoding is to distribute shards
across servers for fault tolerance.

Now -rebalance defaults to true, so shards are actually distributed
after encoding. Users can use -rebalance=false if they only want to
see what would happen without making changes.

* test/erasure_coding: improve Makefile safety and docs

- Narrow pkill pattern for volume servers to use TEST_DIR instead of
  port pattern, avoiding accidental kills of unrelated SeaweedFS processes
- Document external dependencies (curl, jq) in header comments

* shell: refactor buildRackWithEcShards to reuse buildEcShards

Extract common shard bit construction logic to avoid duplication
between buildEcShards and buildRackWithEcShards helper functions.

* shell: update test for EC replication 000 behavior

When DiffRackCount=0 (replication "000"), EC shards should be
distributed freely across racks since erasure coding provides its
own redundancy. Update test expectation to reflect this behavior.

* erasure_coding: add distribution package for proportional EC shard placement

Add a new reusable package for EC shard distribution that:
- Supports configurable EC ratios (not hard-coded 10+4)
- Distributes shards proportionally based on replication policy
- Provides fault tolerance analysis
- Prefers moving parity shards to keep data shards spread out

Key components:
- ECConfig: Configurable data/parity shard counts
- ReplicationConfig: Parsed XYZ replication policy
- ECDistribution: Target shard counts per DC/rack/node
- Rebalancer: Plans shard moves with parity-first strategy

This enables seaweed-enterprise custom EC ratios and weed worker
integration while maintaining a clean, testable architecture.

* shell: integrate distribution package for EC rebalancing

Add shell wrappers around the distribution package:
- ProportionalECRebalancer: Plans moves using distribution.Rebalancer
- NewProportionalECRebalancerWithConfig: Supports custom EC configs
- GetDistributionSummary/GetFaultToleranceAnalysis: Helper functions

The shell layer converts between EcNode types and the generic
TopologyNode types used by the distribution package.

* test setup

* ec: improve data and parity shard distribution across racks

- Add shardsByTypePerRack helper to track data vs parity shards
- Rewrite doBalanceEcShardsAcrossRacks for two-pass balancing:
  1. Balance data shards (0-9) evenly, max ceil(10/6)=2 per rack
  2. Balance parity shards (10-13) evenly, max ceil(4/6)=1 per rack
- Add balanceShardTypeAcrossRacks for generic shard type balancing
- Add pickRackForShardType to select destination with room for type
- Add unit tests for even data/parity distribution verification

This ensures even read load during normal operation by spreading
both data and parity shards across all available racks.

* ec: make data/parity shard counts configurable in ecBalancer

- Add dataShardCount and parityShardCount fields to ecBalancer struct
- Add getDataShardCount() and getParityShardCount() methods with defaults
- Replace direct constant usage with configurable methods
- Fix unused variable warning for parityPerRack

This allows seaweed-enterprise to use custom EC ratios while
defaulting to standard 10+4 scheme.

* Address PR 7812 review comments

Makefile improvements:
- Save PIDs for each volume server for precise termination
- Use PID-based killing in stop target with pkill fallback
- Use more specific pkill patterns with TEST_DIR paths

Documentation:
- Document jq dependency in README.md

Rebalancer fix:
- Fix duplicate shard count updates in applyMovesToAnalysis
- All planners (DC/rack/node) update counts inline during planning
- Remove duplicate updates from applyMovesToAnalysis to avoid double-counting

* test/erasure_coding: use mktemp for test file template

Use mktemp instead of hardcoded /tmp/testfile_template.bin path
to provide better isolation for concurrent test runs.
2025-12-19 13:29:12 -08:00
Chris Lu
f4cdfcc5fd Add cluster.raft.leader.transfer command for graceful leader change (#7819)
* proto: add RaftLeadershipTransfer RPC for forced leader change

Add new gRPC RPC and messages for leadership transfer:
- RaftLeadershipTransferRequest: optional target_id and target_address
- RaftLeadershipTransferResponse: previous_leader and new_leader

This enables graceful leadership transfer before master maintenance,
reducing errors in filers during planned maintenance windows.

Ref: https://github.com/seaweedfs/seaweedfs/issues/7527

* proto: regenerate Go files for RaftLeadershipTransfer

Generated from master.proto changes.

* master: implement RaftLeadershipTransfer gRPC handler

Add gRPC handler for leadership transfer with support for:
- Transfer to any eligible follower (when target_id is empty)
- Transfer to a specific server (when target_id and target_address are provided)

Uses hashicorp/raft LeadershipTransfer() and LeadershipTransferToServer() APIs.

Returns the previous and new leader in the response.

* shell: add cluster.raft.leader.transfer command

Add weed shell command for graceful leadership transfer:
- Displays current cluster status before transfer
- Supports auto-selection of target (any eligible follower)
- Supports targeted transfer with -id and -address flags
- Provides clear feedback on success/failure with troubleshooting tips

Usage:
  cluster.raft.leader.transfer
  cluster.raft.leader.transfer -id <server_id> -address <grpc_address>

* master: add unit tests for raft gRPC handlers

Add tests covering:
- RaftLeadershipTransfer with no raft initialized
- RaftLeadershipTransfer with target_id but no address
- RaftListClusterServers with no raft initialized
- RaftAddServer with no raft initialized
- RaftRemoveServer with no raft initialized

These tests verify error handling when raft is not configured.

* shell: add tests for cluster.raft.leader.transfer command

Add tests covering:
- Command name and help text validation
- HasTag returns false for ResourceHeavy
- Validation of -id without -address
- Argument parsing with unknown flags

* master: clarify that leadership transfer requires -raftHashicorp

The default raft implementation (seaweedfs/raft, a goraft fork) does not
support graceful leadership transfer. This feature is only available when
using hashicorp raft (-raftHashicorp=true).

Update error messages and help text to make this requirement clear:
- gRPC handler returns specific error for goraft users
- Shell command help text notes the requirement
- Added test for goraft case

* test: use strings.Contains instead of custom helper

Replace custom contains/containsHelper functions with the standard
library strings.Contains for better maintainability.

* shell: return flag parsing errors instead of swallowing them

- Return the error from flag.Parse() instead of returning nil
- Update test to explicitly assert error for unknown flags

* test: document integration test scenarios for Raft leadership transfer

Add comments explaining:
- Why these unit tests only cover 'Raft not initialized' scenarios
- What integration tests should cover (with multi-master cluster)
- hashicorp/raft uses concrete types that cannot be easily mocked

* fix: address reviewer feedback on tests and leader routing

- Remove misleading tests that couldn't properly validate their
  documented behavior without a real Raft cluster:
  - TestRaftLeadershipTransfer_GoraftNotSupported
  - TestRaftLeadershipTransfer_ValidationTargetIdWithoutAddress

- Change WithClient(false) to WithClient(true) for RaftLeadershipTransfer
  RPC to ensure the request is routed to the current leader

* Improve cluster.raft.transferLeader command

- Rename command from cluster.raft.leader.transfer to cluster.raft.transferLeader
- Add symmetric validation: -id and -address must be specified together
- Handle case where same leader is re-elected after transfer
- Add test for -address without -id validation
- Add docker compose file for 5-master raft cluster testing
2025-12-19 00:15:39 -08:00
Lisandro Pin
6a1b9ce8cd Give cluster.status detailed file metrics for regular volumes (#7791)
* Implement a `weed shell` command to return a status overview of the cluster.

Detailed file information will be implemented in a follow-up MR. Note also
that masters are currently not reporting back EC shard sizes correctly, via
`master_pb.VolumeEcShardInformationMessage.shard_sizes`.

F.ex:

```
> status

cluster:
	id:       topo
	status:   LOCKED
	nodes:    10
	topology: 1 DC(s)s, 1 disk(s) on 1 rack(s)

volumes:
	total:    3 volumes on 1 collections
	max size: 31457280000 bytes
	regular:  2/80 volumes on 6 replicas, 6 writable (100.00%), 0 read-only (0.00%)
	EC:       1 EC volumes on 14 shards (14.00 shards/volume)

storage:
	total:           186024424 bytes
	regular volumes: 186024424 bytes
	EC volumes:      0 bytes
	raw:             558073152 bytes on volume replicas, 0 bytes on EC shard files
```

* Humanize output for `weed.server` by default.

Makes things more readable :)

```
> cluster.status

cluster:
	id:       topo
	status:   LOCKED
	nodes:    10
	topology: 1 DC, 10 disks on 1 rack

volumes:
	total:    3 volumes, 1 collection
	max size: 32 GB
	regular:  2/80 volumes on 6 replicas, 6 writable (100%), 0 read-only (0%)
	EC:       1 EC volume on 14 shards (14 shards/volume)

storage:
	total:           172 MB
	regular volumes: 172 MB
	EC volumes:      0 B
	raw:             516 MB on volume replicas, 0 B on EC shards
```

```
> cluster.status --humanize=false

cluster:
	id:       topo
	status:   LOCKED
	nodes:    10
	topology: 1 DC(s), 10 disk(s) on 1 rack(s)

volumes:
	total:    3 volume(s), 1 collection(s)
	max size: 31457280000 byte(s)
	regular:  2/80 volume(s) on 6 replica(s), 5 writable (83.33%), 1 read-only (16.67%)
	EC:       1 EC volume(s) on 14 shard(s) (14.00 shards/volume)

storage:
	total:           172128072 byte(s)
	regular volumes: 172128072 byte(s)
	EC volumes:      0 byte(s)
	raw:             516384216 byte(s) on volume replicas, 0 byte(s) on EC shards
```

Also adds unit tests, and reshuffles test files handling for clarity.

* `cluster.status`: Add detailed file metrics for regular volumes.
2025-12-17 16:40:27 -08:00
Chris Lu
347ed7cbfa fix: sync replica entries before ec.encode and volume.tier.move (#7798)
* fix: sync replica entries before ec.encode and volume.tier.move (#7797)

This addresses the data inconsistency risk in multi-replica volumes.

When ec.encode or volume.tier.move operates on a multi-replica volume:
1. Find the replica with the highest file count (the 'best' one)
2. Copy missing entries from other replicas INTO this best replica
3. Use this union replica for the destructive operation

This ensures no data is lost due to replica inconsistency before
EC encoding or tier moving.

Added:
- command_volume_replica_check.go: Core sync and select logic
- command_volume_replica_check_test.go: Test coverage

Modified:
- command_ec_encode.go: Call syncAndSelectBestReplica before encoding
- command_volume_tier_move.go: Call syncAndSelectBestReplica before moving

Fixes #7797

* test: add integration test for replicated volume sync during ec.encode

* test: improve retry logic for replicated volume integration test

* fix: resolve JWT issue in integration tests by using empty security.toml

* address review comments: add readNeedleMeta, parallelize status fetch, fix collection param, fix test issues

* test: use collection parameter consistently in replica sync test

* fix: convert weed binary path to absolute to work with changed working directory

* fix: remove skip behavior, keep tests failing on missing binary

* fix: always check recency for each needle, add divergent replica test
2025-12-16 23:16:07 -08:00
G-OD
504b258258 s3: fix remote object not caching (#7790)
* s3: fix remote object not caching

* s3: address review comments for remote object caching

- Fix leading slash in object name by using strings.TrimPrefix
- Return cached entry from CacheRemoteObjectToLocalCluster to get updated local chunk locations
- Reuse existing helper function instead of inline gRPC call

* s3/filer: add singleflight deduplication for remote object caching

- Add singleflight.Group to FilerServer to deduplicate concurrent cache operations
- Wrap CacheRemoteObjectToLocalCluster with singleflight to ensure only one
  caching operation runs per object when multiple clients request the same file
- Add early-return check for already-cached objects
- S3 API calls filer gRPC with timeout and graceful fallback on error
- Clear negative bucket cache when bucket is created via weed shell
- Add integration tests for remote cache with singleflight deduplication

This benefits all clients (S3, HTTP, Hadoop) accessing remote-mounted objects
by preventing redundant cache operations and improving concurrent access performance.

Fixes: https://github.com/seaweedfs/seaweedfs/discussions/7599

* fix: data race in concurrent remote object caching

- Add mutex to protect chunks slice from concurrent append
- Add mutex to protect fetchAndWriteErr from concurrent read/write
- Fix incorrect error check (was checking assignResult.Error instead of parseErr)
- Rename inner variable to avoid shadowing fetchAndWriteErr

* fix: address code review comments

- Remove duplicate remote caching block in GetObjectHandler, keep only singleflight version
- Add mutex protection for concurrent chunk slice and error access (data race fix)
- Use lazy initialization for S3 client in tests to avoid panic during package load
- Fix markdown linting: add language specifier to code fence, blank lines around tables
- Add 'all' target to Makefile as alias for test-with-server
- Remove unused 'util' import

* style: remove emojis from test files

* fix: add defensive checks and sort chunks by offset

- Add nil check and type assertion check for singleflight result
- Sort chunks by offset after concurrent fetching to maintain file order

* fix: improve test diagnostics and path normalization

- runWeedShell now returns error for better test diagnostics
- Add all targets to .PHONY in Makefile (logs-primary, logs-remote, health)
- Strip leading slash from normalizedObject to avoid double slashes in path

---------

Co-authored-by: chrislu <chris.lu@gmail.com>
Co-authored-by: Chris Lu <chrislusf@users.noreply.github.com>
2025-12-16 12:41:04 -08:00
Lisandro Pin
187ef65e8f Humanize output for weed.server by default (#7758)
* Implement a `weed shell` command to return a status overview of the cluster.

Detailed file information will be implemented in a follow-up MR. Note also
that masters are currently not reporting back EC shard sizes correctly, via
`master_pb.VolumeEcShardInformationMessage.shard_sizes`.

F.ex:

```
> status

cluster:
	id:       topo
	status:   LOCKED
	nodes:    10
	topology: 1 DC(s)s, 1 disk(s) on 1 rack(s)

volumes:
	total:    3 volumes on 1 collections
	max size: 31457280000 bytes
	regular:  2/80 volumes on 6 replicas, 6 writable (100.00%), 0 read-only (0.00%)
	EC:       1 EC volumes on 14 shards (14.00 shards/volume)

storage:
	total:           186024424 bytes
	regular volumes: 186024424 bytes
	EC volumes:      0 bytes
	raw:             558073152 bytes on volume replicas, 0 bytes on EC shard files
```

* Humanize output for `weed.server` by default.

Makes things more readable :)

```
> cluster.status

cluster:
	id:       topo
	status:   LOCKED
	nodes:    10
	topology: 1 DC, 10 disks on 1 rack

volumes:
	total:    3 volumes, 1 collection
	max size: 32 GB
	regular:  2/80 volumes on 6 replicas, 6 writable (100%), 0 read-only (0%)
	EC:       1 EC volume on 14 shards (14 shards/volume)

storage:
	total:           172 MB
	regular volumes: 172 MB
	EC volumes:      0 B
	raw:             516 MB on volume replicas, 0 B on EC shards
```

```
> cluster.status --humanize=false

cluster:
	id:       topo
	status:   LOCKED
	nodes:    10
	topology: 1 DC(s), 10 disk(s) on 1 rack(s)

volumes:
	total:    3 volume(s), 1 collection(s)
	max size: 31457280000 byte(s)
	regular:  2/80 volume(s) on 6 replica(s), 5 writable (83.33%), 1 read-only (16.67%)
	EC:       1 EC volume(s) on 14 shard(s) (14.00 shards/volume)

storage:
	total:           172128072 byte(s)
	regular volumes: 172128072 byte(s)
	EC volumes:      0 byte(s)
	raw:             516384216 byte(s) on volume replicas, 0 byte(s) on EC shards
```

Also adds unit tests, and reshuffles test files handling for clarity.
2025-12-15 11:18:45 -08:00
Chris Lu
7ed7578424 fix(ec.decode): purge EC shards when volume is empty (#7749)
* fix(ec.decode): purge EC shards when volume is empty

When an EC volume has no live entries (all deleted), ec.decode should not generate an empty normal volume. Instead, treat decode as a no-op and allow shard purge to proceed cleanly.\n\nFixes: #7748

* chore: address PR review comments

* test: cover live EC index + avoid magic string

* chore: harden empty-EC handling

- Make shard cleanup best-effort (collect errors)\n- Remove unreachable EOF handling in HasLiveNeedles\n- Add empty ecx test case\n- Share no-live-entries substring between server/client\n

* perf: parallelize EC shard unmount/delete across locations

* refactor: combine unmount+delete into single goroutine per location

* refactor: use errors.Join for multi-error aggregation

* refactor: use existing ErrorWaitGroup for parallel execution

* fix: capture loop variables + clarify SuperBlockSize safety
2025-12-14 17:06:13 -08:00
Chris Lu
f77e6ed2d4 fix: admin UI bucket delete now properly deletes collection and checks Object Lock (#7734)
* fix: admin UI bucket delete now properly deletes collection and checks Object Lock

Fixes #7711

The admin UI's DeleteS3Bucket function was missing two critical behaviors:

1. It did not delete the collection from the master (unlike s3.bucket.delete
   shell command), leaving orphaned volume data that caused fs.verify errors.

2. It did not check for Object Lock protections before deletion, potentially
   allowing deletion of buckets with locked objects.

Changes:
- Add shared Object Lock checking utilities to object_lock_utils.go:
  - EntryHasActiveLock: standalone function to check if an entry has active lock
  - HasObjectsWithActiveLocks: shared function to scan bucket for locked objects
- Refactor S3 API entryHasActiveLock to use shared EntryHasActiveLock function
- Update admin UI DeleteS3Bucket to:
  - Check Object Lock using shared HasObjectsWithActiveLocks utility
  - Delete the collection before deleting filer entries (matching s3.bucket.delete)

* refactor: S3 API uses shared Object Lock utilities

Removes 114 lines of duplicated code from s3api_bucket_handlers.go by
having hasObjectsWithActiveLocks delegate to the shared HasObjectsWithActiveLocks
function in object_lock_utils.go.

Now both S3 API and Admin UI use the same shared utilities:
- EntryHasActiveLock
- HasObjectsWithActiveLocks
- recursivelyCheckLocksWithClient
- checkVersionsForLocksWithClient

* feat: s3.bucket.delete shell command now checks Object Lock

Add Object Lock protection to the s3.bucket.delete shell command.
If the bucket has Object Lock enabled and contains objects with active
retention or legal hold, deletion is prevented.

Also refactors Object Lock checking utilities into a new s3_objectlock
package to avoid import cycles between shell, s3api, and admin packages.

All three components now share the same logic:
- S3 API (DeleteBucketHandler)
- Admin UI (DeleteS3Bucket)
- Shell command (s3.bucket.delete)

* refactor: unified Object Lock checking and consistent deletion parameters

1. Add CheckBucketForLockedObjects() - a unified function that combines:
   - Bucket entry lookup
   - Object Lock enabled check
   - Scan for locked objects

2. All three components now use this single function:
   - S3 API (via s3api.CheckBucketForLockedObjects)
   - Admin UI (via s3api.CheckBucketForLockedObjects)
   - Shell command (via s3_objectlock.CheckBucketForLockedObjects)

3. Aligned deletion parameters across all components:
   - isDeleteData: false (collection already deleted separately)
   - isRecursive: true
   - ignoreRecursiveError: true

* fix: properly handle non-EOF errors in Recv() loops

The Recv() loops in recursivelyCheckLocksWithClient and
checkVersionsForLocksWithClient were breaking on any error, which
could hide real stream errors and incorrectly report 'no locks found'.

Now:
- io.EOF: break loop (normal end of stream)
- any other error: return it so caller knows the stream failed

* fix: address PR review comments

1. Add path traversal protection - validate entry names before building
   subdirectory paths. Skip entries with empty names, '.', '..', or
   containing path separators.

2. Use exact match for .versions folder instead of HasSuffix() to avoid
   mismatching unrelated directories like 'foo.versions'.

3. Replace path.Join with simple string concatenation since we now
   validate entry names.

* refactor: extract paginateEntries helper to reduce duplication

The recursivelyCheckLocksWithClient and checkVersionsForLocksWithClient
functions shared significant structural similarity. Extracted a generic
paginateEntries helper that:
- Handles pagination logic (lastFileName tracking, Limit)
- Handles stream receiving with proper EOF vs error handling
- Validates entry names (path traversal protection)
- Calls a processEntry callback for business logic

This centralizes pagination logic and makes the code more maintainable.

* feat: add context propagation for timeout and cancellation support

All Object Lock checking functions now accept context.Context parameter:
- paginateEntries(ctx, client, dir, processEntry)
- recursivelyCheckLocksWithClient(ctx, client, dir, hasLocks, currentTime)
- checkVersionsForLocksWithClient(ctx, client, versionsDir, hasLocks, currentTime)
- HasObjectsWithActiveLocks(ctx, client, bucketPath)
- CheckBucketForLockedObjects(ctx, client, bucketsPath, bucketName)

This enables:
- Timeout support for large bucket scans
- Cancellation propagation from HTTP requests
- The S3 API handler now uses r.Context() for proper request lifecycle

* fix: address PR review comments

1. Add DefaultBucketsPath constant in admin_server.go instead of
   hardcoding "/buckets" in multiple places.

2. Add defensive normalization in EntryHasActiveLock:
   - TrimSpace to handle whitespace around values
   - ToUpper for case-insensitive comparison of legal hold and
     retention mode values
   - TrimSpace on retention date before parsing

* fix: use ctx variable consistently instead of context.Background()

In both DeleteS3Bucket and command_s3_bucket_delete, use the ctx
variable defined at the start of the function for all gRPC calls
instead of creating new context.Background() instances.
2025-12-13 13:41:25 -08:00
Chris Lu
93cca3a96b volume.fsck: increase default cutoffTimeAgo from 5 minutes to 5 hours (#7730)
* volume.fsck: increase default cutoffTimeAgo from 5 minutes to 5 hours

This change makes the fsck check more conservative by only considering
chunks older than 5 hours as potential orphans. A 5 minute window was
too aggressive and could incorrectly flag recently written chunks,
especially in busy systems or during backup operations.

Addresses #7649

* Update command_volume_fsck.go

* volume.fsck: add help text explaining cutoffTimeAgo parameter

* Update command_volume_fsck.go
2025-12-12 23:42:27 -08:00
Lisandro Pin
662a6ac8ee Implement a weed shell command to return a status overview of the cluster. (#7704)
Detailed file information will be implemented in a follow-up MR. Note also
that masters are currently not reporting back EC shard sizes correctly, via
`master_pb.VolumeEcShardInformationMessage.shard_sizes`.

F.ex:

```
> cluster.status

cluster:
	id:       topo
	status:   LOCKED
	nodes:    10
	topology: 1 DC(s)s, 1 disk(s) on 1 rack(s)

volumes:
	total:    3 volumes on 1 collections
	max size: 31457280000 bytes
	regular:  2/80 volumes on 6 replicas, 6 writable (100.00%), 0 read-only (0.00%)
	EC:       1 EC volumes on 14 shards (14.00 shards/volume)

storage:
	total:           186024424 bytes
	regular volumes: 186024424 bytes
	EC volumes:      0 bytes
	raw:             558073152 bytes on volume replicas, 0 bytes on EC shard files
```
2025-12-12 18:07:59 -08:00
Chris Lu
a1eab5ff99 shell: add -owner flag to s3.bucket.create command (#7728)
* shell: add -owner flag to s3.bucket.create command

This fixes an issue where buckets created via weed shell cannot be accessed
by non-admin S3 users because the bucket has no owner set.

When using S3 IAM authentication, non-admin users can only access buckets
they own. Buckets created via lazy S3 creation automatically have their
owner set from the request context, but buckets created via weed shell
had no owner, making them inaccessible to non-admin users.

The new -owner flag allows setting the bucket owner identity (s3-identity-id)
at creation time:

    s3.bucket.create -name my-bucket -owner my-identity-name

Fixes: https://github.com/seaweedfs/seaweedfs/discussions/7599

* shell: add s3.bucket.owner command to view/change bucket ownership

This command allows viewing and changing the owner of an S3 bucket,
making it easier to manage bucket access for IAM users.

Usage:
    # View the current owner of a bucket
    s3.bucket.owner -name my-bucket

    # Set or change the owner of a bucket
    s3.bucket.owner -name my-bucket -set -owner new-identity

    # Remove the owner (make bucket admin-only)
    s3.bucket.owner -name my-bucket -set -owner ""

* shell: show bucket owner in s3.bucket.list output

Display the bucket owner (s3-identity-id) when listing buckets,
making it easier to see which identity owns each bucket.

Example output:
  my-bucket    size:1024    chunk:5    owner:my-identity

* admin: add bucket owner support to admin UI

- Add Owner field to S3Bucket struct for displaying bucket ownership
- Add Owner field to CreateBucketRequest for setting owner at creation
- Add UpdateBucketOwner API endpoint (PUT /api/s3/buckets/:bucket/owner)
- Add SetBucketOwner function for updating bucket ownership
- Update GetS3Buckets to populate owner from s3-identity-id extended attribute
- Update CreateS3BucketWithObjectLock to set owner when creating bucket

This allows the admin UI to display bucket owners and supports creating/
editing bucket ownership, which is essential for S3 IAM authentication
where non-admin users can only access buckets they own.

* admin: show bucket owner in buckets list and create form

- Add Owner column to buckets table to display bucket ownership
- Add Owner field to create bucket form for setting owner at creation
- Show owner in bucket details modal
- Update JavaScript to include owner when creating buckets

This makes bucket ownership visible and configurable from the admin UI,
which is essential for S3 IAM authentication where non-admin users can
only access buckets they own.

* admin: add bucket owner management with user dropdown

- Add 'Manage Owner' button to bucket actions
- Add modal with dropdown to select owner from existing users
- Fetch users from /api/users endpoint to populate dropdown
- Update create bucket form to use dropdown for owner selection
- Allow setting owner to empty (no owner = admin-only access)

This provides a user-friendly way to manage bucket ownership by selecting
from existing S3 identities rather than manually typing identity names.

* fix: use username instead of name for user dropdown

The /api/users endpoint returns 'username' field, not 'name'.
Fixed both the manage owner modal and create bucket form.

* Update s3_buckets_templ.go

* fix: address code review feedback for s3.bucket.create

- Check if entry.Extended is nil before making a new map to prevent
  overwriting any previously set extended attributes
- Use fmt.Fprintln(writer, ...) instead of println() for consistent
  output handling across the shell command framework

* fix: improve help text and validate owner input

- Add note that -owner value should match identity name in s3.json
- Trim whitespace from owner and treat whitespace-only as empty

* fix: address code review feedback for list and owner commands

- s3.bucket.list: Use %q to escape owner value and prevent malformed
  tabular output from special characters (tabs/newlines/control chars)
- s3.bucket.owner: Use neutral error message for lookup failures since
  they can occur for reasons other than missing bucket (e.g., permission)

* fix: improve s3.bucket.owner CLI UX

- Remove confusing -set flag that was required but not shown in examples
- Add explicit -delete flag to remove owner (safer than empty string)
- Presence of -owner now implies set operation (no extra flag needed)
- Validate that -owner and -delete cannot be used together
- Trim whitespace from owner value
- Update help text with correct examples and add note about identity name
- Clearer success messages for each operation

* fix: address code review feedback for admin UI

- GetBucketDetails: Extract and return owner from extended attributes
- CSV export: Fix column indices after adding Owner column, add Owner to header
- XSS prevention: Add escapeHtml() function to sanitize user data in innerHTML
  (bucket.name, bucket.owner, bucket.object_lock_mode, obj.key, obj.storage_class)

* fix: address additional code review feedback

- types.go: Add omitempty to Owner JSON tag, update comment
- bucket_management.go: Trim and validate owner (max 256 chars) in CreateBucket
- bucket_management.go: Use neutral error message in SetBucketOwner lookup

* fix: improve owner field handling and error recovery

bucket_management.go:
- Use *string pointer for Owner to detect if field was explicitly provided
- Return HTTP 400 if owner field is missing (use empty string to clear)
- Trim and validate owner (max 256 chars) in UpdateBucketOwner

s3_buckets.templ:
- Re-enable owner select dropdown on fetch error
- Reset dropdown to default 'No owner' option on error
- Allow users to retry or continue without selecting an owner

* fix: move modal instance variables to global scope

Move deleteModalInstance, quotaModalInstance, ownerModalInstance,
detailsModalInstance, and cachedUsers to global scope so they are
accessible from both DOMContentLoaded handlers and global functions
like deleteBucket(). This fixes the undefined variable issue.

* refactor: improve modal handling and avoid global window properties

- Initialize modal instances once on DOMContentLoaded and reuse with show()
- Replace window.currentBucket* global properties with data attributes on forms
- Remove modal dispose/recreate pattern and unnecessary cleanup code
- Scope state to relevant DOM elements instead of global namespace

* Update s3_buckets_templ.go

* fix: define MaxOwnerNameLength constant and implement RFC 4180 CSV escaping

bucket_management.go:
- Add MaxOwnerNameLength constant (256) with documentation
- Replace magic number 256 with constant in both validation checks

s3_buckets.templ:
- Add escapeCsvField() helper for RFC 4180 compliant CSV escaping
- Properly handle commas, double quotes, and newlines in field values
- Escape internal quotes by doubling them (")→("")

* Update s3_buckets_templ.go

* refactor: use direct gRPC client methods for consistency

- command_s3_bucket_create.go: Use client.CreateEntry instead of filer_pb.CreateEntry
- command_s3_bucket_owner.go: Use client.LookupDirectoryEntry instead of filer_pb.LookupEntry
- command_s3_bucket_owner.go: Use client.UpdateEntry instead of filer_pb.UpdateEntry

This aligns with the pattern used in weed/admin/dash/bucket_management.go
2025-12-12 18:06:13 -08:00
Chris Lu
e8b7347031 Reduce memory allocations in hot paths (#7725)
* filer: reduce allocations in MatchStorageRule

Optimize MatchStorageRule to avoid allocations in common cases:
- Return singleton emptyPathConf when no rules match (zero allocations)
- Return existing rule directly when only one rule matches (zero allocations)
- Only allocate and merge when multiple rules match (rare case)

Based on heap profile analysis showing 111MB allocated from 1.64M calls
to this function during 180 seconds of operation.

* filer: add fast path for getActualStore when no path-specific stores

Add hasPathSpecificStore flag to FilerStoreWrapper to skip
the MatchPrefix() call and []byte(path) conversion when no
path-specific stores are configured (the common case).

Based on heap profile analysis showing 1.39M calls to this
function during 180 seconds of operation, each requiring a
string-to-byte slice conversion for the MatchPrefix call.

* filer/foundationdb: use sync.Pool for tuple allocation in genKey

Use sync.Pool to reuse tuple.Tuple slices in genKey(), reducing
allocation overhead for every FoundationDB operation.

Based on heap profile analysis showing 102MB allocated from 1.79M
calls to genKey() during 180 seconds of operation. The Pack() call
still allocates internally, but this reduces the tuple slice
allocation overhead by ~50%.

* filer: use sync.Pool for protobuf Entry and FuseAttributes

Add pooling for filer_pb.Entry and filer_pb.FuseAttributes in
EncodeAttributesAndChunks and DecodeAttributesAndChunks to reduce
allocations during filer store operations.

Changes:
- Add pbEntryPool with pre-allocated FuseAttributes
- Add EntryAttributeToExistingPb for in-place attribute conversion
- Update ToExistingProtoEntry to reuse existing Attributes when available

Based on heap profile showing:
- EncodeAttributesAndChunks: 69.5MB cumulative
- DecodeAttributesAndChunks: 46.5MB cumulative
- EntryAttributeToPb: 47.5MB flat allocations

* log_buffer: use sync.Pool for LogEntry in readTs

Add logEntryPool to reuse filer_pb.LogEntry objects in readTs(),
which is called frequently during binary search in ReadFromBuffer.

This function only needs the TsNs field from the unmarshaled entry,
so pooling the LogEntry avoids repeated allocations.

Based on heap profile showing readTs with 188MB cumulative allocations
from timestamp lookups during log buffer reads.

* pb: reduce gRPC metadata allocations in interceptor

Optimize requestIDUnaryInterceptor and WithGrpcClient to reduce
metadata allocations on every gRPC request:

- Use AppendToOutgoingContext instead of NewOutgoingContext + New()
  This avoids creating a new map[string]string for single key-value pairs

- Check FromIncomingContext return value before using metadata

Based on heap profile showing metadata operations contributing 0.45GB
(10.5%) of allocations, with requestIDUnaryInterceptor being the main
source at 0.44GB cumulative.

Expected reduction: ~0.2GB from avoiding map allocations per request.

* filer/log_buffer: address code review feedback

- Use proto.Reset() instead of manual field clearing in resetLogEntry
  for more idiomatic and comprehensive state clearing
- Add resetPbEntry() call before pool return in error path for
  consistency with success path in DecodeAttributesAndChunks

* log_buffer: reduce PreviousBufferCount from 32 to 4

Reduce the number of retained previous buffers from 32 to 4.
Each buffer is 8MB, so this reduces the maximum retained memory
from 256MB to 32MB for previous buffers.

Most subscribers catch up quickly, so 4 buffers (32MB) should
be sufficient while significantly reducing memory footprint.

* filer/foundationdb: use defer for tuple pool cleanup in genKey

Refactor genKey to use defer for returning the pooled tuple.
This ensures the pooled object is always returned even if
store.seaweedfsDir.Pack panics, making the code more robust.

Also simplifies the code by removing the temporary variable.

* filer: early-stop MatchStorageRule prescan after 2 matches

Stop the prescan callback after finding 2 matches since we only
need to know if there are 0, 1, or multiple matches. This avoids
unnecessarily scanning the rest of the trie when many rules exist.

* fix: address critical code review issues

filer_conf.go:
- Remove mutable singleton emptyPathConf that could corrupt shared state
- Return fresh copy for no-match case and cloned copy for single-match case
- Add clonePathConf helper to create shallow copies safely

grpc_client_server.go:
- Remove incorrect AppendToOutgoingContext call in server interceptor
  (that API is for outbound client calls, not server-side handlers)
- Rely on request_id.Set and SetTrailer for request ID propagation

* fix: treat FilerConf_PathConf as immutable

Fix callers that were incorrectly mutating the returned PathConf:

- filer_server_handlers_write.go: Use local variable for MaxFileNameLength
  instead of mutating the shared rule

- command_s3_bucket_quota_check.go: Create new PathConf explicitly when
  modifying config instead of mutating the returned one

This allows MatchStorageRule to safely return the singleton or direct
references without copying, restoring the memory optimization.

Callers must NOT mutate the returned *FilerConf_PathConf.

* filer: add ClonePathConf helper for creating mutable copies

Add reusable ClonePathConf function that creates a mutable copy of
a PathConf. This is useful when callers need to modify config before
calling SetLocationConf.

Update command_s3_bucket_quota_check.go to use the new helper.

Also fix redundant return statement in DeleteLocationConf.

* fmt

* filer: fix protobuf pool reset to clear internal fields

Address code review feedback:

1. resetPbEntry/resetFuseAttributes: Use struct assignment (*e = T{})
   instead of field-by-field reset to clear protobuf internal fields
   (unknownFields, sizeCache) that would otherwise accumulate across
   pool reuses, causing data corruption or memory bloat.

2. EntryAttributeToExistingPb: Add nil guard for attr parameter to
   prevent panic if caller passes nil.

* log_buffer: reset logEntry before pool return in error path

For consistency with success path, reset the logEntry before putting
it back in the pool in the error path. This prevents the pooled object
from holding references to partially unmarshaled data.

* filer: optimize MatchStorageRule and document ClonePathConf

1. Avoid double []byte(path) conversion in multi-match case by
   converting once and reusing pathBytes.

2. Add IMPORTANT comment to ClonePathConf documenting that it must
   be kept in sync with filer_pb.FilerConf_PathConf fields when
   the protobuf evolves.

* filer/log_buffer: fix data race and use defer for pool cleanup

1. entry_codec.go EncodeAttributesAndChunks: Fix critical data race -
   proto.Marshal may return a slice sharing memory with the message.
   Copy the data before returning message to pool to prevent corruption.

2. entry_codec.go DecodeAttributesAndChunks: Use defer for cleaner
   pool management, ensuring message is always returned to pool.

3. log_buffer.go readTs: Use defer for pool cleanup, removing
   duplicated resetLogEntry/Put calls in success and error paths.

* filer: fix ClonePathConf field order and add comprehensive test

1. Fix field order in ClonePathConf to match protobuf struct definition
   (WormGracePeriodSeconds before WormRetentionTimeSeconds).

2. Add TestClonePathConf that constructs a fully-populated PathConf,
   calls ClonePathConf, and asserts equality of all exported fields.
   This will catch future schema drift when new fields are added.

3. Add TestClonePathConfNil to verify nil handling.

* filer: use reflection in ClonePathConf test to detect schema drift

Replace hardcoded field comparisons with reflection-based comparison.
This automatically catches:
1. New fields added to the protobuf but not copied in ClonePathConf
2. Missing non-zero test values for any exported field

The test iterates over all exported fields using reflect and compares
src vs clone values, failing if any field differs.

* filer: update EntryAttributeToExistingPb comment to reflect nil handling

The function safely handles nil attr by returning early, but the comment
incorrectly stated 'attr must not be nil'. Update comment to accurately
describe the defensive behavior.

* Fix review feedback: restore request ID propagation and remove redundant resets

1. grpc_client_server.go: Restore AppendToOutgoingContext for request ID
   so handlers making downstream gRPC calls will automatically propagate
   the request ID to downstream services.

2. entry_codec.go: Remove redundant resetPbEntry calls after Get.
   The defer block ensures reset before Put, so next Get receives clean object.

3. log_buffer.go: Remove redundant resetLogEntry call after Get for
   same reason - defer already handles reset before Put.
2025-12-12 12:51:48 -08:00
Chris Lu
df4f2f7020 ec: add -diskType flag to EC commands for SSD support (#7607)
* ec: add diskType parameter to core EC functions

Add diskType parameter to:
- ecBalancer struct
- collectEcVolumeServersByDc()
- collectEcNodesForDC()
- collectEcNodes()
- EcBalance()

This allows EC operations to target specific disk types (hdd, ssd, etc.)
instead of being hardcoded to HardDriveType only.

For backward compatibility, all callers currently pass types.HardDriveType
as the default value. Subsequent commits will add -diskType flags to
the individual EC commands.

* ec: update helper functions to use configurable diskType

Update the following functions to accept/use diskType parameter:
- findEcVolumeShards()
- addEcVolumeShards()
- deleteEcVolumeShards()
- moveMountedShardToEcNode()
- countShardsByRack()
- pickNEcShardsToMoveFrom()

All ecBalancer methods now use ecb.diskType instead of hardcoded
types.HardDriveType. Non-ecBalancer callers (like volumeServer.evacuate
and ec.rebuild) use types.HardDriveType as the default.

Update all test files to pass diskType where needed.

* ec: add -diskType flag to ec.balance and ec.encode commands

Add -diskType flag to specify the target disk type for EC operations:
- ec.balance -diskType=ssd
- ec.encode -diskType=ssd

The disk type can be 'hdd', 'ssd', or empty for default (hdd).
This allows placing EC shards on SSD or other disk types instead of
only HDD.

Example usage:
  ec.balance -collection=mybucket -diskType=ssd -apply
  ec.encode -collection=mybucket -diskType=ssd -force

* test: add integration tests for EC disk type support

Add integration tests to verify the -diskType flag works correctly:
- TestECDiskTypeSupport: Tests EC encode and balance with SSD disk type
- TestECDiskTypeMixedCluster: Tests EC operations on a mixed HDD/SSD cluster

The tests verify:
- Volume servers can be configured with specific disk types
- ec.encode accepts -diskType flag and encodes to the correct disk type
- ec.balance accepts -diskType flag and balances on the correct disk type
- Mixed disk type clusters work correctly with separate collections

* ec: add -sourceDiskType to ec.encode and -diskType to ec.decode

ec.encode:
- Add -sourceDiskType flag to filter source volumes by disk type
- This enables tier migration scenarios (e.g., SSD volumes → HDD EC shards)
- -diskType specifies target disk type for EC shards

ec.decode:
- Add -diskType flag to specify source disk type where EC shards are stored
- Update collectEcShardIds() and collectEcNodeShardBits() to accept diskType

Examples:
  # Encode SSD volumes to HDD EC shards (tier migration)
  ec.encode -collection=mybucket -sourceDiskType=ssd -diskType=hdd

  # Decode EC shards from SSD
  ec.decode -collection=mybucket -diskType=ssd

Integration tests updated to cover new flags.

* ec: fix variable shadowing and add -diskType to ec.rebuild and volumeServer.evacuate

Address code review comments:

1. Fix variable shadowing in collectEcVolumeServersByDc():
   - Rename loop variable 'diskType' to 'diskTypeKey' and 'diskTypeStr'
     to avoid shadowing the function parameter

2. Fix hardcoded HardDriveType in ecBalancer methods:
   - balanceEcRack(): use ecb.diskType instead of types.HardDriveType
   - collectVolumeIdToEcNodes(): use ecb.diskType

3. Add -diskType flag to ec.rebuild command:
   - Add diskType field to ecRebuilder struct
   - Pass diskType to collectEcNodes() and addEcVolumeShards()

4. Add -diskType flag to volumeServer.evacuate command:
   - Add diskType field to commandVolumeServerEvacuate struct
   - Pass diskType to collectEcVolumeServersByDc() and moveMountedShardToEcNode()

* test: add diskType field to ecBalancer in TestPickEcNodeToBalanceShardsInto

Address nitpick comment: ensure test ecBalancer struct has diskType
field set for consistency with other tests.

* ec: filter disk selection by disk type in pickBestDiskOnNode

When evacuating or rebalancing EC shards, pickBestDiskOnNode now
filters disks by the target disk type. This ensures:

1. EC shards from SSD disks are moved to SSD disks on destination nodes
2. EC shards from HDD disks are moved to HDD disks on destination nodes
3. No cross-disk-type shard movement occurs

This maintains the storage tier isolation when moving EC shards
between nodes during evacuation or rebalancing operations.

* ec: allow disk type fallback during evacuation

Update pickBestDiskOnNode to accept a strictDiskType parameter:

- strictDiskType=true (balancing): Only use disks of matching type.
  This maintains storage tier isolation during normal rebalancing.

- strictDiskType=false (evacuation): Prefer same disk type, but
  fall back to other disk types if no matching disk is available.
  This ensures evacuation can complete even when same-type capacity
  is insufficient.

Priority order for evacuation:
1. Same disk type with lowest shard count (preferred)
2. Different disk type with lowest shard count (fallback)

* test: use defer for lock/unlock to prevent lock leaks

Use defer to ensure locks are always released, even on early returns
or test failures. This prevents lock leaks that could cause subsequent
tests to hang or fail.

Changes:
- Return early if lock acquisition fails
- Immediately defer unlock after successful lock
- Remove redundant explicit unlock calls at end of tests
- Fix unused variable warning (err -> encodeErr/locErr)

* ec: dynamically discover disk types from topology for evacuation

Disk types are free-form tags (e.g., 'ssd', 'nvme', 'archive') that come
from the topology, not a hardcoded set. Only 'hdd' (or empty) is the
default disk type.

Use collectVolumeDiskTypes() to discover all disk types present in the
cluster topology instead of hardcoding [HardDriveType, SsdType].

* test: add evacuation fallback and cross-rack EC placement tests

Add two new integration tests:

1. TestEvacuationFallbackBehavior:
   - Tests that when same disk type has no capacity, shards fall back
     to other disk types during evacuation
   - Creates cluster with 1 SSD + 2 HDD servers (limited SSD capacity)
   - Verifies pickBestDiskOnNode behavior with strictDiskType=false

2. TestCrossRackECPlacement:
   - Tests EC shard distribution across different racks
   - Creates cluster with 4 servers in 4 different racks
   - Verifies shards are spread across multiple racks
   - Tests that ec.balance respects rack placement

Helper functions added:
- startLimitedSsdCluster: 1 SSD + 2 HDD servers
- startMultiRackCluster: 4 servers in 4 racks
- countShardsPerRack: counts EC shards per rack from disk

* test: fix collection mismatch in TestCrossRackECPlacement

The EC commands were using collection 'rack_test' but uploaded test data
uses collection 'test' (default). This caused ec.encode/ec.balance to not
find the uploaded volume.

Fix: Change EC commands to use '-collection test' to match the uploaded data.

Addresses review comment from PR #7607.

* test: close log files in MultiDiskCluster.Stop() to prevent FD leaks

Track log files in MultiDiskCluster.logFiles and close them in Stop()
to prevent file descriptor accumulation in long-running or many-test
scenarios.

Addresses review comment about logging resources cleanup.

* test: improve EC integration tests with proper assertions

- Add assertNoFlagError helper to detect flag parsing regressions
- Update diskType subtests to fail on flag errors (ec.encode, ec.balance, ec.decode)
- Update verify_disktype_flag_parsing to check help output contains diskType
- Remove verify_fallback_disk_selection (was documentation-only, not executable)
- Add assertion to verify_cross_rack_distribution for minimum 2 racks
- Consolidate uploadTestDataWithDiskType to accept collection parameter
- Remove duplicate uploadTestDataWithDiskTypeMixed function

* test: extract captureCommandOutput helper and fix error handling

- Add captureCommandOutput helper to reduce code duplication in diskType tests
- Create commandRunner interface to match shell command Do method
- Update ec_encode_with_ssd_disktype, ec_balance_with_ssd_disktype,
  ec_encode_with_source_disktype, ec_decode_with_disktype to use helper
- Fix filepath.Glob error handling in countShardsPerRack instead of ignoring it

* test: add flag validation to ec_balance_targets_correct_disk_type

Add assertNoFlagError calls after ec.balance commands to ensure
-diskType flag is properly recognized for both SSD and HDD disk types.

* test: add proper assertions for EC command results

- ec_encode_with_ssd_disktype: check for expected volume-related errors
- ec_balance_with_ssd_disktype: require success with require.NoError
- ec_encode_with_source_disktype: check for expected no-volume errors
- ec_decode_with_disktype: check for expected no-ec-volume errors
- upload_to_ssd_and_hdd: use require.NoError for setup validation

Tests now properly fail on unexpected errors rather than just logging.

* test: fix missing unlock in ec_encode_with_disk_awareness

Add defer unlock pattern to ensure lock is always released, matching
the pattern used in other subtests.

* test: improve helper robustness

- Make assertNoFlagError case-insensitive for pattern matching
- Use defer in captureCommandOutput to restore stdout/stderr and close
  pipe ends to avoid FD leaks even if cmd.Do panics
2025-12-10 22:42:52 -08:00
Lisandro Pin
ca1ad9c4c2 Nit: have ec.encode exit immediately if no volumes are processed. (#7654)
* 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>
2025-12-08 11:12:36 -08:00
Chris Lu
f9b4a4c396 fix: check freeEcSlot before evacuating EC shards to prevent data loss (#7621)
* 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.
2025-12-04 16:05:06 -08:00
Lisandro Pin
d59cc1b09f Fix handling of fixed read-only volumes for volume.check.disk. (#7612)
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.
2025-12-03 11:33:35 -08:00
Chris Lu
4f038820dc Add disk-aware EC rebalancing (#7597)
* 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
2025-12-02 12:30:15 -08:00
Lisandro Pin
ebb06a3908 Mutex command output writes for volume.check.disk. (#7605)
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>
2025-12-02 10:14:24 -08:00
Lisandro Pin
ee775825bc Parallelize read-only volume check pass for volume.check.disk. (#7602) 2025-12-02 09:29:27 -08:00
Lisandro Pin
36dd59560b Have volume.check.disk select a random (heathly) source volume when… (#7574)
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.
2025-12-01 10:58:29 -08:00
steve.wei
5c25df20f2 feat(volume.fix): show all replica locations for misplaced volumes (#7560) 2025-11-27 00:04:45 -08:00
Lisandro Pin
2843cb1255 Bootstrap logic to fix read-only volumes with volume.check.disk. (#7531)
* Bootstrap logic to fix read-only volumes with `volume.check.disk`.

The new implementation performs a second pass where read-only volumes are (optionally)
verified and fixed.

For each non-writable volume ID A:
  if volume is not full
    prune late volume entries not matching its index file
    select a writable volume replica B
    append missing entries from B into A
    mark the volume as writable (healthy)

* variable and parameter renaming

---------

Co-authored-by: chrislu <chris.lu@gmail.com>
2025-11-25 13:41:03 -08:00
Lisandro Pin
c89f394aba Parallelize ec.rebuild operations per affected volume. (#7466)
* Parallelize `ec.rebuild` operations per affected volume.

* node.freeEcSlot >= slotsNeeded

* variable names, help messages,

* Protected the read operation with the same mutex

* accurate error message

* fix broken test

---------

Co-authored-by: chrislu <chris.lu@gmail.com>
Co-authored-by: Chris Lu <chrislusf@users.noreply.github.com>
2025-11-21 17:58:37 -08:00
Lisandro Pin
3dd5348616 volume.check.disk: add support for uni- or bi-directional sync between volume replicas. (#7484)
* `volume.check.disk`: add support for uni- or bi-directional sync between volume replicas.

We'll need this to support repairing broken replicas, which involve syncing from a known good source replica without modifying it.

* S3: Lazy Versioning Check, Conditional SSE Entry Fetch, HEAD Request Optimization (#7480)

* Lazy Versioning Check, Conditional SSE Entry Fetch, HEAD Request Optimization

* revert

Reverted the conditional versioning check to always check versioning status
Reverted the conditional SSE entry fetch to always fetch entry metadata
Reverted the conditional versioning check to always check versioning status
Reverted the conditional SSE entry fetch to always fetch entry metadata

* Lazy Entry Fetch for SSE, Skip Conditional Header Check

* SSE-KMS headers are present, this is not an SSE-C request (mutually exclusive)

* SSE-C is mutually exclusive with SSE-S3 and SSE-KMS

* refactor

* Removed Premature Mutual Exclusivity Check

* check for the presence of the X-Amz-Server-Side-Encryption header

* not used

* fmt

* Volume Server: avoid aggressive volume assignment (#7501)

* avoid aggressive volume assignment

* also test ec shards

* separate DiskLocation instances for each subtest

* edge cases

* No volumes plus low disk space
* Multiple EC volumes

* simplify

* chore(deps): bump github.com/getsentry/sentry-go from 0.36.1 to 0.38.0 (#7498)

Bumps [github.com/getsentry/sentry-go](https://github.com/getsentry/sentry-go) from 0.36.1 to 0.38.0.
- [Release notes](https://github.com/getsentry/sentry-go/releases)
- [Changelog](https://github.com/getsentry/sentry-go/blob/master/CHANGELOG.md)
- [Commits](https://github.com/getsentry/sentry-go/compare/v0.36.1...v0.38.0)

---
updated-dependencies:
- dependency-name: github.com/getsentry/sentry-go
  dependency-version: 0.38.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): bump go.etcd.io/etcd/client/v3 from 3.6.5 to 3.6.6 (#7496)

Bumps [go.etcd.io/etcd/client/v3](https://github.com/etcd-io/etcd) from 3.6.5 to 3.6.6.
- [Release notes](https://github.com/etcd-io/etcd/releases)
- [Commits](https://github.com/etcd-io/etcd/compare/v3.6.5...v3.6.6)

---
updated-dependencies:
- dependency-name: go.etcd.io/etcd/client/v3
  dependency-version: 3.6.6
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): bump github.com/hanwen/go-fuse/v2 from 2.8.0 to 2.9.0 (#7495)

Bumps [github.com/hanwen/go-fuse/v2](https://github.com/hanwen/go-fuse) from 2.8.0 to 2.9.0.
- [Commits](https://github.com/hanwen/go-fuse/compare/v2.8.0...v2.9.0)

---
updated-dependencies:
- dependency-name: github.com/hanwen/go-fuse/v2
  dependency-version: 2.9.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): bump github.com/linxGnu/grocksdb from 1.10.2 to 1.10.3 (#7494)

Bumps [github.com/linxGnu/grocksdb](https://github.com/linxGnu/grocksdb) from 1.10.2 to 1.10.3.
- [Release notes](https://github.com/linxGnu/grocksdb/releases)
- [Commits](https://github.com/linxGnu/grocksdb/compare/v1.10.2...v1.10.3)

---
updated-dependencies:
- dependency-name: github.com/linxGnu/grocksdb
  dependency-version: 1.10.3
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): bump actions/dependency-review-action from 4.8.1 to 4.8.2 (#7493)

Bumps [actions/dependency-review-action](https://github.com/actions/dependency-review-action) from 4.8.1 to 4.8.2.
- [Release notes](https://github.com/actions/dependency-review-action/releases)
- [Commits](40c09b7dc9...3c4e3dcb1a)

---
updated-dependencies:
- dependency-name: actions/dependency-review-action
  dependency-version: 4.8.2
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): bump golang.org/x/image from 0.32.0 to 0.33.0 (#7497)

* chore(deps): bump golang.org/x/image from 0.32.0 to 0.33.0

Bumps [golang.org/x/image](https://github.com/golang/image) from 0.32.0 to 0.33.0.
- [Commits](https://github.com/golang/image/compare/v0.32.0...v0.33.0)

---
updated-dependencies:
- dependency-name: golang.org/x/image
  dependency-version: 0.33.0
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>

* go mod tidy

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: chrislu <chris.lu@gmail.com>

* chore: fix the diagram in RDMA sidecar readme (#7503)

* de/compress the fs meta file if filename ends with gz/gzip (#7500)

* de/compress the fs meta file if filename ends with gz/gzip

* gemini code review

* update help msg

* faster master startup

* chore(deps): bump org.apache.hadoop:hadoop-common from 3.2.4 to 3.4.0 in /other/java/hdfs2 (#7502)

chore(deps): bump org.apache.hadoop:hadoop-common in /other/java/hdfs2

Bumps org.apache.hadoop:hadoop-common from 3.2.4 to 3.4.0.

---
updated-dependencies:
- dependency-name: org.apache.hadoop:hadoop-common
  dependency-version: 3.4.0
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* S3: Directly read write volume servers (#7481)

* Lazy Versioning Check, Conditional SSE Entry Fetch, HEAD Request Optimization

* revert

Reverted the conditional versioning check to always check versioning status
Reverted the conditional SSE entry fetch to always fetch entry metadata
Reverted the conditional versioning check to always check versioning status
Reverted the conditional SSE entry fetch to always fetch entry metadata

* Lazy Entry Fetch for SSE, Skip Conditional Header Check

* SSE-KMS headers are present, this is not an SSE-C request (mutually exclusive)

* SSE-C is mutually exclusive with SSE-S3 and SSE-KMS

* refactor

* Removed Premature Mutual Exclusivity Check

* check for the presence of the X-Amz-Server-Side-Encryption header

* not used

* fmt

* directly read write volume servers

* HTTP Range Request Support

* set header

* md5

* copy object

* fix sse

* fmt

* implement sse

* sse continue

* fixed the suffix range bug (bytes=-N for "last N bytes")

* debug logs

* Missing PartsCount Header

* profiling

* url encoding

* test_multipart_get_part

* headers

* debug

* adjust log level

* handle part number

* Update s3api_object_handlers.go

* nil safety

* set ModifiedTsNs

* remove

* nil check

* fix sse header

* same logic as filer

* decode values

* decode ivBase64

* s3: Fix SSE decryption JWT authentication and streaming errors

Critical fix for SSE (Server-Side Encryption) test failures:

1. **JWT Authentication Bug** (Root Cause):
   - Changed from GenJwtForFilerServer to GenJwtForVolumeServer
   - S3 API now uses correct JWT when directly reading from volume servers
   - Matches filer's authentication pattern for direct volume access
   - Fixes 'unexpected EOF' and 500 errors in SSE tests

2. **Streaming Error Handling**:
   - Added error propagation in getEncryptedStreamFromVolumes goroutine
   - Use CloseWithError() to properly communicate stream failures
   - Added debug logging for streaming errors

3. **Response Header Timing**:
   - Removed premature WriteHeader(http.StatusOK) call
   - Let Go's http package write status automatically on first write
   - Prevents header lock when errors occur during streaming

4. **Enhanced SSE Decryption Debugging**:
   - Added IV/Key validation and logging for SSE-C, SSE-KMS, SSE-S3
   - Better error messages for missing or invalid encryption metadata
   - Added glog.V(2) debugging for decryption setup

This fixes SSE integration test failures where encrypted objects
could not be retrieved due to volume server authentication failures.
The JWT bug was causing volume servers to reject requests, resulting
in truncated/empty streams (EOF) or internal errors.

* s3: Fix SSE multipart upload metadata preservation

Critical fix for SSE multipart upload test failures (SSE-C and SSE-KMS):

**Root Cause - Incomplete SSE Metadata Copying**:
The old code only tried to copy 'SeaweedFSSSEKMSKey' from the first
part to the completed object. This had TWO bugs:

1. **Wrong Constant Name** (Key Mismatch Bug):
   - Storage uses: SeaweedFSSSEKMSKeyHeader = 'X-SeaweedFS-SSE-KMS-Key'
   - Old code read: SeaweedFSSSEKMSKey = 'x-seaweedfs-sse-kms-key'
   - Result: SSE-KMS metadata was NEVER copied → 500 errors

2. **Missing SSE-C and SSE-S3 Headers**:
   - SSE-C requires: IV, Algorithm, KeyMD5
   - SSE-S3 requires: encrypted key data + standard headers
   - Old code: copied nothing for SSE-C/SSE-S3 → decryption failures

**Fix - Complete SSE Header Preservation**:
Now copies ALL SSE headers from first part to completed object:

- SSE-C: SeaweedFSSSEIV, CustomerAlgorithm, CustomerKeyMD5
- SSE-KMS: SeaweedFSSSEKMSKeyHeader, AwsKmsKeyId, ServerSideEncryption
- SSE-S3: SeaweedFSSSES3Key, ServerSideEncryption

Applied consistently to all 3 code paths:
1. Versioned buckets (creates version file)
2. Suspended versioning (creates main object with null versionId)
3. Non-versioned buckets (creates main object)

**Why This Is Correct**:
The headers copied EXACTLY match what putToFiler stores during part
upload (lines 496-521 in s3api_object_handlers_put.go). This ensures
detectPrimarySSEType() can correctly identify encrypted multipart
objects and trigger inline decryption with proper metadata.

Fixes: TestSSEMultipartUploadIntegration (SSE-C and SSE-KMS subtests)

* s3: Add debug logging for versioning state diagnosis

Temporary debug logging to diagnose test_versioning_obj_plain_null_version_overwrite_suspended failure.

Added glog.V(0) logging to show:
1. setBucketVersioningStatus: when versioning status is changed
2. PutObjectHandler: what versioning state is detected (Enabled/Suspended/none)
3. PutObjectHandler: which code path is taken (putVersionedObject vs putSuspendedVersioningObject)

This will help identify if:
- The versioning status is being set correctly in bucket config
- The cache is returning stale/incorrect versioning state
- The switch statement is correctly routing to suspended vs enabled handlers

* s3: Enhanced versioning state tracing for suspended versioning diagnosis

Added comprehensive logging across the entire versioning state flow:

PutBucketVersioningHandler:
- Log requested status (Enabled/Suspended)
- Log when calling setBucketVersioningStatus
- Log success/failure of status change

setBucketVersioningStatus:
- Log bucket and status being set
- Log when config is updated
- Log completion with error code

updateBucketConfig:
- Log versioning state being written to cache
- Immediate cache verification after Set
- Log if cache verification fails

getVersioningState:
- Log bucket name and state being returned
- Log if object lock forces VersioningEnabled
- Log errors

This will reveal:
1. If PutBucketVersioning(Suspended) is reaching the handler
2. If the cache update succeeds
3. What state getVersioningState returns during PUT
4. Any cache consistency issues

Expected to show why bucket still reports 'Enabled' after 'Suspended' call.

* s3: Add SSE chunk detection debugging for multipart uploads

Added comprehensive logging to diagnose why TestSSEMultipartUploadIntegration fails:

detectPrimarySSEType now logs:
1. Total chunk count and extended header count
2. All extended headers with 'sse'/'SSE'/'encryption' in the name
3. For each chunk: index, SseType, and whether it has metadata
4. Final SSE type counts (SSE-C, SSE-KMS, SSE-S3)

This will reveal if:
- Chunks are missing SSE metadata after multipart completion
- Extended headers are copied correctly from first part
- The SSE detection logic is working correctly

Expected to show if chunks have SseType=0 (none) or proper SSE types set.

* s3: Trace SSE chunk metadata through multipart completion and retrieval

Added end-to-end logging to track SSE chunk metadata lifecycle:

**During Multipart Completion (filer_multipart.go)**:
1. Log finalParts chunks BEFORE mkFile - shows SseType and metadata
2. Log versionEntry.Chunks INSIDE mkFile callback - shows if mkFile preserves SSE info
3. Log success after mkFile completes

**During GET Retrieval (s3api_object_handlers.go)**:
1. Log retrieved entry chunks - shows SseType and metadata after retrieval
2. Log detected SSE type result

This will reveal at which point SSE chunk metadata is lost:
- If finalParts have SSE metadata but versionEntry.Chunks don't → mkFile bug
- If versionEntry.Chunks have SSE metadata but retrieved chunks don't → storage/retrieval bug
- If chunks never have SSE metadata → multipart completion SSE processing bug

Expected to show chunks with SseType=NONE during retrieval even though
they were created with proper SseType during multipart completion.

* s3: Fix SSE-C multipart IV base64 decoding bug

**Critical Bug Found**: SSE-C multipart uploads were failing because:

Root Cause:
- entry.Extended[SeaweedFSSSEIV] stores base64-encoded IV (24 bytes for 16-byte IV)
- SerializeSSECMetadata expects raw IV bytes (16 bytes)
- During multipart completion, we were passing base64 IV directly → serialization error

Error Message:
"Failed to serialize SSE-C metadata for chunk in part X: invalid IV length: expected 16 bytes, got 24"

Fix:
- Base64-decode IV before passing to SerializeSSECMetadata
- Added error handling for decode failures

Impact:
- SSE-C multipart uploads will now correctly serialize chunk metadata
- Chunks will have proper SSE metadata for decryption during GET

This fixes the SSE-C subtest of TestSSEMultipartUploadIntegration.
SSE-KMS still has a separate issue (error code 23) being investigated.

* fixes

* kms sse

* handle retry if not found in .versions folder and should read the normal object

* quick check (no retries) to see if the .versions/ directory exists

* skip retry if object is not found

* explicit update to avoid sync delay

* fix map update lock

* Remove fmt.Printf debug statements

* Fix SSE-KMS multipart base IV fallback to fail instead of regenerating

* fmt

* Fix ACL grants storage logic

* header handling

* nil handling

* range read for sse content

* test range requests for sse objects

* fmt

* unused code

* upload in chunks

* header case

* fix url

* bucket policy error vs bucket not found

* jwt handling

* fmt

* jwt in request header

* Optimize Case-Insensitive Prefix Check

* dead code

* Eliminated Unnecessary Stream Prefetch for Multipart SSE

* range sse

* sse

* refactor

* context

* fmt

* fix type

* fix SSE-C IV Mismatch

* Fix Headers Being Set After WriteHeader

* fix url parsing

* propergate sse headers

* multipart sse-s3

* aws sig v4 authen

* sse kms

* set content range

* better errors

* Update s3api_object_handlers_copy.go

* Update s3api_object_handlers.go

* Update s3api_object_handlers.go

* avoid magic number

* clean up

* Update s3api_bucket_policy_handlers.go

* fix url parsing

* context

* data and metadata both use background context

* adjust the offset

* SSE Range Request IV Calculation

* adjust logs

* IV relative to offset in each part, not the whole file

* collect logs

* offset

* fix offset

* fix url

* logs

* variable

* jwt

* Multipart ETag semantics: conditionally set object-level Md5 for single-chunk uploads only.

* sse

* adjust IV and offset

* multipart boundaries

* ensures PUT and GET operations return consistent ETags

* Metadata Header Case

* CommonPrefixes Sorting with URL Encoding

* always sort

* remove the extra PathUnescape call

* fix the multipart get part ETag

* the FileChunk is created without setting ModifiedTsNs

* Sort CommonPrefixes lexicographically to match AWS S3 behavior

* set md5 for multipart uploads

* prevents any potential data loss or corruption in the small-file inline storage path

* compiles correctly

* decryptedReader will now be properly closed after use

* Fixed URL encoding and sort order for CommonPrefixes

* Update s3api_object_handlers_list.go

* SSE-x Chunk View Decryption

* Different IV offset calculations for single-part vs multipart objects

* still too verbose in logs

* less logs

* ensure correct conversion

* fix listing

* nil check

* minor fixes

* nil check

* single character delimiter

* optimize

* range on empty object or zero-length

* correct IV based on its position within that part, not its position in the entire object

* adjust offset

* offset

Fetch FULL encrypted chunk (not just the range)
Adjust IV by PartOffset/ChunkOffset only
Decrypt full chunk
Skip in the DECRYPTED stream to reach OffsetInChunk

* look breaking

* refactor

* error on no content

* handle intra-block byte skipping

* Incomplete HTTP Response Error Handling

* multipart SSE

* Update s3api_object_handlers.go

* address comments

* less logs

* handling directory

* Optimized rejectDirectoryObjectWithoutSlash() to avoid unnecessary lookups

* Revert "handling directory"

This reverts commit 3a335f0ac33c63f51975abc63c40e5328857a74b.

* constant

* Consolidate nil entry checks in GetObjectHandler

* add range tests

* Consolidate redundant nil entry checks in HeadObjectHandler

* adjust logs

* SSE type

* large files

* large files

Reverted the plain-object range test

* ErrNoEncryptionConfig

* Fixed SSERangeReader Infinite Loop Vulnerability

* Fixed SSE-KMS Multipart ChunkReader HTTP Body Leak

* handle empty directory in S3, added PyArrow tests

* purge unused code

* Update s3_parquet_test.py

* Update requirements.txt

* According to S3 specifications, when both partNumber and Range are present, the Range should apply within the selected part's boundaries, not to the full object.

* handle errors

* errors after writing header

* https

* fix: Wait for volume assignment readiness before running Parquet tests

The test-implicit-dir-with-server test was failing with an Internal Error
because volume assignment was not ready when tests started. This fix adds
a check that attempts a volume assignment and waits for it to succeed
before proceeding with tests.

This ensures that:
1. Volume servers are registered with the master
2. Volume growth is triggered if needed
3. The system can successfully assign volumes for writes

Fixes the timeout issue where boto3 would retry 4 times and fail with
'We encountered an internal error, please try again.'

* sse tests

* store derived IV

* fix: Clean up gRPC ports between tests to prevent port conflicts

The second test (test-implicit-dir-with-server) was failing because the
volume server's gRPC port (18080 = VOLUME_PORT + 10000) was still in use
from the first test. The cleanup code only killed HTTP port processes,
not gRPC port processes.

Added cleanup for gRPC ports in all stop targets:
- Master gRPC: MASTER_PORT + 10000 (19333)
- Volume gRPC: VOLUME_PORT + 10000 (18080)
- Filer gRPC: FILER_PORT + 10000 (18888)

This ensures clean state between test runs in CI.

* add import

* address comments

* docs: Add placeholder documentation files for Parquet test suite

Added three missing documentation files referenced in test/s3/parquet/README.md:

1. TEST_COVERAGE.md - Documents 43 total test cases (17 Go unit tests,
   6 Python integration tests, 20 Python end-to-end tests)

2. FINAL_ROOT_CAUSE_ANALYSIS.md - Explains the s3fs compatibility issue
   with PyArrow, the implicit directory problem, and how the fix works

3. MINIO_DIRECTORY_HANDLING.md - Compares MinIO's directory handling
   approach with SeaweedFS's implementation

Each file contains:
- Title and overview
- Key technical details relevant to the topic
- TODO sections for future expansion

These placeholder files resolve the broken README links and provide
structure for future detailed documentation.

* clean up if metadata operation failed

* Update s3_parquet_test.py

* clean up

* Update Makefile

* Update s3_parquet_test.py

* Update Makefile

* Handle ivSkip for non-block-aligned offsets

* Update README.md

* stop volume server faster

* stop volume server in 1 second

* different IV for each chunk in SSE-S3 and SSE-KMS

* clean up if fails

* testing upload

* error propagation

* fmt

* simplify

* fix copying

* less logs

* endian

* Added marshaling error handling

* handling invalid ranges

* error handling for adding to log buffer

* fix logging

* avoid returning too quickly and ensure proper cleaning up

* Activity Tracking for Disk Reads

* Cleanup Unused Parameters

* Activity Tracking for Kafka Publishers

* Proper Test Error Reporting

* refactoring

* less logs

* less logs

* go fmt

* guard it with if entry.Attributes.TtlSec > 0 to match the pattern used elsewhere.

* Handle bucket-default encryption config errors explicitly for multipart

* consistent activity tracking

* obsolete code for s3 on filer read/write handlers

* Update weed/s3api/s3api_object_handlers_list.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>

* S3: Add tests for PyArrow with native S3 filesystem (#7508)

* PyArrow native S3 filesystem

* add sse-s3 tests

* update

* minor

* ENABLE_SSE_S3

* Update test_pyarrow_native_s3.py

* clean up

* refactoring

* Update test_pyarrow_native_s3.py

* filer store: add foundationdb (#7178)

* add foundationdb

* Update foundationdb_store.go

* fix

* apply the patch

* avoid panic on error

* address comments

* remove extra data

* address comments

* adds more debug messages

* fix range listing

* delete with prefix range; list with right start key

* fix docker files

* use the more idiomatic FoundationDB KeySelectors

* address comments

* proper errors

* fix API versions

* more efficient

* recursive deletion

* clean up

* clean up

* pagination, one transaction for deletion

* error checking

* Use fdb.Strinc() to compute the lexicographically next string and create a proper range

* fix docker

* Update README.md

* delete in batches

* delete in batches

* fix build

* add foundationdb build

* Updated FoundationDB Version

* Fixed glibc/musl Incompatibility (Alpine → Debian)

* Update container_foundationdb_version.yml

* build SeaweedFS

* build tag

* address comments

* separate transaction

* address comments

* fix build

* empty vs no data

* fixes

* add go test

* Install FoundationDB client libraries

* nil compare

* chore(deps): bump golang.org/x/crypto from 0.43.0 to 0.45.0 in /test/kafka/kafka-client-loadtest (#7510)

chore(deps): bump golang.org/x/crypto

Bumps [golang.org/x/crypto](https://github.com/golang/crypto) from 0.43.0 to 0.45.0.
- [Commits](https://github.com/golang/crypto/compare/v0.43.0...v0.45.0)

---
updated-dependencies:
- dependency-name: golang.org/x/crypto
  dependency-version: 0.45.0
  dependency-type: indirect
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Account Info (#7507)

* Account Info

Add account info on s3.configure

* address comments

* Update command_s3_configure.go

---------

Co-authored-by: chrislu <chris.lu@gmail.com>

* chore(deps): bump org.apache.hadoop:hadoop-common from 3.2.4 to 3.4.0 in /other/java/hdfs-over-ftp (#7513)

chore(deps): bump org.apache.hadoop:hadoop-common

Bumps org.apache.hadoop:hadoop-common from 3.2.4 to 3.4.0.

---
updated-dependencies:
- dependency-name: org.apache.hadoop:hadoop-common
  dependency-version: 3.4.0
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): bump golang.org/x/crypto from 0.43.0 to 0.45.0 (#7511)

* chore(deps): bump golang.org/x/crypto from 0.43.0 to 0.45.0

Bumps [golang.org/x/crypto](https://github.com/golang/crypto) from 0.43.0 to 0.45.0.
- [Commits](https://github.com/golang/crypto/compare/v0.43.0...v0.45.0)

---
updated-dependencies:
- dependency-name: golang.org/x/crypto
  dependency-version: 0.45.0
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <support@github.com>

* go mod tidy

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: chrislu <chris.lu@gmail.com>

* chore(deps): bump org.apache.hadoop:hadoop-common from 3.2.4 to 3.4.0 in /other/java/hdfs3 (#7512)

* chore(deps): bump org.apache.hadoop:hadoop-common in /other/java/hdfs3

Bumps org.apache.hadoop:hadoop-common from 3.2.4 to 3.4.0.

---
updated-dependencies:
- dependency-name: org.apache.hadoop:hadoop-common
  dependency-version: 3.4.0
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <support@github.com>

* add java client unit tests

* Update dependency-reduced-pom.xml

* add java integration tests

* fix

* fix buffer

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: chrislu <chris.lu@gmail.com>

* S3: JWT generation for volume server authentication (#7514)

* Refactor JWT generation for volume server authentication to use centralized function from filer package, improving code clarity and reducing redundancy.

* Update s3api_object_handlers.go

* S3: S3 Object Retention API to include XML namespace support (#7517)

* Refactor S3 Object Retention API to include XML namespace support and improve compatibility with Veeam. Updated XML tags to remove hardcoded namespaces and added test cases for retention and legal hold configurations without namespaces.

* Added XMLNS field setting in both places

* S3: adds FilerClient to use cached volume id (#7518)

* adds FilerClient to use cached volume id

* refactor: MasterClient embeds vidMapClient to eliminate ~150 lines of duplication

- Create masterVolumeProvider that implements VolumeLocationProvider
- MasterClient now embeds vidMapClient instead of maintaining duplicate cache logic
- Removed duplicate methods: LookupVolumeIdsWithFallback, getStableVidMap, etc.
- MasterClient still receives real-time updates via KeepConnected streaming
- Updates call inherited addLocation/deleteLocation from vidMapClient
- Benefits: DRY principle, shared singleflight, cache chain logic reused
- Zero behavioral changes - only architectural improvement

* refactor: mount uses FilerClient for efficient volume location caching

- Add configurable vidMap cache size (default: 5 historical snapshots)
- Add FilerClientOption struct for clean configuration
  * GrpcTimeout: default 5 seconds (prevents hanging requests)
  * UrlPreference: PreferUrl or PreferPublicUrl
  * CacheSize: number of historical vidMap snapshots (for volume moves)
- NewFilerClient uses option struct for better API extensibility
- Improved error handling in filerVolumeProvider.LookupVolumeIds:
  * Distinguish genuine 'not found' from communication failures
  * Log volumes missing from filer response
  * Return proper error context with volume count
  * Document that filer Locations lacks Error field (unlike master)
- FilerClient.GetLookupFileIdFunction() handles URL preference automatically
- Mount (WFS) creates FilerClient with appropriate options
- Benefits for weed mount:
  * Singleflight: Deduplicates concurrent volume lookups
  * Cache history: Old volume locations available briefly when volumes move
  * Configurable cache depth: Tune for different deployment environments
  * Battle-tested vidMap cache with cache chain
  * Better concurrency handling with timeout protection
  * Improved error visibility and debugging
- Old filer.LookupFn() kept for backward compatibility
- Performance improvement for mount operations with high concurrency

* fix: prevent vidMap swap race condition in LookupFileIdWithFallback

- Hold vidMapLock.RLock() during entire vm.LookupFileId() call
- Prevents resetVidMap() from swapping vidMap mid-operation
- Ensures atomic access to the current vidMap instance
- Added documentation warnings to getStableVidMap() about swap risks
- Enhanced withCurrentVidMap() documentation for clarity

This fixes a subtle race condition where:
1. Thread A: acquires lock, gets vm pointer, releases lock
2. Thread B: calls resetVidMap(), swaps vc.vidMap
3. Thread A: calls vm.LookupFileId() on old/stale vidMap

While the old vidMap remains valid (in cache chain), holding the lock
ensures we consistently use the current vidMap for the entire operation.

* fix: FilerClient supports multiple filer addresses for high availability

Critical fix: FilerClient now accepts []ServerAddress instead of single address
- Prevents mount failure when first filer is down (regression fix)
- Implements automatic failover to remaining filers
- Uses round-robin with atomic index tracking (same pattern as WFS.WithFilerClient)
- Retries all configured filers before giving up
- Updates successful filer index for future requests

Changes:
- NewFilerClient([]pb.ServerAddress, ...) instead of (pb.ServerAddress, ...)
- filerVolumeProvider references FilerClient for failover access
- LookupVolumeIds tries all filers with util.Retry pattern
- Mount passes all option.FilerAddresses for HA
- S3 wraps single filer in slice for API consistency

This restores the high availability that existed in the old implementation
where mount would automatically failover between configured filers.

* fix: restore leader change detection in KeepConnected stream loop

Critical fix: Leader change detection was accidentally removed from the streaming loop
- Master can announce leader changes during an active KeepConnected stream
- Without this check, client continues talking to non-leader until connection breaks
- This can lead to stale data or operational errors

The check needs to be in TWO places:
1. Initial response (lines 178-187): Detect redirect on first connect
2. Stream loop (lines 203-209): Detect leader changes during active stream

Restored the loop check that was accidentally removed during refactoring.
This ensures the client immediately reconnects to new leader when announced.

* improve: address code review findings on error handling and documentation

1. Master provider now preserves per-volume errors
   - Surface detailed errors from master (e.g., misconfiguration, deletion)
   - Return partial results with aggregated errors using errors.Join
   - Callers can now distinguish specific volume failures from general errors
   - Addresses issue of losing vidLoc.Error details

2. Document GetMaster initialization contract
   - Add comprehensive documentation explaining blocking behavior
   - Clarify that KeepConnectedToMaster must be started first
   - Provide typical initialization pattern example
   - Prevent confusing timeouts during warm-up

3. Document partial results API contract
   - LookupVolumeIdsWithFallback explicitly documents partial results
   - Clear examples of how to handle result + error combinations
   - Helps prevent callers from discarding valid partial results

4. Add safeguards to legacy filer.LookupFn
   - Add deprecation warning with migration guidance
   - Implement simple 10,000 entry cache limit
   - Log warning when limit reached
   - Recommend wdclient.FilerClient for new code
   - Prevents unbounded memory growth in long-running processes

These changes improve API clarity and operational safety while maintaining
backward compatibility.

* fix: handle partial results correctly in LookupVolumeIdsWithFallback callers

Two callers were discarding partial results by checking err before processing
the result map. While these are currently single-volume lookups (so partial
results aren't possible), the code was fragile and would break if we ever
batched multiple volumes together.

Changes:
- Check result map FIRST, then conditionally check error
- If volume is found in result, use it (ignore errors about other volumes)
- If volume is NOT found and err != nil, include error context with %w
- Add defensive comments explaining the pattern for future maintainers

This makes the code:
1. Correct for future batched lookups
2. More informative (preserves underlying error details)
3. Consistent with filer_grpc_server.go which already handles this correctly

Example: If looking up ["1", "2", "999"] and only 999 fails, callers
looking for volumes 1 or 2 will succeed instead of failing unnecessarily.

* improve: address remaining code review findings

1. Lazy initialize FilerClient in mount for proxy-only setups
   - Only create FilerClient when VolumeServerAccess != "filerProxy"
   - Avoids wasted work when all reads proxy through filer
   - filerClient is nil for proxy mode, initialized for direct access

2. Fix inaccurate deprecation comment in filer.LookupFn
   - Updated comment to reflect current behavior (10k bounded cache)
   - Removed claim of "unbounded growth" after adding size limit
   - Still directs new code to wdclient.FilerClient for better features

3. Audit all MasterClient usages for KeepConnectedToMaster
   - Verified all production callers start KeepConnectedToMaster early
   - Filer, Shell, Master, Broker, Benchmark, Admin all correct
   - IAM creates MasterClient but never uses it (harmless)
   - Test code doesn't need KeepConnectedToMaster (mocks)

All callers properly follow the initialization pattern documented in
GetMaster(), preventing unexpected blocking or timeouts.

* fix: restore observability instrumentation in MasterClient

During the refactoring, several important stats counters and logging
statements were accidentally removed from tryConnectToMaster. These are
critical for monitoring and debugging the health of master client connections.

Restored instrumentation:
1. stats.MasterClientConnectCounter("total") - tracks all connection attempts
2. stats.MasterClientConnectCounter(FailedToKeepConnected) - when KeepConnected stream fails
3. stats.MasterClientConnectCounter(FailedToReceive) - when Recv() fails in loop
4. stats.MasterClientConnectCounter(Failed) - when overall gprcErr occurs
5. stats.MasterClientConnectCounter(OnPeerUpdate) - when peer updates detected

Additionally restored peer update logging:
- "+ filer@host noticed group.type address" for node additions
- "- filer@host noticed group.type address" for node removals
- Only logs updates matching the client's FilerGroup for noise reduction

This information is valuable for:
- Monitoring cluster health and connection stability
- Debugging cluster membership changes
- Tracking master failover and reconnection patterns
- Identifying network issues between clients and masters

No functional changes - purely observability restoration.

* improve: implement gRPC-aware retry for FilerClient volume lookups

The previous implementation used util.Retry which only retries errors
containing the string "transport". This is insufficient for handling
the full range of transient gRPC errors.

Changes:
1. Added isRetryableGrpcError() to properly inspect gRPC status codes
   - Retries: Unavailable, DeadlineExceeded, ResourceExhausted, Aborted
   - Falls back to string matching for non-gRPC network errors

2. Replaced util.Retry with custom retry loop
   - 3 attempts with exponential backoff (1s, 1.5s, 2.25s)
   - Tries all N filers on each attempt (N*3 total attempts max)
   - Fast-fails on non-retryable errors (NotFound, PermissionDenied, etc.)

3. Improved logging
   - Shows both filer attempt (x/N) and retry attempt (y/3)
   - Logs retry reason and wait time for debugging

Benefits:
- Better handling of transient gRPC failures (server restarts, load spikes)
- Faster failure for permanent errors (no wasted retries)
- More informative logs for troubleshooting
- Maintains existing HA failover across multiple filers

Example: If all 3 filers return Unavailable (server overload):
- Attempt 1: try all 3 filers, wait 1s
- Attempt 2: try all 3 filers, wait 1.5s
- Attempt 3: try all 3 filers, fail

Example: If filer returns NotFound (volume doesn't exist):
- Attempt 1: try all 3 filers, fast-fail (no retry)

* fmt

* improve: add circuit breaker to skip known-unhealthy filers

The previous implementation tried all filers on every failure, including
known-unhealthy ones. This wasted time retrying permanently down filers.

Problem scenario (3 filers, filer0 is down):
- Last successful: filer1 (saved as filerIndex=1)
- Next lookup when filer1 fails:
  Retry 1: filer1(fail) → filer2(fail) → filer0(fail, wastes 5s timeout)
  Retry 2: filer1(fail) → filer2(fail) → filer0(fail, wastes 5s timeout)
  Retry 3: filer1(fail) → filer2(fail) → filer0(fail, wastes 5s timeout)
  Total wasted: 15 seconds on known-bad filer!

Solution: Circuit breaker pattern
- Track consecutive failures per filer (atomic int32)
- Skip filers with 3+ consecutive failures
- Re-check unhealthy filers every 30 seconds
- Reset failure count on success

New behavior:
- filer0 fails 3 times → marked unhealthy
- Future lookups skip filer0 for 30 seconds
- After 30s, re-check filer0 (allows recovery)
- If filer0 succeeds, reset failure count to 0

Benefits:
1. Avoids wasting time on known-down filers
2. Still sticks to last healthy filer (via filerIndex)
3. Allows recovery (30s re-check window)
4. No configuration needed (automatic)

Implementation details:
- filerHealth struct tracks failureCount (atomic) + lastFailureTime
- shouldSkipUnhealthyFiler(): checks if we should skip this filer
- recordFilerSuccess(): resets failure count to 0
- recordFilerFailure(): increments count, updates timestamp
- Logs when skipping unhealthy filers (V(2) level)

Example with circuit breaker:
- filer0 down, saved filerIndex=1 (filer1 healthy)
- Lookup 1: filer1(ok) → Done (0.01s)
- Lookup 2: filer1(fail) → filer2(ok) → Done, save filerIndex=2 (0.01s)
- Lookup 3: filer2(fail) → skip filer0 (unhealthy) → filer1(ok) → Done (0.01s)

Much better than wasting 15s trying filer0 repeatedly!

* fix: OnPeerUpdate should only process updates for matching FilerGroup

Critical bug: The OnPeerUpdate callback was incorrectly moved outside the
FilerGroup check when restoring observability instrumentation. This caused
clients to process peer updates for ALL filer groups, not just their own.

Problem:
  Before: mc.OnPeerUpdate only called for update.FilerGroup == mc.FilerGroup
  Bug:    mc.OnPeerUpdate called for ALL updates regardless of FilerGroup

Impact:
- Multi-tenant deployments with separate filer groups would see cross-group
  updates (e.g., group A clients processing group B updates)
- Could cause incorrect cluster membership tracking
- OnPeerUpdate handlers (like Filer's DLM ring updates) would receive
  irrelevant updates from other groups

Example scenario:
  Cluster has two filer groups: "production" and "staging"
  Production filer connects with FilerGroup="production"

  Incorrect behavior (bug):
    - Receives "staging" group updates
    - Incorrectly adds staging filers to production DLM ring
    - Cross-tenant data access issues

  Correct behavior (fixed):
    - Only receives "production" group updates
    - Only adds production filers to production DLM ring
    - Proper isolation between groups

Fix:
  Moved mc.OnPeerUpdate(update, time.Now()) back INSIDE the FilerGroup check
  where it belongs, matching the original implementation.

The logging and stats counter were already correctly scoped to matching
FilerGroup, so they remain inside the if block as intended.

* improve: clarify Aborted error handling in volume lookups

Added documentation and logging to address the concern that codes.Aborted
might not always be retryable in all contexts.

Context-specific justification for treating Aborted as retryable:

Volume location lookups (LookupVolume RPC) are simple, read-only operations:
  - No transactions
  - No write conflicts
  - No application-level state changes
  - Idempotent (safe to retry)

In this context, Aborted is most likely caused by:
  - Filer restarting/recovering (transient)
  - Connection interrupted mid-request (transient)
  - Server-side resource cleanup (transient)

NOT caused by:
  - Application-level conflicts (no writes)
  - Transaction failures (no transactions)
  - Logical errors (read-only lookup)

Changes:
1. Added detailed comment explaining the context-specific reasoning
2. Added V(1) logging when treating Aborted as retryable
   - Helps detect misclassification if it occurs
   - Visible in verbose logs for troubleshooting
3. Split switch statement for clarity (one case per line)

If future analysis shows Aborted should not be retried, operators will
now have visibility via logs to make that determination. The logging
provides evidence for future tuning decisions.

Alternative approaches considered but not implemented:
  - Removing Aborted entirely (too conservative for read-only ops)
  - Message content inspection (adds complexity, no known patterns yet)
  - Different handling per RPC type (premature optimization)

* fix: IAM server must start KeepConnectedToMaster for masterClient usage

The IAM server creates and uses a MasterClient but never started
KeepConnectedToMaster, which could cause blocking if IAM config files
have chunks requiring volume lookups.

Problem flow:
  NewIamApiServerWithStore()
    → creates masterClient
    →  NEVER starts KeepConnectedToMaster

  GetS3ApiConfigurationFromFiler()
    → filer.ReadEntry(iama.masterClient, ...)
      → StreamContent(masterClient, ...) if file has chunks
        → masterClient.GetLookupFileIdFunction()
          → GetMaster(ctx) ← BLOCKS indefinitely waiting for connection!

While IAM config files (identity & policies) are typically small and
stored inline without chunks, the code path exists and would block
if the files ever had chunks.

Fix:
  Start KeepConnectedToMaster in background goroutine right after
  creating masterClient, following the documented pattern:

    mc := wdclient.NewMasterClient(...)
    go mc.KeepConnectedToMaster(ctx)

This ensures masterClient is usable if ReadEntry ever needs to
stream chunked content from volume servers.

Note: This bug was dormant because IAM config files are small (<256 bytes)
and SeaweedFS stores small files inline in Entry.Content, not as chunks.
The bug would only manifest if:
  - IAM config grew > 256 bytes (inline threshold)
  - Config was stored as chunks on volume servers
  - ReadEntry called StreamContent
  - GetMaster blocked indefinitely

Now all 9 production MasterClient instances correctly follow the pattern.

* fix: data race on filerHealth.lastFailureTime in circuit breaker

The circuit breaker tracked lastFailureTime as time.Time, which was
written in recordFilerFailure and read in shouldSkipUnhealthyFiler
without synchronization, causing a data race.

Data race scenario:
  Goroutine 1: recordFilerFailure(0)
    health.lastFailureTime = time.Now()  //  unsynchronized write

  Goroutine 2: shouldSkipUnhealthyFiler(0)
    time.Since(health.lastFailureTime)   //  unsynchronized read

  → RACE DETECTED by -race detector

Fix:
  Changed lastFailureTime from time.Time to int64 (lastFailureTimeNs)
  storing Unix nanoseconds for atomic access:

  Write side (recordFilerFailure):
    atomic.StoreInt64(&health.lastFailureTimeNs, time.Now().UnixNano())

  Read side (shouldSkipUnhealthyFiler):
    lastFailureNs := atomic.LoadInt64(&health.lastFailureTimeNs)
    if lastFailureNs == 0 { return false }  // Never failed
    lastFailureTime := time.Unix(0, lastFailureNs)
    time.Since(lastFailureTime) > 30*time.Second

Benefits:
  - Atomic reads/writes (no data race)
  - Efficient (int64 is 8 bytes, always atomic on 64-bit systems)
  - Zero value (0) naturally means "never failed"
  - No mutex needed (lock-free circuit breaker)

Note: sync/atomic was already imported for failureCount, so no new
import needed.

* fix: create fresh timeout context for each filer retry attempt

The timeout context was created once at function start and reused across
all retry attempts, causing subsequent retries to run with progressively
shorter (or expired) deadlines.

Problem flow:
  Line 244: timeoutCtx, cancel := context.WithTimeout(ctx, 5s)
  defer cancel()

  Retry 1, filer 0: client.LookupVolume(timeoutCtx, ...) ← 5s available 
  Retry 1, filer 1: client.LookupVolume(timeoutCtx, ...) ← 3s left
  Retry 1, filer 2: client.LookupVolume(timeoutCtx, ...) ← 0.5s left
  Retry 2, filer 0: client.LookupVolume(timeoutCtx, ...) ← EXPIRED! 

Result: Retries always fail with DeadlineExceeded, defeating the purpose
of retries.

Fix:
  Moved context.WithTimeout inside the per-filer loop, creating a fresh
  timeout context for each attempt:

    for x := 0; x < n; x++ {
      timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
      err := pb.WithGrpcFilerClient(..., func(client) {
        resp, err := client.LookupVolume(timeoutCtx, ...)
        ...
      })
      cancel()  // Clean up immediately after call
    }

Benefits:
  - Each filer attempt gets full fc.grpcTimeout (default 5s)
  - Retries actually have time to complete
  - No context leaks (cancel called after each attempt)
  - More predictable timeout behavior

Example with fix:
  Retry 1, filer 0: fresh 5s timeout 
  Retry 1, filer 1: fresh 5s timeout 
  Retry 2, filer 0: fresh 5s timeout 

Total max time: 3 retries × 3 filers × 5s = 45s (plus backoff)

Note: The outer ctx (from caller) still provides overall cancellation if
the caller cancels or times out the entire operation.

* fix: always reset vidMap cache on master reconnection

The previous refactoring removed the else block that resets vidMap when
the first message from a newly connected master is not a VolumeLocation.

Problem scenario:
  1. Client connects to master-1 and builds vidMap cache
  2. Master-1 fails, client connects to master-2
  3. First message from master-2 is a ClusterNodeUpdate (not VolumeLocation)
  4. Old code: vidMap is reset and updated 
  5. New code: vidMap is NOT reset 
  6. Result: Client uses stale cache from master-1 → data access errors

Example flow with bug:
  Connect to master-2
  First message: ClusterNodeUpdate {filer.x added}
  → No resetVidMap() call
  → vidMap still has master-1's stale volume locations
  → Client reads from wrong volume servers → 404 errors

Fix:
  Restored the else block that resets vidMap when first message is not
  a VolumeLocation:

    if resp.VolumeLocation != nil {
      // ... check leader, reset, and update ...
    } else {
      // First message is ClusterNodeUpdate or other type
      // Must still reset to avoid stale data
      mc.resetVidMap()
    }

This ensures the cache is always cleared when establishing a new master
connection, regardless of what the first message type is.

Root cause:
  During the vidMapClient refactoring, this else block was accidentally
  dropped, making failover behavior fragile and non-deterministic (depends
  on which message type arrives first from the new master).

Impact:
  - High severity for master failover scenarios
  - Could cause read failures, 404s, or wrong data access
  - Only manifests when first message is not VolumeLocation

* fix: goroutine and connection leak in IAM server shutdown

The IAM server's KeepConnectedToMaster goroutine used context.Background(),
which is non-cancellable, causing the goroutine and its gRPC connections
to leak on server shutdown.

Problem:
  go masterClient.KeepConnectedToMaster(context.Background())

  - context.Background() never cancels
  - KeepConnectedToMaster goroutine runs forever
  - gRPC connection to master stays open
  - No way to stop cleanly on server shutdown

Result: Resource leaks when IAM server is stopped

Fix:
  1. Added shutdownContext and shutdownCancel to IamApiServer struct
  2. Created cancellable context in NewIamApiServerWithStore:
       shutdownCtx, shutdownCancel := context.WithCancel(context.Background())
  3. Pass shutdownCtx to KeepConnectedToMaster:
       go masterClient.KeepConnectedToMaster(shutdownCtx)
  4. Added Shutdown() method to invoke cancel:
       func (iama *IamApiServer) Shutdown() {
           if iama.shutdownCancel != nil {
               iama.shutdownCancel()
           }
       }

  5. Stored masterClient reference on IamApiServer for future use

Benefits:
  - Goroutine stops cleanly when Shutdown() is called
  - gRPC connections are closed properly
  - No resource leaks on server restart/stop
  - Shutdown() is idempotent (safe to call multiple times)

Usage (for future graceful shutdown):
  iamServer, _ := iamapi.NewIamApiServer(...)
  defer iamServer.Shutdown()

  // or in signal handler:
  sigChan := make(chan os.Signal, 1)
  signal.Notify(sigChan, syscall.SIGTERM, syscall.SIGINT)
  go func() {
      <-sigChan
      iamServer.Shutdown()
      os.Exit(0)
  }()

Note: Current command implementations (weed/command/iam.go) don't have
shutdown paths yet, but this makes IAM server ready for proper lifecycle
management when that infrastructure is added.

* refactor: remove unnecessary KeepMasterClientConnected wrapper in filer

The Filer.KeepMasterClientConnected() method was an unnecessary wrapper that
just forwarded to MasterClient.KeepConnectedToMaster(). This wrapper added
no value and created inconsistency with other components that call
KeepConnectedToMaster directly.

Removed:
  filer.go:178-180
    func (fs *Filer) KeepMasterClientConnected(ctx context.Context) {
        fs.MasterClient.KeepConnectedToMaster(ctx)
    }

Updated caller:
  filer_server.go:181
    - go fs.filer.KeepMasterClientConnected(context.Background())
    + go fs.filer.MasterClient.KeepConnectedToMaster(context.Background())

Benefits:
  - Consistent with other components (S3, IAM, Shell, Mount)
  - Removes unnecessary indirection
  - Clearer that KeepConnectedToMaster runs in background goroutine
  - Follows the documented pattern from MasterClient.GetMaster()

Note: shell/commands.go was verified and already correctly starts
KeepConnectedToMaster in a background goroutine (shell_liner.go:51):
  go commandEnv.MasterClient.KeepConnectedToMaster(ctx)

* fix: use client ID instead of timeout for gRPC signature parameter

The pb.WithGrpcFilerClient signature parameter is meant to be a client
identifier for logging and tracking (added as 'sw-client-id' gRPC metadata
in streaming mode), not a timeout value.

Problem:
  timeoutMs := int32(fc.grpcTimeout.Milliseconds())  // 5000 (5 seconds)
  err := pb.WithGrpcFilerClient(false, timeoutMs, filerAddress, ...)

  - Passing timeout (5000ms) as signature/client ID
  - Misuse of API: signature should be a unique client identifier
  - Timeout is already handled by timeoutCtx passed to gRPC call
  - Inconsistent with other callers (all use 0 or proper client ID)

How WithGrpcFilerClient uses signature parameter:
  func WithGrpcClient(..., signature int32, ...) {
    if streamingMode && signature != 0 {
      md := metadata.New(map[string]string{"sw-client-id": fmt.Sprintf("%d", signature)})
      ctx = metadata.NewOutgoingContext(ctx, md)
    }
    ...
  }

It's for client identification, not timeout control!

Fix:
  1. Added clientId int32 field to FilerClient struct
  2. Initialize with rand.Int31() in NewFilerClient for unique ID
  3. Removed timeoutMs variable (and misleading comment)
  4. Use fc.clientId in pb.WithGrpcFilerClient call

Before:
  err := pb.WithGrpcFilerClient(false, timeoutMs, ...)
                                      ^^^^^^^^^ Wrong! (5000)

After:
  err := pb.WithGrpcFilerClient(false, fc.clientId, ...)
                                      ^^^^^^^^^^^^ Correct! (random int31)

Benefits:
  - Correct API usage (signature = client ID, not timeout)
  - Timeout still works via timeoutCtx (unchanged)
  - Consistent with other pb.WithGrpcFilerClient callers
  - Enables proper client tracking on filer side via gRPC metadata
  - Each FilerClient instance has unique ID for debugging

Examples of correct usage elsewhere:
  weed/iamapi/iamapi_server.go:145     pb.WithGrpcFilerClient(false, 0, ...)
  weed/command/s3.go:215               pb.WithGrpcFilerClient(false, 0, ...)
  weed/shell/commands.go:110           pb.WithGrpcFilerClient(streamingMode, 0, ...)

All use 0 (or a proper signature), not a timeout value.

* fix: add timeout to master volume lookup to prevent indefinite blocking

The masterVolumeProvider.LookupVolumeIds method was using the context
directly without a timeout, which could cause it to block indefinitely
if the master is slow to respond or unreachable.

Problem:
  err := pb.WithMasterClient(false, p.masterClient.GetMaster(ctx), ...)
  resp, err := client.LookupVolume(ctx, &master_pb.LookupVolumeRequest{...})

  - No timeout on gRPC call to master
  - Could block indefinitely if master is unresponsive
  - Inconsistent with FilerClient which uses 5s timeout
  - This is a fallback path (cache miss) but still needs protection

Scenarios where this could hang:
  1. Master server under heavy load (slow response)
  2. Network issues between client and master
  3. Master server hung or deadlocked
  4. Master in process of shutting down

Fix:
  timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Second)
  defer cancel()

  err := pb.WithMasterClient(false, p.masterClient.GetMaster(timeoutCtx), ...)
  resp, err := client.LookupVolume(timeoutCtx, &master_pb.LookupVolumeRequest{...})

Benefits:
  - Prevents indefinite blocking on master lookup
  - Consistent with FilerClient timeout pattern (5 seconds)
  - Faster failure detection when master is unresponsive
  - Caller's context still honored (timeout is in addition, not replacement)
  - Improves overall system resilience

Note: 5 seconds is a reasonable default for volume lookups:
  - Long enough for normal master response (~10-50ms)
  - Short enough to fail fast on issues
  - Matches FilerClient's grpcTimeout default

* purge

* refactor: address code review feedback on comments and style

Fixed several code quality issues identified during review:

1. Corrected backoff algorithm description in filer_client.go:
   - Changed "Exponential backoff" to "Multiplicative backoff with 1.5x factor"
   - The formula waitTime * 3/2 produces 1s, 1.5s, 2.25s, not exponential 2^n
   - More accurate terminology prevents confusion

2. Removed redundant nil check in vidmap_client.go:
   - After the for loop, node is guaranteed to be non-nil
   - Loop either returns early or assigns non-nil value to node
   - Simplified: if node != nil { node.cache.Store(nil) } → node.cache.Store(nil)

3. Added startup logging to IAM server for consistency:
   - Log when master client connection starts
   - Matches pattern in S3ApiServer (line 100 in s3api_server.go)
   - Improves operational visibility during startup
   - Added missing glog import

4. Fixed indentation in filer/reader_at.go:
   - Lines 76-91 had incorrect indentation (extra tab level)
   - Line 93 also misaligned
   - Now properly aligned with surrounding code

5. Updated deprecation comment to follow Go convention:
   - Changed "DEPRECATED:" to "Deprecated:" (standard Go format)
   - Tools like staticcheck and IDEs recognize the standard format
   - Enables automated deprecation warnings in tooling
   - Better developer experience

All changes are cosmetic and do not affect functionality.

* fmt

* refactor: make circuit breaker parameters configurable in FilerClient

The circuit breaker failure threshold (3) and reset timeout (30s) were
hardcoded, making it difficult to tune the client's behavior in different
deployment environments without modifying the code.

Problem:
  func shouldSkipUnhealthyFiler(index int32) bool {
    if failureCount < 3 {              // Hardcoded threshold
      return false
    }
    if time.Since(lastFailureTime) > 30*time.Second {  // Hardcoded timeout
      return false
    }
  }

Different environments have different needs:
  - High-traffic production: may want lower threshold (2) for faster failover
  - Development/testing: may want higher threshold (5) to tolerate flaky networks
  - Low-latency services: may want shorter reset timeout (10s)
  - Batch processing: may want longer reset timeout (60s)

Solution:
  1. Added fields to FilerClientOption:
     - FailureThreshold int32 (default: 3)
     - ResetTimeout time.Duration (default: 30s)

  2. Added fields to FilerClient:
     - failureThreshold int32
     - resetTimeout time.Duration

  3. Applied defaults in NewFilerClient with option override:
     failureThreshold := int32(3)
     resetTimeout := 30 * time.Second
     if opt.FailureThreshold > 0 {
       failureThreshold = opt.FailureThreshold
     }
     if opt.ResetTimeout > 0 {
       resetTimeout = opt.ResetTimeout
     }

  4. Updated shouldSkipUnhealthyFiler to use configurable values:
     if failureCount < fc.failureThreshold { ... }
     if time.Since(lastFailureTime) > fc.resetTimeout { ... }

Benefits:
  ✓ Tunable for different deployment environments
  ✓ Backward compatible (defaults match previous hardcoded values)
  ✓ No breaking changes to existing code
  ✓ Better maintainability and flexibility

Example usage:
  // Aggressive failover for low-latency production
  fc := wdclient.NewFilerClient(filers, dialOpt, dc, &wdclient.FilerClientOption{
    FailureThreshold: 2,
    ResetTimeout:     10 * time.Second,
  })

  // Tolerant of flaky networks in development
  fc := wdclient.NewFilerClient(filers, dialOpt, dc, &wdclient.FilerClientOption{
    FailureThreshold: 5,
    ResetTimeout:     60 * time.Second,
  })

* retry parameters

* refactor: make retry and timeout parameters configurable

Made retry logic and gRPC timeouts configurable across FilerClient and
MasterClient to support different deployment environments and network
conditions.

Problem 1: Hardcoded retry parameters in FilerClient
  waitTime := time.Second          // Fixed at 1s
  maxRetries := 3                  // Fixed at 3 attempts
  waitTime = waitTime * 3 / 2      // Fixed 1.5x multiplier

Different environments have different needs:
  - Unstable networks: may want more retries (5) with longer waits (2s)
  - Low-latency production: may want fewer retries (2) with shorter waits (500ms)
  - Batch processing: may want exponential backoff (2x) instead of 1.5x

Problem 2: Hardcoded gRPC timeout in MasterClient
  timeoutCtx, cancel := context.WithTimeout(ctx, 5*time.Second)

Master lookups may need different timeouts:
  - High-latency cross-region: may need 10s timeout
  - Local network: may use 2s timeout for faster failure detection

Solution for FilerClient:
  1. Added fields to FilerClientOption:
     - MaxRetries int (default: 3)
     - InitialRetryWait time.Duration (default: 1s)
     - RetryBackoffFactor float64 (default: 1.5)

  2. Added fields to FilerClient:
     - maxRetries int
     - initialRetryWait time.Duration
     - retryBackoffFactor float64

  3. Updated LookupVolumeIds to use configurable values:
     waitTime := fc.initialRetryWait
     maxRetries := fc.maxRetries
     for retry := 0; retry < maxRetries; retry++ {
       ...
       waitTime = time.Duration(float64(waitTime) * fc.retryBackoffFactor)
     }

Solution for MasterClient:
  1. Added grpcTimeout field to MasterClient (default: 5s)
  2. Initialize in NewMasterClient with 5 * time.Second default
  3. Updated masterVolumeProvider to use p.masterClient.grpcTimeout

Benefits:
  ✓ Tunable for different network conditions and deployment scenarios
  ✓ Backward compatible (defaults match previous hardcoded values)
  ✓ No breaking changes to existing code
  ✓ Consistent configuration pattern across FilerClient and MasterClient

Example usage:
  // Fast-fail for low-latency production with stable network
  fc := wdclient.NewFilerClient(filers, dialOpt, dc, &wdclient.FilerClientOption{
    MaxRetries:         2,
    InitialRetryWait:   500 * time.Millisecond,
    RetryBackoffFactor: 2.0,  // Exponential backoff
    GrpcTimeout:        2 * time.Second,
  })

  // Patient retries for unstable network or batch processing
  fc := wdclient.NewFilerClient(filers, dialOpt, dc, &wdclient.FilerClientOption{
    MaxRetries:         5,
    InitialRetryWait:   2 * time.Second,
    RetryBackoffFactor: 1.5,
    GrpcTimeout:        10 * time.Second,
  })

Note: MasterClient timeout is currently set at construction time and not
user-configurable via NewMasterClient parameters. Future enhancement could
add a MasterClientOption struct similar to FilerClientOption.

* fix: rename vicCacheLock to vidCacheLock for consistency

Fixed typo in variable name for better code consistency and readability.

Problem:
  vidCache := make(map[string]*filer_pb.Locations)
  var vicCacheLock sync.RWMutex  // Typo: vic instead of vid

  vicCacheLock.RLock()
  locations, found := vidCache[vid]
  vicCacheLock.RUnlock()

The variable name 'vicCacheLock' is inconsistent with 'vidCache'.
Both should use 'vid' prefix (volume ID) not 'vic'.

Fix:
  Renamed all 5 occurrences:
  - var vicCacheLock → var vidCacheLock (line 56)
  - vicCacheLock.RLock() → vidCacheLock.RLock() (line 62)
  - vicCacheLock.RUnlock() → vidCacheLock.RUnlock() (line 64)
  - vicCacheLock.Lock() → vidCacheLock.Lock() (line 81)
  - vicCacheLock.Unlock() → vidCacheLock.Unlock() (line 91)

Benefits:
  ✓ Consistent variable naming convention
  ✓ Clearer intent (volume ID cache lock)
  ✓ Better code readability
  ✓ Easier code navigation

* fix: use defer cancel() with anonymous function for proper context cleanup

Fixed context cancellation to use defer pattern correctly in loop iteration.

Problem:
  for x := 0; x < n; x++ {
    timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
    err := pb.WithGrpcFilerClient(...)
    cancel() // Only called on normal return, not on panic
  }

Issues with original approach:
  1. If pb.WithGrpcFilerClient panics, cancel() is never called → context leak
  2. If callback returns early (though unlikely here), cleanup might be missed
  3. Not following Go best practices for context.WithTimeout usage

Problem with naive defer in loop:
  for x := 0; x < n; x++ {
    timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
    defer cancel() //  WRONG: All defers accumulate until function returns
  }

In Go, defer executes when the surrounding *function* returns, not when
the loop iteration ends. This would accumulate n deferred cancel() calls
and leak contexts until LookupVolumeIds returns.

Solution: Wrap in anonymous function
  for x := 0; x < n; x++ {
    err := func() error {
      timeoutCtx, cancel := context.WithTimeout(ctx, fc.grpcTimeout)
      defer cancel() //  Executes when anonymous function returns (per iteration)
      return pb.WithGrpcFilerClient(...)
    }()
  }

Benefits:
  ✓ Context always cancelled, even on panic
  ✓ defer executes after each iteration (not accumulated)
  ✓ Follows Go best practices for context.WithTimeout
  ✓ No resource leaks during retry loop execution
  ✓ Cleaner error handling

Reference:
  Go documentation for context.WithTimeout explicitly shows:
    ctx, cancel := context.WithTimeout(...)
    defer cancel()

This is the idiomatic pattern that should always be followed.

* Can't use defer directly in loop

* improve: add data center preference and URL shuffling for consistent performance

Added missing data center preference and load distribution (URL shuffling)
to ensure consistent performance and behavior across all code paths.

Problem 1: PreferPublicUrl path missing DC preference and shuffling
Location: weed/wdclient/filer_client.go lines 184-192

The custom PreferPublicUrl implementation was simply iterating through
locations and building URLs without considering:
  1. Data center proximity (latency optimization)
  2. Load distribution across volume servers

Before:
  for _, loc := range locations {
    url := loc.PublicUrl
    if url == "" { url = loc.Url }
    fullUrls = append(fullUrls, "http://"+url+"/"+fileId)
  }
  return fullUrls, nil

After:
  var sameDcUrls, otherDcUrls []string
  dataCenter := fc.GetDataCenter()
  for _, loc := range locations {
    url := loc.PublicUrl
    if url == "" { url = loc.Url }
    httpUrl := "http://" + url + "/" + fileId
    if dataCenter != "" && dataCenter == loc.DataCenter {
      sameDcUrls = append(sameDcUrls, httpUrl)
    } else {
      otherDcUrls = append(otherDcUrls, httpUrl)
    }
  }
  rand.Shuffle(len(sameDcUrls), ...)
  rand.Shuffle(len(otherDcUrls), ...)
  fullUrls = append(sameDcUrls, otherDcUrls...)

Problem 2: Cache miss path missing URL shuffling
Location: weed/wdclient/vidmap_client.go lines 95-108

The cache miss path (fallback lookup) was missing URL shuffling, while
the cache hit path (vm.LookupFileId) already shuffles URLs. This
inconsistency meant:
  - Cache hit: URLs shuffled → load distributed
  - Cache miss: URLs not shuffled → first server always hit

Before:
  var sameDcUrls, otherDcUrls []string
  // ... build URLs ...
  fullUrls = append(sameDcUrls, otherDcUrls...)
  return fullUrls, nil

After:
  var sameDcUrls, otherDcUrls []string
  // ... build URLs ...
  rand.Shuffle(len(sameDcUrls), ...)
  rand.Shuffle(len(otherDcUrls), ...)
  fullUrls = append(sameDcUrls, otherDcUrls...)
  return fullUrls, nil

Benefits:
  ✓ Reduced latency by preferring same-DC volume servers
  ✓ Even load distribution across all volume servers
  ✓ Consistent behavior between cache hit/miss paths
  ✓ Consistent behavior between PreferUrl and PreferPublicUrl
  ✓ Matches behavior of existing vidMap.LookupFileId implementation

Impact on performance:
  - Lower read latency (same-DC preference)
  - Better volume server utilization (load spreading)
  - No single volume server becomes a hotspot

Note: Added math/rand import to vidmap_client.go for shuffle support.

* Update weed/wdclient/masterclient.go

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* improve: call IAM server Shutdown() for best-effort cleanup

Added call to iamApiServer.Shutdown() to ensure cleanup happens when possible,
and documented the limitations of the current approach.

Problem:
  The Shutdown() method was defined in IamApiServer but never called anywhere,
  meaning the KeepConnectedToMaster goroutine would continue running even when
  the IAM server stopped, causing resource leaks.

Changes:
  1. Store iamApiServer instance in weed/command/iam.go
     - Changed: _, iamApiServer_err := iamapi.NewIamApiServer(...)
     - To: iamApiServer, iamApiServer_err := iamapi.NewIamApiServer(...)

  2. Added defer call for best-effort cleanup
     - defer iamApiServer.Shutdown()
     - This will execute if startIamServer() returns normally

  3. Added logging in Shutdown() method
     - Log when shutdown is triggered for visibility

  4. Documented limitations and future improvements
     - Added note that defer only works for normal function returns
     - SeaweedFS commands don't currently have signal handling
     - Suggested future enhancement: add SIGTERM/SIGINT handling

Current behavior:
  - ✓ Cleanup happens if HTTP server fails to start (glog.Fatalf path)
  - ✓ Cleanup happens if Serve() returns with error (unlikely)
  - ✗ Cleanup does NOT happen on SIGTERM/SIGINT (process killed)

The last case is a limitation of the current command architecture - all
SeaweedFS commands (s3, filer, volume, master, iam) lack signal handling
for graceful shutdown. This is a systemic issue that affects all services.

Future enhancement:
  To properly handle SIGTERM/SIGINT, the command layer would need:

    sigChan := make(chan os.Signal, 1)
    signal.Notify(sigChan, syscall.SIGTERM, syscall.SIGINT)

    go func() {
      httpServer.Serve(listener) // Non-blocking
    }()

    <-sigChan
    glog.V(0).Infof("Received shutdown signal")
    iamApiServer.Shutdown()
    httpServer.Shutdown(context.Background())

This would require refactoring the command structure for all services,
which is out of scope for this change.

Benefits of current approach:
  ✓ Best-effort cleanup (better than nothing)
  ✓ Proper cleanup in error paths
  ✓ Documented for future improvement
  ✓ Consistent with how other SeaweedFS services handle lifecycle

* data racing in test

---------

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>

* test read write by s3fs and PyArrow native file system for s3 (#7520)

* test read write by s3fs and PyArrow native file system for s3

* address comments

* add github action

* S3: list owned buckets (#7519)

* list owned buckets

* simplify

* add unit tests

* no-owner buckets

* set identity id

* fallback to request header if iam is not enabled

* refactor to test

* fix comparing

* fix security vulnerability

* Update s3api_bucket_handlers.go

* Update s3api_bucket_handlers.go

* Update s3api_bucket_handlers.go

* S3: set identity to request context, and remove obsolete code (#7523)

* list owned buckets

* simplify

* add unit tests

* no-owner buckets

* set identity id

* fallback to request header if iam is not enabled

* refactor to test

* fix comparing

* fix security vulnerability

* Update s3api_bucket_handlers.go

* Update s3api_bucket_handlers.go

* Update s3api_bucket_handlers.go

* set identity to request context

* remove SeaweedFSIsDirectoryKey

* remove obsolete

* simplify

* reuse

* refactor or remove obsolete logic on filer

* Removed the redundant check in GetOrHeadHandler

* surfacing invalid X-Amz-Tagging as a client error

* clean up

* constant

* reuse

* multiple header values

* code reuse

* err on duplicated tag key

* check errors

* read inside filer

* add debugging for InvalidAccessKeyId

* fix read only volumes

* error format

* do not implement checkReadOnlyVolumes

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: Chris Lu <chrislusf@users.noreply.github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: chrislu <chris.lu@gmail.com>
Co-authored-by: Dima Tisnek <dimaqq@gmail.com>
Co-authored-by: Feng Shao <88640691+shaofeng66@users.noreply.github.com>
Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
Co-authored-by: Leonardo Lara <49646901+digitalinfobr@users.noreply.github.com>
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
2025-11-21 17:20:04 -08:00
Leonardo Lara
d8cac1a6cc Account Info (#7507)
* Account Info

Add account info on s3.configure

* address comments

* Update command_s3_configure.go

---------

Co-authored-by: chrislu <chris.lu@gmail.com>
2025-11-19 20:34:38 -08:00
Feng Shao
0299e78de7 de/compress the fs meta file if filename ends with gz/gzip (#7500)
* de/compress the fs meta file if filename ends with gz/gzip

* gemini code review

* update help msg
2025-11-18 07:45:12 -08:00
Lisandro Pin
0e69f7c916 Split logic for volume.check.disk into writable and read-only volume replicas. (#7476) 2025-11-13 17:14:36 -08:00
Lisandro Pin
79fa87bad4 Rework parameters passing for functions within ec.rebuild (#7445)
* Rework parameters passing for functions within `ec.rebuild`

This simplifies the overall codebase and allows to cleanly handle parallelization via waitgroups.

* fix copy source

* add tests

* remove tests not useful

* fmt

* nil check

---------

Co-authored-by: Chris Lu <chrislusf@users.noreply.github.com>
Co-authored-by: chrislu <chris.lu@gmail.com>
2025-11-10 22:43:43 -08:00
Lisandro Pin
9744382a18 Rework parameters passing for functions within volume.check.disk. (#7448)
* Rework parameters passing for functions within `volume.check.disk`.

We'll need to rework this logic to account for read-only volumes, and there're already way too many parameters shuffled around.

Grouping these into a single struct simplifies the overall codebase.

* similar fix

* Improved Error Handling in Tests

* propagate the errors

* edge cases

* edge case on modified time

* clean up

---------

Co-authored-by: chrislu <chris.lu@gmail.com>
2025-11-10 16:03:38 -08:00
Lisandro Pin
76e4a51964 Unify the parameter to disable dry-run on weed shell commands to -apply (instead of -force). (#7450)
* Unify the parameter to disable dry-run on weed shell commands to --apply (instead of --force).

* lint

* refactor

* Execution Order Corrected

* handle deprecated force flag

* fix help messages

* Refactoring]: Using flag.FlagSet.Visit()

* consistent with other commands

* Checks for both flags

* fix toml files

---------

Co-authored-by: chrislu <chris.lu@gmail.com>
2025-11-09 19:58:38 -08:00
Lisandro Pin
f466ff1412 Nit: use time.Durations instead of constants in seconds. (#7438)
Nit: use `time.Durations` instead of constants in seconds. Makes for slightly more readable code.
2025-11-04 13:02:22 -08:00
Lisandro Pin
1668c1042b Rework collection resultion for ec.rebuild, in preparation for parallelization. (#7420)
* Rework collection resultion for `ec.rebuild`, in preparation for parallelization.

See https://github.com/seaweedfs/seaweedfs/issues/7416 .

* simplify

* Update weed/shell/command_ec_rebuild.go

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>

---------

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>
2025-11-02 08:54:37 -08:00
Chris Lu
d745e6e41d Fix masterclient vidmap race condition (#7412)
* fallback to check master

* clean up

* parsing

* refactor

* handle parse error

* return error

* avoid dup lookup

* use batch key

* dedup lookup logic

* address comments

* errors.Join(lookupErrors...)

* add a comment

* Fix: Critical data race in MasterClient vidMap

Fixes a critical data race where resetVidMap() was writing to the vidMap
pointer while other methods were reading it concurrently without synchronization.

Changes:
- Removed embedded *vidMap from MasterClient
- Added vidMapLock (sync.RWMutex) to protect vidMap pointer access
- Created safe accessor methods (GetLocations, GetDataCenter, etc.)
- Updated all direct vidMap accesses to use thread-safe methods
- Updated resetVidMap() to acquire write lock during pointer swap

The vidMap already has internal locking for its operations, but this fix
protects the vidMap pointer itself from concurrent read/write races.

Verified with: go test -race ./weed/wdclient/...

Impact:
- Prevents potential panics from concurrent pointer access
- No performance impact - uses RWMutex for read-heavy workloads
- Maintains backward compatibility through wrapper methods

* fmt

* Fix: Critical data race in MasterClient vidMap

Fixes a critical data race where resetVidMap() was writing to the vidMap
pointer while other methods were reading it concurrently without synchronization.

Changes:
- Removed embedded *vidMap from MasterClient struct
- Added vidMapLock (sync.RWMutex) to protect vidMap pointer access
- Created minimal public accessor methods for external packages:
  * GetLocations, GetLocationsClone, GetVidLocations
  * LookupFileId, LookupVolumeServerUrl
  * GetDataCenter
- Internal code directly locks and accesses vidMap (no extra indirection)
- Updated resetVidMap() to acquire write lock during pointer swap
- Updated shell/commands.go to use GetDataCenter() method

Design Philosophy:
- vidMap already has internal locking for its map operations
- This fix specifically protects the vidMap *pointer* from concurrent access
- Public methods for external callers, direct locking for internal use
- Minimizes wrapper overhead while maintaining thread safety

Verified with: go test -race ./weed/wdclient/... (passes)

Impact:
- Prevents potential panics/crashes from data races
- Minimal performance impact (RWMutex for read-heavy workload)
- Maintains full backward compatibility

* fix more concurrent access

* reduce lock scope

* Optimize vidMap locking for better concurrency

Improved locking strategy based on the understanding that:
- vidMapLock protects the vidMap pointer from concurrent swaps
- vidMap has internal locks that protect its data structures

Changes:
1. Read operations: Grab pointer with RLock, release immediately, then operate
   - Reduces lock hold time
   - Allows resetVidMap to proceed sooner
   - Methods: GetLocations, GetLocationsClone, GetVidLocations,
     LookupVolumeServerUrl, GetDataCenter

2. Write operations: Changed from Lock() to RLock()
   - RLock prevents pointer swap during operation
   - Allows concurrent readers and other writers (serialized by vidMap's lock)
   - Methods: addLocation, deleteLocation, addEcLocation, deleteEcLocation

Benefits:
- Significantly reduced lock contention
- Better concurrent performance under load
- Still prevents all race conditions

Verified with: go test -race ./weed/wdclient/... (passes)

* Further reduce lock contention in LookupVolumeIdsWithFallback

Optimized two loops that were holding RLock for extended periods:

Before:
- Held RLock during entire loop iteration
- Included string parsing and cache lookups
- Could block resetVidMap for significant time with large batches

After:
- Grab vidMap pointer with brief RLock
- Release lock immediately
- Perform all loop operations on local pointer

Impact:
- First loop: Cache check on initial volumeIds
- Second loop: Double-check after singleflight wait

Benefits:
- Minimal lock hold time (just pointer copy)
- resetVidMap no longer blocked by long loops
- Better concurrent performance with large volume ID lists
- Still thread-safe (vidMap methods have internal locks)

Verified with: go test -race ./weed/wdclient/... (passes)

* Add clarifying comments to vidMap helper functions

Added inline documentation to each helper function (addLocation, deleteLocation,
addEcLocation, deleteEcLocation) explaining the two-level locking strategy:

- RLock on vidMapLock prevents resetVidMap from swapping the pointer
- vidMap has internal locks that protect the actual map mutations
- This design provides optimal concurrency

The comments make it clear why RLock (not Lock) is correct and intentional,
preventing future confusion about the locking strategy.

* Improve encapsulation: Add shallowClone() method to vidMap

Added a shallowClone() method to vidMap to improve encapsulation and prevent
MasterClient from directly accessing vidMap's internal fields.

Changes:
1. Added vidMap.shallowClone() in vid_map.go
   - Encapsulates the shallow copy logic within vidMap
   - Makes vidMap responsible for its own state representation
   - Documented that caller is responsible for thread safety

2. Simplified resetVidMap() in masterclient.go
   - Uses tail := mc.vidMap.shallowClone() instead of manual field access
   - Cleaner, more maintainable code
   - Better adherence to encapsulation principles

Benefits:
- Improved code organization and maintainability
- vidMap internals are now properly encapsulated
- Easier to modify vidMap structure in the future
- More self-documenting code

Verified with: go test -race ./weed/wdclient/... (passes)

* Optimize locking: Reduce lock acquisitions and use helper methods

Two optimizations to further reduce lock contention and improve code consistency:

1. LookupFileIdWithFallback: Eliminated redundant lock acquisition
   - Before: Two separate locks to get vidMap and dataCenter
   - After: Single lock gets both values together
   - Benefit: 50% reduction in lock/unlock overhead for this hot path

2. KeepConnected: Use GetDataCenter() helper for consistency
   - Before: Manual lock/unlock to access DataCenter field
   - After: Use existing GetDataCenter() helper method
   - Benefit: Better encapsulation and code consistency

Impact:
- Reduced lock contention in high-traffic lookup path
- More consistent use of accessor methods throughout codebase
- Cleaner, more maintainable code

Verified with: go test -race ./weed/wdclient/... (passes)

* Refactor: Extract common locking patterns into helper methods

Eliminated code duplication by introducing two helper methods that encapsulate
the common locking patterns used throughout MasterClient:

1. getStableVidMap() - For read operations
   - Acquires lock, gets pointer, releases immediately
   - Returns stable snapshot for thread-safe reads
   - Used by: GetLocations, GetLocationsClone, GetVidLocations,
     LookupFileId, LookupVolumeServerUrl, GetDataCenter

2. withCurrentVidMap(f func(vm *vidMap)) - For write operations
   - Holds RLock during callback execution
   - Prevents pointer swap while allowing concurrent operations
   - Used by: addLocation, deleteLocation, addEcLocation, deleteEcLocation

Benefits:
- Reduced code duplication (eliminated 48 lines of repetitive locking code)
- Centralized locking logic makes it easier to understand and maintain
- Self-documenting pattern through named helper methods
- Easier to modify locking strategy in the future (single point of change)
- Improved readability - accessor methods are now one-liners

Code size reduction: ~40% fewer lines for accessor/helper methods

Verified with: go test -race ./weed/wdclient/... (passes)

* consistent

* Fix cache pointer race condition with atomic.Pointer

Use atomic.Pointer for vidMap cache field to prevent data races
during cache trimming in resetVidMap. This addresses the race condition
where concurrent GetLocations calls could read the cache pointer while
resetVidMap is modifying it during cache chain trimming.

Changes:
- Changed cache field from *vidMap to atomic.Pointer[vidMap]
- Updated all cache access to use Load() and Store() atomic operations
- Updated shallowClone, GetLocations, deleteLocation, deleteEcLocation
- Updated resetVidMap to use atomic operations for cache trimming

* Merge: Resolve conflict in deleteEcLocation - keep atomic.Pointer and fix bug

Resolved merge conflict by combining:
1. Atomic pointer access pattern (from HEAD): cache.Load()
2. Correct method call (from fix): deleteEcLocation (not deleteLocation)

Resolution:
- Before (HEAD): cachedMap.deleteLocation() - WRONG, reintroduced bug
- Before (fix): vc.cache.deleteEcLocation() - RIGHT method, old pattern
- After (merged): cachedMap.deleteEcLocation() - RIGHT method, new pattern

This preserves both improvements:
✓ Thread-safe atomic.Pointer access pattern
✓ Correct recursive call to deleteEcLocation

Verified with: go test -race ./weed/wdclient/... (passes)

* Update vid_map.go

* remove shallow clone

* simplify
2025-10-30 23:36:06 -07:00