Files
seaweedFS/weed/s3api/bucket_size_metrics.go
Chris Lu 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
2025-12-15 19:23:25 -08:00

243 lines
7.6 KiB
Go

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++
}
}
}
}
}
}