Fix uninitialized replication stats (#20260)

Services are unfrozen before `initBackgroundReplication` is finished. This means that 
the globalReplicationStats write is racy. Switch to an atomic pointer.

Provide the `ReplicationPool` with the stats, so it doesn't have to be grabbed 
from the atomic pointer on every use.

All other loads and checks are nil, and calls return empty values when stats 
still haven't been initialized.
This commit is contained in:
Klaus Post
2024-08-15 05:04:40 -07:00
committed by GitHub
parent 3b1aa40372
commit f1302c40fe
19 changed files with 175 additions and 101 deletions

View File

@@ -50,6 +50,7 @@ import (
xhttp "github.com/minio/minio/internal/http"
xioutil "github.com/minio/minio/internal/ioutil"
"github.com/minio/minio/internal/logger"
"github.com/minio/minio/internal/once"
"github.com/tinylib/msgp/msgp"
"github.com/zeebo/xxh3"
"golang.org/x/exp/maps"
@@ -478,7 +479,7 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, obj
lk := objectAPI.NewNSLock(bucket, "/[replicate]/"+dobj.ObjectName)
lkctx, err := lk.GetLock(ctx, globalOperationTimeout)
if err != nil {
globalReplicationPool.queueMRFSave(dobj.ToMRFEntry())
globalReplicationPool.Get().queueMRFSave(dobj.ToMRFEntry())
sendEvent(eventArgs{
BucketName: bucket,
Object: ObjectInfo{
@@ -548,7 +549,7 @@ 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, replicationStatus,
globalReplicationStats.Load().Update(dobj.Bucket, rinfo, replicationStatus,
prevStatus)
}
}
@@ -556,7 +557,7 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, obj
eventName := event.ObjectReplicationComplete
if replicationStatus == replication.Failed {
eventName = event.ObjectReplicationFailed
globalReplicationPool.queueMRFSave(dobj.ToMRFEntry())
globalReplicationPool.Get().queueMRFSave(dobj.ToMRFEntry())
}
drs := getReplicationState(rinfos, dobj.ReplicationState, dobj.VersionID)
if replicationStatus != prevStatus {
@@ -1054,7 +1055,7 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje
UserAgent: "Internal: [Replication]",
Host: globalLocalNodeName,
})
globalReplicationPool.queueMRFSave(ri.ToMRFEntry())
globalReplicationPool.Get().queueMRFSave(ri.ToMRFEntry())
return
}
ctx = lkctx.Context()
@@ -1139,7 +1140,7 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje
for _, rinfo := range rinfos.Targets {
if rinfo.ReplicationStatus != rinfo.PrevReplicationStatus {
rinfo.OpType = opType // update optype to reflect correct operation.
globalReplicationStats.Update(bucket, rinfo, rinfo.ReplicationStatus, rinfo.PrevReplicationStatus)
globalReplicationStats.Load().Update(bucket, rinfo, rinfo.ReplicationStatus, rinfo.PrevReplicationStatus)
}
}
}
@@ -1159,7 +1160,7 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje
ri.EventType = ReplicateMRF
ri.ReplicationStatusInternal = rinfos.ReplicationStatusInternal()
ri.RetryCount++
globalReplicationPool.queueMRFSave(ri.ToMRFEntry())
globalReplicationPool.Get().queueMRFSave(ri.ToMRFEntry())
}
}
@@ -1787,8 +1788,8 @@ const (
)
var (
globalReplicationPool *ReplicationPool
globalReplicationStats *ReplicationStats
globalReplicationPool = once.NewSingleton[ReplicationPool]()
globalReplicationStats atomic.Pointer[ReplicationStats]
)
// ReplicationPool describes replication pool
@@ -1803,6 +1804,7 @@ type ReplicationPool struct {
priority string
maxWorkers int
maxLWorkers int
stats *ReplicationStats
mu sync.RWMutex
mrfMU sync.Mutex
@@ -1849,7 +1851,7 @@ const (
)
// NewReplicationPool creates a pool of replication workers of specified size
func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPoolOpts) *ReplicationPool {
func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPoolOpts, stats *ReplicationStats) *ReplicationPool {
var workers, failedWorkers int
priority := "auto"
maxWorkers := WorkerMaxLimit
@@ -1891,6 +1893,7 @@ func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPool
mrfStopCh: make(chan struct{}, 1),
ctx: ctx,
objLayer: o,
stats: stats,
priority: priority,
maxWorkers: maxWorkers,
maxLWorkers: maxLWorkers,
@@ -1918,11 +1921,11 @@ func (p *ReplicationPool) AddMRFWorker() {
}
switch v := oi.(type) {
case ReplicateObjectInfo:
globalReplicationStats.incQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
p.stats.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)
p.stats.decQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
default:
bugLogIf(p.ctx, fmt.Errorf("unknown mrf replication type: %T", oi), "unknown-mrf-replicate-type")
@@ -1950,9 +1953,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)
p.stats.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)
p.stats.decQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
if opTracker != nil {
atomic.AddInt32(opTracker, -1)
}
@@ -1960,10 +1963,10 @@ func (p *ReplicationPool) AddWorker(input <-chan ReplicationWorkerOperation, opT
if opTracker != nil {
atomic.AddInt32(opTracker, 1)
}
globalReplicationStats.incQ(v.Bucket, 0, true, v.OpType)
p.stats.incQ(v.Bucket, 0, true, v.OpType)
replicateDelete(p.ctx, v, p.objLayer)
globalReplicationStats.decQ(v.Bucket, 0, true, v.OpType)
p.stats.decQ(v.Bucket, 0, true, v.OpType)
if opTracker != nil {
atomic.AddInt32(opTracker, -1)
@@ -1990,9 +1993,9 @@ func (p *ReplicationPool) AddLargeWorker(input <-chan ReplicationWorkerOperation
if opTracker != nil {
atomic.AddInt32(opTracker, 1)
}
globalReplicationStats.incQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
p.stats.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)
p.stats.decQ(v.Bucket, v.Size, v.DeleteMarker, v.OpType)
if opTracker != nil {
atomic.AddInt32(opTracker, -1)
}
@@ -2156,7 +2159,7 @@ func (p *ReplicationPool) queueReplicaTask(ri ReplicateObjectInfo) {
case <-p.ctx.Done():
case p.lrgworkers[h%uint64(len(p.lrgworkers))] <- ri:
default:
globalReplicationPool.queueMRFSave(ri.ToMRFEntry())
p.queueMRFSave(ri.ToMRFEntry())
p.mu.RLock()
maxLWorkers := p.maxLWorkers
existing := len(p.lrgworkers)
@@ -2187,7 +2190,7 @@ func (p *ReplicationPool) queueReplicaTask(ri ReplicateObjectInfo) {
case healCh <- ri:
case ch <- ri:
default:
globalReplicationPool.queueMRFSave(ri.ToMRFEntry())
globalReplicationPool.Get().queueMRFSave(ri.ToMRFEntry())
p.mu.RLock()
prio := p.priority
maxWorkers := p.maxWorkers
@@ -2223,7 +2226,7 @@ func queueReplicateDeletesWrapper(doi DeletedObjectReplicationInfo, existingObje
doi.ResetID = v.ResetID
doi.TargetArn = k
globalReplicationPool.queueReplicaDeleteTask(doi)
globalReplicationPool.Get().queueReplicaDeleteTask(doi)
}
}
}
@@ -2244,7 +2247,7 @@ func (p *ReplicationPool) queueReplicaDeleteTask(doi DeletedObjectReplicationInf
case <-p.ctx.Done():
case ch <- doi:
default:
globalReplicationPool.queueMRFSave(doi.ToMRFEntry())
p.queueMRFSave(doi.ToMRFEntry())
p.mu.RLock()
prio := p.priority
maxWorkers := p.maxWorkers
@@ -2274,9 +2277,10 @@ type replicationPoolOpts struct {
}
func initBackgroundReplication(ctx context.Context, objectAPI ObjectLayer) {
globalReplicationPool = NewReplicationPool(ctx, objectAPI, globalAPIConfig.getReplicationOpts())
globalReplicationStats = NewReplicationStats(ctx, objectAPI)
go globalReplicationStats.trackEWMA()
stats := NewReplicationStats(ctx, objectAPI)
globalReplicationPool.Set(NewReplicationPool(ctx, objectAPI, globalAPIConfig.getReplicationOpts(), stats))
globalReplicationStats.Store(stats)
go stats.trackEWMA()
}
type proxyResult struct {
@@ -2482,7 +2486,7 @@ func scheduleReplication(ctx context.Context, oi ObjectInfo, o ObjectLayer, dsc
if dsc.Synchronous() {
replicateObject(ctx, ri, o)
} else {
globalReplicationPool.queueReplicaTask(ri)
globalReplicationPool.Get().queueReplicaTask(ri)
}
}
@@ -2610,9 +2614,9 @@ func proxyGetTaggingToRepTarget(ctx context.Context, bucket, object string, opts
}
func scheduleReplicationDelete(ctx context.Context, dv DeletedObjectReplicationInfo, o ObjectLayer) {
globalReplicationPool.queueReplicaDeleteTask(dv)
globalReplicationPool.Get().queueReplicaDeleteTask(dv)
for arn := range dv.ReplicationState.Targets {
globalReplicationStats.Update(dv.Bucket, replicatedTargetInfo{Arn: arn, Size: 0, Duration: 0, OpType: replication.DeleteReplicationType}, replication.Pending, replication.StatusType(""))
globalReplicationStats.Load().Update(dv.Bucket, replicatedTargetInfo{Arn: arn, Size: 0, Duration: 0, OpType: replication.DeleteReplicationType}, replication.Pending, replication.StatusType(""))
}
}
@@ -3042,9 +3046,9 @@ func (s *replicationResyncer) start(ctx context.Context, objAPI ObjectLayer, opt
if len(tgtArns) == 0 {
return fmt.Errorf("arn %s specified for resync not found in replication config", opts.arn)
}
globalReplicationPool.resyncer.RLock()
data, ok := globalReplicationPool.resyncer.statusMap[opts.bucket]
globalReplicationPool.resyncer.RUnlock()
globalReplicationPool.Get().resyncer.RLock()
data, ok := globalReplicationPool.Get().resyncer.statusMap[opts.bucket]
globalReplicationPool.Get().resyncer.RUnlock()
if !ok {
data, err = loadBucketResyncMetadata(ctx, opts.bucket, objAPI)
if err != nil {
@@ -3070,9 +3074,9 @@ func (s *replicationResyncer) start(ctx context.Context, objAPI ObjectLayer, opt
return err
}
globalReplicationPool.resyncer.Lock()
defer globalReplicationPool.resyncer.Unlock()
brs, ok := globalReplicationPool.resyncer.statusMap[opts.bucket]
globalReplicationPool.Get().resyncer.Lock()
defer globalReplicationPool.Get().resyncer.Unlock()
brs, ok := globalReplicationPool.Get().resyncer.statusMap[opts.bucket]
if !ok {
brs = BucketReplicationResyncStatus{
Version: resyncMetaVersion,
@@ -3080,8 +3084,8 @@ func (s *replicationResyncer) start(ctx context.Context, objAPI ObjectLayer, opt
}
}
brs.TargetsMap[opts.arn] = status
globalReplicationPool.resyncer.statusMap[opts.bucket] = brs
go globalReplicationPool.resyncer.resyncBucket(GlobalContext, objAPI, false, opts)
globalReplicationPool.Get().resyncer.statusMap[opts.bucket] = brs
go globalReplicationPool.Get().resyncer.resyncBucket(GlobalContext, objAPI, false, opts)
return nil
}
@@ -3413,7 +3417,7 @@ func queueReplicationHeal(ctx context.Context, bucket string, oi ObjectInfo, rcf
if roi.ReplicationStatus == replication.Pending ||
roi.ReplicationStatus == replication.Failed ||
roi.VersionPurgeStatus == Failed || roi.VersionPurgeStatus == Pending {
globalReplicationPool.queueReplicaDeleteTask(dv)
globalReplicationPool.Get().queueReplicaDeleteTask(dv)
return
}
// if replication status is Complete on DeleteMarker and existing object resync required
@@ -3429,12 +3433,12 @@ func queueReplicationHeal(ctx context.Context, bucket string, oi ObjectInfo, rcf
switch roi.ReplicationStatus {
case replication.Pending, replication.Failed:
roi.EventType = ReplicateHeal
globalReplicationPool.queueReplicaTask(roi)
globalReplicationPool.Get().queueReplicaTask(roi)
return
}
if roi.ExistingObjResync.mustResync() {
roi.EventType = ReplicateExisting
globalReplicationPool.queueReplicaTask(roi)
globalReplicationPool.Get().queueReplicaTask(roi)
}
return
}
@@ -3499,8 +3503,8 @@ func (p *ReplicationPool) queueMRFSave(entry MRFReplicateEntry) {
return
}
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))
atomic.AddUint64(&p.stats.mrfStats.TotalDroppedCount, 1)
atomic.AddUint64(&p.stats.mrfStats.TotalDroppedBytes, uint64(entry.sz))
return
}
@@ -3513,8 +3517,8 @@ func (p *ReplicationPool) queueMRFSave(entry MRFReplicateEntry) {
select {
case p.mrfSaveCh <- entry:
default:
atomic.AddUint64(&globalReplicationStats.mrfStats.TotalDroppedCount, 1)
atomic.AddUint64(&globalReplicationStats.mrfStats.TotalDroppedBytes, uint64(entry.sz))
atomic.AddUint64(&p.stats.mrfStats.TotalDroppedCount, 1)
atomic.AddUint64(&p.stats.mrfStats.TotalDroppedBytes, uint64(entry.sz))
}
}
}
@@ -3563,7 +3567,7 @@ func (p *ReplicationPool) saveMRFEntries(ctx context.Context, entries map[string
if !p.initialized() {
return
}
atomic.StoreUint64(&globalReplicationStats.mrfStats.LastFailedCount, uint64(len(entries)))
atomic.StoreUint64(&p.stats.mrfStats.LastFailedCount, uint64(len(entries)))
if len(entries) == 0 {
return
}