mirror of
https://github.com/minio/minio.git
synced 2025-11-07 12:52:58 -05:00
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:
@@ -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
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user