2021-04-18 15:41:13 -04:00
|
|
|
// Copyright (c) 2015-2021 MinIO, Inc.
|
|
|
|
//
|
|
|
|
// This file is part of MinIO Object Storage stack
|
|
|
|
//
|
|
|
|
// This program is free software: you can redistribute it and/or modify
|
|
|
|
// it under the terms of the GNU Affero General Public License as published by
|
|
|
|
// the Free Software Foundation, either version 3 of the License, or
|
|
|
|
// (at your option) any later version.
|
|
|
|
//
|
|
|
|
// This program is distributed in the hope that it will be useful
|
|
|
|
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
|
|
|
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
|
|
|
// GNU Affero General Public License for more details.
|
|
|
|
//
|
|
|
|
// You should have received a copy of the GNU Affero General Public License
|
|
|
|
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
2020-06-12 13:28:21 -04:00
|
|
|
|
|
|
|
package cmd
|
|
|
|
|
|
|
|
import (
|
|
|
|
"bytes"
|
|
|
|
"context"
|
|
|
|
"encoding/binary"
|
2022-04-07 11:10:40 -04:00
|
|
|
"encoding/json"
|
2020-06-12 13:28:21 -04:00
|
|
|
"errors"
|
2022-01-26 11:34:56 -05:00
|
|
|
"fmt"
|
2021-11-08 13:25:34 -05:00
|
|
|
"io/fs"
|
2020-12-04 12:32:35 -05:00
|
|
|
"math"
|
2020-09-25 22:21:52 -04:00
|
|
|
"math/rand"
|
2020-06-12 13:28:21 -04:00
|
|
|
"os"
|
|
|
|
"path"
|
|
|
|
"strings"
|
2020-12-04 12:32:35 -05:00
|
|
|
"sync"
|
2021-10-02 12:31:05 -04:00
|
|
|
"sync/atomic"
|
2020-06-12 13:28:21 -04:00
|
|
|
"time"
|
|
|
|
|
2021-05-27 11:02:39 -04:00
|
|
|
"github.com/bits-and-blooms/bloom/v3"
|
2022-01-26 11:34:56 -05:00
|
|
|
"github.com/dustin/go-humanize"
|
2021-05-06 11:52:02 -04:00
|
|
|
"github.com/minio/madmin-go"
|
2021-06-01 17:59:40 -04:00
|
|
|
"github.com/minio/minio/internal/bucket/lifecycle"
|
2022-04-11 16:25:32 -04:00
|
|
|
"github.com/minio/minio/internal/bucket/object/lock"
|
2021-06-01 17:59:40 -04:00
|
|
|
"github.com/minio/minio/internal/bucket/replication"
|
|
|
|
"github.com/minio/minio/internal/color"
|
|
|
|
"github.com/minio/minio/internal/config/heal"
|
|
|
|
"github.com/minio/minio/internal/event"
|
|
|
|
"github.com/minio/minio/internal/logger"
|
2021-05-28 18:17:01 -04:00
|
|
|
"github.com/minio/pkg/console"
|
2020-06-12 13:28:21 -04:00
|
|
|
)
|
|
|
|
|
|
|
|
const (
|
2021-05-11 21:36:15 -04:00
|
|
|
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.
|
2020-06-12 13:28:21 -04:00
|
|
|
|
2020-09-12 03:08:12 -04:00
|
|
|
healDeleteDangling = true
|
|
|
|
healFolderIncludeProb = 32 // Include a clean folder one in n cycles.
|
|
|
|
healObjectSelectProb = 512 // Overall probability of a file being scanned; one in n.
|
|
|
|
)
|
|
|
|
|
|
|
|
var (
|
2021-08-26 17:06:04 -04:00
|
|
|
globalHealConfig heal.Config
|
2020-12-04 12:32:35 -05:00
|
|
|
|
2021-02-17 15:04:11 -05:00
|
|
|
dataScannerLeaderLockTimeout = newDynamicTimeout(30*time.Second, 10*time.Second)
|
2020-12-04 12:32:35 -05:00
|
|
|
// Sleeper values are updated when config is loaded.
|
2021-02-17 15:04:11 -05:00
|
|
|
scannerSleeper = newDynamicSleeper(10, 10*time.Second)
|
2021-04-27 11:24:44 -04:00
|
|
|
scannerCycle = &safeDuration{
|
|
|
|
t: dataScannerStartDelay,
|
|
|
|
}
|
2020-06-12 13:28:21 -04:00
|
|
|
)
|
|
|
|
|
2021-02-17 15:04:11 -05:00
|
|
|
// initDataScanner will start the scanner in the background.
|
|
|
|
func initDataScanner(ctx context.Context, objAPI ObjectLayer) {
|
2021-12-16 11:32:15 -05:00
|
|
|
go func() {
|
2022-01-02 12:15:34 -05:00
|
|
|
r := rand.New(rand.NewSource(time.Now().UnixNano()))
|
2021-12-16 11:32:15 -05:00
|
|
|
// Run the data scanner in a loop
|
|
|
|
for {
|
2022-01-02 12:15:34 -05:00
|
|
|
runDataScanner(ctx, objAPI)
|
|
|
|
duration := time.Duration(r.Float64() * float64(scannerCycle.Get()))
|
|
|
|
if duration < time.Second {
|
|
|
|
// Make sure to sleep atleast a second to avoid high CPU ticks.
|
|
|
|
duration = time.Second
|
2021-12-16 11:32:15 -05:00
|
|
|
}
|
2022-01-02 12:15:34 -05:00
|
|
|
time.Sleep(duration)
|
2021-12-16 11:32:15 -05:00
|
|
|
}
|
|
|
|
}()
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2021-03-30 16:59:02 -04:00
|
|
|
type safeDuration struct {
|
|
|
|
sync.Mutex
|
|
|
|
t time.Duration
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *safeDuration) Update(t time.Duration) {
|
|
|
|
s.Lock()
|
|
|
|
defer s.Unlock()
|
|
|
|
s.t = t
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *safeDuration) Get() time.Duration {
|
|
|
|
s.Lock()
|
|
|
|
defer s.Unlock()
|
|
|
|
return s.t
|
|
|
|
}
|
|
|
|
|
2022-04-07 11:10:40 -04:00
|
|
|
func getCycleScanMode(currentCycle, bitrotStartCycle uint64, bitrotStartTime time.Time) madmin.HealScanMode {
|
|
|
|
bitrotCycle := globalHealConfig.BitrotScanCycle()
|
|
|
|
switch bitrotCycle {
|
|
|
|
case -1:
|
|
|
|
return madmin.HealNormalScan
|
|
|
|
case 0:
|
|
|
|
return madmin.HealDeepScan
|
|
|
|
}
|
|
|
|
|
|
|
|
if currentCycle-bitrotStartCycle < healObjectSelectProb {
|
|
|
|
return madmin.HealDeepScan
|
|
|
|
}
|
|
|
|
|
|
|
|
if time.Since(bitrotStartTime) > bitrotCycle {
|
|
|
|
return madmin.HealDeepScan
|
|
|
|
}
|
|
|
|
|
|
|
|
return madmin.HealNormalScan
|
|
|
|
}
|
|
|
|
|
|
|
|
type backgroundHealInfo struct {
|
|
|
|
BitrotStartTime time.Time `json:"bitrotStartTime"`
|
|
|
|
BitrotStartCycle uint64 `json:"bitrotStartCycle"`
|
|
|
|
CurrentScanMode madmin.HealScanMode `json:"currentScanMode"`
|
|
|
|
}
|
|
|
|
|
|
|
|
func readBackgroundHealInfo(ctx context.Context, objAPI ObjectLayer) backgroundHealInfo {
|
|
|
|
// Get last healing information
|
|
|
|
buf, err := readConfig(ctx, objAPI, backgroundHealInfoPath)
|
|
|
|
if err != nil {
|
|
|
|
if !errors.Is(err, errConfigNotFound) {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
}
|
|
|
|
return backgroundHealInfo{}
|
|
|
|
}
|
|
|
|
var info backgroundHealInfo
|
|
|
|
err = json.Unmarshal(buf, &info)
|
|
|
|
if err != nil {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
return backgroundHealInfo{}
|
|
|
|
}
|
|
|
|
return info
|
|
|
|
}
|
|
|
|
|
|
|
|
func saveBackgroundHealInfo(ctx context.Context, objAPI ObjectLayer, info backgroundHealInfo) {
|
|
|
|
b, err := json.Marshal(info)
|
|
|
|
if err != nil {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
// Get last healing information
|
|
|
|
err = saveConfig(ctx, objAPI, backgroundHealInfoPath, b)
|
|
|
|
if err != nil {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-02-17 15:04:11 -05:00
|
|
|
// runDataScanner will start a data scanner.
|
2020-06-12 13:28:21 -04:00
|
|
|
// The function will block until the context is canceled.
|
2021-02-17 15:04:11 -05:00
|
|
|
// There should only ever be one scanner running per cluster.
|
2021-04-29 23:55:21 -04:00
|
|
|
func runDataScanner(pctx context.Context, objAPI ObjectLayer) {
|
2021-02-17 15:04:11 -05:00
|
|
|
// Make sure only 1 scanner is running on the cluster.
|
2022-01-02 12:15:34 -05:00
|
|
|
locker := objAPI.NewNSLock(minioMetaBucket, "scanner/runDataScanner.lock")
|
|
|
|
lkctx, err := locker.GetLock(pctx, dataScannerLeaderLockTimeout)
|
|
|
|
if err != nil {
|
2022-01-03 13:22:58 -05:00
|
|
|
if intDataUpdateTracker.debug {
|
|
|
|
logger.LogIf(pctx, err)
|
|
|
|
}
|
2022-01-02 12:15:34 -05:00
|
|
|
return
|
2020-09-18 14:15:54 -04:00
|
|
|
}
|
2022-01-02 12:15:34 -05:00
|
|
|
ctx := lkctx.Context()
|
|
|
|
defer lkctx.Cancel()
|
|
|
|
// No unlock for "leader" lock.
|
2020-09-18 14:15:54 -04:00
|
|
|
|
2020-06-12 13:28:21 -04:00
|
|
|
// Load current bloom cycle
|
|
|
|
nextBloomCycle := intDataUpdateTracker.current() + 1
|
2021-02-05 12:57:30 -05:00
|
|
|
|
2022-01-03 13:22:58 -05:00
|
|
|
buf, _ := readConfig(ctx, objAPI, dataUsageBloomNamePath)
|
|
|
|
if len(buf) >= 8 {
|
|
|
|
if err = binary.Read(bytes.NewReader(buf), binary.LittleEndian, &nextBloomCycle); err != nil {
|
2020-06-12 13:28:21 -04:00
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-30 16:59:02 -04:00
|
|
|
scannerTimer := time.NewTimer(scannerCycle.Get())
|
2021-02-26 18:11:42 -05:00
|
|
|
defer scannerTimer.Stop()
|
2020-12-17 15:35:02 -05:00
|
|
|
|
2020-06-12 13:28:21 -04:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
2021-02-26 18:11:42 -05:00
|
|
|
case <-scannerTimer.C:
|
2020-12-17 15:35:02 -05:00
|
|
|
// Reset the timer for next cycle.
|
2021-03-30 16:59:02 -04:00
|
|
|
scannerTimer.Reset(scannerCycle.Get())
|
2020-12-17 15:35:02 -05:00
|
|
|
|
2020-12-17 19:52:47 -05:00
|
|
|
if intDataUpdateTracker.debug {
|
2021-02-17 15:04:11 -05:00
|
|
|
console.Debugln("starting scanner cycle")
|
2020-12-17 19:52:47 -05:00
|
|
|
}
|
|
|
|
|
2022-04-07 11:10:40 -04:00
|
|
|
bgHealInfo := readBackgroundHealInfo(ctx, objAPI)
|
|
|
|
scanMode := getCycleScanMode(nextBloomCycle, 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
|
|
|
|
}
|
|
|
|
saveBackgroundHealInfo(ctx, objAPI, newHealInfo)
|
|
|
|
}
|
|
|
|
|
2020-06-12 13:28:21 -04:00
|
|
|
// Wait before starting next cycle and wait on startup.
|
2021-09-18 16:31:35 -04:00
|
|
|
results := make(chan DataUsageInfo, 1)
|
2020-06-12 13:28:21 -04:00
|
|
|
go storeDataUsageInBackend(ctx, objAPI, results)
|
|
|
|
bf, err := globalNotificationSys.updateBloomFilter(ctx, nextBloomCycle)
|
|
|
|
logger.LogIf(ctx, err)
|
2022-04-07 11:10:40 -04:00
|
|
|
err = objAPI.NSScanner(ctx, bf, results, uint32(nextBloomCycle), scanMode)
|
2020-06-12 13:28:21 -04:00
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
if err == nil {
|
|
|
|
// Store new cycle...
|
|
|
|
nextBloomCycle++
|
|
|
|
var tmp [8]byte
|
|
|
|
binary.LittleEndian.PutUint64(tmp[:], nextBloomCycle)
|
2022-01-03 13:22:58 -05:00
|
|
|
if err = saveConfig(ctx, objAPI, dataUsageBloomNamePath, tmp[:]); err != nil {
|
2020-06-12 13:28:21 -04:00
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
type cachedFolder struct {
|
2020-08-24 16:47:01 -04:00
|
|
|
name string
|
|
|
|
parent *dataUsageHash
|
|
|
|
objectHealProbDiv uint32
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
type folderScanner struct {
|
2021-05-19 17:38:30 -04:00
|
|
|
root string
|
|
|
|
getSize getSizeFn
|
|
|
|
oldCache dataUsageCache
|
|
|
|
newCache dataUsageCache
|
|
|
|
updateCache dataUsageCache
|
|
|
|
withFilter *bloomFilter
|
2020-06-12 13:28:21 -04:00
|
|
|
|
2021-02-26 18:11:42 -05:00
|
|
|
dataUsageScannerDebug bool
|
|
|
|
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
|
2022-04-07 11:10:40 -04:00
|
|
|
scanMode madmin.HealScanMode
|
2020-06-12 13:28:21 -04:00
|
|
|
|
2022-01-26 11:34:56 -05:00
|
|
|
disks []StorageAPI
|
|
|
|
disksQuorum int
|
2021-05-19 17:38:30 -04:00
|
|
|
|
|
|
|
// If set updates will be sent regularly to this channel.
|
|
|
|
// Will not be closed when returned.
|
|
|
|
updates chan<- dataUsageEntry
|
|
|
|
lastUpdate time.Time
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2021-10-02 12:31:05 -04:00
|
|
|
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
|
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
// 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.
|
|
|
|
|
2021-02-26 18:11:42 -05:00
|
|
|
// scanDataFolder will scanner the basepath+cache.Info.Name and return an updated cache.
|
2020-06-12 13:28:21 -04:00
|
|
|
// The returned cache will always be valid, but may not be updated from the existing.
|
2021-02-17 15:04:11 -05:00
|
|
|
// Before each operation sleepDuration is called which can be used to temporarily halt the scanner.
|
2020-06-12 13:28:21 -04:00
|
|
|
// If the supplied context is canceled the function will return at the first chance.
|
2022-04-07 11:10:40 -04:00
|
|
|
func scanDataFolder(ctx context.Context, poolIdx, setIdx int, basePath string, cache dataUsageCache, getSize getSizeFn, scanMode madmin.HealScanMode) (dataUsageCache, error) {
|
2020-06-12 13:28:21 -04:00
|
|
|
t := UTCNow()
|
|
|
|
|
|
|
|
logPrefix := color.Green("data-usage: ")
|
2020-12-29 04:57:28 -05:00
|
|
|
logSuffix := color.Blue("- %v + %v", basePath, cache.Info.Name)
|
2021-10-02 12:31:05 -04:00
|
|
|
atomic.AddUint64(&globalScannerStats.bucketsStarted, 1)
|
|
|
|
defer func() {
|
|
|
|
atomic.AddUint64(&globalScannerStats.bucketsFinished, 1)
|
|
|
|
}()
|
2020-06-12 13:28:21 -04:00
|
|
|
if intDataUpdateTracker.debug {
|
|
|
|
defer func() {
|
2021-02-26 18:11:42 -05:00
|
|
|
console.Debugf(logPrefix+" Scanner time: %v %s\n", time.Since(t), logSuffix)
|
2020-06-12 13:28:21 -04:00
|
|
|
}()
|
|
|
|
}
|
|
|
|
|
|
|
|
switch cache.Info.Name {
|
|
|
|
case "", dataUsageRoot:
|
|
|
|
return cache, errors.New("internal error: root scan attempted")
|
|
|
|
}
|
|
|
|
|
|
|
|
s := folderScanner{
|
2021-02-26 18:11:42 -05:00
|
|
|
root: basePath,
|
|
|
|
getSize: getSize,
|
|
|
|
oldCache: cache,
|
|
|
|
newCache: dataUsageCache{Info: cache.Info},
|
2021-05-19 17:38:30 -04:00
|
|
|
updateCache: dataUsageCache{Info: cache.Info},
|
2021-02-26 18:11:42 -05:00
|
|
|
dataUsageScannerDebug: intDataUpdateTracker.debug,
|
|
|
|
healFolderInclude: 0,
|
|
|
|
healObjectSelect: 0,
|
2022-04-07 11:10:40 -04:00
|
|
|
scanMode: scanMode,
|
2021-05-19 17:38:30 -04:00
|
|
|
updates: cache.Info.updates,
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2020-12-01 15:07:39 -05:00
|
|
|
// Add disks for set healing.
|
2022-01-26 11:34:56 -05:00
|
|
|
if poolIdx >= 0 && setIdx >= 0 {
|
2020-12-01 16:50:33 -05:00
|
|
|
objAPI, ok := newObjectLayerFn().(*erasureServerPools)
|
2020-12-01 15:07:39 -05:00
|
|
|
if ok {
|
2022-01-26 11:34:56 -05:00
|
|
|
if poolIdx < len(objAPI.serverPools) && setIdx < len(objAPI.serverPools[poolIdx].sets) {
|
|
|
|
// Pass the disks belonging to the set.
|
|
|
|
s.disks = objAPI.serverPools[poolIdx].sets[setIdx].getDisks()
|
2022-01-27 20:00:15 -05:00
|
|
|
s.disksQuorum = len(s.disks) / 2
|
2022-01-26 11:34:56 -05:00
|
|
|
} else {
|
|
|
|
logger.LogIf(ctx, fmt.Errorf("Matching pool %s, set %s not found", humanize.Ordinal(poolIdx+1), humanize.Ordinal(setIdx+1)))
|
2020-12-01 15:07:39 -05:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-08-24 16:47:01 -04:00
|
|
|
// Enable healing in XL mode.
|
2021-05-11 21:36:15 -04:00
|
|
|
if globalIsErasure && !cache.Info.SkipHealing {
|
2020-08-24 16:47:01 -04:00
|
|
|
// Include a clean folder one in n cycles.
|
2020-09-12 03:08:12 -04:00
|
|
|
s.healFolderInclude = healFolderIncludeProb
|
2020-08-24 16:47:01 -04:00
|
|
|
// Do a heal check on an object once every n cycles. Must divide into healFolderInclude
|
2020-09-12 03:08:12 -04:00
|
|
|
s.healObjectSelect = healObjectSelectProb
|
2020-08-24 16:47:01 -04:00
|
|
|
}
|
2020-06-12 13:28:21 -04:00
|
|
|
if len(cache.Info.BloomFilter) > 0 {
|
|
|
|
s.withFilter = &bloomFilter{BloomFilter: &bloom.BloomFilter{}}
|
2020-12-27 01:58:06 -05:00
|
|
|
_, err := s.withFilter.ReadFrom(bytes.NewReader(cache.Info.BloomFilter))
|
2020-06-12 13:28:21 -04:00
|
|
|
if err != nil {
|
|
|
|
logger.LogIf(ctx, err, logPrefix+"Error reading bloom filter")
|
|
|
|
s.withFilter = nil
|
|
|
|
}
|
|
|
|
}
|
2021-02-26 18:11:42 -05:00
|
|
|
if s.dataUsageScannerDebug {
|
|
|
|
console.Debugf(logPrefix+"Start scanning. Bloom filter: %v %s\n", s.withFilter != nil, logSuffix)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
done := ctx.Done()
|
2021-02-26 18:11:42 -05:00
|
|
|
if s.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(logPrefix+"Cycle: %v, Entries: %v %s\n", cache.Info.NextCycle, len(cache.Cache), logSuffix)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
// Read top level in bucket.
|
|
|
|
select {
|
|
|
|
case <-done:
|
|
|
|
return cache, ctx.Err()
|
|
|
|
default:
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
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
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2021-02-26 18:11:42 -05:00
|
|
|
if s.dataUsageScannerDebug {
|
2021-05-11 21:36:15 -04:00
|
|
|
console.Debugf(logPrefix+"Finished scanner, %v entries (%+v) %s \n", len(s.newCache.Cache), *s.newCache.sizeRecursive(s.newCache.Info.Name), logSuffix)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
s.newCache.Info.LastUpdate = UTCNow()
|
2021-08-25 11:25:26 -04:00
|
|
|
s.newCache.Info.NextCycle = cache.Info.NextCycle
|
2020-06-12 13:28:21 -04:00
|
|
|
return s.newCache, nil
|
|
|
|
}
|
|
|
|
|
2021-05-19 17:38:30 -04:00
|
|
|
// sendUpdate() should be called on a regular basis when the newCache contains more recent total than previously.
|
|
|
|
// May or may not send an update upstream.
|
|
|
|
func (f *folderScanner) sendUpdate() {
|
|
|
|
// Send at most an update every minute.
|
|
|
|
if f.updates == nil || time.Since(f.lastUpdate) < time.Minute {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if flat := f.updateCache.sizeRecursive(f.newCache.Info.Name); flat != nil {
|
|
|
|
select {
|
|
|
|
case f.updates <- *flat:
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
f.lastUpdate = time.Now()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
// scanFolder will scan the provided folder.
|
2020-06-12 13:28:21 -04:00
|
|
|
// 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.
|
2021-05-11 21:36:15 -04:00
|
|
|
func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, into *dataUsageEntry) error {
|
2020-06-12 13:28:21 -04:00
|
|
|
done := ctx.Done()
|
2020-12-29 04:57:28 -05:00
|
|
|
scannerLogPrefix := color.Green("folder-scanner:")
|
2021-05-11 21:36:15 -04:00
|
|
|
thisHash := hashPath(folder.name)
|
|
|
|
// Store initial compaction state.
|
|
|
|
wasCompacted := into.Compacted
|
2021-10-02 12:31:05 -04:00
|
|
|
atomic.AddUint64(&globalScannerStats.accFolders, 1)
|
2021-05-19 17:38:30 -04:00
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
for {
|
2020-06-12 13:28:21 -04:00
|
|
|
select {
|
|
|
|
case <-done:
|
2021-05-11 21:36:15 -04:00
|
|
|
return ctx.Err()
|
2020-06-12 13:28:21 -04:00
|
|
|
default:
|
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
existing, ok := f.oldCache.Cache[thisHash.Key()]
|
|
|
|
var abandonedChildren dataUsageHashMap
|
|
|
|
if !into.Compacted {
|
|
|
|
abandonedChildren = f.oldCache.findChildrenCopy(thisHash)
|
|
|
|
}
|
2020-06-12 13:28:21 -04:00
|
|
|
|
|
|
|
// If there are lifecycle rules for the prefix, remove the filter.
|
|
|
|
filter := f.withFilter
|
2020-12-13 15:05:54 -05:00
|
|
|
_, prefix := path2BucketObjectWithBasePath(f.root, folder.name)
|
2020-06-12 13:28:21 -04:00
|
|
|
var activeLifeCycle *lifecycle.Lifecycle
|
2020-12-13 15:05:54 -05:00
|
|
|
if f.oldCache.Info.lifeCycle != nil && f.oldCache.Info.lifeCycle.HasActiveRules(prefix, true) {
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(scannerLogPrefix+" Prefix %q has active rules\n", prefix)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2020-12-13 15:05:54 -05:00
|
|
|
activeLifeCycle = f.oldCache.Info.lifeCycle
|
|
|
|
filter = nil
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-06-01 22:59:11 -04:00
|
|
|
// If there are replication rules for the prefix, remove the filter.
|
|
|
|
var replicationCfg replicationConfig
|
|
|
|
if !f.oldCache.Info.replication.Empty() && f.oldCache.Info.replication.Config.HasActiveRules(prefix, true) {
|
|
|
|
replicationCfg = f.oldCache.Info.replication
|
|
|
|
filter = nil
|
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
// Check if we can skip it due to bloom filter...
|
|
|
|
if filter != nil && ok && existing.Compacted {
|
2020-06-12 13:28:21 -04:00
|
|
|
// If folder isn't in filter and we have data, skip it completely.
|
|
|
|
if folder.name != dataUsageRoot && !filter.containsDir(folder.name) {
|
2022-03-07 12:25:53 -05:00
|
|
|
if f.healObjectSelect == 0 || !thisHash.modAlt(f.oldCache.Info.NextCycle/folder.objectHealProbDiv, f.healFolderInclude/folder.objectHealProbDiv) {
|
2020-08-24 16:47:01 -04:00
|
|
|
f.newCache.copyWithChildren(&f.oldCache, thisHash, folder.parent)
|
2021-05-19 17:38:30 -04:00
|
|
|
f.updateCache.copyWithChildren(&f.oldCache, thisHash, folder.parent)
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(scannerLogPrefix+" Skipping non-updated folder: %v\n", folder.name)
|
2020-08-24 16:47:01 -04:00
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
return nil
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
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
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
}
|
2021-02-26 18:11:42 -05:00
|
|
|
scannerSleeper.Sleep(ctx, dataScannerSleepPerFolder)
|
2020-06-12 13:28:21 -04:00
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
var existingFolders, newFolders []cachedFolder
|
|
|
|
var foundObjects bool
|
2020-06-12 13:28:21 -04:00
|
|
|
err := readDirFn(path.Join(f.root, folder.name), func(entName string, typ os.FileMode) error {
|
|
|
|
// Parse
|
2021-04-15 19:32:13 -04:00
|
|
|
entName = pathClean(path.Join(folder.name, entName))
|
2021-09-15 12:24:41 -04:00
|
|
|
if entName == "" || entName == folder.name {
|
2021-04-15 19:32:13 -04:00
|
|
|
if f.dataUsageScannerDebug {
|
2021-09-15 12:24:41 -04:00
|
|
|
console.Debugf(scannerLogPrefix+" no entity (%s,%s)\n", f.root, entName)
|
2021-04-15 19:32:13 -04:00
|
|
|
}
|
2021-09-15 12:24:41 -04:00
|
|
|
return nil
|
2021-04-15 19:32:13 -04:00
|
|
|
}
|
2020-06-12 13:28:21 -04:00
|
|
|
bucket, prefix := path2BucketObjectWithBasePath(f.root, entName)
|
|
|
|
if bucket == "" {
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(scannerLogPrefix+" no bucket (%s,%s)\n", f.root, entName)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-02-17 18:34:42 -05:00
|
|
|
return errDoneForNow
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
if isReservedOrInvalidBucket(bucket, false) {
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(scannerLogPrefix+" invalid bucket: %v, entry: %v\n", bucket, entName)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-02-17 18:34:42 -05:00
|
|
|
return errDoneForNow
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
select {
|
|
|
|
case <-done:
|
2021-02-17 18:34:42 -05:00
|
|
|
return errDoneForNow
|
2020-06-12 13:28:21 -04:00
|
|
|
default:
|
|
|
|
}
|
|
|
|
|
|
|
|
if typ&os.ModeDir != 0 {
|
|
|
|
h := hashPath(entName)
|
|
|
|
_, exists := f.oldCache.Cache[h.Key()]
|
2021-09-15 12:24:41 -04:00
|
|
|
if h == thisHash {
|
|
|
|
return nil
|
|
|
|
}
|
2020-08-24 16:47:01 -04:00
|
|
|
this := cachedFolder{name: entName, parent: &thisHash, objectHealProbDiv: folder.objectHealProbDiv}
|
2021-05-11 21:36:15 -04:00
|
|
|
delete(abandonedChildren, h.Key()) // h.Key() already accounted for.
|
|
|
|
if exists {
|
|
|
|
existingFolders = append(existingFolders, this)
|
2021-05-19 17:38:30 -04:00
|
|
|
f.updateCache.copyWithChildren(&f.oldCache, h, &thisHash)
|
2020-06-12 13:28:21 -04:00
|
|
|
} else {
|
2021-05-11 21:36:15 -04:00
|
|
|
newFolders = append(newFolders, this)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
2020-10-22 16:36:24 -04:00
|
|
|
|
2020-06-12 13:28:21 -04:00
|
|
|
// Dynamic time delay.
|
2021-02-17 15:04:11 -05:00
|
|
|
wait := scannerSleeper.Timer(ctx)
|
2020-06-12 13:28:21 -04:00
|
|
|
|
|
|
|
// Get file size, ignore errors.
|
2021-02-26 18:11:42 -05:00
|
|
|
item := scannerItem{
|
2021-06-01 22:59:11 -04:00
|
|
|
Path: path.Join(f.root, entName),
|
|
|
|
Typ: typ,
|
|
|
|
bucket: bucket,
|
|
|
|
prefix: path.Dir(prefix),
|
|
|
|
objectName: path.Base(entName),
|
|
|
|
debug: f.dataUsageScannerDebug,
|
|
|
|
lifeCycle: activeLifeCycle,
|
|
|
|
replication: replicationCfg,
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2022-04-07 11:10:40 -04:00
|
|
|
|
|
|
|
item.heal.enabled = thisHash.modAlt(f.oldCache.Info.NextCycle/folder.objectHealProbDiv, f.healObjectSelect/folder.objectHealProbDiv) && globalIsErasure
|
|
|
|
item.heal.bitrot = f.scanMode == madmin.HealDeepScan
|
|
|
|
|
2021-01-19 05:40:52 -05:00
|
|
|
// if the drive belongs to an erasure set
|
|
|
|
// that is already being healed, skip the
|
|
|
|
// healing attempt on this drive.
|
2022-04-07 11:10:40 -04:00
|
|
|
item.heal.enabled = item.heal.enabled && f.healObjectSelect > 0
|
2021-01-19 05:40:52 -05:00
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
sz, err := f.getSize(item)
|
2021-09-15 12:24:41 -04:00
|
|
|
if err != nil {
|
2020-12-28 13:31:00 -05:00
|
|
|
wait() // wait to proceed to next entry.
|
2021-09-15 12:24:41 -04:00
|
|
|
if err != errSkipFile && f.dataUsageScannerDebug {
|
|
|
|
console.Debugf(scannerLogPrefix+" getSize \"%v/%v\" returned err: %v\n", bucket, item.objectPath(), err)
|
|
|
|
}
|
2020-06-12 13:28:21 -04:00
|
|
|
return nil
|
|
|
|
}
|
2021-09-15 12:24:41 -04:00
|
|
|
|
2020-12-28 13:31:00 -05:00
|
|
|
// successfully read means we have a valid object.
|
2021-05-11 21:36:15 -04:00
|
|
|
foundObjects = true
|
2020-12-28 13:31:00 -05:00
|
|
|
// 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.
|
2021-05-11 21:36:15 -04:00
|
|
|
delete(abandonedChildren, path.Join(item.bucket, item.objectPath()))
|
2020-12-28 13:31:00 -05:00
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
into.addSizes(sz)
|
|
|
|
into.Objects++
|
2020-06-12 13:28:21 -04:00
|
|
|
|
2020-12-28 13:31:00 -05:00
|
|
|
wait() // wait to proceed to next entry.
|
|
|
|
|
2020-06-12 13:28:21 -04:00
|
|
|
return nil
|
|
|
|
})
|
|
|
|
if err != nil {
|
2021-05-11 21:36:15 -04:00
|
|
|
return err
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2020-08-24 16:47:01 -04:00
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
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 {
|
2021-08-04 12:14:14 -04:00
|
|
|
h := dataUsageHash(folder.name)
|
|
|
|
into.addChild(h)
|
|
|
|
// We scanned a folder, optionally send update.
|
|
|
|
f.updateCache.deleteRecursive(h)
|
|
|
|
f.updateCache.copyWithChildren(&f.newCache, h, folder.parent)
|
|
|
|
f.sendUpdate()
|
2021-05-11 21:36:15 -04:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-08-04 12:14:14 -04:00
|
|
|
// Transfer existing
|
|
|
|
if !into.Compacted {
|
|
|
|
for _, folder := range existingFolders {
|
|
|
|
h := hashPath(folder.name)
|
|
|
|
f.updateCache.copyWithChildren(&f.oldCache, h, folder.parent)
|
|
|
|
}
|
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
// Scan new...
|
|
|
|
for _, folder := range newFolders {
|
2021-05-19 17:38:30 -04:00
|
|
|
h := hashPath(folder.name)
|
|
|
|
// Add new folders to the update tree so totals update for these.
|
|
|
|
if !into.Compacted {
|
|
|
|
var foundAny bool
|
|
|
|
parent := thisHash
|
|
|
|
for parent != hashPath(f.updateCache.Info.Name) {
|
|
|
|
e := f.updateCache.find(parent.Key())
|
|
|
|
if e == nil || e.Compacted {
|
|
|
|
foundAny = true
|
|
|
|
break
|
|
|
|
}
|
|
|
|
if next := f.updateCache.searchParent(parent); next == nil {
|
|
|
|
foundAny = true
|
|
|
|
break
|
|
|
|
} else {
|
|
|
|
parent = *next
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if !foundAny {
|
|
|
|
// Add non-compacted empty entry.
|
|
|
|
f.updateCache.replaceHashed(h, &thisHash, dataUsageEntry{})
|
|
|
|
}
|
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
scanFolder(folder)
|
2021-05-19 17:38:30 -04:00
|
|
|
// Add new folders if this is new and we don't have existing.
|
|
|
|
if !into.Compacted {
|
|
|
|
parent := f.updateCache.find(thisHash.Key())
|
|
|
|
if parent != nil && !parent.Compacted {
|
|
|
|
f.updateCache.deleteRecursive(h)
|
|
|
|
f.updateCache.copyWithChildren(&f.newCache, h, &thisHash)
|
|
|
|
}
|
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
// 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) {
|
2022-03-07 12:25:53 -05:00
|
|
|
if f.healObjectSelect == 0 || !h.modAlt(f.oldCache.Info.NextCycle/folder.objectHealProbDiv, f.healFolderInclude/folder.objectHealProbDiv) {
|
2021-05-11 21:36:15 -04:00
|
|
|
// Transfer and add as child...
|
|
|
|
f.newCache.copyWithChildren(&f.oldCache, h, folder.parent)
|
|
|
|
into.addChild(h)
|
|
|
|
continue
|
|
|
|
}
|
2022-03-07 12:25:53 -05:00
|
|
|
folder.objectHealProbDiv = f.healFolderInclude
|
2021-05-11 21:36:15 -04:00
|
|
|
}
|
|
|
|
}
|
|
|
|
scanFolder(folder)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Scan for healing
|
|
|
|
if f.healObjectSelect == 0 || len(abandonedChildren) == 0 {
|
|
|
|
// If we are not heal scanning, return now.
|
|
|
|
break
|
2020-08-24 16:47:01 -04:00
|
|
|
}
|
|
|
|
|
2020-12-01 16:50:33 -05:00
|
|
|
objAPI, ok := newObjectLayerFn().(*erasureServerPools)
|
2022-01-26 11:34:56 -05:00
|
|
|
if !ok || len(f.disks) == 0 || f.disksQuorum == 0 {
|
2021-05-11 21:36:15 -04:00
|
|
|
break
|
2020-08-24 16:47:01 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
bgSeq, found := globalBackgroundHealState.getHealSequenceByToken(bgHealingUUID)
|
|
|
|
if !found {
|
2021-05-11 21:36:15 -04:00
|
|
|
break
|
2020-08-24 16:47:01 -04:00
|
|
|
}
|
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
// Whatever remains in 'abandonedChildren' are folders at this level
|
2020-08-24 16:47:01 -04:00
|
|
|
// that existed in the previous run but wasn't found now.
|
|
|
|
//
|
|
|
|
// This may be because of 2 reasons:
|
|
|
|
//
|
|
|
|
// 1) The folder/object was deleted.
|
|
|
|
// 2) We come from another disk and this disk missed the write.
|
|
|
|
//
|
|
|
|
// We therefore perform a heal check.
|
|
|
|
// If that doesn't bring it back we remove the folder and assume it was deleted.
|
|
|
|
// This means that the next run will not look for it.
|
2020-12-01 15:07:39 -05:00
|
|
|
// How to resolve results.
|
|
|
|
resolver := metadataResolutionParams{
|
2022-01-26 11:34:56 -05:00
|
|
|
dirQuorum: f.disksQuorum,
|
|
|
|
objQuorum: f.disksQuorum,
|
2020-12-01 15:07:39 -05:00
|
|
|
bucket: "",
|
2021-12-02 14:29:16 -05:00
|
|
|
strict: false,
|
2020-12-01 15:07:39 -05:00
|
|
|
}
|
|
|
|
|
2020-12-29 04:57:28 -05:00
|
|
|
healObjectsPrefix := color.Green("healObjects:")
|
2021-05-11 21:36:15 -04:00
|
|
|
for k := range abandonedChildren {
|
2020-08-24 16:47:01 -04:00
|
|
|
bucket, prefix := path2BucketObject(k)
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(scannerLogPrefix+" checking disappeared folder: %v/%v\n", bucket, prefix)
|
2020-08-24 16:47:01 -04:00
|
|
|
}
|
|
|
|
|
2021-08-26 17:06:04 -04:00
|
|
|
if bucket != resolver.bucket {
|
|
|
|
// Bucket might be missing as well with abandoned children.
|
|
|
|
// make sure it is created first otherwise healing won't proceed
|
|
|
|
// for objects.
|
|
|
|
_, _ = objAPI.HealBucket(ctx, bucket, madmin.HealOpts{})
|
|
|
|
}
|
|
|
|
|
2020-12-01 15:07:39 -05:00
|
|
|
resolver.bucket = bucket
|
|
|
|
|
|
|
|
foundObjs := false
|
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
2021-03-06 12:25:48 -05:00
|
|
|
|
2020-12-01 15:07:39 -05:00
|
|
|
err := listPathRaw(ctx, listPathRawOptions{
|
|
|
|
disks: f.disks,
|
|
|
|
bucket: bucket,
|
|
|
|
path: prefix,
|
|
|
|
recursive: true,
|
|
|
|
reportNotFound: true,
|
2022-01-26 11:34:56 -05:00
|
|
|
minDisks: f.disksQuorum,
|
2020-12-01 15:07:39 -05:00
|
|
|
agreed: func(entry metaCacheEntry) {
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(healObjectsPrefix+" got agreement: %v\n", entry.name)
|
2020-12-01 15:07:39 -05:00
|
|
|
}
|
|
|
|
},
|
|
|
|
// Some disks have data for this.
|
|
|
|
partial: func(entries metaCacheEntries, nAgreed int, errs []error) {
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(healObjectsPrefix+" got partial, %d agreed, errs: %v\n", nAgreed, errs)
|
2020-12-01 15:07:39 -05:00
|
|
|
}
|
2020-12-28 13:31:00 -05:00
|
|
|
|
2020-12-01 15:07:39 -05:00
|
|
|
entry, ok := entries.resolve(&resolver)
|
|
|
|
if !ok {
|
2021-08-26 23:32:58 -04:00
|
|
|
// check if we can get one entry atleast
|
2021-09-21 17:55:17 -04:00
|
|
|
// proceed to heal nonetheless, since
|
|
|
|
// this object might be dangling.
|
2020-12-01 15:07:39 -05:00
|
|
|
entry, _ = entries.firstFound()
|
|
|
|
}
|
2020-10-22 16:36:24 -04:00
|
|
|
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(healObjectsPrefix+" resolved to: %v, dir: %v\n", entry.name, entry.isDir())
|
2020-12-01 15:07:39 -05:00
|
|
|
}
|
2020-12-28 13:31:00 -05:00
|
|
|
|
2020-12-01 15:07:39 -05:00
|
|
|
if entry.isDir() {
|
|
|
|
return
|
|
|
|
}
|
2021-04-27 11:24:44 -04:00
|
|
|
|
|
|
|
// wait on timer per object.
|
|
|
|
wait := scannerSleeper.Timer(ctx)
|
|
|
|
|
2020-12-01 15:07:39 -05:00
|
|
|
// We got an entry which we should be able to heal.
|
|
|
|
fiv, err := entry.fileInfoVersions(bucket)
|
|
|
|
if err != nil {
|
2021-04-27 11:24:44 -04:00
|
|
|
wait()
|
2020-12-01 15:07:39 -05:00
|
|
|
err := bgSeq.queueHealTask(healSource{
|
|
|
|
bucket: bucket,
|
|
|
|
object: entry.name,
|
|
|
|
versionID: "",
|
|
|
|
}, madmin.HealItemObject)
|
2021-09-02 23:56:13 -04:00
|
|
|
logger.LogIf(ctx, err)
|
2020-12-01 15:07:39 -05:00
|
|
|
foundObjs = foundObjs || err == nil
|
|
|
|
return
|
|
|
|
}
|
2021-04-27 11:24:44 -04:00
|
|
|
|
2020-12-01 15:07:39 -05:00
|
|
|
for _, ver := range fiv.Versions {
|
|
|
|
// Sleep and reset.
|
2020-12-04 12:32:35 -05:00
|
|
|
wait()
|
2021-02-17 15:04:11 -05:00
|
|
|
wait = scannerSleeper.Timer(ctx)
|
2021-04-27 11:24:44 -04:00
|
|
|
|
2020-12-01 15:07:39 -05:00
|
|
|
err := bgSeq.queueHealTask(healSource{
|
|
|
|
bucket: bucket,
|
|
|
|
object: fiv.Name,
|
|
|
|
versionID: ver.VersionID,
|
|
|
|
}, madmin.HealItemObject)
|
2021-09-02 23:56:13 -04:00
|
|
|
logger.LogIf(ctx, err)
|
2020-12-01 15:07:39 -05:00
|
|
|
foundObjs = foundObjs || err == nil
|
|
|
|
}
|
|
|
|
},
|
|
|
|
// Too many disks failed.
|
|
|
|
finished: func(errs []error) {
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(healObjectsPrefix+" too many errors: %v\n", errs)
|
2020-12-01 15:07:39 -05:00
|
|
|
}
|
|
|
|
cancel()
|
|
|
|
},
|
|
|
|
})
|
2020-10-22 16:36:24 -04:00
|
|
|
|
2021-02-26 18:11:42 -05:00
|
|
|
if f.dataUsageScannerDebug && err != nil && err != errFileNotFound {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(healObjectsPrefix+" checking returned value %v (%T)\n", err, err)
|
2020-12-01 15:07:39 -05:00
|
|
|
}
|
2020-08-24 16:47:01 -04:00
|
|
|
|
|
|
|
// Add unless healing returned an error.
|
2020-12-01 15:07:39 -05:00
|
|
|
if foundObjs {
|
2021-05-11 21:36:15 -04:00
|
|
|
this := cachedFolder{name: k, parent: &thisHash, objectHealProbDiv: 1}
|
|
|
|
scanFolder(this)
|
2020-08-24 16:47:01 -04:00
|
|
|
}
|
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
break
|
|
|
|
}
|
|
|
|
if !wasCompacted {
|
|
|
|
f.newCache.replaceHashed(thisHash, folder.parent, *into)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2021-05-11 21:36:15 -04:00
|
|
|
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:
|
2021-11-16 12:28:29 -05:00
|
|
|
// console.Debugf(scannerLogPrefix+" Only %d objects, compacting %s -> %+v\n", flat.Objects, folder.name, flat)
|
2021-05-11 21:36:15 -04:00
|
|
|
}
|
|
|
|
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
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if f.dataUsageScannerDebug && compact {
|
|
|
|
// Disabled, rather chatty:
|
2021-11-16 12:28:29 -05:00
|
|
|
// console.Debugf(scannerLogPrefix+" Only objects (%d), compacting %s -> %+v\n", flat.Objects, folder.name, flat)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-01-19 05:40:52 -05:00
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
if compact {
|
|
|
|
f.newCache.deleteRecursive(thisHash)
|
|
|
|
f.newCache.replaceHashed(thisHash, folder.parent, *flat)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2020-12-29 04:57:28 -05:00
|
|
|
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
// Compact if too many children...
|
|
|
|
if !into.Compacted {
|
|
|
|
f.newCache.reduceChildrenOf(thisHash, dataScannerCompactAtChildren, f.newCache.Info.Name != folder.name)
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-05-19 17:38:30 -04:00
|
|
|
if _, ok := f.updateCache.Cache[thisHash.Key()]; !wasCompacted && ok {
|
|
|
|
// Replace if existed before.
|
|
|
|
if flat := f.newCache.sizeRecursive(thisHash.Key()); flat != nil {
|
|
|
|
f.updateCache.deleteRecursive(thisHash)
|
|
|
|
f.updateCache.replaceHashed(thisHash, folder.parent, *flat)
|
|
|
|
}
|
|
|
|
}
|
2021-05-11 21:36:15 -04:00
|
|
|
|
|
|
|
return nil
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2021-02-26 18:11:42 -05:00
|
|
|
// scannerItem represents each file while walking.
|
|
|
|
type scannerItem struct {
|
2021-11-08 13:25:34 -05:00
|
|
|
Path string
|
2021-06-01 22:59:11 -04:00
|
|
|
bucket string // Bucket.
|
|
|
|
prefix string // Only the prefix if any, does not have final object name.
|
|
|
|
objectName string // Only the object name without prefixes.
|
|
|
|
replication replicationConfig
|
2021-11-08 13:25:34 -05:00
|
|
|
lifeCycle *lifecycle.Lifecycle
|
|
|
|
Typ fs.FileMode
|
2022-04-07 11:10:40 -04:00
|
|
|
heal struct {
|
|
|
|
enabled bool
|
|
|
|
bitrot bool
|
|
|
|
} // Has the object been selected for heal check?
|
|
|
|
debug bool
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2020-12-07 16:47:48 -05:00
|
|
|
type sizeSummary struct {
|
2021-09-18 16:31:35 -04:00
|
|
|
totalSize int64
|
|
|
|
versions uint64
|
|
|
|
replicatedSize int64
|
|
|
|
pendingSize int64
|
|
|
|
failedSize int64
|
|
|
|
replicaSize int64
|
|
|
|
pendingCount uint64
|
|
|
|
failedCount uint64
|
|
|
|
replTargetStats map[string]replTargetSizeSummary
|
2021-10-23 21:38:33 -04:00
|
|
|
tiers map[string]tierStats
|
2021-09-18 16:31:35 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
// replTargetSizeSummary holds summary of replication stats by target
|
|
|
|
type replTargetSizeSummary struct {
|
2020-12-07 16:47:48 -05:00
|
|
|
replicatedSize int64
|
|
|
|
pendingSize int64
|
|
|
|
failedSize int64
|
2021-04-03 12:03:42 -04:00
|
|
|
pendingCount uint64
|
|
|
|
failedCount uint64
|
2020-12-07 16:47:48 -05:00
|
|
|
}
|
|
|
|
|
2021-02-26 18:11:42 -05:00
|
|
|
type getSizeFn func(item scannerItem) (sizeSummary, error)
|
2020-06-12 13:28:21 -04:00
|
|
|
|
|
|
|
// transformMetaDir will transform a directory to prefix/file.ext
|
2021-02-26 18:11:42 -05:00
|
|
|
func (i *scannerItem) transformMetaDir() {
|
2020-06-12 13:28:21 -04:00
|
|
|
split := strings.Split(i.prefix, SlashSeparator)
|
|
|
|
if len(split) > 1 {
|
|
|
|
i.prefix = path.Join(split[:len(split)-1]...)
|
|
|
|
} else {
|
|
|
|
i.prefix = ""
|
|
|
|
}
|
|
|
|
// Object name is last element
|
|
|
|
i.objectName = split[len(split)-1]
|
|
|
|
}
|
|
|
|
|
2022-01-02 12:15:06 -05:00
|
|
|
var (
|
|
|
|
applyActionsLogPrefix = color.Green("applyActions:")
|
|
|
|
applyVersionActionsLogPrefix = color.Green("applyVersionActions:")
|
|
|
|
)
|
2021-02-01 12:52:11 -05:00
|
|
|
|
2021-08-26 17:06:04 -04:00
|
|
|
func (i *scannerItem) applyHealing(ctx context.Context, o ObjectLayer, oi ObjectInfo) (size int64) {
|
2020-06-12 13:28:21 -04:00
|
|
|
if i.debug {
|
2021-08-26 17:06:04 -04:00
|
|
|
if oi.VersionID != "" {
|
|
|
|
console.Debugf(applyActionsLogPrefix+" heal checking: %v/%v v(%s)\n", i.bucket, i.objectPath(), oi.VersionID)
|
2021-03-31 05:15:08 -04:00
|
|
|
} else {
|
|
|
|
console.Debugf(applyActionsLogPrefix+" heal checking: %v/%v\n", i.bucket, i.objectPath())
|
|
|
|
}
|
|
|
|
}
|
2022-04-07 11:10:40 -04:00
|
|
|
scanMode := madmin.HealNormalScan
|
|
|
|
if i.heal.bitrot {
|
|
|
|
scanMode = madmin.HealDeepScan
|
|
|
|
}
|
2021-08-26 17:06:04 -04:00
|
|
|
healOpts := madmin.HealOpts{
|
|
|
|
Remove: healDeleteDangling,
|
2022-04-07 11:10:40 -04:00
|
|
|
ScanMode: scanMode,
|
2021-03-31 05:15:08 -04:00
|
|
|
}
|
2021-08-26 17:06:04 -04:00
|
|
|
res, err := o.HealObject(ctx, i.bucket, i.objectPath(), oi.VersionID, healOpts)
|
2021-03-31 05:15:08 -04:00
|
|
|
if err != nil && !errors.Is(err, NotImplemented{}) {
|
2020-06-12 13:28:21 -04:00
|
|
|
logger.LogIf(ctx, err)
|
2021-03-31 05:15:08 -04:00
|
|
|
return 0
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-03-31 05:15:08 -04:00
|
|
|
return res.ObjectSize
|
|
|
|
}
|
|
|
|
|
2021-08-26 17:06:04 -04:00
|
|
|
func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, oi ObjectInfo) (applied bool, size int64) {
|
|
|
|
size, err := oi.GetActualSize()
|
2021-03-31 05:15:08 -04:00
|
|
|
if i.debug {
|
|
|
|
logger.LogIf(ctx, err)
|
2020-08-24 16:47:01 -04:00
|
|
|
}
|
2020-06-12 13:28:21 -04:00
|
|
|
if i.lifeCycle == nil {
|
2020-12-13 15:05:54 -05:00
|
|
|
if i.debug {
|
2021-05-11 21:36:15 -04:00
|
|
|
// disabled, very chatty:
|
|
|
|
// console.Debugf(applyActionsLogPrefix+" no lifecycle rules to apply: %q\n", i.objectPath())
|
2020-12-13 15:05:54 -05:00
|
|
|
}
|
2021-03-31 05:15:08 -04:00
|
|
|
return false, size
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2021-10-02 12:31:05 -04:00
|
|
|
atomic.AddUint64(&globalScannerStats.ilmChecks, 1)
|
2021-08-26 17:06:04 -04:00
|
|
|
versionID := oi.VersionID
|
2022-04-12 00:55:56 -04:00
|
|
|
rCfg, _ := globalBucketObjectLockSys.Get(i.bucket)
|
|
|
|
action := evalActionFromLifecycle(ctx, *i.lifeCycle, rCfg, oi, false)
|
2020-06-12 13:28:21 -04:00
|
|
|
if i.debug {
|
2020-12-13 15:05:54 -05:00
|
|
|
if versionID != "" {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(applyActionsLogPrefix+" lifecycle: %q (version-id=%s), Initial scan: %v\n", i.objectPath(), versionID, action)
|
2020-12-13 15:05:54 -05:00
|
|
|
} else {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(applyActionsLogPrefix+" lifecycle: %q Initial scan: %v\n", i.objectPath(), action)
|
2020-12-13 15:05:54 -05:00
|
|
|
}
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-10-02 12:31:05 -04:00
|
|
|
atomic.AddUint64(&globalScannerStats.actions[action], 1)
|
|
|
|
|
2020-06-12 13:28:21 -04:00
|
|
|
switch action {
|
2021-08-27 20:06:47 -04:00
|
|
|
case lifecycle.DeleteAction, lifecycle.DeleteVersionAction, lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:
|
|
|
|
return applyLifecycleAction(action, oi), 0
|
2020-11-12 15:12:09 -05:00
|
|
|
case lifecycle.TransitionAction, lifecycle.TransitionVersionAction:
|
2021-08-27 20:06:47 -04:00
|
|
|
return applyLifecycleAction(action, oi), size
|
2020-06-12 13:28:21 -04:00
|
|
|
default:
|
|
|
|
// No action.
|
2021-03-31 05:15:08 -04:00
|
|
|
return false, size
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
2021-03-31 05:15:08 -04:00
|
|
|
}
|
|
|
|
|
2021-06-30 22:32:07 -04:00
|
|
|
// applyTierObjSweep removes remote object pending deletion and the free-version
|
|
|
|
// tracking this information.
|
2021-08-26 17:06:04 -04:00
|
|
|
func (i *scannerItem) applyTierObjSweep(ctx context.Context, o ObjectLayer, oi ObjectInfo) {
|
|
|
|
if !oi.TransitionedObject.FreeVersion {
|
2021-06-30 22:32:07 -04:00
|
|
|
// nothing to be done
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
ignoreNotFoundErr := func(err error) error {
|
|
|
|
switch {
|
|
|
|
case isErrVersionNotFound(err), isErrObjectNotFound(err):
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
// Remove the remote object
|
2021-08-26 17:06:04 -04:00
|
|
|
err := deleteObjectFromRemoteTier(ctx, oi.TransitionedObject.Name, oi.TransitionedObject.VersionID, oi.TransitionedObject.Tier)
|
2021-06-30 22:32:07 -04:00
|
|
|
if ignoreNotFoundErr(err) != nil {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// Remove this free version
|
2021-08-26 17:06:04 -04:00
|
|
|
_, err = o.DeleteObject(ctx, oi.Bucket, oi.Name, ObjectOptions{
|
|
|
|
VersionID: oi.VersionID,
|
2021-08-24 15:24:00 -04:00
|
|
|
})
|
2021-06-30 22:32:07 -04:00
|
|
|
if err == nil {
|
2021-08-26 17:06:04 -04:00
|
|
|
auditLogLifecycle(ctx, oi, ILMFreeVersionDelete)
|
2021-06-30 22:32:07 -04:00
|
|
|
}
|
|
|
|
if ignoreNotFoundErr(err) != nil {
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-12-14 12:41:44 -05:00
|
|
|
// applyNewerNoncurrentVersionLimit removes noncurrent versions older than the most recent NewerNoncurrentVersions configured.
|
2021-11-19 20:54:10 -05:00
|
|
|
// Note: This function doesn't update sizeSummary since it always removes versions that it doesn't return.
|
2021-12-14 12:41:44 -05:00
|
|
|
func (i *scannerItem) applyNewerNoncurrentVersionLimit(ctx context.Context, _ ObjectLayer, fivs []FileInfo) ([]FileInfo, error) {
|
2021-11-19 20:54:10 -05:00
|
|
|
if i.lifeCycle == nil {
|
|
|
|
return fivs, nil
|
|
|
|
}
|
|
|
|
|
2021-12-14 12:41:44 -05:00
|
|
|
_, days, lim := i.lifeCycle.NoncurrentVersionsExpirationLimit(lifecycle.ObjectOpts{Name: i.objectPath()})
|
2021-11-19 20:54:10 -05:00
|
|
|
if lim == 0 || len(fivs) <= lim+1 { // fewer than lim _noncurrent_ versions
|
|
|
|
return fivs, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
overflowVersions := fivs[lim+1:]
|
|
|
|
// current version + most recent lim noncurrent versions
|
|
|
|
fivs = fivs[:lim+1]
|
|
|
|
|
|
|
|
rcfg, _ := globalBucketObjectLockSys.Get(i.bucket)
|
|
|
|
toDel := make([]ObjectToDelete, 0, len(overflowVersions))
|
|
|
|
for _, fi := range overflowVersions {
|
|
|
|
obj := fi.ToObjectInfo(i.bucket, i.objectPath())
|
2021-12-14 12:41:44 -05:00
|
|
|
// skip versions with object locking enabled
|
2021-11-19 20:54:10 -05:00
|
|
|
if rcfg.LockEnabled && enforceRetentionForDeletion(ctx, obj) {
|
|
|
|
if i.debug {
|
|
|
|
if obj.VersionID != "" {
|
|
|
|
console.Debugf(applyVersionActionsLogPrefix+" lifecycle: %s v(%s) is locked, not deleting\n", obj.Name, obj.VersionID)
|
|
|
|
} else {
|
|
|
|
console.Debugf(applyVersionActionsLogPrefix+" lifecycle: %s is locked, not deleting\n", obj.Name)
|
|
|
|
}
|
|
|
|
}
|
2021-12-14 12:41:44 -05:00
|
|
|
// add this version back to remaining versions for
|
|
|
|
// subsequent lifecycle policy applications
|
|
|
|
fivs = append(fivs, fi)
|
2021-11-19 20:54:10 -05:00
|
|
|
continue
|
|
|
|
}
|
2021-12-14 12:41:44 -05:00
|
|
|
|
|
|
|
// NoncurrentDays not passed yet.
|
|
|
|
if time.Now().UTC().Before(lifecycle.ExpectedExpiryTime(obj.SuccessorModTime, days)) {
|
|
|
|
// add this version back to remaining versions for
|
|
|
|
// subsequent lifecycle policy applications
|
|
|
|
fivs = append(fivs, fi)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2021-11-19 20:54:10 -05:00
|
|
|
toDel = append(toDel, ObjectToDelete{
|
2022-01-03 04:28:52 -05:00
|
|
|
ObjectV: ObjectV{
|
|
|
|
ObjectName: fi.Name,
|
|
|
|
VersionID: fi.VersionID,
|
|
|
|
},
|
2021-11-19 20:54:10 -05:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2021-12-14 12:41:44 -05:00
|
|
|
globalExpiryState.enqueueByNewerNoncurrent(i.bucket, toDel)
|
2021-11-19 20:54:10 -05:00
|
|
|
return fivs, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// applyVersionActions will apply lifecycle checks on all versions of a scanned item. Returns versions that remain
|
|
|
|
// after applying lifecycle checks configured.
|
|
|
|
func (i *scannerItem) applyVersionActions(ctx context.Context, o ObjectLayer, fivs []FileInfo) ([]FileInfo, error) {
|
2021-12-14 12:41:44 -05:00
|
|
|
return i.applyNewerNoncurrentVersionLimit(ctx, o, fivs)
|
2021-11-19 20:54:10 -05:00
|
|
|
}
|
|
|
|
|
2021-03-31 05:15:08 -04:00
|
|
|
// applyActions will apply lifecycle checks on to a scanned item.
|
|
|
|
// The resulting size on disk will always be returned.
|
|
|
|
// 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.
|
2021-08-26 17:06:04 -04:00
|
|
|
func (i *scannerItem) applyActions(ctx context.Context, o ObjectLayer, oi ObjectInfo, sizeS *sizeSummary) int64 {
|
|
|
|
applied, size := i.applyLifecycle(ctx, o, oi)
|
2021-03-31 05:15:08 -04:00
|
|
|
// 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.
|
2021-04-27 11:24:44 -04:00
|
|
|
if !applied {
|
2022-04-07 11:10:40 -04:00
|
|
|
if i.heal.enabled {
|
2021-08-26 17:06:04 -04:00
|
|
|
size = i.applyHealing(ctx, o, oi)
|
2021-04-27 11:24:44 -04:00
|
|
|
}
|
|
|
|
// replicate only if lifecycle rules are not applied.
|
2021-08-26 17:06:04 -04:00
|
|
|
i.healReplication(ctx, o, oi.Clone(), sizeS)
|
2021-02-01 12:52:11 -05:00
|
|
|
}
|
|
|
|
return size
|
|
|
|
}
|
|
|
|
|
2022-04-11 16:25:32 -04:00
|
|
|
func evalActionFromLifecycle(ctx context.Context, lc lifecycle.Lifecycle, lr lock.Retention, obj ObjectInfo, debug bool) (action lifecycle.Action) {
|
2021-07-21 22:12:44 -04:00
|
|
|
action = lc.ComputeAction(obj.ToLifecycleOpts())
|
2021-02-01 12:52:11 -05:00
|
|
|
if debug {
|
2020-12-29 04:57:28 -05:00
|
|
|
console.Debugf(applyActionsLogPrefix+" lifecycle: Secondary scan: %v\n", action)
|
2020-08-04 02:04:40 -04:00
|
|
|
}
|
2021-02-01 12:52:11 -05:00
|
|
|
|
|
|
|
if action == lifecycle.NoneAction {
|
|
|
|
return action
|
2020-08-04 02:04:40 -04:00
|
|
|
}
|
2020-06-12 13:28:21 -04:00
|
|
|
|
2020-07-04 23:56:02 -04:00
|
|
|
switch action {
|
2020-11-12 15:12:09 -05:00
|
|
|
case lifecycle.DeleteVersionAction, lifecycle.DeleteRestoredVersionAction:
|
2020-08-04 02:04:40 -04:00
|
|
|
// Defensive code, should never happen
|
|
|
|
if obj.VersionID == "" {
|
2021-02-01 12:52:11 -05:00
|
|
|
return lifecycle.NoneAction
|
2020-08-04 02:04:40 -04:00
|
|
|
}
|
2022-04-11 16:25:32 -04:00
|
|
|
if lr.LockEnabled && enforceRetentionForDeletion(ctx, obj) {
|
|
|
|
if debug {
|
|
|
|
if obj.VersionID != "" {
|
|
|
|
console.Debugf(applyActionsLogPrefix+" lifecycle: %s v(%s) is locked, not deleting\n", obj.Name, obj.VersionID)
|
|
|
|
} else {
|
|
|
|
console.Debugf(applyActionsLogPrefix+" lifecycle: %s is locked, not deleting\n", obj.Name)
|
2020-08-04 02:04:40 -04:00
|
|
|
}
|
2020-11-12 15:12:09 -05:00
|
|
|
}
|
2022-04-11 16:25:32 -04:00
|
|
|
return lifecycle.NoneAction
|
2020-11-12 15:12:09 -05:00
|
|
|
}
|
2020-07-04 23:56:02 -04:00
|
|
|
}
|
2020-12-13 15:05:54 -05:00
|
|
|
|
2021-02-01 12:52:11 -05:00
|
|
|
return action
|
|
|
|
}
|
|
|
|
|
2021-08-25 11:25:26 -04:00
|
|
|
func applyTransitionRule(obj ObjectInfo) bool {
|
2021-08-24 15:24:00 -04:00
|
|
|
if obj.DeleteMarker {
|
|
|
|
return false
|
2020-12-13 15:05:54 -05:00
|
|
|
}
|
2021-02-01 12:52:11 -05:00
|
|
|
globalTransitionState.queueTransitionTask(obj)
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
func applyExpiryOnTransitionedObject(ctx context.Context, objLayer ObjectLayer, obj ObjectInfo, restoredObject bool) bool {
|
2021-04-19 13:30:42 -04:00
|
|
|
action := expireObj
|
|
|
|
if restoredObject {
|
|
|
|
action = expireRestoredObj
|
|
|
|
}
|
2021-07-21 22:12:44 -04:00
|
|
|
if err := expireTransitionedObject(ctx, objLayer, &obj, obj.ToLifecycleOpts(), action); err != nil {
|
2021-02-01 12:52:11 -05:00
|
|
|
if isErrObjectNotFound(err) || isErrVersionNotFound(err) {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
logger.LogIf(ctx, err)
|
|
|
|
return false
|
|
|
|
}
|
2021-04-19 13:30:42 -04:00
|
|
|
// Notification already sent in *expireTransitionedObject*, just return 'true' here.
|
2021-02-01 12:52:11 -05:00
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
2021-02-06 19:10:33 -05:00
|
|
|
func applyExpiryOnNonTransitionedObjects(ctx context.Context, objLayer ObjectLayer, obj ObjectInfo, applyOnVersion bool) bool {
|
2021-08-27 20:06:47 -04:00
|
|
|
opts := ObjectOptions{
|
|
|
|
Expiration: ExpirationOptions{Expire: true},
|
|
|
|
}
|
2021-02-01 12:52:11 -05:00
|
|
|
|
2021-02-06 19:10:33 -05:00
|
|
|
if applyOnVersion {
|
|
|
|
opts.VersionID = obj.VersionID
|
|
|
|
}
|
2021-02-01 12:52:11 -05:00
|
|
|
if opts.VersionID == "" {
|
2022-05-06 22:05:28 -04:00
|
|
|
opts.Versioned = globalBucketVersioningSys.PrefixEnabled(obj.Bucket, obj.Name)
|
2021-02-01 12:52:11 -05:00
|
|
|
}
|
|
|
|
|
|
|
|
obj, err := objLayer.DeleteObject(ctx, obj.Bucket, obj.Name, opts)
|
2020-12-13 15:05:54 -05:00
|
|
|
if err != nil {
|
2021-01-17 16:58:41 -05:00
|
|
|
if isErrObjectNotFound(err) || isErrVersionNotFound(err) {
|
2021-02-01 12:52:11 -05:00
|
|
|
return false
|
2021-01-17 16:58:41 -05:00
|
|
|
}
|
2020-12-13 15:05:54 -05:00
|
|
|
// Assume it is still there.
|
|
|
|
logger.LogIf(ctx, err)
|
2021-02-01 12:52:11 -05:00
|
|
|
return false
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
2021-04-23 12:51:12 -04:00
|
|
|
// Send audit for the lifecycle delete operation
|
2021-07-01 17:02:44 -04:00
|
|
|
auditLogLifecycle(ctx, obj, ILMExpiry)
|
2021-04-23 12:51:12 -04:00
|
|
|
|
2020-10-17 00:22:12 -04:00
|
|
|
eventName := event.ObjectRemovedDelete
|
|
|
|
if obj.DeleteMarker {
|
|
|
|
eventName = event.ObjectRemovedDeleteMarkerCreated
|
|
|
|
}
|
|
|
|
|
2020-06-12 13:28:21 -04:00
|
|
|
// Notify object deleted event.
|
|
|
|
sendEvent(eventArgs{
|
2020-10-17 00:22:12 -04:00
|
|
|
EventName: eventName,
|
2021-02-01 12:52:11 -05:00
|
|
|
BucketName: obj.Bucket,
|
2020-06-12 23:04:01 -04:00
|
|
|
Object: obj,
|
|
|
|
Host: "Internal: [ILM-EXPIRY]",
|
2020-06-12 13:28:21 -04:00
|
|
|
})
|
2021-02-01 12:52:11 -05:00
|
|
|
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
// Apply object, object version, restored object or restored object version action on the given object
|
2021-08-27 20:06:47 -04:00
|
|
|
func applyExpiryRule(obj ObjectInfo, restoredObject, applyOnVersion bool) bool {
|
2021-11-19 20:54:10 -05:00
|
|
|
globalExpiryState.enqueueByDays(obj, restoredObject, applyOnVersion)
|
2021-08-27 20:06:47 -04:00
|
|
|
return true
|
2021-02-01 12:52:11 -05:00
|
|
|
}
|
|
|
|
|
2021-03-05 17:15:53 -05:00
|
|
|
// Perform actions (removal or transitioning of objects), return true the action is successfully performed
|
2021-08-27 20:06:47 -04:00
|
|
|
func applyLifecycleAction(action lifecycle.Action, obj ObjectInfo) (success bool) {
|
2021-02-01 12:52:11 -05:00
|
|
|
switch action {
|
|
|
|
case lifecycle.DeleteVersionAction, lifecycle.DeleteAction:
|
2021-08-27 20:06:47 -04:00
|
|
|
success = applyExpiryRule(obj, false, action == lifecycle.DeleteVersionAction)
|
2021-02-01 12:52:11 -05:00
|
|
|
case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:
|
2021-08-27 20:06:47 -04:00
|
|
|
success = applyExpiryRule(obj, true, action == lifecycle.DeleteRestoredVersionAction)
|
2021-02-01 12:52:11 -05:00
|
|
|
case lifecycle.TransitionAction, lifecycle.TransitionVersionAction:
|
2021-08-25 11:25:26 -04:00
|
|
|
success = applyTransitionRule(obj)
|
2021-02-01 12:52:11 -05:00
|
|
|
}
|
|
|
|
return
|
2020-06-12 13:28:21 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
// objectPath returns the prefix and object name.
|
2021-02-26 18:11:42 -05:00
|
|
|
func (i *scannerItem) objectPath() string {
|
2020-06-12 13:28:21 -04:00
|
|
|
return path.Join(i.prefix, i.objectName)
|
|
|
|
}
|
|
|
|
|
2020-07-21 20:49:56 -04:00
|
|
|
// healReplication will heal a scanned item that has failed replication.
|
2021-02-26 18:11:42 -05:00
|
|
|
func (i *scannerItem) healReplication(ctx context.Context, o ObjectLayer, oi ObjectInfo, sizeS *sizeSummary) {
|
2021-09-18 16:31:35 -04:00
|
|
|
roi := getHealReplicateObjectInfo(oi, i.replication)
|
2022-05-06 22:05:28 -04:00
|
|
|
if !roi.Dsc.ReplicateAny() {
|
|
|
|
return
|
|
|
|
}
|
2021-09-18 16:31:35 -04:00
|
|
|
|
2020-12-28 13:31:00 -05:00
|
|
|
if oi.DeleteMarker || !oi.VersionPurgeStatus.Empty() {
|
2020-12-13 15:05:54 -05:00
|
|
|
// heal delete marker replication failure or versioned delete replication failure
|
2020-12-28 13:31:00 -05:00
|
|
|
if oi.ReplicationStatus == replication.Pending ||
|
|
|
|
oi.ReplicationStatus == replication.Failed ||
|
|
|
|
oi.VersionPurgeStatus == Failed || oi.VersionPurgeStatus == Pending {
|
2021-09-18 16:31:35 -04:00
|
|
|
i.healReplicationDeletes(ctx, o, roi)
|
2020-11-19 21:43:58 -05:00
|
|
|
return
|
|
|
|
}
|
2021-06-01 22:59:11 -04:00
|
|
|
// if replication status is Complete on DeleteMarker and existing object resync required
|
2021-09-18 16:31:35 -04:00
|
|
|
if roi.ExistingObjResync.mustResync() && (oi.ReplicationStatus == replication.Completed || oi.ReplicationStatus.Empty()) {
|
|
|
|
i.healReplicationDeletes(ctx, o, roi)
|
2021-06-01 22:59:11 -04:00
|
|
|
return
|
|
|
|
}
|
|
|
|
return
|
|
|
|
}
|
2021-09-18 16:31:35 -04:00
|
|
|
if roi.ExistingObjResync.mustResync() {
|
2021-06-01 22:59:11 -04:00
|
|
|
roi.OpType = replication.ExistingObjectReplicationType
|
2020-11-19 21:43:58 -05:00
|
|
|
}
|
2021-09-18 16:31:35 -04:00
|
|
|
|
2022-04-11 16:25:32 -04:00
|
|
|
if sizeS.replTargetStats == nil && len(roi.TargetStatuses) > 0 {
|
|
|
|
sizeS.replTargetStats = make(map[string]replTargetSizeSummary)
|
|
|
|
}
|
|
|
|
|
|
|
|
for arn, tgtStatus := range roi.TargetStatuses {
|
|
|
|
tgtSizeS, ok := sizeS.replTargetStats[arn]
|
|
|
|
if !ok {
|
|
|
|
tgtSizeS = replTargetSizeSummary{}
|
2021-09-18 16:31:35 -04:00
|
|
|
}
|
2022-04-11 16:25:32 -04:00
|
|
|
switch tgtStatus {
|
|
|
|
case replication.Pending:
|
|
|
|
tgtSizeS.pendingCount++
|
|
|
|
tgtSizeS.pendingSize += oi.Size
|
|
|
|
sizeS.pendingCount++
|
|
|
|
sizeS.pendingSize += oi.Size
|
|
|
|
case replication.Failed:
|
|
|
|
tgtSizeS.failedSize += oi.Size
|
|
|
|
tgtSizeS.failedCount++
|
|
|
|
sizeS.failedSize += oi.Size
|
|
|
|
sizeS.failedCount++
|
|
|
|
case replication.Completed, replication.CompletedLegacy:
|
|
|
|
tgtSizeS.replicatedSize += oi.Size
|
|
|
|
sizeS.replicatedSize += oi.Size
|
2021-09-18 16:31:35 -04:00
|
|
|
}
|
2022-04-11 16:25:32 -04:00
|
|
|
sizeS.replTargetStats[arn] = tgtSizeS
|
2021-09-18 16:31:35 -04:00
|
|
|
}
|
|
|
|
|
2020-12-28 13:31:00 -05:00
|
|
|
switch oi.ReplicationStatus {
|
2021-09-22 13:48:45 -04:00
|
|
|
case replication.Pending, replication.Failed:
|
2021-06-01 22:59:11 -04:00
|
|
|
globalReplicationPool.queueReplicaTask(roi)
|
|
|
|
return
|
2020-12-07 16:47:48 -05:00
|
|
|
case replication.Replica:
|
2020-12-28 13:31:00 -05:00
|
|
|
sizeS.replicaSize += oi.Size
|
2020-07-21 20:49:56 -04:00
|
|
|
}
|
2021-09-18 16:31:35 -04:00
|
|
|
if roi.ExistingObjResync.mustResync() {
|
2021-06-01 22:59:11 -04:00
|
|
|
globalReplicationPool.queueReplicaTask(roi)
|
|
|
|
}
|
2020-07-21 20:49:56 -04:00
|
|
|
}
|
2020-11-19 21:43:58 -05:00
|
|
|
|
|
|
|
// healReplicationDeletes will heal a scanned deleted item that failed to replicate deletes.
|
2021-09-18 16:31:35 -04:00
|
|
|
func (i *scannerItem) healReplicationDeletes(ctx context.Context, o ObjectLayer, roi ReplicateObjectInfo) {
|
2020-11-19 21:43:58 -05:00
|
|
|
// handle soft delete and permanent delete failures here.
|
2021-09-18 16:31:35 -04:00
|
|
|
if roi.DeleteMarker || !roi.VersionPurgeStatus.Empty() {
|
2020-11-19 21:43:58 -05:00
|
|
|
versionID := ""
|
|
|
|
dmVersionID := ""
|
2021-09-18 16:31:35 -04:00
|
|
|
if roi.VersionPurgeStatus.Empty() {
|
|
|
|
dmVersionID = roi.VersionID
|
2020-11-19 21:43:58 -05:00
|
|
|
} else {
|
2021-09-18 16:31:35 -04:00
|
|
|
versionID = roi.VersionID
|
2020-11-19 21:43:58 -05:00
|
|
|
}
|
2021-09-18 16:31:35 -04:00
|
|
|
|
2021-06-01 22:59:11 -04:00
|
|
|
doi := DeletedObjectReplicationInfo{
|
2020-11-19 21:43:58 -05:00
|
|
|
DeletedObject: DeletedObject{
|
2021-09-18 16:31:35 -04:00
|
|
|
ObjectName: roi.Name,
|
|
|
|
DeleteMarkerVersionID: dmVersionID,
|
|
|
|
VersionID: versionID,
|
|
|
|
ReplicationState: roi.getReplicationState(roi.Dsc.String(), versionID, true),
|
|
|
|
DeleteMarkerMTime: DeleteMarkerMTime{roi.ModTime},
|
|
|
|
DeleteMarker: roi.DeleteMarker,
|
2020-11-19 21:43:58 -05:00
|
|
|
},
|
2021-09-18 16:31:35 -04:00
|
|
|
Bucket: roi.Bucket,
|
2021-12-16 18:34:55 -05:00
|
|
|
OpType: replication.HealReplicationType,
|
2021-06-01 22:59:11 -04:00
|
|
|
}
|
2021-09-18 16:31:35 -04:00
|
|
|
if roi.ExistingObjResync.mustResync() {
|
2021-06-01 22:59:11 -04:00
|
|
|
doi.OpType = replication.ExistingObjectReplicationType
|
2021-09-18 16:31:35 -04:00
|
|
|
queueReplicateDeletesWrapper(doi, roi.ExistingObjResync)
|
|
|
|
return
|
2021-06-01 22:59:11 -04:00
|
|
|
}
|
|
|
|
globalReplicationPool.queueReplicaDeleteTask(doi)
|
2020-11-19 21:43:58 -05:00
|
|
|
}
|
|
|
|
}
|
2020-12-04 12:32:35 -05:00
|
|
|
|
|
|
|
type dynamicSleeper struct {
|
|
|
|
mu sync.RWMutex
|
|
|
|
|
|
|
|
// Sleep factor
|
|
|
|
factor float64
|
|
|
|
|
|
|
|
// maximum sleep cap,
|
|
|
|
// set to <= 0 to disable.
|
|
|
|
maxSleep time.Duration
|
|
|
|
|
|
|
|
// Don't sleep at all, if time taken is below this value.
|
|
|
|
// This is to avoid too small costly sleeps.
|
|
|
|
minSleep time.Duration
|
|
|
|
|
|
|
|
// cycle will be closed
|
|
|
|
cycle chan struct{}
|
|
|
|
}
|
|
|
|
|
|
|
|
// newDynamicSleeper
|
|
|
|
func newDynamicSleeper(factor float64, maxWait time.Duration) *dynamicSleeper {
|
|
|
|
return &dynamicSleeper{
|
|
|
|
factor: factor,
|
|
|
|
cycle: make(chan struct{}),
|
|
|
|
maxSleep: maxWait,
|
|
|
|
minSleep: 100 * time.Microsecond,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Timer returns a timer that has started.
|
|
|
|
// When the returned function is called it will wait.
|
|
|
|
func (d *dynamicSleeper) Timer(ctx context.Context) func() {
|
|
|
|
t := time.Now()
|
|
|
|
return func() {
|
|
|
|
doneAt := time.Now()
|
|
|
|
for {
|
|
|
|
// Grab current values
|
|
|
|
d.mu.RLock()
|
|
|
|
minWait, maxWait := d.minSleep, d.maxSleep
|
|
|
|
factor := d.factor
|
|
|
|
cycle := d.cycle
|
|
|
|
d.mu.RUnlock()
|
|
|
|
elapsed := doneAt.Sub(t)
|
|
|
|
// Don't sleep for really small amount of time
|
|
|
|
wantSleep := time.Duration(float64(elapsed) * factor)
|
|
|
|
if wantSleep <= minWait {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if maxWait > 0 && wantSleep > maxWait {
|
|
|
|
wantSleep = maxWait
|
|
|
|
}
|
|
|
|
timer := time.NewTimer(wantSleep)
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
if !timer.Stop() {
|
|
|
|
<-timer.C
|
|
|
|
}
|
|
|
|
return
|
|
|
|
case <-timer.C:
|
|
|
|
return
|
|
|
|
case <-cycle:
|
|
|
|
if !timer.Stop() {
|
|
|
|
// We expired.
|
|
|
|
<-timer.C
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Sleep sleeps the specified time multiplied by the sleep factor.
|
|
|
|
// If the factor is updated the sleep will be done again with the new factor.
|
|
|
|
func (d *dynamicSleeper) Sleep(ctx context.Context, base time.Duration) {
|
|
|
|
for {
|
|
|
|
// Grab current values
|
|
|
|
d.mu.RLock()
|
|
|
|
minWait, maxWait := d.minSleep, d.maxSleep
|
|
|
|
factor := d.factor
|
|
|
|
cycle := d.cycle
|
|
|
|
d.mu.RUnlock()
|
|
|
|
// Don't sleep for really small amount of time
|
|
|
|
wantSleep := time.Duration(float64(base) * factor)
|
|
|
|
if wantSleep <= minWait {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if maxWait > 0 && wantSleep > maxWait {
|
|
|
|
wantSleep = maxWait
|
|
|
|
}
|
|
|
|
timer := time.NewTimer(wantSleep)
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
if !timer.Stop() {
|
|
|
|
<-timer.C
|
|
|
|
}
|
|
|
|
return
|
|
|
|
case <-timer.C:
|
|
|
|
return
|
|
|
|
case <-cycle:
|
|
|
|
if !timer.Stop() {
|
|
|
|
// We expired.
|
|
|
|
<-timer.C
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update the current settings and cycle all waiting.
|
|
|
|
// Parameters are the same as in the contructor.
|
|
|
|
func (d *dynamicSleeper) Update(factor float64, maxWait time.Duration) error {
|
|
|
|
d.mu.Lock()
|
|
|
|
defer d.mu.Unlock()
|
|
|
|
if math.Abs(d.factor-factor) < 1e-10 && d.maxSleep == maxWait {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
// Update values and cycle waiting.
|
|
|
|
close(d.cycle)
|
|
|
|
d.factor = factor
|
|
|
|
d.maxSleep = maxWait
|
|
|
|
d.cycle = make(chan struct{})
|
|
|
|
return nil
|
|
|
|
}
|
2021-04-23 12:51:12 -04:00
|
|
|
|
2021-06-30 22:32:07 -04:00
|
|
|
const (
|
2021-07-01 17:02:44 -04:00
|
|
|
// ILMExpiry - audit trail for ILM expiry
|
|
|
|
ILMExpiry = "ilm:expiry"
|
|
|
|
// ILMFreeVersionDelete - audit trail for ILM free-version delete
|
|
|
|
ILMFreeVersionDelete = "ilm:free-version-delete"
|
2021-07-30 15:45:25 -04:00
|
|
|
// ILMTransition - audit trail for ILM transitioning.
|
|
|
|
ILMTransition = " ilm:transition"
|
2021-06-30 22:32:07 -04:00
|
|
|
)
|
2021-06-29 02:58:08 -04:00
|
|
|
|
2021-07-01 17:02:44 -04:00
|
|
|
func auditLogLifecycle(ctx context.Context, oi ObjectInfo, trigger string) {
|
2021-06-30 22:32:07 -04:00
|
|
|
var apiName string
|
|
|
|
switch trigger {
|
2021-07-01 17:02:44 -04:00
|
|
|
case ILMExpiry:
|
|
|
|
apiName = "ILMExpiry"
|
|
|
|
case ILMFreeVersionDelete:
|
|
|
|
apiName = "ILMFreeVersionDelete"
|
2021-07-30 15:45:25 -04:00
|
|
|
case ILMTransition:
|
|
|
|
apiName = "ILMTransition"
|
2021-06-30 22:32:07 -04:00
|
|
|
}
|
2021-07-01 17:02:44 -04:00
|
|
|
auditLogInternal(ctx, oi.Bucket, oi.Name, AuditLogOptions{
|
2021-06-30 22:32:07 -04:00
|
|
|
Trigger: trigger,
|
|
|
|
APIName: apiName,
|
2021-07-01 17:02:44 -04:00
|
|
|
VersionID: oi.VersionID,
|
2021-06-29 02:58:08 -04:00
|
|
|
})
|
2021-04-23 12:51:12 -04:00
|
|
|
}
|