feat: add dynamic usage cache (#12229)

A cache structure will be kept with a tree of usages.
The cache is a tree structure where each keeps track 
of its children.

An uncompacted branch contains a count of the files 
only directly at the branch level, and contains link to 
children branches or leaves.

The leaves are "compacted" based on a number of properties.
A compacted leaf contains the totals of all files beneath it.

A leaf is only scanned once every dataUsageUpdateDirCycles,
rarer if the bloom filter for the path is clean and no lifecycles 
are applied. Skipped leaves have their totals transferred from 
the previous cycle.

A clean leaf will be included once every healFolderIncludeProb 
for partial heal scans. When selected there is a one in 
healObjectSelectProb that any object will be chosen for heal scan.

Compaction happens when either:

- The folder (and subfolders) contains less than dataScannerCompactLeastObject objects.
- The folder itself contains more than dataScannerCompactAtFolders folders.
- The folder only contains objects and no subfolders.
- A bucket root will never be compacted.

Furthermore, if a has more than dataScannerCompactAtChildren recursive 
children (uncompacted folders) the tree will be recursively scanned and the 
branches with the least number of objects will be compacted until the limit 
is reached.

This ensures that any branch will never contain an unreasonable amount 
of other branches, and also that small branches with few objects don't 
take up unreasonable amounts of space.

Whenever a branch is scanned, it is assumed that it will be un-compacted
before it hits any of the above limits. This will make the branch rebalance 
itself when scanned if the distribution of objects has changed.

TLDR; With current values: No bucket will ever have more than 10000 
child nodes recursively. No single folder will have more than 2500 child 
nodes by itself. All subfolders are compacted if they have less than 500 
objects in them recursively.

We accumulate the (non-deletemarker) version count for paths as well, 
since we are changing the structure anyway.
This commit is contained in:
Klaus Post
2021-05-12 03:36:15 +02:00
committed by GitHub
parent f63eedb2b4
commit 229d83bb75
9 changed files with 1047 additions and 1328 deletions

View File

@@ -45,9 +45,12 @@ import (
)
const (
dataScannerSleepPerFolder = 20 * time.Millisecond // Time to wait between folders.
dataScannerStartDelay = 1 * time.Minute // Time to wait on startup and between cycles.
dataUsageUpdateDirCycles = 16 // Visit all folders every n cycles.
dataScannerSleepPerFolder = time.Millisecond // Time to wait between folders.
dataUsageUpdateDirCycles = 16 // Visit all folders every n cycles.
dataScannerCompactLeastObject = 500 // Compact when there is less than this many objects in a branch.
dataScannerCompactAtChildren = 10000 // Compact when there are this many children in a branch.
dataScannerCompactAtFolders = dataScannerCompactAtChildren / 4 // Compact when this many subfolders in a single folder.
dataScannerStartDelay = 1 * time.Minute // Time to wait on startup and between cycles.
healDeleteDangling = true
healFolderIncludeProb = 32 // Include a clean folder one in n cycles.
@@ -185,11 +188,47 @@ type folderScanner struct {
healFolderInclude uint32 // Include a clean folder one in n cycles.
healObjectSelect uint32 // Do a heal check on an object once every n cycles. Must divide into healFolderInclude
newFolders []cachedFolder
existingFolders []cachedFolder
disks []StorageAPI
disks []StorageAPI
}
// Cache structure and compaction:
//
// A cache structure will be kept with a tree of usages.
// The cache is a tree structure where each keeps track of its children.
//
// An uncompacted branch contains a count of the files only directly at the
// branch level, and contains link to children branches or leaves.
//
// The leaves are "compacted" based on a number of properties.
// A compacted leaf contains the totals of all files beneath it.
//
// A leaf is only scanned once every dataUsageUpdateDirCycles,
// rarer if the bloom filter for the path is clean and no lifecycles are applied.
// Skipped leaves have their totals transferred from the previous cycle.
//
// A clean leaf will be included once every healFolderIncludeProb for partial heal scans.
// When selected there is a one in healObjectSelectProb that any object will be chosen for heal scan.
//
// Compaction happens when either:
//
// 1) The folder (and subfolders) contains less than dataScannerCompactLeastObject objects.
// 2) The folder itself contains more than dataScannerCompactAtFolders folders.
// 3) The folder only contains objects and no subfolders.
//
// A bucket root will never be compacted.
//
// Furthermore if a has more than dataScannerCompactAtChildren recursive children (uncompacted folders)
// the tree will be recursively scanned and the branches with the least number of objects will be
// compacted until the limit is reached.
//
// This ensures that any branch will never contain an unreasonable amount of other branches,
// and also that small branches with few objects don't take up unreasonable amounts of space.
// This keeps the cache size at a reasonable size for all buckets.
//
// Whenever a branch is scanned, it is assumed that it will be un-compacted
// before it hits any of the above limits.
// This will make the branch rebalance itself when scanned if the distribution of objects has changed.
// scanDataFolder will scanner the basepath+cache.Info.Name and return an updated cache.
// The returned cache will always be valid, but may not be updated from the existing.
// Before each operation sleepDuration is called which can be used to temporarily halt the scanner.
@@ -211,15 +250,11 @@ func scanDataFolder(ctx context.Context, basePath string, cache dataUsageCache,
return cache, errors.New("internal error: root scan attempted")
}
skipHeal := cache.Info.SkipHealing
s := folderScanner{
root: basePath,
getSize: getSize,
oldCache: cache,
newCache: dataUsageCache{Info: cache.Info},
newFolders: nil,
existingFolders: nil,
dataUsageScannerDebug: intDataUpdateTracker.debug,
healFolderInclude: 0,
healObjectSelect: 0,
@@ -238,7 +273,7 @@ func scanDataFolder(ctx context.Context, basePath string, cache dataUsageCache,
}
// Enable healing in XL mode.
if globalIsErasure {
if globalIsErasure && !cache.Info.SkipHealing {
// Include a clean folder one in n cycles.
s.healFolderInclude = healFolderIncludeProb
// Do a heal check on an object once every n cycles. Must divide into healFolderInclude
@@ -257,139 +292,53 @@ func scanDataFolder(ctx context.Context, basePath string, cache dataUsageCache,
}
done := ctx.Done()
var flattenLevels = 2
if s.dataUsageScannerDebug {
console.Debugf(logPrefix+"Cycle: %v, Entries: %v %s\n", cache.Info.NextCycle, len(cache.Cache), logSuffix)
}
// Always scan flattenLevels deep. Cache root is level 0.
todo := []cachedFolder{{name: cache.Info.Name, objectHealProbDiv: 1}}
for i := 0; i < flattenLevels; i++ {
if s.dataUsageScannerDebug {
console.Debugf(logPrefix+"Level %v, scanning %v directories. %s\n", i, len(todo), logSuffix)
}
select {
case <-done:
return cache, ctx.Err()
default:
}
var err error
todo, err = s.scanQueuedLevels(ctx, todo, i == flattenLevels-1, skipHeal)
if err != nil {
// No useful information...
return cache, err
}
// Read top level in bucket.
select {
case <-done:
return cache, ctx.Err()
default:
}
root := dataUsageEntry{}
folder := cachedFolder{name: cache.Info.Name, objectHealProbDiv: 1}
err := s.scanFolder(ctx, folder, &root)
if err != nil {
// No useful information...
return cache, err
}
if s.dataUsageScannerDebug {
console.Debugf(logPrefix+"New folders: %v %s\n", s.newFolders, logSuffix)
}
// Add new folders first
for _, folder := range s.newFolders {
select {
case <-done:
return s.newCache, ctx.Err()
default:
}
du, err := s.deepScanFolder(ctx, folder, skipHeal)
if err != nil {
logger.LogIf(ctx, err)
continue
}
if du == nil {
console.Debugln(logPrefix + "no disk usage provided" + logSuffix)
continue
}
s.newCache.replace(folder.name, "", *du)
// Add to parent manually
if folder.parent != nil {
parent := s.newCache.Cache[folder.parent.Key()]
parent.addChildString(folder.name)
}
}
if s.dataUsageScannerDebug {
console.Debugf(logPrefix+"Existing folders: %v %s\n", len(s.existingFolders), logSuffix)
}
// Do selective scanning of existing folders.
for _, folder := range s.existingFolders {
select {
case <-done:
return s.newCache, ctx.Err()
default:
}
h := hashPath(folder.name)
if !h.mod(s.oldCache.Info.NextCycle, dataUsageUpdateDirCycles) {
if !h.mod(s.oldCache.Info.NextCycle, s.healFolderInclude/folder.objectHealProbDiv) {
s.newCache.replaceHashed(h, folder.parent, s.oldCache.Cache[h.Key()])
continue
} else {
folder.objectHealProbDiv = s.healFolderInclude
}
folder.objectHealProbDiv = dataUsageUpdateDirCycles
}
if s.withFilter != nil {
_, prefix := path2BucketObjectWithBasePath(basePath, folder.name)
if s.oldCache.Info.lifeCycle == nil || !s.oldCache.Info.lifeCycle.HasActiveRules(prefix, true) {
// If folder isn't in filter, skip it completely.
if !s.withFilter.containsDir(folder.name) {
if !h.mod(s.oldCache.Info.NextCycle, s.healFolderInclude/folder.objectHealProbDiv) {
if s.dataUsageScannerDebug {
console.Debugf(logPrefix+"Skipping non-updated folder: %v %s\n", folder, logSuffix)
}
s.newCache.replaceHashed(h, folder.parent, s.oldCache.Cache[h.Key()])
continue
} else {
if s.dataUsageScannerDebug {
console.Debugf(logPrefix+"Adding non-updated folder to heal check: %v %s\n", folder.name, logSuffix)
}
// Update probability of including objects
folder.objectHealProbDiv = s.healFolderInclude
}
}
}
}
// Update on this cycle...
du, err := s.deepScanFolder(ctx, folder, skipHeal)
if err != nil {
logger.LogIf(ctx, err)
continue
}
if du == nil {
logger.LogIf(ctx, errors.New("data-usage: no disk usage provided"))
continue
}
s.newCache.replaceHashed(h, folder.parent, *du)
}
if s.dataUsageScannerDebug {
console.Debugf(logPrefix+"Finished scanner, %v entries %s\n", len(s.newCache.Cache), logSuffix)
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++
return s.newCache, nil
}
// scanQueuedLevels will scan the provided folders.
// scanFolder will scan the provided folder.
// Files found in the folders will be added to f.newCache.
// If final is provided folders will be put into f.newFolders or f.existingFolders.
// If final is not provided the folders found are returned from the function.
func (f *folderScanner) scanQueuedLevels(ctx context.Context, folders []cachedFolder, final bool, skipHeal bool) ([]cachedFolder, error) {
var nextFolders []cachedFolder
func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, into *dataUsageEntry) error {
done := ctx.Done()
scannerLogPrefix := color.Green("folder-scanner:")
for _, folder := range folders {
thisHash := hashPath(folder.name)
// Store initial compaction state.
wasCompacted := into.Compacted
for {
select {
case <-done:
return nil, ctx.Err()
return ctx.Err()
default:
}
thisHash := hashPath(folder.name)
existing := f.oldCache.findChildrenCopy(thisHash)
existing, ok := f.oldCache.Cache[thisHash.Key()]
var abandonedChildren dataUsageHashMap
if !into.Compacted {
abandonedChildren = f.oldCache.findChildrenCopy(thisHash)
}
// If there are lifecycle rules for the prefix, remove the filter.
filter := f.withFilter
@@ -402,28 +351,29 @@ func (f *folderScanner) scanQueuedLevels(ctx context.Context, folders []cachedFo
activeLifeCycle = f.oldCache.Info.lifeCycle
filter = nil
}
if _, ok := f.oldCache.Cache[thisHash.Key()]; filter != nil && ok {
// Check if we can skip it due to bloom filter...
if filter != nil && ok && existing.Compacted {
// If folder isn't in filter and we have data, skip it completely.
if folder.name != dataUsageRoot && !filter.containsDir(folder.name) {
if !thisHash.mod(f.oldCache.Info.NextCycle, f.healFolderInclude/folder.objectHealProbDiv) {
if f.healObjectSelect == 0 || !thisHash.mod(f.oldCache.Info.NextCycle, f.healFolderInclude/folder.objectHealProbDiv) {
f.newCache.copyWithChildren(&f.oldCache, thisHash, folder.parent)
if f.dataUsageScannerDebug {
console.Debugf(scannerLogPrefix+" Skipping non-updated folder: %v\n", folder.name)
}
continue
} else {
if f.dataUsageScannerDebug {
console.Debugf(scannerLogPrefix+" Adding non-updated folder to heal check: %v\n", folder.name)
}
// If probability was already scannerHealFolderInclude, keep it.
folder.objectHealProbDiv = f.healFolderInclude
return nil
}
if f.dataUsageScannerDebug {
console.Debugf(scannerLogPrefix+" Adding non-updated folder to heal check: %v\n", folder.name)
}
// If probability was already scannerHealFolderInclude, keep it.
folder.objectHealProbDiv = f.healFolderInclude
}
}
scannerSleeper.Sleep(ctx, dataScannerSleepPerFolder)
cache := dataUsageEntry{}
var existingFolders, newFolders []cachedFolder
var foundObjects bool
err := readDirFn(path.Join(f.root, folder.name), func(entName string, typ os.FileMode) error {
// Parse
entName = pathClean(path.Join(folder.name, entName))
@@ -455,26 +405,15 @@ func (f *folderScanner) scanQueuedLevels(ctx context.Context, folders []cachedFo
}
if typ&os.ModeDir != 0 {
scannerSleeper.Sleep(ctx, dataScannerSleepPerFolder)
h := hashPath(entName)
_, exists := f.oldCache.Cache[h.Key()]
cache.addChildString(entName)
this := cachedFolder{name: entName, parent: &thisHash, objectHealProbDiv: folder.objectHealProbDiv}
delete(existing, h.Key()) // h.Key() already accounted for.
cache.addChild(h)
if final {
if exists {
f.existingFolders = append(f.existingFolders, this)
} else {
f.newFolders = append(f.newFolders, this)
}
delete(abandonedChildren, h.Key()) // h.Key() already accounted for.
if exists {
existingFolders = append(existingFolders, this)
} else {
nextFolders = append(nextFolders, this)
newFolders = append(newFolders, this)
}
return nil
}
@@ -497,54 +436,111 @@ func (f *folderScanner) scanQueuedLevels(ctx context.Context, folders []cachedFo
// if the drive belongs to an erasure set
// that is already being healed, skip the
// healing attempt on this drive.
item.heal = item.heal && !skipHeal
item.heal = item.heal && f.healObjectSelect > 0
sizeSummary, err := f.getSize(item)
sz, err := f.getSize(item)
if err == errSkipFile {
wait() // wait to proceed to next entry.
return nil
}
// successfully read means we have a valid object.
foundObjects = true
// Remove filename i.e is the meta file to construct object name
item.transformMetaDir()
// Object already accounted for, remove from heal map,
// simply because getSize() function already heals the
// object.
delete(existing, path.Join(item.bucket, item.objectPath()))
delete(abandonedChildren, path.Join(item.bucket, item.objectPath()))
cache.addSizes(sizeSummary)
cache.Objects++
cache.ObjSizes.add(sizeSummary.totalSize)
into.addSizes(sz)
into.Objects++
wait() // wait to proceed to next entry.
return nil
})
if err != nil {
return nil, err
return err
}
if f.healObjectSelect == 0 {
// If we are not scanning, return now.
f.newCache.replaceHashed(thisHash, folder.parent, cache)
continue
if foundObjects && globalIsErasure {
// If we found an object in erasure mode, we skip subdirs (only datadirs)...
break
}
// If we have many subfolders, compact ourself.
if !into.Compacted &&
f.newCache.Info.Name != folder.name &&
len(existingFolders)+len(newFolders) >= dataScannerCompactAtFolders {
into.Compacted = true
newFolders = append(newFolders, existingFolders...)
existingFolders = nil
if f.dataUsageScannerDebug {
console.Debugf(scannerLogPrefix+" Preemptively compacting: %v, entries: %v\n", folder.name, len(existingFolders)+len(newFolders))
}
}
scanFolder := func(folder cachedFolder) {
if contextCanceled(ctx) {
return
}
dst := into
if !into.Compacted {
dst = &dataUsageEntry{Compacted: false}
}
if err := f.scanFolder(ctx, folder, dst); err != nil {
logger.LogIf(ctx, err)
return
}
if !into.Compacted {
into.addChild(dataUsageHash(folder.name))
}
}
// Scan new...
for _, folder := range newFolders {
scanFolder(folder)
}
// Scan existing...
for _, folder := range existingFolders {
h := hashPath(folder.name)
// Check if we should skip scanning folder...
// We can only skip if we are not indexing into a compacted destination
// and the entry itself is compacted.
if !into.Compacted && f.oldCache.isCompacted(h) {
if !h.mod(f.oldCache.Info.NextCycle, dataUsageUpdateDirCycles) {
if !h.mod(f.oldCache.Info.NextCycle, f.healFolderInclude/folder.objectHealProbDiv) {
// Transfer and add as child...
f.newCache.copyWithChildren(&f.oldCache, h, folder.parent)
into.addChild(h)
continue
}
folder.objectHealProbDiv = dataUsageUpdateDirCycles
}
}
scanFolder(folder)
}
// Scan for healing
if f.healObjectSelect == 0 || len(abandonedChildren) == 0 {
// If we are not heal scanning, return now.
break
}
objAPI, ok := newObjectLayerFn().(*erasureServerPools)
if !ok || len(f.disks) == 0 {
continue
break
}
bgSeq, found := globalBackgroundHealState.getHealSequenceByToken(bgHealingUUID)
if !found {
continue
break
}
// Whatever remains in 'existing' are folders at this level
// Whatever remains in 'abandonedChildren' are folders at this level
// that existed in the previous run but wasn't found now.
//
// This may be because of 2 reasons:
@@ -563,7 +559,7 @@ func (f *folderScanner) scanQueuedLevels(ctx context.Context, folders []cachedFo
}
healObjectsPrefix := color.Green("healObjects:")
for k := range existing {
for k := range abandonedChildren {
bucket, prefix := path2BucketObject(k)
if f.dataUsageScannerDebug {
console.Debugf(scannerLogPrefix+" checking disappeared folder: %v/%v\n", bucket, prefix)
@@ -677,115 +673,68 @@ func (f *folderScanner) scanQueuedLevels(ctx context.Context, folders []cachedFo
objAPI.HealObjects(ctx, bucket, prefix, madmin.HealOpts{
Recursive: true,
Remove: healDeleteDangling,
},
func(bucket, object, versionID string) error {
// Wait for each heal as per scanner frequency.
wait()
return bgSeq.queueHealTask(healSource{
bucket: bucket,
object: object,
versionID: versionID,
}, madmin.HealItemObject)
})
}, func(bucket, object, versionID string) error {
// Wait for each heal as per scanner frequency.
wait()
wait = scannerSleeper.Timer(ctx)
return bgSeq.queueHealTask(healSource{
bucket: bucket,
object: object,
versionID: versionID,
}, madmin.HealItemObject)
})
}
// Add unless healing returned an error.
if foundObjs {
this := cachedFolder{name: k, parent: &thisHash, objectHealProbDiv: folder.objectHealProbDiv}
cache.addChild(hashPath(k))
if final {
f.existingFolders = append(f.existingFolders, this)
} else {
nextFolders = append(nextFolders, this)
this := cachedFolder{name: k, parent: &thisHash, objectHealProbDiv: 1}
scanFolder(this)
}
}
break
}
if !wasCompacted {
f.newCache.replaceHashed(thisHash, folder.parent, *into)
}
if !into.Compacted && f.newCache.Info.Name != folder.name {
flat := f.newCache.sizeRecursive(thisHash.Key())
flat.Compacted = true
var compact bool
if flat.Objects < dataScannerCompactLeastObject {
if f.dataUsageScannerDebug && flat.Objects > 1 {
// Disabled, rather chatty:
//console.Debugf(scannerLogPrefix+" Only %d objects, compacting %s -> %+v\n", flat.Objects, folder.name, flat)
}
compact = true
} else {
// Compact if we only have objects as children...
compact = true
for k := range into.Children {
if v, ok := f.newCache.Cache[k]; ok {
if len(v.Children) > 0 || v.Objects > 1 {
compact = false
break
}
}
}
}
f.newCache.replaceHashed(thisHash, folder.parent, cache)
}
return nextFolders, nil
}
// deepScanFolder will deep scan a folder and return the size if no error occurs.
func (f *folderScanner) deepScanFolder(ctx context.Context, folder cachedFolder, skipHeal bool) (*dataUsageEntry, error) {
var cache dataUsageEntry
done := ctx.Done()
var addDir func(entName string, typ os.FileMode) error
var dirStack = []string{f.root, folder.name}
deepScannerLogPrefix := color.Green("deep-scanner:")
addDir = func(entName string, typ os.FileMode) error {
select {
case <-done:
return errDoneForNow
default:
}
if typ&os.ModeDir != 0 {
dirStack = append(dirStack, entName)
err := readDirFn(path.Join(dirStack...), addDir)
dirStack = dirStack[:len(dirStack)-1]
scannerSleeper.Sleep(ctx, dataScannerSleepPerFolder)
return err
}
// Dynamic time delay.
wait := scannerSleeper.Timer(ctx)
// Get file size, ignore errors.
dirStack = append(dirStack, entName)
fileName := path.Join(dirStack...)
dirStack = dirStack[:len(dirStack)-1]
bucket, prefix := path2BucketObjectWithBasePath(f.root, fileName)
var activeLifeCycle *lifecycle.Lifecycle
if f.oldCache.Info.lifeCycle != nil && f.oldCache.Info.lifeCycle.HasActiveRules(prefix, false) {
if f.dataUsageScannerDebug {
console.Debugf(deepScannerLogPrefix+" Prefix %q has active rules\n", prefix)
if f.dataUsageScannerDebug && compact {
// Disabled, rather chatty:
//console.Debugf(scannerLogPrefix+" Only objects (%d), compacting %s -> %+v\n", flat.Objects, folder.name, flat)
}
activeLifeCycle = f.oldCache.Info.lifeCycle
}
if compact {
f.newCache.deleteRecursive(thisHash)
f.newCache.replaceHashed(thisHash, folder.parent, *flat)
}
item := scannerItem{
Path: fileName,
Typ: typ,
bucket: bucket,
prefix: path.Dir(prefix),
objectName: path.Base(entName),
debug: f.dataUsageScannerDebug,
lifeCycle: activeLifeCycle,
heal: hashPath(path.Join(prefix, entName)).mod(f.oldCache.Info.NextCycle, f.healObjectSelect/folder.objectHealProbDiv) && globalIsErasure,
}
// if the drive belongs to an erasure set
// that is already being healed, skip the
// healing attempt on this drive.
item.heal = item.heal && !skipHeal
sizeSummary, err := f.getSize(item)
if err == errSkipFile {
// Wait to throttle IO
wait()
return nil
}
logger.LogIf(ctx, err)
cache.addSizes(sizeSummary)
cache.Objects++
cache.ObjSizes.add(sizeSummary.totalSize)
// Wait to throttle IO
wait()
return nil
}
err := readDirFn(path.Join(dirStack...), addDir)
if err != nil {
return nil, err
// Compact if too many children...
if !into.Compacted {
f.newCache.reduceChildrenOf(thisHash, dataScannerCompactAtChildren, f.newCache.Info.Name != folder.name)
}
return &cache, nil
return nil
}
// scannerItem represents each file while walking.
@@ -803,6 +752,7 @@ type scannerItem struct {
type sizeSummary struct {
totalSize int64
versions uint64
replicatedSize int64
pendingSize int64
failedSize int64
@@ -863,7 +813,8 @@ func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, meta ac
}
if i.lifeCycle == nil {
if i.debug {
console.Debugf(applyActionsLogPrefix+" no lifecycle rules to apply: %q\n", i.objectPath())
// disabled, very chatty:
// console.Debugf(applyActionsLogPrefix+" no lifecycle rules to apply: %q\n", i.objectPath())
}
return false, size
}