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
This commit is contained in:
@@ -3361,6 +3361,209 @@
|
||||
],
|
||||
"title": "Filer Go Routines",
|
||||
"type": "timeseries"
|
||||
},
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"color": {
|
||||
"mode": "palette-classic"
|
||||
},
|
||||
"custom": {
|
||||
"axisBorderShow": false,
|
||||
"axisCenteredZero": false,
|
||||
"axisColorMode": "text",
|
||||
"axisLabel": "",
|
||||
"axisPlacement": "auto",
|
||||
"barAlignment": 0,
|
||||
"drawStyle": "line",
|
||||
"fillOpacity": 10,
|
||||
"gradientMode": "none",
|
||||
"hideFrom": {
|
||||
"legend": false,
|
||||
"tooltip": false,
|
||||
"viz": false
|
||||
},
|
||||
"insertNulls": false,
|
||||
"lineInterpolation": "linear",
|
||||
"lineWidth": 1,
|
||||
"pointSize": 5,
|
||||
"scaleDistribution": {
|
||||
"type": "linear"
|
||||
},
|
||||
"showPoints": "never",
|
||||
"spanNulls": false,
|
||||
"stacking": {
|
||||
"group": "A",
|
||||
"mode": "none"
|
||||
},
|
||||
"thresholdsStyle": {
|
||||
"mode": "off"
|
||||
}
|
||||
},
|
||||
"mappings": [],
|
||||
"thresholds": {
|
||||
"mode": "absolute",
|
||||
"steps": [
|
||||
{
|
||||
"color": "green",
|
||||
"value": null
|
||||
},
|
||||
{
|
||||
"color": "red",
|
||||
"value": 80
|
||||
}
|
||||
]
|
||||
},
|
||||
"unit": "decbytes"
|
||||
},
|
||||
"overrides": []
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 0,
|
||||
"y": 48
|
||||
},
|
||||
"id": 89,
|
||||
"options": {
|
||||
"legend": {
|
||||
"calcs": [],
|
||||
"displayMode": "list",
|
||||
"placement": "bottom",
|
||||
"showLegend": true
|
||||
},
|
||||
"tooltip": {
|
||||
"mode": "multi",
|
||||
"sort": "none"
|
||||
}
|
||||
},
|
||||
"pluginVersion": "10.3.1",
|
||||
"targets": [
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"editorMode": "code",
|
||||
"expr": "max(SeaweedFS_s3_bucket_size_bytes{namespace=\"$NAMESPACE\"}) by (bucket)",
|
||||
"legendFormat": "{{bucket}} (logical)",
|
||||
"range": true,
|
||||
"refId": "A"
|
||||
},
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"editorMode": "code",
|
||||
"expr": "max(SeaweedFS_s3_bucket_physical_size_bytes{namespace=\"$NAMESPACE\"}) by (bucket)",
|
||||
"legendFormat": "{{bucket}} (physical)",
|
||||
"range": true,
|
||||
"refId": "B"
|
||||
}
|
||||
],
|
||||
"title": "S3 Bucket Size",
|
||||
"type": "timeseries"
|
||||
},
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"color": {
|
||||
"mode": "palette-classic"
|
||||
},
|
||||
"custom": {
|
||||
"axisBorderShow": false,
|
||||
"axisCenteredZero": false,
|
||||
"axisColorMode": "text",
|
||||
"axisLabel": "",
|
||||
"axisPlacement": "auto",
|
||||
"barAlignment": 0,
|
||||
"drawStyle": "line",
|
||||
"fillOpacity": 10,
|
||||
"gradientMode": "none",
|
||||
"hideFrom": {
|
||||
"legend": false,
|
||||
"tooltip": false,
|
||||
"viz": false
|
||||
},
|
||||
"insertNulls": false,
|
||||
"lineInterpolation": "linear",
|
||||
"lineWidth": 1,
|
||||
"pointSize": 5,
|
||||
"scaleDistribution": {
|
||||
"type": "linear"
|
||||
},
|
||||
"showPoints": "never",
|
||||
"spanNulls": false,
|
||||
"stacking": {
|
||||
"group": "A",
|
||||
"mode": "none"
|
||||
},
|
||||
"thresholdsStyle": {
|
||||
"mode": "off"
|
||||
}
|
||||
},
|
||||
"mappings": [],
|
||||
"thresholds": {
|
||||
"mode": "absolute",
|
||||
"steps": [
|
||||
{
|
||||
"color": "green",
|
||||
"value": null
|
||||
},
|
||||
{
|
||||
"color": "red",
|
||||
"value": 80
|
||||
}
|
||||
]
|
||||
},
|
||||
"unit": "short"
|
||||
},
|
||||
"overrides": []
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"y": 48
|
||||
},
|
||||
"id": 90,
|
||||
"options": {
|
||||
"legend": {
|
||||
"calcs": [],
|
||||
"displayMode": "list",
|
||||
"placement": "bottom",
|
||||
"showLegend": true
|
||||
},
|
||||
"tooltip": {
|
||||
"mode": "multi",
|
||||
"sort": "none"
|
||||
}
|
||||
},
|
||||
"pluginVersion": "10.3.1",
|
||||
"targets": [
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"editorMode": "code",
|
||||
"expr": "max(SeaweedFS_s3_bucket_object_count{namespace=\"$NAMESPACE\"}) by (bucket)",
|
||||
"legendFormat": "{{bucket}}",
|
||||
"range": true,
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"title": "S3 Bucket Object Count",
|
||||
"type": "timeseries"
|
||||
}
|
||||
],
|
||||
"refresh": "",
|
||||
|
||||
@@ -3053,6 +3053,200 @@
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_PROMETHEUS}",
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"unit": "decbytes"
|
||||
},
|
||||
"overrides": []
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 0,
|
||||
"y": 37
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 87,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"links": [],
|
||||
"nullPointMode": "null as zero",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "8.1.2",
|
||||
"pointradius": 5,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"expr": "max(SeaweedFS_s3_bucket_size_bytes) by (bucket)",
|
||||
"format": "time_series",
|
||||
"hide": false,
|
||||
"intervalFactor": 2,
|
||||
"legendFormat": "{{bucket}} (logical)",
|
||||
"refId": "A"
|
||||
},
|
||||
{
|
||||
"expr": "max(SeaweedFS_s3_bucket_physical_size_bytes) by (bucket)",
|
||||
"format": "time_series",
|
||||
"hide": false,
|
||||
"intervalFactor": 2,
|
||||
"legendFormat": "{{bucket}} (physical)",
|
||||
"refId": "B"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "S3 Bucket Size",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "decbytes",
|
||||
"logBase": 1,
|
||||
"min": 0,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"logBase": 1,
|
||||
"show": false
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_PROMETHEUS}",
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"unit": "short"
|
||||
},
|
||||
"overrides": []
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"y": 37
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 88,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"links": [],
|
||||
"nullPointMode": "null as zero",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "8.1.2",
|
||||
"pointradius": 5,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"expr": "max(SeaweedFS_s3_bucket_object_count) by (bucket)",
|
||||
"format": "time_series",
|
||||
"hide": false,
|
||||
"intervalFactor": 2,
|
||||
"legendFormat": "{{bucket}}",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "S3 Bucket Object Count",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "short",
|
||||
"logBase": 1,
|
||||
"min": 0,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"logBase": 1,
|
||||
"show": false
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
}
|
||||
],
|
||||
"refresh": "30s",
|
||||
|
||||
@@ -1166,6 +1166,202 @@
|
||||
"show": false
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_PROMETHEUS}",
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"unit": "decbytes"
|
||||
},
|
||||
"overrides": []
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 0,
|
||||
"y": 37
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 87,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"links": [],
|
||||
"nullPointMode": "null as zero",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "8.1.2",
|
||||
"pointradius": 5,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"span": 6,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"expr": "max(SeaweedFS_s3_bucket_size_bytes) by (bucket)",
|
||||
"format": "time_series",
|
||||
"hide": false,
|
||||
"intervalFactor": 2,
|
||||
"legendFormat": "{{bucket}} (logical)",
|
||||
"refId": "A"
|
||||
},
|
||||
{
|
||||
"expr": "max(SeaweedFS_s3_bucket_physical_size_bytes) by (bucket)",
|
||||
"format": "time_series",
|
||||
"hide": false,
|
||||
"intervalFactor": 2,
|
||||
"legendFormat": "{{bucket}} (physical)",
|
||||
"refId": "B"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "S3 Bucket Size",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "decbytes",
|
||||
"logBase": 1,
|
||||
"min": 0,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"logBase": 1,
|
||||
"show": false
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_PROMETHEUS}",
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"unit": "short"
|
||||
},
|
||||
"overrides": []
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"y": 37
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 88,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"links": [],
|
||||
"nullPointMode": "null as zero",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "8.1.2",
|
||||
"pointradius": 5,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"span": 6,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"expr": "max(SeaweedFS_s3_bucket_object_count) by (bucket)",
|
||||
"format": "time_series",
|
||||
"hide": false,
|
||||
"intervalFactor": 2,
|
||||
"legendFormat": "{{bucket}}",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "S3 Bucket Object Count",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "short",
|
||||
"logBase": 1,
|
||||
"min": 0,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"logBase": 1,
|
||||
"show": false
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
}
|
||||
],
|
||||
"repeat": null,
|
||||
|
||||
@@ -2447,6 +2447,209 @@
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
},
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"color": {
|
||||
"mode": "palette-classic"
|
||||
},
|
||||
"custom": {
|
||||
"axisBorderShow": false,
|
||||
"axisCenteredZero": false,
|
||||
"axisColorMode": "text",
|
||||
"axisLabel": "",
|
||||
"axisPlacement": "auto",
|
||||
"barAlignment": 0,
|
||||
"drawStyle": "line",
|
||||
"fillOpacity": 10,
|
||||
"gradientMode": "none",
|
||||
"hideFrom": {
|
||||
"legend": false,
|
||||
"tooltip": false,
|
||||
"viz": false
|
||||
},
|
||||
"insertNulls": false,
|
||||
"lineInterpolation": "linear",
|
||||
"lineWidth": 1,
|
||||
"pointSize": 5,
|
||||
"scaleDistribution": {
|
||||
"type": "linear"
|
||||
},
|
||||
"showPoints": "never",
|
||||
"spanNulls": false,
|
||||
"stacking": {
|
||||
"group": "A",
|
||||
"mode": "none"
|
||||
},
|
||||
"thresholdsStyle": {
|
||||
"mode": "off"
|
||||
}
|
||||
},
|
||||
"mappings": [],
|
||||
"thresholds": {
|
||||
"mode": "absolute",
|
||||
"steps": [
|
||||
{
|
||||
"color": "green",
|
||||
"value": null
|
||||
},
|
||||
{
|
||||
"color": "red",
|
||||
"value": 80
|
||||
}
|
||||
]
|
||||
},
|
||||
"unit": "decbytes"
|
||||
},
|
||||
"overrides": []
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 0,
|
||||
"y": 41
|
||||
},
|
||||
"id": 89,
|
||||
"options": {
|
||||
"legend": {
|
||||
"calcs": [],
|
||||
"displayMode": "list",
|
||||
"placement": "bottom",
|
||||
"showLegend": true
|
||||
},
|
||||
"tooltip": {
|
||||
"mode": "multi",
|
||||
"sort": "none"
|
||||
}
|
||||
},
|
||||
"pluginVersion": "10.3.1",
|
||||
"targets": [
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"editorMode": "code",
|
||||
"expr": "max(SeaweedFS_s3_bucket_size_bytes{namespace=\"$NAMESPACE\"}) by (bucket)",
|
||||
"legendFormat": "{{bucket}} (logical)",
|
||||
"range": true,
|
||||
"refId": "A"
|
||||
},
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"editorMode": "code",
|
||||
"expr": "max(SeaweedFS_s3_bucket_physical_size_bytes{namespace=\"$NAMESPACE\"}) by (bucket)",
|
||||
"legendFormat": "{{bucket}} (physical)",
|
||||
"range": true,
|
||||
"refId": "B"
|
||||
}
|
||||
],
|
||||
"title": "S3 Bucket Size",
|
||||
"type": "timeseries"
|
||||
},
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"color": {
|
||||
"mode": "palette-classic"
|
||||
},
|
||||
"custom": {
|
||||
"axisBorderShow": false,
|
||||
"axisCenteredZero": false,
|
||||
"axisColorMode": "text",
|
||||
"axisLabel": "",
|
||||
"axisPlacement": "auto",
|
||||
"barAlignment": 0,
|
||||
"drawStyle": "line",
|
||||
"fillOpacity": 10,
|
||||
"gradientMode": "none",
|
||||
"hideFrom": {
|
||||
"legend": false,
|
||||
"tooltip": false,
|
||||
"viz": false
|
||||
},
|
||||
"insertNulls": false,
|
||||
"lineInterpolation": "linear",
|
||||
"lineWidth": 1,
|
||||
"pointSize": 5,
|
||||
"scaleDistribution": {
|
||||
"type": "linear"
|
||||
},
|
||||
"showPoints": "never",
|
||||
"spanNulls": false,
|
||||
"stacking": {
|
||||
"group": "A",
|
||||
"mode": "none"
|
||||
},
|
||||
"thresholdsStyle": {
|
||||
"mode": "off"
|
||||
}
|
||||
},
|
||||
"mappings": [],
|
||||
"thresholds": {
|
||||
"mode": "absolute",
|
||||
"steps": [
|
||||
{
|
||||
"color": "green",
|
||||
"value": null
|
||||
},
|
||||
{
|
||||
"color": "red",
|
||||
"value": 80
|
||||
}
|
||||
]
|
||||
},
|
||||
"unit": "short"
|
||||
},
|
||||
"overrides": []
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"y": 41
|
||||
},
|
||||
"id": 90,
|
||||
"options": {
|
||||
"legend": {
|
||||
"calcs": [],
|
||||
"displayMode": "list",
|
||||
"placement": "bottom",
|
||||
"showLegend": true
|
||||
},
|
||||
"tooltip": {
|
||||
"mode": "multi",
|
||||
"sort": "none"
|
||||
}
|
||||
},
|
||||
"pluginVersion": "10.3.1",
|
||||
"targets": [
|
||||
{
|
||||
"datasource": {
|
||||
"type": "prometheus",
|
||||
"uid": "${DS_PROMETHEUS}"
|
||||
},
|
||||
"editorMode": "code",
|
||||
"expr": "max(SeaweedFS_s3_bucket_object_count{namespace=\"$NAMESPACE\"}) by (bucket)",
|
||||
"legendFormat": "{{bucket}}",
|
||||
"range": true,
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"title": "S3 Bucket Object Count",
|
||||
"type": "timeseries"
|
||||
}
|
||||
],
|
||||
"refresh": "30s",
|
||||
|
||||
@@ -36,7 +36,7 @@
|
||||
"sts": {
|
||||
"tokenDuration": "15m",
|
||||
"issuer": "seaweedfs-sts",
|
||||
"signingKey": "test-sts-signing-key-for-integration-tests"
|
||||
"signingKey": "dGVzdC1zaWduaW5nLWtleS0zMi1jaGFyYWN0ZXJzLWxvbmc="
|
||||
},
|
||||
"policy": {
|
||||
"defaultEffect": "Deny"
|
||||
|
||||
@@ -3,6 +3,7 @@ package cluster
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/cluster/lock_manager"
|
||||
@@ -36,7 +37,7 @@ type LiveLock struct {
|
||||
hostFiler pb.ServerAddress
|
||||
cancelCh chan struct{}
|
||||
grpcDialOption grpc.DialOption
|
||||
isLocked bool
|
||||
isLocked int32 // 0 = unlocked, 1 = locked; use atomic operations
|
||||
self string
|
||||
lc *LockClient
|
||||
owner string
|
||||
@@ -84,10 +85,12 @@ func (lc *LockClient) StartLongLivedLock(key string, owner string, onLockOwnerCh
|
||||
if err := lock.AttemptToLock(lock_manager.LiveLockTTL); err != nil {
|
||||
glog.V(0).Infof("Lost lock %s: %v", key, err)
|
||||
isLocked = false
|
||||
atomic.StoreInt32(&lock.isLocked, 0)
|
||||
}
|
||||
} else {
|
||||
if err := lock.AttemptToLock(lock_manager.LiveLockTTL); err == nil {
|
||||
isLocked = true
|
||||
// Note: AttemptToLock already sets lock.isLocked atomically on success
|
||||
}
|
||||
}
|
||||
if lockOwner != lock.LockOwner() && lock.LockOwner() != "" {
|
||||
@@ -130,20 +133,20 @@ func (lock *LiveLock) AttemptToLock(lockDuration time.Duration) error {
|
||||
time.Sleep(time.Second)
|
||||
return fmt.Errorf("%v", errorMessage)
|
||||
}
|
||||
if !lock.isLocked {
|
||||
if atomic.LoadInt32(&lock.isLocked) == 0 {
|
||||
// Only log when transitioning from unlocked to locked
|
||||
glog.V(1).Infof("LOCK: Successfully acquired key=%s owner=%s", lock.key, lock.self)
|
||||
}
|
||||
lock.isLocked = true
|
||||
atomic.StoreInt32(&lock.isLocked, 1)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (lock *LiveLock) StopShortLivedLock() error {
|
||||
if !lock.isLocked {
|
||||
if atomic.LoadInt32(&lock.isLocked) == 0 {
|
||||
return nil
|
||||
}
|
||||
defer func() {
|
||||
lock.isLocked = false
|
||||
atomic.StoreInt32(&lock.isLocked, 0)
|
||||
}()
|
||||
return pb.WithFilerClient(false, 0, lock.hostFiler, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
_, err := client.DistributedUnlock(context.Background(), &filer_pb.UnlockRequest{
|
||||
@@ -228,3 +231,8 @@ func (lock *LiveLock) doLock(lockDuration time.Duration) (errorMessage string, e
|
||||
func (lock *LiveLock) LockOwner() string {
|
||||
return lock.owner
|
||||
}
|
||||
|
||||
// IsLocked returns true if this instance currently holds the lock
|
||||
func (lock *LiveLock) IsLocked() bool {
|
||||
return atomic.LoadInt32(&lock.isLocked) == 1
|
||||
}
|
||||
|
||||
242
weed/s3api/bucket_size_metrics.go
Normal file
242
weed/s3api/bucket_size_metrics.go
Normal file
@@ -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++
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -226,6 +226,10 @@ func NewS3ApiServerWithStore(router *mux.Router, option *S3ApiServerOption, expl
|
||||
}
|
||||
|
||||
go s3ApiServer.subscribeMetaEvents("s3", startTsNs, filer.DirectoryEtcRoot, []string{option.BucketsPath})
|
||||
|
||||
// Start bucket size metrics collection in background
|
||||
go s3ApiServer.startBucketSizeMetricsLoop(context.Background())
|
||||
|
||||
return s3ApiServer, nil
|
||||
}
|
||||
|
||||
|
||||
@@ -434,6 +434,30 @@ var (
|
||||
Name: "uploaded_objects",
|
||||
Help: "Number of objects uploaded in each bucket.",
|
||||
}, []string{"bucket"})
|
||||
|
||||
S3BucketSizeBytesGauge = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: Namespace,
|
||||
Subsystem: "s3",
|
||||
Name: "bucket_size_bytes",
|
||||
Help: "Current size of each S3 bucket in bytes (logical size, deduplicated across replicas).",
|
||||
}, []string{"bucket"})
|
||||
|
||||
S3BucketPhysicalSizeBytesGauge = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: Namespace,
|
||||
Subsystem: "s3",
|
||||
Name: "bucket_physical_size_bytes",
|
||||
Help: "Current physical size of each S3 bucket in bytes (including all replicas).",
|
||||
}, []string{"bucket"})
|
||||
|
||||
S3BucketObjectCountGauge = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: Namespace,
|
||||
Subsystem: "s3",
|
||||
Name: "bucket_object_count",
|
||||
Help: "Current number of objects in each S3 bucket (logical count, deduplicated across replicas).",
|
||||
}, []string{"bucket"})
|
||||
)
|
||||
|
||||
func init() {
|
||||
@@ -491,6 +515,9 @@ func init() {
|
||||
Gather.MustRegister(S3BucketTrafficSentBytesCounter)
|
||||
Gather.MustRegister(S3DeletedObjectsCounter)
|
||||
Gather.MustRegister(S3UploadedObjectsCounter)
|
||||
Gather.MustRegister(S3BucketSizeBytesGauge)
|
||||
Gather.MustRegister(S3BucketPhysicalSizeBytesGauge)
|
||||
Gather.MustRegister(S3BucketObjectCountGauge)
|
||||
|
||||
go bucketMetricTTLControl()
|
||||
}
|
||||
@@ -576,6 +603,9 @@ func bucketMetricTTLControl() {
|
||||
c += S3BucketTrafficSentBytesCounter.DeletePartialMatch(labels)
|
||||
c += S3DeletedObjectsCounter.DeletePartialMatch(labels)
|
||||
c += S3UploadedObjectsCounter.DeletePartialMatch(labels)
|
||||
c += S3BucketSizeBytesGauge.DeletePartialMatch(labels)
|
||||
c += S3BucketPhysicalSizeBytesGauge.DeletePartialMatch(labels)
|
||||
c += S3BucketObjectCountGauge.DeletePartialMatch(labels)
|
||||
glog.V(0).Infof("delete inactive bucket metrics, %s: %d", bucket, c)
|
||||
}
|
||||
}
|
||||
@@ -585,3 +615,14 @@ func bucketMetricTTLControl() {
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// UpdateBucketSizeMetrics updates the bucket size gauges
|
||||
// logicalSize is the deduplicated size (accounting for replication)
|
||||
// physicalSize is the raw size including all replicas
|
||||
// objectCount is the number of objects in the bucket (deduplicated)
|
||||
func UpdateBucketSizeMetrics(bucket string, logicalSize, physicalSize float64, objectCount float64) {
|
||||
S3BucketSizeBytesGauge.WithLabelValues(bucket).Set(logicalSize)
|
||||
S3BucketPhysicalSizeBytesGauge.WithLabelValues(bucket).Set(physicalSize)
|
||||
S3BucketObjectCountGauge.WithLabelValues(bucket).Set(objectCount)
|
||||
RecordBucketActiveTime(bucket)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user