mirror of
https://github.com/minio/minio.git
synced 2025-11-21 02:09:08 -05:00
Add detailed scanner metrics (#15161)
This commit is contained in:
@@ -31,7 +31,6 @@ import (
|
||||
"path"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/bits-and-blooms/bloom/v3"
|
||||
@@ -66,7 +65,7 @@ var (
|
||||
|
||||
dataScannerLeaderLockTimeout = newDynamicTimeout(30*time.Second, 10*time.Second)
|
||||
// Sleeper values are updated when config is loaded.
|
||||
scannerSleeper = newDynamicSleeper(10, 10*time.Second)
|
||||
scannerSleeper = newDynamicSleeper(10, 10*time.Second, true)
|
||||
scannerCycle = uatomic.NewDuration(dataScannerStartDelay)
|
||||
)
|
||||
|
||||
@@ -162,35 +161,45 @@ func runDataScanner(pctx context.Context, objAPI ObjectLayer) {
|
||||
// No unlock for "leader" lock.
|
||||
|
||||
// Load current bloom cycle
|
||||
nextBloomCycle := intDataUpdateTracker.current() + 1
|
||||
var cycleInfo currentScannerCycle
|
||||
cycleInfo.next = intDataUpdateTracker.current() + 1
|
||||
|
||||
buf, _ := readConfig(ctx, objAPI, dataUsageBloomNamePath)
|
||||
if len(buf) >= 8 {
|
||||
if err = binary.Read(bytes.NewReader(buf), binary.LittleEndian, &nextBloomCycle); err != nil {
|
||||
logger.LogIf(ctx, err)
|
||||
}
|
||||
if len(buf) == 8 {
|
||||
cycleInfo.next = binary.LittleEndian.Uint64(buf)
|
||||
} else if len(buf) > 8 {
|
||||
cycleInfo.next = binary.LittleEndian.Uint64(buf[:8])
|
||||
buf = buf[8:]
|
||||
_, err = cycleInfo.UnmarshalMsg(buf)
|
||||
logger.LogIf(ctx, err)
|
||||
}
|
||||
|
||||
scannerTimer := time.NewTimer(scannerCycle.Load())
|
||||
defer scannerTimer.Stop()
|
||||
defer globalScannerMetrics.setCycle(nil)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-scannerTimer.C:
|
||||
if intDataUpdateTracker.debug {
|
||||
console.Debugln("starting scanner cycle")
|
||||
}
|
||||
// Reset the timer for next cycle.
|
||||
// If scanner takes longer we start at once.
|
||||
scannerTimer.Reset(scannerCycle.Load())
|
||||
|
||||
stopFn := globalScannerMetrics.log(scannerMetricScanCycle)
|
||||
cycleInfo.current = cycleInfo.next
|
||||
cycleInfo.started = time.Now()
|
||||
globalScannerMetrics.setCycle(&cycleInfo)
|
||||
|
||||
bgHealInfo := readBackgroundHealInfo(ctx, objAPI)
|
||||
scanMode := getCycleScanMode(nextBloomCycle, bgHealInfo.BitrotStartCycle, bgHealInfo.BitrotStartTime)
|
||||
scanMode := getCycleScanMode(cycleInfo.current, bgHealInfo.BitrotStartCycle, bgHealInfo.BitrotStartTime)
|
||||
if bgHealInfo.CurrentScanMode != scanMode {
|
||||
newHealInfo := bgHealInfo
|
||||
newHealInfo.CurrentScanMode = scanMode
|
||||
if scanMode == madmin.HealDeepScan {
|
||||
newHealInfo.BitrotStartTime = time.Now().UTC()
|
||||
newHealInfo.BitrotStartCycle = nextBloomCycle
|
||||
newHealInfo.BitrotStartCycle = cycleInfo.current
|
||||
}
|
||||
saveBackgroundHealInfo(ctx, objAPI, newHealInfo)
|
||||
}
|
||||
@@ -198,22 +207,27 @@ func runDataScanner(pctx context.Context, objAPI ObjectLayer) {
|
||||
// Wait before starting next cycle and wait on startup.
|
||||
results := make(chan DataUsageInfo, 1)
|
||||
go storeDataUsageInBackend(ctx, objAPI, results)
|
||||
bf, err := globalNotificationSys.updateBloomFilter(ctx, nextBloomCycle)
|
||||
bf, err := globalNotificationSys.updateBloomFilter(ctx, cycleInfo.current)
|
||||
logger.LogIf(ctx, err)
|
||||
err = objAPI.NSScanner(ctx, bf, results, uint32(nextBloomCycle), scanMode)
|
||||
err = objAPI.NSScanner(ctx, bf, results, uint32(cycleInfo.current), scanMode)
|
||||
logger.LogIf(ctx, err)
|
||||
stopFn()
|
||||
if err == nil {
|
||||
// Store new cycle...
|
||||
nextBloomCycle++
|
||||
var tmp [8]byte
|
||||
binary.LittleEndian.PutUint64(tmp[:], nextBloomCycle)
|
||||
if err = saveConfig(ctx, objAPI, dataUsageBloomNamePath, tmp[:]); err != nil {
|
||||
logger.LogIf(ctx, err)
|
||||
cycleInfo.next++
|
||||
cycleInfo.current = 0
|
||||
cycleInfo.cycleCompleted = append(cycleInfo.cycleCompleted, time.Now())
|
||||
if len(cycleInfo.cycleCompleted) > dataUsageUpdateDirCycles {
|
||||
cycleInfo.cycleCompleted = cycleInfo.cycleCompleted[len(cycleInfo.cycleCompleted)-dataUsageUpdateDirCycles:]
|
||||
}
|
||||
globalScannerMetrics.setCycle(&cycleInfo)
|
||||
tmp := make([]byte, 8, 8+cycleInfo.Msgsize())
|
||||
// Cycle for backward compat.
|
||||
binary.LittleEndian.PutUint64(tmp, cycleInfo.next)
|
||||
tmp, _ = cycleInfo.MarshalMsg(tmp)
|
||||
err = saveConfig(ctx, objAPI, dataUsageBloomNamePath, tmp)
|
||||
logger.LogIf(ctx, err)
|
||||
}
|
||||
|
||||
// Reset the timer for next cycle.
|
||||
scannerTimer.Reset(scannerCycle.Load())
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -244,24 +258,11 @@ type folderScanner struct {
|
||||
// Will not be closed when returned.
|
||||
updates chan<- dataUsageEntry
|
||||
lastUpdate time.Time
|
||||
|
||||
// updateCurrentPath should be called whenever a new path is scanned.
|
||||
updateCurrentPath func(string)
|
||||
}
|
||||
|
||||
type scannerStats struct {
|
||||
// All fields must be accessed atomically and aligned.
|
||||
|
||||
accTotalObjects uint64
|
||||
accTotalVersions uint64
|
||||
accFolders uint64
|
||||
bucketsStarted uint64
|
||||
bucketsFinished uint64
|
||||
ilmChecks uint64
|
||||
|
||||
// actions records actions performed.
|
||||
actions [lifecycle.ActionCount]uint64
|
||||
}
|
||||
|
||||
var globalScannerStats scannerStats
|
||||
|
||||
// Cache structure and compaction:
|
||||
//
|
||||
// A cache structure will be kept with a tree of usages.
|
||||
@@ -305,24 +306,14 @@ var globalScannerStats scannerStats
|
||||
// Before each operation sleepDuration is called which can be used to temporarily halt the scanner.
|
||||
// If the supplied context is canceled the function will return at the first chance.
|
||||
func scanDataFolder(ctx context.Context, poolIdx, setIdx int, basePath string, cache dataUsageCache, getSize getSizeFn, scanMode madmin.HealScanMode) (dataUsageCache, error) {
|
||||
t := UTCNow()
|
||||
|
||||
logPrefix := color.Green("data-usage: ")
|
||||
logSuffix := color.Blue("- %v + %v", basePath, cache.Info.Name)
|
||||
atomic.AddUint64(&globalScannerStats.bucketsStarted, 1)
|
||||
defer func() {
|
||||
atomic.AddUint64(&globalScannerStats.bucketsFinished, 1)
|
||||
}()
|
||||
if intDataUpdateTracker.debug {
|
||||
defer func() {
|
||||
console.Debugf(logPrefix+" Scanner time: %v %s\n", time.Since(t), logSuffix)
|
||||
}()
|
||||
}
|
||||
|
||||
switch cache.Info.Name {
|
||||
case "", dataUsageRoot:
|
||||
return cache, errors.New("internal error: root scan attempted")
|
||||
}
|
||||
updatePath, closeDisk := globalScannerMetrics.currentPathUpdater(basePath, cache.Info.Name)
|
||||
defer closeDisk()
|
||||
|
||||
s := folderScanner{
|
||||
root: basePath,
|
||||
@@ -335,6 +326,7 @@ func scanDataFolder(ctx context.Context, poolIdx, setIdx int, basePath string, c
|
||||
healObjectSelect: 0,
|
||||
scanMode: scanMode,
|
||||
updates: cache.Info.updates,
|
||||
updateCurrentPath: updatePath,
|
||||
}
|
||||
|
||||
// Add disks for set healing.
|
||||
@@ -366,14 +358,8 @@ func scanDataFolder(ctx context.Context, poolIdx, setIdx int, basePath string, c
|
||||
s.withFilter = nil
|
||||
}
|
||||
}
|
||||
if s.dataUsageScannerDebug {
|
||||
console.Debugf(logPrefix+"Start scanning. Bloom filter: %v %s\n", s.withFilter != nil, logSuffix)
|
||||
}
|
||||
|
||||
done := ctx.Done()
|
||||
if s.dataUsageScannerDebug {
|
||||
console.Debugf(logPrefix+"Cycle: %v, Entries: %v %s\n", cache.Info.NextCycle, len(cache.Cache), logSuffix)
|
||||
}
|
||||
|
||||
// Read top level in bucket.
|
||||
select {
|
||||
@@ -389,9 +375,6 @@ func scanDataFolder(ctx context.Context, poolIdx, setIdx int, basePath string, c
|
||||
return cache, err
|
||||
}
|
||||
|
||||
if s.dataUsageScannerDebug {
|
||||
console.Debugf(logPrefix+"Finished scanner, %v entries (%+v) %s \n", len(s.newCache.Cache), *s.newCache.sizeRecursive(s.newCache.Info.Name), logSuffix)
|
||||
}
|
||||
s.newCache.Info.LastUpdate = UTCNow()
|
||||
s.newCache.Info.NextCycle = cache.Info.NextCycle
|
||||
return s.newCache, nil
|
||||
@@ -420,10 +403,10 @@ func (f *folderScanner) sendUpdate() {
|
||||
func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, into *dataUsageEntry) error {
|
||||
done := ctx.Done()
|
||||
scannerLogPrefix := color.Green("folder-scanner:")
|
||||
|
||||
thisHash := hashPath(folder.name)
|
||||
// Store initial compaction state.
|
||||
wasCompacted := into.Compacted
|
||||
atomic.AddUint64(&globalScannerStats.accFolders, 1)
|
||||
|
||||
for {
|
||||
select {
|
||||
@@ -648,7 +631,10 @@ func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, int
|
||||
f.updateCache.replaceHashed(h, &thisHash, dataUsageEntry{})
|
||||
}
|
||||
}
|
||||
f.updateCurrentPath(folder.name)
|
||||
stopFn := globalScannerMetrics.log(scannerMetricScanFolder, f.root, folder.name)
|
||||
scanFolder(folder)
|
||||
stopFn()
|
||||
// Add new folders if this is new and we don't have existing.
|
||||
if !into.Compacted {
|
||||
parent := f.updateCache.find(thisHash.Key())
|
||||
@@ -676,7 +662,10 @@ func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, int
|
||||
folder.objectHealProbDiv = f.healFolderInclude
|
||||
}
|
||||
}
|
||||
f.updateCurrentPath(folder.name)
|
||||
stopFn := globalScannerMetrics.log(scannerMetricScanFolder, f.root, folder.name, "EXISTING")
|
||||
scanFolder(folder)
|
||||
stopFn()
|
||||
}
|
||||
|
||||
// Scan for healing
|
||||
@@ -717,9 +706,8 @@ func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, int
|
||||
healObjectsPrefix := color.Green("healObjects:")
|
||||
for k := range abandonedChildren {
|
||||
bucket, prefix := path2BucketObject(k)
|
||||
if f.dataUsageScannerDebug {
|
||||
console.Debugf(scannerLogPrefix+" checking disappeared folder: %v/%v\n", bucket, prefix)
|
||||
}
|
||||
stopFn := globalScannerMetrics.time(scannerMetricCheckMissing)
|
||||
f.updateCurrentPath(k)
|
||||
|
||||
if bucket != resolver.bucket {
|
||||
// Bucket might be missing as well with abandoned children.
|
||||
@@ -807,6 +795,7 @@ func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, int
|
||||
},
|
||||
})
|
||||
|
||||
stopFn()
|
||||
if f.dataUsageScannerDebug && err != nil && err != errFileNotFound {
|
||||
console.Debugf(healObjectsPrefix+" checking returned value %v (%T)\n", err, err)
|
||||
}
|
||||
@@ -814,7 +803,9 @@ func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, int
|
||||
// Add unless healing returned an error.
|
||||
if foundObjs {
|
||||
this := cachedFolder{name: k, parent: &thisHash, objectHealProbDiv: 1}
|
||||
stopFn := globalScannerMetrics.log(scannerMetricScanFolder, f.root, this.name, "HEALED")
|
||||
scanFolder(this)
|
||||
stopFn()
|
||||
}
|
||||
}
|
||||
break
|
||||
@@ -964,7 +955,6 @@ func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, oi Obje
|
||||
return false, size
|
||||
}
|
||||
|
||||
atomic.AddUint64(&globalScannerStats.ilmChecks, 1)
|
||||
versionID := oi.VersionID
|
||||
rCfg, _ := globalBucketObjectLockSys.Get(i.bucket)
|
||||
action := evalActionFromLifecycle(ctx, *i.lifeCycle, rCfg, oi, false)
|
||||
@@ -975,7 +965,7 @@ func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, oi Obje
|
||||
console.Debugf(applyActionsLogPrefix+" lifecycle: %q Initial scan: %v\n", i.objectPath(), action)
|
||||
}
|
||||
}
|
||||
atomic.AddUint64(&globalScannerStats.actions[action], 1)
|
||||
defer globalScannerMetrics.timeILM(action)
|
||||
|
||||
switch action {
|
||||
case lifecycle.DeleteAction, lifecycle.DeleteVersionAction, lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:
|
||||
@@ -1092,16 +1082,23 @@ func (i *scannerItem) applyVersionActions(ctx context.Context, o ObjectLayer, fi
|
||||
// The metadata will be compared to consensus on the object layer before any changes are applied.
|
||||
// If no metadata is supplied, -1 is returned if no action is taken.
|
||||
func (i *scannerItem) applyActions(ctx context.Context, o ObjectLayer, oi ObjectInfo, sizeS *sizeSummary) int64 {
|
||||
done := globalScannerMetrics.time(scannerMetricILM)
|
||||
applied, size := i.applyLifecycle(ctx, o, oi)
|
||||
done()
|
||||
|
||||
// For instance, an applied lifecycle means we remove/transitioned an object
|
||||
// from the current deployment, which means we don't have to call healing
|
||||
// routine even if we are asked to do via heal flag.
|
||||
if !applied {
|
||||
if i.heal.enabled {
|
||||
done := globalScannerMetrics.time(scannerMetricHealCheck)
|
||||
size = i.applyHealing(ctx, o, oi)
|
||||
done()
|
||||
}
|
||||
// replicate only if lifecycle rules are not applied.
|
||||
done := globalScannerMetrics.time(scannerMetricCheckReplication)
|
||||
i.healReplication(ctx, o, oi.Clone(), sizeS)
|
||||
done()
|
||||
}
|
||||
return size
|
||||
}
|
||||
@@ -1341,15 +1338,20 @@ type dynamicSleeper struct {
|
||||
|
||||
// cycle will be closed
|
||||
cycle chan struct{}
|
||||
|
||||
// isScanner should be set when this is used by the scanner
|
||||
// to record metrics.
|
||||
isScanner bool
|
||||
}
|
||||
|
||||
// newDynamicSleeper
|
||||
func newDynamicSleeper(factor float64, maxWait time.Duration) *dynamicSleeper {
|
||||
func newDynamicSleeper(factor float64, maxWait time.Duration, isScanner bool) *dynamicSleeper {
|
||||
return &dynamicSleeper{
|
||||
factor: factor,
|
||||
cycle: make(chan struct{}),
|
||||
maxSleep: maxWait,
|
||||
minSleep: 100 * time.Microsecond,
|
||||
factor: factor,
|
||||
cycle: make(chan struct{}),
|
||||
maxSleep: maxWait,
|
||||
minSleep: 100 * time.Microsecond,
|
||||
isScanner: isScanner,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1379,15 +1381,24 @@ func (d *dynamicSleeper) Timer(ctx context.Context) func() {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
if !timer.Stop() {
|
||||
if d.isScanner {
|
||||
globalScannerMetrics.incTime(scannerMetricYield, wantSleep)
|
||||
}
|
||||
<-timer.C
|
||||
}
|
||||
return
|
||||
case <-timer.C:
|
||||
if d.isScanner {
|
||||
globalScannerMetrics.incTime(scannerMetricYield, wantSleep)
|
||||
}
|
||||
return
|
||||
case <-cycle:
|
||||
if !timer.Stop() {
|
||||
// We expired.
|
||||
<-timer.C
|
||||
if d.isScanner {
|
||||
globalScannerMetrics.incTime(scannerMetricYield, wantSleep)
|
||||
}
|
||||
return
|
||||
}
|
||||
}
|
||||
@@ -1418,14 +1429,23 @@ func (d *dynamicSleeper) Sleep(ctx context.Context, base time.Duration) {
|
||||
case <-ctx.Done():
|
||||
if !timer.Stop() {
|
||||
<-timer.C
|
||||
if d.isScanner {
|
||||
globalScannerMetrics.incTime(scannerMetricYield, wantSleep)
|
||||
}
|
||||
}
|
||||
return
|
||||
case <-timer.C:
|
||||
if d.isScanner {
|
||||
globalScannerMetrics.incTime(scannerMetricYield, wantSleep)
|
||||
}
|
||||
return
|
||||
case <-cycle:
|
||||
if !timer.Stop() {
|
||||
// We expired.
|
||||
<-timer.C
|
||||
if d.isScanner {
|
||||
globalScannerMetrics.incTime(scannerMetricYield, wantSleep)
|
||||
}
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user