deprecate disk tokens, instead rely on deadlines and active monitoring (#18947)

disk tokens usage is not necessary anymore with the implementation
of deadlines for storage calls and active monitoring of the drive
for I/O timeouts.

Functionality kicking off a bad drive is still supported, it's just that 
we do not have to serialize I/O in the manner tokens would do.
This commit is contained in:
Harshavardhana 2024-02-02 10:10:54 -08:00 committed by GitHub
parent ce0cb913bc
commit 99fde2ba85
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 13 additions and 162 deletions

View File

@ -33,7 +33,6 @@ import (
"github.com/minio/madmin-go/v3"
xioutil "github.com/minio/minio/internal/ioutil"
"github.com/minio/minio/internal/logger"
"github.com/minio/pkg/v2/env"
)
//go:generate stringer -type=storageMetric -trimprefix=storageMetric $GOFILE
@ -90,14 +89,6 @@ type xlStorageDiskIDCheck struct {
health *diskHealthTracker
healthCheck bool
// driveStartChecking is a threshold above which we will start to check
// the state of disks, generally this value is less than driveMaxConcurrent
driveStartChecking int
// driveMaxConcurrent represents maximum number of running concurrent
// operations for local and (incoming) remote disk operations.
driveMaxConcurrent int
metricsCache timedValue
diskCtx context.Context
diskCancel context.CancelFunc
@ -127,8 +118,6 @@ func (p *xlStorageDiskIDCheck) getMetrics() DiskMetrics {
}
// Do not need this value to be cached.
diskMetric.TotalTokens = uint32(p.driveMaxConcurrent)
diskMetric.TotalWaiting = uint32(p.health.waiting.Load())
diskMetric.TotalErrorsTimeout = p.totalErrsTimeout.Load()
diskMetric.TotalErrorsAvailability = p.totalErrsAvailability.Load()
@ -189,61 +178,11 @@ func (e *lockedLastMinuteLatency) total() AccElem {
return e.lastMinuteLatency.getTotal()
}
var maxConcurrentOnce sync.Once
func newXLStorageDiskIDCheck(storage *xlStorage, healthCheck bool) *xlStorageDiskIDCheck {
// driveMaxConcurrent represents maximum number of running concurrent
// operations for local and (incoming) remote disk operations.
//
// this value is a placeholder it is overridden via ENV for custom settings
// or this default value is used to pick the correct value HDDs v/s NVMe's
driveMaxConcurrent := -1
maxConcurrentOnce.Do(func() {
s := env.Get("_MINIO_DRIVE_MAX_CONCURRENT", "")
if s == "" {
s = env.Get("_MINIO_DISK_MAX_CONCURRENT", "")
}
if s != "" {
driveMaxConcurrent, _ = strconv.Atoi(s)
}
})
if driveMaxConcurrent <= 0 {
// nr_requests is for both READ and WRITE separately
// so we this 2x tokens on our end.
//
// https://www.kernel.org/doc/Documentation/block/queue-sysfs.txt
//
// nr_requests (RW)
// ----------------
// This controls how many requests may be allocated in the block layer for
// read or write requests. Note that the total allocated number may be twice
// this amount, since it applies only to reads or writes (not the accumulated
// sum).
driveMaxConcurrent = int(storage.nrRequests) * 2
if driveMaxConcurrent <= 0 {
driveMaxConcurrent = 1023 * 2 // Default value on Linux for most NVMe
}
if storage.rotational {
// use 80% of the available nr_requests on HDDs
driveMaxConcurrent = int(float64(storage.nrRequests)*0.8) * 2
if driveMaxConcurrent < 32 {
driveMaxConcurrent = 32
}
}
}
driveStartChecking := 16 + driveMaxConcurrent/8
if driveStartChecking > driveMaxConcurrent {
driveStartChecking = driveMaxConcurrent
}
xl := xlStorageDiskIDCheck{
storage: storage,
health: newDiskHealthTracker(driveMaxConcurrent),
healthCheck: healthCheck && globalDriveMonitoring,
driveMaxConcurrent: driveMaxConcurrent,
driveStartChecking: driveStartChecking,
storage: storage,
health: newDiskHealthTracker(),
healthCheck: healthCheck && globalDriveMonitoring,
}
xl.totalWrites.Store(xl.storage.getWriteAttribute())
@ -362,7 +301,6 @@ func (p *xlStorageDiskIDCheck) DiskInfo(ctx context.Context, opts DiskInfoOption
if opts.NoOp {
info.Metrics.TotalWrites = p.totalWrites.Load()
info.Metrics.TotalDeletes = p.totalDeletes.Load()
info.Metrics.TotalTokens = uint32(p.driveMaxConcurrent)
info.Metrics.TotalWaiting = uint32(p.health.waiting.Load())
info.Metrics.TotalErrorsTimeout = p.totalErrsTimeout.Load()
info.Metrics.TotalErrorsAvailability = p.totalErrsAvailability.Load()
@ -375,7 +313,6 @@ func (p *xlStorageDiskIDCheck) DiskInfo(ctx context.Context, opts DiskInfoOption
}
info.Metrics.TotalWrites = p.totalWrites.Load()
info.Metrics.TotalDeletes = p.totalDeletes.Load()
info.Metrics.TotalTokens = uint32(p.driveMaxConcurrent)
info.Metrics.TotalWaiting = uint32(p.health.waiting.Load())
info.Metrics.TotalErrorsTimeout = p.totalErrsTimeout.Load()
info.Metrics.TotalErrorsAvailability = p.totalErrsAvailability.Load()
@ -847,24 +784,17 @@ type diskHealthTracker struct {
// Atomic status of disk.
status atomic.Int32
// Atomic number of requests waiting for a token.
// Atomic number indicates if a disk is hung
waiting atomic.Int32
// Concurrency tokens.
tokens chan struct{}
}
// newDiskHealthTracker creates a new disk health tracker.
func newDiskHealthTracker(driveMaxConcurrent int) *diskHealthTracker {
func newDiskHealthTracker() *diskHealthTracker {
d := diskHealthTracker{
lastSuccess: time.Now().UnixNano(),
lastStarted: time.Now().UnixNano(),
tokens: make(chan struct{}, driveMaxConcurrent),
}
d.status.Store(diskHealthOK)
for i := 0; i < driveMaxConcurrent; i++ {
d.tokens <- struct{}{}
}
return &d
}
@ -906,9 +836,8 @@ func (p *xlStorageDiskIDCheck) TrackDiskHealth(ctx context.Context, s storageMet
return ctx, done, ctx.Err()
}
// Return early if disk is faulty already.
if err := p.checkHealth(ctx); err != nil {
return ctx, done, err
if p.health.status.Load() != diskHealthOK {
return ctx, done, errFaultyDisk
}
// Verify if the disk is not stale
@ -924,27 +853,19 @@ func (p *xlStorageDiskIDCheck) TrackDiskHealth(ctx context.Context, s storageMet
return ctx, done, nil
}
select {
case <-ctx.Done():
if contextCanceled(ctx) {
return ctx, done, ctx.Err()
case <-p.health.tokens:
// Fast path, got token.
default:
// We ran out of tokens, check health before blocking.
err = p.waitForToken(ctx)
if err != nil {
return ctx, done, err
}
}
// We only progress here if we got a token.
atomic.StoreInt64(&p.health.lastStarted, time.Now().UnixNano())
p.health.waiting.Add(1)
ctx = context.WithValue(ctx, healthDiskCtxKey{}, &healthDiskCtxValue{lastSuccess: &p.health.lastSuccess})
si := p.updateStorageMetrics(s, paths...)
var once sync.Once
return ctx, func(errp *error) {
p.health.waiting.Add(-1)
once.Do(func() {
p.health.tokens <- struct{}{}
if errp != nil {
err := *errp
if err == nil || errors.Is(err, io.EOF) {
@ -956,71 +877,6 @@ func (p *xlStorageDiskIDCheck) TrackDiskHealth(ctx context.Context, s storageMet
}, nil
}
// waitForToken will wait for a token, while periodically
// checking the disk status.
// If nil is returned a token was picked up.
func (p *xlStorageDiskIDCheck) waitForToken(ctx context.Context) (err error) {
p.health.waiting.Add(1)
defer p.health.waiting.Add(-1)
// Avoid stampeding herd...
ticker := time.NewTicker(5*time.Second + time.Duration(rand.Int63n(int64(5*time.Second))))
defer ticker.Stop()
for {
err = p.checkHealth(ctx)
if err != nil {
return err
}
select {
case <-ticker.C:
// Ticker expired, check health again.
case <-ctx.Done():
return ctx.Err()
case <-p.health.tokens:
return nil
}
}
}
// checkHealth should only be called when tokens have run out.
// This will check if disk should be taken offline.
func (p *xlStorageDiskIDCheck) checkHealth(ctx context.Context) (err error) {
if p.health.status.Load() == diskHealthFaulty {
return errFaultyDisk
}
// Check if there are tokens.
if p.driveMaxConcurrent-len(p.health.tokens) < p.driveStartChecking {
return nil
}
const maxTimeSinceLastSuccess = 30 * time.Second
const minTimeSinceLastOpStarted = 15 * time.Second
// To avoid stampeding herd (100s of simultaneous starting requests)
// there must be a delay between the last started request and now
// for the last lastSuccess to be useful.
t := time.Since(time.Unix(0, atomic.LoadInt64(&p.health.lastStarted)))
if t < minTimeSinceLastOpStarted {
return nil
}
// If also more than 15 seconds since last success, take disk offline.
t = time.Since(time.Unix(0, atomic.LoadInt64(&p.health.lastSuccess)))
if t > maxTimeSinceLastSuccess {
if p.health.status.CompareAndSwap(diskHealthOK, diskHealthFaulty) {
logger.LogAlwaysIf(ctx, fmt.Errorf("node(%s): taking drive %s offline, time since last response %v", globalLocalNodeName, p.storage.String(), t.Round(time.Millisecond)))
p.health.waiting.Add(1)
go p.monitorDiskStatus(0, mustGetUUID())
}
return errFaultyDisk
}
return nil
}
// Make sure we do not write O_DIRECT aligned I/O because WrIteAll() ends
// up using O_DIRECT codepath which internally utilizes p.health.tokens
// we need to avoid using incoming I/O tokens as part of the healthcheck
// monitoring I/O.
var toWrite = []byte{2048: 42}
// monitorDiskStatus should be called once when a drive has been marked offline.
@ -1034,11 +890,6 @@ func (p *xlStorageDiskIDCheck) monitorDiskStatus(spent time.Duration, fn string)
return
}
if len(p.health.tokens) == 0 {
// Queue is still full, no need to check.
continue
}
err := p.storage.WriteAll(context.Background(), minioMetaTmpBucket, fn, toWrite)
if err != nil {
continue

View File

@ -73,7 +73,7 @@ func LookupConfig(kvs config.KVS) (cfg *Config, err error) {
}
dur, _ := time.ParseDuration(d)
if dur < time.Second {
cfg.MaxTimeout = time.Minute * 2
cfg.MaxTimeout = 30 * time.Second
} else {
cfg.MaxTimeout = getMaxTimeout(dur)
}
@ -89,7 +89,7 @@ func getMaxTimeout(t time.Duration) time.Duration {
}
dur, _ := time.ParseDuration(d)
if dur < time.Second {
return time.Minute * 2
return 30 * time.Second
}
return dur
}