Add additional info for replication metrics API (#17293)

to track the replication transfer rate across different nodes,
number of active workers in use and in-queue stats to get
an idea of the current workload.

This PR also adds replication metrics to the site replication
status API. For site replication, prometheus metrics are
no longer at the bucket level - but at the cluster level.

Add prometheus metric to track credential errors since uptime
This commit is contained in:
Poorna 2023-08-30 01:00:59 -07:00 committed by GitHub
parent cce90cb2b7
commit b48bbe08b2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
31 changed files with 8779 additions and 743 deletions

View File

@ -431,6 +431,7 @@ func getSRStatusOptions(r *http.Request) (opts madmin.SRStatusOptions) {
opts.Entity = madmin.GetSREntityType(q.Get("entity"))
opts.EntityValue = q.Get("entityvalue")
opts.ShowDeleted = q.Get("showDeleted") == "true"
opts.Metrics = q.Get("metrics") == "true"
return
}

View File

@ -461,9 +461,12 @@ func registerAPIRouter(router *mux.Router) {
// MinIO extension API for replication.
//
// GetBucketReplicationMetrics
router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("getbucketreplicationmetrics", maxClients(gz(httpTraceAll(api.GetBucketReplicationMetricsV2Handler))))).Queries("replication-metrics", "2")
// deprecated handler
router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("getbucketreplicationmetrics", maxClients(gz(httpTraceAll(api.GetBucketReplicationMetricsHandler))))).Queries("replication-metrics", "")
// ValidateBucketReplicationCreds
router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("checkbucketreplicationconfiguration", maxClients(gz(httpTraceAll(api.ValidateBucketReplicationCredsHandler))))).Queries("replication-check", "")

View File

@ -193,7 +193,7 @@ func (api objectAPIHandlers) DeleteBucketReplicationConfigHandler(w http.Respons
writeSuccessResponseHeadersOnly(w)
}
// GetBucketReplicationMetricsHandler - GET Bucket replication metrics.
// GetBucketReplicationMetricsHandler - GET Bucket replication metrics. // Deprecated Aug 2023
// ----------
// Gets the replication metrics for a bucket.
func (api objectAPIHandlers) GetBucketReplicationMetricsHandler(w http.ResponseWriter, r *http.Request) {
@ -227,29 +227,81 @@ func (api objectAPIHandlers) GetBucketReplicationMetricsHandler(w http.ResponseW
return
}
var usageInfo BucketUsageInfo
dataUsageInfo, err := loadDataUsageFromBackend(ctx, objectAPI)
if err == nil && !dataUsageInfo.LastUpdate.IsZero() {
usageInfo = dataUsageInfo.BucketsUsage[bucket]
w.Header().Set(xhttp.ContentType, string(mimeJSON))
enc := json.NewEncoder(w)
stats := globalReplicationStats.getLatestReplicationStats(bucket)
bwRpt := globalNotificationSys.GetBandwidthReports(ctx, bucket)
bwMap := bwRpt.BucketStats[bucket]
for arn, st := range stats.ReplicationStats.Stats {
if bwMap != nil {
if bw, ok := bwMap[arn]; ok {
st.BandWidthLimitInBytesPerSecond = bw.LimitInBytesPerSecond
st.CurrentBandwidthInBytesPerSecond = bw.CurrentBandwidthInBytesPerSecond
stats.ReplicationStats.Stats[arn] = st
}
}
}
if err := enc.Encode(stats.ReplicationStats); err != nil {
writeErrorResponseJSON(ctx, w, toAPIError(ctx, err), r.URL)
return
}
}
// GetBucketReplicationMetricsV2Handler - GET Bucket replication metrics.
// ----------
// Gets the replication metrics for a bucket.
func (api objectAPIHandlers) GetBucketReplicationMetricsV2Handler(w http.ResponseWriter, r *http.Request) {
ctx := newContext(r, w, "GetBucketReplicationMetricsV2")
defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r))
vars := mux.Vars(r)
bucket := vars["bucket"]
objectAPI := api.ObjectAPI()
if objectAPI == nil {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(ErrServerNotInitialized), r.URL)
return
}
// check if user has permissions to perform this operation
if s3Error := checkRequestAuthType(ctx, r, policy.GetReplicationConfigurationAction, bucket, ""); s3Error != ErrNone {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL)
return
}
// Check if bucket exists.
if _, err := objectAPI.GetBucketInfo(ctx, bucket, BucketOptions{}); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return
}
if _, _, err := globalBucketMetadataSys.GetReplicationConfig(ctx, bucket); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return
}
w.Header().Set(xhttp.ContentType, string(mimeJSON))
enc := json.NewEncoder(w)
stats := globalReplicationStats.getLatestReplicationStats(bucket, usageInfo)
stats := globalReplicationStats.getLatestReplicationStats(bucket)
bwRpt := globalNotificationSys.GetBandwidthReports(ctx, bucket)
bwMap := bwRpt.BucketStats[bucket]
for arn, st := range stats.Stats {
for arn, st := range stats.ReplicationStats.Stats {
if bwMap != nil {
if bw, ok := bwMap[arn]; ok {
st.BandWidthLimitInBytesPerSecond = bw.LimitInBytesPerSecond
st.CurrentBandwidthInBytesPerSecond = bw.CurrentBandwidthInBytesPerSecond
stats.Stats[arn] = st
stats.ReplicationStats.Stats[arn] = st
}
}
}
if err = enc.Encode(stats); err != nil {
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
stats.Uptime = UTCNow().Unix() - globalBootTime.Unix()
if err := enc.Encode(stats); err != nil {
writeErrorResponseJSON(ctx, w, toAPIError(ctx, err), r.URL)
return
}
}
@ -533,7 +585,7 @@ func (api objectAPIHandlers) ValidateBucketReplicationCredsHandler(w http.Respon
if rule.Status == replication.Disabled {
continue
}
clnt := globalBucketTargetSys.GetRemoteTargetClient(ctx, rule.Destination.Bucket)
clnt := globalBucketTargetSys.GetRemoteTargetClient(rule.Destination.Bucket)
if clnt == nil {
writeErrorResponse(ctx, w, errorCodes.ToAPIErrWithErr(ErrRemoteTargetNotFoundError, fmt.Errorf("replication config with rule ID %s has a stale target", rule.ID)), r.URL)
return

View File

@ -0,0 +1,381 @@
// Copyright (c) 2015-2023 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package cmd
import (
"fmt"
"sync"
"sync/atomic"
"time"
"github.com/rcrowley/go-metrics"
)
//go:generate msgp -file $GOFILE
const (
// beta is the weight used to calculate exponential moving average
beta = 0.1 // Number of averages considered = 1/(1-beta)
)
// rateMeasurement captures the transfer details for one bucket/target
//msgp:ignore rateMeasurement
type rateMeasurement struct {
lock sync.Mutex
bytesSinceLastWindow uint64 // Total bytes since last window was processed
startTime time.Time // Start time for window
expMovingAvg float64 // Previously calculated exponential moving average
}
// newRateMeasurement creates a new instance of the measurement with the initial start time.
func newRateMeasurement(initTime time.Time) *rateMeasurement {
return &rateMeasurement{
startTime: initTime,
}
}
// incrementBytes add bytes reported for a bucket/target.
func (m *rateMeasurement) incrementBytes(bytes uint64) {
atomic.AddUint64(&m.bytesSinceLastWindow, bytes)
}
// updateExponentialMovingAverage processes the measurements captured so far.
func (m *rateMeasurement) updateExponentialMovingAverage(endTime time.Time) {
// Calculate aggregate avg bandwidth and exp window avg
m.lock.Lock()
defer func() {
m.startTime = endTime
m.lock.Unlock()
}()
if m.startTime.IsZero() {
return
}
if endTime.Before(m.startTime) {
return
}
duration := endTime.Sub(m.startTime)
bytesSinceLastWindow := atomic.SwapUint64(&m.bytesSinceLastWindow, 0)
if m.expMovingAvg == 0 {
// Should address initial calculation and should be fine for resuming from 0
m.expMovingAvg = float64(bytesSinceLastWindow) / duration.Seconds()
return
}
increment := float64(bytesSinceLastWindow) / duration.Seconds()
m.expMovingAvg = exponentialMovingAverage(beta, m.expMovingAvg, increment)
}
// exponentialMovingAverage calculates the exponential moving average
func exponentialMovingAverage(beta, previousAvg, incrementAvg float64) float64 {
return (1-beta)*incrementAvg + beta*previousAvg
}
// getExpMovingAvgBytesPerSecond returns the exponential moving average for the bucket/target in bytes
func (m *rateMeasurement) getExpMovingAvgBytesPerSecond() float64 {
m.lock.Lock()
defer m.lock.Unlock()
return m.expMovingAvg
}
// ActiveWorkerStat is stat for active replication workers
type ActiveWorkerStat struct {
Curr int `json:"curr"`
Avg float32 `json:"avg"`
Max int `json:"max"`
hist metrics.Histogram
}
func newActiveWorkerStat(r metrics.Registry) *ActiveWorkerStat {
h := metrics.NewHistogram(metrics.NewUniformSample(100))
r.Register("replication.active_workers", h)
return &ActiveWorkerStat{
hist: h,
}
}
// update curr and max workers;
func (a *ActiveWorkerStat) update() {
if a == nil {
return
}
a.Curr = globalReplicationPool.ActiveWorkers()
a.hist.Update(int64(a.Curr))
a.Avg = float32(a.hist.Mean())
a.Max = int(a.hist.Max())
}
func (a *ActiveWorkerStat) get() ActiveWorkerStat {
w := ActiveWorkerStat{
Curr: a.Curr,
Avg: a.Avg,
Max: a.Max,
}
return w
}
// QStat holds queue stats for replication
type QStat struct {
Count float64 `json:"count"`
Bytes float64 `json:"bytes"`
}
func (q *QStat) add(o QStat) QStat {
return QStat{Bytes: q.Bytes + o.Bytes, Count: q.Count + o.Count}
}
// InQueueMetric holds queue stats for replication
type InQueueMetric struct {
Curr QStat `json:"curr" msg:"cq"`
Avg QStat `json:"avg" msg:"aq"`
Max QStat `json:"max" msg:"pq"`
}
func (qm InQueueMetric) merge(o InQueueMetric) InQueueMetric {
return InQueueMetric{
Curr: qm.Curr.add(o.Curr),
Avg: qm.Avg.add(o.Avg),
Max: qm.Max.add(o.Max),
}
}
type queueCache struct {
srQueueStats InQueueStats
bucketStats map[string]InQueueStats
sync.RWMutex // mutex for queue stats
}
func newQueueCache(r metrics.Registry) queueCache {
return queueCache{
bucketStats: make(map[string]InQueueStats),
srQueueStats: newInQueueStats(r, "site"),
}
}
func (q *queueCache) update() {
q.Lock()
defer q.Unlock()
q.srQueueStats.update()
for _, s := range q.bucketStats {
s.update()
}
}
func (q *queueCache) getBucketStats(bucket string) InQueueMetric {
q.RLock()
defer q.RUnlock()
v, ok := q.bucketStats[bucket]
if !ok {
return InQueueMetric{}
}
return InQueueMetric{
Curr: QStat{Bytes: float64(v.nowBytes), Count: float64(v.nowCount)},
Max: QStat{Bytes: float64(v.histBytes.Max()), Count: float64(v.histCounts.Max())},
Avg: QStat{Bytes: v.histBytes.Mean(), Count: v.histCounts.Mean()},
}
}
func (q *queueCache) getSiteStats() InQueueMetric {
q.RLock()
defer q.RUnlock()
v := q.srQueueStats
return InQueueMetric{
Curr: QStat{Bytes: float64(v.nowBytes), Count: float64(v.nowCount)},
Max: QStat{Bytes: float64(v.histBytes.Max()), Count: float64(v.histCounts.Max())},
Avg: QStat{Bytes: v.histBytes.Mean(), Count: v.histCounts.Mean()},
}
}
// InQueueStats holds queue stats for replication
type InQueueStats struct {
nowBytes int64 `json:"-"`
nowCount int64 `json:"-"`
histCounts metrics.Histogram
histBytes metrics.Histogram
}
func newInQueueStats(r metrics.Registry, lbl string) InQueueStats {
histCounts := metrics.NewHistogram(metrics.NewUniformSample(100))
histBytes := metrics.NewHistogram(metrics.NewUniformSample(100))
r.Register("replication.queue.counts."+lbl, histCounts)
r.Register("replication.queue.bytes."+lbl, histBytes)
return InQueueStats{
histCounts: histCounts,
histBytes: histBytes,
}
}
func (q *InQueueStats) update() {
q.histBytes.Update(atomic.LoadInt64(&q.nowBytes))
q.histCounts.Update(atomic.LoadInt64(&q.nowCount))
}
// XferStats has transfer stats for replication
type XferStats struct {
Curr float64 `json:"currRate" msg:"cr"`
Avg float64 `json:"avgRate" msg:"av"`
Peak float64 `json:"peakRate" msg:"p"`
N int64 `json:"n" msg:"n"`
measure *rateMeasurement `json:"-"`
sma *SMA `json:"-"`
}
// Clone returns a copy of XferStats
func (rx *XferStats) Clone() *XferStats {
curr := rx.curr()
peak := rx.Peak
if curr > peak {
peak = curr
}
return &XferStats{
Curr: curr,
Avg: rx.Avg,
Peak: peak,
N: rx.N,
measure: rx.measure,
}
}
func newXferStats() *XferStats {
return &XferStats{
measure: newRateMeasurement(time.Now()),
sma: newSMA(50),
}
}
func (rx *XferStats) String() string {
return fmt.Sprintf("curr=%f avg=%f, peak=%f", rx.curr(), rx.Avg, rx.Peak)
}
func (rx *XferStats) curr() float64 {
if rx.measure == nil {
return 0.0
}
return rx.measure.getExpMovingAvgBytesPerSecond()
}
func (rx *XferStats) merge(o XferStats) XferStats {
curr := calcAvg(rx.curr(), o.curr(), rx.N, o.N)
peak := rx.Peak
if o.Peak > peak {
peak = o.Peak
}
if curr > peak {
peak = curr
}
return XferStats{
Avg: calcAvg(rx.Avg, o.Avg, rx.N, o.N),
Peak: peak,
Curr: curr,
measure: rx.measure,
N: rx.N + o.N,
}
}
func calcAvg(x, y float64, n1, n2 int64) float64 {
if n1+n2 == 0 {
return 0
}
avg := (x*float64(n1) + y*float64(n2)) / float64(n1+n2)
return avg
}
// Add a new transfer
func (rx *XferStats) addSize(sz int64, t time.Duration) {
if rx.measure == nil {
rx.measure = newRateMeasurement(time.Now())
}
rx.measure.incrementBytes(uint64(sz))
rx.Curr = rx.measure.getExpMovingAvgBytesPerSecond()
rx.sma.addSample(rx.Curr)
rx.Avg = rx.sma.simpleMovingAvg()
if rx.Curr > rx.Peak {
rx.Peak = rx.Curr
}
rx.N++
}
// ReplicationMRFStats holds stats of MRF backlog saved to disk in the last 5 minutes
// and number of entries that failed replication after 3 retries
type ReplicationMRFStats struct {
LastFailedCount uint64 `json:"failedCount_last5min"`
// Count of unreplicated entries that were dropped after MRF retry limit reached since cluster start.
TotalDroppedCount uint64 `json:"droppedCount_since_uptime"`
// Bytes of unreplicated entries that were dropped after MRF retry limit reached since cluster start.
TotalDroppedBytes uint64 `json:"droppedBytes_since_uptime"`
}
// SMA struct for calculating simple moving average
type SMA struct {
buf []float64
window int // len of buf
idx int // current index in buf
CAvg float64 // cumulative average
prevSMA float64
filledBuf bool
}
func newSMA(len int) *SMA {
if len <= 0 {
len = defaultWindowSize
}
return &SMA{
buf: make([]float64, len),
window: len,
idx: 0,
}
}
func (s *SMA) addSample(next float64) {
prev := s.buf[s.idx]
s.buf[s.idx] = next
if s.filledBuf {
s.prevSMA += (next - prev) / float64(s.window)
s.CAvg += (next - s.CAvg) / float64(s.window)
} else {
s.CAvg = s.simpleMovingAvg()
s.prevSMA = s.CAvg
}
if s.idx == s.window-1 {
s.filledBuf = true
}
s.idx = (s.idx + 1) % s.window
}
func (s *SMA) simpleMovingAvg() float64 {
if s.filledBuf {
return s.prevSMA
}
var tot float64
for _, r := range s.buf {
tot += r
}
return tot / float64(s.idx+1)
}
const (
defaultWindowSize = 10
)

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,801 @@
package cmd
// Code generated by github.com/tinylib/msgp DO NOT EDIT.
import (
"bytes"
"testing"
"github.com/tinylib/msgp/msgp"
)
func TestMarshalUnmarshalActiveWorkerStat(t *testing.T) {
v := ActiveWorkerStat{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgActiveWorkerStat(b *testing.B) {
v := ActiveWorkerStat{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgActiveWorkerStat(b *testing.B) {
v := ActiveWorkerStat{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalActiveWorkerStat(b *testing.B) {
v := ActiveWorkerStat{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeActiveWorkerStat(t *testing.T) {
v := ActiveWorkerStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeActiveWorkerStat Msgsize() is inaccurate")
}
vn := ActiveWorkerStat{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeActiveWorkerStat(b *testing.B) {
v := ActiveWorkerStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeActiveWorkerStat(b *testing.B) {
v := ActiveWorkerStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalInQueueMetric(t *testing.T) {
v := InQueueMetric{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgInQueueMetric(b *testing.B) {
v := InQueueMetric{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgInQueueMetric(b *testing.B) {
v := InQueueMetric{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalInQueueMetric(b *testing.B) {
v := InQueueMetric{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeInQueueMetric(t *testing.T) {
v := InQueueMetric{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeInQueueMetric Msgsize() is inaccurate")
}
vn := InQueueMetric{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeInQueueMetric(b *testing.B) {
v := InQueueMetric{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeInQueueMetric(b *testing.B) {
v := InQueueMetric{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalInQueueStats(t *testing.T) {
v := InQueueStats{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgInQueueStats(b *testing.B) {
v := InQueueStats{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgInQueueStats(b *testing.B) {
v := InQueueStats{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalInQueueStats(b *testing.B) {
v := InQueueStats{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeInQueueStats(t *testing.T) {
v := InQueueStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeInQueueStats Msgsize() is inaccurate")
}
vn := InQueueStats{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeInQueueStats(b *testing.B) {
v := InQueueStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeInQueueStats(b *testing.B) {
v := InQueueStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalQStat(t *testing.T) {
v := QStat{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgQStat(b *testing.B) {
v := QStat{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgQStat(b *testing.B) {
v := QStat{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalQStat(b *testing.B) {
v := QStat{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeQStat(t *testing.T) {
v := QStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeQStat Msgsize() is inaccurate")
}
vn := QStat{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeQStat(b *testing.B) {
v := QStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeQStat(b *testing.B) {
v := QStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalReplicationMRFStats(t *testing.T) {
v := ReplicationMRFStats{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgReplicationMRFStats(b *testing.B) {
v := ReplicationMRFStats{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgReplicationMRFStats(b *testing.B) {
v := ReplicationMRFStats{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalReplicationMRFStats(b *testing.B) {
v := ReplicationMRFStats{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeReplicationMRFStats(t *testing.T) {
v := ReplicationMRFStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeReplicationMRFStats Msgsize() is inaccurate")
}
vn := ReplicationMRFStats{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeReplicationMRFStats(b *testing.B) {
v := ReplicationMRFStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeReplicationMRFStats(b *testing.B) {
v := ReplicationMRFStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalSMA(t *testing.T) {
v := SMA{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgSMA(b *testing.B) {
v := SMA{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgSMA(b *testing.B) {
v := SMA{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalSMA(b *testing.B) {
v := SMA{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeSMA(t *testing.T) {
v := SMA{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeSMA Msgsize() is inaccurate")
}
vn := SMA{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeSMA(b *testing.B) {
v := SMA{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeSMA(b *testing.B) {
v := SMA{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalXferStats(t *testing.T) {
v := XferStats{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgXferStats(b *testing.B) {
v := XferStats{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgXferStats(b *testing.B) {
v := XferStats{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalXferStats(b *testing.B) {
v := XferStats{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeXferStats(t *testing.T) {
v := XferStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeXferStats Msgsize() is inaccurate")
}
vn := XferStats{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeXferStats(b *testing.B) {
v := XferStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeXferStats(b *testing.B) {
v := XferStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}

View File

@ -19,12 +19,12 @@ package cmd
import (
"context"
"encoding/binary"
"math"
"sync"
"sync/atomic"
"time"
"github.com/minio/minio/internal/bucket/replication"
"github.com/rcrowley/go-metrics"
)
func (b *BucketReplicationStats) hasReplicationUsage() bool {
@ -38,12 +38,93 @@ func (b *BucketReplicationStats) hasReplicationUsage() bool {
// ReplicationStats holds the global in-memory replication stats
type ReplicationStats struct {
// map of site deployment ID to site replication status
// for site replication - maintain stats at global level
srStats *SRStats
// active worker stats
workers *ActiveWorkerStat
// queue stats cache
qCache queueCache
// mrf backlog stats
mrfStats ReplicationMRFStats
// for bucket replication, continue to use existing cache
Cache map[string]*BucketReplicationStats
UsageCache map[string]*BucketReplicationStats
mostRecentStats BucketStatsMap
sync.RWMutex // mutex for Cache
ulock sync.RWMutex // mutex for UsageCache
mostRecentStatsMu sync.Mutex // mutex for mostRecentStats
registry metrics.Registry
sync.RWMutex // mutex for Cache
mostRecentStatsMu sync.Mutex // mutex for mostRecentStats
wlock sync.RWMutex // mutex for active workers
movingAvgTicker *time.Ticker // Ticker for calculating moving averages
wTimer *time.Ticker // ticker for calculating active workers
qTimer *time.Ticker // ticker for calculating queue stats
}
func (r *ReplicationStats) trackEWMA() {
for {
select {
case <-r.movingAvgTicker.C:
r.updateMovingAvg()
case <-GlobalContext.Done():
return
}
}
}
func (r *ReplicationStats) updateMovingAvg() {
r.RLock()
for _, s := range r.Cache {
for _, st := range s.Stats {
st.XferRateLrg.measure.updateExponentialMovingAverage(time.Now())
st.XferRateSml.measure.updateExponentialMovingAverage(time.Now())
}
}
r.RUnlock()
}
// ActiveWorkers returns worker stats
func (r *ReplicationStats) ActiveWorkers() ActiveWorkerStat {
r.wlock.RLock()
defer r.wlock.RUnlock()
w := r.workers.get()
return ActiveWorkerStat{
Curr: w.Curr,
Max: w.Max,
Avg: w.Avg,
}
}
func (r *ReplicationStats) collectWorkerMetrics(ctx context.Context) {
if r == nil {
return
}
for {
select {
case <-ctx.Done():
return
case <-r.wTimer.C:
r.wlock.Lock()
r.workers.update()
r.wlock.Unlock()
}
}
}
func (r *ReplicationStats) collectQueueMetrics(ctx context.Context) {
if r == nil {
return
}
for {
select {
case <-ctx.Done():
return
case <-r.qTimer.C:
r.qCache.update()
}
}
}
// Delete deletes in-memory replication statistics for a bucket.
@ -55,10 +136,6 @@ func (r *ReplicationStats) Delete(bucket string) {
r.Lock()
defer r.Unlock()
delete(r.Cache, bucket)
r.ulock.Lock()
defer r.ulock.Unlock()
delete(r.UsageCache, bucket)
}
// UpdateReplicaStat updates in-memory replica statistics with new values.
@ -71,83 +148,130 @@ func (r *ReplicationStats) UpdateReplicaStat(bucket string, n int64) {
defer r.Unlock()
bs, ok := r.Cache[bucket]
if !ok {
bs = &BucketReplicationStats{Stats: make(map[string]*BucketReplicationStat)}
bs = newBucketReplicationStats()
}
bs.ReplicaSize += n
bs.ReplicaCount++
r.Cache[bucket] = bs
r.srUpdateReplicaStat(n)
}
// Update updates in-memory replication statistics with new values.
func (r *ReplicationStats) Update(bucket string, arn string, n int64, duration time.Duration, status, prevStatus replication.StatusType, opType replication.Type) {
func (r *ReplicationStats) srUpdateReplicaStat(sz int64) {
if r == nil {
return
}
r.Lock()
defer r.Unlock()
atomic.AddInt64(&r.srStats.ReplicaSize, sz)
atomic.AddInt64(&r.srStats.ReplicaCount, 1)
}
bs, ok := r.Cache[bucket]
if !ok {
bs = &BucketReplicationStats{Stats: make(map[string]*BucketReplicationStat)}
r.Cache[bucket] = bs
}
b, ok := bs.Stats[arn]
if !ok {
b = &BucketReplicationStat{}
bs.Stats[arn] = b
}
switch status {
case replication.Pending:
if opType.IsDataReplication() && prevStatus != status {
b.PendingSize += n
b.PendingCount++
}
case replication.Completed:
switch prevStatus { // adjust counters based on previous state
case replication.Pending:
b.PendingCount--
case replication.Failed:
b.FailedCount--
}
if opType.IsDataReplication() {
b.ReplicatedSize += n
switch prevStatus {
case replication.Pending:
b.PendingSize -= n
case replication.Failed:
b.FailedSize -= n
}
if duration > 0 {
b.Latency.update(n, duration)
}
}
case replication.Failed:
if opType.IsDataReplication() {
if prevStatus == replication.Pending {
b.FailedSize += n
b.FailedCount++
b.PendingSize -= n
b.PendingCount--
}
}
case replication.Replica:
if opType == replication.ObjectReplicationType {
b.ReplicaSize += n
}
func (r *ReplicationStats) srUpdate(sr replStat) {
dID, err := globalSiteReplicationSys.getDeplIDForEndpoint(sr.endpoint())
if err == nil {
r.srStats.update(sr, dID)
}
}
// GetInitialUsage get replication metrics available at the time of cluster initialization
func (r *ReplicationStats) GetInitialUsage(bucket string) BucketReplicationStats {
// Update updates in-memory replication statistics with new values.
func (r *ReplicationStats) Update(bucket string, ri replicatedTargetInfo, status, prevStatus replication.StatusType) {
if r == nil {
return BucketReplicationStats{}
return
}
r.ulock.RLock()
defer r.ulock.RUnlock()
st, ok := r.UsageCache[bucket]
var rs replStat
switch status {
case replication.Pending:
if ri.OpType.IsDataReplication() && prevStatus != status {
rs.set(ri.Arn, ri.Size, 0, status, ri.OpType, ri.endpoint, ri.secure, ri.Err)
}
case replication.Completed:
if ri.OpType.IsDataReplication() {
rs.set(ri.Arn, ri.Size, ri.Duration, status, ri.OpType, ri.endpoint, ri.secure, ri.Err)
}
case replication.Failed:
if ri.OpType.IsDataReplication() && prevStatus == replication.Pending {
rs.set(ri.Arn, ri.Size, ri.Duration, status, ri.OpType, ri.endpoint, ri.secure, ri.Err)
}
case replication.Replica:
if ri.OpType == replication.ObjectReplicationType {
rs.set(ri.Arn, ri.Size, 0, status, ri.OpType, "", false, ri.Err)
}
}
// update site-replication in-memory stats
if rs.Completed || rs.Failed {
r.srUpdate(rs)
}
r.Lock()
defer r.Unlock()
// update bucket replication in-memory stats
bs, ok := r.Cache[bucket]
if !ok {
return BucketReplicationStats{}
bs = newBucketReplicationStats()
r.Cache[bucket] = bs
}
b, ok := bs.Stats[ri.Arn]
if !ok {
b = &BucketReplicationStat{
XferRateLrg: newXferStats(),
XferRateSml: newXferStats(),
}
bs.Stats[ri.Arn] = b
}
switch {
case rs.Completed:
b.ReplicatedSize += rs.TransferSize
b.ReplicatedCount++
if rs.TransferDuration > 0 {
b.Latency.update(rs.TransferSize, rs.TransferDuration)
b.updateXferRate(rs.TransferSize, rs.TransferDuration)
}
case rs.Failed:
b.FailStats.addsize(rs.TransferSize, rs.Err)
case rs.Pending:
}
}
type replStat struct {
Arn string
Completed bool
Pending bool
Failed bool
opType replication.Type
// transfer size
TransferSize int64
// transfer duration
TransferDuration time.Duration
Endpoint string
Secure bool
Err error
}
func (rs *replStat) endpoint() string {
scheme := "http"
if rs.Secure {
scheme = "https"
}
return scheme + "://" + rs.Endpoint
}
func (rs *replStat) set(arn string, n int64, duration time.Duration, status replication.StatusType, opType replication.Type, endpoint string, secure bool, err error) {
rs.Endpoint = endpoint
rs.Secure = secure
rs.TransferSize = n
rs.Arn = arn
rs.TransferDuration = duration
rs.opType = opType
switch status {
case replication.Completed:
rs.Completed = true
case replication.Pending:
rs.Pending = true
case replication.Failed:
rs.Failed = true
rs.Err = err
}
return st.Clone()
}
// GetAll returns replication metrics for all buckets at once.
@ -157,16 +281,36 @@ func (r *ReplicationStats) GetAll() map[string]BucketReplicationStats {
}
r.RLock()
defer r.RUnlock()
bucketReplicationStats := make(map[string]BucketReplicationStats, len(r.Cache))
for k, v := range r.Cache {
bucketReplicationStats[k] = v.Clone()
}
r.RUnlock()
for k, v := range bucketReplicationStats {
v.QStat = r.qCache.getBucketStats(k)
bucketReplicationStats[k] = v
}
return bucketReplicationStats
}
func (r *ReplicationStats) getSRMetricsForNode() SRMetricsSummary {
if r == nil {
return SRMetricsSummary{}
}
m := SRMetricsSummary{
Uptime: UTCNow().Unix() - globalBootTime.Unix(),
Queued: r.qCache.getSiteStats(),
ActiveWorkers: r.ActiveWorkers(),
Metrics: r.srStats.get(),
ReplicaSize: atomic.LoadInt64(&r.srStats.ReplicaSize),
ReplicaCount: atomic.LoadInt64(&r.srStats.ReplicaCount),
}
return m
}
// Get replication metrics for a bucket from this node since this node came up.
func (r *ReplicationStats) Get(bucket string) BucketReplicationStats {
if r == nil {
@ -178,99 +322,35 @@ func (r *ReplicationStats) Get(bucket string) BucketReplicationStats {
st, ok := r.Cache[bucket]
if !ok {
return BucketReplicationStats{}
return BucketReplicationStats{Stats: make(map[string]*BucketReplicationStat)}
}
return st.Clone()
}
// NewReplicationStats initialize in-memory replication statistics
func NewReplicationStats(ctx context.Context, objectAPI ObjectLayer) *ReplicationStats {
return &ReplicationStats{
Cache: make(map[string]*BucketReplicationStats),
UsageCache: make(map[string]*BucketReplicationStats),
r := metrics.NewRegistry()
rs := ReplicationStats{
Cache: make(map[string]*BucketReplicationStats),
qCache: newQueueCache(r),
srStats: newSRStats(),
movingAvgTicker: time.NewTicker(2 * time.Second),
wTimer: time.NewTicker(2 * time.Second),
qTimer: time.NewTicker(2 * time.Second),
workers: newActiveWorkerStat(r),
registry: r,
}
go rs.collectWorkerMetrics(ctx)
go rs.collectQueueMetrics(ctx)
return &rs
}
// load replication metrics at cluster start from latest replication stats saved in .minio.sys/buckets/replication/node-name.stats
// fallback to replication stats in data usage to be backward compatible
func (r *ReplicationStats) loadInitialReplicationMetrics(ctx context.Context) {
m := make(map[string]*BucketReplicationStats)
if stats, err := globalReplicationPool.loadStatsFromDisk(); err == nil {
for b, st := range stats {
c := st.Clone()
m[b] = &c
}
r.ulock.Lock()
r.UsageCache = m
r.ulock.Unlock()
return
}
rTimer := time.NewTimer(time.Second * 5)
defer rTimer.Stop()
var (
dui DataUsageInfo
err error
)
outer:
for {
select {
case <-ctx.Done():
return
case <-rTimer.C:
dui, err = loadDataUsageFromBackend(GlobalContext, newObjectLayerFn())
// If LastUpdate is set, data usage is available.
if err == nil {
break outer
}
rTimer.Reset(time.Second * 5)
}
}
for bucket, usage := range dui.BucketsUsage {
b := &BucketReplicationStats{
Stats: make(map[string]*BucketReplicationStat, len(usage.ReplicationInfo)),
}
for arn, uinfo := range usage.ReplicationInfo {
b.Stats[arn] = &BucketReplicationStat{
FailedSize: int64(uinfo.ReplicationFailedSize),
ReplicatedSize: int64(uinfo.ReplicatedSize),
ReplicaSize: int64(uinfo.ReplicaSize),
FailedCount: int64(uinfo.ReplicationFailedCount),
}
}
b.ReplicaSize += int64(usage.ReplicaSize)
if b.hasReplicationUsage() {
m[bucket] = b
}
}
r.ulock.Lock()
r.UsageCache = m
r.ulock.Unlock()
}
// serializeStats will serialize the current stats.
// Will return (nil, nil) if no data.
func (r *ReplicationStats) serializeStats() ([]byte, error) {
if r == nil {
return nil, nil
}
r.mostRecentStatsMu.Lock()
defer r.mostRecentStatsMu.Unlock()
if len(r.mostRecentStats.Stats) == 0 {
return nil, nil
}
data := make([]byte, 4, 4+r.mostRecentStats.Msgsize())
// Add the replication stats meta header.
binary.LittleEndian.PutUint16(data[0:2], replStatsMetaFormat)
binary.LittleEndian.PutUint16(data[2:4], replStatsVersion)
// Add data
return r.mostRecentStats.MarshalMsg(data)
}
func (r *ReplicationStats) getAllLatest(bucketsUsage map[string]BucketUsageInfo) (bucketsReplicationStats map[string]BucketReplicationStats) {
func (r *ReplicationStats) getAllLatest(bucketsUsage map[string]BucketUsageInfo) (bucketsReplicationStats map[string]BucketStats) {
peerBucketStatsList := globalNotificationSys.GetClusterAllBucketStats(GlobalContext)
bucketsReplicationStats = make(map[string]BucketReplicationStats, len(bucketsUsage))
bucketsReplicationStats = make(map[string]BucketStats, len(bucketsUsage))
for bucket, u := range bucketsUsage {
for bucket := range bucketsUsage {
bucketStats := make([]BucketStats, len(peerBucketStatsList))
for i, peerBucketStats := range peerBucketStatsList {
bucketStat, ok := peerBucketStats.Stats[bucket]
@ -279,110 +359,126 @@ func (r *ReplicationStats) getAllLatest(bucketsUsage map[string]BucketUsageInfo)
}
bucketStats[i] = bucketStat
}
bucketsReplicationStats[bucket] = r.calculateBucketReplicationStats(bucket, u, bucketStats)
bucketsReplicationStats[bucket] = r.calculateBucketReplicationStats(bucket, bucketStats)
}
return bucketsReplicationStats
}
func (r *ReplicationStats) calculateBucketReplicationStats(bucket string, u BucketUsageInfo, bucketStats []BucketStats) (s BucketReplicationStats) {
func (r *ReplicationStats) calculateBucketReplicationStats(bucket string, bucketStats []BucketStats) (bs BucketStats) {
if r == nil {
s = BucketReplicationStats{
Stats: make(map[string]*BucketReplicationStat),
bs = BucketStats{
ReplicationStats: BucketReplicationStats{
Stats: make(map[string]*BucketReplicationStat),
},
QueueStats: ReplicationQueueStats{},
}
return s
return bs
}
var s BucketReplicationStats
// accumulate cluster bucket stats
stats := make(map[string]*BucketReplicationStat)
var totReplicaSize int64
var (
totReplicaSize, totReplicatedSize int64
totReplicaCount, totReplicatedCount int64
totFailed RTimedMetrics
tq InQueueMetric
)
for _, bucketStat := range bucketStats {
totReplicaSize += bucketStat.ReplicationStats.ReplicaSize
totReplicaCount += bucketStat.ReplicationStats.ReplicaCount
for _, q := range bucketStat.QueueStats.Nodes {
tq = tq.merge(q.QStats)
}
for arn, stat := range bucketStat.ReplicationStats.Stats {
oldst := stats[arn]
if oldst == nil {
oldst = &BucketReplicationStat{}
oldst = &BucketReplicationStat{
XferRateLrg: newXferStats(),
XferRateSml: newXferStats(),
}
}
fstats := stat.FailStats.merge(oldst.FailStats)
lrg := oldst.XferRateLrg.merge(*stat.XferRateLrg)
sml := oldst.XferRateSml.merge(*stat.XferRateSml)
stats[arn] = &BucketReplicationStat{
FailedCount: stat.FailedCount + oldst.FailedCount,
FailedSize: stat.FailedSize + oldst.FailedSize,
ReplicatedSize: stat.ReplicatedSize + oldst.ReplicatedSize,
Latency: stat.Latency.merge(oldst.Latency),
PendingCount: stat.PendingCount + oldst.PendingCount,
PendingSize: stat.PendingSize + oldst.PendingSize,
Failed: fstats.toMetric(),
FailStats: fstats,
ReplicatedSize: stat.ReplicatedSize + oldst.ReplicatedSize,
ReplicatedCount: stat.ReplicatedCount + oldst.ReplicatedCount,
Latency: stat.Latency.merge(oldst.Latency),
XferRateLrg: &lrg,
XferRateSml: &sml,
}
totReplicatedSize += stat.ReplicatedSize
totReplicatedCount += stat.ReplicatedCount
totFailed = totFailed.merge(stat.FailStats)
}
}
// add initial usage stat to cluster stats
usageStat := globalReplicationStats.GetInitialUsage(bucket)
totReplicaSize += usageStat.ReplicaSize
for arn, stat := range usageStat.Stats {
st, ok := stats[arn]
if !ok {
st = &BucketReplicationStat{}
stats[arn] = st
}
st.ReplicatedSize += stat.ReplicatedSize
st.FailedSize += stat.FailedSize
st.FailedCount += stat.FailedCount
st.PendingSize += stat.PendingSize
st.PendingCount += stat.PendingCount
}
s = BucketReplicationStats{
Stats: make(map[string]*BucketReplicationStat, len(stats)),
Stats: stats,
QStat: tq,
ReplicaSize: totReplicaSize,
ReplicaCount: totReplicaCount,
ReplicatedSize: totReplicatedSize,
ReplicatedCount: totReplicatedCount,
Failed: totFailed.toMetric(),
}
var latestTotReplicatedSize int64
for _, st := range u.ReplicationInfo {
latestTotReplicatedSize += int64(st.ReplicatedSize)
var qs ReplicationQueueStats
for _, bs := range bucketStats {
qs.Nodes = append(qs.Nodes, bs.QueueStats.Nodes...)
}
// normalize computed real time stats with latest usage stat
for arn, tgtstat := range stats {
st := BucketReplicationStat{}
bu, ok := u.ReplicationInfo[arn]
if !ok {
bu = BucketTargetUsageInfo{}
}
// use in memory replication stats if it is ahead of usage info.
st.ReplicatedSize = int64(bu.ReplicatedSize)
if tgtstat.ReplicatedSize >= int64(bu.ReplicatedSize) {
st.ReplicatedSize = tgtstat.ReplicatedSize
}
s.ReplicatedSize += st.ReplicatedSize
// Reset FailedSize and FailedCount to 0 for negative overflows which can
// happen since data usage picture can lag behind actual usage state at the time of cluster start
st.FailedSize = int64(math.Max(float64(tgtstat.FailedSize), 0))
st.FailedCount = int64(math.Max(float64(tgtstat.FailedCount), 0))
st.PendingSize = int64(math.Max(float64(tgtstat.PendingSize), 0))
st.PendingCount = int64(math.Max(float64(tgtstat.PendingCount), 0))
st.Latency = tgtstat.Latency
s.Stats[arn] = &st
s.FailedSize += st.FailedSize
s.FailedCount += st.FailedCount
s.PendingCount += st.PendingCount
s.PendingSize += st.PendingSize
qs.Uptime = UTCNow().Unix() - globalBootTime.Unix()
bs = BucketStats{
ReplicationStats: s,
QueueStats: qs,
}
// normalize overall stats
s.ReplicaSize = int64(math.Max(float64(totReplicaSize), float64(u.ReplicaSize)))
s.ReplicatedSize = int64(math.Max(float64(s.ReplicatedSize), float64(latestTotReplicatedSize)))
r.mostRecentStatsMu.Lock()
if len(r.mostRecentStats.Stats) == 0 {
r.mostRecentStats = BucketStatsMap{Stats: make(map[string]BucketStats, 1), Timestamp: UTCNow()}
}
if len(s.Stats) > 0 {
r.mostRecentStats.Stats[bucket] = BucketStats{ReplicationStats: s}
if len(bs.ReplicationStats.Stats) > 0 {
r.mostRecentStats.Stats[bucket] = bs
}
r.mostRecentStats.Timestamp = UTCNow()
r.mostRecentStatsMu.Unlock()
return s
return bs
}
// get the most current of in-memory replication stats and data usage info from crawler.
func (r *ReplicationStats) getLatestReplicationStats(bucket string, u BucketUsageInfo) (s BucketReplicationStats) {
func (r *ReplicationStats) getLatestReplicationStats(bucket string) (s BucketStats) {
bucketStats := globalNotificationSys.GetClusterBucketStats(GlobalContext, bucket)
return r.calculateBucketReplicationStats(bucket, u, bucketStats)
return r.calculateBucketReplicationStats(bucket, bucketStats)
}
func (r *ReplicationStats) incQ(bucket string, sz int64, isDeleleRepl bool, opType replication.Type) {
r.qCache.Lock()
defer r.qCache.Unlock()
v, ok := r.qCache.bucketStats[bucket]
if !ok {
v = newInQueueStats(r.registry, bucket)
}
atomic.AddInt64(&v.nowBytes, sz)
atomic.AddInt64(&v.nowCount, 1)
r.qCache.bucketStats[bucket] = v
atomic.AddInt64(&r.qCache.srQueueStats.nowBytes, sz)
atomic.AddInt64(&r.qCache.srQueueStats.nowCount, 1)
}
func (r *ReplicationStats) decQ(bucket string, sz int64, isDelMarker bool, opType replication.Type) {
r.qCache.Lock()
defer r.qCache.Unlock()
v, ok := r.qCache.bucketStats[bucket]
if !ok {
v = newInQueueStats(r.registry, bucket)
}
atomic.AddInt64(&v.nowBytes, -1*sz)
atomic.AddInt64(&v.nowCount, -1)
r.qCache.bucketStats[bucket] = v
atomic.AddInt64(&r.qCache.srQueueStats.nowBytes, -1*sz)
atomic.AddInt64(&r.qCache.srQueueStats.nowCount, -1)
}

View File

@ -49,6 +49,9 @@ type replicatedTargetInfo struct {
VersionPurgeStatus VersionPurgeStatusType
ResyncTimestamp string
ReplicationResynced bool // true only if resync attempted for this target
endpoint string
secure bool
Err error // replication error if any
}
// Empty returns true for a target if arn is empty
@ -320,7 +323,7 @@ func parseReplicateDecision(ctx context.Context, bucket, s string) (r ReplicateD
if err != nil {
return r, err
}
tgtClnt := globalBucketTargetSys.GetRemoteTargetClient(ctx, slc[0])
tgtClnt := globalBucketTargetSys.GetRemoteTargetClient(slc[0])
if tgtClnt == nil {
// Skip stale targets if any and log them to be missing atleast once.
logger.LogOnceIf(ctx, fmt.Errorf("failed to get target for bucket:%s arn:%s", bucket, slc[0]), slc[0])
@ -800,6 +803,7 @@ type MRFReplicateEntry struct {
Object string `json:"object" msg:"o"`
versionID string `json:"-"`
RetryCount int `json:"retryCount" msg:"rc"`
sz int64 `json:"-"`
}
// MRFReplicateEntries has the map of MRF entries to save to disk
@ -814,17 +818,7 @@ func (ri ReplicateObjectInfo) ToMRFEntry() MRFReplicateEntry {
Bucket: ri.Bucket,
Object: ri.Name,
versionID: ri.VersionID,
sz: ri.Size,
RetryCount: int(ri.RetryCount),
}
}
func getReplicationStatsPath() string {
return bucketMetaPrefix + SlashSeparator + replicationDir + SlashSeparator + "replication.stats"
}
const (
replStatsMetaFormat = 1
replStatsVersionV1 = 1
replStatsVersion = replStatsVersionV1
replStatsSaveInterval = time.Minute * 5
)

View File

@ -112,7 +112,7 @@ func validateReplicationDestination(ctx context.Context, bucket string, rCfg *re
if arn.Type != madmin.ReplicationService {
return sameTarget, toAPIError(ctx, BucketRemoteArnTypeInvalid{Bucket: bucket})
}
clnt := globalBucketTargetSys.GetRemoteTargetClient(ctx, arnStr)
clnt := globalBucketTargetSys.GetRemoteTargetClient(arnStr)
if clnt == nil {
return sameTarget, toAPIError(ctx, BucketRemoteTargetNotFound{Bucket: bucket})
}
@ -137,7 +137,7 @@ func validateReplicationDestination(ctx context.Context, bucket string, rCfg *re
}
}
// validate replication ARN against target endpoint
c := globalBucketTargetSys.GetRemoteTargetClient(ctx, arnStr)
c := globalBucketTargetSys.GetRemoteTargetClient(arnStr)
if c != nil {
if err := checkRemoteEndpoint(ctx, c.EndpointURL()); err != nil {
switch err.(type) {
@ -281,7 +281,7 @@ func mustReplicate(ctx context.Context, bucket, object string, mopts mustReplica
}
tgtArns := cfg.FilterTargetArns(opts)
for _, tgtArn := range tgtArns {
tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, tgtArn)
tgt := globalBucketTargetSys.GetRemoteTargetClient(tgtArn)
// the target online status should not be used here while deciding
// whether to replicate as the target could be temporarily down
opts.TargetArn = tgtArn
@ -380,7 +380,7 @@ func checkReplicateDelete(ctx context.Context, bucket string, dobj ObjectToDelet
continue
}
}
tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, tgtArn)
tgt := globalBucketTargetSys.GetRemoteTargetClient(tgtArn)
// the target online status should not be used here while deciding
// whether to replicate deletes as the target could be temporarily down
tgtDsc := newReplicateTargetDecision(tgtArn, false, false)
@ -517,8 +517,8 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, obj
// to decrement pending count later.
for _, rinfo := range rinfos.Targets {
if rinfo.ReplicationStatus != rinfo.PrevReplicationStatus {
globalReplicationStats.Update(dobj.Bucket, rinfo.Arn, 0, 0, replicationStatus,
prevStatus, replication.DeleteReplicationType)
globalReplicationStats.Update(dobj.Bucket, rinfo, replicationStatus,
prevStatus)
}
}
@ -575,6 +575,8 @@ func replicateDeleteToTarget(ctx context.Context, dobj DeletedObjectReplicationI
rinfo = dobj.ReplicationState.targetState(tgt.ARN)
rinfo.OpType = dobj.OpType
rinfo.endpoint = tgt.EndpointURL().Host
rinfo.secure = tgt.EndpointURL().Scheme == "https"
defer func() {
if rinfo.ReplicationStatus == replication.Completed && tgt.ResetID != "" && dobj.OpType == replication.ExistingObjectReplicationType {
rinfo.ResyncTimestamp = fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), tgt.ResetID)
@ -639,6 +641,7 @@ func replicateDeleteToTarget(ctx context.Context, dobj DeletedObjectReplicationI
// this request yet (object version not replicated yet)
if err != nil && !toi.ReplicationReady {
rinfo.ReplicationStatus = replication.Failed
rinfo.Err = err
return
}
}
@ -653,6 +656,7 @@ func replicateDeleteToTarget(ctx context.Context, dobj DeletedObjectReplicationI
},
})
if rmErr != nil {
rinfo.Err = rmErr
if dobj.VersionID == "" {
rinfo.ReplicationStatus = replication.Failed
} else {
@ -1003,7 +1007,7 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje
var rinfos replicatedInfos
rinfos.Targets = make([]replicatedTargetInfo, len(tgtArns))
for i, tgtArn := range tgtArns {
tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, tgtArn)
tgt := globalBucketTargetSys.GetRemoteTargetClient(tgtArn)
if tgt == nil {
logger.LogOnceIf(ctx, fmt.Errorf("failed to get target for bucket:%s arn:%s", bucket, tgtArn), tgtArn)
sendEvent(eventArgs{
@ -1066,7 +1070,8 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje
}
for _, rinfo := range rinfos.Targets {
if rinfo.ReplicationStatus != rinfo.PrevReplicationStatus {
globalReplicationStats.Update(bucket, rinfo.Arn, rinfo.Size, rinfo.Duration, rinfo.ReplicationStatus, rinfo.PrevReplicationStatus, opType)
rinfo.OpType = opType // update optype to reflect correct operation.
globalReplicationStats.Update(bucket, rinfo, rinfo.ReplicationStatus, rinfo.PrevReplicationStatus)
}
}
}
@ -1107,8 +1112,9 @@ func (ri ReplicateObjectInfo) replicateObject(ctx context.Context, objectAPI Obj
ReplicationStatus: replication.Failed,
OpType: ri.OpType,
ReplicationAction: rAction,
endpoint: tgt.EndpointURL().Host,
secure: tgt.EndpointURL().Scheme == "https",
}
if ri.ObjectInfo.TargetReplicationStatus(tgt.ARN) == replication.Completed && !ri.ExistingObjResync.Empty() && !ri.ExistingObjResync.mustResyncTarget(tgt.ARN) {
rinfo.ReplicationStatus = replication.Completed
rinfo.ReplicationResynced = true
@ -1223,22 +1229,22 @@ func (ri ReplicateObjectInfo) replicateObject(ctx context.Context, objectAPI Obj
}
r := bandwidth.NewMonitoredReader(newCtx, globalBucketMonitor, gr, opts)
if objInfo.isMultipart() {
if err = replicateObjectWithMultipart(ctx, c, tgt.Bucket, object,
r, objInfo, putOpts); err != nil {
if minio.ToErrorResponse(err).Code != "PreconditionFailed" {
if rinfo.Err = replicateObjectWithMultipart(ctx, c, tgt.Bucket, object,
r, objInfo, putOpts); rinfo.Err != nil {
if minio.ToErrorResponse(rinfo.Err).Code != "PreconditionFailed" {
rinfo.ReplicationStatus = replication.Failed
logger.LogIf(ctx, fmt.Errorf("unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, err))
logger.LogIf(ctx, fmt.Errorf("unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, rinfo.Err))
}
}
} else {
if _, err = c.PutObject(ctx, tgt.Bucket, object, r, size, "", "", putOpts); err != nil {
if minio.ToErrorResponse(err).Code != "PreconditionFailed" {
if _, rinfo.Err = c.PutObject(ctx, tgt.Bucket, object, r, size, "", "", putOpts); rinfo.Err != nil {
if minio.ToErrorResponse(rinfo.Err).Code != "PreconditionFailed" {
rinfo.ReplicationStatus = replication.Failed
logger.LogIf(ctx, fmt.Errorf("unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, err))
logger.LogIf(ctx, fmt.Errorf("unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, rinfo.Err))
}
}
}
if err != nil && minio.IsNetworkOrHostDown(err, true) && !globalBucketTargetSys.isOffline(tgt.EndpointURL()) {
if rinfo.Err != nil && minio.IsNetworkOrHostDown(rinfo.Err, true) && !globalBucketTargetSys.isOffline(tgt.EndpointURL()) {
globalBucketTargetSys.markOffline(tgt.EndpointURL())
}
return
@ -1266,6 +1272,8 @@ func (ri ReplicateObjectInfo) replicateAll(ctx context.Context, objectAPI Object
ReplicationStatus: replication.Failed,
OpType: ri.OpType,
ReplicationAction: rAction,
endpoint: tgt.EndpointURL().Host,
secure: tgt.EndpointURL().Scheme == "https",
}
if globalBucketTargetSys.isOffline(tgt.EndpointURL()) {
@ -1393,6 +1401,7 @@ func (ri ReplicateObjectInfo) replicateAll(ctx context.Context, objectAPI Object
case "NoSuchKey", "NoSuchVersion", "SlowDownRead":
rAction = replicateAll
default:
rinfo.Err = cerr
logger.LogIf(ctx, fmt.Errorf("unable to replicate %s/%s (%s). Target (%s) returned %s error on HEAD",
bucket, object, objInfo.VersionID, tgt.EndpointURL(), cerr))
sendEvent(eventArgs{
@ -1423,9 +1432,9 @@ func (ri ReplicateObjectInfo) replicateAll(ctx context.Context, objectAPI Object
ReplicationRequest: true, // always set this to distinguish between `mc mirror` replication and serverside
},
}
if _, err = c.CopyObject(ctx, tgt.Bucket, object, tgt.Bucket, object, getCopyObjMetadata(objInfo, tgt.StorageClass), srcOpts, dstOpts); err != nil {
if _, rinfo.Err = c.CopyObject(ctx, tgt.Bucket, object, tgt.Bucket, object, getCopyObjMetadata(objInfo, tgt.StorageClass), srcOpts, dstOpts); rinfo.Err != nil {
rinfo.ReplicationStatus = replication.Failed
logger.LogIf(ctx, fmt.Errorf("unable to replicate metadata for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, err))
logger.LogIf(ctx, fmt.Errorf("unable to replicate metadata for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, rinfo.Err))
}
} else {
var putOpts minio.PutObjectOptions
@ -1459,26 +1468,26 @@ func (ri ReplicateObjectInfo) replicateAll(ctx context.Context, objectAPI Object
}
r := bandwidth.NewMonitoredReader(newCtx, globalBucketMonitor, gr, opts)
if objInfo.isMultipart() {
if err = replicateObjectWithMultipart(ctx, c, tgt.Bucket, object,
r, objInfo, putOpts); err != nil {
if minio.ToErrorResponse(err).Code != "PreconditionFailed" {
if rinfo.Err = replicateObjectWithMultipart(ctx, c, tgt.Bucket, object,
r, objInfo, putOpts); rinfo.Err != nil {
if minio.ToErrorResponse(rinfo.Err).Code != "PreconditionFailed" {
rinfo.ReplicationStatus = replication.Failed
logger.LogIf(ctx, fmt.Errorf("unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, err))
logger.LogIf(ctx, fmt.Errorf("unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, rinfo.Err))
} else {
rinfo.ReplicationStatus = replication.Completed
}
}
} else {
if _, err = c.PutObject(ctx, tgt.Bucket, object, r, size, "", "", putOpts); err != nil {
if minio.ToErrorResponse(err).Code != "PreconditionFailed" {
if _, rinfo.Err = c.PutObject(ctx, tgt.Bucket, object, r, size, "", "", putOpts); rinfo.Err != nil {
if minio.ToErrorResponse(rinfo.Err).Code != "PreconditionFailed" {
rinfo.ReplicationStatus = replication.Failed
logger.LogIf(ctx, fmt.Errorf("unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, err))
logger.LogIf(ctx, fmt.Errorf("unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, rinfo.Err))
} else {
rinfo.ReplicationStatus = replication.Completed
}
}
}
if err != nil && minio.IsNetworkOrHostDown(err, true) && !globalBucketTargetSys.isOffline(tgt.EndpointURL()) {
if rinfo.Err != nil && minio.IsNetworkOrHostDown(rinfo.Err, true) && !globalBucketTargetSys.isOffline(tgt.EndpointURL()) {
globalBucketTargetSys.markOffline(tgt.EndpointURL())
}
}
@ -1730,7 +1739,6 @@ func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPool
go pool.resyncer.PersistToDisk(ctx, o)
go pool.processMRF()
go pool.persistMRF()
go pool.saveStatsToDisk()
return pool
}
@ -1747,9 +1755,12 @@ func (p *ReplicationPool) AddMRFWorker() {
}
switch v := oi.(type) {
case ReplicateObjectInfo:
globalReplicationStats.incQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
atomic.AddInt32(&p.activeMRFWorkers, 1)
replicateObject(p.ctx, v, p.objLayer)
atomic.AddInt32(&p.activeMRFWorkers, -1)
globalReplicationStats.decQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
default:
logger.LogOnceIf(p.ctx, fmt.Errorf("unknown mrf replication type: %T", oi), "unknown-mrf-replicate-type")
}
@ -1776,7 +1787,9 @@ func (p *ReplicationPool) AddWorker(input <-chan ReplicationWorkerOperation, opT
if opTracker != nil {
atomic.AddInt32(opTracker, 1)
}
globalReplicationStats.incQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
replicateObject(p.ctx, v, p.objLayer)
globalReplicationStats.decQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
if opTracker != nil {
atomic.AddInt32(opTracker, -1)
}
@ -1784,7 +1797,11 @@ func (p *ReplicationPool) AddWorker(input <-chan ReplicationWorkerOperation, opT
if opTracker != nil {
atomic.AddInt32(opTracker, 1)
}
globalReplicationStats.incQ(v.Bucket, 0, true, v.OpType)
replicateDelete(p.ctx, v, p.objLayer)
globalReplicationStats.decQ(v.Bucket, 0, true, v.OpType)
if opTracker != nil {
atomic.AddInt32(opTracker, -1)
}
@ -1822,7 +1839,9 @@ func (p *ReplicationPool) AddLargeWorker(input <-chan ReplicationWorkerOperation
}
switch v := oi.(type) {
case ReplicateObjectInfo:
globalReplicationStats.incQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
replicateObject(p.ctx, v, p.objLayer)
globalReplicationStats.decQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
case DeletedObjectReplicationInfo:
replicateDelete(p.ctx, v, p.objLayer)
default:
@ -2046,7 +2065,7 @@ func initBackgroundReplication(ctx context.Context, objectAPI ObjectLayer) {
Priority: globalAPIConfig.getReplicationPriority(),
})
globalReplicationStats = NewReplicationStats(ctx, objectAPI)
go globalReplicationStats.loadInitialReplicationMetrics(ctx)
go globalReplicationStats.trackEWMA()
}
type proxyResult struct {
@ -2136,7 +2155,7 @@ func proxyHeadToRepTarget(ctx context.Context, bucket, object string, rs *HTTPRa
return nil, oi, proxy
}
for _, t := range proxyTargets.Targets {
tgt = globalBucketTargetSys.GetRemoteTargetClient(ctx, t.Arn)
tgt = globalBucketTargetSys.GetRemoteTargetClient(t.Arn)
if tgt == nil || globalBucketTargetSys.isOffline(tgt.EndpointURL()) {
continue
}
@ -2217,20 +2236,12 @@ func scheduleReplication(ctx context.Context, objInfo ObjectInfo, o ObjectLayer,
} else {
globalReplicationPool.queueReplicaTask(ri)
}
if sz, err := objInfo.GetActualSize(); err == nil {
for arn := range dsc.targetsMap {
globalReplicationStats.Update(objInfo.Bucket, arn, sz, 0, objInfo.ReplicationStatus, replication.StatusType(""), opType)
}
}
}
func scheduleReplicationDelete(ctx context.Context, dv DeletedObjectReplicationInfo, o ObjectLayer) {
globalReplicationPool.queueReplicaDeleteTask(dv)
for arn := range dv.ReplicationState.Targets {
globalReplicationStats.Update(dv.Bucket, arn, 0, 0, replication.Pending, replication.StatusType(""), replication.DeleteReplicationType)
}
for arn := range dv.ReplicationState.PurgeTargets {
globalReplicationStats.Update(dv.Bucket, arn, 0, 0, replication.Pending, replication.StatusType(""), replication.DeleteReplicationType)
globalReplicationStats.Update(dv.Bucket, replicatedTargetInfo{Arn: arn, Size: 0, Duration: 0, OpType: replication.DeleteReplicationType}, replication.Pending, replication.StatusType(""))
}
}
@ -2488,7 +2499,7 @@ func (s *replicationResyncer) resyncBucket(ctx context.Context, objectAPI Object
logger.LogIf(ctx, fmt.Errorf("replication resync failed for %s - arn specified %s is missing in the replication config", opts.bucket, opts.arn))
return
}
tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, opts.arn)
tgt := globalBucketTargetSys.GetRemoteTargetClient(opts.arn)
if tgt == nil {
logger.LogIf(ctx, fmt.Errorf("replication resync failed for %s - target could not be created for arn %s", opts.bucket, opts.arn))
return
@ -3102,9 +3113,12 @@ func (p *ReplicationPool) queueMRFSave(entry MRFReplicateEntry) {
if !p.initialized() {
return
}
if entry.RetryCount > mrfRetryLimit {
if entry.RetryCount > mrfRetryLimit { // let scanner catch up if retry count exceeded
atomic.AddUint64(&globalReplicationStats.mrfStats.TotalDroppedCount, 1)
atomic.AddUint64(&globalReplicationStats.mrfStats.TotalDroppedBytes, uint64(entry.sz))
return
}
select {
case <-GlobalContext.Done():
return
@ -3123,7 +3137,7 @@ func (p *ReplicationPool) saveMRFEntries(ctx context.Context, entries map[string
if !p.initialized() {
return nil
}
atomic.StoreUint64(&globalReplicationStats.mrfStats.LastFailedCount, uint64(len(entries)))
if len(entries) == 0 {
return nil
}
@ -3263,87 +3277,10 @@ func (p *ReplicationPool) queueMRFHeal() error {
return nil
}
// load replication stats from disk
func (p *ReplicationPool) loadStatsFromDisk() (rs map[string]BucketReplicationStats, e error) {
if !p.initialized() {
return map[string]BucketReplicationStats{}, nil
}
data, err := readConfig(p.ctx, p.objLayer, getReplicationStatsPath())
if err != nil {
if errors.Is(err, errConfigNotFound) {
return rs, nil
}
return rs, err
}
if len(data) <= 4 {
logger.LogIf(p.ctx, fmt.Errorf("replication stats: no data"))
return map[string]BucketReplicationStats{}, nil
}
// Read repl stats meta header
switch binary.LittleEndian.Uint16(data[0:2]) {
case replStatsMetaFormat:
default:
return rs, fmt.Errorf("replication stats: unknown format: %d", binary.LittleEndian.Uint16(data[0:2]))
}
switch binary.LittleEndian.Uint16(data[2:4]) {
case replStatsVersion:
default:
return rs, fmt.Errorf("replication stats: unknown version: %d", binary.LittleEndian.Uint16(data[2:4]))
}
ss := BucketStatsMap{}
if _, err = ss.UnmarshalMsg(data[4:]); err != nil {
return rs, err
}
rs = make(map[string]BucketReplicationStats, len(ss.Stats))
for bucket, st := range ss.Stats {
rs[bucket] = st.ReplicationStats
}
return rs, nil
}
func (p *ReplicationPool) initialized() bool {
return !(p == nil || p.objLayer == nil)
}
func (p *ReplicationPool) saveStatsToDisk() {
if !p.initialized() {
return
}
ctx, cancel := globalLeaderLock.GetLock(p.ctx)
defer cancel()
sTimer := time.NewTimer(replStatsSaveInterval)
defer sTimer.Stop()
for {
select {
case <-sTimer.C:
dui, err := loadDataUsageFromBackend(GlobalContext, newObjectLayerFn())
if err == nil && !dui.LastUpdate.IsZero() {
globalReplicationStats.getAllLatest(dui.BucketsUsage)
}
p.saveStats(p.ctx)
sTimer.Reset(replStatsSaveInterval)
case <-ctx.Done():
return
}
}
}
// save replication stats to .minio.sys/buckets/replication/node-name.stats
func (p *ReplicationPool) saveStats(ctx context.Context) error {
if !p.initialized() {
return nil
}
data, err := globalReplicationStats.serializeStats()
if data == nil {
return err
}
return saveConfig(ctx, p.objLayer, getReplicationStatsPath(), data)
}
// getMRF returns MRF entries for this node.
func (p *ReplicationPool) getMRF(ctx context.Context, bucket string) (ch chan madmin.ReplicationMRF, err error) {
mrfCh := make(chan madmin.ReplicationMRF, 100)

View File

@ -19,7 +19,10 @@ package cmd
import (
"fmt"
"math"
"time"
"github.com/minio/madmin-go/v3"
)
//go:generate msgp -file $GOFILE
@ -52,6 +55,94 @@ func (rl *ReplicationLatency) update(size int64, duration time.Duration) {
rl.UploadHistogram.Add(size, duration)
}
// ReplicationLastMinute has last minute replication counters
type ReplicationLastMinute struct {
LastMinute lastMinuteLatency
}
func (rl ReplicationLastMinute) merge(other ReplicationLastMinute) (nl ReplicationLastMinute) {
nl = ReplicationLastMinute{rl.LastMinute.merge(other.LastMinute)}
return
}
func (rl *ReplicationLastMinute) addsize(n int64) {
t := time.Now().Unix()
rl.LastMinute.addAll(t-1, AccElem{Total: t - 1, Size: n, N: 1})
}
func (rl *ReplicationLastMinute) String() string {
t := rl.LastMinute.getTotal()
return fmt.Sprintf("ReplicationLastMinute sz= %d, n=%d , dur=%d", t.Size, t.N, t.Total)
}
func (rl *ReplicationLastMinute) getTotal() AccElem {
return rl.LastMinute.getTotal()
}
// ReplicationLastHour keeps track of replication counts over the last hour
type ReplicationLastHour struct {
Totals [60]AccElem
LastMin int64
}
// Merge data of two ReplicationLastHour structure
func (l ReplicationLastHour) merge(o ReplicationLastHour) (merged ReplicationLastHour) {
if l.LastMin > o.LastMin {
o.forwardTo(l.LastMin)
merged.LastMin = l.LastMin
} else {
l.forwardTo(o.LastMin)
merged.LastMin = o.LastMin
}
for i := range merged.Totals {
merged.Totals[i] = AccElem{
Total: l.Totals[i].Total + o.Totals[i].Total,
N: l.Totals[i].N + o.Totals[i].N,
Size: l.Totals[i].Size + o.Totals[i].Size,
}
}
return merged
}
// Add a new duration data
func (l *ReplicationLastHour) addsize(sz int64) {
min := time.Now().Unix() / 60
l.forwardTo(min)
winIdx := min % 60
l.Totals[winIdx].merge(AccElem{Total: min, Size: sz, N: 1})
l.LastMin = min
}
// Merge all recorded counts of last hour into one
func (l *ReplicationLastHour) getTotal() AccElem {
var res AccElem
min := time.Now().Unix() / 60
l.forwardTo(min)
for _, elem := range l.Totals[:] {
res.merge(elem)
}
return res
}
// forwardTo time t, clearing any entries in between.
func (l *ReplicationLastHour) forwardTo(t int64) {
tMin := t / 60
if l.LastMin >= tMin {
return
}
if t-l.LastMin >= 60 {
l.Totals = [60]AccElem{}
return
}
for l.LastMin != t {
// Clear next element.
idx := (l.LastMin + 1) % 60
l.Totals[idx] = AccElem{}
l.LastMin++
}
}
// BucketStatsMap captures bucket statistics for all buckets
type BucketStatsMap struct {
Stats map[string]BucketStats
@ -60,19 +151,32 @@ type BucketStatsMap struct {
// BucketStats bucket statistics
type BucketStats struct {
ReplicationStats BucketReplicationStats
Uptime int64 `json:"uptime"`
ReplicationStats BucketReplicationStats `json:"currStats"` // current replication stats since cluster startup
QueueStats ReplicationQueueStats `json:"queueStats"` // replication queue stats
}
// BucketReplicationStats represents inline replication statistics
// such as pending, failed and completed bytes in total for a bucket
type BucketReplicationStats struct {
Stats map[string]*BucketReplicationStat `json:",omitempty"`
// Pending size in bytes
PendingSize int64 `json:"pendingReplicationSize"`
// Completed size in bytes
ReplicatedSize int64 `json:"completedReplicationSize"`
// Total Replica size in bytes
ReplicaSize int64 `json:"replicaSize"`
// Total failed operations including metadata updates for various time frames
Failed madmin.TimedErrStats `json:"failed"`
// Total number of completed operations
ReplicatedCount int64 `json:"replicationCount"`
// Total number of replica received
ReplicaCount int64 `json:"replicaCount"`
// in Queue stats for bucket - from qCache
QStat InQueueMetric `json:"queued"`
// Deprecated fields
// Pending size in bytes
PendingSize int64 `json:"pendingReplicationSize"`
// Failed size in bytes
FailedSize int64 `json:"failedReplicationSize"`
// Total number of pending operations including metadata updates
@ -81,6 +185,12 @@ type BucketReplicationStats struct {
FailedCount int64 `json:"failedReplicationCount"`
}
func newBucketReplicationStats() *BucketReplicationStats {
return &BucketReplicationStats{
Stats: make(map[string]*BucketReplicationStat),
}
}
// Empty returns true if there are no target stats
func (brs *BucketReplicationStats) Empty() bool {
return len(brs.Stats) == 0 && brs.ReplicaSize == 0
@ -96,7 +206,24 @@ func (brs BucketReplicationStats) Clone() (c BucketReplicationStats) {
c.Stats = make(map[string]*BucketReplicationStat, len(brs.Stats))
for arn, st := range brs.Stats {
// make a copy of `*st`
s := *st
s := BucketReplicationStat{
ReplicatedSize: st.ReplicatedSize,
ReplicaSize: st.ReplicaSize,
Latency: st.Latency,
BandWidthLimitInBytesPerSecond: st.BandWidthLimitInBytesPerSecond,
CurrentBandwidthInBytesPerSecond: st.CurrentBandwidthInBytesPerSecond,
XferRateLrg: st.XferRateLrg.Clone(),
XferRateSml: st.XferRateSml.Clone(),
ReplicatedCount: st.ReplicatedCount,
Failed: st.Failed,
FailStats: st.FailStats,
}
if s.Failed.ErrCounts == nil {
s.Failed.ErrCounts = make(map[string]int)
for k, v := range st.Failed.ErrCounts {
s.Failed.ErrCounts[k] = v
}
}
c.Stats[arn] = &s
}
return c
@ -107,38 +234,189 @@ func (brs BucketReplicationStats) Clone() (c BucketReplicationStats) {
// remote target
type BucketReplicationStat struct {
// Pending size in bytes
PendingSize int64 `json:"pendingReplicationSize"`
// PendingSize int64 `json:"pendingReplicationSize"`
// Completed size in bytes
ReplicatedSize int64 `json:"completedReplicationSize"`
// Total Replica size in bytes
ReplicaSize int64 `json:"replicaSize"`
// Failed size in bytes
FailedSize int64 `json:"failedReplicationSize"`
// Total number of pending operations including metadata updates
PendingCount int64 `json:"pendingReplicationCount"`
// Total number of failed operations including metadata updates
FailedCount int64 `json:"failedReplicationCount"`
// Collect stats for failures
FailStats RTimedMetrics `json:"-"`
// Total number of failed operations including metadata updates in the last minute
Failed madmin.TimedErrStats `json:"failed"`
// Total number of completed operations
ReplicatedCount int64 `json:"replicationCount"`
// Replication latency information
Latency ReplicationLatency `json:"replicationLatency"`
// bandwidth limit for target
BandWidthLimitInBytesPerSecond int64 `json:"limitInBits"`
// current bandwidth reported
CurrentBandwidthInBytesPerSecond float64 `json:"currentBandwidth"`
// transfer rate for large uploads
XferRateLrg *XferStats `json:"-" msg:"lt"`
// transfer rate for small uploads
XferRateSml *XferStats `json:"-" msg:"st"`
// Deprecated fields
// Pending size in bytes
PendingSize int64 `json:"pendingReplicationSize"`
// Failed size in bytes
FailedSize int64 `json:"failedReplicationSize"`
// Total number of pending operations including metadata updates
PendingCount int64 `json:"pendingReplicationCount"`
// Total number of failed operations including metadata updates
FailedCount int64 `json:"failedReplicationCount"`
}
func (bs *BucketReplicationStat) hasReplicationUsage() bool {
return bs.FailedSize > 0 ||
return bs.FailStats.SinceUptime.Count > 0 ||
bs.ReplicatedSize > 0 ||
bs.ReplicaSize > 0 ||
bs.FailedCount > 0 ||
bs.PendingCount > 0 ||
bs.PendingSize > 0
bs.ReplicaSize > 0
}
func (brs BucketReplicationStats) String() string {
s := "ReplicatedSize=" + fmt.Sprintf("%d", brs.ReplicatedSize) + "+\n ReplicaSize=" + fmt.Sprintf("%d", brs.ReplicaSize)
for arn, st := range brs.Stats {
s += "\n arn: " + arn + " ReplicatedSize=" + fmt.Sprintf("%d", st.ReplicatedSize) + " +::ReplicaSize=" + fmt.Sprintf("%d", st.ReplicaSize)
func (bs *BucketReplicationStat) updateXferRate(sz int64, duration time.Duration) {
if sz > minLargeObjSize {
bs.XferRateLrg.addSize(sz, duration)
} else {
bs.XferRateSml.addSize(sz, duration)
}
return s
}
// RMetricName - name of replication metric
type RMetricName string
const (
// Large - objects larger than 128MiB
Large RMetricName = "Large"
// Small - objects smaller than 128MiB
Small RMetricName = "Small"
// Total - metric pertaining to totals
Total RMetricName = "Total"
)
// ReplQNodeStats holds queue stats for replication per node
type ReplQNodeStats struct {
NodeName string `json:"nodeName"`
Uptime int64 `json:"uptime"`
ActiveWorkers ActiveWorkerStat `json:"activeWorkers"`
XferStats map[RMetricName]XferStats `json:"transferSummary"`
TgtXferStats map[string]map[RMetricName]XferStats `json:"tgtTransferStats"`
QStats InQueueMetric `json:"queueStats"`
MRFStats ReplicationMRFStats `json:"mrfStats"`
}
// getNodeQueueStats returns replication operational stats at the node level
func (r *ReplicationStats) getNodeQueueStats(bucket string) (qs ReplQNodeStats) {
qs.NodeName = globalLocalNodeName
qs.Uptime = UTCNow().Unix() - globalBootTime.Unix()
qs.ActiveWorkers = globalReplicationStats.ActiveWorkers()
qs.XferStats = make(map[RMetricName]XferStats)
qs.QStats = r.qCache.getBucketStats(bucket)
qs.TgtXferStats = make(map[string]map[RMetricName]XferStats)
r.RLock()
defer r.RUnlock()
brs, ok := r.Cache[bucket]
if !ok {
return qs
}
for arn := range brs.Stats {
qs.TgtXferStats[arn] = make(map[RMetricName]XferStats)
}
count := 0
var totPeak float64
// calculate large, small transfers and total transfer rates per replication target at bucket level
for arn, v := range brs.Stats {
lcurrTgt := v.XferRateLrg.curr()
scurrTgt := v.XferRateSml.curr()
totPeak = math.Max(math.Max(v.XferRateLrg.Peak, v.XferRateSml.Peak), totPeak)
totPeak = math.Max(math.Max(lcurrTgt, scurrTgt), totPeak)
tcount := 0
if v.XferRateLrg.Peak > 0 {
tcount++
}
if v.XferRateSml.Peak > 0 {
tcount++
}
qs.TgtXferStats[arn][Large] = XferStats{
Avg: v.XferRateLrg.Avg,
Curr: lcurrTgt,
Peak: math.Max(v.XferRateLrg.Peak, lcurrTgt),
}
qs.TgtXferStats[arn][Small] = XferStats{
Avg: v.XferRateSml.Avg,
Curr: scurrTgt,
Peak: math.Max(v.XferRateSml.Peak, scurrTgt),
}
if tcount > 0 {
qs.TgtXferStats[arn][Total] = XferStats{
Avg: (v.XferRateLrg.Avg + v.XferRateSml.Avg) / float64(tcount),
Curr: (scurrTgt + lcurrTgt) / float64(tcount),
Peak: totPeak,
}
}
}
// calculate large, small and total transfer rates for a minio node
var lavg, lcurr, lpeak, savg, scurr, speak, totpeak float64
for _, v := range qs.TgtXferStats {
tot := v[Total]
lavg += v[Large].Avg
lcurr += v[Large].Curr
savg += v[Small].Avg
scurr += v[Small].Curr
totpeak = math.Max(math.Max(tot.Peak, totpeak), tot.Curr)
lpeak = math.Max(math.Max(v[Large].Peak, lpeak), v[Large].Curr)
speak = math.Max(math.Max(v[Small].Peak, speak), v[Small].Curr)
if lpeak > 0 || speak > 0 {
count++
}
}
if count > 0 {
lrg := XferStats{
Avg: lavg / float64(count),
Curr: lcurr / float64(count),
Peak: lpeak,
}
sml := XferStats{
Avg: savg / float64(count),
Curr: scurr / float64(count),
Peak: speak,
}
qs.XferStats[Large] = lrg
qs.XferStats[Small] = sml
qs.XferStats[Total] = XferStats{
Avg: (savg + lavg) / float64(count),
Curr: (lcurr + scurr) / float64(count),
Peak: totpeak,
}
}
return qs
}
// populate queue totals for node and active workers in use for metrics
func (r *ReplicationStats) getNodeQueueStatsSummary() (qs ReplQNodeStats) {
qs.NodeName = globalLocalNodeName
qs.Uptime = UTCNow().Unix() - globalBootTime.Unix()
qs.ActiveWorkers = globalReplicationStats.ActiveWorkers()
qs.XferStats = make(map[RMetricName]XferStats)
qs.QStats = r.qCache.getSiteStats()
r.RLock()
defer r.RUnlock()
tx := newXferStats()
for _, brs := range r.Cache {
for _, v := range brs.Stats {
tx := tx.merge(*v.XferRateLrg)
tx = tx.merge(*v.XferRateSml)
}
}
qs.XferStats[Total] = *tx
return qs
}
// ReplicationQueueStats holds overall queue stats for replication
type ReplicationQueueStats struct {
Nodes []ReplQNodeStats `json:"nodes"`
Uptime int64 `json:"uptime"`
}

File diff suppressed because it is too large Load Diff

View File

@ -461,6 +461,345 @@ func BenchmarkDecodeBucketStatsMap(b *testing.B) {
}
}
func TestMarshalUnmarshalReplQNodeStats(t *testing.T) {
v := ReplQNodeStats{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgReplQNodeStats(b *testing.B) {
v := ReplQNodeStats{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgReplQNodeStats(b *testing.B) {
v := ReplQNodeStats{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalReplQNodeStats(b *testing.B) {
v := ReplQNodeStats{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeReplQNodeStats(t *testing.T) {
v := ReplQNodeStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeReplQNodeStats Msgsize() is inaccurate")
}
vn := ReplQNodeStats{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeReplQNodeStats(b *testing.B) {
v := ReplQNodeStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeReplQNodeStats(b *testing.B) {
v := ReplQNodeStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalReplicationLastHour(t *testing.T) {
v := ReplicationLastHour{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgReplicationLastHour(b *testing.B) {
v := ReplicationLastHour{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgReplicationLastHour(b *testing.B) {
v := ReplicationLastHour{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalReplicationLastHour(b *testing.B) {
v := ReplicationLastHour{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeReplicationLastHour(t *testing.T) {
v := ReplicationLastHour{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeReplicationLastHour Msgsize() is inaccurate")
}
vn := ReplicationLastHour{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeReplicationLastHour(b *testing.B) {
v := ReplicationLastHour{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeReplicationLastHour(b *testing.B) {
v := ReplicationLastHour{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalReplicationLastMinute(t *testing.T) {
v := ReplicationLastMinute{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgReplicationLastMinute(b *testing.B) {
v := ReplicationLastMinute{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgReplicationLastMinute(b *testing.B) {
v := ReplicationLastMinute{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalReplicationLastMinute(b *testing.B) {
v := ReplicationLastMinute{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeReplicationLastMinute(t *testing.T) {
v := ReplicationLastMinute{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeReplicationLastMinute Msgsize() is inaccurate")
}
vn := ReplicationLastMinute{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeReplicationLastMinute(b *testing.B) {
v := ReplicationLastMinute{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeReplicationLastMinute(b *testing.B) {
v := ReplicationLastMinute{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalReplicationLatency(t *testing.T) {
v := ReplicationLatency{}
bts, err := v.MarshalMsg(nil)
@ -573,3 +912,116 @@ func BenchmarkDecodeReplicationLatency(b *testing.B) {
}
}
}
func TestMarshalUnmarshalReplicationQueueStats(t *testing.T) {
v := ReplicationQueueStats{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgReplicationQueueStats(b *testing.B) {
v := ReplicationQueueStats{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgReplicationQueueStats(b *testing.B) {
v := ReplicationQueueStats{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalReplicationQueueStats(b *testing.B) {
v := ReplicationQueueStats{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeReplicationQueueStats(t *testing.T) {
v := ReplicationQueueStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeReplicationQueueStats Msgsize() is inaccurate")
}
vn := ReplicationQueueStats{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeReplicationQueueStats(b *testing.B) {
v := ReplicationQueueStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeReplicationQueueStats(b *testing.B) {
v := ReplicationQueueStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}

View File

@ -50,11 +50,33 @@ type BucketTargetSys struct {
hcClient *madmin.AnonymousClient
}
type latencyStat struct {
lastmin lastMinuteLatency
curr time.Duration
avg time.Duration
peak time.Duration
N int64
}
func (l *latencyStat) update(d time.Duration) {
l.lastmin.add(d)
l.N++
if d > l.peak {
l.peak = d
}
l.curr = l.lastmin.getTotal().avg()
l.avg = time.Duration((int64(l.avg)*(l.N-1) + int64(l.curr)) / l.N)
}
// epHealth struct represents health of a replication target endpoint.
type epHealth struct {
Endpoint string
Scheme string
Online bool
Endpoint string
Scheme string
Online bool
lastOnline time.Time
lastHCAt time.Time
offlineDuration time.Duration
latency latencyStat
}
// isOffline returns current liveness result of remote target. Add endpoint to
@ -117,11 +139,40 @@ func (sys *BucketTargetSys) heartBeat(ctx context.Context) {
if len(eps) > 0 {
cctx, cancel := context.WithTimeout(ctx, 30*time.Second)
m := make(map[string]epHealth, len(eps))
start := time.Now()
for result := range sys.hcClient.Alive(cctx, madmin.AliveOpts{}, eps...) {
var lastOnline time.Time
var offline time.Duration
// var deploymentID string
sys.hMutex.RLock()
prev, ok := sys.hc[result.Endpoint.Host]
sys.hMutex.RUnlock()
if ok {
if prev.Online != result.Online || !result.Online {
if !prev.lastHCAt.IsZero() {
offline = time.Since(prev.lastHCAt) + prev.offlineDuration
} else {
offline = prev.offlineDuration
}
} else if result.Online {
offline = prev.offlineDuration
}
}
lastOnline = prev.lastOnline
if result.Online {
lastOnline = time.Now()
}
l := prev.latency
l.update(time.Since(start))
m[result.Endpoint.Host] = epHealth{
Endpoint: result.Endpoint.Host,
Scheme: result.Endpoint.Scheme,
Online: result.Online,
Endpoint: result.Endpoint.Host,
Scheme: result.Endpoint.Scheme,
Online: result.Online,
lastOnline: lastOnline,
offlineDuration: offline,
lastHCAt: time.Now(),
latency: l,
}
}
cancel()
@ -141,32 +192,61 @@ func (sys *BucketTargetSys) heartBeat(ctx context.Context) {
func (sys *BucketTargetSys) reloadHealthCheckers(ctx context.Context) {
m := make(map[string]epHealth)
tgts := sys.ListTargets(ctx, "", "")
sys.hMutex.Lock()
for _, t := range tgts {
if _, ok := m[t.Endpoint]; !ok {
scheme := "http"
if t.Secure {
scheme = "https"
}
m[t.Endpoint] = epHealth{
epHealth := epHealth{
Online: true,
Endpoint: t.Endpoint,
Scheme: scheme,
}
if prev, ok := sys.hc[t.Endpoint]; ok {
epHealth.lastOnline = prev.lastOnline
epHealth.offlineDuration = prev.offlineDuration
epHealth.lastHCAt = prev.lastHCAt
epHealth.latency = prev.latency
}
m[t.Endpoint] = epHealth
}
}
sys.hMutex.Lock()
// swap out the map
sys.hc = m
sys.hMutex.Unlock()
}
func (sys *BucketTargetSys) healthStats() map[string]epHealth {
sys.hMutex.RLock()
defer sys.hMutex.RUnlock()
m := make(map[string]epHealth, len(sys.hc))
for k, v := range sys.hc {
m[k] = v
}
return m
}
// ListTargets lists bucket targets across tenant or for individual bucket, and returns
// results filtered by arnType
func (sys *BucketTargetSys) ListTargets(ctx context.Context, bucket, arnType string) (targets []madmin.BucketTarget) {
h := sys.healthStats()
if bucket != "" {
if ts, err := sys.ListBucketTargets(ctx, bucket); err == nil {
for _, t := range ts.Targets {
if string(t.Type) == arnType || arnType == "" {
if hs, ok := h[t.URL().Host]; ok {
t.TotalDowntime = hs.offlineDuration
t.Online = hs.Online
t.LastOnline = hs.lastOnline
t.Latency = madmin.LatencyStat{
Curr: hs.latency.curr,
Avg: hs.latency.avg,
Max: hs.latency.peak,
}
}
targets = append(targets, t.Clone())
}
}
@ -178,6 +258,16 @@ func (sys *BucketTargetSys) ListTargets(ctx context.Context, bucket, arnType str
for _, tgts := range sys.targetsMap {
for _, t := range tgts {
if string(t.Type) == arnType || arnType == "" {
if hs, ok := h[t.URL().Host]; ok {
t.TotalDowntime = hs.offlineDuration
t.Online = hs.Online
t.LastOnline = hs.lastOnline
t.Latency = madmin.LatencyStat{
Curr: hs.latency.curr,
Avg: hs.latency.avg,
Max: hs.latency.peak,
}
}
targets = append(targets, t.Clone())
}
}
@ -343,7 +433,7 @@ func (sys *BucketTargetSys) RemoveTarget(ctx context.Context, bucket, arnStr str
}
// GetRemoteTargetClient returns minio-go client for replication target instance
func (sys *BucketTargetSys) GetRemoteTargetClient(ctx context.Context, arn string) *TargetClient {
func (sys *BucketTargetSys) GetRemoteTargetClient(arn string) *TargetClient {
sys.RLock()
defer sys.RUnlock()
return sys.arnRemotesMap[arn]
@ -489,6 +579,8 @@ func (sys *BucketTargetSys) getRemoteARN(bucket string, target *madmin.BucketTar
if target == nil {
return
}
sys.RLock()
defer sys.RUnlock()
tgts := sys.targetsMap[bucket]
for _, tgt := range tgts {
if tgt.Type == target.Type &&
@ -506,6 +598,8 @@ func (sys *BucketTargetSys) getRemoteARN(bucket string, target *madmin.BucketTar
// getRemoteARNForPeer returns the remote target for a peer site in site replication
func (sys *BucketTargetSys) getRemoteARNForPeer(bucket string, peer madmin.PeerInfo) string {
sys.RLock()
defer sys.RUnlock()
tgts := sys.targetsMap[bucket]
for _, target := range tgts {
ep, _ := url.Parse(peer.Endpoint)

View File

@ -846,9 +846,11 @@ type sizeSummary struct {
versions uint64
deleteMarkers uint64
replicatedSize int64
replicatedCount int64
pendingSize int64
failedSize int64
replicaSize int64
replicaCount int64
pendingCount uint64
failedCount uint64
replTargetStats map[string]replTargetSizeSummary
@ -857,11 +859,12 @@ type sizeSummary struct {
// replTargetSizeSummary holds summary of replication stats by target
type replTargetSizeSummary struct {
replicatedSize int64
pendingSize int64
failedSize int64
pendingCount uint64
failedCount uint64
replicatedSize int64
replicatedCount int64
pendingSize int64
failedSize int64
pendingCount uint64
failedCount uint64
}
type getSizeFn func(item scannerItem) (sizeSummary, error)
@ -1286,13 +1289,16 @@ func (i *scannerItem) healReplication(ctx context.Context, o ObjectLayer, oi Obj
sizeS.failedCount++
case replication.Completed, replication.CompletedLegacy:
tgtSizeS.replicatedSize += oi.Size
tgtSizeS.replicatedCount++
sizeS.replicatedSize += oi.Size
sizeS.replicatedCount++
}
sizeS.replTargetStats[arn] = tgtSizeS
}
if oi.ReplicationStatus == replication.Replica {
sizeS.replicaSize += oi.Size
sizeS.replicaCount++
}
}

View File

@ -164,6 +164,7 @@ type replicationStats struct {
AfterThresholdSize uint64
MissedThresholdCount uint64
AfterThresholdCount uint64
ReplicatedCount uint64
}
func (rs replicationStats) Empty() bool {
@ -173,14 +174,16 @@ func (rs replicationStats) Empty() bool {
}
type replicationAllStats struct {
Targets map[string]replicationStats `msg:"t,omitempty"`
ReplicaSize uint64 `msg:"r,omitempty"`
Targets map[string]replicationStats `msg:"t,omitempty"`
ReplicaSize uint64 `msg:"r,omitempty"`
ReplicaCount uint64 `msg:"rc,omitempty"`
}
//msgp:tuple replicationAllStatsV1
type replicationAllStatsV1 struct {
Targets map[string]replicationStats
ReplicaSize uint64 `msg:"ReplicaSize,omitempty"`
Targets map[string]replicationStats
ReplicaSize uint64 `msg:"ReplicaSize,omitempty"`
ReplicaCount uint64 `msg:"ReplicaCount,omitempty"`
}
// clone creates a deep-copy clone.
@ -341,6 +344,7 @@ func (e *dataUsageEntry) addSizes(summary sizeSummary) {
e.ReplicationStats.Targets = make(map[string]replicationStats)
}
e.ReplicationStats.ReplicaSize += uint64(summary.replicaSize)
e.ReplicationStats.ReplicaCount += uint64(summary.replicaCount)
if summary.replTargetStats != nil {
for arn, st := range summary.replTargetStats {
@ -351,6 +355,7 @@ func (e *dataUsageEntry) addSizes(summary sizeSummary) {
tgtStat.PendingSize += uint64(st.pendingSize)
tgtStat.FailedSize += uint64(st.failedSize)
tgtStat.ReplicatedSize += uint64(st.replicatedSize)
tgtStat.ReplicatedCount += uint64(st.replicatedCount)
tgtStat.FailedCount += st.failedCount
tgtStat.PendingCount += st.pendingCount
e.ReplicationStats.Targets[arn] = tgtStat
@ -377,14 +382,16 @@ func (e *dataUsageEntry) merge(other dataUsageEntry) {
e.ReplicationStats.Targets = make(map[string]replicationStats)
}
e.ReplicationStats.ReplicaSize += other.ReplicationStats.ReplicaSize
e.ReplicationStats.ReplicaCount += other.ReplicationStats.ReplicaCount
for arn, stat := range other.ReplicationStats.Targets {
st := e.ReplicationStats.Targets[arn]
e.ReplicationStats.Targets[arn] = replicationStats{
PendingSize: stat.PendingSize + st.PendingSize,
FailedSize: stat.FailedSize + st.FailedSize,
ReplicatedSize: stat.ReplicatedSize + st.ReplicatedSize,
PendingCount: stat.PendingCount + st.PendingCount,
FailedCount: stat.FailedCount + st.FailedCount,
PendingSize: stat.PendingSize + st.PendingSize,
FailedSize: stat.FailedSize + st.FailedSize,
ReplicatedSize: stat.ReplicatedSize + st.ReplicatedSize,
PendingCount: stat.PendingCount + st.PendingCount,
FailedCount: stat.FailedCount + st.FailedCount,
ReplicatedCount: stat.ReplicatedCount + st.ReplicatedCount,
}
}
}
@ -815,6 +822,8 @@ func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]Bucke
}
if flat.ReplicationStats != nil {
bui.ReplicaSize = flat.ReplicationStats.ReplicaSize
bui.ReplicaCount = flat.ReplicationStats.ReplicaCount
bui.ReplicationInfo = make(map[string]BucketTargetUsageInfo, len(flat.ReplicationStats.Targets))
for arn, stat := range flat.ReplicationStats.Targets {
bui.ReplicationInfo[arn] = BucketTargetUsageInfo{
@ -823,6 +832,7 @@ func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]Bucke
ReplicationFailedSize: stat.FailedSize,
ReplicationPendingCount: stat.PendingCount,
ReplicationFailedCount: stat.FailedCount,
ReplicatedCount: stat.ReplicatedCount,
}
}
}
@ -1236,8 +1246,9 @@ func (d *dataUsageCache) deserialize(r io.Reader) error {
var replicationStats *replicationAllStats
if v.ReplicationStats != nil {
replicationStats = &replicationAllStats{
Targets: v.ReplicationStats.Targets,
ReplicaSize: v.ReplicationStats.ReplicaSize,
Targets: v.ReplicationStats.Targets,
ReplicaSize: v.ReplicationStats.ReplicaSize,
ReplicaCount: v.ReplicationStats.ReplicaCount,
}
}
due := dataUsageEntry{

View File

@ -2790,6 +2790,12 @@ func (z *replicationAllStats) DecodeMsg(dc *msgp.Reader) (err error) {
err = msgp.WrapError(err, "ReplicaSize")
return
}
case "rc":
z.ReplicaCount, err = dc.ReadUint64()
if err != nil {
err = msgp.WrapError(err, "ReplicaCount")
return
}
default:
err = dc.Skip()
if err != nil {
@ -2804,8 +2810,8 @@ func (z *replicationAllStats) DecodeMsg(dc *msgp.Reader) (err error) {
// EncodeMsg implements msgp.Encodable
func (z *replicationAllStats) EncodeMsg(en *msgp.Writer) (err error) {
// omitempty: check for empty values
zb0001Len := uint32(2)
var zb0001Mask uint8 /* 2 bits */
zb0001Len := uint32(3)
var zb0001Mask uint8 /* 3 bits */
_ = zb0001Mask
if z.Targets == nil {
zb0001Len--
@ -2815,6 +2821,10 @@ func (z *replicationAllStats) EncodeMsg(en *msgp.Writer) (err error) {
zb0001Len--
zb0001Mask |= 0x2
}
if z.ReplicaCount == 0 {
zb0001Len--
zb0001Mask |= 0x4
}
// variable map header, size zb0001Len
err = en.Append(0x80 | uint8(zb0001Len))
if err != nil {
@ -2859,6 +2869,18 @@ func (z *replicationAllStats) EncodeMsg(en *msgp.Writer) (err error) {
return
}
}
if (zb0001Mask & 0x4) == 0 { // if not empty
// write "rc"
err = en.Append(0xa2, 0x72, 0x63)
if err != nil {
return
}
err = en.WriteUint64(z.ReplicaCount)
if err != nil {
err = msgp.WrapError(err, "ReplicaCount")
return
}
}
return
}
@ -2866,8 +2888,8 @@ func (z *replicationAllStats) EncodeMsg(en *msgp.Writer) (err error) {
func (z *replicationAllStats) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.Require(b, z.Msgsize())
// omitempty: check for empty values
zb0001Len := uint32(2)
var zb0001Mask uint8 /* 2 bits */
zb0001Len := uint32(3)
var zb0001Mask uint8 /* 3 bits */
_ = zb0001Mask
if z.Targets == nil {
zb0001Len--
@ -2877,6 +2899,10 @@ func (z *replicationAllStats) MarshalMsg(b []byte) (o []byte, err error) {
zb0001Len--
zb0001Mask |= 0x2
}
if z.ReplicaCount == 0 {
zb0001Len--
zb0001Mask |= 0x4
}
// variable map header, size zb0001Len
o = append(o, 0x80|uint8(zb0001Len))
if zb0001Len == 0 {
@ -2900,6 +2926,11 @@ func (z *replicationAllStats) MarshalMsg(b []byte) (o []byte, err error) {
o = append(o, 0xa1, 0x72)
o = msgp.AppendUint64(o, z.ReplicaSize)
}
if (zb0001Mask & 0x4) == 0 { // if not empty
// string "rc"
o = append(o, 0xa2, 0x72, 0x63)
o = msgp.AppendUint64(o, z.ReplicaCount)
}
return
}
@ -2957,6 +2988,12 @@ func (z *replicationAllStats) UnmarshalMsg(bts []byte) (o []byte, err error) {
err = msgp.WrapError(err, "ReplicaSize")
return
}
case "rc":
z.ReplicaCount, bts, err = msgp.ReadUint64Bytes(bts)
if err != nil {
err = msgp.WrapError(err, "ReplicaCount")
return
}
default:
bts, err = msgp.Skip(bts)
if err != nil {
@ -2978,7 +3015,7 @@ func (z *replicationAllStats) Msgsize() (s int) {
s += msgp.StringPrefixSize + len(za0001) + za0002.Msgsize()
}
}
s += 2 + msgp.Uint64Size
s += 2 + msgp.Uint64Size + 3 + msgp.Uint64Size
return
}
@ -2990,8 +3027,8 @@ func (z *replicationAllStatsV1) DecodeMsg(dc *msgp.Reader) (err error) {
err = msgp.WrapError(err)
return
}
if zb0001 != 2 {
err = msgp.ArrayError{Wanted: 2, Got: zb0001}
if zb0001 != 3 {
err = msgp.ArrayError{Wanted: 3, Got: zb0001}
return
}
var zb0002 uint32
@ -3028,13 +3065,18 @@ func (z *replicationAllStatsV1) DecodeMsg(dc *msgp.Reader) (err error) {
err = msgp.WrapError(err, "ReplicaSize")
return
}
z.ReplicaCount, err = dc.ReadUint64()
if err != nil {
err = msgp.WrapError(err, "ReplicaCount")
return
}
return
}
// EncodeMsg implements msgp.Encodable
func (z *replicationAllStatsV1) EncodeMsg(en *msgp.Writer) (err error) {
// array header, size 2
err = en.Append(0x92)
// array header, size 3
err = en.Append(0x93)
if err != nil {
return
}
@ -3060,14 +3102,19 @@ func (z *replicationAllStatsV1) EncodeMsg(en *msgp.Writer) (err error) {
err = msgp.WrapError(err, "ReplicaSize")
return
}
err = en.WriteUint64(z.ReplicaCount)
if err != nil {
err = msgp.WrapError(err, "ReplicaCount")
return
}
return
}
// MarshalMsg implements msgp.Marshaler
func (z *replicationAllStatsV1) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.Require(b, z.Msgsize())
// array header, size 2
o = append(o, 0x92)
// array header, size 3
o = append(o, 0x93)
o = msgp.AppendMapHeader(o, uint32(len(z.Targets)))
for za0001, za0002 := range z.Targets {
o = msgp.AppendString(o, za0001)
@ -3078,6 +3125,7 @@ func (z *replicationAllStatsV1) MarshalMsg(b []byte) (o []byte, err error) {
}
}
o = msgp.AppendUint64(o, z.ReplicaSize)
o = msgp.AppendUint64(o, z.ReplicaCount)
return
}
@ -3089,8 +3137,8 @@ func (z *replicationAllStatsV1) UnmarshalMsg(bts []byte) (o []byte, err error) {
err = msgp.WrapError(err)
return
}
if zb0001 != 2 {
err = msgp.ArrayError{Wanted: 2, Got: zb0001}
if zb0001 != 3 {
err = msgp.ArrayError{Wanted: 3, Got: zb0001}
return
}
var zb0002 uint32
@ -3127,6 +3175,11 @@ func (z *replicationAllStatsV1) UnmarshalMsg(bts []byte) (o []byte, err error) {
err = msgp.WrapError(err, "ReplicaSize")
return
}
z.ReplicaCount, bts, err = msgp.ReadUint64Bytes(bts)
if err != nil {
err = msgp.WrapError(err, "ReplicaCount")
return
}
o = bts
return
}
@ -3140,7 +3193,7 @@ func (z *replicationAllStatsV1) Msgsize() (s int) {
s += msgp.StringPrefixSize + len(za0001) + za0002.Msgsize()
}
}
s += msgp.Uint64Size
s += msgp.Uint64Size + msgp.Uint64Size
return
}
@ -3152,8 +3205,8 @@ func (z *replicationStats) DecodeMsg(dc *msgp.Reader) (err error) {
err = msgp.WrapError(err)
return
}
if zb0001 != 9 {
err = msgp.ArrayError{Wanted: 9, Got: zb0001}
if zb0001 != 10 {
err = msgp.ArrayError{Wanted: 10, Got: zb0001}
return
}
z.PendingSize, err = dc.ReadUint64()
@ -3201,13 +3254,18 @@ func (z *replicationStats) DecodeMsg(dc *msgp.Reader) (err error) {
err = msgp.WrapError(err, "AfterThresholdCount")
return
}
z.ReplicatedCount, err = dc.ReadUint64()
if err != nil {
err = msgp.WrapError(err, "ReplicatedCount")
return
}
return
}
// EncodeMsg implements msgp.Encodable
func (z *replicationStats) EncodeMsg(en *msgp.Writer) (err error) {
// array header, size 9
err = en.Append(0x99)
// array header, size 10
err = en.Append(0x9a)
if err != nil {
return
}
@ -3256,14 +3314,19 @@ func (z *replicationStats) EncodeMsg(en *msgp.Writer) (err error) {
err = msgp.WrapError(err, "AfterThresholdCount")
return
}
err = en.WriteUint64(z.ReplicatedCount)
if err != nil {
err = msgp.WrapError(err, "ReplicatedCount")
return
}
return
}
// MarshalMsg implements msgp.Marshaler
func (z *replicationStats) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.Require(b, z.Msgsize())
// array header, size 9
o = append(o, 0x99)
// array header, size 10
o = append(o, 0x9a)
o = msgp.AppendUint64(o, z.PendingSize)
o = msgp.AppendUint64(o, z.ReplicatedSize)
o = msgp.AppendUint64(o, z.FailedSize)
@ -3273,6 +3336,7 @@ func (z *replicationStats) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.AppendUint64(o, z.AfterThresholdSize)
o = msgp.AppendUint64(o, z.MissedThresholdCount)
o = msgp.AppendUint64(o, z.AfterThresholdCount)
o = msgp.AppendUint64(o, z.ReplicatedCount)
return
}
@ -3284,8 +3348,8 @@ func (z *replicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) {
err = msgp.WrapError(err)
return
}
if zb0001 != 9 {
err = msgp.ArrayError{Wanted: 9, Got: zb0001}
if zb0001 != 10 {
err = msgp.ArrayError{Wanted: 10, Got: zb0001}
return
}
z.PendingSize, bts, err = msgp.ReadUint64Bytes(bts)
@ -3333,13 +3397,18 @@ func (z *replicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) {
err = msgp.WrapError(err, "AfterThresholdCount")
return
}
z.ReplicatedCount, bts, err = msgp.ReadUint64Bytes(bts)
if err != nil {
err = msgp.WrapError(err, "ReplicatedCount")
return
}
o = bts
return
}
// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message
func (z *replicationStats) Msgsize() (s int) {
s = 1 + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size
s = 1 + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size
return
}

View File

@ -38,6 +38,7 @@ type BucketTargetUsageInfo struct {
ReplicaSize uint64 `json:"objectReplicaTotalSize"`
ReplicationPendingCount uint64 `json:"objectsPendingReplicationCount"`
ReplicationFailedCount uint64 `json:"objectsFailedReplicationCount"`
ReplicatedCount uint64 `json:"objectsReplicatedCount"`
}
// BucketUsageInfo - bucket usage info provides
@ -64,6 +65,7 @@ type BucketUsageInfo struct {
VersionsCount uint64 `json:"versionsCount"`
DeleteMarkersCount uint64 `json:"deleteMarkersCount"`
ReplicaSize uint64 `json:"objectReplicaTotalSize"`
ReplicaCount uint64 `json:"objectReplicaCount"`
ReplicationInfo map[string]BucketTargetUsageInfo `json:"objectsReplicationInfo"`
}

View File

@ -55,6 +55,7 @@ func init() {
getKMSMetrics(),
getClusterHealthMetrics(),
getIAMNodeMetrics(),
getReplicationSiteMetrics(),
}
peerMetricsGroups = []*MetricsGroup{
@ -73,6 +74,7 @@ func init() {
getKMSNodeMetrics(),
getMinioHealingMetrics(),
getWebhookMetrics(),
getReplicationClusterMetrics(),
}
allMetricsGroups := func() (allMetrics []*MetricsGroup) {
@ -186,8 +188,39 @@ const (
total MetricName = "total"
freeInodes MetricName = "free_inodes"
failedCount MetricName = "failed_count"
failedBytes MetricName = "failed_bytes"
lastMinFailedCount MetricName = "last_minute_failed_count"
lastMinFailedBytes MetricName = "last_minute_failed_bytes"
lastHourFailedCount MetricName = "last_hour_failed_count"
lastHourFailedBytes MetricName = "last_hour_failed_bytes"
totalFailedCount MetricName = "total_failed_count"
totalFailedBytes MetricName = "total_failed_bytes"
currActiveWorkers MetricName = "current_active_workers"
avgActiveWorkers MetricName = "average_active_workers"
maxActiveWorkers MetricName = "max_active_workers"
recentBacklogCount MetricName = "recent_backlog_count"
currInQueueCount MetricName = "last_minute_queued_count"
currInQueueBytes MetricName = "last_minute_queued_bytes"
receivedCount MetricName = "received_count"
sentCount MetricName = "sent_count"
currTransferRate MetricName = "current_transfer_rate"
avgTransferRate MetricName = "average_transfer_rate"
maxTransferRate MetricName = "max_transfer_rate"
credentialErrors MetricName = "credential_errors"
currLinkLatency MetricName = "current_link_latency_ms"
avgLinkLatency MetricName = "average_link_latency_ms"
maxLinkLatency MetricName = "max_link_latency_ms"
linkOnline MetricName = "link_online"
linkOfflineDuration MetricName = "link_offline_duration_seconds"
linkDowntimeTotalDuration MetricName = "link_downtime_duration_seconds"
avgInQueueCount MetricName = "average_queued_count"
avgInQueueBytes MetricName = "average_queued_bytes"
maxInQueueCount MetricName = "max_queued_count"
maxInQueueBytes MetricName = "max_queued_bytes"
freeBytes MetricName = "free_bytes"
readBytes MetricName = "read_bytes"
rcharBytes MetricName = "rchar_bytes"
@ -646,56 +679,6 @@ func getBucketUsageDeleteMarkersTotalMD() MetricDescription {
}
}
func getBucketRepLatencyMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
Subsystem: replicationSubsystem,
Name: latencyMilliSec,
Help: "Replication latency in milliseconds",
Type: histogramMetric,
}
}
func getBucketRepFailedBytesMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
Subsystem: replicationSubsystem,
Name: failedBytes,
Help: "Total number of bytes failed at least once to replicate",
Type: gaugeMetric,
}
}
func getBucketRepSentBytesMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
Subsystem: replicationSubsystem,
Name: sentBytes,
Help: "Total number of bytes replicated to the target bucket",
Type: gaugeMetric,
}
}
func getBucketRepReceivedBytesMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
Subsystem: replicationSubsystem,
Name: receivedBytes,
Help: "Total number of bytes replicated to this bucket from another source bucket",
Type: gaugeMetric,
}
}
func getBucketRepFailedOperationsMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
Subsystem: replicationSubsystem,
Name: failedCount,
Help: "Total number of objects which failed replication",
Type: gaugeMetric,
}
}
func getClusterObjectDistributionMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
@ -716,6 +699,324 @@ func getClusterObjectVersionsMD() MetricDescription {
}
}
func getClusterRepLinkLatencyCurrMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: currLinkLatency,
Help: "Replication current link latency in milliseconds",
Type: gaugeMetric,
}
}
func getClusterRepLinkOnlineMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: linkOnline,
Help: "Reports whether replication link is online (1) or offline(0)",
Type: gaugeMetric,
}
}
func getClusterRepLinkCurrOfflineDurationMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: linkOfflineDuration,
Help: "Duration of replication link being offline in seconds since last offline event",
Type: gaugeMetric,
}
}
func getClusterRepLinkTotalOfflineDurationMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: linkDowntimeTotalDuration,
Help: "Total downtime of replication link in seconds since server start",
Type: gaugeMetric,
}
}
func getBucketRepLatencyMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
Subsystem: replicationSubsystem,
Name: latencyMilliSec,
Help: "Replication latency in milliseconds",
Type: histogramMetric,
}
}
func getRepFailedBytesLastMinuteMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: lastMinFailedBytes,
Help: "Total number of bytes failed at least once to replicate in the last full minute",
Type: gaugeMetric,
}
}
func getRepFailedOperationsLastMinuteMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: lastMinFailedCount,
Help: "Total number of objects which failed replication in the last full minute",
Type: gaugeMetric,
}
}
func getRepFailedBytesLastHourMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: lastHourFailedBytes,
Help: "Total number of bytes failed at least once to replicate in the last hour",
Type: gaugeMetric,
}
}
func getRepFailedOperationsLastHourMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: lastHourFailedCount,
Help: "Total number of objects which failed replication in the last hour",
Type: gaugeMetric,
}
}
func getRepFailedBytesTotalMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: totalFailedBytes,
Help: "Total number of bytes failed at least once to replicate since server start",
Type: gaugeMetric,
}
}
func getRepFailedOperationsTotalMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: totalFailedCount,
Help: "Total number of objects which failed replication since server start",
Type: gaugeMetric,
}
}
func getRepSentBytesMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: sentBytes,
Help: "Total number of bytes replicated to the target",
Type: gaugeMetric,
}
}
func getRepSentOperationsMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: sentCount,
Help: "Total number of objects replicated to the target",
Type: gaugeMetric,
}
}
func getRepReceivedBytesMD(namespace MetricNamespace) MetricDescription {
helpText := "Total number of bytes replicated to this bucket from another source bucket"
if namespace == clusterMetricNamespace {
helpText = "Total number of bytes replicated to this cluster from site replication peer"
}
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: receivedBytes,
Help: helpText,
Type: gaugeMetric,
}
}
func getRepReceivedOperationsMD(namespace MetricNamespace) MetricDescription {
help := "Total number of objects received by this cluster"
if namespace == bucketMetricNamespace {
help = "Total number of objects received by this bucket from another source bucket"
}
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: receivedCount,
Help: help,
Type: gaugeMetric,
}
}
func getClusterReplMRFFailedOperationsMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: recentBacklogCount,
Help: "Total number of objects seen in replication backlog in the last 5 minutes",
Type: gaugeMetric,
}
}
func getClusterRepCredentialErrorsMD(namespace MetricNamespace) MetricDescription {
return MetricDescription{
Namespace: namespace,
Subsystem: replicationSubsystem,
Name: credentialErrors,
Help: "Total number of replication credential errors since server start",
Type: counterMetric,
}
}
func getClusterReplCurrQueuedOperationsMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: currInQueueCount,
Help: "Total number of objects queued for replication in the last full minute",
Type: gaugeMetric,
}
}
func getClusterReplCurrQueuedBytesMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: currInQueueBytes,
Help: "Total number of bytes queued for replication in the last full minute",
Type: gaugeMetric,
}
}
func getClusterReplActiveWorkersCountMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: currActiveWorkers,
Help: "Total number of active replication workers",
Type: gaugeMetric,
}
}
func getClusterReplAvgActiveWorkersCountMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: avgActiveWorkers,
Help: "Average number of active replication workers",
Type: gaugeMetric,
}
}
func getClusterReplMaxActiveWorkersCountMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: maxActiveWorkers,
Help: "Maximum number of active replication workers seen since server start",
Type: gaugeMetric,
}
}
func getClusterReplCurrentTransferRateMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: currTransferRate,
Help: "Current replication transfer rate in bytes/sec",
Type: gaugeMetric,
}
}
func getClusterRepLinkLatencyMaxMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: maxLinkLatency,
Help: "Maximum replication link latency in milliseconds seen since server start",
Type: gaugeMetric,
}
}
func getClusterRepLinkLatencyAvgMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: avgLinkLatency,
Help: "Average replication link latency in milliseconds",
Type: gaugeMetric,
}
}
func getClusterReplAvgQueuedOperationsMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: avgInQueueCount,
Help: "Average number of objects queued for replication since server start",
Type: gaugeMetric,
}
}
func getClusterReplAvgQueuedBytesMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: avgInQueueBytes,
Help: "Average number of bytes queued for replication since server start",
Type: gaugeMetric,
}
}
func getClusterReplMaxQueuedOperationsMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: maxInQueueCount,
Help: "Maximum number of objects queued for replication since server start",
Type: gaugeMetric,
}
}
func getClusterReplMaxQueuedBytesMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: maxInQueueBytes,
Help: "Maximum number of bytes queued for replication since server start",
Type: gaugeMetric,
}
}
func getClusterReplAvgTransferRateMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: avgTransferRate,
Help: "Average replication transfer rate in bytes/sec",
Type: gaugeMetric,
}
}
func getClusterReplMaxTransferRateMD() MetricDescription {
return MetricDescription{
Namespace: clusterMetricNamespace,
Subsystem: replicationSubsystem,
Name: maxTransferRate,
Help: "Maximum replication transfer rate in bytes/sec seen since server start",
Type: gaugeMetric,
}
}
func getBucketObjectDistributionMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
@ -1773,6 +2074,273 @@ func getIAMNodeMetrics() *MetricsGroup {
return mg
}
// replication metrics for each node - published to the cluster endpoint with nodename as label
func getReplicationClusterMetrics() *MetricsGroup {
mg := &MetricsGroup{
cacheInterval: 1 * time.Minute,
}
const (
Online = 1
Offline = 0
)
mg.RegisterRead(func(_ context.Context) []Metric {
// common operational metrics for bucket replication and site replication - published
// at cluster level
qs := globalReplicationStats.getNodeQueueStatsSummary()
activeWorkersCount := Metric{
Description: getClusterReplActiveWorkersCountMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
avgActiveWorkersCount := Metric{
Description: getClusterReplAvgActiveWorkersCountMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
maxActiveWorkersCount := Metric{
Description: getClusterReplMaxActiveWorkersCountMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
currInQueueCount := Metric{
Description: getClusterReplCurrQueuedOperationsMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
currInQueueBytes := Metric{
Description: getClusterReplCurrQueuedBytesMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
currTransferRate := Metric{
Description: getClusterReplCurrentTransferRateMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
avgQueueCount := Metric{
Description: getClusterReplAvgQueuedOperationsMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
avgQueueBytes := Metric{
Description: getClusterReplAvgQueuedBytesMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
maxQueueCount := Metric{
Description: getClusterReplMaxQueuedOperationsMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
maxQueueBytes := Metric{
Description: getClusterReplMaxQueuedBytesMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
avgTransferRate := Metric{
Description: getClusterReplAvgTransferRateMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
maxTransferRate := Metric{
Description: getClusterReplMaxTransferRateMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
}
mrfCount := Metric{
Description: getClusterReplMRFFailedOperationsMD(),
VariableLabels: map[string]string{serverName: qs.NodeName},
Value: float64(qs.MRFStats.LastFailedCount),
}
if qs.QStats.Avg.Count > 0 || qs.QStats.Curr.Count > 0 {
qt := qs.QStats
currInQueueBytes.Value = qt.Curr.Bytes
currInQueueCount.Value = qt.Curr.Count
avgQueueBytes.Value = qt.Avg.Bytes
avgQueueCount.Value = qt.Avg.Count
maxQueueBytes.Value = qt.Max.Bytes
maxQueueCount.Value = qt.Max.Count
}
activeWorkersCount.Value = float64(qs.ActiveWorkers.Curr)
avgActiveWorkersCount.Value = float64(qs.ActiveWorkers.Avg)
maxActiveWorkersCount.Value = float64(qs.ActiveWorkers.Max)
if len(qs.XferStats) > 0 {
tots := qs.XferStats[Total]
currTransferRate.Value = tots.Curr
avgTransferRate.Value = tots.Avg
maxTransferRate.Value = tots.Peak
}
ml := []Metric{
activeWorkersCount,
avgActiveWorkersCount,
maxActiveWorkersCount,
currInQueueCount,
currInQueueBytes,
avgQueueCount,
avgQueueBytes,
maxQueueCount,
maxQueueBytes,
currTransferRate,
avgTransferRate,
maxTransferRate,
mrfCount,
}
for ep, health := range globalBucketTargetSys.healthStats() {
// link latency current
m := Metric{
Description: getClusterRepLinkLatencyCurrMD(),
VariableLabels: map[string]string{
"endpoint": ep,
serverName: globalLocalNodeName,
},
}
m.Value = float64(health.latency.curr / time.Millisecond)
ml = append(ml, m)
// link latency average
m = Metric{
Description: getClusterRepLinkLatencyAvgMD(),
VariableLabels: map[string]string{
"endpoint": ep,
serverName: globalLocalNodeName,
},
}
m.Value = float64(health.latency.avg / time.Millisecond)
ml = append(ml, m)
// link latency max
m = Metric{
Description: getClusterRepLinkLatencyMaxMD(),
VariableLabels: map[string]string{
"endpoint": ep,
serverName: globalLocalNodeName,
},
}
m.Value = float64(health.latency.peak / time.Millisecond)
ml = append(ml, m)
linkOnline := Metric{
Description: getClusterRepLinkOnlineMD(),
VariableLabels: map[string]string{
"endpoint": ep,
serverName: globalLocalNodeName,
},
}
online := Offline
if health.Online {
online = Online
}
linkOnline.Value = float64(online)
ml = append(ml, linkOnline)
offlineDuration := Metric{
Description: getClusterRepLinkCurrOfflineDurationMD(),
VariableLabels: map[string]string{
"endpoint": ep,
serverName: globalLocalNodeName,
},
}
currDowntime := time.Duration(0)
if !health.Online && !health.lastOnline.IsZero() {
currDowntime = UTCNow().Sub(health.lastOnline)
}
offlineDuration.Value = float64(currDowntime / time.Second)
ml = append(ml, offlineDuration)
downtimeDuration := Metric{
Description: getClusterRepLinkTotalOfflineDurationMD(),
VariableLabels: map[string]string{
"endpoint": ep,
serverName: globalLocalNodeName,
},
}
dwntime := currDowntime
if health.offlineDuration > currDowntime {
dwntime = health.offlineDuration
}
downtimeDuration.Value = float64(dwntime / time.Second)
ml = append(ml, downtimeDuration)
}
return ml
})
return mg
}
// replication metrics for site replication
func getReplicationSiteMetrics() *MetricsGroup {
mg := &MetricsGroup{
cacheInterval: 1 * time.Minute,
}
mg.RegisterRead(func(_ context.Context) []Metric {
ml := []Metric{}
// metrics pertinent to site replication - overall roll up.
if globalSiteReplicationSys.isEnabled() {
m, err := globalSiteReplicationSys.getSiteMetrics(GlobalContext)
if err != nil {
logger.LogIf(GlobalContext, err)
return ml
}
ml = append(ml, Metric{
Description: getRepReceivedBytesMD(clusterMetricNamespace),
Value: float64(m.ReplicaSize),
})
ml = append(ml, Metric{
Description: getRepReceivedOperationsMD(clusterMetricNamespace),
Value: float64(m.ReplicaCount),
})
for _, stat := range m.Metrics {
ml = append(ml, Metric{
Description: getRepFailedBytesLastMinuteMD(clusterMetricNamespace),
Value: float64(stat.Failed.LastMinute.Bytes),
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
ml = append(ml, Metric{
Description: getRepFailedOperationsLastMinuteMD(clusterMetricNamespace),
Value: stat.Failed.LastMinute.Count,
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
ml = append(ml, Metric{
Description: getRepFailedBytesLastHourMD(clusterMetricNamespace),
Value: float64(stat.Failed.LastHour.Bytes),
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
ml = append(ml, Metric{
Description: getRepFailedOperationsLastHourMD(clusterMetricNamespace),
Value: stat.Failed.LastHour.Count,
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
ml = append(ml, Metric{
Description: getRepFailedBytesTotalMD(clusterMetricNamespace),
Value: float64(stat.Failed.Totals.Bytes),
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
ml = append(ml, Metric{
Description: getRepFailedOperationsTotalMD(clusterMetricNamespace),
Value: stat.Failed.Totals.Count,
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
ml = append(ml, Metric{
Description: getRepSentBytesMD(clusterMetricNamespace),
Value: float64(stat.ReplicatedSize),
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
ml = append(ml, Metric{
Description: getRepSentOperationsMD(clusterMetricNamespace),
Value: float64(stat.ReplicatedCount),
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
if c, ok := stat.Failed.ErrCounts["AccessDenied"]; ok {
ml = append(ml, Metric{
Description: getClusterRepCredentialErrorsMD(clusterMetricNamespace),
Value: float64(c),
VariableLabels: map[string]string{"endpoint": stat.Endpoint},
})
}
}
}
return ml
})
return mg
}
func getMinioVersionMetrics() *MetricsGroup {
mg := &MetricsGroup{
cacheInterval: 10 * time.Second,
@ -2412,10 +2980,11 @@ func getBucketUsageMetrics() *MetricsGroup {
Value: float64(time.Since(dataUsageInfo.LastUpdate)),
})
bucketReplStats := globalReplicationStats.getAllLatest(dataUsageInfo.BucketsUsage)
var bucketReplStats map[string]BucketStats
if !globalSiteReplicationSys.isEnabled() {
bucketReplStats = globalReplicationStats.getAllLatest(dataUsageInfo.BucketsUsage)
}
for bucket, usage := range dataUsageInfo.BucketsUsage {
stats := bucketReplStats[bucket]
quota, _ := globalBucketQuotaSys.Get(ctx, bucket)
metrics = append(metrics, Metric{
@ -2442,12 +3011,6 @@ func getBucketUsageMetrics() *MetricsGroup {
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
Description: getBucketRepReceivedBytesMD(),
Value: float64(stats.ReplicaSize),
VariableLabels: map[string]string{"bucket": bucket},
})
if quota != nil && quota.Quota > 0 {
metrics = append(metrics, Metric{
Description: getBucketUsageQuotaTotalBytesMD(),
@ -2455,47 +3018,89 @@ func getBucketUsageMetrics() *MetricsGroup {
VariableLabels: map[string]string{"bucket": bucket},
})
}
if stats.hasReplicationUsage() {
for arn, stat := range stats.Stats {
metrics = append(metrics, Metric{
Description: getBucketRepFailedBytesMD(),
Value: float64(stat.FailedSize),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getBucketRepSentBytesMD(),
Value: float64(stat.ReplicatedSize),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getBucketRepFailedOperationsMD(),
Value: float64(stat.FailedCount),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getBucketRepLatencyMD(),
HistogramBucketLabel: "range",
Histogram: stat.Latency.getUploadLatency(),
VariableLabels: map[string]string{"bucket": bucket, "operation": "upload", "targetArn": arn},
})
if !globalSiteReplicationSys.isEnabled() {
stats := bucketReplStats[bucket].ReplicationStats
metrics = append(metrics, Metric{
Description: getRepReceivedBytesMD(bucketMetricNamespace),
Value: float64(stats.ReplicaSize),
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
Description: getRepReceivedOperationsMD(bucketMetricNamespace),
Value: float64(stats.ReplicaCount),
VariableLabels: map[string]string{"bucket": bucket},
})
if stats.hasReplicationUsage() {
for arn, stat := range stats.Stats {
metrics = append(metrics, Metric{
Description: getRepFailedBytesLastMinuteMD(bucketMetricNamespace),
Value: float64(stat.Failed.LastMinute.Bytes),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getRepFailedOperationsLastMinuteMD(bucketMetricNamespace),
Value: stat.Failed.LastMinute.Count,
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getRepFailedBytesLastHourMD(bucketMetricNamespace),
Value: float64(stat.Failed.LastHour.Bytes),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getRepFailedOperationsLastHourMD(bucketMetricNamespace),
Value: stat.Failed.LastHour.Count,
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getRepFailedBytesTotalMD(bucketMetricNamespace),
Value: float64(stat.Failed.Totals.Bytes),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getRepFailedOperationsTotalMD(bucketMetricNamespace),
Value: stat.Failed.Totals.Count,
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getRepSentBytesMD(bucketMetricNamespace),
Value: float64(stat.ReplicatedSize),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getRepSentOperationsMD(bucketMetricNamespace),
Value: float64(stat.ReplicatedCount),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
metrics = append(metrics, Metric{
Description: getBucketRepLatencyMD(),
HistogramBucketLabel: "range",
Histogram: stat.Latency.getUploadLatency(),
VariableLabels: map[string]string{"bucket": bucket, "operation": "upload", "targetArn": arn},
})
if c, ok := stat.Failed.ErrCounts["AccessDenied"]; ok {
metrics = append(metrics, Metric{
Description: getClusterRepCredentialErrorsMD(bucketMetricNamespace),
Value: float64(c),
VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn},
})
}
}
}
metrics = append(metrics, Metric{
Description: getBucketObjectDistributionMD(),
Histogram: usage.ObjectSizesHistogram,
HistogramBucketLabel: "range",
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
Description: getBucketObjectVersionsMD(),
Histogram: usage.ObjectVersionsHistogram,
HistogramBucketLabel: "range",
VariableLabels: map[string]string{"bucket": bucket},
})
}
metrics = append(metrics, Metric{
Description: getBucketObjectDistributionMD(),
Histogram: usage.ObjectSizesHistogram,
HistogramBucketLabel: "range",
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
Description: getBucketObjectVersionsMD(),
Histogram: usage.ObjectVersionsHistogram,
HistogramBucketLabel: "range",
VariableLabels: map[string]string{"bucket": bucket},
})
}
return
})

View File

@ -381,7 +381,7 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
}
for bucket, usageInfo := range dataUsageInfo.BucketsUsage {
stat := globalReplicationStats.getLatestReplicationStats(bucket, usageInfo)
stat := globalReplicationStats.getLatestReplicationStats(bucket)
// Total space used by bucket
ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc(
@ -401,22 +401,13 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
float64(usageInfo.ObjectsCount),
bucket,
)
ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc(
prometheus.BuildFQName("bucket", "replication", "failed_size"),
"Total capacity failed to replicate at least once",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(stat.FailedSize),
bucket,
)
ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc(
prometheus.BuildFQName("bucket", "replication", "successful_size"),
"Total capacity replicated to destination",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(stat.ReplicatedSize),
float64(stat.ReplicationStats.ReplicatedSize),
bucket,
)
ch <- prometheus.MustNewConstMetric(
@ -425,18 +416,10 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity replicated to this instance",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(stat.ReplicaSize),
bucket,
)
ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc(
prometheus.BuildFQName("bucket", "replication", "failed_count"),
"Total replication operations failed",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(stat.FailedCount),
float64(stat.ReplicationStats.ReplicaSize),
bucket,
)
for k, v := range usageInfo.ObjectSizesHistogram {
ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc(

View File

@ -549,10 +549,40 @@ func (sys *NotificationSys) GetClusterBucketStats(ctx context.Context, bucketNam
}
bucketStats = append(bucketStats, BucketStats{
ReplicationStats: globalReplicationStats.Get(bucketName),
QueueStats: ReplicationQueueStats{Nodes: []ReplQNodeStats{globalReplicationStats.getNodeQueueStats(bucketName)}},
})
return bucketStats
}
// GetClusterSiteMetrics - calls GetClusterSiteMetrics call on all peers for a cluster statistics view.
func (sys *NotificationSys) GetClusterSiteMetrics(ctx context.Context) []SRMetricsSummary {
ng := WithNPeers(len(sys.peerClients)).WithRetries(1)
siteStats := make([]SRMetricsSummary, len(sys.peerClients))
for index, client := range sys.peerClients {
index := index
client := client
ng.Go(ctx, func() error {
if client == nil {
return errPeerNotReachable
}
sm, err := client.GetSRMetrics()
if err != nil {
return err
}
siteStats[index] = sm
return nil
}, index, *client.host)
}
for _, nErr := range ng.Wait() {
reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", nErr.Host.String())
if nErr.Err != nil {
logger.LogIf(logger.SetReqInfo(ctx, reqInfo), nErr.Err)
}
}
siteStats = append(siteStats, globalReplicationStats.getSRMetricsForNode())
return siteStats
}
// ReloadPoolMeta reloads on disk updates on pool metadata
func (sys *NotificationSys) ReloadPoolMeta(ctx context.Context) {
ng := WithNPeers(len(sys.peerClients))

View File

@ -277,6 +277,19 @@ func (client *peerRESTClient) GetBucketStats(bucket string) (BucketStats, error)
return bs, msgp.Decode(respBody, &bs)
}
// GetSRMetrics- loads site replication metrics, optionally for a specific bucket
func (client *peerRESTClient) GetSRMetrics() (SRMetricsSummary, error) {
values := make(url.Values)
respBody, err := client.call(peerRESTMethodGetSRMetrics, values, nil, -1)
if err != nil {
return SRMetricsSummary{}, err
}
var sm SRMetricsSummary
defer xhttp.DrainBody(respBody)
return sm, msgp.Decode(respBody, &sm)
}
// GetAllBucketStats - load replication stats for all buckets
func (client *peerRESTClient) GetAllBucketStats() (BucketStatsMap, error) {
values := make(url.Values)

View File

@ -18,7 +18,7 @@
package cmd
const (
peerRESTVersion = "v32" // Add bucket peer metrics
peerRESTVersion = "v33" // Add SRMetrics
peerRESTVersionPrefix = SlashSeparator + peerRESTVersion
peerRESTPrefix = minioReservedBucketPath + "/peer"
@ -78,6 +78,7 @@ const (
peerRESTMethodNetperf = "/netperf"
peerRESTMethodMetrics = "/metrics"
peerRESTMethodGetReplicationMRF = "/getreplicationmrf"
peerRESTMethodGetSRMetrics = "/getsrmetrics"
)
const (

View File

@ -598,10 +598,27 @@ func (s *peerRESTServer) GetBucketStatsHandler(w http.ResponseWriter, r *http.Re
bs := BucketStats{
ReplicationStats: globalReplicationStats.Get(bucketName),
QueueStats: ReplicationQueueStats{Nodes: []ReplQNodeStats{globalReplicationStats.getNodeQueueStats(bucketName)}},
}
logger.LogIf(r.Context(), msgp.Encode(w, &bs))
}
// GetSRMetricsHandler - fetches current in-memory replication stats at site level from this peer
func (s *peerRESTServer) GetSRMetricsHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("Invalid request"))
return
}
objAPI := newObjectLayerFn()
if objAPI == nil {
s.writeErrorResponse(w, errServerNotInitialized)
return
}
sm := globalReplicationStats.getSRMetricsForNode()
logger.LogIf(r.Context(), msgp.Encode(w, &sm))
}
// LoadBucketMetadataHandler - reloads in memory bucket metadata
func (s *peerRESTServer) LoadBucketMetadataHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
@ -1443,6 +1460,7 @@ func registerPeerRESTHandlers(router *mux.Router) {
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodLoadServiceAccount).HandlerFunc(h(server.LoadServiceAccountHandler)).Queries(restQueries(peerRESTUser)...)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodLoadGroup).HandlerFunc(h(server.LoadGroupHandler)).Queries(restQueries(peerRESTGroup)...)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodGetReplicationMRF).HandlerFunc(httpTraceHdrs(server.GetReplicationMRFHandler)).Queries(restQueries(peerRESTBucket)...)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodGetSRMetrics).HandlerFunc(h(server.GetSRMetricsHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodStartProfiling).HandlerFunc(h(server.StartProfilingHandler)).Queries(restQueries(peerRESTProfiler)...)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodDownloadProfilingData).HandlerFunc(h(server.DownloadProfilingDataHandler))

View File

@ -0,0 +1,289 @@
// Copyright (c) 2015-2023 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package cmd
import (
"fmt"
"sync"
"sync/atomic"
"time"
"github.com/minio/madmin-go/v3"
"github.com/minio/minio-go/v7"
)
//go:generate msgp -file $GOFILE
// RStat has replication error stats
type RStat struct {
Count int64 `json:"count"`
Bytes int64 `json:"bytes"`
}
// RTimedMetrics has replication error stats for various time windows
type RTimedMetrics struct {
LastHour ReplicationLastHour `json:"lastHour"`
SinceUptime RStat `json:"sinceUptime"`
LastMinute ReplicationLastMinute
// Error counts
ErrCounts map[string]int `json:"errCounts"` // Count of credential errors
}
func (rt *RTimedMetrics) String() string {
s := rt.toMetric()
return fmt.Sprintf("Errors in LastMinute: %v, LastHour: %v, SinceUptime: %v", s.LastMinute.Count, s.LastHour.Count, s.Totals.Count)
}
func (rt *RTimedMetrics) toMetric() madmin.TimedErrStats {
if rt == nil {
return madmin.TimedErrStats{}
}
errCounts := make(map[string]int)
for k, v := range rt.ErrCounts {
errCounts[k] = v
}
minuteTotals := rt.LastMinute.getTotal()
hourTotals := rt.LastHour.getTotal()
return madmin.TimedErrStats{
LastMinute: madmin.RStat{
Count: float64(minuteTotals.N),
Bytes: minuteTotals.Size,
},
LastHour: madmin.RStat{
Count: float64(hourTotals.N),
Bytes: hourTotals.Size,
},
Totals: madmin.RStat{
Count: float64(rt.SinceUptime.Count),
Bytes: rt.SinceUptime.Bytes,
},
ErrCounts: errCounts,
}
}
func (rt *RTimedMetrics) addsize(size int64, err error) {
// failures seen since uptime
atomic.AddInt64(&rt.SinceUptime.Bytes, size)
atomic.AddInt64(&rt.SinceUptime.Count, 1)
rt.LastMinute.addsize(size)
rt.LastHour.addsize(size)
if err != nil && minio.ToErrorResponse(err).Code == "AccessDenied" {
if rt.ErrCounts == nil {
rt.ErrCounts = make(map[string]int)
}
rt.ErrCounts["AccessDenied"]++
}
}
func (rt *RTimedMetrics) merge(o RTimedMetrics) (n RTimedMetrics) {
n.SinceUptime.Bytes = atomic.LoadInt64(&rt.SinceUptime.Bytes) + atomic.LoadInt64(&o.SinceUptime.Bytes)
n.SinceUptime.Count = atomic.LoadInt64(&rt.SinceUptime.Count) + atomic.LoadInt64(&o.SinceUptime.Count)
n.LastMinute = n.LastMinute.merge(rt.LastMinute)
n.LastMinute = n.LastMinute.merge(o.LastMinute)
n.LastHour = n.LastHour.merge(rt.LastHour)
n.LastHour = n.LastHour.merge(o.LastHour)
n.ErrCounts = make(map[string]int)
for k, v := range rt.ErrCounts {
n.ErrCounts[k] = v
}
for k, v := range o.ErrCounts {
n.ErrCounts[k] += v
}
return n
}
// SRStats has replication stats at site level
type SRStats struct {
// Total Replica size in bytes
ReplicaSize int64 `json:"replicaSize"`
// Total Replica received
ReplicaCount int64 `json:"replicaCount"`
M map[string]*SRStatus `json:"srStatusMap"`
movingAvgTicker *time.Ticker // Ticker for calculating moving averages
lock sync.RWMutex // mutex for srStats
}
// SRStatus has replication stats at deployment level
type SRStatus struct {
ReplicatedSize int64 `json:"completedReplicationSize"`
// Total number of failed operations including metadata updates in the last minute
Failed RTimedMetrics `json:"failedReplication"`
// Total number of completed operations
ReplicatedCount int64 `json:"replicationCount"`
// Replication latency information
Latency ReplicationLatency `json:"replicationLatency"`
// transfer rate for large uploads
XferRateLrg *XferStats `json:"largeTransferRate" msg:"lt"`
// transfer rate for small uploads
XferRateSml *XferStats `json:"smallTransferRate" msg:"st"`
// Endpoint is the replication target endpoint
Endpoint string `json:"-"`
// Secure is true if the replication target endpoint is secure
Secure bool `json:"-"`
}
func (sr *SRStats) update(st replStat, dID string) {
sr.lock.Lock()
defer sr.lock.Unlock()
srs, ok := sr.M[dID]
if !ok {
srs = &SRStatus{
XferRateLrg: newXferStats(),
XferRateSml: newXferStats(),
}
}
srs.Endpoint = st.Endpoint
srs.Secure = st.Secure
switch {
case st.Completed:
srs.ReplicatedSize += st.TransferSize
srs.ReplicatedCount++
if st.TransferDuration > 0 {
srs.Latency.update(st.TransferSize, st.TransferDuration)
srs.updateXferRate(st.TransferSize, st.TransferDuration)
}
case st.Failed:
srs.Failed.addsize(st.TransferSize, st.Err)
case st.Pending:
}
sr.M[dID] = srs
}
func (sr *SRStats) get() map[string]SRMetric {
epMap := globalBucketTargetSys.healthStats()
sr.lock.RLock()
defer sr.lock.RUnlock()
m := make(map[string]SRMetric, len(sr.M))
for dID, v := range sr.M {
t := newXferStats()
mx := make(map[RMetricName]XferStats)
if v.XferRateLrg != nil {
mx[Large] = *v.XferRateLrg.Clone()
m := t.merge(*v.XferRateLrg)
t = &m
}
if v.XferRateSml != nil {
mx[Small] = *v.XferRateSml.Clone()
m := t.merge(*v.XferRateSml)
t = &m
}
mx[Total] = *t
metric := SRMetric{
ReplicatedSize: v.ReplicatedSize,
ReplicatedCount: v.ReplicatedCount,
DeploymentID: dID,
Failed: v.Failed.toMetric(),
XferStats: mx,
}
epHealth, ok := epMap[v.Endpoint]
if ok {
metric.Endpoint = epHealth.Endpoint
metric.TotalDowntime = epHealth.offlineDuration
metric.LastOnline = epHealth.lastOnline
metric.Online = epHealth.Online
metric.Latency = madmin.LatencyStat{
Curr: epHealth.latency.curr,
Avg: epHealth.latency.avg,
Max: epHealth.latency.peak,
}
}
m[dID] = metric
}
return m
}
func (srs *SRStatus) updateXferRate(sz int64, duration time.Duration) {
if sz > minLargeObjSize {
srs.XferRateLrg.addSize(sz, duration)
} else {
srs.XferRateSml.addSize(sz, duration)
}
}
func newSRStats() *SRStats {
s := SRStats{
M: make(map[string]*SRStatus),
movingAvgTicker: time.NewTicker(time.Second * 2),
}
go s.trackEWMA()
return &s
}
func (sr *SRStats) trackEWMA() {
for {
select {
case <-sr.movingAvgTicker.C:
sr.updateMovingAvg()
case <-GlobalContext.Done():
return
}
}
}
func (sr *SRStats) updateMovingAvg() {
sr.lock.Lock()
defer sr.lock.Unlock()
for _, s := range sr.M {
s.XferRateLrg.measure.updateExponentialMovingAverage(time.Now())
s.XferRateSml.measure.updateExponentialMovingAverage(time.Now())
}
}
// SRMetric captures replication metrics for a deployment
type SRMetric struct {
DeploymentID string `json:"deploymentID"`
Endpoint string `json:"endpoint"`
TotalDowntime time.Duration `json:"totalDowntime"`
LastOnline time.Time `json:"lastOnline"`
Online bool `json:"isOnline"`
Latency madmin.LatencyStat `json:"latency"`
// replication metrics across buckets roll up
ReplicatedSize int64 `json:"replicatedSize"`
// Total number of completed operations
ReplicatedCount int64 `json:"replicatedCount"`
// Failed captures replication errors in various time windows
Failed madmin.TimedErrStats `json:"failed,omitempty"`
XferStats map[RMetricName]XferStats `json:"transferSummary"`
}
// SRMetricsSummary captures summary of replication counts across buckets on site
// along with op metrics rollup.
type SRMetricsSummary struct {
// op metrics roll up
ActiveWorkers ActiveWorkerStat `json:"activeWorkers"`
// Total Replica size in bytes
ReplicaSize int64 `json:"replicaSize"`
// Total number of replica received
ReplicaCount int64 `json:"replicaCount"`
// Queued operations
Queued InQueueMetric `json:"queued"`
// replication metrics summary for each site replication peer
Metrics map[string]SRMetric `json:"replMetrics"`
// uptime of node being queried for site replication metrics
Uptime int64 `json:"uptime"`
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,688 @@
package cmd
// Code generated by github.com/tinylib/msgp DO NOT EDIT.
import (
"bytes"
"testing"
"github.com/tinylib/msgp/msgp"
)
func TestMarshalUnmarshalRStat(t *testing.T) {
v := RStat{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgRStat(b *testing.B) {
v := RStat{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgRStat(b *testing.B) {
v := RStat{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalRStat(b *testing.B) {
v := RStat{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeRStat(t *testing.T) {
v := RStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeRStat Msgsize() is inaccurate")
}
vn := RStat{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeRStat(b *testing.B) {
v := RStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeRStat(b *testing.B) {
v := RStat{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalRTimedMetrics(t *testing.T) {
v := RTimedMetrics{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgRTimedMetrics(b *testing.B) {
v := RTimedMetrics{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgRTimedMetrics(b *testing.B) {
v := RTimedMetrics{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalRTimedMetrics(b *testing.B) {
v := RTimedMetrics{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeRTimedMetrics(t *testing.T) {
v := RTimedMetrics{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeRTimedMetrics Msgsize() is inaccurate")
}
vn := RTimedMetrics{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeRTimedMetrics(b *testing.B) {
v := RTimedMetrics{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeRTimedMetrics(b *testing.B) {
v := RTimedMetrics{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalSRMetric(t *testing.T) {
v := SRMetric{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgSRMetric(b *testing.B) {
v := SRMetric{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgSRMetric(b *testing.B) {
v := SRMetric{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalSRMetric(b *testing.B) {
v := SRMetric{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeSRMetric(t *testing.T) {
v := SRMetric{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeSRMetric Msgsize() is inaccurate")
}
vn := SRMetric{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeSRMetric(b *testing.B) {
v := SRMetric{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeSRMetric(b *testing.B) {
v := SRMetric{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalSRMetricsSummary(t *testing.T) {
v := SRMetricsSummary{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgSRMetricsSummary(b *testing.B) {
v := SRMetricsSummary{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgSRMetricsSummary(b *testing.B) {
v := SRMetricsSummary{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalSRMetricsSummary(b *testing.B) {
v := SRMetricsSummary{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeSRMetricsSummary(t *testing.T) {
v := SRMetricsSummary{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeSRMetricsSummary Msgsize() is inaccurate")
}
vn := SRMetricsSummary{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeSRMetricsSummary(b *testing.B) {
v := SRMetricsSummary{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeSRMetricsSummary(b *testing.B) {
v := SRMetricsSummary{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalSRStats(t *testing.T) {
v := SRStats{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgSRStats(b *testing.B) {
v := SRStats{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgSRStats(b *testing.B) {
v := SRStats{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalSRStats(b *testing.B) {
v := SRStats{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeSRStats(t *testing.T) {
v := SRStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeSRStats Msgsize() is inaccurate")
}
vn := SRStats{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeSRStats(b *testing.B) {
v := SRStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeSRStats(b *testing.B) {
v := SRStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalSRStatus(t *testing.T) {
v := SRStatus{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgSRStatus(b *testing.B) {
v := SRStatus{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgSRStatus(b *testing.B) {
v := SRStatus{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalSRStatus(b *testing.B) {
v := SRStatus{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeSRStatus(t *testing.T) {
v := SRStatus{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeSRStatus Msgsize() is inaccurate")
}
vn := SRStatus{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeSRStatus(b *testing.B) {
v := SRStatus{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeSRStatus(b *testing.B) {
v := SRStatus{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}

View File

@ -989,6 +989,7 @@ func (c *SiteReplicationSys) PeerBucketConfigureReplHandler(ctx context.Context,
Type: madmin.ReplicationService,
Region: "",
ReplicationSync: peer.SyncState == madmin.SyncEnabled,
DeploymentID: d,
}
var exists bool // true if ARN already exists
bucketTarget.Arn, exists = globalBucketTargetSys.getRemoteARN(bucket, &bucketTarget, peer.DeploymentID)
@ -2511,6 +2512,7 @@ func (c *SiteReplicationSys) SiteReplicationStatus(ctx context.Context, objAPI O
MaxPolicies: sinfo.MaxPolicies,
Sites: sinfo.Sites,
StatsSummary: sinfo.StatsSummary,
Metrics: sinfo.Metrics,
}
info.BucketStats = make(map[string]map[string]madmin.SRBucketStatsSummary, len(sinfo.Sites))
info.PolicyStats = make(map[string]map[string]madmin.SRPolicyStatsSummary)
@ -2617,7 +2619,6 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
},
replicationStatus,
)
if err := errors.Unwrap(metaInfoConcErr); err != nil {
return info, errSRBackendIssue(err)
}
@ -3093,6 +3094,14 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
}
}
if opts.Metrics {
m, err := globalSiteReplicationSys.getSiteMetrics(ctx)
if err != nil {
return info, err
}
info.Metrics = m
}
// maximum buckets users etc seen across sites
info.MaxBuckets = len(bucketStats)
info.MaxUsers = len(userInfoStats)
@ -3846,6 +3855,7 @@ type srStatusInfo struct {
// GroupStats map of group to slice of deployment IDs with stats. This is populated only if there are
// mismatches or if a specific bucket's stats are requested
GroupStats map[string]map[string]srGroupStatsSummary
Metrics madmin.SRMetricsSummary
}
// SRBucketDeleteOp - type of delete op
@ -5385,3 +5395,88 @@ func saveSiteResyncMetadata(ctx context.Context, ss SiteResyncStatus, objectAPI
func getSRResyncFilePath(dID string) string {
return pathJoin(siteResyncPrefix, dID+".meta")
}
func (c *SiteReplicationSys) getDeplIDForEndpoint(ep string) (dID string, err error) {
if ep == "" {
return dID, fmt.Errorf("no deployment id found for endpoint %s", ep)
}
c.RLock()
defer c.RUnlock()
if !c.enabled {
return dID, errSRNotEnabled
}
for _, peer := range c.state.Peers {
if ep == peer.Endpoint {
return peer.DeploymentID, nil
}
}
return dID, fmt.Errorf("no deployment id found for endpoint %s", ep)
}
func (c *SiteReplicationSys) getSiteMetrics(ctx context.Context) (madmin.SRMetricsSummary, error) {
if !c.isEnabled() {
return madmin.SRMetricsSummary{}, errSRNotEnabled
}
peerSMetricsList := globalNotificationSys.GetClusterSiteMetrics(ctx)
var sm madmin.SRMetricsSummary
sm.Metrics = make(map[string]madmin.SRMetric)
for _, peer := range peerSMetricsList {
sm.ActiveWorkers.Avg += peer.ActiveWorkers.Avg
sm.ActiveWorkers.Curr += peer.ActiveWorkers.Curr
if peer.ActiveWorkers.Max > sm.ActiveWorkers.Max {
sm.ActiveWorkers.Max += peer.ActiveWorkers.Max
}
sm.Queued.Avg.Bytes += peer.Queued.Avg.Bytes
sm.Queued.Avg.Count += peer.Queued.Avg.Count
sm.Queued.Curr.Bytes += peer.Queued.Curr.Bytes
sm.Queued.Curr.Count += peer.Queued.Curr.Count
if peer.Queued.Max.Count > sm.Queued.Max.Count {
sm.Queued.Max.Bytes = peer.Queued.Max.Bytes
sm.Queued.Max.Count = peer.Queued.Max.Count
}
sm.ReplicaCount += peer.ReplicaCount
sm.ReplicaSize += peer.ReplicaSize
for dID, v := range peer.Metrics {
v2, ok := sm.Metrics[dID]
if !ok {
v2 = madmin.SRMetric{}
v2.Failed.ErrCounts = make(map[string]int)
}
// use target endpoint metrics from node which has been up the longest
if v2.LastOnline.After(v.LastOnline) || v2.LastOnline.IsZero() {
v2.Endpoint = v.Endpoint
v2.LastOnline = v.LastOnline
v2.Latency = v.Latency
v2.Online = v.Online
v2.TotalDowntime = v.TotalDowntime
v2.DeploymentID = v.DeploymentID
}
v2.ReplicatedCount += v.ReplicatedCount
v2.ReplicatedSize += v.ReplicatedSize
v2.Failed = v2.Failed.Add(v.Failed)
for k, v := range v.Failed.ErrCounts {
v2.Failed.ErrCounts[k] += v
}
if v2.XferStats == nil {
v2.XferStats = make(map[replication.MetricName]replication.XferStats)
}
for rm, x := range v.XferStats {
x2, ok := v2.XferStats[replication.MetricName(rm)]
if !ok {
x2 = replication.XferStats{}
}
x2.AvgRate += x.Avg
x2.CurrRate += x.Curr
if x.Peak > x2.PeakRate {
x2.PeakRate = x.Peak
}
v2.XferStats[replication.MetricName(rm)] = x2
}
sm.Metrics[dID] = v2
}
}
sm.Uptime = UTCNow().Unix() - globalBootTime.Unix()
return sm, nil
}

View File

@ -619,6 +619,7 @@ func (s *xlStorage) NSScanner(ctx context.Context, cache dataUsageCache, updates
}
for tgt, st := range sizeS.replTargetStats {
res["repl-size-"+tgt] = strconv.FormatInt(st.replicatedSize, 10)
res["repl-count-"+tgt] = strconv.FormatInt(st.replicatedCount, 10)
if st.failedCount > 0 {
res["repl-failed-"+tgt] = fmt.Sprintf("%d versions, %d bytes", st.failedCount, st.failedSize)
}

View File

@ -4,6 +4,9 @@ Each metric includes a label for the server that calculated the metric. Each met
These metrics can be obtained from any MinIO server once per collection.
The replication metrics marked with * are only relevant for site replication, where metrics are published at the cluster level and not at bucket level. If bucket
replication is in use, these metrics are exported at the bucket level.
| Name | Description |
|:----------------------------------------------|:----------------------------------------------------------------------------------------------------------------|
| `minio_audit_failed_messages` | Total number of messages that failed to send since start. |
@ -42,7 +45,38 @@ These metrics can be obtained from any MinIO server once per collection.
| `minio_cluster_nodes_online_total` | Total number of MinIO nodes online. |
| `minio_cluster_write_quorum` | Maximum write quorum across all pools and sets |
| `minio_cluster_health_status` | Get current cluster health status |
| `minio_cluster_replication_current_active_workers` | Total number of active replication workers |
| `minio_cluster_replication_average_active_workers` | Average number of active replication workers |
| `minio_cluster_replication_max_active_workers` | Maximum number of active replication workers seen since server start |
| `minio_cluster_replication_link_online` | Reports whether the replication link is online (1) or offline (0). |
| `minio_cluster_replication_link_offline_duration_seconds` | Total duration of replication link being offline in seconds since last offline event|
| `minio_cluster_replication_link_downtime_duration_seconds` | Total downtime of replication link in seconds since server start|
| `minio_cluster_replication_average_link_latency_ms` | Average replication link latency in milliseconds |
| `minio_cluster_replication_max_link_latency_ms` | Maximum replication link latency in milliseconds seen since server start |
| `minio_cluster_replication_current_link_latency_ms` | Current replication link latency in milliseconds |
| `minio_cluster_replication_current_transfer_rate` | Current replication transfer rate in bytes/sec |
| `minio_cluster_replication_average_transfer_rate` | Average replication transfer rate in bytes/sec |
| `minio_cluster_replication_max_transfer_rate` | Maximum replication transfer rate in bytes/sec seen since server start |
| `minio_cluster_replication_last_minute_queued_count` | Total number of objects queued for replication in the last full minute |
| `minio_cluster_replication_last_minute_queued_bytes` | Total number of bytes queued for replication in the last full minute |
| `minio_cluster_replication_average_queued_count` | Average number of objects queued for replication since server start |
| `minio_cluster_replication_average_queued_bytes` | Average number of bytes queued for replication since server start |
| `minio_cluster_replication_max_queued_bytes` | Maximum number of bytes queued for replication seen since server start |
| `minio_cluster_replication_max_queued_count` | Maximum number of objects queued for replication seen since server start |
| `minio_cluster_replication_recent_backlog_count` | Total number of objects seen in replication backlog in the last 5 minutes |
| `minio_heal_objects_errors_total` | Objects for which healing failed in current self healing run. |
| `minio_cluster_replication_last_minute_failed_bytes` | Total number of bytes failed at least once to replicate in the last full minute. |
| `minio_cluster_replication_last_minute_failed_count` | Total number of objects which failed replication in the last full minute. |
| `minio_cluster_replication_last_hour_failed_bytes` | * Total number of bytes failed at least once to replicate in the last full hour. |
| `minio_cluster_replication_last_hour_failed_count` | * Total number of objects which failed replication in the last full hour. |
| `minio_cluster_replication_total_failed_bytes` | * Total number of bytes failed at least once to replicate since server start. |
| `minio_cluster_replication_total_failed_count` | * Total number of objects which failed replication since server start. |
| `minio_cluster_replication_received_bytes` | * Total number of bytes replicated to this cluster from another source cluster. |
| `minio_cluster_replication_received_count` | * Total number of objects received by this cluster from another source cluster. |
| `minio_cluster_replication_sent_bytes` | * Total number of bytes replicated to the target cluster. | |
| `minio_cluster_replication_sent_count` | * Total number of objects replicated to the target cluster. | |
| `minio_cluster_replication_credential_errors` | * Total number of replication credential errors since server start |
| `minio_heal_objects_heal_total` | Objects healed in current self healing run. |
| `minio_heal_objects_total` | Objects scanned in current self healing run. |
| `minio_heal_time_last_activity_nano_seconds` | Time elapsed (in nano seconds) since last self healing activity. |
@ -118,11 +152,17 @@ These metrics can be obtained from any MinIO server once per collection.
| `minio_bucket_objects_size_distribution` | Distribution of object sizes in the bucket, includes label for the bucket name. |
| `minio_bucket_objects_version_distribution` | Distribution of object sizes in a bucket, by number of versions |
| `minio_bucket_quota_total_bytes` | Total bucket quota size in bytes. |
| `minio_bucket_replication_failed_bytes` | Total number of bytes failed at least once to replicate. |
| `minio_bucket_replication_failed_count` | Total number of objects which failed replication. |
| `minio_bucket_replication_last_minute_failed_bytes` | Total number of bytes failed at least once to replicate in the last full minute. |
| `minio_bucket_replication_last_minute_failed_count` | Total number of objects which failed replication in the last full minute. |
| `minio_bucket_replication_last_hour_failed_bytes` | Total number of bytes failed at least once to replicate in the last full hour. |
| `minio_bucket_replication_last_hour_failed_count` | Total number of objects which failed replication in the last full hour. |
| `minio_bucket_replication_total_failed_bytes` | Total number of bytes failed at least once to replicate since server start. |
| `minio_bucket_replication_total_failed_count` | Total number of objects which failed replication since server start. |
| `minio_bucket_replication_latency_ms` | Replication latency in milliseconds. |
| `minio_bucket_replication_received_bytes` | Total number of bytes replicated to this bucket from another source bucket. |
| `minio_bucket_replication_sent_bytes` | Total number of bytes replicated to the target bucket. |
| `minio_bucket_replication_received_count` | Total number of objects received by this bucket from another source bucket. |
| `minio_bucket_replication_sent_bytes` | Total number of bytes replicated to the target bucket. | |
| `minio_bucket_replication_sent_count` | Total number of objects replicated to the target bucket. | |
| `minio_bucket_traffic_received_bytes` | Total number of S3 bytes received for this bucket. |
| `minio_bucket_traffic_sent_bytes` | Total number of S3 bytes sent for this bucket. |
| `minio_bucket_usage_object_total` | Total number of objects. |
@ -135,3 +175,4 @@ These metrics can be obtained from any MinIO server once per collection.
| `minio_bucket_requests_total` | Total number of S3 requests on a bucket. |
| `minio_bucket_requests_canceled_total` | Total number S3 requests canceled by the client. |
| `minio_bucket_requests_ttfb_seconds_distribution` | Distribution of time to first byte across API calls per bucket. |
| `minio_bucket_replication_credential_errors` | Total number of replication credential errors since server start |

12
go.mod
View File

@ -49,7 +49,7 @@ require (
github.com/minio/dperf v0.5.0
github.com/minio/highwayhash v1.0.2
github.com/minio/kes-go v0.2.0
github.com/minio/madmin-go/v3 v3.0.11
github.com/minio/madmin-go/v3 v3.0.17
github.com/minio/minio-go/v7 v7.0.63
github.com/minio/mux v1.9.0
github.com/minio/pkg v1.7.5
@ -74,9 +74,10 @@ require (
github.com/prometheus/common v0.44.0
github.com/prometheus/procfs v0.11.1
github.com/rabbitmq/amqp091-go v1.8.1
github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475
github.com/rs/cors v1.9.0
github.com/secure-io/sio-go v0.3.1
github.com/shirou/gopsutil/v3 v3.23.6
github.com/shirou/gopsutil/v3 v3.23.7
github.com/tidwall/gjson v1.15.0
github.com/tinylib/msgp v1.1.8
github.com/valyala/bytebufferpool v1.0.0
@ -204,7 +205,6 @@ require (
github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect
github.com/pquerna/cachecontrol v0.2.0 // indirect
github.com/prometheus/prom2json v1.3.3 // indirect
github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect
github.com/rivo/tview v0.0.0-20230621164836-6cc0565babaf // indirect
github.com/rivo/uniseg v0.4.4 // indirect
github.com/rjeczalik/notify v0.9.3 // indirect
@ -230,9 +230,9 @@ require (
golang.org/x/tools v0.11.0 // indirect
golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
google.golang.org/appengine v1.6.7 // indirect
google.golang.org/genproto v0.0.0-20230726155614-23370e0ffb3e // indirect
google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e // indirect
google.golang.org/genproto/googleapis/rpc v0.0.0-20230726155614-23370e0ffb3e // indirect
google.golang.org/genproto v0.0.0-20230731193218-e0aa005b6bdf // indirect
google.golang.org/genproto/googleapis/api v0.0.0-20230731193218-e0aa005b6bdf // indirect
google.golang.org/genproto/googleapis/rpc v0.0.0-20230731193218-e0aa005b6bdf // indirect
google.golang.org/grpc v1.57.0 // indirect
google.golang.org/protobuf v1.31.0 // indirect
gopkg.in/h2non/filetype.v1 v1.0.5 // indirect

20
go.sum
View File

@ -484,8 +484,8 @@ github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA
github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY=
github.com/minio/kes-go v0.2.0 h1:HA33arq9s3MErbsj3PAXFVfFo4U4yw7lTKQ5kWFrpCA=
github.com/minio/kes-go v0.2.0/go.mod h1:VorHLaIYis9/MxAHAtXN4d8PUMNKhIxTIlvFt0hBOEo=
github.com/minio/madmin-go/v3 v3.0.11 h1:7QrZkgbQ5+qTKGy6Nok2A8OgLAcn/lcMYYuSgiZrgBE=
github.com/minio/madmin-go/v3 v3.0.11/go.mod h1:DMXyWO670OXwZNN0v4ZrEodl9oLOcaPJIZhpoHpu7aw=
github.com/minio/madmin-go/v3 v3.0.17 h1:fm3TIMK7hxCpgztO2GHN6Lxq66zYPWlmrS8M9YWvoSE=
github.com/minio/madmin-go/v3 v3.0.17/go.mod h1:B2EgtEGrfWx+AkXv+OAcS6IHwoIJcd1p75QfDPSPd6Q=
github.com/minio/mc v0.0.0-20230726035150-6b8680a2f7ca h1:y/dJJuWhlVYN9tqBxoHvGCpJ7olVuPa1whg+GS1m1a8=
github.com/minio/mc v0.0.0-20230726035150-6b8680a2f7ca/go.mod h1:9x/wxYmFZCP+FMVZe57igzxENRGNOFBrucj3m4NqByg=
github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34=
@ -651,8 +651,8 @@ github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR
github.com/scylladb/termtables v0.0.0-20191203121021-c4c0b6d42ff4/go.mod h1:C1a7PQSMz9NShzorzCiG2fk9+xuCgLkPeCvMHYR2OWg=
github.com/secure-io/sio-go v0.3.1 h1:dNvY9awjabXTYGsTF1PiCySl9Ltofk9GA3VdWlo7rRc=
github.com/secure-io/sio-go v0.3.1/go.mod h1:+xbkjDzPjwh4Axd07pRKSNriS9SCiYksWnZqdnfpQxs=
github.com/shirou/gopsutil/v3 v3.23.6 h1:5y46WPI9QBKBbK7EEccUPNXpJpNrvPuTD0O2zHEHT08=
github.com/shirou/gopsutil/v3 v3.23.6/go.mod h1:j7QX50DrXYggrpN30W0Mo+I4/8U2UUIQrnrhqUeWrAU=
github.com/shirou/gopsutil/v3 v3.23.7 h1:C+fHO8hfIppoJ1WdsVm1RoI0RwXoNdfTK7yWXV0wVj4=
github.com/shirou/gopsutil/v3 v3.23.7/go.mod h1:c4gnmoRC0hQuaLqvxnx1//VXQ0Ms/X9UnJF8pddY5z4=
github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM=
github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ=
github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU=
@ -965,12 +965,12 @@ google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoA
google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
google.golang.org/genproto v0.0.0-20230726155614-23370e0ffb3e h1:xIXmWJ303kJCuogpj0bHq+dcjcZHU+XFyc1I0Yl9cRg=
google.golang.org/genproto v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:0ggbjUrZYpy1q+ANUS30SEoGZ53cdfwtbuG7Ptgy108=
google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e h1:z3vDksarJxsAKM5dmEGv0GHwE2hKJ096wZra71Vs4sw=
google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ=
google.golang.org/genproto/googleapis/rpc v0.0.0-20230726155614-23370e0ffb3e h1:S83+ibolgyZ0bqz7KEsUOPErxcv4VzlszxY+31OfB/E=
google.golang.org/genproto/googleapis/rpc v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:TUfxEVdsvPg18p6AslUXFoLdpED4oBnGwyqk3dV1XzM=
google.golang.org/genproto v0.0.0-20230731193218-e0aa005b6bdf h1:v5Cf4E9+6tawYrs/grq1q1hFpGtzlGFzgWHqwt6NFiU=
google.golang.org/genproto v0.0.0-20230731193218-e0aa005b6bdf/go.mod h1:oH/ZOT02u4kWEp7oYBGYFFkCdKS/uYR9Z7+0/xuuFp8=
google.golang.org/genproto/googleapis/api v0.0.0-20230731193218-e0aa005b6bdf h1:xkVZ5FdZJF4U82Q/JS+DcZA83s/GRVL+QrFMlexk9Yo=
google.golang.org/genproto/googleapis/api v0.0.0-20230731193218-e0aa005b6bdf/go.mod h1:5DZzOUPCLYL3mNkQ0ms0F3EuUNZ7py1Bqeq6sxzI7/Q=
google.golang.org/genproto/googleapis/rpc v0.0.0-20230731193218-e0aa005b6bdf h1:guOdSPaeFgN+jEJwTo1dQ71hdBm+yKSCCKuTRkJzcVo=
google.golang.org/genproto/googleapis/rpc v0.0.0-20230731193218-e0aa005b6bdf/go.mod h1:zBEcrKX2ZOcEkHWxBPAIvYUWOKKMIhYcmNiUIu2ji3I=
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=