Browse Source
feat: add S3 bucket size and object count metrics (#7776)
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 RPCpull/7782/head
committed by
GitHub
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
9 changed files with 1100 additions and 9 deletions
-
203k8s/charts/seaweedfs/dashboards/seaweedfs-grafana-dashboard.json
-
194other/metrics/grafana_seaweedfs.json
-
196other/metrics/grafana_seaweedfs_heartbeat.json
-
203other/metrics/grafana_seaweedfs_k8s.json
-
2test/s3/iam/test_config.json
-
18weed/cluster/lock_client.go
-
242weed/s3api/bucket_size_metrics.go
-
4weed/s3api/s3api_server.go
-
41weed/stats/metrics.go
@ -0,0 +1,242 @@ |
|||
package s3api |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"io" |
|||
"strings" |
|||
"time" |
|||
|
|||
"github.com/seaweedfs/seaweedfs/weed/cluster" |
|||
"github.com/seaweedfs/seaweedfs/weed/glog" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb" |
|||
"github.com/seaweedfs/seaweedfs/weed/stats" |
|||
) |
|||
|
|||
const ( |
|||
bucketSizeMetricsInterval = 1 * time.Minute |
|||
listBucketPageSize = 1000 // Page size for paginated bucket listing
|
|||
s3MetricsLockName = "s3.leader" |
|||
) |
|||
|
|||
// CollectionInfo holds collection statistics
|
|||
// Used for both metrics collection and quota enforcement
|
|||
type CollectionInfo struct { |
|||
FileCount float64 |
|||
DeleteCount float64 |
|||
DeletedByteCount float64 |
|||
Size float64 // Logical size (deduplicated by volume ID)
|
|||
PhysicalSize float64 // Physical size (including all replicas)
|
|||
VolumeCount int // Logical volume count (deduplicated by volume ID)
|
|||
} |
|||
|
|||
// volumeKey uniquely identifies a volume for deduplication
|
|||
type volumeKey struct { |
|||
collection string |
|||
volumeId uint32 |
|||
} |
|||
|
|||
// startBucketSizeMetricsLoop periodically collects bucket size metrics and updates Prometheus gauges.
|
|||
// Uses a distributed lock to ensure only one S3 instance collects metrics at a time.
|
|||
// Should be called as a goroutine; stops when the provided context is cancelled.
|
|||
func (s3a *S3ApiServer) startBucketSizeMetricsLoop(ctx context.Context) { |
|||
// Initial delay to let the system stabilize
|
|||
select { |
|||
case <-time.After(10 * time.Second): |
|||
case <-ctx.Done(): |
|||
return |
|||
} |
|||
|
|||
// Create lock client for distributed lock
|
|||
if len(s3a.option.Filers) == 0 { |
|||
glog.V(1).Infof("No filers configured, skipping bucket size metrics collection") |
|||
return |
|||
} |
|||
filer := s3a.option.Filers[0] |
|||
lockClient := cluster.NewLockClient(s3a.option.GrpcDialOption, filer) |
|||
owner := string(filer) + "-s3-metrics" |
|||
|
|||
// Start long-lived lock - this S3 instance will only collect metrics when it holds the lock
|
|||
lock := lockClient.StartLongLivedLock(s3MetricsLockName, owner, func(newLockOwner string) { |
|||
glog.V(1).Infof("S3 bucket size metrics lock owner changed to: %s", newLockOwner) |
|||
}) |
|||
defer lock.Stop() |
|||
|
|||
ticker := time.NewTicker(bucketSizeMetricsInterval) |
|||
defer ticker.Stop() |
|||
|
|||
for { |
|||
select { |
|||
case <-ctx.Done(): |
|||
glog.V(1).Infof("Stopping bucket size metrics collection") |
|||
return |
|||
case <-ticker.C: |
|||
// Only collect metrics if we hold the lock
|
|||
if lock.IsLocked() { |
|||
s3a.collectAndUpdateBucketSizeMetrics(ctx) |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
// collectAndUpdateBucketSizeMetrics collects bucket sizes from master topology
|
|||
// and updates Prometheus metrics. Uses the same approach as quota enforcement.
|
|||
func (s3a *S3ApiServer) collectAndUpdateBucketSizeMetrics(ctx context.Context) { |
|||
// Collect collection info from master topology (same as quota enforcement)
|
|||
collectionInfos, err := s3a.collectCollectionInfoFromMaster(ctx) |
|||
if err != nil { |
|||
glog.V(2).Infof("Failed to collect collection info from master: %v", err) |
|||
return |
|||
} |
|||
|
|||
// Get list of buckets
|
|||
buckets, err := s3a.listBucketNames(ctx) |
|||
if err != nil { |
|||
glog.V(2).Infof("Failed to list buckets for size metrics: %v", err) |
|||
return |
|||
} |
|||
|
|||
// Map collections to buckets and update metrics
|
|||
for _, bucket := range buckets { |
|||
collection := s3a.getCollectionName(bucket) |
|||
if info, found := collectionInfos[collection]; found { |
|||
stats.UpdateBucketSizeMetrics(bucket, info.Size, info.PhysicalSize, info.FileCount) |
|||
glog.V(3).Infof("Updated bucket size metrics: bucket=%s, logicalSize=%.0f, physicalSize=%.0f, objects=%.0f", |
|||
bucket, info.Size, info.PhysicalSize, info.FileCount) |
|||
} else { |
|||
// Bucket exists but no collection data (empty bucket)
|
|||
stats.UpdateBucketSizeMetrics(bucket, 0, 0, 0) |
|||
} |
|||
} |
|||
} |
|||
|
|||
// collectCollectionInfoFromMaster queries the master for topology info and extracts collection sizes.
|
|||
// This is the same approach used by shell command s3.bucket.quota.enforce.
|
|||
func (s3a *S3ApiServer) collectCollectionInfoFromMaster(ctx context.Context) (map[string]*CollectionInfo, error) { |
|||
if len(s3a.option.Masters) == 0 { |
|||
return nil, fmt.Errorf("no masters configured") |
|||
} |
|||
|
|||
// Convert masters slice to map for WithOneOfGrpcMasterClients
|
|||
masterMap := make(map[string]pb.ServerAddress) |
|||
for _, master := range s3a.option.Masters { |
|||
masterMap[string(master)] = master |
|||
} |
|||
|
|||
// Connect to any available master and get volume list with topology
|
|||
collectionInfos := make(map[string]*CollectionInfo) |
|||
|
|||
err := pb.WithOneOfGrpcMasterClients(false, masterMap, s3a.option.GrpcDialOption, func(client master_pb.SeaweedClient) error { |
|||
resp, err := client.VolumeList(ctx, &master_pb.VolumeListRequest{}) |
|||
if err != nil { |
|||
return fmt.Errorf("failed to get volume list: %w", err) |
|||
} |
|||
if resp == nil || resp.TopologyInfo == nil { |
|||
return fmt.Errorf("empty topology info from master") |
|||
} |
|||
collectCollectionInfoFromTopology(resp.TopologyInfo, collectionInfos) |
|||
return nil |
|||
}) |
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
|
|||
return collectionInfos, nil |
|||
} |
|||
|
|||
// listBucketNames returns a list of all bucket names using pagination
|
|||
func (s3a *S3ApiServer) listBucketNames(ctx context.Context) ([]string, error) { |
|||
var buckets []string |
|||
|
|||
err := s3a.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error { |
|||
lastFileName := "" |
|||
for { |
|||
request := &filer_pb.ListEntriesRequest{ |
|||
Directory: s3a.option.BucketsPath, |
|||
StartFromFileName: lastFileName, |
|||
Limit: listBucketPageSize, |
|||
InclusiveStartFrom: lastFileName == "", |
|||
} |
|||
|
|||
stream, err := client.ListEntries(ctx, request) |
|||
if err != nil { |
|||
return err |
|||
} |
|||
|
|||
entriesReceived := 0 |
|||
for { |
|||
resp, err := stream.Recv() |
|||
if err != nil { |
|||
if err == io.EOF { |
|||
break |
|||
} |
|||
return fmt.Errorf("error receiving bucket list entries: %w", err) |
|||
} |
|||
entriesReceived++ |
|||
if resp.Entry != nil { |
|||
lastFileName = resp.Entry.Name |
|||
if resp.Entry.IsDirectory { |
|||
// Skip .uploads and other hidden directories
|
|||
if !strings.HasPrefix(resp.Entry.Name, ".") { |
|||
buckets = append(buckets, resp.Entry.Name) |
|||
} |
|||
} |
|||
} |
|||
} |
|||
|
|||
// If we got fewer entries than the limit, we're done
|
|||
if entriesReceived < listBucketPageSize { |
|||
break |
|||
} |
|||
} |
|||
return nil |
|||
}) |
|||
|
|||
return buckets, err |
|||
} |
|||
|
|||
// collectCollectionInfoFromTopology extracts collection info from topology.
|
|||
// Deduplicates by volume ID to correctly handle missing replicas.
|
|||
// Unlike dividing by copyCount (which would give wrong results if replicas are missing),
|
|||
// we track seen volume IDs and only count each volume once for logical size/count.
|
|||
func collectCollectionInfoFromTopology(t *master_pb.TopologyInfo, collectionInfos map[string]*CollectionInfo) { |
|||
// Track which volumes we've already seen to deduplicate by volume ID
|
|||
seenVolumes := make(map[volumeKey]bool) |
|||
|
|||
for _, dc := range t.DataCenterInfos { |
|||
for _, r := range dc.RackInfos { |
|||
for _, dn := range r.DataNodeInfos { |
|||
for _, diskInfo := range dn.DiskInfos { |
|||
for _, vi := range diskInfo.VolumeInfos { |
|||
c := vi.Collection |
|||
cif, found := collectionInfos[c] |
|||
if !found { |
|||
cif = &CollectionInfo{} |
|||
collectionInfos[c] = cif |
|||
} |
|||
|
|||
// Always add to physical size (all replicas)
|
|||
cif.PhysicalSize += float64(vi.Size) |
|||
|
|||
// Check if we've already counted this volume for logical stats
|
|||
key := volumeKey{collection: c, volumeId: vi.Id} |
|||
if seenVolumes[key] { |
|||
// Already counted this volume, skip logical stats
|
|||
continue |
|||
} |
|||
seenVolumes[key] = true |
|||
|
|||
// First time seeing this volume - add to logical stats
|
|||
cif.Size += float64(vi.Size) |
|||
cif.FileCount += float64(vi.FileCount) |
|||
cif.DeleteCount += float64(vi.DeleteCount) |
|||
cif.DeletedByteCount += float64(vi.DeletedByteCount) |
|||
cif.VolumeCount++ |
|||
} |
|||
} |
|||
} |
|||
} |
|||
} |
|||
} |
|||
Write
Preview
Loading…
Cancel
Save
Reference in new issue