From b48bbe08b2b9c99ec900f6c1351cf67bca260476 Mon Sep 17 00:00:00 2001 From: Poorna Date: Wed, 30 Aug 2023 01:00:59 -0700 Subject: [PATCH] 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 --- cmd/admin-handlers-site-replication.go | 1 + cmd/api-router.go | 5 +- cmd/bucket-replication-handlers.go | 74 +- cmd/bucket-replication-metrics.go | 381 +++++ cmd/bucket-replication-metrics_gen.go | 1198 ++++++++++++++ cmd/bucket-replication-metrics_gen_test.go | 801 +++++++++ cmd/bucket-replication-stats.go | 542 +++--- cmd/bucket-replication-utils.go | 18 +- cmd/bucket-replication.go | 175 +- cmd/bucket-stats.go | 320 +++- cmd/bucket-stats_gen.go | 1496 +++++++++++++++-- cmd/bucket-stats_gen_test.go | 452 +++++ cmd/bucket-targets.go | 112 +- cmd/data-scanner.go | 16 +- cmd/data-usage-cache.go | 33 +- cmd/data-usage-cache_gen.go | 115 +- cmd/data-usage-utils.go | 2 + cmd/metrics-v2.go | 805 +++++++-- cmd/metrics.go | 25 +- cmd/notification.go | 30 + cmd/peer-rest-client.go | 13 + cmd/peer-rest-common.go | 3 +- cmd/peer-rest-server.go | 18 + cmd/site-replication-metrics.go | 289 ++++ cmd/site-replication-metrics_gen.go | 1733 ++++++++++++++++++++ cmd/site-replication-metrics_gen_test.go | 688 ++++++++ cmd/site-replication.go | 97 +- cmd/xl-storage.go | 1 + docs/metrics/prometheus/list.md | 47 +- go.mod | 12 +- go.sum | 20 +- 31 files changed, 8779 insertions(+), 743 deletions(-) create mode 100644 cmd/bucket-replication-metrics.go create mode 100644 cmd/bucket-replication-metrics_gen.go create mode 100644 cmd/bucket-replication-metrics_gen_test.go create mode 100644 cmd/site-replication-metrics.go create mode 100644 cmd/site-replication-metrics_gen.go create mode 100644 cmd/site-replication-metrics_gen_test.go diff --git a/cmd/admin-handlers-site-replication.go b/cmd/admin-handlers-site-replication.go index 9d07453ba..726888e85 100644 --- a/cmd/admin-handlers-site-replication.go +++ b/cmd/admin-handlers-site-replication.go @@ -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 } diff --git a/cmd/api-router.go b/cmd/api-router.go index d4e118e1c..accf48891 100644 --- a/cmd/api-router.go +++ b/cmd/api-router.go @@ -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", "") diff --git a/cmd/bucket-replication-handlers.go b/cmd/bucket-replication-handlers.go index caa54978c..0b7fa3d34 100644 --- a/cmd/bucket-replication-handlers.go +++ b/cmd/bucket-replication-handlers.go @@ -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 diff --git a/cmd/bucket-replication-metrics.go b/cmd/bucket-replication-metrics.go new file mode 100644 index 000000000..74c0c8c23 --- /dev/null +++ b/cmd/bucket-replication-metrics.go @@ -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 . + +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 +) diff --git a/cmd/bucket-replication-metrics_gen.go b/cmd/bucket-replication-metrics_gen.go new file mode 100644 index 000000000..c67a3c097 --- /dev/null +++ b/cmd/bucket-replication-metrics_gen.go @@ -0,0 +1,1198 @@ +package cmd + +// Code generated by github.com/tinylib/msgp DO NOT EDIT. + +import ( + "github.com/tinylib/msgp/msgp" +) + +// DecodeMsg implements msgp.Decodable +func (z *ActiveWorkerStat) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Curr": + z.Curr, err = dc.ReadInt() + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + case "Avg": + z.Avg, err = dc.ReadFloat32() + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + case "Max": + z.Max, err = dc.ReadInt() + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z ActiveWorkerStat) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 3 + // write "Curr" + err = en.Append(0x83, 0xa4, 0x43, 0x75, 0x72, 0x72) + if err != nil { + return + } + err = en.WriteInt(z.Curr) + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + // write "Avg" + err = en.Append(0xa3, 0x41, 0x76, 0x67) + if err != nil { + return + } + err = en.WriteFloat32(z.Avg) + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + // write "Max" + err = en.Append(0xa3, 0x4d, 0x61, 0x78) + if err != nil { + return + } + err = en.WriteInt(z.Max) + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z ActiveWorkerStat) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 3 + // string "Curr" + o = append(o, 0x83, 0xa4, 0x43, 0x75, 0x72, 0x72) + o = msgp.AppendInt(o, z.Curr) + // string "Avg" + o = append(o, 0xa3, 0x41, 0x76, 0x67) + o = msgp.AppendFloat32(o, z.Avg) + // string "Max" + o = append(o, 0xa3, 0x4d, 0x61, 0x78) + o = msgp.AppendInt(o, z.Max) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ActiveWorkerStat) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Curr": + z.Curr, bts, err = msgp.ReadIntBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + case "Avg": + z.Avg, bts, err = msgp.ReadFloat32Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + case "Max": + z.Max, bts, err = msgp.ReadIntBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z ActiveWorkerStat) Msgsize() (s int) { + s = 1 + 5 + msgp.IntSize + 4 + msgp.Float32Size + 4 + msgp.IntSize + return +} + +// DecodeMsg implements msgp.Decodable +func (z *InQueueMetric) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "cq": + var zb0002 uint32 + zb0002, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + for zb0002 > 0 { + zb0002-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Curr.Count, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Curr", "Count") + return + } + case "Bytes": + z.Curr.Bytes, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Curr", "Bytes") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + } + } + case "aq": + var zb0003 uint32 + zb0003, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + for zb0003 > 0 { + zb0003-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Avg.Count, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Avg", "Count") + return + } + case "Bytes": + z.Avg.Bytes, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Avg", "Bytes") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + } + } + case "pq": + var zb0004 uint32 + zb0004, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + for zb0004 > 0 { + zb0004-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Max.Count, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Max", "Count") + return + } + case "Bytes": + z.Max.Bytes, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Max", "Bytes") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + } + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *InQueueMetric) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 3 + // write "cq" + err = en.Append(0x83, 0xa2, 0x63, 0x71) + if err != nil { + return + } + // map header, size 2 + // write "Count" + err = en.Append(0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteFloat64(z.Curr.Count) + if err != nil { + err = msgp.WrapError(err, "Curr", "Count") + return + } + // write "Bytes" + err = en.Append(0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteFloat64(z.Curr.Bytes) + if err != nil { + err = msgp.WrapError(err, "Curr", "Bytes") + return + } + // write "aq" + err = en.Append(0xa2, 0x61, 0x71) + if err != nil { + return + } + // map header, size 2 + // write "Count" + err = en.Append(0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteFloat64(z.Avg.Count) + if err != nil { + err = msgp.WrapError(err, "Avg", "Count") + return + } + // write "Bytes" + err = en.Append(0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteFloat64(z.Avg.Bytes) + if err != nil { + err = msgp.WrapError(err, "Avg", "Bytes") + return + } + // write "pq" + err = en.Append(0xa2, 0x70, 0x71) + if err != nil { + return + } + // map header, size 2 + // write "Count" + err = en.Append(0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteFloat64(z.Max.Count) + if err != nil { + err = msgp.WrapError(err, "Max", "Count") + return + } + // write "Bytes" + err = en.Append(0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteFloat64(z.Max.Bytes) + if err != nil { + err = msgp.WrapError(err, "Max", "Bytes") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *InQueueMetric) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 3 + // string "cq" + o = append(o, 0x83, 0xa2, 0x63, 0x71) + // map header, size 2 + // string "Count" + o = append(o, 0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendFloat64(o, z.Curr.Count) + // string "Bytes" + o = append(o, 0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + o = msgp.AppendFloat64(o, z.Curr.Bytes) + // string "aq" + o = append(o, 0xa2, 0x61, 0x71) + // map header, size 2 + // string "Count" + o = append(o, 0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendFloat64(o, z.Avg.Count) + // string "Bytes" + o = append(o, 0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + o = msgp.AppendFloat64(o, z.Avg.Bytes) + // string "pq" + o = append(o, 0xa2, 0x70, 0x71) + // map header, size 2 + // string "Count" + o = append(o, 0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendFloat64(o, z.Max.Count) + // string "Bytes" + o = append(o, 0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + o = msgp.AppendFloat64(o, z.Max.Bytes) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *InQueueMetric) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "cq": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + for zb0002 > 0 { + zb0002-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Curr.Count, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Curr", "Count") + return + } + case "Bytes": + z.Curr.Bytes, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Curr", "Bytes") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + } + } + case "aq": + var zb0003 uint32 + zb0003, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + for zb0003 > 0 { + zb0003-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Avg.Count, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Avg", "Count") + return + } + case "Bytes": + z.Avg.Bytes, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Avg", "Bytes") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + } + } + case "pq": + var zb0004 uint32 + zb0004, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + for zb0004 > 0 { + zb0004-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Max.Count, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Max", "Count") + return + } + case "Bytes": + z.Max.Bytes, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Max", "Bytes") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err, "Max") + return + } + } + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *InQueueMetric) Msgsize() (s int) { + s = 1 + 3 + 1 + 6 + msgp.Float64Size + 6 + msgp.Float64Size + 3 + 1 + 6 + msgp.Float64Size + 6 + msgp.Float64Size + 3 + 1 + 6 + msgp.Float64Size + 6 + msgp.Float64Size + return +} + +// DecodeMsg implements msgp.Decodable +func (z *InQueueStats) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z InQueueStats) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 0 + err = en.Append(0x80) + if err != nil { + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z InQueueStats) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 0 + o = append(o, 0x80) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *InQueueStats) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z InQueueStats) Msgsize() (s int) { + s = 1 + return +} + +// DecodeMsg implements msgp.Decodable +func (z *QStat) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Count, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Count") + return + } + case "Bytes": + z.Bytes, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Bytes") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z QStat) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 2 + // write "Count" + err = en.Append(0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteFloat64(z.Count) + if err != nil { + err = msgp.WrapError(err, "Count") + return + } + // write "Bytes" + err = en.Append(0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteFloat64(z.Bytes) + if err != nil { + err = msgp.WrapError(err, "Bytes") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z QStat) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 2 + // string "Count" + o = append(o, 0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendFloat64(o, z.Count) + // string "Bytes" + o = append(o, 0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + o = msgp.AppendFloat64(o, z.Bytes) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *QStat) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Count, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Count") + return + } + case "Bytes": + z.Bytes, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Bytes") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z QStat) Msgsize() (s int) { + s = 1 + 6 + msgp.Float64Size + 6 + msgp.Float64Size + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ReplicationMRFStats) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "LastFailedCount": + z.LastFailedCount, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "LastFailedCount") + return + } + case "TotalDroppedCount": + z.TotalDroppedCount, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "TotalDroppedCount") + return + } + case "TotalDroppedBytes": + z.TotalDroppedBytes, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "TotalDroppedBytes") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z ReplicationMRFStats) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 3 + // write "LastFailedCount" + err = en.Append(0x83, 0xaf, 0x4c, 0x61, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteUint64(z.LastFailedCount) + if err != nil { + err = msgp.WrapError(err, "LastFailedCount") + return + } + // write "TotalDroppedCount" + err = en.Append(0xb1, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteUint64(z.TotalDroppedCount) + if err != nil { + err = msgp.WrapError(err, "TotalDroppedCount") + return + } + // write "TotalDroppedBytes" + err = en.Append(0xb1, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x42, 0x79, 0x74, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteUint64(z.TotalDroppedBytes) + if err != nil { + err = msgp.WrapError(err, "TotalDroppedBytes") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z ReplicationMRFStats) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 3 + // string "LastFailedCount" + o = append(o, 0x83, 0xaf, 0x4c, 0x61, 0x73, 0x74, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendUint64(o, z.LastFailedCount) + // string "TotalDroppedCount" + o = append(o, 0xb1, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendUint64(o, z.TotalDroppedCount) + // string "TotalDroppedBytes" + o = append(o, 0xb1, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x42, 0x79, 0x74, 0x65, 0x73) + o = msgp.AppendUint64(o, z.TotalDroppedBytes) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ReplicationMRFStats) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "LastFailedCount": + z.LastFailedCount, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "LastFailedCount") + return + } + case "TotalDroppedCount": + z.TotalDroppedCount, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "TotalDroppedCount") + return + } + case "TotalDroppedBytes": + z.TotalDroppedBytes, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "TotalDroppedBytes") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z ReplicationMRFStats) Msgsize() (s int) { + s = 1 + 16 + msgp.Uint64Size + 18 + msgp.Uint64Size + 18 + msgp.Uint64Size + return +} + +// DecodeMsg implements msgp.Decodable +func (z *SMA) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "CAvg": + z.CAvg, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "CAvg") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z SMA) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 1 + // write "CAvg" + err = en.Append(0x81, 0xa4, 0x43, 0x41, 0x76, 0x67) + if err != nil { + return + } + err = en.WriteFloat64(z.CAvg) + if err != nil { + err = msgp.WrapError(err, "CAvg") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z SMA) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 1 + // string "CAvg" + o = append(o, 0x81, 0xa4, 0x43, 0x41, 0x76, 0x67) + o = msgp.AppendFloat64(o, z.CAvg) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *SMA) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "CAvg": + z.CAvg, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "CAvg") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z SMA) Msgsize() (s int) { + s = 1 + 5 + msgp.Float64Size + return +} + +// DecodeMsg implements msgp.Decodable +func (z *XferStats) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "cr": + z.Curr, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + case "av": + z.Avg, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + case "p": + z.Peak, err = dc.ReadFloat64() + if err != nil { + err = msgp.WrapError(err, "Peak") + return + } + case "n": + z.N, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "N") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *XferStats) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 4 + // write "cr" + err = en.Append(0x84, 0xa2, 0x63, 0x72) + if err != nil { + return + } + err = en.WriteFloat64(z.Curr) + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + // write "av" + err = en.Append(0xa2, 0x61, 0x76) + if err != nil { + return + } + err = en.WriteFloat64(z.Avg) + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + // write "p" + err = en.Append(0xa1, 0x70) + if err != nil { + return + } + err = en.WriteFloat64(z.Peak) + if err != nil { + err = msgp.WrapError(err, "Peak") + return + } + // write "n" + err = en.Append(0xa1, 0x6e) + if err != nil { + return + } + err = en.WriteInt64(z.N) + if err != nil { + err = msgp.WrapError(err, "N") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *XferStats) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 4 + // string "cr" + o = append(o, 0x84, 0xa2, 0x63, 0x72) + o = msgp.AppendFloat64(o, z.Curr) + // string "av" + o = append(o, 0xa2, 0x61, 0x76) + o = msgp.AppendFloat64(o, z.Avg) + // string "p" + o = append(o, 0xa1, 0x70) + o = msgp.AppendFloat64(o, z.Peak) + // string "n" + o = append(o, 0xa1, 0x6e) + o = msgp.AppendInt64(o, z.N) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *XferStats) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "cr": + z.Curr, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Curr") + return + } + case "av": + z.Avg, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Avg") + return + } + case "p": + z.Peak, bts, err = msgp.ReadFloat64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Peak") + return + } + case "n": + z.N, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "N") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *XferStats) Msgsize() (s int) { + s = 1 + 3 + msgp.Float64Size + 3 + msgp.Float64Size + 2 + msgp.Float64Size + 2 + msgp.Int64Size + return +} diff --git a/cmd/bucket-replication-metrics_gen_test.go b/cmd/bucket-replication-metrics_gen_test.go new file mode 100644 index 000000000..3401d7dba --- /dev/null +++ b/cmd/bucket-replication-metrics_gen_test.go @@ -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) + } + } +} diff --git a/cmd/bucket-replication-stats.go b/cmd/bucket-replication-stats.go index 36d22b4bc..ca65bb58e 100644 --- a/cmd/bucket-replication-stats.go +++ b/cmd/bucket-replication-stats.go @@ -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) } diff --git a/cmd/bucket-replication-utils.go b/cmd/bucket-replication-utils.go index 3024740be..1d623ae00 100644 --- a/cmd/bucket-replication-utils.go +++ b/cmd/bucket-replication-utils.go @@ -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 -) diff --git a/cmd/bucket-replication.go b/cmd/bucket-replication.go index e9d015ede..bc984f813 100644 --- a/cmd/bucket-replication.go +++ b/cmd/bucket-replication.go @@ -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) diff --git a/cmd/bucket-stats.go b/cmd/bucket-stats.go index 773b5a064..7c4b94e36 100644 --- a/cmd/bucket-stats.go +++ b/cmd/bucket-stats.go @@ -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"` } diff --git a/cmd/bucket-stats_gen.go b/cmd/bucket-stats_gen.go index 72f7fc342..85997dcf6 100644 --- a/cmd/bucket-stats_gen.go +++ b/cmd/bucket-stats_gen.go @@ -24,12 +24,6 @@ func (z *BucketReplicationStat) DecodeMsg(dc *msgp.Reader) (err error) { return } switch msgp.UnsafeString(field) { - case "PendingSize": - z.PendingSize, err = dc.ReadInt64() - if err != nil { - err = msgp.WrapError(err, "PendingSize") - return - } case "ReplicatedSize": z.ReplicatedSize, err = dc.ReadInt64() if err != nil { @@ -42,22 +36,22 @@ func (z *BucketReplicationStat) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "ReplicaSize") return } - case "FailedSize": - z.FailedSize, err = dc.ReadInt64() + case "FailStats": + err = z.FailStats.DecodeMsg(dc) if err != nil { - err = msgp.WrapError(err, "FailedSize") + err = msgp.WrapError(err, "FailStats") return } - case "PendingCount": - z.PendingCount, err = dc.ReadInt64() + case "Failed": + err = z.Failed.DecodeMsg(dc) if err != nil { - err = msgp.WrapError(err, "PendingCount") + err = msgp.WrapError(err, "Failed") return } - case "FailedCount": - z.FailedCount, err = dc.ReadInt64() + case "ReplicatedCount": + z.ReplicatedCount, err = dc.ReadInt64() if err != nil { - err = msgp.WrapError(err, "FailedCount") + err = msgp.WrapError(err, "ReplicatedCount") return } case "Latency": @@ -101,6 +95,66 @@ func (z *BucketReplicationStat) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "CurrentBandwidthInBytesPerSecond") return } + case "lt": + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + z.XferRateLrg = nil + } else { + if z.XferRateLrg == nil { + z.XferRateLrg = new(XferStats) + } + err = z.XferRateLrg.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + } + case "st": + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + z.XferRateSml = nil + } else { + if z.XferRateSml == nil { + z.XferRateSml = new(XferStats) + } + err = z.XferRateSml.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + } + case "PendingSize": + z.PendingSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } + case "FailedSize": + z.FailedSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "FailedSize") + return + } + case "PendingCount": + z.PendingCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "PendingCount") + return + } + case "FailedCount": + z.FailedCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "FailedCount") + return + } default: err = dc.Skip() if err != nil { @@ -114,19 +168,9 @@ func (z *BucketReplicationStat) DecodeMsg(dc *msgp.Reader) (err error) { // EncodeMsg implements msgp.Encodable func (z *BucketReplicationStat) EncodeMsg(en *msgp.Writer) (err error) { - // map header, size 9 - // write "PendingSize" - err = en.Append(0x89, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) - if err != nil { - return - } - err = en.WriteInt64(z.PendingSize) - if err != nil { - err = msgp.WrapError(err, "PendingSize") - return - } + // map header, size 14 // write "ReplicatedSize" - err = en.Append(0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + err = en.Append(0x8e, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) if err != nil { return } @@ -145,34 +189,34 @@ func (z *BucketReplicationStat) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "ReplicaSize") return } - // write "FailedSize" - err = en.Append(0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + // write "FailStats" + err = en.Append(0xa9, 0x46, 0x61, 0x69, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x73) if err != nil { return } - err = en.WriteInt64(z.FailedSize) + err = z.FailStats.EncodeMsg(en) if err != nil { - err = msgp.WrapError(err, "FailedSize") + err = msgp.WrapError(err, "FailStats") return } - // write "PendingCount" - err = en.Append(0xac, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74) + // write "Failed" + err = en.Append(0xa6, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64) if err != nil { return } - err = en.WriteInt64(z.PendingCount) + err = z.Failed.EncodeMsg(en) if err != nil { - err = msgp.WrapError(err, "PendingCount") + err = msgp.WrapError(err, "Failed") return } - // write "FailedCount" - err = en.Append(0xab, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + // write "ReplicatedCount" + err = en.Append(0xaf, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) if err != nil { return } - err = en.WriteInt64(z.FailedCount) + err = en.WriteInt64(z.ReplicatedCount) if err != nil { - err = msgp.WrapError(err, "FailedCount") + err = msgp.WrapError(err, "ReplicatedCount") return } // write "Latency" @@ -211,31 +255,110 @@ func (z *BucketReplicationStat) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "CurrentBandwidthInBytesPerSecond") return } + // write "lt" + err = en.Append(0xa2, 0x6c, 0x74) + if err != nil { + return + } + if z.XferRateLrg == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = z.XferRateLrg.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + } + // write "st" + err = en.Append(0xa2, 0x73, 0x74) + if err != nil { + return + } + if z.XferRateSml == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = z.XferRateSml.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + } + // write "PendingSize" + err = en.Append(0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.PendingSize) + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } + // write "FailedSize" + err = en.Append(0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.FailedSize) + if err != nil { + err = msgp.WrapError(err, "FailedSize") + return + } + // write "PendingCount" + err = en.Append(0xac, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.PendingCount) + if err != nil { + err = msgp.WrapError(err, "PendingCount") + return + } + // write "FailedCount" + err = en.Append(0xab, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.FailedCount) + if err != nil { + err = msgp.WrapError(err, "FailedCount") + return + } return } // MarshalMsg implements msgp.Marshaler func (z *BucketReplicationStat) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // map header, size 9 - // string "PendingSize" - o = append(o, 0x89, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) - o = msgp.AppendInt64(o, z.PendingSize) + // map header, size 14 // string "ReplicatedSize" - o = append(o, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + o = append(o, 0x8e, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) o = msgp.AppendInt64(o, z.ReplicatedSize) // string "ReplicaSize" o = append(o, 0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) o = msgp.AppendInt64(o, z.ReplicaSize) - // string "FailedSize" - o = append(o, 0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) - o = msgp.AppendInt64(o, z.FailedSize) - // string "PendingCount" - o = append(o, 0xac, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74) - o = msgp.AppendInt64(o, z.PendingCount) - // string "FailedCount" - o = append(o, 0xab, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) - o = msgp.AppendInt64(o, z.FailedCount) + // string "FailStats" + o = append(o, 0xa9, 0x46, 0x61, 0x69, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x73) + o, err = z.FailStats.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "FailStats") + return + } + // string "Failed" + o = append(o, 0xa6, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64) + o, err = z.Failed.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + // string "ReplicatedCount" + o = append(o, 0xaf, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.ReplicatedCount) // string "Latency" o = append(o, 0xa7, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79) // map header, size 1 @@ -252,6 +375,40 @@ func (z *BucketReplicationStat) MarshalMsg(b []byte) (o []byte, err error) { // string "CurrentBandwidthInBytesPerSecond" o = append(o, 0xd9, 0x20, 0x43, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x42, 0x61, 0x6e, 0x64, 0x77, 0x69, 0x64, 0x74, 0x68, 0x49, 0x6e, 0x42, 0x79, 0x74, 0x65, 0x73, 0x50, 0x65, 0x72, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64) o = msgp.AppendFloat64(o, z.CurrentBandwidthInBytesPerSecond) + // string "lt" + o = append(o, 0xa2, 0x6c, 0x74) + if z.XferRateLrg == nil { + o = msgp.AppendNil(o) + } else { + o, err = z.XferRateLrg.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + } + // string "st" + o = append(o, 0xa2, 0x73, 0x74) + if z.XferRateSml == nil { + o = msgp.AppendNil(o) + } else { + o, err = z.XferRateSml.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + } + // string "PendingSize" + o = append(o, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.PendingSize) + // string "FailedSize" + o = append(o, 0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.FailedSize) + // string "PendingCount" + o = append(o, 0xac, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.PendingCount) + // string "FailedCount" + o = append(o, 0xab, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.FailedCount) return } @@ -273,12 +430,6 @@ func (z *BucketReplicationStat) UnmarshalMsg(bts []byte) (o []byte, err error) { return } switch msgp.UnsafeString(field) { - case "PendingSize": - z.PendingSize, bts, err = msgp.ReadInt64Bytes(bts) - if err != nil { - err = msgp.WrapError(err, "PendingSize") - return - } case "ReplicatedSize": z.ReplicatedSize, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { @@ -291,22 +442,22 @@ func (z *BucketReplicationStat) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err, "ReplicaSize") return } - case "FailedSize": - z.FailedSize, bts, err = msgp.ReadInt64Bytes(bts) + case "FailStats": + bts, err = z.FailStats.UnmarshalMsg(bts) if err != nil { - err = msgp.WrapError(err, "FailedSize") + err = msgp.WrapError(err, "FailStats") return } - case "PendingCount": - z.PendingCount, bts, err = msgp.ReadInt64Bytes(bts) + case "Failed": + bts, err = z.Failed.UnmarshalMsg(bts) if err != nil { - err = msgp.WrapError(err, "PendingCount") + err = msgp.WrapError(err, "Failed") return } - case "FailedCount": - z.FailedCount, bts, err = msgp.ReadInt64Bytes(bts) + case "ReplicatedCount": + z.ReplicatedCount, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { - err = msgp.WrapError(err, "FailedCount") + err = msgp.WrapError(err, "ReplicatedCount") return } case "Latency": @@ -350,6 +501,64 @@ func (z *BucketReplicationStat) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err, "CurrentBandwidthInBytesPerSecond") return } + case "lt": + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + z.XferRateLrg = nil + } else { + if z.XferRateLrg == nil { + z.XferRateLrg = new(XferStats) + } + bts, err = z.XferRateLrg.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + } + case "st": + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + z.XferRateSml = nil + } else { + if z.XferRateSml == nil { + z.XferRateSml = new(XferStats) + } + bts, err = z.XferRateSml.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + } + case "PendingSize": + z.PendingSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } + case "FailedSize": + z.FailedSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "FailedSize") + return + } + case "PendingCount": + z.PendingCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "PendingCount") + return + } + case "FailedCount": + z.FailedCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "FailedCount") + return + } default: bts, err = msgp.Skip(bts) if err != nil { @@ -364,7 +573,19 @@ func (z *BucketReplicationStat) UnmarshalMsg(bts []byte) (o []byte, err error) { // Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message func (z *BucketReplicationStat) Msgsize() (s int) { - s = 1 + 12 + msgp.Int64Size + 15 + msgp.Int64Size + 12 + msgp.Int64Size + 11 + msgp.Int64Size + 13 + msgp.Int64Size + 12 + msgp.Int64Size + 8 + 1 + 16 + z.Latency.UploadHistogram.Msgsize() + 31 + msgp.Int64Size + 34 + msgp.Float64Size + s = 1 + 15 + msgp.Int64Size + 12 + msgp.Int64Size + 10 + z.FailStats.Msgsize() + 7 + z.Failed.Msgsize() + 16 + msgp.Int64Size + 8 + 1 + 16 + z.Latency.UploadHistogram.Msgsize() + 31 + msgp.Int64Size + 34 + msgp.Float64Size + 3 + if z.XferRateLrg == nil { + s += msgp.NilSize + } else { + s += z.XferRateLrg.Msgsize() + } + s += 3 + if z.XferRateSml == nil { + s += msgp.NilSize + } else { + s += z.XferRateSml.Msgsize() + } + s += 12 + msgp.Int64Size + 11 + msgp.Int64Size + 13 + msgp.Int64Size + 12 + msgp.Int64Size return } @@ -428,12 +649,6 @@ func (z *BucketReplicationStats) DecodeMsg(dc *msgp.Reader) (err error) { } z.Stats[za0001] = za0002 } - case "PendingSize": - z.PendingSize, err = dc.ReadInt64() - if err != nil { - err = msgp.WrapError(err, "PendingSize") - return - } case "ReplicatedSize": z.ReplicatedSize, err = dc.ReadInt64() if err != nil { @@ -446,6 +661,36 @@ func (z *BucketReplicationStats) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "ReplicaSize") return } + case "Failed": + err = z.Failed.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + case "ReplicatedCount": + z.ReplicatedCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + case "ReplicaCount": + z.ReplicaCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + case "QStat": + err = z.QStat.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "QStat") + return + } + case "PendingSize": + z.PendingSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } case "FailedSize": z.FailedSize, err = dc.ReadInt64() if err != nil { @@ -477,9 +722,9 @@ func (z *BucketReplicationStats) DecodeMsg(dc *msgp.Reader) (err error) { // EncodeMsg implements msgp.Encodable func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { - // map header, size 7 + // map header, size 11 // write "Stats" - err = en.Append(0x87, 0xa5, 0x53, 0x74, 0x61, 0x74, 0x73) + err = en.Append(0x8b, 0xa5, 0x53, 0x74, 0x61, 0x74, 0x73) if err != nil { return } @@ -507,16 +752,6 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { } } } - // write "PendingSize" - err = en.Append(0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) - if err != nil { - return - } - err = en.WriteInt64(z.PendingSize) - if err != nil { - err = msgp.WrapError(err, "PendingSize") - return - } // write "ReplicatedSize" err = en.Append(0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) if err != nil { @@ -537,6 +772,56 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "ReplicaSize") return } + // write "Failed" + err = en.Append(0xa6, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64) + if err != nil { + return + } + err = z.Failed.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + // write "ReplicatedCount" + err = en.Append(0xaf, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicatedCount) + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + // write "ReplicaCount" + err = en.Append(0xac, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicaCount) + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + // write "QStat" + err = en.Append(0xa5, 0x51, 0x53, 0x74, 0x61, 0x74) + if err != nil { + return + } + err = z.QStat.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "QStat") + return + } + // write "PendingSize" + err = en.Append(0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.PendingSize) + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } // write "FailedSize" err = en.Append(0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) if err != nil { @@ -573,9 +858,9 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { // MarshalMsg implements msgp.Marshaler func (z *BucketReplicationStats) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // map header, size 7 + // map header, size 11 // string "Stats" - o = append(o, 0x87, 0xa5, 0x53, 0x74, 0x61, 0x74, 0x73) + o = append(o, 0x8b, 0xa5, 0x53, 0x74, 0x61, 0x74, 0x73) o = msgp.AppendMapHeader(o, uint32(len(z.Stats))) for za0001, za0002 := range z.Stats { o = msgp.AppendString(o, za0001) @@ -589,15 +874,35 @@ func (z *BucketReplicationStats) MarshalMsg(b []byte) (o []byte, err error) { } } } - // string "PendingSize" - o = append(o, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) - o = msgp.AppendInt64(o, z.PendingSize) // string "ReplicatedSize" o = append(o, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) o = msgp.AppendInt64(o, z.ReplicatedSize) // string "ReplicaSize" o = append(o, 0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) o = msgp.AppendInt64(o, z.ReplicaSize) + // string "Failed" + o = append(o, 0xa6, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64) + o, err = z.Failed.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + // string "ReplicatedCount" + o = append(o, 0xaf, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.ReplicatedCount) + // string "ReplicaCount" + o = append(o, 0xac, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.ReplicaCount) + // string "QStat" + o = append(o, 0xa5, 0x51, 0x53, 0x74, 0x61, 0x74) + o, err = z.QStat.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "QStat") + return + } + // string "PendingSize" + o = append(o, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.PendingSize) // string "FailedSize" o = append(o, 0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) o = msgp.AppendInt64(o, z.FailedSize) @@ -669,12 +974,6 @@ func (z *BucketReplicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) } z.Stats[za0001] = za0002 } - case "PendingSize": - z.PendingSize, bts, err = msgp.ReadInt64Bytes(bts) - if err != nil { - err = msgp.WrapError(err, "PendingSize") - return - } case "ReplicatedSize": z.ReplicatedSize, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { @@ -687,6 +986,36 @@ func (z *BucketReplicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) err = msgp.WrapError(err, "ReplicaSize") return } + case "Failed": + bts, err = z.Failed.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + case "ReplicatedCount": + z.ReplicatedCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + case "ReplicaCount": + z.ReplicaCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + case "QStat": + bts, err = z.QStat.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "QStat") + return + } + case "PendingSize": + z.PendingSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } case "FailedSize": z.FailedSize, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { @@ -731,7 +1060,7 @@ func (z *BucketReplicationStats) Msgsize() (s int) { } } } - s += 12 + msgp.Int64Size + 15 + msgp.Int64Size + 12 + msgp.Int64Size + 11 + msgp.Int64Size + 13 + msgp.Int64Size + 12 + msgp.Int64Size + s += 15 + msgp.Int64Size + 12 + msgp.Int64Size + 7 + z.Failed.Msgsize() + 16 + msgp.Int64Size + 13 + msgp.Int64Size + 6 + z.QStat.Msgsize() + 12 + msgp.Int64Size + 11 + msgp.Int64Size + 13 + msgp.Int64Size + 12 + msgp.Int64Size return } @@ -753,12 +1082,66 @@ func (z *BucketStats) DecodeMsg(dc *msgp.Reader) (err error) { return } switch msgp.UnsafeString(field) { + case "Uptime": + z.Uptime, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } case "ReplicationStats": err = z.ReplicationStats.DecodeMsg(dc) if err != nil { err = msgp.WrapError(err, "ReplicationStats") return } + case "QueueStats": + var zb0002 uint32 + zb0002, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "QueueStats") + return + } + for zb0002 > 0 { + zb0002-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err, "QueueStats") + return + } + switch msgp.UnsafeString(field) { + case "Nodes": + var zb0003 uint32 + zb0003, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Nodes") + return + } + if cap(z.QueueStats.Nodes) >= int(zb0003) { + z.QueueStats.Nodes = (z.QueueStats.Nodes)[:zb0003] + } else { + z.QueueStats.Nodes = make([]ReplQNodeStats, zb0003) + } + for za0001 := range z.QueueStats.Nodes { + err = z.QueueStats.Nodes[za0001].DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Nodes", za0001) + return + } + } + case "Uptime": + z.QueueStats.Uptime, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Uptime") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err, "QueueStats") + return + } + } + } default: err = dc.Skip() if err != nil { @@ -772,9 +1155,19 @@ func (z *BucketStats) DecodeMsg(dc *msgp.Reader) (err error) { // EncodeMsg implements msgp.Encodable func (z *BucketStats) EncodeMsg(en *msgp.Writer) (err error) { - // map header, size 1 + // map header, size 3 + // write "Uptime" + err = en.Append(0x83, 0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.Uptime) + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } // write "ReplicationStats" - err = en.Append(0x81, 0xb0, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73) + err = en.Append(0xb0, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73) if err != nil { return } @@ -783,20 +1176,72 @@ func (z *BucketStats) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "ReplicationStats") return } + // write "QueueStats" + err = en.Append(0xaa, 0x51, 0x75, 0x65, 0x75, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73) + if err != nil { + return + } + // map header, size 2 + // write "Nodes" + err = en.Append(0x82, 0xa5, 0x4e, 0x6f, 0x64, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteArrayHeader(uint32(len(z.QueueStats.Nodes))) + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Nodes") + return + } + for za0001 := range z.QueueStats.Nodes { + err = z.QueueStats.Nodes[za0001].EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Nodes", za0001) + return + } + } + // write "Uptime" + err = en.Append(0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.QueueStats.Uptime) + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Uptime") + return + } return } // MarshalMsg implements msgp.Marshaler func (z *BucketStats) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // map header, size 1 + // map header, size 3 + // string "Uptime" + o = append(o, 0x83, 0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + o = msgp.AppendInt64(o, z.Uptime) // string "ReplicationStats" - o = append(o, 0x81, 0xb0, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73) + o = append(o, 0xb0, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73) o, err = z.ReplicationStats.MarshalMsg(o) if err != nil { err = msgp.WrapError(err, "ReplicationStats") return } + // string "QueueStats" + o = append(o, 0xaa, 0x51, 0x75, 0x65, 0x75, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73) + // map header, size 2 + // string "Nodes" + o = append(o, 0x82, 0xa5, 0x4e, 0x6f, 0x64, 0x65, 0x73) + o = msgp.AppendArrayHeader(o, uint32(len(z.QueueStats.Nodes))) + for za0001 := range z.QueueStats.Nodes { + o, err = z.QueueStats.Nodes[za0001].MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Nodes", za0001) + return + } + } + // string "Uptime" + o = append(o, 0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + o = msgp.AppendInt64(o, z.QueueStats.Uptime) return } @@ -818,12 +1263,66 @@ func (z *BucketStats) UnmarshalMsg(bts []byte) (o []byte, err error) { return } switch msgp.UnsafeString(field) { + case "Uptime": + z.Uptime, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } case "ReplicationStats": bts, err = z.ReplicationStats.UnmarshalMsg(bts) if err != nil { err = msgp.WrapError(err, "ReplicationStats") return } + case "QueueStats": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "QueueStats") + return + } + for zb0002 > 0 { + zb0002-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err, "QueueStats") + return + } + switch msgp.UnsafeString(field) { + case "Nodes": + var zb0003 uint32 + zb0003, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Nodes") + return + } + if cap(z.QueueStats.Nodes) >= int(zb0003) { + z.QueueStats.Nodes = (z.QueueStats.Nodes)[:zb0003] + } else { + z.QueueStats.Nodes = make([]ReplQNodeStats, zb0003) + } + for za0001 := range z.QueueStats.Nodes { + bts, err = z.QueueStats.Nodes[za0001].UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Nodes", za0001) + return + } + } + case "Uptime": + z.QueueStats.Uptime, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "QueueStats", "Uptime") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err, "QueueStats") + return + } + } + } default: bts, err = msgp.Skip(bts) if err != nil { @@ -838,7 +1337,11 @@ func (z *BucketStats) UnmarshalMsg(bts []byte) (o []byte, err error) { // Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message func (z *BucketStats) Msgsize() (s int) { - s = 1 + 17 + z.ReplicationStats.Msgsize() + s = 1 + 7 + msgp.Int64Size + 17 + z.ReplicationStats.Msgsize() + 11 + 1 + 6 + msgp.ArrayHeaderSize + for za0001 := range z.QueueStats.Nodes { + s += z.QueueStats.Nodes[za0001].Msgsize() + } + s += 7 + msgp.Int64Size return } @@ -883,34 +1386,11 @@ func (z *BucketStatsMap) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "Stats") return } - var zb0003 uint32 - zb0003, err = dc.ReadMapHeader() + err = za0002.DecodeMsg(dc) if err != nil { err = msgp.WrapError(err, "Stats", za0001) return } - for zb0003 > 0 { - zb0003-- - field, err = dc.ReadMapKeyPtr() - if err != nil { - err = msgp.WrapError(err, "Stats", za0001) - return - } - switch msgp.UnsafeString(field) { - case "ReplicationStats": - err = za0002.ReplicationStats.DecodeMsg(dc) - if err != nil { - err = msgp.WrapError(err, "Stats", za0001, "ReplicationStats") - return - } - default: - err = dc.Skip() - if err != nil { - err = msgp.WrapError(err, "Stats", za0001) - return - } - } - } z.Stats[za0001] = za0002 } case "Timestamp": @@ -949,15 +1429,9 @@ func (z *BucketStatsMap) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "Stats") return } - // map header, size 1 - // write "ReplicationStats" - err = en.Append(0x81, 0xb0, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73) + err = za0002.EncodeMsg(en) if err != nil { - return - } - err = za0002.ReplicationStats.EncodeMsg(en) - if err != nil { - err = msgp.WrapError(err, "Stats", za0001, "ReplicationStats") + err = msgp.WrapError(err, "Stats", za0001) return } } @@ -983,12 +1457,9 @@ func (z *BucketStatsMap) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.AppendMapHeader(o, uint32(len(z.Stats))) for za0001, za0002 := range z.Stats { o = msgp.AppendString(o, za0001) - // map header, size 1 - // string "ReplicationStats" - o = append(o, 0x81, 0xb0, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73) - o, err = za0002.ReplicationStats.MarshalMsg(o) + o, err = za0002.MarshalMsg(o) if err != nil { - err = msgp.WrapError(err, "Stats", za0001, "ReplicationStats") + err = msgp.WrapError(err, "Stats", za0001) return } } @@ -1039,34 +1510,11 @@ func (z *BucketStatsMap) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err, "Stats") return } - var zb0003 uint32 - zb0003, bts, err = msgp.ReadMapHeaderBytes(bts) + bts, err = za0002.UnmarshalMsg(bts) if err != nil { err = msgp.WrapError(err, "Stats", za0001) return } - for zb0003 > 0 { - zb0003-- - field, bts, err = msgp.ReadMapKeyZC(bts) - if err != nil { - err = msgp.WrapError(err, "Stats", za0001) - return - } - switch msgp.UnsafeString(field) { - case "ReplicationStats": - bts, err = za0002.ReplicationStats.UnmarshalMsg(bts) - if err != nil { - err = msgp.WrapError(err, "Stats", za0001, "ReplicationStats") - return - } - default: - bts, err = msgp.Skip(bts) - if err != nil { - err = msgp.WrapError(err, "Stats", za0001) - return - } - } - } z.Stats[za0001] = za0002 } case "Timestamp": @@ -1093,13 +1541,557 @@ func (z *BucketStatsMap) Msgsize() (s int) { if z.Stats != nil { for za0001, za0002 := range z.Stats { _ = za0002 - s += msgp.StringPrefixSize + len(za0001) + 1 + 17 + za0002.ReplicationStats.Msgsize() + s += msgp.StringPrefixSize + len(za0001) + za0002.Msgsize() } } s += 10 + msgp.TimeSize return } +// DecodeMsg implements msgp.Decodable +func (z *RMetricName) DecodeMsg(dc *msgp.Reader) (err error) { + { + var zb0001 string + zb0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err) + return + } + (*z) = RMetricName(zb0001) + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z RMetricName) EncodeMsg(en *msgp.Writer) (err error) { + err = en.WriteString(string(z)) + if err != nil { + err = msgp.WrapError(err) + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z RMetricName) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + o = msgp.AppendString(o, string(z)) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *RMetricName) UnmarshalMsg(bts []byte) (o []byte, err error) { + { + var zb0001 string + zb0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + (*z) = RMetricName(zb0001) + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z RMetricName) Msgsize() (s int) { + s = msgp.StringPrefixSize + len(string(z)) + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ReplQNodeStats) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "NodeName": + z.NodeName, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "NodeName") + return + } + case "Uptime": + z.Uptime, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + case "ActiveWorkers": + err = z.ActiveWorkers.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "ActiveWorkers") + return + } + case "QStats": + err = z.QStats.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "QStats") + return + } + case "MRFStats": + err = z.MRFStats.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "MRFStats") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *ReplQNodeStats) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 5 + // write "NodeName" + err = en.Append(0x85, 0xa8, 0x4e, 0x6f, 0x64, 0x65, 0x4e, 0x61, 0x6d, 0x65) + if err != nil { + return + } + err = en.WriteString(z.NodeName) + if err != nil { + err = msgp.WrapError(err, "NodeName") + return + } + // write "Uptime" + err = en.Append(0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.Uptime) + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + // write "ActiveWorkers" + err = en.Append(0xad, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73) + if err != nil { + return + } + err = z.ActiveWorkers.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "ActiveWorkers") + return + } + // write "QStats" + err = en.Append(0xa6, 0x51, 0x53, 0x74, 0x61, 0x74, 0x73) + if err != nil { + return + } + err = z.QStats.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "QStats") + return + } + // write "MRFStats" + err = en.Append(0xa8, 0x4d, 0x52, 0x46, 0x53, 0x74, 0x61, 0x74, 0x73) + if err != nil { + return + } + err = z.MRFStats.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "MRFStats") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *ReplQNodeStats) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 5 + // string "NodeName" + o = append(o, 0x85, 0xa8, 0x4e, 0x6f, 0x64, 0x65, 0x4e, 0x61, 0x6d, 0x65) + o = msgp.AppendString(o, z.NodeName) + // string "Uptime" + o = append(o, 0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + o = msgp.AppendInt64(o, z.Uptime) + // string "ActiveWorkers" + o = append(o, 0xad, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73) + o, err = z.ActiveWorkers.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "ActiveWorkers") + return + } + // string "QStats" + o = append(o, 0xa6, 0x51, 0x53, 0x74, 0x61, 0x74, 0x73) + o, err = z.QStats.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "QStats") + return + } + // string "MRFStats" + o = append(o, 0xa8, 0x4d, 0x52, 0x46, 0x53, 0x74, 0x61, 0x74, 0x73) + o, err = z.MRFStats.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "MRFStats") + return + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ReplQNodeStats) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "NodeName": + z.NodeName, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "NodeName") + return + } + case "Uptime": + z.Uptime, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + case "ActiveWorkers": + bts, err = z.ActiveWorkers.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "ActiveWorkers") + return + } + case "QStats": + bts, err = z.QStats.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "QStats") + return + } + case "MRFStats": + bts, err = z.MRFStats.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "MRFStats") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *ReplQNodeStats) Msgsize() (s int) { + s = 1 + 9 + msgp.StringPrefixSize + len(z.NodeName) + 7 + msgp.Int64Size + 14 + z.ActiveWorkers.Msgsize() + 7 + z.QStats.Msgsize() + 9 + z.MRFStats.Msgsize() + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ReplicationLastHour) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Totals": + var zb0002 uint32 + zb0002, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err, "Totals") + return + } + if zb0002 != uint32(60) { + err = msgp.ArrayError{Wanted: uint32(60), Got: zb0002} + return + } + for za0001 := range z.Totals { + err = z.Totals[za0001].DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Totals", za0001) + return + } + } + case "LastMin": + z.LastMin, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "LastMin") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *ReplicationLastHour) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 2 + // write "Totals" + err = en.Append(0x82, 0xa6, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x73) + if err != nil { + return + } + err = en.WriteArrayHeader(uint32(60)) + if err != nil { + err = msgp.WrapError(err, "Totals") + return + } + for za0001 := range z.Totals { + err = z.Totals[za0001].EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Totals", za0001) + return + } + } + // write "LastMin" + err = en.Append(0xa7, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x69, 0x6e) + if err != nil { + return + } + err = en.WriteInt64(z.LastMin) + if err != nil { + err = msgp.WrapError(err, "LastMin") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *ReplicationLastHour) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 2 + // string "Totals" + o = append(o, 0x82, 0xa6, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x73) + o = msgp.AppendArrayHeader(o, uint32(60)) + for za0001 := range z.Totals { + o, err = z.Totals[za0001].MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Totals", za0001) + return + } + } + // string "LastMin" + o = append(o, 0xa7, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x69, 0x6e) + o = msgp.AppendInt64(o, z.LastMin) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ReplicationLastHour) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Totals": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Totals") + return + } + if zb0002 != uint32(60) { + err = msgp.ArrayError{Wanted: uint32(60), Got: zb0002} + return + } + for za0001 := range z.Totals { + bts, err = z.Totals[za0001].UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Totals", za0001) + return + } + } + case "LastMin": + z.LastMin, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "LastMin") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *ReplicationLastHour) Msgsize() (s int) { + s = 1 + 7 + msgp.ArrayHeaderSize + for za0001 := range z.Totals { + s += z.Totals[za0001].Msgsize() + } + s += 8 + msgp.Int64Size + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ReplicationLastMinute) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "LastMinute": + err = z.LastMinute.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "LastMinute") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *ReplicationLastMinute) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 1 + // write "LastMinute" + err = en.Append(0x81, 0xaa, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x69, 0x6e, 0x75, 0x74, 0x65) + if err != nil { + return + } + err = z.LastMinute.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "LastMinute") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *ReplicationLastMinute) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 1 + // string "LastMinute" + o = append(o, 0x81, 0xaa, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x69, 0x6e, 0x75, 0x74, 0x65) + o, err = z.LastMinute.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "LastMinute") + return + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ReplicationLastMinute) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "LastMinute": + bts, err = z.LastMinute.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "LastMinute") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *ReplicationLastMinute) Msgsize() (s int) { + s = 1 + 11 + z.LastMinute.Msgsize() + return +} + // DecodeMsg implements msgp.Decodable func (z *ReplicationLatency) DecodeMsg(dc *msgp.Reader) (err error) { var field []byte @@ -1206,3 +2198,175 @@ func (z *ReplicationLatency) Msgsize() (s int) { s = 1 + 16 + z.UploadHistogram.Msgsize() return } + +// DecodeMsg implements msgp.Decodable +func (z *ReplicationQueueStats) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Nodes": + var zb0002 uint32 + zb0002, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err, "Nodes") + return + } + if cap(z.Nodes) >= int(zb0002) { + z.Nodes = (z.Nodes)[:zb0002] + } else { + z.Nodes = make([]ReplQNodeStats, zb0002) + } + for za0001 := range z.Nodes { + err = z.Nodes[za0001].DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Nodes", za0001) + return + } + } + case "Uptime": + z.Uptime, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *ReplicationQueueStats) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 2 + // write "Nodes" + err = en.Append(0x82, 0xa5, 0x4e, 0x6f, 0x64, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteArrayHeader(uint32(len(z.Nodes))) + if err != nil { + err = msgp.WrapError(err, "Nodes") + return + } + for za0001 := range z.Nodes { + err = z.Nodes[za0001].EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Nodes", za0001) + return + } + } + // write "Uptime" + err = en.Append(0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.Uptime) + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *ReplicationQueueStats) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 2 + // string "Nodes" + o = append(o, 0x82, 0xa5, 0x4e, 0x6f, 0x64, 0x65, 0x73) + o = msgp.AppendArrayHeader(o, uint32(len(z.Nodes))) + for za0001 := range z.Nodes { + o, err = z.Nodes[za0001].MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Nodes", za0001) + return + } + } + // string "Uptime" + o = append(o, 0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + o = msgp.AppendInt64(o, z.Uptime) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ReplicationQueueStats) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Nodes": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Nodes") + return + } + if cap(z.Nodes) >= int(zb0002) { + z.Nodes = (z.Nodes)[:zb0002] + } else { + z.Nodes = make([]ReplQNodeStats, zb0002) + } + for za0001 := range z.Nodes { + bts, err = z.Nodes[za0001].UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Nodes", za0001) + return + } + } + case "Uptime": + z.Uptime, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *ReplicationQueueStats) Msgsize() (s int) { + s = 1 + 6 + msgp.ArrayHeaderSize + for za0001 := range z.Nodes { + s += z.Nodes[za0001].Msgsize() + } + s += 7 + msgp.Int64Size + return +} diff --git a/cmd/bucket-stats_gen_test.go b/cmd/bucket-stats_gen_test.go index f50ff31d0..2116c19eb 100644 --- a/cmd/bucket-stats_gen_test.go +++ b/cmd/bucket-stats_gen_test.go @@ -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) + } + } +} diff --git a/cmd/bucket-targets.go b/cmd/bucket-targets.go index a4fd9833d..acf04f2cf 100644 --- a/cmd/bucket-targets.go +++ b/cmd/bucket-targets.go @@ -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) diff --git a/cmd/data-scanner.go b/cmd/data-scanner.go index aacdd6a83..ce97f1fdd 100644 --- a/cmd/data-scanner.go +++ b/cmd/data-scanner.go @@ -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++ } } diff --git a/cmd/data-usage-cache.go b/cmd/data-usage-cache.go index a989a74d3..bf631e050 100644 --- a/cmd/data-usage-cache.go +++ b/cmd/data-usage-cache.go @@ -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{ diff --git a/cmd/data-usage-cache_gen.go b/cmd/data-usage-cache_gen.go index 4c747046d..e66e1edcc 100644 --- a/cmd/data-usage-cache_gen.go +++ b/cmd/data-usage-cache_gen.go @@ -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 } diff --git a/cmd/data-usage-utils.go b/cmd/data-usage-utils.go index f1c94d8f3..aed9186e9 100644 --- a/cmd/data-usage-utils.go +++ b/cmd/data-usage-utils.go @@ -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"` } diff --git a/cmd/metrics-v2.go b/cmd/metrics-v2.go index b76f2a27f..14106cb62 100644 --- a/cmd/metrics-v2.go +++ b/cmd/metrics-v2.go @@ -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 }) diff --git a/cmd/metrics.go b/cmd/metrics.go index a9ea7f088..0bc647924 100644 --- a/cmd/metrics.go +++ b/cmd/metrics.go @@ -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( diff --git a/cmd/notification.go b/cmd/notification.go index 417813257..7a15253c8 100644 --- a/cmd/notification.go +++ b/cmd/notification.go @@ -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)) diff --git a/cmd/peer-rest-client.go b/cmd/peer-rest-client.go index cfc2add9b..89261d46e 100644 --- a/cmd/peer-rest-client.go +++ b/cmd/peer-rest-client.go @@ -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) diff --git a/cmd/peer-rest-common.go b/cmd/peer-rest-common.go index 25a9e4272..f4b469d79 100644 --- a/cmd/peer-rest-common.go +++ b/cmd/peer-rest-common.go @@ -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 ( diff --git a/cmd/peer-rest-server.go b/cmd/peer-rest-server.go index 96be14ca6..3ada67dc0 100644 --- a/cmd/peer-rest-server.go +++ b/cmd/peer-rest-server.go @@ -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)) diff --git a/cmd/site-replication-metrics.go b/cmd/site-replication-metrics.go new file mode 100644 index 000000000..3f1d61d94 --- /dev/null +++ b/cmd/site-replication-metrics.go @@ -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 . + +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"` +} diff --git a/cmd/site-replication-metrics_gen.go b/cmd/site-replication-metrics_gen.go new file mode 100644 index 000000000..d18d6e2fd --- /dev/null +++ b/cmd/site-replication-metrics_gen.go @@ -0,0 +1,1733 @@ +package cmd + +// Code generated by github.com/tinylib/msgp DO NOT EDIT. + +import ( + "github.com/tinylib/msgp/msgp" +) + +// DecodeMsg implements msgp.Decodable +func (z *RStat) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Count, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "Count") + return + } + case "Bytes": + z.Bytes, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "Bytes") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z RStat) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 2 + // write "Count" + err = en.Append(0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.Count) + if err != nil { + err = msgp.WrapError(err, "Count") + return + } + // write "Bytes" + err = en.Append(0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteInt64(z.Bytes) + if err != nil { + err = msgp.WrapError(err, "Bytes") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z RStat) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 2 + // string "Count" + o = append(o, 0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.Count) + // string "Bytes" + o = append(o, 0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + o = msgp.AppendInt64(o, z.Bytes) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *RStat) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.Count, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Count") + return + } + case "Bytes": + z.Bytes, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Bytes") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z RStat) Msgsize() (s int) { + s = 1 + 6 + msgp.Int64Size + 6 + msgp.Int64Size + return +} + +// DecodeMsg implements msgp.Decodable +func (z *RTimedMetrics) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "LastHour": + err = z.LastHour.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "LastHour") + return + } + case "SinceUptime": + var zb0002 uint32 + zb0002, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "SinceUptime") + return + } + for zb0002 > 0 { + zb0002-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err, "SinceUptime") + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.SinceUptime.Count, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "SinceUptime", "Count") + return + } + case "Bytes": + z.SinceUptime.Bytes, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "SinceUptime", "Bytes") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err, "SinceUptime") + return + } + } + } + case "LastMinute": + err = z.LastMinute.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "LastMinute") + return + } + case "ErrCounts": + var zb0003 uint32 + zb0003, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "ErrCounts") + return + } + if z.ErrCounts == nil { + z.ErrCounts = make(map[string]int, zb0003) + } else if len(z.ErrCounts) > 0 { + for key := range z.ErrCounts { + delete(z.ErrCounts, key) + } + } + for zb0003 > 0 { + zb0003-- + var za0001 string + var za0002 int + za0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "ErrCounts") + return + } + za0002, err = dc.ReadInt() + if err != nil { + err = msgp.WrapError(err, "ErrCounts", za0001) + return + } + z.ErrCounts[za0001] = za0002 + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *RTimedMetrics) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 4 + // write "LastHour" + err = en.Append(0x84, 0xa8, 0x4c, 0x61, 0x73, 0x74, 0x48, 0x6f, 0x75, 0x72) + if err != nil { + return + } + err = z.LastHour.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "LastHour") + return + } + // write "SinceUptime" + err = en.Append(0xab, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + if err != nil { + return + } + // map header, size 2 + // write "Count" + err = en.Append(0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.SinceUptime.Count) + if err != nil { + err = msgp.WrapError(err, "SinceUptime", "Count") + return + } + // write "Bytes" + err = en.Append(0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + if err != nil { + return + } + err = en.WriteInt64(z.SinceUptime.Bytes) + if err != nil { + err = msgp.WrapError(err, "SinceUptime", "Bytes") + return + } + // write "LastMinute" + err = en.Append(0xaa, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x69, 0x6e, 0x75, 0x74, 0x65) + if err != nil { + return + } + err = z.LastMinute.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "LastMinute") + return + } + // write "ErrCounts" + err = en.Append(0xa9, 0x45, 0x72, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73) + if err != nil { + return + } + err = en.WriteMapHeader(uint32(len(z.ErrCounts))) + if err != nil { + err = msgp.WrapError(err, "ErrCounts") + return + } + for za0001, za0002 := range z.ErrCounts { + err = en.WriteString(za0001) + if err != nil { + err = msgp.WrapError(err, "ErrCounts") + return + } + err = en.WriteInt(za0002) + if err != nil { + err = msgp.WrapError(err, "ErrCounts", za0001) + return + } + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *RTimedMetrics) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 4 + // string "LastHour" + o = append(o, 0x84, 0xa8, 0x4c, 0x61, 0x73, 0x74, 0x48, 0x6f, 0x75, 0x72) + o, err = z.LastHour.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "LastHour") + return + } + // string "SinceUptime" + o = append(o, 0xab, 0x53, 0x69, 0x6e, 0x63, 0x65, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + // map header, size 2 + // string "Count" + o = append(o, 0x82, 0xa5, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.SinceUptime.Count) + // string "Bytes" + o = append(o, 0xa5, 0x42, 0x79, 0x74, 0x65, 0x73) + o = msgp.AppendInt64(o, z.SinceUptime.Bytes) + // string "LastMinute" + o = append(o, 0xaa, 0x4c, 0x61, 0x73, 0x74, 0x4d, 0x69, 0x6e, 0x75, 0x74, 0x65) + o, err = z.LastMinute.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "LastMinute") + return + } + // string "ErrCounts" + o = append(o, 0xa9, 0x45, 0x72, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x73) + o = msgp.AppendMapHeader(o, uint32(len(z.ErrCounts))) + for za0001, za0002 := range z.ErrCounts { + o = msgp.AppendString(o, za0001) + o = msgp.AppendInt(o, za0002) + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *RTimedMetrics) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "LastHour": + bts, err = z.LastHour.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "LastHour") + return + } + case "SinceUptime": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "SinceUptime") + return + } + for zb0002 > 0 { + zb0002-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err, "SinceUptime") + return + } + switch msgp.UnsafeString(field) { + case "Count": + z.SinceUptime.Count, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "SinceUptime", "Count") + return + } + case "Bytes": + z.SinceUptime.Bytes, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "SinceUptime", "Bytes") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err, "SinceUptime") + return + } + } + } + case "LastMinute": + bts, err = z.LastMinute.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "LastMinute") + return + } + case "ErrCounts": + var zb0003 uint32 + zb0003, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ErrCounts") + return + } + if z.ErrCounts == nil { + z.ErrCounts = make(map[string]int, zb0003) + } else if len(z.ErrCounts) > 0 { + for key := range z.ErrCounts { + delete(z.ErrCounts, key) + } + } + for zb0003 > 0 { + var za0001 string + var za0002 int + zb0003-- + za0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ErrCounts") + return + } + za0002, bts, err = msgp.ReadIntBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ErrCounts", za0001) + return + } + z.ErrCounts[za0001] = za0002 + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *RTimedMetrics) Msgsize() (s int) { + s = 1 + 9 + z.LastHour.Msgsize() + 12 + 1 + 6 + msgp.Int64Size + 6 + msgp.Int64Size + 11 + z.LastMinute.Msgsize() + 10 + msgp.MapHeaderSize + if z.ErrCounts != nil { + for za0001, za0002 := range z.ErrCounts { + _ = za0002 + s += msgp.StringPrefixSize + len(za0001) + msgp.IntSize + } + } + return +} + +// DecodeMsg implements msgp.Decodable +func (z *SRMetric) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "DeploymentID": + z.DeploymentID, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "DeploymentID") + return + } + case "Endpoint": + z.Endpoint, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Endpoint") + return + } + case "TotalDowntime": + z.TotalDowntime, err = dc.ReadDuration() + if err != nil { + err = msgp.WrapError(err, "TotalDowntime") + return + } + case "LastOnline": + z.LastOnline, err = dc.ReadTime() + if err != nil { + err = msgp.WrapError(err, "LastOnline") + return + } + case "Online": + z.Online, err = dc.ReadBool() + if err != nil { + err = msgp.WrapError(err, "Online") + return + } + case "Latency": + err = z.Latency.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Latency") + return + } + case "ReplicatedSize": + z.ReplicatedSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + case "ReplicatedCount": + z.ReplicatedCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + case "Failed": + err = z.Failed.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *SRMetric) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 9 + // write "DeploymentID" + err = en.Append(0x89, 0xac, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44) + if err != nil { + return + } + err = en.WriteString(z.DeploymentID) + if err != nil { + err = msgp.WrapError(err, "DeploymentID") + return + } + // write "Endpoint" + err = en.Append(0xa8, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteString(z.Endpoint) + if err != nil { + err = msgp.WrapError(err, "Endpoint") + return + } + // write "TotalDowntime" + err = en.Append(0xad, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x44, 0x6f, 0x77, 0x6e, 0x74, 0x69, 0x6d, 0x65) + if err != nil { + return + } + err = en.WriteDuration(z.TotalDowntime) + if err != nil { + err = msgp.WrapError(err, "TotalDowntime") + return + } + // write "LastOnline" + err = en.Append(0xaa, 0x4c, 0x61, 0x73, 0x74, 0x4f, 0x6e, 0x6c, 0x69, 0x6e, 0x65) + if err != nil { + return + } + err = en.WriteTime(z.LastOnline) + if err != nil { + err = msgp.WrapError(err, "LastOnline") + return + } + // write "Online" + err = en.Append(0xa6, 0x4f, 0x6e, 0x6c, 0x69, 0x6e, 0x65) + if err != nil { + return + } + err = en.WriteBool(z.Online) + if err != nil { + err = msgp.WrapError(err, "Online") + return + } + // write "Latency" + err = en.Append(0xa7, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79) + if err != nil { + return + } + err = z.Latency.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Latency") + return + } + // write "ReplicatedSize" + err = en.Append(0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicatedSize) + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + // write "ReplicatedCount" + err = en.Append(0xaf, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicatedCount) + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + // write "Failed" + err = en.Append(0xa6, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64) + if err != nil { + return + } + err = z.Failed.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *SRMetric) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 9 + // string "DeploymentID" + o = append(o, 0x89, 0xac, 0x44, 0x65, 0x70, 0x6c, 0x6f, 0x79, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44) + o = msgp.AppendString(o, z.DeploymentID) + // string "Endpoint" + o = append(o, 0xa8, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74) + o = msgp.AppendString(o, z.Endpoint) + // string "TotalDowntime" + o = append(o, 0xad, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x44, 0x6f, 0x77, 0x6e, 0x74, 0x69, 0x6d, 0x65) + o = msgp.AppendDuration(o, z.TotalDowntime) + // string "LastOnline" + o = append(o, 0xaa, 0x4c, 0x61, 0x73, 0x74, 0x4f, 0x6e, 0x6c, 0x69, 0x6e, 0x65) + o = msgp.AppendTime(o, z.LastOnline) + // string "Online" + o = append(o, 0xa6, 0x4f, 0x6e, 0x6c, 0x69, 0x6e, 0x65) + o = msgp.AppendBool(o, z.Online) + // string "Latency" + o = append(o, 0xa7, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79) + o, err = z.Latency.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Latency") + return + } + // string "ReplicatedSize" + o = append(o, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.ReplicatedSize) + // string "ReplicatedCount" + o = append(o, 0xaf, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.ReplicatedCount) + // string "Failed" + o = append(o, 0xa6, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64) + o, err = z.Failed.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *SRMetric) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "DeploymentID": + z.DeploymentID, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "DeploymentID") + return + } + case "Endpoint": + z.Endpoint, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Endpoint") + return + } + case "TotalDowntime": + z.TotalDowntime, bts, err = msgp.ReadDurationBytes(bts) + if err != nil { + err = msgp.WrapError(err, "TotalDowntime") + return + } + case "LastOnline": + z.LastOnline, bts, err = msgp.ReadTimeBytes(bts) + if err != nil { + err = msgp.WrapError(err, "LastOnline") + return + } + case "Online": + z.Online, bts, err = msgp.ReadBoolBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Online") + return + } + case "Latency": + bts, err = z.Latency.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Latency") + return + } + case "ReplicatedSize": + z.ReplicatedSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + case "ReplicatedCount": + z.ReplicatedCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + case "Failed": + bts, err = z.Failed.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *SRMetric) Msgsize() (s int) { + s = 1 + 13 + msgp.StringPrefixSize + len(z.DeploymentID) + 9 + msgp.StringPrefixSize + len(z.Endpoint) + 14 + msgp.DurationSize + 11 + msgp.TimeSize + 7 + msgp.BoolSize + 8 + z.Latency.Msgsize() + 15 + msgp.Int64Size + 16 + msgp.Int64Size + 7 + z.Failed.Msgsize() + return +} + +// DecodeMsg implements msgp.Decodable +func (z *SRMetricsSummary) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "ActiveWorkers": + err = z.ActiveWorkers.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "ActiveWorkers") + return + } + case "ReplicaSize": + z.ReplicaSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + case "ReplicaCount": + z.ReplicaCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + case "Queued": + err = z.Queued.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Queued") + return + } + case "Metrics": + var zb0002 uint32 + zb0002, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Metrics") + return + } + if z.Metrics == nil { + z.Metrics = make(map[string]SRMetric, zb0002) + } else if len(z.Metrics) > 0 { + for key := range z.Metrics { + delete(z.Metrics, key) + } + } + for zb0002 > 0 { + zb0002-- + var za0001 string + var za0002 SRMetric + za0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Metrics") + return + } + err = za0002.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Metrics", za0001) + return + } + z.Metrics[za0001] = za0002 + } + case "Uptime": + z.Uptime, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *SRMetricsSummary) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 6 + // write "ActiveWorkers" + err = en.Append(0x86, 0xad, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73) + if err != nil { + return + } + err = z.ActiveWorkers.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "ActiveWorkers") + return + } + // write "ReplicaSize" + err = en.Append(0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicaSize) + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + // write "ReplicaCount" + err = en.Append(0xac, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicaCount) + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + // write "Queued" + err = en.Append(0xa6, 0x51, 0x75, 0x65, 0x75, 0x65, 0x64) + if err != nil { + return + } + err = z.Queued.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Queued") + return + } + // write "Metrics" + err = en.Append(0xa7, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73) + if err != nil { + return + } + err = en.WriteMapHeader(uint32(len(z.Metrics))) + if err != nil { + err = msgp.WrapError(err, "Metrics") + return + } + for za0001, za0002 := range z.Metrics { + err = en.WriteString(za0001) + if err != nil { + err = msgp.WrapError(err, "Metrics") + return + } + err = za0002.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Metrics", za0001) + return + } + } + // write "Uptime" + err = en.Append(0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.Uptime) + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *SRMetricsSummary) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 6 + // string "ActiveWorkers" + o = append(o, 0x86, 0xad, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73) + o, err = z.ActiveWorkers.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "ActiveWorkers") + return + } + // string "ReplicaSize" + o = append(o, 0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.ReplicaSize) + // string "ReplicaCount" + o = append(o, 0xac, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.ReplicaCount) + // string "Queued" + o = append(o, 0xa6, 0x51, 0x75, 0x65, 0x75, 0x65, 0x64) + o, err = z.Queued.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Queued") + return + } + // string "Metrics" + o = append(o, 0xa7, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73) + o = msgp.AppendMapHeader(o, uint32(len(z.Metrics))) + for za0001, za0002 := range z.Metrics { + o = msgp.AppendString(o, za0001) + o, err = za0002.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Metrics", za0001) + return + } + } + // string "Uptime" + o = append(o, 0xa6, 0x55, 0x70, 0x74, 0x69, 0x6d, 0x65) + o = msgp.AppendInt64(o, z.Uptime) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *SRMetricsSummary) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "ActiveWorkers": + bts, err = z.ActiveWorkers.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "ActiveWorkers") + return + } + case "ReplicaSize": + z.ReplicaSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + case "ReplicaCount": + z.ReplicaCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + case "Queued": + bts, err = z.Queued.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Queued") + return + } + case "Metrics": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Metrics") + return + } + if z.Metrics == nil { + z.Metrics = make(map[string]SRMetric, zb0002) + } else if len(z.Metrics) > 0 { + for key := range z.Metrics { + delete(z.Metrics, key) + } + } + for zb0002 > 0 { + var za0001 string + var za0002 SRMetric + zb0002-- + za0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Metrics") + return + } + bts, err = za0002.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Metrics", za0001) + return + } + z.Metrics[za0001] = za0002 + } + case "Uptime": + z.Uptime, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Uptime") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *SRMetricsSummary) Msgsize() (s int) { + s = 1 + 14 + z.ActiveWorkers.Msgsize() + 12 + msgp.Int64Size + 13 + msgp.Int64Size + 7 + z.Queued.Msgsize() + 8 + msgp.MapHeaderSize + if z.Metrics != nil { + for za0001, za0002 := range z.Metrics { + _ = za0002 + s += msgp.StringPrefixSize + len(za0001) + za0002.Msgsize() + } + } + s += 7 + msgp.Int64Size + return +} + +// DecodeMsg implements msgp.Decodable +func (z *SRStats) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "ReplicaSize": + z.ReplicaSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + case "ReplicaCount": + z.ReplicaCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + case "M": + var zb0002 uint32 + zb0002, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "M") + return + } + if z.M == nil { + z.M = make(map[string]*SRStatus, zb0002) + } else if len(z.M) > 0 { + for key := range z.M { + delete(z.M, key) + } + } + for zb0002 > 0 { + zb0002-- + var za0001 string + var za0002 *SRStatus + za0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "M") + return + } + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "M", za0001) + return + } + za0002 = nil + } else { + if za0002 == nil { + za0002 = new(SRStatus) + } + err = za0002.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "M", za0001) + return + } + } + z.M[za0001] = za0002 + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *SRStats) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 3 + // write "ReplicaSize" + err = en.Append(0x83, 0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicaSize) + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + // write "ReplicaCount" + err = en.Append(0xac, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicaCount) + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + // write "M" + err = en.Append(0xa1, 0x4d) + if err != nil { + return + } + err = en.WriteMapHeader(uint32(len(z.M))) + if err != nil { + err = msgp.WrapError(err, "M") + return + } + for za0001, za0002 := range z.M { + err = en.WriteString(za0001) + if err != nil { + err = msgp.WrapError(err, "M") + return + } + if za0002 == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = za0002.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "M", za0001) + return + } + } + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *SRStats) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 3 + // string "ReplicaSize" + o = append(o, 0x83, 0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.ReplicaSize) + // string "ReplicaCount" + o = append(o, 0xac, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.ReplicaCount) + // string "M" + o = append(o, 0xa1, 0x4d) + o = msgp.AppendMapHeader(o, uint32(len(z.M))) + for za0001, za0002 := range z.M { + o = msgp.AppendString(o, za0001) + if za0002 == nil { + o = msgp.AppendNil(o) + } else { + o, err = za0002.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "M", za0001) + return + } + } + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *SRStats) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "ReplicaSize": + z.ReplicaSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + case "ReplicaCount": + z.ReplicaCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaCount") + return + } + case "M": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "M") + return + } + if z.M == nil { + z.M = make(map[string]*SRStatus, zb0002) + } else if len(z.M) > 0 { + for key := range z.M { + delete(z.M, key) + } + } + for zb0002 > 0 { + var za0001 string + var za0002 *SRStatus + zb0002-- + za0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "M") + return + } + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + za0002 = nil + } else { + if za0002 == nil { + za0002 = new(SRStatus) + } + bts, err = za0002.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "M", za0001) + return + } + } + z.M[za0001] = za0002 + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *SRStats) Msgsize() (s int) { + s = 1 + 12 + msgp.Int64Size + 13 + msgp.Int64Size + 2 + msgp.MapHeaderSize + if z.M != nil { + for za0001, za0002 := range z.M { + _ = za0002 + s += msgp.StringPrefixSize + len(za0001) + if za0002 == nil { + s += msgp.NilSize + } else { + s += za0002.Msgsize() + } + } + } + return +} + +// DecodeMsg implements msgp.Decodable +func (z *SRStatus) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "ReplicatedSize": + z.ReplicatedSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + case "Failed": + err = z.Failed.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + case "ReplicatedCount": + z.ReplicatedCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + case "Latency": + err = z.Latency.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Latency") + return + } + case "lt": + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + z.XferRateLrg = nil + } else { + if z.XferRateLrg == nil { + z.XferRateLrg = new(XferStats) + } + err = z.XferRateLrg.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + } + case "st": + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + z.XferRateSml = nil + } else { + if z.XferRateSml == nil { + z.XferRateSml = new(XferStats) + } + err = z.XferRateSml.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + } + case "Endpoint": + z.Endpoint, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Endpoint") + return + } + case "Secure": + z.Secure, err = dc.ReadBool() + if err != nil { + err = msgp.WrapError(err, "Secure") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *SRStatus) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 8 + // write "ReplicatedSize" + err = en.Append(0x88, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicatedSize) + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + // write "Failed" + err = en.Append(0xa6, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64) + if err != nil { + return + } + err = z.Failed.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + // write "ReplicatedCount" + err = en.Append(0xaf, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicatedCount) + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + // write "Latency" + err = en.Append(0xa7, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79) + if err != nil { + return + } + err = z.Latency.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Latency") + return + } + // write "lt" + err = en.Append(0xa2, 0x6c, 0x74) + if err != nil { + return + } + if z.XferRateLrg == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = z.XferRateLrg.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + } + // write "st" + err = en.Append(0xa2, 0x73, 0x74) + if err != nil { + return + } + if z.XferRateSml == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = z.XferRateSml.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + } + // write "Endpoint" + err = en.Append(0xa8, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteString(z.Endpoint) + if err != nil { + err = msgp.WrapError(err, "Endpoint") + return + } + // write "Secure" + err = en.Append(0xa6, 0x53, 0x65, 0x63, 0x75, 0x72, 0x65) + if err != nil { + return + } + err = en.WriteBool(z.Secure) + if err != nil { + err = msgp.WrapError(err, "Secure") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *SRStatus) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 8 + // string "ReplicatedSize" + o = append(o, 0x88, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.ReplicatedSize) + // string "Failed" + o = append(o, 0xa6, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64) + o, err = z.Failed.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + // string "ReplicatedCount" + o = append(o, 0xaf, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.ReplicatedCount) + // string "Latency" + o = append(o, 0xa7, 0x4c, 0x61, 0x74, 0x65, 0x6e, 0x63, 0x79) + o, err = z.Latency.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Latency") + return + } + // string "lt" + o = append(o, 0xa2, 0x6c, 0x74) + if z.XferRateLrg == nil { + o = msgp.AppendNil(o) + } else { + o, err = z.XferRateLrg.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + } + // string "st" + o = append(o, 0xa2, 0x73, 0x74) + if z.XferRateSml == nil { + o = msgp.AppendNil(o) + } else { + o, err = z.XferRateSml.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + } + // string "Endpoint" + o = append(o, 0xa8, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74) + o = msgp.AppendString(o, z.Endpoint) + // string "Secure" + o = append(o, 0xa6, 0x53, 0x65, 0x63, 0x75, 0x72, 0x65) + o = msgp.AppendBool(o, z.Secure) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *SRStatus) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "ReplicatedSize": + z.ReplicatedSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + case "Failed": + bts, err = z.Failed.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Failed") + return + } + case "ReplicatedCount": + z.ReplicatedCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicatedCount") + return + } + case "Latency": + bts, err = z.Latency.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Latency") + return + } + case "lt": + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + z.XferRateLrg = nil + } else { + if z.XferRateLrg == nil { + z.XferRateLrg = new(XferStats) + } + bts, err = z.XferRateLrg.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "XferRateLrg") + return + } + } + case "st": + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + z.XferRateSml = nil + } else { + if z.XferRateSml == nil { + z.XferRateSml = new(XferStats) + } + bts, err = z.XferRateSml.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "XferRateSml") + return + } + } + case "Endpoint": + z.Endpoint, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Endpoint") + return + } + case "Secure": + z.Secure, bts, err = msgp.ReadBoolBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Secure") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *SRStatus) Msgsize() (s int) { + s = 1 + 15 + msgp.Int64Size + 7 + z.Failed.Msgsize() + 16 + msgp.Int64Size + 8 + z.Latency.Msgsize() + 3 + if z.XferRateLrg == nil { + s += msgp.NilSize + } else { + s += z.XferRateLrg.Msgsize() + } + s += 3 + if z.XferRateSml == nil { + s += msgp.NilSize + } else { + s += z.XferRateSml.Msgsize() + } + s += 9 + msgp.StringPrefixSize + len(z.Endpoint) + 7 + msgp.BoolSize + return +} diff --git a/cmd/site-replication-metrics_gen_test.go b/cmd/site-replication-metrics_gen_test.go new file mode 100644 index 000000000..0aa1598d7 --- /dev/null +++ b/cmd/site-replication-metrics_gen_test.go @@ -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) + } + } +} diff --git a/cmd/site-replication.go b/cmd/site-replication.go index 59af6526a..206dc29fb 100644 --- a/cmd/site-replication.go +++ b/cmd/site-replication.go @@ -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 +} diff --git a/cmd/xl-storage.go b/cmd/xl-storage.go index ddef7dea4..87f09eb30 100644 --- a/cmd/xl-storage.go +++ b/cmd/xl-storage.go @@ -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) } diff --git a/docs/metrics/prometheus/list.md b/docs/metrics/prometheus/list.md index 1eeb1ef93..664dc629a 100644 --- a/docs/metrics/prometheus/list.md +++ b/docs/metrics/prometheus/list.md @@ -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 | diff --git a/go.mod b/go.mod index c01a0a62c..eb2b1646f 100644 --- a/go.mod +++ b/go.mod @@ -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 diff --git a/go.sum b/go.sum index 0b5dadfb7..d988ee929 100644 --- a/go.sum +++ b/go.sum @@ -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=