mirror of
https://github.com/minio/minio.git
synced 2024-12-24 22:25:54 -05:00
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:
parent
ea18e51f4d
commit
cfc9cfd84a
@ -226,7 +226,7 @@ func getClaimsFromToken(r *http.Request) (map[string]interface{}, error) {
|
||||
if err != nil {
|
||||
// Base64 decoding fails, we should log to indicate
|
||||
// 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
|
||||
}
|
||||
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)
|
||||
if err != nil {
|
||||
return nil, toAPIErrorCode(context.Background(), err)
|
||||
return nil, toAPIErrorCode(r.Context(), err)
|
||||
}
|
||||
return claims, ErrNone
|
||||
}
|
||||
@ -460,7 +460,7 @@ func (a authHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
a.handler.ServeHTTP(w, r)
|
||||
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
|
||||
|
@ -66,8 +66,7 @@ func waitForLowHTTPReq(tolerance int32) {
|
||||
}
|
||||
|
||||
// Wait for heal requests and process them
|
||||
func (h *healRoutine) run() {
|
||||
ctx := context.Background()
|
||||
func (h *healRoutine) run(ctx context.Context, objAPI ObjectLayer) {
|
||||
for {
|
||||
select {
|
||||
case task, ok := <-h.tasks:
|
||||
@ -83,18 +82,18 @@ func (h *healRoutine) run() {
|
||||
bucket, object := path2BucketObject(task.path)
|
||||
switch {
|
||||
case bucket == "" && object == "":
|
||||
res, err = bgHealDiskFormat(ctx, task.opts)
|
||||
res, err = healDiskFormat(ctx, objAPI, task.opts)
|
||||
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 != "":
|
||||
res, err = bgHealObject(ctx, bucket, object, task.opts)
|
||||
res, err = objAPI.HealObject(ctx, bucket, object, task.opts)
|
||||
}
|
||||
if task.responseCh != nil {
|
||||
task.responseCh <- healResult{result: res, err: err}
|
||||
}
|
||||
case <-h.doneCh:
|
||||
return
|
||||
case <-GlobalServiceDoneCh:
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
@ -108,22 +107,10 @@ func initHealRoutine() *healRoutine {
|
||||
|
||||
}
|
||||
|
||||
func startBackgroundHealing() {
|
||||
ctx := context.Background()
|
||||
|
||||
var objAPI ObjectLayer
|
||||
for {
|
||||
objAPI = newObjectLayerWithoutSafeModeFn()
|
||||
if objAPI == nil {
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
func startBackgroundHealing(ctx context.Context, objAPI ObjectLayer) {
|
||||
// Run the background healer
|
||||
globalBackgroundHealRoutine = initHealRoutine()
|
||||
go globalBackgroundHealRoutine.run()
|
||||
go globalBackgroundHealRoutine.run(ctx, objAPI)
|
||||
|
||||
// Launch the background healer sequence to track
|
||||
// background healing operations
|
||||
@ -133,20 +120,14 @@ func startBackgroundHealing() {
|
||||
globalBackgroundHealState.LaunchNewHealSequence(nh)
|
||||
}
|
||||
|
||||
func initBackgroundHealing() {
|
||||
go startBackgroundHealing()
|
||||
func initBackgroundHealing(ctx context.Context, objAPI ObjectLayer) {
|
||||
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.
|
||||
func bgHealDiskFormat(ctx context.Context, opts madmin.HealOpts) (madmin.HealResultItem, error) {
|
||||
// Get current object layer instance.
|
||||
objectAPI := newObjectLayerWithoutSafeModeFn()
|
||||
if objectAPI == nil {
|
||||
return madmin.HealResultItem{}, errServerNotInitialized
|
||||
}
|
||||
|
||||
res, err := objectAPI.HealFormat(ctx, opts.DryRun)
|
||||
func healDiskFormat(ctx context.Context, objAPI ObjectLayer, opts madmin.HealOpts) (madmin.HealResultItem, error) {
|
||||
res, err := objAPI.HealFormat(ctx, opts.DryRun)
|
||||
|
||||
// return any error, ignore error returned when disks have
|
||||
// already healed.
|
||||
@ -167,24 +148,3 @@ func bgHealDiskFormat(ctx context.Context, opts madmin.HealOpts) (madmin.HealRes
|
||||
|
||||
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)
|
||||
}
|
||||
|
@ -25,32 +25,19 @@ import (
|
||||
|
||||
const defaultMonitorNewDiskInterval = time.Minute * 10
|
||||
|
||||
func initLocalDisksAutoHeal() {
|
||||
go monitorLocalDisksAndHeal()
|
||||
func initLocalDisksAutoHeal(ctx context.Context, objAPI ObjectLayer) {
|
||||
go monitorLocalDisksAndHeal(ctx, objAPI)
|
||||
}
|
||||
|
||||
// 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 = newObjectLayerWithoutSafeModeFn()
|
||||
if objAPI == nil {
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
func monitorLocalDisksAndHeal(ctx context.Context, objAPI ObjectLayer) {
|
||||
z, ok := objAPI.(*xlZones)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
ctx := context.Background()
|
||||
|
||||
var bgSeq *healSequence
|
||||
var found bool
|
||||
|
||||
@ -64,8 +51,10 @@ func monitorLocalDisksAndHeal() {
|
||||
|
||||
// Perform automatic disk healing when a disk is replaced locally.
|
||||
for {
|
||||
time.Sleep(defaultMonitorNewDiskInterval)
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-time.After(defaultMonitorNewDiskInterval):
|
||||
// Attempt a heal as the server starts-up first.
|
||||
localDisksInZoneHeal := make([]Endpoints, len(z.zones))
|
||||
for i, ep := range globalEndpoints {
|
||||
@ -126,4 +115,5 @@ func monitorLocalDisksAndHeal() {
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -30,75 +30,29 @@ const (
|
||||
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
|
||||
// listing of all objects and applies any matching bucket lifecycle
|
||||
// rules.
|
||||
func initDailyLifecycle() {
|
||||
go startDailyLifecycle()
|
||||
func initDailyLifecycle(ctx context.Context, objAPI ObjectLayer) {
|
||||
go startDailyLifecycle(ctx, objAPI)
|
||||
}
|
||||
|
||||
func startDailyLifecycle() {
|
||||
var objAPI ObjectLayer
|
||||
var ctx = context.Background()
|
||||
|
||||
// Wait until the object API is ready
|
||||
func startDailyLifecycle(ctx context.Context, objAPI ObjectLayer) {
|
||||
for {
|
||||
objAPI = newObjectLayerWithoutSafeModeFn()
|
||||
if objAPI == nil {
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
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)
|
||||
}
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-time.NewTimer(bgLifecycleInterval).C:
|
||||
// Perform one lifecycle operation
|
||||
err := lifecycleRound(ctx, objAPI)
|
||||
switch err.(type) {
|
||||
// Unable to hold a lock means there is another
|
||||
// instance doing the lifecycle round round
|
||||
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:
|
||||
logger.LogIf(ctx, err)
|
||||
time.Sleep(time.Minute)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@ -107,13 +61,6 @@ func startDailyLifecycle() {
|
||||
var lifecycleLockTimeout = newDynamicTimeout(60*time.Second, time.Second)
|
||||
|
||||
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)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -49,50 +49,19 @@ const (
|
||||
)
|
||||
|
||||
// initDataUsageStats will start the crawler unless disabled.
|
||||
func initDataUsageStats() {
|
||||
func initDataUsageStats(ctx context.Context, objAPI ObjectLayer) {
|
||||
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) {
|
||||
// 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 {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
locker.Unlock()
|
||||
return
|
||||
// Wait before starting next cycle and wait on startup.
|
||||
case <-time.NewTimer(dataUsageStartDelay).C:
|
||||
// Wait before starting next cycle and wait on startup.
|
||||
results := make(chan DataUsageInfo, 1)
|
||||
go storeDataUsageInBackend(ctx, objAPI, results)
|
||||
err := objAPI.CrawlAndGetDataUsage(ctx, results)
|
||||
|
@ -229,7 +229,7 @@ func (c *diskCache) toClear() uint64 {
|
||||
}
|
||||
|
||||
// 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() {
|
||||
return
|
||||
}
|
||||
|
@ -689,17 +689,17 @@ func newServerCacheObjects(ctx context.Context, config cache.Config) (CacheObjec
|
||||
if migrateSw {
|
||||
go c.migrateCacheFromV1toV2(ctx)
|
||||
}
|
||||
go c.gc(ctx, GlobalServiceDoneCh)
|
||||
go c.gc(ctx)
|
||||
return c, nil
|
||||
}
|
||||
|
||||
func (c *cacheObjects) gc(ctx context.Context, doneCh <-chan struct{}) {
|
||||
func (c *cacheObjects) gc(ctx context.Context) {
|
||||
ticker := time.NewTicker(cacheGCInterval)
|
||||
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-doneCh:
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
if c.migrating {
|
||||
@ -714,7 +714,7 @@ func (c *cacheObjects) gc(ctx context.Context, doneCh <-chan struct{}) {
|
||||
go func(d *diskCache) {
|
||||
defer wg.Done()
|
||||
d.resetGCCounter()
|
||||
d.purge(ctx, doneCh)
|
||||
d.purge(ctx)
|
||||
}(dcache)
|
||||
}
|
||||
wg.Wait()
|
||||
|
@ -267,7 +267,7 @@ func StartGateway(ctx *cli.Context, gw Gateway) {
|
||||
if globalCacheConfig.Enabled {
|
||||
// initialize the new disk cache objects.
|
||||
var cacheAPI CacheObjectLayer
|
||||
cacheAPI, err = newServerCacheObjects(context.Background(), globalCacheConfig)
|
||||
cacheAPI, err = newServerCacheObjects(GlobalContext, globalCacheConfig)
|
||||
logger.FatalIf(err, "Unable to initialize disk caching")
|
||||
|
||||
globalObjLayerMutex.Lock()
|
||||
@ -277,7 +277,7 @@ func StartGateway(ctx *cli.Context, gw Gateway) {
|
||||
|
||||
// Populate existing buckets to the etcd backend
|
||||
if globalDNSConfig != nil {
|
||||
buckets, err := newObject.ListBuckets(context.Background())
|
||||
buckets, err := newObject.ListBuckets(GlobalContext)
|
||||
if err != nil {
|
||||
logger.Fatal(err, "Unable to list buckets")
|
||||
}
|
||||
|
@ -87,7 +87,7 @@ func setRequestHeaderSizeLimitHandler(h http.Handler) http.Handler {
|
||||
// of the user-defined metadata to 2 KB.
|
||||
func (h requestHeaderSizeLimitHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
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
|
||||
}
|
||||
h.Handler.ServeHTTP(w, r)
|
||||
@ -130,7 +130,7 @@ func filterReservedMetadata(h http.Handler) http.Handler {
|
||||
// would be treated as metadata.
|
||||
func (h reservedMetadataHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
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
|
||||
}
|
||||
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
|
||||
// header, for all requests where Date header is not
|
||||
// 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
|
||||
}
|
||||
// Verify if the request date header is shifted by less than globalMaxSkewTime parameter in the past
|
||||
// or in the future, reject request otherwise.
|
||||
curTime := UTCNow()
|
||||
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
|
||||
}
|
||||
}
|
||||
@ -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 != "" && objectName == "" {
|
||||
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
|
||||
}
|
||||
}
|
||||
// If bucketName and objectName are present check for its resource queries.
|
||||
if bucketName != "" && objectName != "" {
|
||||
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
|
||||
}
|
||||
}
|
||||
@ -589,20 +589,20 @@ func hasMultipleAuth(r *http.Request) bool {
|
||||
func (h requestValidityHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
// Check for bad components in 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
|
||||
}
|
||||
// Check for bad components in URL query values.
|
||||
for _, vv := range r.URL.Query() {
|
||||
for _, v := range vv {
|
||||
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
|
||||
}
|
||||
}
|
||||
}
|
||||
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
|
||||
}
|
||||
h.handler.ServeHTTP(w, r)
|
||||
@ -648,10 +648,10 @@ func (f bucketForwardingHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques
|
||||
sr, err := globalDNSConfig.Get(bucket)
|
||||
if err != nil {
|
||||
if err == dns.ErrNoEntriesFound {
|
||||
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrNoSuchBucket),
|
||||
writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrNoSuchBucket),
|
||||
r.URL, guessIsBrowserReq(r))
|
||||
} else {
|
||||
writeErrorResponse(context.Background(), w, toAPIError(context.Background(), err),
|
||||
writeErrorResponse(r.Context(), w, toAPIError(r.Context(), err),
|
||||
r.URL, guessIsBrowserReq(r))
|
||||
}
|
||||
return
|
||||
@ -697,9 +697,9 @@ func (f bucketForwardingHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques
|
||||
sr, err := globalDNSConfig.Get(bucket)
|
||||
if err != nil {
|
||||
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 {
|
||||
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
|
||||
}
|
||||
@ -772,7 +772,7 @@ type criticalErrorHandler struct{ handler http.Handler }
|
||||
func (h criticalErrorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
defer func() {
|
||||
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 {
|
||||
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 {
|
||||
writeErrorResponseHeadersOnly(w, errorCodes.ToAPIErr(ErrInsecureSSECustomerRequest))
|
||||
} else {
|
||||
writeErrorResponse(context.Background(), w, errorCodes.ToAPIErr(ErrInsecureSSECustomerRequest), r.URL, guessIsBrowserReq(r))
|
||||
writeErrorResponse(r.Context(), w, errorCodes.ToAPIErr(ErrInsecureSSECustomerRequest), r.URL, guessIsBrowserReq(r))
|
||||
}
|
||||
return
|
||||
}
|
||||
|
@ -18,7 +18,6 @@ package cmd
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -126,65 +125,36 @@ func durationToNextHealRound(lastHeal time.Time) time.Duration {
|
||||
}
|
||||
|
||||
// Healing leader will take the charge of healing all erasure sets
|
||||
func execLeaderTasks(z *xlZones) {
|
||||
ctx := context.Background()
|
||||
|
||||
// Hold a lock so only one server performs auto-healing
|
||||
leaderLock := z.NewNSLock(ctx, minioMetaBucket, "leader")
|
||||
func execLeaderTasks(ctx context.Context, z *xlZones) {
|
||||
lastScanTime := UTCNow() // So that we don't heal immediately, but after one month.
|
||||
for {
|
||||
err := leaderLock.GetLock(leaderLockTimeout)
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
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))
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-time.NewTimer(durationToNextHealRound(lastScanTime)).C:
|
||||
for _, zone := range z.zones {
|
||||
// Heal set by set
|
||||
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 {
|
||||
setLock.Unlock()
|
||||
logger.LogIf(ctx, err)
|
||||
continue
|
||||
}
|
||||
setLock.Unlock()
|
||||
}
|
||||
}
|
||||
lastScanTime = time.Now()
|
||||
lastScanTime = UTCNow()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func startGlobalHeal() {
|
||||
var objAPI ObjectLayer
|
||||
for {
|
||||
objAPI = newObjectLayerWithoutSafeModeFn()
|
||||
if objAPI == nil {
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
func startGlobalHeal(ctx context.Context, objAPI ObjectLayer) {
|
||||
zones, ok := objAPI.(*xlZones)
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
execLeaderTasks(zones)
|
||||
execLeaderTasks(ctx, zones)
|
||||
}
|
||||
|
||||
func initGlobalHeal() {
|
||||
go startGlobalHeal()
|
||||
func initGlobalHeal(ctx context.Context, objAPI ObjectLayer) {
|
||||
go startGlobalHeal(ctx, objAPI)
|
||||
}
|
||||
|
@ -293,9 +293,7 @@ func lockMaintenance(ctx context.Context, interval time.Duration) error {
|
||||
}
|
||||
|
||||
// Start lock maintenance from all lock servers.
|
||||
func startLockMaintenance() {
|
||||
var ctx = context.Background()
|
||||
|
||||
func startLockMaintenance(ctx context.Context) {
|
||||
// Wait until the object API is ready
|
||||
// no need to start the lock maintenance
|
||||
// if ObjectAPI is not initialized.
|
||||
@ -317,7 +315,7 @@ func startLockMaintenance() {
|
||||
for {
|
||||
// Verifies every minute for locks held more than 2 minutes.
|
||||
select {
|
||||
case <-GlobalServiceDoneCh:
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
// 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)
|
||||
}
|
||||
|
@ -267,24 +267,6 @@ func (sys *NotificationSys) BackgroundHealStatus() []madmin.BgHealState {
|
||||
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.
|
||||
func (sys *NotificationSys) StartProfiling(profiler string) []NotificationPeerErr {
|
||||
ng := WithNPeers(len(sys.peerClients))
|
||||
|
@ -614,32 +614,6 @@ func (client *peerRESTClient) BackgroundHealStatus() (madmin.BgHealState, error)
|
||||
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) {
|
||||
values := make(url.Values)
|
||||
values.Set(peerRESTTraceAll, strconv.FormatBool(trcAll))
|
||||
|
@ -34,7 +34,6 @@ const (
|
||||
peerRESTMethodServerUpdate = "/serverupdate"
|
||||
peerRESTMethodSignalService = "/signalservice"
|
||||
peerRESTMethodBackgroundHealStatus = "/backgroundhealstatus"
|
||||
peerRESTMethodBackgroundOpsStatus = "/backgroundopsstatus"
|
||||
peerRESTMethodGetLocks = "/getlocks"
|
||||
peerRESTMethodBucketPolicyRemove = "/removebucketpolicy"
|
||||
peerRESTMethodLoadUser = "/loaduser"
|
||||
|
@ -1129,22 +1129,6 @@ func (s *peerRESTServer) BackgroundHealStatusHandler(w http.ResponseWriter, r *h
|
||||
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
|
||||
func (s *peerRESTServer) ConsoleLogHandler(w http.ResponseWriter, r *http.Request) {
|
||||
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 + 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 + peerRESTMethodBackgroundOpsStatus).HandlerFunc(server.BackgroundOpsStatusHandler)
|
||||
|
||||
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 + peerRESTMethodBackgroundHealStatus).HandlerFunc(server.BackgroundHealStatusHandler)
|
||||
|
@ -672,11 +672,15 @@ func (s *posix) Walk(volume, dirPath, marker string, recursive bool, leafFile st
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ch = make(chan FileInfo)
|
||||
// buffer channel matches the S3 ListObjects implementation
|
||||
ch = make(chan FileInfo, maxObjectList)
|
||||
go func() {
|
||||
defer close(ch)
|
||||
listDir := func(volume, dirPath, dirEntry string) (emptyDir bool, entries []string) {
|
||||
// Dynamic time delay.
|
||||
t := UTCNow()
|
||||
entries, err := s.ListDir(volume, dirPath, -1, leafFile)
|
||||
sleepDuration(time.Since(t), 10.0)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
@ -701,7 +705,10 @@ func (s *posix) Walk(volume, dirPath, marker string, recursive bool, leafFile st
|
||||
Mode: os.ModeDir,
|
||||
}
|
||||
} else {
|
||||
// Dynamic time delay.
|
||||
t := UTCNow()
|
||||
buf, err := s.ReadAll(volume, pathJoin(walkResult.entry, leafFile))
|
||||
sleepDuration(time.Since(t), 10.0)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
|
@ -26,6 +26,7 @@ import (
|
||||
"os/signal"
|
||||
"strings"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/minio/cli"
|
||||
"github.com/minio/minio/cmd/config"
|
||||
@ -285,6 +286,27 @@ func initAllSubsystems(buckets []BucketInfo, newObject ObjectLayer) (err error)
|
||||
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.
|
||||
func serverMain(ctx *cli.Context) {
|
||||
if ctx.Args().First() == "help" || !endpointsPresent(ctx) {
|
||||
@ -401,12 +423,11 @@ func serverMain(ctx *cli.Context) {
|
||||
|
||||
// Enable healing to heal drives if possible
|
||||
if globalIsXL {
|
||||
initBackgroundHealing()
|
||||
initLocalDisksAutoHeal()
|
||||
initGlobalHeal()
|
||||
initBackgroundHealing(GlobalContext, newObject)
|
||||
initLocalDisksAutoHeal(GlobalContext, newObject)
|
||||
}
|
||||
|
||||
buckets, err := newObject.ListBuckets(context.Background())
|
||||
buckets, err := newObject.ListBuckets(GlobalContext)
|
||||
if err != nil {
|
||||
logger.Fatal(err, "Unable to list buckets")
|
||||
}
|
||||
@ -416,7 +437,7 @@ func serverMain(ctx *cli.Context) {
|
||||
if globalCacheConfig.Enabled {
|
||||
// initialize the new disk cache objects.
|
||||
var cacheAPI CacheObjectLayer
|
||||
cacheAPI, err = newServerCacheObjects(context.Background(), globalCacheConfig)
|
||||
cacheAPI, err = newServerCacheObjects(GlobalContext, globalCacheConfig)
|
||||
logger.FatalIf(err, "Unable to initialize disk caching")
|
||||
|
||||
globalObjLayerMutex.Lock()
|
||||
@ -429,8 +450,7 @@ func serverMain(ctx *cli.Context) {
|
||||
initFederatorBackend(buckets, newObject)
|
||||
}
|
||||
|
||||
initDataUsageStats()
|
||||
initDailyLifecycle()
|
||||
go startBackgroundOps(GlobalContext, newObject)
|
||||
|
||||
// Disable safe mode operation, after all initialization is over.
|
||||
globalObjLayerMutex.Lock()
|
||||
|
@ -392,6 +392,11 @@ func (s *xlSets) StorageInfo(ctx context.Context, local bool) StorageInfo {
|
||||
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)
|
||||
defer closeStorageDisks(storageDisks)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user