xl: Rewrite auto-healing and implement auto new-disk healer (#8114)

The new auto healing model selects one node always responsible
for auto-healing the whole cluster, erasure set by erasure set.
If that node dies, another node will be elected as a leading
operator to perform healing.

This code also adds a goroutine which checks each 10 minutes
if there are any new unformatted disks and performs its healing
in that case, only the erasure set which has the new disk will
be healed.
This commit is contained in:
Anis Elleuch 2019-10-28 18:27:49 +01:00 committed by kannappanr
parent 10b526ed86
commit a49d4a9cb2
7 changed files with 339 additions and 113 deletions

View file

@ -53,6 +53,10 @@ const (
// time-duration to keep heal sequence state after it
// completes.
keepHealSeqStateDuration = time.Minute * 10
// nopHeal is a no operating healing action to
// wait for the current healing operation to finish
nopHeal = ""
)
var (
@ -548,7 +552,7 @@ func (h *healSequence) queueHealTask(path string, healType madmin.HealItemType)
var respCh = make(chan healResult)
defer close(respCh)
// Send heal request
globalBackgroundHealing.queueHealTask(healTask{path: path, responseCh: respCh, opts: h.settings})
globalBackgroundHealRoutine.queueHealTask(healTask{path: path, responseCh: respCh, opts: h.settings})
// Wait for answer and push result to the client
res := <-respCh
if !h.reportProgress {
@ -587,6 +591,8 @@ func (h *healSequence) healItemsFromSourceCh() error {
var itemType madmin.HealItemType
switch {
case path == nopHeal:
continue
case path == SlashSeparator:
itemType = madmin.HealItemMetadata
case !strings.Contains(path, SlashSeparator):

View file

@ -83,7 +83,9 @@ func (h *healRoutine) run() {
case bucket != "" && object != "":
res, err = bgHealObject(ctx, bucket, object, task.opts)
}
task.responseCh <- healResult{result: res, err: err}
if task.responseCh != nil {
task.responseCh <- healResult{result: res, err: err}
}
case <-h.doneCh:
return
case <-GlobalServiceDoneCh:
@ -100,11 +102,33 @@ func initHealRoutine() *healRoutine {
}
func initBackgroundHealing() {
healBg := initHealRoutine()
go healBg.run()
func startBackgroundHealing() {
ctx := context.Background()
globalBackgroundHealing = healBg
var objAPI ObjectLayer
for {
objAPI = newObjectLayerFn()
if objAPI == nil {
time.Sleep(time.Second)
continue
}
break
}
// Run the background healer
globalBackgroundHealRoutine = initHealRoutine()
go globalBackgroundHealRoutine.run()
// Launch the background healer sequence to track
// background healing operations
info := objAPI.StorageInfo(ctx)
numDisks := info.Backend.OnlineDisks.Sum() + info.Backend.OfflineDisks.Sum()
nh := newBgHealSequence(numDisks)
globalBackgroundHealState.LaunchNewHealSequence(nh)
}
func initBackgroundHealing() {
go startBackgroundHealing()
}
// bgHealDiskFormat - heals format.json, return value indicates if a

View file

@ -0,0 +1,119 @@
/*
* MinIO Cloud Storage, (C) 2019 MinIO, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package cmd
import (
"context"
"time"
"github.com/minio/minio/cmd/logger"
)
const defaultMonitorNewDiskInterval = time.Minute * 10
func initLocalDisksAutoHeal() {
go monitorLocalDisksAndHeal()
}
// monitorLocalDisksAndHeal - ensures that detected new disks are healed
// 1. Only the concerned erasure set will be listed and healed
// 2. Only the node hosting the disk is responsible to perform the heal
func monitorLocalDisksAndHeal() {
// Wait until the object layer is ready
var objAPI ObjectLayer
for {
objAPI = newObjectLayerFn()
if objAPI == nil {
time.Sleep(time.Second)
continue
}
break
}
sets, ok := objAPI.(*xlSets)
if !ok {
return
}
ctx := context.Background()
var bgSeq *healSequence
var found bool
for {
bgSeq, found = globalBackgroundHealState.getHealSequenceByToken(bgHealingUUID)
if found {
break
}
time.Sleep(time.Second)
}
// Perform automatic disk healing when a new one is inserted
for {
time.Sleep(defaultMonitorNewDiskInterval)
localDisksToHeal := []Endpoint{}
for _, endpoint := range globalEndpoints {
if !endpoint.IsLocal {
continue
}
// Try to connect to the current endpoint
// and reformat if the current disk is not formatted
_, _, err := connectEndpoint(endpoint)
if err == errUnformattedDisk {
localDisksToHeal = append(localDisksToHeal, endpoint)
}
}
if len(localDisksToHeal) == 0 {
continue
}
// Reformat disks
bgSeq.sourceCh <- SlashSeparator
// Ensure that reformatting disks is finished
bgSeq.sourceCh <- nopHeal
// Compute the list of erasure set to heal
var erasureSetToHeal []int
for _, endpoint := range localDisksToHeal {
// Load the new format of this passed endpoint
_, format, err := connectEndpoint(endpoint)
if err != nil {
logger.LogIf(ctx, err)
continue
}
// Calculate the set index where the current endpoint belongs
setIndex, _, err := findDiskIndex(sets.format, format)
if err != nil {
logger.LogIf(ctx, err)
continue
}
erasureSetToHeal = append(erasureSetToHeal, setIndex)
}
// Heal all erasure sets that need
for _, setIndex := range erasureSetToHeal {
xlObj := sets.sets[setIndex]
err := healErasureSet(ctx, setIndex, xlObj)
if err != nil {
logger.LogIf(ctx, err)
}
}
}
}

View file

@ -1,101 +0,0 @@
/*
* MinIO Cloud Storage, (C) 2019 MinIO, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package cmd
import (
"context"
"sync"
"time"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/madmin"
)
const (
bgHealingUUID = "0000-0000-0000-0000"
)
// NewBgHealSequence creates a background healing sequence
// operation which crawls all objects and heal them.
func newBgHealSequence(numDisks int) *healSequence {
reqInfo := &logger.ReqInfo{API: "BackgroundHeal"}
ctx := logger.SetReqInfo(context.Background(), reqInfo)
hs := madmin.HealOpts{
// Remove objects that do not have read-quorum
Remove: true,
ScanMode: madmin.HealNormalScan,
}
return &healSequence{
sourceCh: make(chan string),
startTime: UTCNow(),
clientToken: bgHealingUUID,
settings: hs,
currentStatus: healSequenceStatus{
Summary: healNotStartedStatus,
HealSettings: hs,
NumDisks: numDisks,
updateLock: &sync.RWMutex{},
},
traverseAndHealDoneCh: make(chan error),
stopSignalCh: make(chan struct{}),
ctx: ctx,
reportProgress: false,
}
}
func getLocalBackgroundHealStatus() madmin.BgHealState {
backgroundSequence, ok := globalSweepHealState.getHealSequenceByToken(bgHealingUUID)
if !ok {
return madmin.BgHealState{}
}
return madmin.BgHealState{
ScannedItemsCount: backgroundSequence.scannedItemsCount,
LastHealActivity: backgroundSequence.lastHealActivity,
}
}
func initDailyHeal() {
go startDailyHeal()
}
func startDailyHeal() {
var objAPI ObjectLayer
var ctx = context.Background()
// Wait until the object API is ready
for {
objAPI = newObjectLayerFn()
if objAPI == nil {
time.Sleep(time.Second)
continue
}
break
}
// Find number of disks in the setup
info := objAPI.StorageInfo(ctx)
numDisks := info.Backend.OnlineDisks.Sum() + info.Backend.OfflineDisks.Sum()
nh := newBgHealSequence(numDisks)
globalSweepHealState.LaunchNewHealSequence(nh)
registerDailySweepListener(nh.sourceCh)
}

177
cmd/global-heal.go Normal file
View file

@ -0,0 +1,177 @@
/*
* MinIO Cloud Storage, (C) 2019 MinIO, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package cmd
import (
"context"
"fmt"
"sync"
"time"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/madmin"
)
const (
bgHealingUUID = "0000-0000-0000-0000"
leaderTick = time.Hour
healTick = time.Hour
healInterval = 30 * 24 * time.Hour
)
var leaderLockTimeout = newDynamicTimeout(time.Minute, time.Minute)
// NewBgHealSequence creates a background healing sequence
// operation which crawls all objects and heal them.
func newBgHealSequence(numDisks int) *healSequence {
reqInfo := &logger.ReqInfo{API: "BackgroundHeal"}
ctx := logger.SetReqInfo(context.Background(), reqInfo)
hs := madmin.HealOpts{
// Remove objects that do not have read-quorum
Remove: true,
ScanMode: madmin.HealNormalScan,
}
return &healSequence{
sourceCh: make(chan string),
startTime: UTCNow(),
clientToken: bgHealingUUID,
settings: hs,
currentStatus: healSequenceStatus{
Summary: healNotStartedStatus,
HealSettings: hs,
NumDisks: numDisks,
updateLock: &sync.RWMutex{},
},
traverseAndHealDoneCh: make(chan error),
stopSignalCh: make(chan struct{}),
ctx: ctx,
reportProgress: false,
}
}
func getLocalBackgroundHealStatus() madmin.BgHealState {
bgSeq, ok := globalBackgroundHealState.getHealSequenceByToken(bgHealingUUID)
if !ok {
return madmin.BgHealState{}
}
return madmin.BgHealState{
ScannedItemsCount: bgSeq.scannedItemsCount,
LastHealActivity: bgSeq.lastHealActivity,
}
}
// healErasureSet lists and heals all objects in a specific erasure set
func healErasureSet(ctx context.Context, setIndex int, xlObj *xlObjects) error {
// Hold a lock for healing the erasure set
zeroDuration := time.Millisecond
zeroDynamicTimeout := newDynamicTimeout(zeroDuration, zeroDuration)
erasureSetHealLock := globalNSMutex.NewNSLock(ctx, "system", fmt.Sprintf("erasure-set-heal-%d", setIndex))
if err := erasureSetHealLock.GetLock(zeroDynamicTimeout); err != nil {
return err
}
defer erasureSetHealLock.Unlock()
buckets, err := xlObj.ListBuckets(ctx)
if err != nil {
return err
}
// Get background heal sequence to send elements to heal
var bgSeq *healSequence
var ok bool
for {
bgSeq, ok = globalBackgroundHealState.getHealSequenceByToken(bgHealingUUID)
if ok {
break
}
time.Sleep(time.Second)
}
// Heal all buckets with all objects
for _, bucket := range buckets {
// Heal current bucket
bgSeq.sourceCh <- bucket.Name
// List all objects in the current bucket and heal them
listDir := listDirFactory(ctx, xlObj.getLoadBalancedDisks()...)
walkResultCh := startTreeWalk(ctx, bucket.Name, "", "", true, listDir, nil)
for walkEntry := range walkResultCh {
bgSeq.sourceCh <- pathJoin(bucket.Name, walkEntry.entry)
}
}
return nil
}
// Healing leader will take the charge of healing all erasure sets
func execLeaderTasks(sets *xlSets) {
ctx := context.Background()
// Hold a lock so only one server performs auto-healing
leaderLock := globalNSMutex.NewNSLock(ctx, minioMetaBucket, "leader")
for {
err := leaderLock.GetLock(leaderLockTimeout)
if err == nil {
break
}
time.Sleep(leaderTick)
}
lastScanTime := time.Now() // So that we don't heal immediately, but after one month.
for {
if time.Since(lastScanTime) < healInterval {
time.Sleep(healTick)
continue
}
// Heal set by set
for i, set := range sets.sets {
err := healErasureSet(ctx, i, set)
if err != nil {
logger.LogIf(ctx, err)
continue
}
}
lastScanTime = time.Now()
}
}
func startGlobalHeal() {
var objAPI ObjectLayer
for {
objAPI = newObjectLayerFn()
if objAPI == nil {
time.Sleep(time.Second)
continue
}
break
}
sets, ok := objAPI.(*xlSets)
if !ok {
return
}
execLeaderTasks(sets)
}
func initGlobalHeal() {
go startGlobalHeal()
}

View file

@ -244,10 +244,11 @@ var (
// GlobalGatewaySSE sse options
GlobalGatewaySSE gatewaySSE
globalAllHealState *allHealState
// The always present healing routine ready to heal objects
globalBackgroundHealing *healRoutine
globalAllHealState *allHealState
globalSweepHealState *allHealState
globalBackgroundHealRoutine *healRoutine
globalBackgroundHealState *allHealState
// Add new variable global values here.
)

View file

@ -262,7 +262,7 @@ func serverMain(ctx *cli.Context) {
if globalIsXL {
// Init global heal state
globalAllHealState = initHealState()
globalSweepHealState = initHealState()
globalBackgroundHealState = initHealState()
}
// Initialize globalConsoleSys system
@ -363,8 +363,8 @@ func serverMain(ctx *cli.Context) {
if globalIsXL {
initBackgroundHealing()
initDailyHeal()
initDailySweeper()
initLocalDisksAutoHeal()
initGlobalHeal()
}
globalObjLayerMutex.Lock()