Add detailed scanner metrics (#15161)

This commit is contained in:
Klaus Post
2022-07-05 14:45:49 -07:00
committed by GitHub
parent df42914da6
commit ac055b09e9
55 changed files with 1735 additions and 1753 deletions

View File

@@ -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
}
}