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=