mirror of
https://github.com/minio/minio.git
synced 2024-12-24 22:25:54 -05:00
refactor healing to remove certain structs (#13079)
- remove sourceCh usage from healing we already have tasks and resp channel - use read locks to lookup globalHealConfig - fix healing resolver to pick candidates quickly that need healing, without this resolver was unexpectedly skipping.
This commit is contained in:
parent
2f9ab26372
commit
c11a2ac396
@ -396,9 +396,6 @@ type healSequence struct {
|
|||||||
// bucket, and object on which heal seq. was initiated
|
// bucket, and object on which heal seq. was initiated
|
||||||
bucket, object string
|
bucket, object string
|
||||||
|
|
||||||
// A channel of entities (format, buckets, objects) to heal
|
|
||||||
sourceCh chan healSource
|
|
||||||
|
|
||||||
// A channel of entities with heal result
|
// A channel of entities with heal result
|
||||||
respCh chan healResult
|
respCh chan healResult
|
||||||
|
|
||||||
@ -648,11 +645,7 @@ func (h *healSequence) healSequenceStart(objAPI ObjectLayer) {
|
|||||||
h.currentStatus.StartTime = UTCNow()
|
h.currentStatus.StartTime = UTCNow()
|
||||||
h.mutex.Unlock()
|
h.mutex.Unlock()
|
||||||
|
|
||||||
if h.sourceCh == nil {
|
|
||||||
go h.traverseAndHeal(objAPI)
|
go h.traverseAndHeal(objAPI)
|
||||||
} else {
|
|
||||||
go h.healFromSourceCh()
|
|
||||||
}
|
|
||||||
|
|
||||||
select {
|
select {
|
||||||
case err, ok := <-h.traverseAndHealDoneCh:
|
case err, ok := <-h.traverseAndHealDoneCh:
|
||||||
@ -696,10 +689,6 @@ func (h *healSequence) logHeal(healType madmin.HealItemType) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (h *healSequence) queueHealTask(source healSource, healType madmin.HealItemType) error {
|
func (h *healSequence) queueHealTask(source healSource, healType madmin.HealItemType) error {
|
||||||
globalHealConfigMu.Lock()
|
|
||||||
opts := globalHealConfig
|
|
||||||
globalHealConfigMu.Unlock()
|
|
||||||
|
|
||||||
// Send heal request
|
// Send heal request
|
||||||
task := healTask{
|
task := healTask{
|
||||||
bucket: source.bucket,
|
bucket: source.bucket,
|
||||||
@ -711,9 +700,7 @@ func (h *healSequence) queueHealTask(source healSource, healType madmin.HealItem
|
|||||||
if source.opts != nil {
|
if source.opts != nil {
|
||||||
task.opts = *source.opts
|
task.opts = *source.opts
|
||||||
}
|
}
|
||||||
if opts.Bitrot {
|
task.opts.ScanMode = globalHealConfig.ScanMode()
|
||||||
task.opts.ScanMode = madmin.HealDeepScan
|
|
||||||
}
|
|
||||||
|
|
||||||
h.mutex.Lock()
|
h.mutex.Lock()
|
||||||
h.scannedItemsMap[healType]++
|
h.scannedItemsMap[healType]++
|
||||||
@ -773,48 +760,6 @@ func (h *healSequence) queueHealTask(source healSource, healType madmin.HealItem
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *healSequence) healItemsFromSourceCh() error {
|
|
||||||
for {
|
|
||||||
select {
|
|
||||||
case source, ok := <-h.sourceCh:
|
|
||||||
if !ok {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
var itemType madmin.HealItemType
|
|
||||||
switch source.bucket {
|
|
||||||
case nopHeal:
|
|
||||||
continue
|
|
||||||
case SlashSeparator:
|
|
||||||
itemType = madmin.HealItemMetadata
|
|
||||||
default:
|
|
||||||
if source.object == "" {
|
|
||||||
itemType = madmin.HealItemBucket
|
|
||||||
} else {
|
|
||||||
itemType = madmin.HealItemObject
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := h.queueHealTask(source, itemType); err != nil {
|
|
||||||
switch err.(type) {
|
|
||||||
case ObjectExistsAsDirectory:
|
|
||||||
case ObjectNotFound:
|
|
||||||
case VersionNotFound:
|
|
||||||
default:
|
|
||||||
logger.LogIf(h.ctx, fmt.Errorf("Heal attempt failed for %s: %w",
|
|
||||||
pathJoin(source.bucket, source.object), err))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
case <-h.ctx.Done():
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *healSequence) healFromSourceCh() {
|
|
||||||
h.healItemsFromSourceCh()
|
|
||||||
}
|
|
||||||
|
|
||||||
func (h *healSequence) healDiskMeta(objAPI ObjectLayer) error {
|
func (h *healSequence) healDiskMeta(objAPI ObjectLayer) error {
|
||||||
// Start healing the config prefix.
|
// Start healing the config prefix.
|
||||||
return h.healMinioSysMeta(objAPI, minioConfigPrefix)()
|
return h.healMinioSysMeta(objAPI, minioConfigPrefix)()
|
||||||
|
@ -19,10 +19,8 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"time"
|
|
||||||
|
|
||||||
"github.com/minio/madmin-go"
|
"github.com/minio/madmin-go"
|
||||||
"github.com/minio/minio/internal/logger"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// healTask represents what to heal along with options
|
// healTask represents what to heal along with options
|
||||||
@ -52,51 +50,22 @@ type healRoutine struct {
|
|||||||
|
|
||||||
// Add a new task in the tasks queue
|
// Add a new task in the tasks queue
|
||||||
func (h *healRoutine) queueHealTask(task healTask) {
|
func (h *healRoutine) queueHealTask(task healTask) {
|
||||||
select {
|
h.tasks <- task
|
||||||
case h.tasks <- task:
|
}
|
||||||
default:
|
|
||||||
}
|
func systemIO() int {
|
||||||
|
// Bucket notification and http trace are not costly, it is okay to ignore them
|
||||||
|
// while counting the number of concurrent connections
|
||||||
|
return int(globalHTTPListen.NumSubscribers()) + int(globalTrace.NumSubscribers())
|
||||||
}
|
}
|
||||||
|
|
||||||
func waitForLowHTTPReq() {
|
func waitForLowHTTPReq() {
|
||||||
globalHealConfigMu.Lock()
|
var currentIO func() int
|
||||||
maxIO, maxWait := globalHealConfig.IOCount, globalHealConfig.Sleep
|
|
||||||
globalHealConfigMu.Unlock()
|
|
||||||
|
|
||||||
// No need to wait run at full speed.
|
|
||||||
if maxIO <= 0 {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
// At max 10 attempts to wait with 100 millisecond interval before proceeding
|
|
||||||
waitTick := 100 * time.Millisecond
|
|
||||||
|
|
||||||
// Bucket notification and http trace are not costly, it is okay to ignore them
|
|
||||||
// while counting the number of concurrent connections
|
|
||||||
maxIOFn := func() int {
|
|
||||||
return maxIO + int(globalHTTPListen.NumSubscribers()) + int(globalTrace.NumSubscribers())
|
|
||||||
}
|
|
||||||
|
|
||||||
tmpMaxWait := maxWait
|
|
||||||
if httpServer := newHTTPServerFn(); httpServer != nil {
|
if httpServer := newHTTPServerFn(); httpServer != nil {
|
||||||
// Any requests in progress, delay the heal.
|
currentIO = httpServer.GetRequestCount
|
||||||
for httpServer.GetRequestCount() >= maxIOFn() {
|
|
||||||
if tmpMaxWait > 0 {
|
|
||||||
if tmpMaxWait < waitTick {
|
|
||||||
time.Sleep(tmpMaxWait)
|
|
||||||
} else {
|
|
||||||
time.Sleep(waitTick)
|
|
||||||
}
|
|
||||||
tmpMaxWait = tmpMaxWait - waitTick
|
|
||||||
}
|
|
||||||
if tmpMaxWait <= 0 {
|
|
||||||
if intDataUpdateTracker.debug {
|
|
||||||
logger.Info("waitForLowHTTPReq: waited max %s, resuming", maxWait)
|
|
||||||
}
|
|
||||||
break
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
globalHealConfig.Wait(currentIO, systemIO)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Wait for heal requests and process them
|
// Wait for heal requests and process them
|
||||||
@ -123,10 +92,7 @@ func (h *healRoutine) run(ctx context.Context, objAPI ObjectLayer) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
select {
|
task.responseCh <- healResult{result: res, err: err}
|
||||||
case task.responseCh <- healResult{result: res, err: err}:
|
|
||||||
default:
|
|
||||||
}
|
|
||||||
|
|
||||||
case <-h.doneCh:
|
case <-h.doneCh:
|
||||||
return
|
return
|
||||||
@ -138,7 +104,7 @@ func (h *healRoutine) run(ctx context.Context, objAPI ObjectLayer) {
|
|||||||
|
|
||||||
func newHealRoutine() *healRoutine {
|
func newHealRoutine() *healRoutine {
|
||||||
return &healRoutine{
|
return &healRoutine{
|
||||||
tasks: make(chan healTask, 50000),
|
tasks: make(chan healTask),
|
||||||
doneCh: make(chan struct{}),
|
doneCh: make(chan struct{}),
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -337,10 +337,10 @@ func monitorLocalDisksAndHeal(ctx context.Context, z *erasureServerPools, bgSeq
|
|||||||
healDisks := globalBackgroundHealState.getHealLocalDiskEndpoints()
|
healDisks := globalBackgroundHealState.getHealLocalDiskEndpoints()
|
||||||
if len(healDisks) > 0 {
|
if len(healDisks) > 0 {
|
||||||
// Reformat disks
|
// Reformat disks
|
||||||
bgSeq.sourceCh <- healSource{bucket: SlashSeparator}
|
bgSeq.queueHealTask(healSource{bucket: SlashSeparator}, madmin.HealItemMetadata)
|
||||||
|
|
||||||
// Ensure that reformatting disks is finished
|
// Ensure that reformatting disks is finished
|
||||||
bgSeq.sourceCh <- healSource{bucket: nopHeal}
|
bgSeq.queueHealTask(healSource{bucket: nopHeal}, madmin.HealItemMetadata)
|
||||||
|
|
||||||
logger.Info(fmt.Sprintf("Found drives to heal %d, proceeding to heal content...",
|
logger.Info(fmt.Sprintf("Found drives to heal %d, proceeding to heal content...",
|
||||||
len(healDisks)))
|
len(healDisks)))
|
||||||
|
@ -579,9 +579,7 @@ func applyDynamicConfig(ctx context.Context, objAPI ObjectLayer, s config.Config
|
|||||||
globalCompressConfig = cmpCfg
|
globalCompressConfig = cmpCfg
|
||||||
globalCompressConfigMu.Unlock()
|
globalCompressConfigMu.Unlock()
|
||||||
|
|
||||||
globalHealConfigMu.Lock()
|
globalHealConfig.Update(healCfg)
|
||||||
globalHealConfig = healCfg
|
|
||||||
globalHealConfigMu.Unlock()
|
|
||||||
|
|
||||||
// update dynamic scanner values.
|
// update dynamic scanner values.
|
||||||
scannerCycle.Update(scannerCfg.Cycle)
|
scannerCycle.Update(scannerCfg.Cycle)
|
||||||
|
@ -58,7 +58,6 @@ const (
|
|||||||
|
|
||||||
var (
|
var (
|
||||||
globalHealConfig heal.Config
|
globalHealConfig heal.Config
|
||||||
globalHealConfigMu sync.Mutex
|
|
||||||
|
|
||||||
dataScannerLeaderLockTimeout = newDynamicTimeout(30*time.Second, 10*time.Second)
|
dataScannerLeaderLockTimeout = newDynamicTimeout(30*time.Second, 10*time.Second)
|
||||||
// Sleeper values are updated when config is loaded.
|
// Sleeper values are updated when config is loaded.
|
||||||
@ -638,6 +637,13 @@ func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, int
|
|||||||
console.Debugf(scannerLogPrefix+" checking disappeared folder: %v/%v\n", bucket, prefix)
|
console.Debugf(scannerLogPrefix+" checking disappeared folder: %v/%v\n", bucket, prefix)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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{})
|
||||||
|
}
|
||||||
|
|
||||||
resolver.bucket = bucket
|
resolver.bucket = bucket
|
||||||
|
|
||||||
foundObjs := false
|
foundObjs := false
|
||||||
@ -856,27 +862,21 @@ func (i *scannerItem) transformMetaDir() {
|
|||||||
i.objectName = split[len(split)-1]
|
i.objectName = split[len(split)-1]
|
||||||
}
|
}
|
||||||
|
|
||||||
// actionMeta contains information used to apply actions.
|
|
||||||
type actionMeta struct {
|
|
||||||
oi ObjectInfo
|
|
||||||
bitRotScan bool // indicates if bitrot check was requested.
|
|
||||||
}
|
|
||||||
|
|
||||||
var applyActionsLogPrefix = color.Green("applyActions:")
|
var applyActionsLogPrefix = color.Green("applyActions:")
|
||||||
|
|
||||||
func (i *scannerItem) applyHealing(ctx context.Context, o ObjectLayer, meta actionMeta) (size int64) {
|
func (i *scannerItem) applyHealing(ctx context.Context, o ObjectLayer, oi ObjectInfo) (size int64) {
|
||||||
if i.debug {
|
if i.debug {
|
||||||
if meta.oi.VersionID != "" {
|
if oi.VersionID != "" {
|
||||||
console.Debugf(applyActionsLogPrefix+" heal checking: %v/%v v(%s)\n", i.bucket, i.objectPath(), meta.oi.VersionID)
|
console.Debugf(applyActionsLogPrefix+" heal checking: %v/%v v(%s)\n", i.bucket, i.objectPath(), oi.VersionID)
|
||||||
} else {
|
} else {
|
||||||
console.Debugf(applyActionsLogPrefix+" heal checking: %v/%v\n", i.bucket, i.objectPath())
|
console.Debugf(applyActionsLogPrefix+" heal checking: %v/%v\n", i.bucket, i.objectPath())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
healOpts := madmin.HealOpts{Remove: healDeleteDangling}
|
healOpts := madmin.HealOpts{
|
||||||
if meta.bitRotScan {
|
Remove: healDeleteDangling,
|
||||||
healOpts.ScanMode = madmin.HealDeepScan
|
ScanMode: globalHealConfig.ScanMode(),
|
||||||
}
|
}
|
||||||
res, err := o.HealObject(ctx, i.bucket, i.objectPath(), meta.oi.VersionID, healOpts)
|
res, err := o.HealObject(ctx, i.bucket, i.objectPath(), oi.VersionID, healOpts)
|
||||||
if isErrObjectNotFound(err) || isErrVersionNotFound(err) {
|
if isErrObjectNotFound(err) || isErrVersionNotFound(err) {
|
||||||
return 0
|
return 0
|
||||||
}
|
}
|
||||||
@ -887,8 +887,8 @@ func (i *scannerItem) applyHealing(ctx context.Context, o ObjectLayer, meta acti
|
|||||||
return res.ObjectSize
|
return res.ObjectSize
|
||||||
}
|
}
|
||||||
|
|
||||||
func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, meta actionMeta) (applied bool, size int64) {
|
func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, oi ObjectInfo) (applied bool, size int64) {
|
||||||
size, err := meta.oi.GetActualSize()
|
size, err := oi.GetActualSize()
|
||||||
if i.debug {
|
if i.debug {
|
||||||
logger.LogIf(ctx, err)
|
logger.LogIf(ctx, err)
|
||||||
}
|
}
|
||||||
@ -900,20 +900,20 @@ func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, meta ac
|
|||||||
return false, size
|
return false, size
|
||||||
}
|
}
|
||||||
|
|
||||||
versionID := meta.oi.VersionID
|
versionID := oi.VersionID
|
||||||
action := i.lifeCycle.ComputeAction(
|
action := i.lifeCycle.ComputeAction(
|
||||||
lifecycle.ObjectOpts{
|
lifecycle.ObjectOpts{
|
||||||
Name: i.objectPath(),
|
Name: i.objectPath(),
|
||||||
UserTags: meta.oi.UserTags,
|
UserTags: oi.UserTags,
|
||||||
ModTime: meta.oi.ModTime,
|
ModTime: oi.ModTime,
|
||||||
VersionID: meta.oi.VersionID,
|
VersionID: oi.VersionID,
|
||||||
DeleteMarker: meta.oi.DeleteMarker,
|
DeleteMarker: oi.DeleteMarker,
|
||||||
IsLatest: meta.oi.IsLatest,
|
IsLatest: oi.IsLatest,
|
||||||
NumVersions: meta.oi.NumVersions,
|
NumVersions: oi.NumVersions,
|
||||||
SuccessorModTime: meta.oi.SuccessorModTime,
|
SuccessorModTime: oi.SuccessorModTime,
|
||||||
RestoreOngoing: meta.oi.RestoreOngoing,
|
RestoreOngoing: oi.RestoreOngoing,
|
||||||
RestoreExpires: meta.oi.RestoreExpires,
|
RestoreExpires: oi.RestoreExpires,
|
||||||
TransitionStatus: meta.oi.TransitionedObject.Status,
|
TransitionStatus: oi.TransitionedObject.Status,
|
||||||
RemoteTiersImmediately: globalDebugRemoteTiersImmediately,
|
RemoteTiersImmediately: globalDebugRemoteTiersImmediately,
|
||||||
})
|
})
|
||||||
if i.debug {
|
if i.debug {
|
||||||
@ -972,8 +972,8 @@ func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, meta ac
|
|||||||
|
|
||||||
// applyTierObjSweep removes remote object pending deletion and the free-version
|
// applyTierObjSweep removes remote object pending deletion and the free-version
|
||||||
// tracking this information.
|
// tracking this information.
|
||||||
func (i *scannerItem) applyTierObjSweep(ctx context.Context, o ObjectLayer, meta actionMeta) {
|
func (i *scannerItem) applyTierObjSweep(ctx context.Context, o ObjectLayer, oi ObjectInfo) {
|
||||||
if !meta.oi.TransitionedObject.FreeVersion {
|
if !oi.TransitionedObject.FreeVersion {
|
||||||
// nothing to be done
|
// nothing to be done
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -986,18 +986,18 @@ func (i *scannerItem) applyTierObjSweep(ctx context.Context, o ObjectLayer, meta
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
// Remove the remote object
|
// Remove the remote object
|
||||||
err := deleteObjectFromRemoteTier(ctx, meta.oi.TransitionedObject.Name, meta.oi.TransitionedObject.VersionID, meta.oi.TransitionedObject.Tier)
|
err := deleteObjectFromRemoteTier(ctx, oi.TransitionedObject.Name, oi.TransitionedObject.VersionID, oi.TransitionedObject.Tier)
|
||||||
if ignoreNotFoundErr(err) != nil {
|
if ignoreNotFoundErr(err) != nil {
|
||||||
logger.LogIf(ctx, err)
|
logger.LogIf(ctx, err)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
// Remove this free version
|
// Remove this free version
|
||||||
_, err = o.DeleteObject(ctx, meta.oi.Bucket, meta.oi.Name, ObjectOptions{
|
_, err = o.DeleteObject(ctx, oi.Bucket, oi.Name, ObjectOptions{
|
||||||
VersionID: meta.oi.VersionID,
|
VersionID: oi.VersionID,
|
||||||
})
|
})
|
||||||
if err == nil {
|
if err == nil {
|
||||||
auditLogLifecycle(ctx, meta.oi, ILMFreeVersionDelete)
|
auditLogLifecycle(ctx, oi, ILMFreeVersionDelete)
|
||||||
}
|
}
|
||||||
if ignoreNotFoundErr(err) != nil {
|
if ignoreNotFoundErr(err) != nil {
|
||||||
logger.LogIf(ctx, err)
|
logger.LogIf(ctx, err)
|
||||||
@ -1009,19 +1009,19 @@ func (i *scannerItem) applyTierObjSweep(ctx context.Context, o ObjectLayer, meta
|
|||||||
// The resulting size on disk will always be returned.
|
// 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.
|
// 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.
|
// If no metadata is supplied, -1 is returned if no action is taken.
|
||||||
func (i *scannerItem) applyActions(ctx context.Context, o ObjectLayer, meta actionMeta, sizeS *sizeSummary) int64 {
|
func (i *scannerItem) applyActions(ctx context.Context, o ObjectLayer, oi ObjectInfo, sizeS *sizeSummary) int64 {
|
||||||
i.applyTierObjSweep(ctx, o, meta)
|
i.applyTierObjSweep(ctx, o, oi)
|
||||||
|
|
||||||
applied, size := i.applyLifecycle(ctx, o, meta)
|
applied, size := i.applyLifecycle(ctx, o, oi)
|
||||||
// For instance, an applied lifecycle means we remove/transitioned an object
|
// For instance, an applied lifecycle means we remove/transitioned an object
|
||||||
// from the current deployment, which means we don't have to call healing
|
// from the current deployment, which means we don't have to call healing
|
||||||
// routine even if we are asked to do via heal flag.
|
// routine even if we are asked to do via heal flag.
|
||||||
if !applied {
|
if !applied {
|
||||||
if i.heal {
|
if i.heal {
|
||||||
size = i.applyHealing(ctx, o, meta)
|
size = i.applyHealing(ctx, o, oi)
|
||||||
}
|
}
|
||||||
// replicate only if lifecycle rules are not applied.
|
// replicate only if lifecycle rules are not applied.
|
||||||
i.healReplication(ctx, o, meta.oi.Clone(), sizeS)
|
i.healReplication(ctx, o, oi.Clone(), sizeS)
|
||||||
}
|
}
|
||||||
return size
|
return size
|
||||||
}
|
}
|
||||||
|
@ -386,7 +386,7 @@ func (fs *FSObjects) scanBucket(ctx context.Context, bucket string, cache dataUs
|
|||||||
}
|
}
|
||||||
|
|
||||||
oi := fsMeta.ToObjectInfo(bucket, object, fi)
|
oi := fsMeta.ToObjectInfo(bucket, object, fi)
|
||||||
sz := item.applyActions(ctx, fs, actionMeta{oi: oi}, &sizeSummary{})
|
sz := item.applyActions(ctx, fs, oi, &sizeSummary{})
|
||||||
if sz >= 0 {
|
if sz >= 0 {
|
||||||
return sizeSummary{totalSize: sz, versions: 1}, nil
|
return sizeSummary{totalSize: sz, versions: 1}, nil
|
||||||
}
|
}
|
||||||
|
@ -41,24 +41,14 @@ func newBgHealSequence() *healSequence {
|
|||||||
reqInfo := &logger.ReqInfo{API: "BackgroundHeal"}
|
reqInfo := &logger.ReqInfo{API: "BackgroundHeal"}
|
||||||
ctx, cancelCtx := context.WithCancel(logger.SetReqInfo(GlobalContext, reqInfo))
|
ctx, cancelCtx := context.WithCancel(logger.SetReqInfo(GlobalContext, reqInfo))
|
||||||
|
|
||||||
globalHealConfigMu.Lock()
|
|
||||||
opts := globalHealConfig
|
|
||||||
globalHealConfigMu.Unlock()
|
|
||||||
|
|
||||||
scanMode := madmin.HealNormalScan
|
|
||||||
if opts.Bitrot {
|
|
||||||
scanMode = madmin.HealDeepScan
|
|
||||||
}
|
|
||||||
|
|
||||||
hs := madmin.HealOpts{
|
hs := madmin.HealOpts{
|
||||||
// Remove objects that do not have read-quorum
|
// Remove objects that do not have read-quorum
|
||||||
Remove: healDeleteDangling,
|
Remove: healDeleteDangling,
|
||||||
ScanMode: scanMode,
|
ScanMode: globalHealConfig.ScanMode(),
|
||||||
}
|
}
|
||||||
|
|
||||||
return &healSequence{
|
return &healSequence{
|
||||||
sourceCh: make(chan healSource, 50000),
|
respCh: make(chan healResult),
|
||||||
respCh: make(chan healResult, 50000),
|
|
||||||
startTime: UTCNow(),
|
startTime: UTCNow(),
|
||||||
clientToken: bgHealingUUID,
|
clientToken: bgHealingUUID,
|
||||||
// run-background heal with reserved bucket
|
// run-background heal with reserved bucket
|
||||||
@ -179,14 +169,7 @@ func (er *erasureObjects) healErasureSet(ctx context.Context, buckets []BucketIn
|
|||||||
Name: pathJoin(minioMetaBucket, minioConfigPrefix),
|
Name: pathJoin(minioMetaBucket, minioConfigPrefix),
|
||||||
})
|
})
|
||||||
|
|
||||||
globalHealConfigMu.Lock()
|
scanMode := globalHealConfig.ScanMode()
|
||||||
opts := globalHealConfig
|
|
||||||
globalHealConfigMu.Unlock()
|
|
||||||
|
|
||||||
scanMode := madmin.HealNormalScan
|
|
||||||
if opts.Bitrot {
|
|
||||||
scanMode = madmin.HealDeepScan
|
|
||||||
}
|
|
||||||
|
|
||||||
// Heal all buckets with all objects
|
// Heal all buckets with all objects
|
||||||
for _, bucket := range buckets {
|
for _, bucket := range buckets {
|
||||||
@ -332,11 +315,8 @@ func (er *erasureObjects) healErasureSet(ctx context.Context, buckets []BucketIn
|
|||||||
func healObject(bucket, object, versionID string, scan madmin.HealScanMode) {
|
func healObject(bucket, object, versionID string, scan madmin.HealScanMode) {
|
||||||
// Get background heal sequence to send elements to heal
|
// Get background heal sequence to send elements to heal
|
||||||
bgSeq, ok := globalBackgroundHealState.getHealSequenceByToken(bgHealingUUID)
|
bgSeq, ok := globalBackgroundHealState.getHealSequenceByToken(bgHealingUUID)
|
||||||
if !ok {
|
if ok {
|
||||||
return
|
bgSeq.queueHealTask(healSource{
|
||||||
}
|
|
||||||
select {
|
|
||||||
case bgSeq.sourceCh <- healSource{
|
|
||||||
bucket: bucket,
|
bucket: bucket,
|
||||||
object: object,
|
object: object,
|
||||||
versionID: versionID,
|
versionID: versionID,
|
||||||
@ -344,7 +324,6 @@ func healObject(bucket, object, versionID string, scan madmin.HealScanMode) {
|
|||||||
Remove: healDeleteDangling, // if found dangling purge it.
|
Remove: healDeleteDangling, // if found dangling purge it.
|
||||||
ScanMode: scan,
|
ScanMode: scan,
|
||||||
},
|
},
|
||||||
}:
|
}, madmin.HealItemObject)
|
||||||
default:
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -163,6 +163,10 @@ func (e *metaCacheEntry) fileInfo(bucket string) (*FileInfo, error) {
|
|||||||
}, nil
|
}, nil
|
||||||
}
|
}
|
||||||
if e.cached == nil {
|
if e.cached == nil {
|
||||||
|
if len(e.metadata) == 0 {
|
||||||
|
// only happens if the entry is not found.
|
||||||
|
return nil, errFileNotFound
|
||||||
|
}
|
||||||
fi, err := getFileInfo(e.metadata, bucket, e.name, "", false)
|
fi, err := getFileInfo(e.metadata, bucket, e.name, "", false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
@ -308,15 +312,21 @@ func (m metaCacheEntries) resolve(r *metadataResolutionParams) (selected *metaCa
|
|||||||
sort.Slice(r.candidates, func(i, j int) bool {
|
sort.Slice(r.candidates, func(i, j int) bool {
|
||||||
return r.candidates[i].n > r.candidates[j].n
|
return r.candidates[i].n > r.candidates[j].n
|
||||||
})
|
})
|
||||||
|
|
||||||
// Check if we have enough.
|
// Check if we have enough.
|
||||||
if r.candidates[0].n < r.objQuorum {
|
if r.candidates[0].n < r.objQuorum {
|
||||||
return nil, false
|
return nil, false
|
||||||
}
|
}
|
||||||
|
|
||||||
if r.candidates[0].n > r.candidates[1].n {
|
if r.candidates[0].n > r.candidates[1].n {
|
||||||
return r.candidates[0].e, true
|
ok := r.candidates[0].e != nil && r.candidates[0].e.name != ""
|
||||||
|
return r.candidates[0].e, ok
|
||||||
}
|
}
|
||||||
|
|
||||||
|
e := resolveEntries(r.candidates[0].e, r.candidates[1].e, r.bucket)
|
||||||
// Tie between two, resolve using modtime+versions.
|
// Tie between two, resolve using modtime+versions.
|
||||||
return resolveEntries(r.candidates[0].e, r.candidates[1].e, r.bucket), true
|
ok := e != nil && e.name != ""
|
||||||
|
return e, ok
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
11
cmd/mrf.go
11
cmd/mrf.go
@ -183,17 +183,8 @@ func (m *mrfState) healRoutine() {
|
|||||||
idler := time.NewTimer(mrfInfoResetInterval)
|
idler := time.NewTimer(mrfInfoResetInterval)
|
||||||
defer idler.Stop()
|
defer idler.Stop()
|
||||||
|
|
||||||
globalHealConfigMu.Lock()
|
|
||||||
opts := globalHealConfig
|
|
||||||
globalHealConfigMu.Unlock()
|
|
||||||
|
|
||||||
scanMode := madmin.HealNormalScan
|
|
||||||
if opts.Bitrot {
|
|
||||||
scanMode = madmin.HealDeepScan
|
|
||||||
}
|
|
||||||
|
|
||||||
var mrfHealingOpts = madmin.HealOpts{
|
var mrfHealingOpts = madmin.HealOpts{
|
||||||
ScanMode: scanMode,
|
ScanMode: globalHealConfig.ScanMode(),
|
||||||
Remove: healDeleteDangling,
|
Remove: healDeleteDangling,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -453,9 +453,6 @@ func (s *xlStorage) NSScanner(ctx context.Context, cache dataUsageCache, updates
|
|||||||
return cache, errServerNotInitialized
|
return cache, errServerNotInitialized
|
||||||
}
|
}
|
||||||
|
|
||||||
globalHealConfigMu.Lock()
|
|
||||||
healOpts := globalHealConfig
|
|
||||||
globalHealConfigMu.Unlock()
|
|
||||||
cache.Info.updates = updates
|
cache.Info.updates = updates
|
||||||
|
|
||||||
dataUsageInfo, err := scanDataFolder(ctx, s.diskPath, cache, func(item scannerItem) (sizeSummary, error) {
|
dataUsageInfo, err := scanDataFolder(ctx, s.diskPath, cache, func(item scannerItem) (sizeSummary, error) {
|
||||||
@ -488,10 +485,7 @@ func (s *xlStorage) NSScanner(ctx context.Context, cache dataUsageCache, updates
|
|||||||
sizeS := sizeSummary{}
|
sizeS := sizeSummary{}
|
||||||
for _, version := range fivs.Versions {
|
for _, version := range fivs.Versions {
|
||||||
oi := version.ToObjectInfo(item.bucket, item.objectPath())
|
oi := version.ToObjectInfo(item.bucket, item.objectPath())
|
||||||
sz := item.applyActions(ctx, objAPI, actionMeta{
|
sz := item.applyActions(ctx, objAPI, oi, &sizeS)
|
||||||
oi: oi,
|
|
||||||
bitRotScan: healOpts.Bitrot,
|
|
||||||
}, &sizeS)
|
|
||||||
if !oi.DeleteMarker && sz == oi.Size {
|
if !oi.DeleteMarker && sz == oi.Size {
|
||||||
sizeS.versions++
|
sizeS.versions++
|
||||||
}
|
}
|
||||||
|
@ -20,8 +20,10 @@ package heal
|
|||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"strconv"
|
"strconv"
|
||||||
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/minio/madmin-go"
|
||||||
"github.com/minio/minio/internal/config"
|
"github.com/minio/minio/internal/config"
|
||||||
"github.com/minio/pkg/env"
|
"github.com/minio/pkg/env"
|
||||||
)
|
)
|
||||||
@ -37,6 +39,8 @@ const (
|
|||||||
EnvIOCount = "MINIO_HEAL_MAX_IO"
|
EnvIOCount = "MINIO_HEAL_MAX_IO"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var configMutex sync.RWMutex
|
||||||
|
|
||||||
// Config represents the heal settings.
|
// Config represents the heal settings.
|
||||||
type Config struct {
|
type Config struct {
|
||||||
// Bitrot will perform bitrot scan on local disk when checking objects.
|
// Bitrot will perform bitrot scan on local disk when checking objects.
|
||||||
@ -46,6 +50,61 @@ type Config struct {
|
|||||||
IOCount int `json:"iocount"`
|
IOCount int `json:"iocount"`
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ScanMode returns configured scan mode
|
||||||
|
func (opts Config) ScanMode() madmin.HealScanMode {
|
||||||
|
configMutex.RLock()
|
||||||
|
defer configMutex.RUnlock()
|
||||||
|
if opts.Bitrot {
|
||||||
|
return madmin.HealDeepScan
|
||||||
|
}
|
||||||
|
return madmin.HealNormalScan
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wait waits for IOCount to go down or max sleep to elapse before returning.
|
||||||
|
// usually used in healing paths to wait for specified amount of time to
|
||||||
|
// throttle healing.
|
||||||
|
func (opts Config) Wait(currentIO func() int, systemIO func() int) {
|
||||||
|
configMutex.RLock()
|
||||||
|
maxIO, maxWait := opts.IOCount, opts.Sleep
|
||||||
|
configMutex.RUnlock()
|
||||||
|
|
||||||
|
// No need to wait run at full speed.
|
||||||
|
if maxIO <= 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
// At max 10 attempts to wait with 100 millisecond interval before proceeding
|
||||||
|
waitTick := 100 * time.Millisecond
|
||||||
|
|
||||||
|
tmpMaxWait := maxWait
|
||||||
|
|
||||||
|
if currentIO != nil {
|
||||||
|
for currentIO() >= maxIO+systemIO() {
|
||||||
|
if tmpMaxWait > 0 {
|
||||||
|
if tmpMaxWait < waitTick {
|
||||||
|
time.Sleep(tmpMaxWait)
|
||||||
|
} else {
|
||||||
|
time.Sleep(waitTick)
|
||||||
|
}
|
||||||
|
tmpMaxWait = tmpMaxWait - waitTick
|
||||||
|
}
|
||||||
|
if tmpMaxWait <= 0 {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Update updates opts with nopts
|
||||||
|
func (opts *Config) Update(nopts Config) {
|
||||||
|
configMutex.Lock()
|
||||||
|
defer configMutex.Unlock()
|
||||||
|
|
||||||
|
opts.Bitrot = nopts.Bitrot
|
||||||
|
opts.IOCount = nopts.IOCount
|
||||||
|
opts.Sleep = nopts.Sleep
|
||||||
|
}
|
||||||
|
|
||||||
var (
|
var (
|
||||||
// DefaultKVS - default KV config for heal settings
|
// DefaultKVS - default KV config for heal settings
|
||||||
DefaultKVS = config.KVS{
|
DefaultKVS = config.KVS{
|
||||||
|
Loading…
Reference in New Issue
Block a user