fix: various optimizations, idiomatic changes (#9179)

- acquire since leader lock for all background operations
  - healing, crawling and applying lifecycle policies.

- simplify lifecyle to avoid network calls, which was a
  bug in implementation - we should hold a leader and
  do everything from there, we have access to entire
  name space.

- make listing, walking not interfere by slowing itself
  down like the crawler.

- effectively use global context everywhere to ensure
  proper shutdown, in cache, lifecycle, healing

- don't read `format.json` for prometheus metrics in
  StorageInfo() call.
This commit is contained in:
Harshavardhana 2020-03-22 12:16:36 -07:00 committed by GitHub
parent ea18e51f4d
commit cfc9cfd84a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 172 additions and 369 deletions

View File

@ -226,7 +226,7 @@ func getClaimsFromToken(r *http.Request) (map[string]interface{}, error) {
if err != nil { if err != nil {
// Base64 decoding fails, we should log to indicate // Base64 decoding fails, we should log to indicate
// something is malforming the request sent by client. // something is malforming the request sent by client.
logger.LogIf(context.Background(), err, logger.Application) logger.LogIf(r.Context(), err, logger.Application)
return nil, errAuthentication return nil, errAuthentication
} }
claims.MapClaims[iampolicy.SessionPolicyName] = string(spBytes) claims.MapClaims[iampolicy.SessionPolicyName] = string(spBytes)
@ -246,7 +246,7 @@ func checkClaimsFromToken(r *http.Request, cred auth.Credentials) (map[string]in
} }
claims, err := getClaimsFromToken(r) claims, err := getClaimsFromToken(r)
if err != nil { if err != nil {
return nil, toAPIErrorCode(context.Background(), err) return nil, toAPIErrorCode(r.Context(), err)
} }
return claims, ErrNone return claims, ErrNone
} }
@ -460,7 +460,7 @@ func (a authHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
a.handler.ServeHTTP(w, r) a.handler.ServeHTTP(w, r)
return return
} }
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrSignatureVersionNotSupported), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrSignatureVersionNotSupported), r.URL, guessIsBrowserReq(r))
} }
// isPutActionAllowed - check if PUT operation is allowed on the resource, this // isPutActionAllowed - check if PUT operation is allowed on the resource, this

View File

@ -66,8 +66,7 @@ func waitForLowHTTPReq(tolerance int32) {
} }
// Wait for heal requests and process them // Wait for heal requests and process them
func (h *healRoutine) run() { func (h *healRoutine) run(ctx context.Context, objAPI ObjectLayer) {
ctx := context.Background()
for { for {
select { select {
case task, ok := <-h.tasks: case task, ok := <-h.tasks:
@ -83,18 +82,18 @@ func (h *healRoutine) run() {
bucket, object := path2BucketObject(task.path) bucket, object := path2BucketObject(task.path)
switch { switch {
case bucket == "" && object == "": case bucket == "" && object == "":
res, err = bgHealDiskFormat(ctx, task.opts) res, err = healDiskFormat(ctx, objAPI, task.opts)
case bucket != "" && object == "": case bucket != "" && object == "":
res, err = bgHealBucket(ctx, bucket, task.opts) res, err = objAPI.HealBucket(ctx, bucket, task.opts.DryRun, task.opts.Remove)
case bucket != "" && object != "": case bucket != "" && object != "":
res, err = bgHealObject(ctx, bucket, object, task.opts) res, err = objAPI.HealObject(ctx, bucket, object, task.opts)
} }
if task.responseCh != nil { if task.responseCh != nil {
task.responseCh <- healResult{result: res, err: err} task.responseCh <- healResult{result: res, err: err}
} }
case <-h.doneCh: case <-h.doneCh:
return return
case <-GlobalServiceDoneCh: case <-ctx.Done():
return return
} }
} }
@ -108,22 +107,10 @@ func initHealRoutine() *healRoutine {
} }
func startBackgroundHealing() { func startBackgroundHealing(ctx context.Context, objAPI ObjectLayer) {
ctx := context.Background()
var objAPI ObjectLayer
for {
objAPI = newObjectLayerWithoutSafeModeFn()
if objAPI == nil {
time.Sleep(time.Second)
continue
}
break
}
// Run the background healer // Run the background healer
globalBackgroundHealRoutine = initHealRoutine() globalBackgroundHealRoutine = initHealRoutine()
go globalBackgroundHealRoutine.run() go globalBackgroundHealRoutine.run(ctx, objAPI)
// Launch the background healer sequence to track // Launch the background healer sequence to track
// background healing operations // background healing operations
@ -133,20 +120,14 @@ func startBackgroundHealing() {
globalBackgroundHealState.LaunchNewHealSequence(nh) globalBackgroundHealState.LaunchNewHealSequence(nh)
} }
func initBackgroundHealing() { func initBackgroundHealing(ctx context.Context, objAPI ObjectLayer) {
go startBackgroundHealing() go startBackgroundHealing(ctx, objAPI)
} }
// bgHealDiskFormat - heals format.json, return value indicates if a // healDiskFormat - heals format.json, return value indicates if a
// failure error occurred. // failure error occurred.
func bgHealDiskFormat(ctx context.Context, opts madmin.HealOpts) (madmin.HealResultItem, error) { func healDiskFormat(ctx context.Context, objAPI ObjectLayer, opts madmin.HealOpts) (madmin.HealResultItem, error) {
// Get current object layer instance. res, err := objAPI.HealFormat(ctx, opts.DryRun)
objectAPI := newObjectLayerWithoutSafeModeFn()
if objectAPI == nil {
return madmin.HealResultItem{}, errServerNotInitialized
}
res, err := objectAPI.HealFormat(ctx, opts.DryRun)
// return any error, ignore error returned when disks have // return any error, ignore error returned when disks have
// already healed. // already healed.
@ -167,24 +148,3 @@ func bgHealDiskFormat(ctx context.Context, opts madmin.HealOpts) (madmin.HealRes
return res, nil return res, nil
} }
// bghealBucket - traverses and heals given bucket
func bgHealBucket(ctx context.Context, bucket string, opts madmin.HealOpts) (madmin.HealResultItem, error) {
// Get current object layer instance.
objectAPI := newObjectLayerWithoutSafeModeFn()
if objectAPI == nil {
return madmin.HealResultItem{}, errServerNotInitialized
}
return objectAPI.HealBucket(ctx, bucket, opts.DryRun, opts.Remove)
}
// bgHealObject - heal the given object and record result
func bgHealObject(ctx context.Context, bucket, object string, opts madmin.HealOpts) (madmin.HealResultItem, error) {
// Get current object layer instance.
objectAPI := newObjectLayerWithoutSafeModeFn()
if objectAPI == nil {
return madmin.HealResultItem{}, errServerNotInitialized
}
return objectAPI.HealObject(ctx, bucket, object, opts)
}

View File

@ -25,32 +25,19 @@ import (
const defaultMonitorNewDiskInterval = time.Minute * 10 const defaultMonitorNewDiskInterval = time.Minute * 10
func initLocalDisksAutoHeal() { func initLocalDisksAutoHeal(ctx context.Context, objAPI ObjectLayer) {
go monitorLocalDisksAndHeal() go monitorLocalDisksAndHeal(ctx, objAPI)
} }
// monitorLocalDisksAndHeal - ensures that detected new disks are healed // monitorLocalDisksAndHeal - ensures that detected new disks are healed
// 1. Only the concerned erasure set will be listed and 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 // 2. Only the node hosting the disk is responsible to perform the heal
func monitorLocalDisksAndHeal() { func monitorLocalDisksAndHeal(ctx context.Context, objAPI ObjectLayer) {
// Wait until the object layer is ready
var objAPI ObjectLayer
for {
objAPI = newObjectLayerWithoutSafeModeFn()
if objAPI == nil {
time.Sleep(time.Second)
continue
}
break
}
z, ok := objAPI.(*xlZones) z, ok := objAPI.(*xlZones)
if !ok { if !ok {
return return
} }
ctx := context.Background()
var bgSeq *healSequence var bgSeq *healSequence
var found bool var found bool
@ -64,8 +51,10 @@ func monitorLocalDisksAndHeal() {
// Perform automatic disk healing when a disk is replaced locally. // Perform automatic disk healing when a disk is replaced locally.
for { for {
time.Sleep(defaultMonitorNewDiskInterval) select {
case <-ctx.Done():
return
case <-time.After(defaultMonitorNewDiskInterval):
// Attempt a heal as the server starts-up first. // Attempt a heal as the server starts-up first.
localDisksInZoneHeal := make([]Endpoints, len(z.zones)) localDisksInZoneHeal := make([]Endpoints, len(z.zones))
for i, ep := range globalEndpoints { for i, ep := range globalEndpoints {
@ -127,3 +116,4 @@ func monitorLocalDisksAndHeal() {
} }
} }
} }
}

View File

@ -30,75 +30,29 @@ const (
bgLifecycleTick = time.Hour bgLifecycleTick = time.Hour
) )
type lifecycleOps struct {
LastActivity time.Time
}
// Register to the daily objects listing
var globalLifecycleOps = &lifecycleOps{}
func getLocalBgLifecycleOpsStatus() BgLifecycleOpsStatus {
return BgLifecycleOpsStatus{
LastActivity: globalLifecycleOps.LastActivity,
}
}
// initDailyLifecycle starts the routine that receives the daily // initDailyLifecycle starts the routine that receives the daily
// listing of all objects and applies any matching bucket lifecycle // listing of all objects and applies any matching bucket lifecycle
// rules. // rules.
func initDailyLifecycle() { func initDailyLifecycle(ctx context.Context, objAPI ObjectLayer) {
go startDailyLifecycle() go startDailyLifecycle(ctx, objAPI)
} }
func startDailyLifecycle() { func startDailyLifecycle(ctx context.Context, objAPI ObjectLayer) {
var objAPI ObjectLayer
var ctx = context.Background()
// Wait until the object API is ready
for { for {
objAPI = newObjectLayerWithoutSafeModeFn() select {
if objAPI == nil { case <-ctx.Done():
time.Sleep(time.Second) return
continue case <-time.NewTimer(bgLifecycleInterval).C:
}
break
}
// Calculate the time of the last lifecycle operation in all peers node of the cluster
computeLastLifecycleActivity := func(status []BgOpsStatus) time.Time {
var lastAct time.Time
for _, st := range status {
if st.LifecycleOps.LastActivity.After(lastAct) {
lastAct = st.LifecycleOps.LastActivity
}
}
return lastAct
}
for {
// Check if we should perform lifecycle ops based on the last lifecycle activity, sleep one hour otherwise
allLifecycleStatus := []BgOpsStatus{
{LifecycleOps: getLocalBgLifecycleOpsStatus()},
}
if globalIsDistXL {
allLifecycleStatus = append(allLifecycleStatus, globalNotificationSys.BackgroundOpsStatus()...)
}
lastAct := computeLastLifecycleActivity(allLifecycleStatus)
if !lastAct.IsZero() && time.Since(lastAct) < bgLifecycleInterval {
time.Sleep(bgLifecycleTick)
}
// Perform one lifecycle operation // Perform one lifecycle operation
err := lifecycleRound(ctx, objAPI) err := lifecycleRound(ctx, objAPI)
switch err.(type) { switch err.(type) {
// Unable to hold a lock means there is another
// instance doing the lifecycle round round
case OperationTimedOut: case OperationTimedOut:
time.Sleep(bgLifecycleTick) // Unable to hold a lock means there is another
// caller holding write lock, ignore and try next round.
continue
default: default:
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
time.Sleep(time.Minute) }
continue
} }
} }
@ -107,13 +61,6 @@ func startDailyLifecycle() {
var lifecycleLockTimeout = newDynamicTimeout(60*time.Second, time.Second) var lifecycleLockTimeout = newDynamicTimeout(60*time.Second, time.Second)
func lifecycleRound(ctx context.Context, objAPI ObjectLayer) error { func lifecycleRound(ctx context.Context, objAPI ObjectLayer) error {
// Lock to avoid concurrent lifecycle ops from other nodes
sweepLock := objAPI.NewNSLock(ctx, "system", "daily-lifecycle-ops")
if err := sweepLock.GetLock(lifecycleLockTimeout); err != nil {
return err
}
defer sweepLock.Unlock()
buckets, err := objAPI.ListBuckets(ctx) buckets, err := objAPI.ListBuckets(ctx)
if err != nil { if err != nil {
return err return err

View File

@ -49,50 +49,19 @@ const (
) )
// initDataUsageStats will start the crawler unless disabled. // initDataUsageStats will start the crawler unless disabled.
func initDataUsageStats() { func initDataUsageStats(ctx context.Context, objAPI ObjectLayer) {
if env.Get(envDataUsageCrawlConf, config.EnableOn) == config.EnableOn { if env.Get(envDataUsageCrawlConf, config.EnableOn) == config.EnableOn {
go runDataUsageInfoUpdateRoutine() go runDataUsageInfo(ctx, objAPI)
} }
} }
// runDataUsageInfoUpdateRoutine will contain the main crawler.
func runDataUsageInfoUpdateRoutine() {
// Wait until the object layer is ready
var objAPI ObjectLayer
for {
objAPI = newObjectLayerWithoutSafeModeFn()
if objAPI == nil {
time.Sleep(time.Second)
continue
}
break
}
runDataUsageInfo(GlobalContext, objAPI)
}
var dataUsageLockTimeout = lifecycleLockTimeout
func runDataUsageInfo(ctx context.Context, objAPI ObjectLayer) { func runDataUsageInfo(ctx context.Context, objAPI ObjectLayer) {
// Make sure only 1 crawler is running on the cluster.
locker := objAPI.NewNSLock(ctx, minioMetaBucket, "leader-data-usage-info")
for {
err := locker.GetLock(dataUsageLockTimeout)
if err != nil {
time.Sleep(5 * time.Minute)
continue
}
// Break without unlocking, this node will acquire
// data usage calculator role for its lifetime.
break
}
for { for {
select { select {
case <-ctx.Done(): case <-ctx.Done():
locker.Unlock()
return return
// Wait before starting next cycle and wait on startup.
case <-time.NewTimer(dataUsageStartDelay).C: case <-time.NewTimer(dataUsageStartDelay).C:
// Wait before starting next cycle and wait on startup.
results := make(chan DataUsageInfo, 1) results := make(chan DataUsageInfo, 1)
go storeDataUsageInBackend(ctx, objAPI, results) go storeDataUsageInBackend(ctx, objAPI, results)
err := objAPI.CrawlAndGetDataUsage(ctx, results) err := objAPI.CrawlAndGetDataUsage(ctx, results)

View File

@ -229,7 +229,7 @@ func (c *diskCache) toClear() uint64 {
} }
// Purge cache entries that were not accessed. // Purge cache entries that were not accessed.
func (c *diskCache) purge(ctx context.Context, doneCh <-chan struct{}) { func (c *diskCache) purge(ctx context.Context) {
if c.diskUsageLow() { if c.diskUsageLow() {
return return
} }

View File

@ -689,17 +689,17 @@ func newServerCacheObjects(ctx context.Context, config cache.Config) (CacheObjec
if migrateSw { if migrateSw {
go c.migrateCacheFromV1toV2(ctx) go c.migrateCacheFromV1toV2(ctx)
} }
go c.gc(ctx, GlobalServiceDoneCh) go c.gc(ctx)
return c, nil return c, nil
} }
func (c *cacheObjects) gc(ctx context.Context, doneCh <-chan struct{}) { func (c *cacheObjects) gc(ctx context.Context) {
ticker := time.NewTicker(cacheGCInterval) ticker := time.NewTicker(cacheGCInterval)
defer ticker.Stop() defer ticker.Stop()
for { for {
select { select {
case <-doneCh: case <-ctx.Done():
return return
case <-ticker.C: case <-ticker.C:
if c.migrating { if c.migrating {
@ -714,7 +714,7 @@ func (c *cacheObjects) gc(ctx context.Context, doneCh <-chan struct{}) {
go func(d *diskCache) { go func(d *diskCache) {
defer wg.Done() defer wg.Done()
d.resetGCCounter() d.resetGCCounter()
d.purge(ctx, doneCh) d.purge(ctx)
}(dcache) }(dcache)
} }
wg.Wait() wg.Wait()

View File

@ -267,7 +267,7 @@ func StartGateway(ctx *cli.Context, gw Gateway) {
if globalCacheConfig.Enabled { if globalCacheConfig.Enabled {
// initialize the new disk cache objects. // initialize the new disk cache objects.
var cacheAPI CacheObjectLayer var cacheAPI CacheObjectLayer
cacheAPI, err = newServerCacheObjects(context.Background(), globalCacheConfig) cacheAPI, err = newServerCacheObjects(GlobalContext, globalCacheConfig)
logger.FatalIf(err, "Unable to initialize disk caching") logger.FatalIf(err, "Unable to initialize disk caching")
globalObjLayerMutex.Lock() globalObjLayerMutex.Lock()
@ -277,7 +277,7 @@ func StartGateway(ctx *cli.Context, gw Gateway) {
// Populate existing buckets to the etcd backend // Populate existing buckets to the etcd backend
if globalDNSConfig != nil { if globalDNSConfig != nil {
buckets, err := newObject.ListBuckets(context.Background()) buckets, err := newObject.ListBuckets(GlobalContext)
if err != nil { if err != nil {
logger.Fatal(err, "Unable to list buckets") logger.Fatal(err, "Unable to list buckets")
} }

View File

@ -87,7 +87,7 @@ func setRequestHeaderSizeLimitHandler(h http.Handler) http.Handler {
// of the user-defined metadata to 2 KB. // of the user-defined metadata to 2 KB.
func (h requestHeaderSizeLimitHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { func (h requestHeaderSizeLimitHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
if isHTTPHeaderSizeTooLarge(r.Header) { if isHTTPHeaderSizeTooLarge(r.Header) {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrMetadataTooLarge), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrMetadataTooLarge), r.URL, guessIsBrowserReq(r))
return return
} }
h.Handler.ServeHTTP(w, r) h.Handler.ServeHTTP(w, r)
@ -130,7 +130,7 @@ func filterReservedMetadata(h http.Handler) http.Handler {
// would be treated as metadata. // would be treated as metadata.
func (h reservedMetadataHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { func (h reservedMetadataHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
if containsReservedMetadata(r.Header) { if containsReservedMetadata(r.Header) {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrUnsupportedMetadata), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrUnsupportedMetadata), r.URL, guessIsBrowserReq(r))
return return
} }
h.Handler.ServeHTTP(w, r) h.Handler.ServeHTTP(w, r)
@ -371,14 +371,14 @@ func (h timeValidityHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
// All our internal APIs are sensitive towards Date // All our internal APIs are sensitive towards Date
// header, for all requests where Date header is not // header, for all requests where Date header is not
// present we will reject such clients. // present we will reject such clients.
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(errCode), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(errCode), r.URL, guessIsBrowserReq(r))
return return
} }
// Verify if the request date header is shifted by less than globalMaxSkewTime parameter in the past // Verify if the request date header is shifted by less than globalMaxSkewTime parameter in the past
// or in the future, reject request otherwise. // or in the future, reject request otherwise.
curTime := UTCNow() curTime := UTCNow()
if curTime.Sub(amzDate) > globalMaxSkewTime || amzDate.Sub(curTime) > globalMaxSkewTime { if curTime.Sub(amzDate) > globalMaxSkewTime || amzDate.Sub(curTime) > globalMaxSkewTime {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrRequestTimeTooSkewed), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrRequestTimeTooSkewed), r.URL, guessIsBrowserReq(r))
return return
} }
} }
@ -509,14 +509,14 @@ func (h resourceHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
// If bucketName is present and not objectName check for bucket level resource queries. // If bucketName is present and not objectName check for bucket level resource queries.
if bucketName != "" && objectName == "" { if bucketName != "" && objectName == "" {
if ignoreNotImplementedBucketResources(r) { if ignoreNotImplementedBucketResources(r) {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrNotImplemented), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrNotImplemented), r.URL, guessIsBrowserReq(r))
return return
} }
} }
// If bucketName and objectName are present check for its resource queries. // If bucketName and objectName are present check for its resource queries.
if bucketName != "" && objectName != "" { if bucketName != "" && objectName != "" {
if ignoreNotImplementedObjectResources(r) { if ignoreNotImplementedObjectResources(r) {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrNotImplemented), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrNotImplemented), r.URL, guessIsBrowserReq(r))
return return
} }
} }
@ -589,20 +589,20 @@ func hasMultipleAuth(r *http.Request) bool {
func (h requestValidityHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { func (h requestValidityHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
// Check for bad components in URL path. // Check for bad components in URL path.
if hasBadPathComponent(r.URL.Path) { if hasBadPathComponent(r.URL.Path) {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrInvalidResourceName), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrInvalidResourceName), r.URL, guessIsBrowserReq(r))
return return
} }
// Check for bad components in URL query values. // Check for bad components in URL query values.
for _, vv := range r.URL.Query() { for _, vv := range r.URL.Query() {
for _, v := range vv { for _, v := range vv {
if hasBadPathComponent(v) { if hasBadPathComponent(v) {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrInvalidResourceName), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrInvalidResourceName), r.URL, guessIsBrowserReq(r))
return return
} }
} }
} }
if hasMultipleAuth(r) { if hasMultipleAuth(r) {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrInvalidRequest), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrInvalidRequest), r.URL, guessIsBrowserReq(r))
return return
} }
h.handler.ServeHTTP(w, r) h.handler.ServeHTTP(w, r)
@ -648,10 +648,10 @@ func (f bucketForwardingHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques
sr, err := globalDNSConfig.Get(bucket) sr, err := globalDNSConfig.Get(bucket)
if err != nil { if err != nil {
if err == dns.ErrNoEntriesFound { if err == dns.ErrNoEntriesFound {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrNoSuchBucket), writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrNoSuchBucket),
r.URL, guessIsBrowserReq(r)) r.URL, guessIsBrowserReq(r))
} else { } else {
writeErrorResponse(context.Background(), w, toAPIError(context.Background(), err), writeErrorResponse(r.Context(), w, toAPIError(r.Context(), err),
r.URL, guessIsBrowserReq(r)) r.URL, guessIsBrowserReq(r))
} }
return return
@ -697,9 +697,9 @@ func (f bucketForwardingHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques
sr, err := globalDNSConfig.Get(bucket) sr, err := globalDNSConfig.Get(bucket)
if err != nil { if err != nil {
if err == dns.ErrNoEntriesFound { if err == dns.ErrNoEntriesFound {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrNoSuchBucket), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrNoSuchBucket), r.URL, guessIsBrowserReq(r))
} else { } else {
writeErrorResponse(context.Background(), w, toAPIError(context.Background(), err), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, toAPIError(r.Context(), err), r.URL, guessIsBrowserReq(r))
} }
return return
} }
@ -772,7 +772,7 @@ type criticalErrorHandler struct{ handler http.Handler }
func (h criticalErrorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { func (h criticalErrorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
defer func() { defer func() {
if err := recover(); err == logger.ErrCritical { // handle if err := recover(); err == logger.ErrCritical { // handle
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrInternalError), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrInternalError), r.URL, guessIsBrowserReq(r))
} else if err != nil { } else if err != nil {
panic(err) // forward other panic calls panic(err) // forward other panic calls
} }
@ -791,7 +791,7 @@ func (h sseTLSHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
if r.Method == http.MethodHead { if r.Method == http.MethodHead {
writeErrorResponseHeadersOnly(w, errorCodes.ToAPIErr(ErrInsecureSSECustomerRequest)) writeErrorResponseHeadersOnly(w, errorCodes.ToAPIErr(ErrInsecureSSECustomerRequest))
} else { } else {
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrInsecureSSECustomerRequest), r.URL, guessIsBrowserReq(r)) writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrInsecureSSECustomerRequest), r.URL, guessIsBrowserReq(r))
} }
return return
} }

View File

@ -18,7 +18,6 @@ package cmd
import ( import (
"context" "context"
"fmt"
"sync" "sync"
"time" "time"
@ -126,65 +125,36 @@ func durationToNextHealRound(lastHeal time.Time) time.Duration {
} }
// Healing leader will take the charge of healing all erasure sets // Healing leader will take the charge of healing all erasure sets
func execLeaderTasks(z *xlZones) { func execLeaderTasks(ctx context.Context, z *xlZones) {
ctx := context.Background() lastScanTime := UTCNow() // So that we don't heal immediately, but after one month.
// Hold a lock so only one server performs auto-healing
leaderLock := z.NewNSLock(ctx, minioMetaBucket, "leader")
for { for {
err := leaderLock.GetLock(leaderLockTimeout) select {
if err == nil { case <-ctx.Done():
break return
} case <-time.NewTimer(durationToNextHealRound(lastScanTime)).C:
time.Sleep(leaderTick)
}
// Hold a lock for healing the erasure set
zeroDuration := time.Millisecond
zeroDynamicTimeout := newDynamicTimeout(zeroDuration, zeroDuration)
lastScanTime := time.Now() // So that we don't heal immediately, but after one month.
for {
time.Sleep(durationToNextHealRound(lastScanTime))
for _, zone := range z.zones { for _, zone := range z.zones {
// Heal set by set // Heal set by set
for i, set := range zone.sets { for i, set := range zone.sets {
setLock := z.zones[0].NewNSLock(ctx, "system", fmt.Sprintf("erasure-set-heal-%d", i))
if err := setLock.GetLock(zeroDynamicTimeout); err != nil {
logger.LogIf(ctx, err)
continue
}
if err := healErasureSet(ctx, i, set); err != nil { if err := healErasureSet(ctx, i, set); err != nil {
setLock.Unlock()
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
continue continue
} }
setLock.Unlock()
} }
} }
lastScanTime = time.Now() lastScanTime = UTCNow()
}
} }
} }
func startGlobalHeal() { func startGlobalHeal(ctx context.Context, objAPI ObjectLayer) {
var objAPI ObjectLayer
for {
objAPI = newObjectLayerWithoutSafeModeFn()
if objAPI == nil {
time.Sleep(time.Second)
continue
}
break
}
zones, ok := objAPI.(*xlZones) zones, ok := objAPI.(*xlZones)
if !ok { if !ok {
return return
} }
execLeaderTasks(zones) execLeaderTasks(ctx, zones)
} }
func initGlobalHeal() { func initGlobalHeal(ctx context.Context, objAPI ObjectLayer) {
go startGlobalHeal() go startGlobalHeal(ctx, objAPI)
} }

View File

@ -293,9 +293,7 @@ func lockMaintenance(ctx context.Context, interval time.Duration) error {
} }
// Start lock maintenance from all lock servers. // Start lock maintenance from all lock servers.
func startLockMaintenance() { func startLockMaintenance(ctx context.Context) {
var ctx = context.Background()
// Wait until the object API is ready // Wait until the object API is ready
// no need to start the lock maintenance // no need to start the lock maintenance
// if ObjectAPI is not initialized. // if ObjectAPI is not initialized.
@ -317,7 +315,7 @@ func startLockMaintenance() {
for { for {
// Verifies every minute for locks held more than 2 minutes. // Verifies every minute for locks held more than 2 minutes.
select { select {
case <-GlobalServiceDoneCh: case <-ctx.Done():
return return
case <-ticker.C: case <-ticker.C:
// Start with random sleep time, so as to avoid // Start with random sleep time, so as to avoid
@ -357,5 +355,5 @@ func registerLockRESTHandlers(router *mux.Router, endpointZones EndpointZones) {
} }
} }
go startLockMaintenance() go startLockMaintenance(GlobalContext)
} }

View File

@ -267,24 +267,6 @@ func (sys *NotificationSys) BackgroundHealStatus() []madmin.BgHealState {
return states return states
} }
// BackgroundOpsStatus - returns the status of all background operations of all peers
func (sys *NotificationSys) BackgroundOpsStatus() []BgOpsStatus {
states := make([]BgOpsStatus, len(sys.peerClients))
for idx, client := range sys.peerClients {
if client == nil {
continue
}
st, err := client.BackgroundOpsStatus()
if err != nil {
logger.LogIf(context.Background(), err)
} else {
states[idx] = st
}
}
return states
}
// StartProfiling - start profiling on remote peers, by initiating a remote RPC. // StartProfiling - start profiling on remote peers, by initiating a remote RPC.
func (sys *NotificationSys) StartProfiling(profiler string) []NotificationPeerErr { func (sys *NotificationSys) StartProfiling(profiler string) []NotificationPeerErr {
ng := WithNPeers(len(sys.peerClients)) ng := WithNPeers(len(sys.peerClients))

View File

@ -614,32 +614,6 @@ func (client *peerRESTClient) BackgroundHealStatus() (madmin.BgHealState, error)
return state, err return state, err
} }
// BgLifecycleOpsStatus describes the status
// of the background lifecycle operations
type BgLifecycleOpsStatus struct {
LastActivity time.Time
}
// BgOpsStatus describes the status of all operations performed
// in background such as auto-healing and lifecycle.
// Notice: We need to increase peer REST API version when adding
// new fields to this struct.
type BgOpsStatus struct {
LifecycleOps BgLifecycleOpsStatus
}
func (client *peerRESTClient) BackgroundOpsStatus() (BgOpsStatus, error) {
respBody, err := client.call(peerRESTMethodBackgroundOpsStatus, nil, nil, -1)
if err != nil {
return BgOpsStatus{}, err
}
defer http.DrainBody(respBody)
state := BgOpsStatus{}
err = gob.NewDecoder(respBody).Decode(&state)
return state, err
}
func (client *peerRESTClient) doTrace(traceCh chan interface{}, doneCh chan struct{}, trcAll, trcErr bool) { func (client *peerRESTClient) doTrace(traceCh chan interface{}, doneCh chan struct{}, trcAll, trcErr bool) {
values := make(url.Values) values := make(url.Values)
values.Set(peerRESTTraceAll, strconv.FormatBool(trcAll)) values.Set(peerRESTTraceAll, strconv.FormatBool(trcAll))

View File

@ -34,7 +34,6 @@ const (
peerRESTMethodServerUpdate = "/serverupdate" peerRESTMethodServerUpdate = "/serverupdate"
peerRESTMethodSignalService = "/signalservice" peerRESTMethodSignalService = "/signalservice"
peerRESTMethodBackgroundHealStatus = "/backgroundhealstatus" peerRESTMethodBackgroundHealStatus = "/backgroundhealstatus"
peerRESTMethodBackgroundOpsStatus = "/backgroundopsstatus"
peerRESTMethodGetLocks = "/getlocks" peerRESTMethodGetLocks = "/getlocks"
peerRESTMethodBucketPolicyRemove = "/removebucketpolicy" peerRESTMethodBucketPolicyRemove = "/removebucketpolicy"
peerRESTMethodLoadUser = "/loaduser" peerRESTMethodLoadUser = "/loaduser"

View File

@ -1129,22 +1129,6 @@ func (s *peerRESTServer) BackgroundHealStatusHandler(w http.ResponseWriter, r *h
logger.LogIf(ctx, gob.NewEncoder(w).Encode(state)) logger.LogIf(ctx, gob.NewEncoder(w).Encode(state))
} }
func (s *peerRESTServer) BackgroundOpsStatusHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
s.writeErrorResponse(w, errors.New("invalid request"))
return
}
ctx := newContext(r, w, "BackgroundOpsStatus")
state := BgOpsStatus{
LifecycleOps: getLocalBgLifecycleOpsStatus(),
}
defer w.(http.Flusher).Flush()
logger.LogIf(ctx, gob.NewEncoder(w).Encode(state))
}
// ConsoleLogHandler sends console logs of this node back to peer rest client // ConsoleLogHandler sends console logs of this node back to peer rest client
func (s *peerRESTServer) ConsoleLogHandler(w http.ResponseWriter, r *http.Request) { func (s *peerRESTServer) ConsoleLogHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) { if !s.IsValid(w, r) {
@ -1230,8 +1214,6 @@ func registerPeerRESTHandlers(router *mux.Router) {
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBucketLifecycleRemove).HandlerFunc(httpTraceHdrs(server.RemoveBucketLifecycleHandler)).Queries(restQueries(peerRESTBucket)...) subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBucketLifecycleRemove).HandlerFunc(httpTraceHdrs(server.RemoveBucketLifecycleHandler)).Queries(restQueries(peerRESTBucket)...)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBucketEncryptionSet).HandlerFunc(httpTraceHdrs(server.SetBucketSSEConfigHandler)).Queries(restQueries(peerRESTBucket)...) subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBucketEncryptionSet).HandlerFunc(httpTraceHdrs(server.SetBucketSSEConfigHandler)).Queries(restQueries(peerRESTBucket)...)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBucketEncryptionRemove).HandlerFunc(httpTraceHdrs(server.RemoveBucketSSEConfigHandler)).Queries(restQueries(peerRESTBucket)...) subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBucketEncryptionRemove).HandlerFunc(httpTraceHdrs(server.RemoveBucketSSEConfigHandler)).Queries(restQueries(peerRESTBucket)...)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBackgroundOpsStatus).HandlerFunc(server.BackgroundOpsStatusHandler)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodTrace).HandlerFunc(server.TraceHandler) subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodTrace).HandlerFunc(server.TraceHandler)
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodListen).HandlerFunc(httpTraceHdrs(server.ListenHandler)) subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodListen).HandlerFunc(httpTraceHdrs(server.ListenHandler))
subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBackgroundHealStatus).HandlerFunc(server.BackgroundHealStatusHandler) subrouter.Methods(http.MethodPost).Path(peerRESTVersionPrefix + peerRESTMethodBackgroundHealStatus).HandlerFunc(server.BackgroundHealStatusHandler)

View File

@ -672,11 +672,15 @@ func (s *posix) Walk(volume, dirPath, marker string, recursive bool, leafFile st
return nil, err return nil, err
} }
ch = make(chan FileInfo) // buffer channel matches the S3 ListObjects implementation
ch = make(chan FileInfo, maxObjectList)
go func() { go func() {
defer close(ch) defer close(ch)
listDir := func(volume, dirPath, dirEntry string) (emptyDir bool, entries []string) { listDir := func(volume, dirPath, dirEntry string) (emptyDir bool, entries []string) {
// Dynamic time delay.
t := UTCNow()
entries, err := s.ListDir(volume, dirPath, -1, leafFile) entries, err := s.ListDir(volume, dirPath, -1, leafFile)
sleepDuration(time.Since(t), 10.0)
if err != nil { if err != nil {
return return
} }
@ -701,7 +705,10 @@ func (s *posix) Walk(volume, dirPath, marker string, recursive bool, leafFile st
Mode: os.ModeDir, Mode: os.ModeDir,
} }
} else { } else {
// Dynamic time delay.
t := UTCNow()
buf, err := s.ReadAll(volume, pathJoin(walkResult.entry, leafFile)) buf, err := s.ReadAll(volume, pathJoin(walkResult.entry, leafFile))
sleepDuration(time.Since(t), 10.0)
if err != nil { if err != nil {
continue continue
} }

View File

@ -26,6 +26,7 @@ import (
"os/signal" "os/signal"
"strings" "strings"
"syscall" "syscall"
"time"
"github.com/minio/cli" "github.com/minio/cli"
"github.com/minio/minio/cmd/config" "github.com/minio/minio/cmd/config"
@ -285,6 +286,27 @@ func initAllSubsystems(buckets []BucketInfo, newObject ObjectLayer) (err error)
return nil return nil
} }
func startBackgroundOps(ctx context.Context, objAPI ObjectLayer) {
// Make sure only 1 crawler is running on the cluster.
locker := objAPI.NewNSLock(ctx, minioMetaBucket, "leader")
for {
err := locker.GetLock(leaderLockTimeout)
if err != nil {
time.Sleep(leaderTick)
continue
}
break
// No unlock for "leader" lock.
}
if globalIsXL {
initGlobalHeal(ctx, objAPI)
}
initDataUsageStats(ctx, objAPI)
initDailyLifecycle(ctx, objAPI)
}
// serverMain handler called for 'minio server' command. // serverMain handler called for 'minio server' command.
func serverMain(ctx *cli.Context) { func serverMain(ctx *cli.Context) {
if ctx.Args().First() == "help" || !endpointsPresent(ctx) { if ctx.Args().First() == "help" || !endpointsPresent(ctx) {
@ -401,12 +423,11 @@ func serverMain(ctx *cli.Context) {
// Enable healing to heal drives if possible // Enable healing to heal drives if possible
if globalIsXL { if globalIsXL {
initBackgroundHealing() initBackgroundHealing(GlobalContext, newObject)
initLocalDisksAutoHeal() initLocalDisksAutoHeal(GlobalContext, newObject)
initGlobalHeal()
} }
buckets, err := newObject.ListBuckets(context.Background()) buckets, err := newObject.ListBuckets(GlobalContext)
if err != nil { if err != nil {
logger.Fatal(err, "Unable to list buckets") logger.Fatal(err, "Unable to list buckets")
} }
@ -416,7 +437,7 @@ func serverMain(ctx *cli.Context) {
if globalCacheConfig.Enabled { if globalCacheConfig.Enabled {
// initialize the new disk cache objects. // initialize the new disk cache objects.
var cacheAPI CacheObjectLayer var cacheAPI CacheObjectLayer
cacheAPI, err = newServerCacheObjects(context.Background(), globalCacheConfig) cacheAPI, err = newServerCacheObjects(GlobalContext, globalCacheConfig)
logger.FatalIf(err, "Unable to initialize disk caching") logger.FatalIf(err, "Unable to initialize disk caching")
globalObjLayerMutex.Lock() globalObjLayerMutex.Lock()
@ -429,8 +450,7 @@ func serverMain(ctx *cli.Context) {
initFederatorBackend(buckets, newObject) initFederatorBackend(buckets, newObject)
} }
initDataUsageStats() go startBackgroundOps(GlobalContext, newObject)
initDailyLifecycle()
// Disable safe mode operation, after all initialization is over. // Disable safe mode operation, after all initialization is over.
globalObjLayerMutex.Lock() globalObjLayerMutex.Lock()

View File

@ -392,6 +392,11 @@ func (s *xlSets) StorageInfo(ctx context.Context, local bool) StorageInfo {
storageInfo.Backend.Sets[i] = make([]madmin.DriveInfo, s.drivesPerSet) storageInfo.Backend.Sets[i] = make([]madmin.DriveInfo, s.drivesPerSet)
} }
if local {
// if local is true, we don't need to read format.json
return storageInfo
}
storageDisks, dErrs := initStorageDisksWithErrors(s.endpoints) storageDisks, dErrs := initStorageDisksWithErrors(s.endpoints)
defer closeStorageDisks(storageDisks) defer closeStorageDisks(storageDisks)