Tree:
4a764dbb37
add-ec-vacuum
add_fasthttp_client
add_remote_storage
adding-message-queue-integration-tests
adjust-fsck-cutoff-default
also-delete-parent-directory-if-empty
avoid_releasing_temp_file_on_write
changing-to-zap
collect-public-metrics
copilot/fix-helm-chart-installation
copilot/fix-s3-object-tagging-issue
copilot/sub-pr-7677
create-table-snapshot-api-design
data_query_pushdown
dependabot/maven/other/java/client/com.google.protobuf-protobuf-java-3.25.5
dependabot/maven/other/java/examples/org.apache.hadoop-hadoop-common-3.4.0
detect-and-plan-ec-tasks
do-not-retry-if-error-is-NotFound
ec-disk-type-support
enhance-erasure-coding
fasthttp
filer1_maintenance_branch
fix-GetObjectLockConfigurationHandler
fix-mount-http-parallelism
fix-mount-read-throughput-7504
fix-s3-object-tagging-issue-7589
fix-versioning-listing-only
ftp
gh-pages
improve-fuse-mount
improve-fuse-mount2
logrus
master
message_send
mount2
mq-subscribe
mq2
nfs-cookie-prefix-list-fixes
optimize-delete-lookups
original_weed_mount
pr-7412
raft-dual-write
random_access_file
refactor-needle-read-operations
refactor-volume-write
remote_overlay
remove-implicit-directory-handling
revert-5134-patch-1
revert-5819-patch-1
revert-6434-bugfix-missing-s3-audit
s3-remote-cache-singleflight
s3-select
sub
tcp_read
test-reverting-lock-table
test_udp
testing
testing-sdx-generation
tikv
track-mount-e2e
upgrade-versions-to-4.00
volume_buffered_writes
worker-execute-ec-tasks
0.72
0.72.release
0.73
0.74
0.75
0.76
0.77
0.90
0.91
0.92
0.93
0.94
0.95
0.96
0.97
0.98
0.99
1.00
1.01
1.02
1.03
1.04
1.05
1.06
1.07
1.08
1.09
1.10
1.11
1.12
1.14
1.15
1.16
1.17
1.18
1.19
1.20
1.21
1.22
1.23
1.24
1.25
1.26
1.27
1.28
1.29
1.30
1.31
1.32
1.33
1.34
1.35
1.36
1.37
1.38
1.40
1.41
1.42
1.43
1.44
1.45
1.46
1.47
1.48
1.49
1.50
1.51
1.52
1.53
1.54
1.55
1.56
1.57
1.58
1.59
1.60
1.61
1.61RC
1.62
1.63
1.64
1.65
1.66
1.67
1.68
1.69
1.70
1.71
1.72
1.73
1.74
1.75
1.76
1.77
1.78
1.79
1.80
1.81
1.82
1.83
1.84
1.85
1.86
1.87
1.88
1.90
1.91
1.92
1.93
1.94
1.95
1.96
1.97
1.98
1.99
1;70
2.00
2.01
2.02
2.03
2.04
2.05
2.06
2.07
2.08
2.09
2.10
2.11
2.12
2.13
2.14
2.15
2.16
2.17
2.18
2.19
2.20
2.21
2.22
2.23
2.24
2.25
2.26
2.27
2.28
2.29
2.30
2.31
2.32
2.33
2.34
2.35
2.36
2.37
2.38
2.39
2.40
2.41
2.42
2.43
2.47
2.48
2.49
2.50
2.51
2.52
2.53
2.54
2.55
2.56
2.57
2.58
2.59
2.60
2.61
2.62
2.63
2.64
2.65
2.66
2.67
2.68
2.69
2.70
2.71
2.72
2.73
2.74
2.75
2.76
2.77
2.78
2.79
2.80
2.81
2.82
2.83
2.84
2.85
2.86
2.87
2.88
2.89
2.90
2.91
2.92
2.93
2.94
2.95
2.96
2.97
2.98
2.99
3.00
3.01
3.02
3.03
3.04
3.05
3.06
3.07
3.08
3.09
3.10
3.11
3.12
3.13
3.14
3.15
3.16
3.18
3.19
3.20
3.21
3.22
3.23
3.24
3.25
3.26
3.27
3.28
3.29
3.30
3.31
3.32
3.33
3.34
3.35
3.36
3.37
3.38
3.39
3.40
3.41
3.42
3.43
3.44
3.45
3.46
3.47
3.48
3.50
3.51
3.52
3.53
3.54
3.55
3.56
3.57
3.58
3.59
3.60
3.61
3.62
3.63
3.64
3.65
3.66
3.67
3.68
3.69
3.71
3.72
3.73
3.74
3.75
3.76
3.77
3.78
3.79
3.80
3.81
3.82
3.83
3.84
3.85
3.86
3.87
3.88
3.89
3.90
3.91
3.92
3.93
3.94
3.95
3.96
3.97
3.98
3.99
4.00
4.01
4.02
4.03
dev
helm-3.65.1
v0.69
v0.70beta
v3.33
${ noResults }
12331 Commits (4a764dbb3718fd42cd8a096aaa28294ef9ded41b)
| Author | SHA1 | Message | Date |
|---|---|---|---|
|
|
4a764dbb37 |
fmt
|
11 hours ago |
|
|
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. |
13 hours ago |
|
|
77a56c2857 |
adjust default concurrent reader and writer
related to https://github.com/seaweedfs/seaweedfs-csi-driver/pull/221 |
13 hours ago |
|
|
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 |
1 day ago |
|
|
134fd6a1ae
|
fix: S3 remote storage cold-cache read fails with 'size reported but no content available' (#7817)
fix: S3 remote storage cold-cache read fails with 'size reported but no content available' (#7815) When a remote-only entry's initial caching attempt times out or fails, streamFromVolumeServers() now detects this case and retries caching synchronously before streaming, similar to how the filer server handles remote-only entries. Changes: - Modified streamFromVolumeServers() to check entry.IsInRemoteOnly() before treating missing chunks as a data integrity error - Added doCacheRemoteObject() as the core caching function (calls filer gRPC) - Added buildRemoteObjectPath() helper to reduce code duplication - Refactored cacheRemoteObjectWithDedup() and cacheRemoteObjectForStreaming() to reuse the shared functions - Added integration tests for remote storage scenarios Fixes https://github.com/seaweedfs/seaweedfs/issues/7815 |
1 day ago |
|
|
6442da6f17
|
mount: efficient file lookup in large directories, skipping directory caching (#7818)
* mount: skip directory caching on file lookup and write When opening or creating a file in a directory that hasn't been cached yet, don't list the entire directory. Instead: - For reads: fetch only the single file's metadata directly from the filer - For writes: create on filer but skip local cache insertion This fixes a performance issue where opening a file in a directory with millions of files would hang because EnsureVisited() had to list all entries before the open could complete. The directory will still be cached when explicitly listed (ReadDir), but individual file operations now bypass the full directory caching. Key optimizations: - Extract shared lookupEntry() method to eliminate code duplication - Skip EnsureVisited on Lookup (file open) - Skip cache insertion on Mknod, Mkdir, Symlink, Link if dir not cached - Skip cache update on file sync/flush if dir not cached - If directory IS cached and entry not found, return ENOENT immediately Fixes #7145 * mount: add error handling for meta cache insert/update operations Handle errors from metaCache.InsertEntry and metaCache.UpdateEntry calls instead of silently ignoring them. This prevents silent cache inconsistencies and ensures errors are properly propagated. Files updated: - filehandle_read.go: handle InsertEntry error in downloadRemoteEntry - weedfs_file_sync.go: handle InsertEntry error in doFlush - weedfs_link.go: handle UpdateEntry and InsertEntry errors in Link - weedfs_symlink.go: handle InsertEntry error in Symlink * mount: use error wrapping (%w) for consistent error handling Use %w instead of %v in fmt.Errorf to preserve the original error, allowing it to be inspected up the call stack with errors.Is/As. |
1 day ago |
|
|
ed1da07665
|
Add consistent -debug and -debug.port flags to commands (#7816)
* Add consistent -debug and -debug.port flags to commands Add -debug and -debug.port flags to weed master, weed volume, weed s3, weed mq.broker, and weed filer.sync commands for consistency with weed filer. When -debug is enabled, an HTTP server starts on the specified port (default 6060) serving runtime profiling data at /debug/pprof/. For mq.broker, replaced the older -port.pprof flag with the new -debug and -debug.port pattern for consistency. * Update weed/util/grace/pprof.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> |
1 day ago |
|
|
bccef78082
|
fix: reduce N+1 queries in S3 versioned object list operations (#7814)
* fix: achieve single-scan efficiency for S3 versioned object listing When listing objects in a versioning-enabled bucket, the original code triggered multiple getEntry calls per versioned object (up to 12 with retries), causing excessive 'find' operations visible in Grafana and leading to high memory usage. This fix achieves single-scan efficiency by caching list metadata (size, ETag, mtime, owner) directly in the .versions directory: 1. Add new Extended keys for caching list metadata in .versions dir 2. Update upload/copy/multipart paths to cache metadata when creating versions 3. Update getLatestVersionEntryFromDirectoryEntry to use cached metadata (zero getEntry calls when cache is available) 4. Update updateLatestVersionAfterDeletion to maintain cache consistency Performance improvement for N versioned objects: - Before: N×1 to N×12 find operations per list request - After: 0 extra find operations (all metadata from single scan) This matches the efficiency of normal (non-versioned) object listing. * Update s3api_object_versioning.go * s3api: fix ETag handling for versioned objects and simplify delete marker creation - Add Md5 attribute to synthetic logicalEntry for single-part uploads to ensure filer.ETag() returns correct value in ListObjects response - Simplify delete marker creation by initializing entry directly in mkFile callback - Add bytes and encoding/hex imports for ETag parsing * s3api: preserve default attributes in delete marker mkFile callback Only modify Mtime field instead of replacing the entire Attributes struct, preserving default values like Crtime, FileMode, Uid, and Gid that mkFile initializes. * s3api: fix ETag handling in newListEntry for multipart uploads Prioritize ExtETagKey from Extended attributes before falling back to filer.ETag(). This properly handles multipart upload ETags (format: md5-parts) for versioned objects, where the synthetic entry has cached ETag metadata but no chunks to calculate from. * s3api: reduce code duplication in delete marker creation Extract deleteMarkerExtended map to be reused in both mkFile callback and deleteMarkerEntry construction. * test: add multipart upload versioning tests for ETag verification Add tests to verify that multipart uploaded objects in versioned buckets have correct ETags when listed: - TestMultipartUploadVersioningListETag: Basic multipart upload with 2 parts - TestMultipartUploadMultipleVersionsListETag: Multiple multipart versions - TestMixedSingleAndMultipartVersionsListETag: Mix of single-part and multipart These tests cover a bug where synthetic entries for versioned objects didn't include proper ETag handling for multipart uploads. * test: add delete marker test for multipart uploaded versioned objects TestMultipartUploadDeleteMarkerListBehavior verifies: - Delete marker creation hides object from ListObjectsV2 - ListObjectVersions shows both version and delete marker - Version ETag (multipart format) is preserved after delete marker - Object can be accessed by version ID after delete marker - Removing delete marker restores object visibility * refactor: address code review feedback - test: use assert.ElementsMatch for ETag verification (more idiomatic) - s3api: optimize newListEntry ETag logic (check ExtETagKey first) - s3api: fix edge case in ETag parsing (>= 2 instead of > 2) * s3api: prevent stale cached metadata and preserve existing extended attrs - setCachedListMetadata: clear old cached keys before setting new values to prevent stale data when new version lacks certain fields (e.g., owner) - createDeleteMarker: merge extended attributes instead of overwriting to preserve any existing metadata on the entry * s3api: extract clearCachedVersionMetadata to reduce code duplication - clearCachedVersionMetadata: clears only metadata fields (size, mtime, etag, owner, deleteMarker) - clearCachedListMetadata: now reuses clearCachedVersionMetadata + clears ID/filename - setCachedListMetadata: uses clearCachedVersionMetadata (not clearCachedListMetadata because caller has already set ID/filename) * s3api: share timestamp between version entry and cache entry Capture versionMtime once before mkFile and reuse for both: - versionEntry.Attributes.Mtime in the mkFile callback - versionEntryForCache.Attributes.Mtime for list caching This keeps list vs. HEAD LastModified timestamps aligned. * s3api: remove amzAccountId variable shadowing in multipart upload Extract amzAccountId before mkFile callback and reuse in both places, similar to how versionMtime is handled. Avoids confusion from redeclaring the same variable. |
1 day ago |
|
|
414cda4215
|
fix: S3 versioning memory leak in ListObjectVersions pagination (#7813)
* fix: S3 versioning memory leak in ListObjectVersions pagination This commit fixes a memory leak issue in S3 versioning buckets where ListObjectVersions with pagination (key-marker set) would collect ALL versions in the bucket before filtering, causing O(N) memory usage. Root cause: - When keyMarker was set, maxCollect was set to 0 (unlimited) - This caused findVersionsRecursively to traverse the entire bucket - All versions were collected into memory, sorted, then filtered Fix: - Updated findVersionsRecursively to accept keyMarker and versionIdMarker - Skips objects/versions before the marker during recursion (not after) - Always respects maxCollect limit (never unlimited) - Memory usage is now O(maxKeys) instead of O(total versions) Refactoring: - Introduced versionCollector struct to encapsulate collection state - Extracted helper methods for cleaner, more testable code: - matchesPrefixFilter: prefix matching logic - shouldSkipObjectForMarker: keyMarker filtering - shouldSkipVersionForMarker: versionIdMarker filtering - processVersionsDirectory: .versions directory handling - processExplicitDirectory: S3 directory object handling - processRegularFile: pre-versioning file handling - collectVersions: main recursive collection loop - processDirectory: directory entry dispatch This reduces the high QPS on 'find' and 'prefixList' operations by skipping irrelevant objects during traversal. Fixes customer-reported memory leak with high find/prefixList QPS in Grafana for S3 versioning buckets. * s3: infer version ID format from ExtLatestVersionIdKey metadata Simplified version format detection: - Removed ExtVersionIdFormatKey - no longer needed - getVersionIdFormat() now infers format from ExtLatestVersionIdKey - Uses isNewFormatVersionId() to check if latest version uses inverted format This approach is simpler because: - ExtLatestVersionIdKey is already stored in .versions directory metadata - No need for separate format metadata field - Format is naturally determined by the existing version IDs |
2 days ago |
|
|
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. |
2 days ago |
|
|
0e998e07d0
|
Upgrade raft to v1.1.6 to fix panic on log compaction (#7811)
Fixes #7810 The raft library would panic when prevLogIndex was beyond the end of the log after compaction. The fix in raft v1.1.6 returns nil instead, triggering the snapshot fallback mechanism. |
3 days ago |
|
|
22271358c6
|
Fix worker and admin ca (#7807)
* Fix Worker and Admin CA in helm chart * Fix Worker and Admin CA in helm chart - add security.toml modification * Fix Worker and Admin CA in helm chart - fix security.toml modification error * Fix Worker and Admin CA in helm chart - fix errors in volume mounts * Fix Worker and Admin CA in helm chart - address review comments - Remove worker-cert from admin pod (principle of least privilege) - Remove admin-cert from worker pod (principle of least privilege) - Remove overly broad namespace wildcards from admin-cert dnsNames - Remove overly broad namespace wildcards from worker-cert dnsNames --------- Co-authored-by: chrislu <chris.lu@gmail.com> |
3 days ago |
|
|
df0ea18084
|
fix: use consistent telemetryUrl default in master.follower (#7809)
Update telemetryUrl to use the same default value as the master command for consistency and maintainability. Addresses review feedback from PR #7808 |
3 days ago |
|
|
0b8fdab1e3
|
fix: initialize missing MasterOptions fields in master.follower (#7808)
Fix nil pointer dereference panic when starting master.follower. The init() function was missing initialization for: - maxParallelVacuumPerServer - telemetryUrl - telemetryEnabled These fields are dereferenced in toMasterOption() causing a panic. Fixes #7806 |
3 days ago |
|
|
ec3378f7a6
|
fix: improve mount quota enforcement to prevent overflow (#7804)
* fix: improve mount quota enforcement to prevent overflow (fixes seaweedfs-csi-driver#218) * test: add unit tests for quota enforcement |
3 days ago |
|
|
99a2e79efc
|
fix: authenticate before parsing form in IAM API (#7803)
fix: authenticate before parsing form in IAM API (#7802) The AuthIam middleware was calling ParseForm() before AuthSignatureOnly(), which consumed the request body before signature verification could hash it. For IAM requests (service != 's3'), the signature verification needs to hash the request body. When ParseForm() was called first, the body was already consumed, resulting in an empty body hash and SignatureDoesNotMatch error. The fix moves authentication before form parsing. The streamHashRequestBody function preserves the body after reading, so ParseForm() works correctly after authentication. Fixes #7802 |
3 days ago |
|
|
2763f105f4
|
fix: use unique bucket name in TestS3IAMPresignedURLIntegration to avoid flaky test (#7801)
The test was using a static bucket name 'test-iam-bucket' that could conflict with buckets created by other tests or previous runs. Each test framework creates new RSA keys for JWT signing, so the 'admin-user' identity differs between runs. When the bucket exists from a previous test, the new admin cannot access or delete it, causing AccessDenied errors. Changed to use GenerateUniqueBucketName() which ensures each test run gets its own bucket, avoiding cross-test conflicts. |
3 days ago |
|
|
a77b145590
|
fix: ListBuckets returns empty for users with bucket-specific permissions (#7799)
* fix: ListBuckets returns empty for users with bucket-specific permissions (#7796) The ListBucketsHandler was using sequential AND logic where ownership check happened before permission check. If a user had 'List:bucketname' permission but didn't own the bucket (different AmzIdentityId or missing owner metadata), the bucket was filtered out before the permission check could run. Changed to OR logic: a bucket is now visible if the user owns it OR has explicit permission to list it. This allows users with bucket-specific permissions like 'List:geoserver' to see buckets they have access to, even if they don't own them. Changes: - Modified ListBucketsHandler to check both ownership and permission, including bucket if either check passes - Renamed isBucketVisibleToIdentity to isBucketOwnedByIdentity for clarity - Added comprehensive tests in TestListBucketsIssue7796 Fixes #7796 * address review comments: optimize permission check and add integration test - Skip permission check if user is already the owner (performance optimization) - Add integration test that simulates the complete handler filtering logic to verify the combination of ownership OR permission check works correctly * add visibility assertions to each sub-test for self-contained verification Each sub-test now verifies the final outcome using isOwner || canList logic, making tests more robust and independently verifiable. |
3 days ago |
|
|
9e9c97ec61 |
fix bucket link
|
3 days ago |
|
|
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 |
3 days ago |
|
|
9c4a2e1b1a
|
fix: JWT validation failures during replication (#7788) (#7795)
fix: add debug logging for JWT validation failures (#7788) When JWT file ID validation fails during replication, add a log message showing both the expected and actual file IDs to help diagnose issues. Ref #7788 |
4 days ago |
|
|
02f7d3f3e2
|
Fix S3 server panic when -s3.port.https equals -s3.port (#7794)
* Fix volume repeatedly toggling between crowded and uncrowded Fixes #6712 The issue was that removeFromCrowded() was called in removeFromWritable(), which is invoked whenever a volume temporarily becomes unwritable (due to replica count fluctuations, heartbeat issues, or read-only state changes). This caused unnecessary toggling: 1. Volume becomes temporarily unwritable → removeFromWritable() → removeFromCrowded() logs 'becomes uncrowded' 2. Volume becomes writable again 3. CollectDeadNodeAndFullVolumes() runs → setVolumeCrowded() logs 'becomes crowded' The fix: - Remove removeFromCrowded() call from removeFromWritable() - Only clear crowded status when volume is fully unregistered from the layout (when location.Length() == 0 in UnRegisterVolume) This ensures transient state changes don't cause log spam and the crowded status accurately reflects the volume's size relative to the grow threshold. * Refactor test to use subtests for better readability Address review feedback: use t.Run subtests to make the test's intent clearer by giving each verification step a descriptive name. * Fix S3 server panic when -s3.port.https equals -s3.port When starting the S3 server with -s3.port.https=8333 (same as default -s3.port), the server would panic with nil pointer dereference because: 1. The HTTP listener was already bound to port 8333 2. NewIpAndLocalListeners for HTTPS failed but error was discarded 3. ServeTLS was called on nil listener causing panic This fix: - Adds early validation to prevent using same port for HTTP and HTTPS - Properly handles the error from NewIpAndLocalListeners for HTTPS Fixes #7792 |
4 days ago |
|
|
8518f06777
|
Fix volume repeatedly toggling between crowded and uncrowded (#7793)
* Fix volume repeatedly toggling between crowded and uncrowded Fixes #6712 The issue was that removeFromCrowded() was called in removeFromWritable(), which is invoked whenever a volume temporarily becomes unwritable (due to replica count fluctuations, heartbeat issues, or read-only state changes). This caused unnecessary toggling: 1. Volume becomes temporarily unwritable → removeFromWritable() → removeFromCrowded() logs 'becomes uncrowded' 2. Volume becomes writable again 3. CollectDeadNodeAndFullVolumes() runs → setVolumeCrowded() logs 'becomes crowded' The fix: - Remove removeFromCrowded() call from removeFromWritable() - Only clear crowded status when volume is fully unregistered from the layout (when location.Length() == 0 in UnRegisterVolume) This ensures transient state changes don't cause log spam and the crowded status accurately reflects the volume's size relative to the grow threshold. * Refactor test to use subtests for better readability Address review feedback: use t.Run subtests to make the test's intent clearer by giving each verification step a descriptive name. |
4 days ago |
|
|
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> |
4 days ago |
|
|
697b56003d
|
s3: reduce ObjectVersion memory by not retaining full Entry (#7786)
s3: fix fallback owner lookup to use specific version Address review feedback: the fallback logic was incorrectly using getLatestObjectVersion which returns the wrong owner when different versions have different owners. Fix by using getSpecificObjectVersion with the version.VersionId to fetch the correct entry for the specific version being processed. This also simplifies the code by removing the separate null version handling since getSpecificObjectVersion already handles that case. |
4 days ago |
|
|
956c5a1626 |
s3: fix pagination by collecting all versions when keyMarker is set
When paginating with keyMarker, we must collect all versions first because filtering happens after sorting. Previously, we limited collection to maxKeys+1 which caused us to miss versions beyond the marker when there were many versions before it. |
4 days ago |
|
|
daa3af826f |
ci: fix stress tests by adding server start/stop
|
4 days ago |
|
|
aff144f8b5 |
ci: run versioning stress tests on all PRs, not just master pushes
|
4 days ago |
|
|
9150d84eea |
test: use -master.peers=none for faster test server startup
|
4 days ago |
|
|
5dd34e3260 |
s3: fix ListObjectVersions pagination by implementing key-marker filtering
The ListObjectVersions API was receiving key-marker and version-id-marker parameters but not using them to filter results. This caused infinite pagination loops when clients tried to paginate through results. Fix by adding filtering logic after sorting: - Skip versions with key < keyMarker (already returned in previous pages) - For key == keyMarker, skip versions with versionId >= versionIdMarker - Include versions with key > keyMarker or (key == keyMarker and versionId < versionIdMarker) This respects the S3 sort order (key ascending, versionId descending for same key) and correctly returns only versions that come AFTER the marker position. |
4 days ago |
|
|
26121c55c9 |
test: improve pagination stress test with QUICK_TEST option and better assertions
|
4 days ago |
|
|
f517bc39fc |
test: fix nil pointer dereference and add debugging to pagination stress tests
|
4 days ago |
|
|
8236df1368 |
ci: enable pagination stress tests in GitHub CI
Add pagination stress tests (>1000 versions) to the S3 versioning stress test job in GitHub CI. These tests run on master branch pushes to validate that ListObjectVersions correctly handles objects with more than 1000 versions using pagination. |
4 days ago |
|
|
0972a0acf3 |
test: add pagination stress tests for S3 versioning with >1000 versions
|
4 days ago |
|
|
3f62240976
|
s3: add pagination to getObjectVersionList and reduce memory (#7787)
* s3: add pagination to getObjectVersionList and reduce memory
This commit makes two improvements to S3 version listing:
1. Add pagination to getObjectVersionList:
- Previously hardcoded limit of 1000 versions per object
- Now paginates through all versions using startFrom marker
- Fixes correctness issue where objects with >1000 versions would
have some versions missing from list results
2. Reduce memory by not retaining full Entry:
- Replace Entry field with OwnerID string in ObjectVersion struct
- Extract owner ID when creating ObjectVersion
- Avoids retaining Chunks array which can be large for big files
- Clear seenVersionIds map after use to help GC
3. Update getObjectOwnerFromVersion:
- Use new OwnerID field instead of accessing Entry.Extended
- Maintains backward compatibility with fallback lookups
* s3: propagate errors from list operation instead of returning partial results
Address review feedback: when s3a.list fails during version listing,
the function was logging at V(2) level and returning partial results
with nil error. This hides the error and could lead to silent data
inconsistencies.
Fix by:
1. Log at Warningf level for better visibility
2. Return nil versions slice with the error to prevent partial results
from being processed as complete
|
4 days ago |
|
|
d26c260041
|
s3: fix memory leak in ListObjectVersions with early termination (#7785)
* s3: fix memory leak in ListObjectVersions with early termination This fixes a critical memory leak in S3 versioned bucket listing operations: 1. Add maxCollect parameter to findVersionsRecursively() to stop collecting versions once we have enough for the response + truncation detection 2. Add early termination checks throughout the recursive traversal to prevent scanning entire buckets when only a small number of results are requested 3. Use clear() on tracking maps after collection to help GC reclaim memory 4. Create new slice with exact capacity when truncating results instead of re-slicing, which allows GC to reclaim the excess backing array memory 5. Pre-allocate result slice with reasonable initial capacity to reduce reallocations during collection Before this fix, listing versions on a bucket with many objects and versions would load ALL versions into memory before pagination, causing OOM crashes. Fixes memory exhaustion when calling ListObjectVersions on large versioned buckets. * s3: fix pre-allocation capacity to be consistent with maxCollect Address review feedback: the previous capping logic caused an inconsistency where initialCap was capped to 1000 but maxCollect was maxKeys+1, leading to unnecessary reallocations when maxKeys was 1000. Fix by: 1. Cap maxKeys to 1000 (S3 API limit) at the start of the function 2. Use maxKeys+1 directly for slice capacity, ensuring consistency with the maxCollect parameter passed to findVersionsRecursively |
4 days ago |
|
|
ef28f49ec3
|
fix: correctly detect missing source file during volume copy (#7784)
* fix: correctly detect missing source file during volume copy
The previous fix (commit
|
4 days ago |
|
|
7920ffa98c
|
Fix uncleanable size=0 orphans with volume.fsck -forcePurging (#7783)
This is a follow-up fix to PR #7332 which partially addressed the issue. The problem is that size=0 needles are in a gray area: - IsValid() returns false for size=0 (because size must be > 0) - IsDeleted() returns false for size=0 (because size must be < 0 or == TombstoneFileSize) PR #7332 only fixed 2 places, but several other places still had the same bug: 1. needle_map_memory.go:doLoading - line 43 still used oldSize.IsValid() 2. needle_map_memory.go:DoOffsetLoading - used during vacuum and incremental loading 3. needle_map_leveldb.go:generateLevelDbFile - used when generating LevelDB needle maps 4. needle_map_leveldb.go:DoOffsetLoading - used during incremental loading for LevelDB 5. needle_map/compact_map.go:delete - couldn't delete size=0 entries because: - The condition 'size > 0' failed for size=0 - Even if it passed, negating 0 gives 0 (not marking as deleted) Changes: - Changed size.IsValid() to !size.IsDeleted() in doLoading and DoOffsetLoading functions - Fixed compact_map delete to use TombstoneFileSize for size=0 entries Fixes #7293 |
4 days ago |
|
|
93499cd944
|
Fix admin GUI list ordering on refresh (#7782)
Sort lists of filers, volume servers, masters, and message brokers by address to ensure consistent ordering on page refresh. This fixes the non-deterministic ordering caused by iterating over Go maps with range. Fixes #7781 |
4 days ago |
|
|
44cd07f835 |
Update cluster_ec_volumes_templ.go
|
4 days ago |
|
|
95ef041bfb
|
Fix EC Volumes page header styling to match admin theme (#7780)
* Fix EC Volumes page header styling to match admin theme Fixes #7779 The EC Volumes page was rendering with bright Bootstrap default colors instead of the admin dark theme because it was structured as a standalone HTML document with its own DOCTYPE, head, and body tags. This fix converts the template to be a content fragment (like other properly styled templates such as cluster_ec_shards.templ) so it correctly inherits the admin.css styling when rendered within the layout. * Address review comments: fix URL interpolation and falsy value check - Fix collection filter link to use templ.URL() for proper interpolation - Change updateUrl() falsy check from 'if (params[key])' to 'if (params[key] != null)' to handle 0 and false values correctly * Address additional review comments - Use erasure_coding.TotalShardsCount constant instead of hardcoded '14' for shard count displays (lines 88 and 214) - Improve error handling in repairVolume() to check response.ok before parsing JSON, preventing confusing errors on non-JSON responses - Remove unused totalSize variable in formatShardRangesWithSizes() - Simplify redundant pagination conditions * Remove unused code: displayShardLocationsHTML, groupShardsByServerWithSizes, formatShardRangesWithSizes These functions and templates were defined but never called anywhere in the codebase. Removing them reduces code maintenance burden. * Address review feedback: improve code quality - Add defensive JSON response validation in repairVolume function - Replace O(n²) bubble sorts with Go's standard sort.Ints and sort.Slice - Document volume status thresholds explaining EC recovery logic: * Critical: unrecoverable (more than DataShardsCount missing) * Degraded: high risk (more than half DataShardsCount missing) * Incomplete: reduced redundancy (more than half ParityShardsCount missing) * Minor: fully recoverable with good margin * Fix redundant shard count display in Healthy Volumes card Changed from 'Complete (14/14 shards)' to 'All 14 shards present' since the numerator and denominator were always the same value. * Use templ.URL for default collection link for consistency * Fix Clear Filter link to stay on EC Volumes page Changed href from /cluster/ec-shards to /cluster/ec-volumes so clearing the filter stays on the current page instead of navigating away. |
4 days ago |
|
|
f5c666052e
|
feat: add S3 bucket size and object count metrics (#7776)
* feat: add S3 bucket size and object count metrics Adds periodic collection of bucket size metrics: - SeaweedFS_s3_bucket_size_bytes: logical size (deduplicated across replicas) - SeaweedFS_s3_bucket_physical_size_bytes: physical size (including replicas) - SeaweedFS_s3_bucket_object_count: object count (deduplicated) Collection runs every 1 minute via background goroutine that queries filer Statistics RPC for each bucket's collection. Also adds Grafana dashboard panels for: - S3 Bucket Size (logical vs physical) - S3 Bucket Object Count * address PR comments: fix bucket size metrics collection 1. Fix collectCollectionInfoFromMaster to use master VolumeList API - Now properly queries master for topology info - Uses WithMasterClient to get volume list from master - Correctly calculates logical vs physical size based on replication 2. Return error when filerClient is nil to trigger fallback - Changed from 'return nil, nil' to 'return nil, error' - Ensures fallback to filer stats is properly triggered 3. Implement pagination in listBucketNames - Added listBucketPageSize constant (1000) - Uses StartFromFileName for pagination - Continues fetching until fewer entries than limit returned 4. Handle NewReplicaPlacementFromByte error and prevent division by zero - Check error return from NewReplicaPlacementFromByte - Default to 1 copy if error occurs - Add explicit check for copyCount == 0 * simplify bucket size metrics: remove filer fallback, align with quota enforcement - Remove fallback to filer Statistics RPC - Use only master topology for collection info (same as s3.bucket.quota.enforce) - Updated comments to clarify this runs the same collection logic as quota enforcement - Simplified code by removing collectBucketSizeFromFilerStats * use s3a.option.Masters directly instead of querying filer * address PR comments: fix dashboard overlaps and improve metrics collection Grafana dashboard fixes: - Fix overlapping panels 55 and 59 in grafana_seaweedfs.json (moved 59 to y=30) - Fix grid collision in k8s dashboard (moved panel 72 to y=48) - Aggregate bucket metrics with max() by (bucket) for multi-instance S3 gateways Go code improvements: - Add graceful shutdown support via context cancellation - Use ticker instead of time.Sleep for better shutdown responsiveness - Distinguish EOF from actual errors in stream handling * improve bucket size metrics: multi-master failover and proper error handling - Initial delay now respects context cancellation using select with time.After - Use WithOneOfGrpcMasterClients for multi-master failover instead of hardcoding Masters[0] - Properly propagate stream errors instead of just logging them (EOF vs real errors) * improve bucket size metrics: distributed lock and volume ID deduplication - Add distributed lock (LiveLock) so only one S3 instance collects metrics at a time - Add IsLocked() method to LiveLock for checking lock status - Fix deduplication: use volume ID tracking instead of dividing by copyCount - Previous approach gave wrong results if replicas were missing - Now tracks seen volume IDs and counts each volume only once - Physical size still includes all replicas for accurate disk usage reporting * rename lock to s3.leader * simplify: remove StartBucketSizeMetricsCollection wrapper function * fix data race: use atomic operations for LiveLock.isLocked field - Change isLocked from bool to int32 - Use atomic.LoadInt32/StoreInt32 for all reads/writes - Sync shared isLocked field in StartLongLivedLock goroutine * add nil check for topology info to prevent panic * fix bucket metrics: use Ticker for consistent intervals, fix pagination logic - Use time.Ticker instead of time.After for consistent interval execution - Fix pagination: count all entries (not just directories) for proper termination - Update lastFileName for all entries to prevent pagination issues * address PR comments: remove redundant atomic store, propagate context - Remove redundant atomic.StoreInt32 in StartLongLivedLock (AttemptToLock already sets it) - Propagate context through metrics collection for proper cancellation on shutdown - collectAndUpdateBucketSizeMetrics now accepts ctx - collectCollectionInfoFromMaster uses ctx for VolumeList RPC - listBucketNames uses ctx for ListEntries RPC |
4 days ago |
|
|
4dcd33bbc8
|
fix: handle missing idx file for empty volumes during copy (#7777) (#7778)
When copying/evacuating empty volumes, the .idx file may not exist on disk (this is allowed by checkIdxFile for volumes with only super block in .dat). This fix: 1. Uses os.IsNotExist() instead of err == os.ErrNotExist for proper wrapped error checking in CopyFile 2. Treats missing source file as success when StopOffset == 0 (empty file) 3. Allows checkCopyFiles to pass when idx file doesn't exist but IdxFileSize == 0 (empty volume) Fixes volumeServer.evacuate and volume.fix.replication for empty volumes. |
4 days ago |
|
|
93d0779318
|
fix: add S3 bucket traffic sent metric tracking (#7774)
* fix: add S3 bucket traffic sent metric tracking The BucketTrafficSent() function was defined but never called, causing the S3 Bucket Traffic Sent Grafana dashboard panel to not display data. Added BucketTrafficSent() calls in the streaming functions: - streamFromVolumeServers: for inline and chunked content - streamFromVolumeServersWithSSE: for encrypted range and full object requests The traffic received metric already worked because BucketTrafficReceived() was properly called in putToFiler() for both regular and multipart uploads. * feat: add S3 API Calls per Bucket panel to Grafana dashboards Added a new panel showing API calls per bucket using the existing SeaweedFS_s3_request_total metric aggregated by bucket. Updated all Grafana dashboard files: - other/metrics/grafana_seaweedfs.json - other/metrics/grafana_seaweedfs_k8s.json - other/metrics/grafana_seaweedfs_heartbeat.json - k8s/charts/seaweedfs/dashboards/seaweedfs-grafana-dashboard.json * address PR comments: use actual bytes written for traffic metrics - Use actual bytes written from w.Write instead of expected size for inline content - Add countingWriter wrapper to track actual bytes for chunked content streaming - Update streamDecryptedRangeFromChunks to return actual bytes written for SSE - Remove redundant nil check that caused linter warning - Fix duplicate panel id 86 in grafana_seaweedfs.json (changed to 90) - Fix overlapping panel positions in grafana_seaweedfs_k8s.json (rebalanced x positions) * fix grafana k8s dashboard: rebalance S3 panels to avoid overlap - Panel 86 (S3 API Calls per Bucket): w:6, x:0, y:15 - Panel 67 (S3 Request Duration 95th): w:6, x:6, y:15 - Panel 68 (S3 Request Duration 80th): w:6, x:12, y:15 - Panel 65 (S3 Request Duration 99th): w:6, x:18, y:15 All four S3 panels now fit in a single row (y:15) with width 6 each. Filer row header at y:22 and subsequent panels remain correctly positioned. * add input validation and clarify comments in adjustRangeForPart - Add validation that partStartOffset <= partEndOffset at function start - Add clarifying comments for suffix-range handling where clientEnd temporarily holds the suffix length before being reassigned * align pluginVersion for panel 86 to 10.3.1 in k8s dashboard * track partial writes for accurate egress traffic accounting - Change condition from 'err == nil' to 'written > 0' for inline content - Move BucketTrafficSent before error check for chunked content streaming - Track traffic even on partial SSE range writes - Track traffic even on partial full SSE object copies This ensures egress traffic is counted even when writes fail partway through, providing more accurate bandwidth metrics. |
4 days ago |
|
|
d0cc51e7c6
|
chore(deps): bump io.netty:netty-codec-http from 4.1.125.Final to 4.1.129.Final in /test/java/spark (#7773)
chore(deps): bump io.netty:netty-codec-http in /test/java/spark Bumps [io.netty:netty-codec-http](https://github.com/netty/netty) from 4.1.125.Final to 4.1.129.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.125.Final...netty-4.1.129.Final) --- updated-dependencies: - dependency-name: io.netty:netty-codec-http dependency-version: 4.1.129.Final dependency-type: direct:production ... Signed-off-by: dependabot[bot] <support@github.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> |
4 days ago |
|
|
c6e07429e7
|
chore(deps): bump golang.org/x/image from 0.33.0 to 0.34.0 (#7764)
* chore(deps): bump golang.org/x/image from 0.33.0 to 0.34.0 Bumps [golang.org/x/image](https://github.com/golang/image) from 0.33.0 to 0.34.0. - [Commits](https://github.com/golang/image/compare/v0.33.0...v0.34.0) --- updated-dependencies: - dependency-name: golang.org/x/image dependency-version: 0.34.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] <support@github.com> * 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> |
5 days ago |
|
|
b53e50485f
|
s3: warm bucket config cache on startup for multi-filer consistency (#7772)
* s3: warm bucket config cache on startup for multi-filer consistency In multi-filer clusters, the bucket configuration cache (storing Object Lock, versioning, and other settings) was not being pre-populated on S3 API server startup. This caused issues where: 1. After server restart, Object Lock and versioning settings appeared lost until the bucket was accessed (lazy loading) 2. In multi-filer clusters, race conditions during bucket creation could result in inconsistent Object Lock configuration This fix warms the bucketConfigCache during BucketRegistry initialization, ensuring all bucket configurations (including Object Lock and versioning) are immediately available after restart without waiting for first access. The fix piggybacks on the existing BucketRegistry.init() which already iterates through all buckets, adding a call to update the config cache with each bucket's extended attributes. * s3: add visibility logging for bucket config cache warming - Add bucket count tracking during initialization - Log error if bucket listing fails - Log INFO message with count of warmed buckets on successful init This improves observability for the cache warming process and addresses review feedback about error handling visibility. * s3: fix bucket deletion not invalidating config cache Bug fix: The metadata subscription handler had an early return when NewEntry was nil, which skipped the onBucketMetadataChange call for bucket deletions. This caused deleted buckets to remain in the config cache. The fix moves onBucketMetadataChange before the nil check so it's called for all events (create, update, delete). The IAM and circuit breaker updates still require NewEntry content, so they remain after the check. * s3: handle config file deletions for IAM and circuit breaker Refactored the metadata subscription handlers to properly handle all event types (create, update, delete) for IAM and circuit breaker configs: - Renamed onIamConfigUpdate -> onIamConfigChange - Renamed onCircuitBreakerConfigUpdate -> onCircuitBreakerConfigChange - Both handlers now check for deletions (newEntry == nil && oldEntry != nil) - On config file deletion, reset to empty config by loading empty bytes - Simplified processEventFn to call all handlers unconditionally - Each handler checks for nil entries internally This ensures that deleting identity.json or circuit_breaker.json will clear the in-memory config rather than leaving stale data. * s3: restore NewParentPath handling for rename/move operations The directory resolution logic was accidentally removed. This restores the check for NewParentPath which is needed when files are renamed or moved - in such cases, NewParentPath contains the destination directory which should be used for directory matching in the handlers. |
5 days ago |
|
|
5a03b5538f
|
filer: improve FoundationDB performance by disabling batch by default (#7770)
* filer: improve FoundationDB performance by disabling batch by default This PR addresses a performance issue where FoundationDB filer was achieving only ~757 ops/sec with 12 concurrent S3 clients, despite FDB being capable of 17,000+ ops/sec. Root cause: The write batcher was waiting up to 5ms for each operation to batch, even though S3 semantics require waiting for durability confirmation. This added artificial latency that defeated the purpose of batching. Changes: - Disable write batching by default (batch_enabled = false) - Each write now commits immediately in its own transaction - Reduce batch interval from 5ms to 1ms when batching is enabled - Add batch_enabled config option to toggle behavior - Improve batcher to collect available ops without blocking - Add benchmarks comparing batch vs no-batch performance Benchmark results (16 concurrent goroutines): - With batch: 2,924 ops/sec (342,032 ns/op) - Without batch: 4,625 ops/sec (216,219 ns/op) - Improvement: +58% faster Configuration: - Default: batch_enabled = false (optimal for S3 PUT latency) - For bulk ingestion: set batch_enabled = true Also fixes ARM64 Docker test setup (shell compatibility, fdbserver path). * fix: address review comments - use atomic counter and remove duplicate batcher - Use sync/atomic.Uint64 for unique filenames in concurrent benchmarks - Remove duplicate batcher creation in createBenchmarkStoreWithBatching (initialize() already creates batcher when batchEnabled=true) * fix: add realistic default values to benchmark store helper Set directoryPrefix, timeout, and maxRetryDelay to reasonable defaults for more realistic benchmark conditions. |
5 days ago |
|
|
44beb42eb9
|
s3: fix PutObject ETag format for multi-chunk uploads (#7771)
* s3: fix PutObject ETag format for multi-chunk uploads Fix issue #7768: AWS S3 SDK for Java fails with 'Invalid base 16 character: -' when performing PutObject on files that are internally auto-chunked. The issue was that SeaweedFS returned a composite ETag format (<md5hash>-<count>) for regular PutObject when the file was split into multiple chunks due to auto-chunking. However, per AWS S3 spec, the composite ETag format should only be used for multipart uploads (CreateMultipartUpload/UploadPart/CompleteMultipartUpload API). Regular PutObject should always return a pure MD5 hash as the ETag, regardless of how the file is stored internally. The fix ensures the MD5 hash is always stored in entry.Attributes.Md5 for regular PutObject operations, so filer.ETag() returns the pure MD5 hash instead of falling back to ETagChunks() composite format. * test: add comprehensive ETag format tests for issue #7768 Add integration tests to ensure PutObject ETag format compatibility: Go tests (test/s3/etag/): - TestPutObjectETagFormat_SmallFile: 1KB single chunk - TestPutObjectETagFormat_LargeFile: 10MB auto-chunked (critical for #7768) - TestPutObjectETagFormat_ExtraLargeFile: 25MB multi-chunk - TestMultipartUploadETagFormat: verify composite ETag for multipart - TestPutObjectETagConsistency: ETag consistency across PUT/HEAD/GET - TestETagHexValidation: simulate AWS SDK v2 hex decoding - TestMultipleLargeFileUploads: stress test multiple large uploads Java tests (other/java/s3copier/): - Update pom.xml to include AWS SDK v2 (2.20.127) - Add ETagValidationTest.java with comprehensive SDK v2 tests - Add README.md documenting SDK versions and test coverage Documentation: - Add test/s3/SDK_COMPATIBILITY.md documenting validated SDK versions - Add test/s3/etag/README.md explaining test coverage These tests ensure large file PutObject (>8MB) returns pure MD5 ETags (not composite format), which is required for AWS SDK v2 compatibility. * fix: lower Java version requirement to 11 for CI compatibility * address CodeRabbit review comments - s3_etag_test.go: Handle rand.Read error, fix multipart part-count logging - Makefile: Add 'all' target, pass S3_ENDPOINT to test commands - SDK_COMPATIBILITY.md: Add language tag to fenced code block - ETagValidationTest.java: Add pagination to cleanup logic - README.md: Clarify Go SDK tests are in separate location * ci: add s3copier ETag validation tests to Java integration tests - Enable S3 API (-s3 -s3.port=8333) in SeaweedFS test server - Add S3 API readiness check to wait loop - Add step to run ETagValidationTest from s3copier This ensures the fix for issue #7768 is continuously tested against AWS SDK v2 for Java in CI. * ci: add S3 config with credentials for s3copier tests - Add -s3.config pointing to docker/compose/s3.json - Add -s3.allowDeleteBucketNotEmpty for test cleanup - Set S3_ACCESS_KEY and S3_SECRET_KEY env vars for tests * ci: pass S3 config as Maven system properties Pass S3_ENDPOINT, S3_ACCESS_KEY, S3_SECRET_KEY via -D flags so they're available via System.getProperty() in Java tests |
5 days ago |
|
|
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. |
5 days ago |