Enforce quota checks after crawl (#10036)

Enforce bucket quotas when crawling has finished. 
This ensures that we will not do quota enforcement on old data.

Additionally, delete less if we are closer to quota than we thought.
master
Klaus Post 4 years ago committed by GitHub
parent 14ff7f5fcf
commit 00d3cc4b69
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 85
      cmd/bucket-quota.go
  2. 15
      cmd/data-usage-cache.go
  3. 9
      cmd/disk-cache-utils.go
  4. 12
      cmd/erasure-zones.go
  5. 1
      cmd/fs-v1.go
  6. 1
      cmd/server-main.go

@ -22,9 +22,7 @@ import (
"fmt" "fmt"
"time" "time"
"github.com/minio/minio/cmd/config"
"github.com/minio/minio/cmd/logger" "github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/env"
"github.com/minio/minio/pkg/event" "github.com/minio/minio/pkg/event"
"github.com/minio/minio/pkg/madmin" "github.com/minio/minio/pkg/madmin"
) )
@ -120,68 +118,17 @@ func enforceBucketQuota(ctx context.Context, bucket string, size int64) error {
return globalBucketQuotaSys.check(ctx, bucket, size) return globalBucketQuotaSys.check(ctx, bucket, size)
} }
const (
bgQuotaInterval = 1 * time.Hour
)
// initQuotaEnforcement starts the routine that deletes objects in bucket
// that exceeds the FIFO quota
func initQuotaEnforcement(ctx context.Context, objAPI ObjectLayer) {
if env.Get(envDataUsageCrawlConf, config.EnableOn) == config.EnableOn {
go startBucketQuotaEnforcement(ctx, objAPI)
}
}
func startBucketQuotaEnforcement(ctx context.Context, objAPI ObjectLayer) {
for {
select {
case <-ctx.Done():
return
case <-time.NewTimer(bgQuotaInterval).C:
enforceFIFOQuota(ctx, objAPI)
}
}
}
// enforceFIFOQuota deletes objects in FIFO order until sufficient objects // enforceFIFOQuota deletes objects in FIFO order until sufficient objects
// have been deleted so as to bring bucket usage within quota // have been deleted so as to bring bucket usage within quota.
func enforceFIFOQuota(ctx context.Context, objectAPI ObjectLayer) { func enforceFIFOQuotaBucket(ctx context.Context, objectAPI ObjectLayer, bucket string, bui BucketUsageInfo) {
// Turn off quota enforcement if data usage info is unavailable.
if env.Get(envDataUsageCrawlConf, config.EnableOn) == config.EnableOff {
return
}
buckets, err := objectAPI.ListBuckets(ctx)
if err != nil {
logger.LogIf(ctx, err)
return
}
dataUsageInfo, err := loadDataUsageFromBackend(ctx, objectAPI)
if err != nil {
logger.LogIf(ctx, err)
return
}
for _, binfo := range buckets {
bucket := binfo.Name
bui, ok := dataUsageInfo.BucketsUsage[bucket]
if !ok {
// bucket doesn't exist anymore, or we
// do not have any information to proceed.
continue
}
// Check if the current bucket has quota restrictions, if not skip it // Check if the current bucket has quota restrictions, if not skip it
cfg, err := globalBucketQuotaSys.Get(bucket) cfg, err := globalBucketQuotaSys.Get(bucket)
if err != nil { if err != nil {
continue return
} }
if cfg.Type != madmin.FIFOQuota { if cfg.Type != madmin.FIFOQuota {
continue return
} }
var toFree uint64 var toFree uint64
@ -189,8 +136,8 @@ func enforceFIFOQuota(ctx context.Context, objectAPI ObjectLayer) {
toFree = bui.Size - cfg.Quota toFree = bui.Size - cfg.Quota
} }
if toFree == 0 { if toFree <= 0 {
continue return
} }
// Allocate new results channel to receive ObjectInfo. // Allocate new results channel to receive ObjectInfo.
@ -201,7 +148,7 @@ func enforceFIFOQuota(ctx context.Context, objectAPI ObjectLayer) {
// Walk through all objects // Walk through all objects
if err := objectAPI.Walk(ctx, bucket, "", objInfoCh, ObjectOptions{WalkVersions: versioned}); err != nil { if err := objectAPI.Walk(ctx, bucket, "", objInfoCh, ObjectOptions{WalkVersions: versioned}); err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
continue return
} }
// reuse the fileScorer used by disk cache to score entries by // reuse the fileScorer used by disk cache to score entries by
@ -211,7 +158,7 @@ func enforceFIFOQuota(ctx context.Context, objectAPI ObjectLayer) {
scorer, err := newFileScorer(toFree, time.Now().Unix(), 1) scorer, err := newFileScorer(toFree, time.Now().Unix(), 1)
if err != nil { if err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
continue return
} }
rcfg, _ := globalBucketObjectLockSys.Get(bucket) rcfg, _ := globalBucketObjectLockSys.Get(bucket)
@ -228,6 +175,21 @@ func enforceFIFOQuota(ctx context.Context, objectAPI ObjectLayer) {
scorer.addFileWithObjInfo(obj, 1) scorer.addFileWithObjInfo(obj, 1)
} }
// If we saw less than quota we are good.
if scorer.seenBytes <= cfg.Quota {
return
}
// Calculate how much we want to delete now.
toFreeNow := scorer.seenBytes - cfg.Quota
// We were less over quota than we thought. Adjust so we delete less.
// If we are more over, leave it for the next run to pick up.
if toFreeNow < toFree {
if !scorer.adjustSaveBytes(int64(toFreeNow) - int64(toFree)) {
// We got below or at quota.
return
}
}
var objects []ObjectToDelete var objects []ObjectToDelete
numKeys := len(scorer.fileObjInfos()) numKeys := len(scorer.fileObjInfos())
for i, obj := range scorer.fileObjInfos() { for i, obj := range scorer.fileObjInfos() {
@ -264,5 +226,4 @@ func enforceFIFOQuota(ctx context.Context, objectAPI ObjectLayer) {
} }
objects = nil objects = nil
} }
}
} }

@ -335,6 +335,21 @@ func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]Bucke
return dst return dst
} }
// bucketUsageInfo returns the buckets usage info.
// If not found all values returned are zero values.
func (d *dataUsageCache) bucketUsageInfo(bucket string) BucketUsageInfo {
e := d.find(bucket)
if e == nil {
return BucketUsageInfo{}
}
flat := d.flatten(*e)
return BucketUsageInfo{
Size: uint64(flat.Size),
ObjectsCount: uint64(flat.Objects),
ObjectSizesHistogram: flat.ObjSizes.toMap(),
}
}
// sizeRecursive returns the path as a flattened entry. // sizeRecursive returns the path as a flattened entry.
func (d *dataUsageCache) sizeRecursive(path string) *dataUsageEntry { func (d *dataUsageCache) sizeRecursive(path string) *dataUsageEntry {
root := d.find(path) root := d.find(path)

@ -295,6 +295,7 @@ type fileScorer struct {
// The list is kept sorted according to score, highest at top, lowest at bottom. // The list is kept sorted according to score, highest at top, lowest at bottom.
queue list.List queue list.List
queuedBytes uint64 queuedBytes uint64
seenBytes uint64
} }
type queuedFile struct { type queuedFile struct {
@ -337,6 +338,7 @@ func (f *fileScorer) addFileWithObjInfo(objInfo ObjectInfo, hits int) {
versionID: objInfo.VersionID, versionID: objInfo.VersionID,
size: uint64(objInfo.Size), size: uint64(objInfo.Size),
} }
f.seenBytes += uint64(objInfo.Size)
var score float64 var score float64
if objInfo.ModTime.IsZero() { if objInfo.ModTime.IsZero() {
@ -369,9 +371,14 @@ func (f *fileScorer) addFileWithObjInfo(objInfo ObjectInfo, hits int) {
// adjustSaveBytes allows to adjust the number of bytes to save. // adjustSaveBytes allows to adjust the number of bytes to save.
// This can be used to adjust the count on the fly. // This can be used to adjust the count on the fly.
// Returns true if there still is a need to delete files (saveBytes >0), // Returns true if there still is a need to delete files (n+saveBytes >0),
// false if no more bytes needs to be saved. // false if no more bytes needs to be saved.
func (f *fileScorer) adjustSaveBytes(n int64) bool { func (f *fileScorer) adjustSaveBytes(n int64) bool {
if int64(f.saveBytes)+n <= 0 {
f.saveBytes = 0
f.trimQueue()
return false
}
if n < 0 { if n < 0 {
f.saveBytes -= ^uint64(n - 1) f.saveBytes -= ^uint64(n - 1)
} else { } else {

@ -337,13 +337,15 @@ func (z *erasureZones) CrawlAndGetDataUsage(ctx context.Context, bf *bloomFilter
updateTicker := time.NewTicker(30 * time.Second) updateTicker := time.NewTicker(30 * time.Second)
defer updateTicker.Stop() defer updateTicker.Stop()
var lastUpdate time.Time var lastUpdate time.Time
update := func() {
mu.Lock()
defer mu.Unlock()
// We need to merge since we will get the same buckets from each zone. // We need to merge since we will get the same buckets from each zone.
// Therefore to get the exact bucket sizes we must merge before we can convert. // Therefore to get the exact bucket sizes we must merge before we can convert.
allMerged := dataUsageCache{Info: dataUsageCacheInfo{Name: dataUsageRoot}} allMerged := dataUsageCache{Info: dataUsageCacheInfo{Name: dataUsageRoot}}
update := func() {
mu.Lock()
defer mu.Unlock()
for _, info := range results { for _, info := range results {
if info.Info.LastUpdate.IsZero() { if info.Info.LastUpdate.IsZero() {
// Not filled yet. // Not filled yet.
@ -362,6 +364,10 @@ func (z *erasureZones) CrawlAndGetDataUsage(ctx context.Context, bf *bloomFilter
return return
case v := <-updateCloser: case v := <-updateCloser:
update() update()
// Enforce quotas when all is done.
for _, b := range allBuckets {
enforceFIFOQuotaBucket(ctx, z, b.Name, allMerged.bucketUsageInfo(b.Name))
}
close(v) close(v)
return return
case <-updateTicker.C: case <-updateTicker.C:

@ -301,6 +301,7 @@ func (fs *FSObjects) CrawlAndGetDataUsage(ctx context.Context, bf *bloomFilter,
logger.LogIf(ctx, totalCache.save(ctx, fs, dataUsageCacheName)) logger.LogIf(ctx, totalCache.save(ctx, fs, dataUsageCacheName))
cloned := totalCache.clone() cloned := totalCache.clone()
updates <- cloned.dui(dataUsageRoot, buckets) updates <- cloned.dui(dataUsageRoot, buckets)
enforceFIFOQuotaBucket(ctx, fs, b.Name, cloned.bucketUsageInfo(b.Name))
} }
return nil return nil

@ -360,7 +360,6 @@ func startBackgroundOps(ctx context.Context, objAPI ObjectLayer) {
} }
initDataCrawler(ctx, objAPI) initDataCrawler(ctx, objAPI)
initQuotaEnforcement(ctx, objAPI)
} }
// serverMain handler called for 'minio server' command. // serverMain handler called for 'minio server' command.

Loading…
Cancel
Save