mirror of
https://github.com/minio/minio.git
synced 2025-04-20 02:27:50 -04:00
[feat] Add configurable deadline for writers (#11822)
This PR adds deadlines per Write() calls, such that slow drives are timed-out appropriately and the overall responsiveness for Writes() is always up to a predefined threshold providing applications sustained latency even if one of the drives is slow to respond.
This commit is contained in:
parent
d46c3c07a8
commit
51a8619a79
@ -23,8 +23,11 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
"hash"
|
"hash"
|
||||||
"io"
|
"io"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
|
"github.com/minio/minio/pkg/env"
|
||||||
|
xioutil "github.com/minio/minio/pkg/ioutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
type errHashMismatch struct {
|
type errHashMismatch struct {
|
||||||
@ -37,7 +40,7 @@ func (err *errHashMismatch) Error() string {
|
|||||||
|
|
||||||
// Calculates bitrot in chunks and writes the hash into the stream.
|
// Calculates bitrot in chunks and writes the hash into the stream.
|
||||||
type streamingBitrotWriter struct {
|
type streamingBitrotWriter struct {
|
||||||
iow *io.PipeWriter
|
iow io.WriteCloser
|
||||||
h hash.Hash
|
h hash.Hash
|
||||||
shardSize int64
|
shardSize int64
|
||||||
canClose chan struct{} // Needed to avoid race explained in Close() call.
|
canClose chan struct{} // Needed to avoid race explained in Close() call.
|
||||||
@ -70,19 +73,28 @@ func (b *streamingBitrotWriter) Close() error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var (
|
||||||
|
ioDeadline, _ = time.ParseDuration(env.Get("MINIO_IO_DEADLINE", ""))
|
||||||
|
)
|
||||||
|
|
||||||
// Returns streaming bitrot writer implementation.
|
// Returns streaming bitrot writer implementation.
|
||||||
func newStreamingBitrotWriter(disk StorageAPI, volume, filePath string, length int64, algo BitrotAlgorithm, shardSize int64) io.WriteCloser {
|
func newStreamingBitrotWriter(disk StorageAPI, volume, filePath string, length int64, algo BitrotAlgorithm, shardSize int64, heal bool) io.Writer {
|
||||||
r, w := io.Pipe()
|
r, w := io.Pipe()
|
||||||
h := algo.New()
|
h := algo.New()
|
||||||
bw := &streamingBitrotWriter{w, h, shardSize, make(chan struct{})}
|
|
||||||
|
var wc io.WriteCloser = w
|
||||||
|
if ioDeadline > 0 && !heal {
|
||||||
|
wc = xioutil.NewDeadlineWriter(w, ioDeadline)
|
||||||
|
}
|
||||||
|
|
||||||
|
bw := &streamingBitrotWriter{wc, h, shardSize, make(chan struct{})}
|
||||||
go func() {
|
go func() {
|
||||||
totalFileSize := int64(-1) // For compressed objects length will be unknown (represented by length=-1)
|
totalFileSize := int64(-1) // For compressed objects length will be unknown (represented by length=-1)
|
||||||
if length != -1 {
|
if length != -1 {
|
||||||
bitrotSumsTotalSize := ceilFrac(length, shardSize) * int64(h.Size()) // Size used for storing bitrot checksums.
|
bitrotSumsTotalSize := ceilFrac(length, shardSize) * int64(h.Size()) // Size used for storing bitrot checksums.
|
||||||
totalFileSize = bitrotSumsTotalSize + length
|
totalFileSize = bitrotSumsTotalSize + length
|
||||||
}
|
}
|
||||||
err := disk.CreateFile(context.TODO(), volume, filePath, totalFileSize, r)
|
r.CloseWithError(disk.CreateFile(context.TODO(), volume, filePath, totalFileSize, r))
|
||||||
r.CloseWithError(err)
|
|
||||||
close(bw.canClose)
|
close(bw.canClose)
|
||||||
}()
|
}()
|
||||||
return bw
|
return bw
|
||||||
|
@ -96,9 +96,9 @@ func BitrotAlgorithmFromString(s string) (a BitrotAlgorithm) {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
func newBitrotWriter(disk StorageAPI, volume, filePath string, length int64, algo BitrotAlgorithm, shardSize int64) io.Writer {
|
func newBitrotWriter(disk StorageAPI, volume, filePath string, length int64, algo BitrotAlgorithm, shardSize int64, heal bool) io.Writer {
|
||||||
if algo == HighwayHash256S {
|
if algo == HighwayHash256S {
|
||||||
return newStreamingBitrotWriter(disk, volume, filePath, length, algo, shardSize)
|
return newStreamingBitrotWriter(disk, volume, filePath, length, algo, shardSize, heal)
|
||||||
}
|
}
|
||||||
return newWholeBitrotWriter(disk, volume, filePath, algo, shardSize)
|
return newWholeBitrotWriter(disk, volume, filePath, algo, shardSize)
|
||||||
}
|
}
|
||||||
|
@ -41,7 +41,7 @@ func testBitrotReaderWriterAlgo(t *testing.T, bitrotAlgo BitrotAlgorithm) {
|
|||||||
|
|
||||||
disk.MakeVol(context.Background(), volume)
|
disk.MakeVol(context.Background(), volume)
|
||||||
|
|
||||||
writer := newBitrotWriter(disk, volume, filePath, 35, bitrotAlgo, 10)
|
writer := newBitrotWriter(disk, volume, filePath, 35, bitrotAlgo, 10, false)
|
||||||
|
|
||||||
_, err = writer.Write([]byte("aaaaaaaaaa"))
|
_, err = writer.Write([]byte("aaaaaaaaaa"))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -108,7 +108,8 @@ func TestErasureDecode(t *testing.T) {
|
|||||||
buffer := make([]byte, test.blocksize, 2*test.blocksize)
|
buffer := make([]byte, test.blocksize, 2*test.blocksize)
|
||||||
writers := make([]io.Writer, len(disks))
|
writers := make([]io.Writer, len(disks))
|
||||||
for i, disk := range disks {
|
for i, disk := range disks {
|
||||||
writers[i] = newBitrotWriter(disk, "testbucket", "object", erasure.ShardFileSize(test.data), writeAlgorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, "testbucket", "object",
|
||||||
|
erasure.ShardFileSize(test.data), writeAlgorithm, erasure.ShardSize(), false)
|
||||||
}
|
}
|
||||||
n, err := erasure.Encode(context.Background(), bytes.NewReader(data[:]), writers, buffer, erasure.dataBlocks+1)
|
n, err := erasure.Encode(context.Background(), bytes.NewReader(data[:]), writers, buffer, erasure.dataBlocks+1)
|
||||||
closeBitrotWriters(writers)
|
closeBitrotWriters(writers)
|
||||||
@ -234,7 +235,8 @@ func TestErasureDecodeRandomOffsetLength(t *testing.T) {
|
|||||||
if disk == nil {
|
if disk == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
writers[i] = newBitrotWriter(disk, "testbucket", "object", erasure.ShardFileSize(length), DefaultBitrotAlgorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, "testbucket", "object",
|
||||||
|
erasure.ShardFileSize(length), DefaultBitrotAlgorithm, erasure.ShardSize(), false)
|
||||||
}
|
}
|
||||||
|
|
||||||
// 10000 iterations with random offsets and lengths.
|
// 10000 iterations with random offsets and lengths.
|
||||||
@ -304,7 +306,8 @@ func benchmarkErasureDecode(data, parity, dataDown, parityDown int, size int64,
|
|||||||
if disk == nil {
|
if disk == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
writers[i] = newBitrotWriter(disk, "testbucket", "object", erasure.ShardFileSize(size), DefaultBitrotAlgorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, "testbucket", "object",
|
||||||
|
erasure.ShardFileSize(size), DefaultBitrotAlgorithm, erasure.ShardSize(), false)
|
||||||
}
|
}
|
||||||
|
|
||||||
content := make([]byte, size)
|
content := make([]byte, size)
|
||||||
|
@ -108,7 +108,7 @@ func TestErasureEncode(t *testing.T) {
|
|||||||
if disk == OfflineDisk {
|
if disk == OfflineDisk {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
writers[i] = newBitrotWriter(disk, "testbucket", "object", erasure.ShardFileSize(int64(len(data[test.offset:]))), test.algorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, "testbucket", "object", erasure.ShardFileSize(int64(len(data[test.offset:]))), test.algorithm, erasure.ShardSize(), false)
|
||||||
}
|
}
|
||||||
n, err := erasure.Encode(context.Background(), bytes.NewReader(data[test.offset:]), writers, buffer, erasure.dataBlocks+1)
|
n, err := erasure.Encode(context.Background(), bytes.NewReader(data[test.offset:]), writers, buffer, erasure.dataBlocks+1)
|
||||||
closeBitrotWriters(writers)
|
closeBitrotWriters(writers)
|
||||||
@ -132,14 +132,14 @@ func TestErasureEncode(t *testing.T) {
|
|||||||
if disk == nil {
|
if disk == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
writers[i] = newBitrotWriter(disk, "testbucket", "object2", erasure.ShardFileSize(int64(len(data[test.offset:]))), test.algorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, "testbucket", "object2", erasure.ShardFileSize(int64(len(data[test.offset:]))), test.algorithm, erasure.ShardSize(), false)
|
||||||
}
|
}
|
||||||
for j := range disks[:test.offDisks] {
|
for j := range disks[:test.offDisks] {
|
||||||
switch w := writers[j].(type) {
|
switch w := writers[j].(type) {
|
||||||
case *wholeBitrotWriter:
|
case *wholeBitrotWriter:
|
||||||
w.disk = badDisk{nil}
|
w.disk = badDisk{nil}
|
||||||
case *streamingBitrotWriter:
|
case *streamingBitrotWriter:
|
||||||
w.iow.CloseWithError(errFaultyDisk)
|
w.iow.(*io.PipeWriter).CloseWithError(errFaultyDisk)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if test.offDisks > 0 {
|
if test.offDisks > 0 {
|
||||||
@ -196,7 +196,8 @@ func benchmarkErasureEncode(data, parity, dataDown, parityDown int, size int64,
|
|||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
disk.Delete(context.Background(), "testbucket", "object", false)
|
disk.Delete(context.Background(), "testbucket", "object", false)
|
||||||
writers[i] = newBitrotWriter(disk, "testbucket", "object", erasure.ShardFileSize(size), DefaultBitrotAlgorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, "testbucket", "object",
|
||||||
|
erasure.ShardFileSize(size), DefaultBitrotAlgorithm, erasure.ShardSize(), false)
|
||||||
}
|
}
|
||||||
_, err := erasure.Encode(context.Background(), bytes.NewReader(content), writers, buffer, erasure.dataBlocks+1)
|
_, err := erasure.Encode(context.Background(), bytes.NewReader(content), writers, buffer, erasure.dataBlocks+1)
|
||||||
closeBitrotWriters(writers)
|
closeBitrotWriters(writers)
|
||||||
|
@ -87,7 +87,8 @@ func TestErasureHeal(t *testing.T) {
|
|||||||
buffer := make([]byte, test.blocksize, 2*test.blocksize)
|
buffer := make([]byte, test.blocksize, 2*test.blocksize)
|
||||||
writers := make([]io.Writer, len(disks))
|
writers := make([]io.Writer, len(disks))
|
||||||
for i, disk := range disks {
|
for i, disk := range disks {
|
||||||
writers[i] = newBitrotWriter(disk, "testbucket", "testobject", erasure.ShardFileSize(test.size), test.algorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, "testbucket", "testobject",
|
||||||
|
erasure.ShardFileSize(test.size), test.algorithm, erasure.ShardSize(), true)
|
||||||
}
|
}
|
||||||
_, err = erasure.Encode(context.Background(), bytes.NewReader(data), writers, buffer, erasure.dataBlocks+1)
|
_, err = erasure.Encode(context.Background(), bytes.NewReader(data), writers, buffer, erasure.dataBlocks+1)
|
||||||
closeBitrotWriters(writers)
|
closeBitrotWriters(writers)
|
||||||
@ -130,7 +131,8 @@ func TestErasureHeal(t *testing.T) {
|
|||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
os.Remove(pathJoin(disk.String(), "testbucket", "testobject"))
|
os.Remove(pathJoin(disk.String(), "testbucket", "testobject"))
|
||||||
staleWriters[i] = newBitrotWriter(disk, "testbucket", "testobject", erasure.ShardFileSize(test.size), test.algorithm, erasure.ShardSize())
|
staleWriters[i] = newBitrotWriter(disk, "testbucket", "testobject",
|
||||||
|
erasure.ShardFileSize(test.size), test.algorithm, erasure.ShardSize(), true)
|
||||||
}
|
}
|
||||||
|
|
||||||
// test case setup is complete - now call Heal()
|
// test case setup is complete - now call Heal()
|
||||||
|
@ -422,7 +422,8 @@ func (er erasureObjects) healObject(ctx context.Context, bucket string, object s
|
|||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
partPath := pathJoin(tmpID, dataDir, fmt.Sprintf("part.%d", partNumber))
|
partPath := pathJoin(tmpID, dataDir, fmt.Sprintf("part.%d", partNumber))
|
||||||
writers[i] = newBitrotWriter(disk, minioMetaTmpBucket, partPath, tillOffset, DefaultBitrotAlgorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, minioMetaTmpBucket, partPath,
|
||||||
|
tillOffset, DefaultBitrotAlgorithm, erasure.ShardSize(), true)
|
||||||
}
|
}
|
||||||
err = erasure.Heal(ctx, readers, writers, partSize)
|
err = erasure.Heal(ctx, readers, writers, partSize)
|
||||||
closeBitrotReaders(readers)
|
closeBitrotReaders(readers)
|
||||||
|
@ -476,7 +476,8 @@ func (er erasureObjects) PutObjectPart(ctx context.Context, bucket, object, uplo
|
|||||||
if disk == nil {
|
if disk == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
writers[i] = newBitrotWriter(disk, minioMetaTmpBucket, tmpPartPath, erasure.ShardFileSize(data.Size()), DefaultBitrotAlgorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, minioMetaTmpBucket, tmpPartPath,
|
||||||
|
erasure.ShardFileSize(data.Size()), DefaultBitrotAlgorithm, erasure.ShardSize(), false)
|
||||||
}
|
}
|
||||||
|
|
||||||
n, err := erasure.Encode(ctx, data, writers, buffer, writeQuorum)
|
n, err := erasure.Encode(ctx, data, writers, buffer, writeQuorum)
|
||||||
|
@ -718,7 +718,8 @@ func (er erasureObjects) putObject(ctx context.Context, bucket string, object st
|
|||||||
if disk == nil {
|
if disk == nil {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
writers[i] = newBitrotWriter(disk, minioMetaTmpBucket, tempErasureObj, erasure.ShardFileSize(data.Size()), DefaultBitrotAlgorithm, erasure.ShardSize())
|
writers[i] = newBitrotWriter(disk, minioMetaTmpBucket, tempErasureObj,
|
||||||
|
erasure.ShardFileSize(data.Size()), DefaultBitrotAlgorithm, erasure.ShardSize(), false)
|
||||||
}
|
}
|
||||||
|
|
||||||
n, erasureErr := erasure.Encode(ctx, data, writers, buffer, writeQuorum)
|
n, erasureErr := erasure.Encode(ctx, data, writers, buffer, writeQuorum)
|
||||||
|
23
cmd/iam.go
23
cmd/iam.go
@ -577,15 +577,6 @@ func (sys *IAMSys) Init(ctx context.Context, objAPI ObjectLayer) {
|
|||||||
// Hold the lock for migration only.
|
// Hold the lock for migration only.
|
||||||
txnLk := objAPI.NewNSLock(minioMetaBucket, minioConfigPrefix+"/iam.lock")
|
txnLk := objAPI.NewNSLock(minioMetaBucket, minioConfigPrefix+"/iam.lock")
|
||||||
|
|
||||||
// Initializing IAM sub-system needs a retry mechanism for
|
|
||||||
// the following reasons:
|
|
||||||
// - Read quorum is lost just after the initialization
|
|
||||||
// of the object layer.
|
|
||||||
// - Write quorum not met when upgrading configuration
|
|
||||||
// version is needed, migration is needed etc.
|
|
||||||
rquorum := InsufficientReadQuorum{}
|
|
||||||
wquorum := InsufficientWriteQuorum{}
|
|
||||||
|
|
||||||
// allocate dynamic timeout once before the loop
|
// allocate dynamic timeout once before the loop
|
||||||
iamLockTimeout := newDynamicTimeout(5*time.Second, 3*time.Second)
|
iamLockTimeout := newDynamicTimeout(5*time.Second, 3*time.Second)
|
||||||
|
|
||||||
@ -620,12 +611,7 @@ func (sys *IAMSys) Init(ctx context.Context, objAPI ObjectLayer) {
|
|||||||
// Migrate IAM configuration, if necessary.
|
// Migrate IAM configuration, if necessary.
|
||||||
if err := sys.doIAMConfigMigration(ctx); err != nil {
|
if err := sys.doIAMConfigMigration(ctx); err != nil {
|
||||||
txnLk.Unlock()
|
txnLk.Unlock()
|
||||||
if errors.Is(err, errDiskNotFound) ||
|
if configRetriableErrors(err) {
|
||||||
errors.Is(err, errConfigNotFound) ||
|
|
||||||
errors.Is(err, context.DeadlineExceeded) ||
|
|
||||||
errors.As(err, &rquorum) ||
|
|
||||||
errors.As(err, &wquorum) ||
|
|
||||||
isErrBucketNotFound(err) {
|
|
||||||
logger.Info("Waiting for all MinIO IAM sub-system to be initialized.. possible cause (%v)", err)
|
logger.Info("Waiting for all MinIO IAM sub-system to be initialized.. possible cause (%v)", err)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
@ -641,12 +627,7 @@ func (sys *IAMSys) Init(ctx context.Context, objAPI ObjectLayer) {
|
|||||||
|
|
||||||
for {
|
for {
|
||||||
if err := sys.store.loadAll(ctx, sys); err != nil {
|
if err := sys.store.loadAll(ctx, sys); err != nil {
|
||||||
if errors.Is(err, errDiskNotFound) ||
|
if configRetriableErrors(err) {
|
||||||
errors.Is(err, errConfigNotFound) ||
|
|
||||||
errors.Is(err, context.DeadlineExceeded) ||
|
|
||||||
errors.As(err, &rquorum) ||
|
|
||||||
errors.As(err, &wquorum) ||
|
|
||||||
isErrBucketNotFound(err) {
|
|
||||||
logger.Info("Waiting for all MinIO IAM sub-system to be initialized.. possible cause (%v)", err)
|
logger.Info("Waiting for all MinIO IAM sub-system to be initialized.. possible cause (%v)", err)
|
||||||
time.Sleep(time.Duration(r.Float64() * float64(5*time.Second)))
|
time.Sleep(time.Duration(r.Float64() * float64(5*time.Second)))
|
||||||
continue
|
continue
|
||||||
|
@ -237,6 +237,28 @@ func newAllSubsystems() {
|
|||||||
globalBucketTargetSys = NewBucketTargetSys()
|
globalBucketTargetSys = NewBucketTargetSys()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func configRetriableErrors(err error) bool {
|
||||||
|
// Initializing sub-systems needs a retry mechanism for
|
||||||
|
// the following reasons:
|
||||||
|
// - Read quorum is lost just after the initialization
|
||||||
|
// of the object layer.
|
||||||
|
// - Write quorum not met when upgrading configuration
|
||||||
|
// version is needed, migration is needed etc.
|
||||||
|
rquorum := InsufficientReadQuorum{}
|
||||||
|
wquorum := InsufficientWriteQuorum{}
|
||||||
|
|
||||||
|
// One of these retriable errors shall be retried.
|
||||||
|
return errors.Is(err, errDiskNotFound) ||
|
||||||
|
errors.Is(err, errConfigNotFound) ||
|
||||||
|
errors.Is(err, context.DeadlineExceeded) ||
|
||||||
|
errors.Is(err, errErasureWriteQuorum) ||
|
||||||
|
errors.Is(err, errErasureReadQuorum) ||
|
||||||
|
errors.As(err, &rquorum) ||
|
||||||
|
errors.As(err, &wquorum) ||
|
||||||
|
isErrBucketNotFound(err) ||
|
||||||
|
errors.Is(err, os.ErrDeadlineExceeded)
|
||||||
|
}
|
||||||
|
|
||||||
func initServer(ctx context.Context, newObject ObjectLayer) error {
|
func initServer(ctx context.Context, newObject ObjectLayer) error {
|
||||||
// Once the config is fully loaded, initialize the new object layer.
|
// Once the config is fully loaded, initialize the new object layer.
|
||||||
setObjectLayer(newObject)
|
setObjectLayer(newObject)
|
||||||
@ -252,15 +274,6 @@ func initServer(ctx context.Context, newObject ObjectLayer) error {
|
|||||||
// Migrating to encrypted backend should happen before initialization of any
|
// Migrating to encrypted backend should happen before initialization of any
|
||||||
// sub-systems, make sure that we do not move the above codeblock elsewhere.
|
// sub-systems, make sure that we do not move the above codeblock elsewhere.
|
||||||
|
|
||||||
// Initializing sub-systems needs a retry mechanism for
|
|
||||||
// the following reasons:
|
|
||||||
// - Read quorum is lost just after the initialization
|
|
||||||
// of the object layer.
|
|
||||||
// - Write quorum not met when upgrading configuration
|
|
||||||
// version is needed, migration is needed etc.
|
|
||||||
rquorum := InsufficientReadQuorum{}
|
|
||||||
wquorum := InsufficientWriteQuorum{}
|
|
||||||
|
|
||||||
r := rand.New(rand.NewSource(time.Now().UnixNano()))
|
r := rand.New(rand.NewSource(time.Now().UnixNano()))
|
||||||
|
|
||||||
lockTimeout := newDynamicTimeout(5*time.Second, 3*time.Second)
|
lockTimeout := newDynamicTimeout(5*time.Second, 3*time.Second)
|
||||||
@ -307,15 +320,7 @@ func initServer(ctx context.Context, newObject ObjectLayer) error {
|
|||||||
|
|
||||||
txnLk.Unlock() // Unlock the transaction lock and allow other nodes to acquire the lock if possible.
|
txnLk.Unlock() // Unlock the transaction lock and allow other nodes to acquire the lock if possible.
|
||||||
|
|
||||||
// One of these retriable errors shall be retried.
|
if configRetriableErrors(err) {
|
||||||
if errors.Is(err, errDiskNotFound) ||
|
|
||||||
errors.Is(err, errConfigNotFound) ||
|
|
||||||
errors.Is(err, context.DeadlineExceeded) ||
|
|
||||||
errors.Is(err, errErasureWriteQuorum) ||
|
|
||||||
errors.Is(err, errErasureReadQuorum) ||
|
|
||||||
errors.As(err, &rquorum) ||
|
|
||||||
errors.As(err, &wquorum) ||
|
|
||||||
isErrBucketNotFound(err) {
|
|
||||||
logger.Info("Waiting for all MinIO sub-systems to be initialized.. possible cause (%v)", err)
|
logger.Info("Waiting for all MinIO sub-systems to be initialized.. possible cause (%v)", err)
|
||||||
time.Sleep(time.Duration(r.Float64() * float64(5*time.Second)))
|
time.Sleep(time.Duration(r.Float64() * float64(5*time.Second)))
|
||||||
continue
|
continue
|
||||||
@ -333,8 +338,6 @@ func initAllSubsystems(ctx context.Context, newObject ObjectLayer) (err error) {
|
|||||||
// you want to add extra context to your error. This
|
// you want to add extra context to your error. This
|
||||||
// ensures top level retry works accordingly.
|
// ensures top level retry works accordingly.
|
||||||
// List buckets to heal, and be re-used for loading configs.
|
// List buckets to heal, and be re-used for loading configs.
|
||||||
rquorum := InsufficientReadQuorum{}
|
|
||||||
wquorum := InsufficientWriteQuorum{}
|
|
||||||
|
|
||||||
buckets, err := newObject.ListBuckets(ctx)
|
buckets, err := newObject.ListBuckets(ctx)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -368,14 +371,7 @@ func initAllSubsystems(ctx context.Context, newObject ObjectLayer) (err error) {
|
|||||||
|
|
||||||
// Initialize config system.
|
// Initialize config system.
|
||||||
if err = globalConfigSys.Init(newObject); err != nil {
|
if err = globalConfigSys.Init(newObject); err != nil {
|
||||||
if errors.Is(err, errDiskNotFound) ||
|
if configRetriableErrors(err) {
|
||||||
errors.Is(err, errConfigNotFound) ||
|
|
||||||
errors.Is(err, context.DeadlineExceeded) ||
|
|
||||||
errors.Is(err, errErasureWriteQuorum) ||
|
|
||||||
errors.Is(err, errErasureReadQuorum) ||
|
|
||||||
errors.As(err, &rquorum) ||
|
|
||||||
errors.As(err, &wquorum) ||
|
|
||||||
isErrBucketNotFound(err) {
|
|
||||||
return fmt.Errorf("Unable to initialize config system: %w", err)
|
return fmt.Errorf("Unable to initialize config system: %w", err)
|
||||||
}
|
}
|
||||||
// Any other config errors we simply print a message and proceed forward.
|
// Any other config errors we simply print a message and proceed forward.
|
||||||
|
@ -62,7 +62,7 @@ const (
|
|||||||
|
|
||||||
// Detects change in underlying disk.
|
// Detects change in underlying disk.
|
||||||
type xlStorageDiskIDCheck struct {
|
type xlStorageDiskIDCheck struct {
|
||||||
storage *xlStorage
|
storage StorageAPI
|
||||||
diskID string
|
diskID string
|
||||||
|
|
||||||
apiCalls [metricLast]uint64
|
apiCalls [metricLast]uint64
|
||||||
|
@ -71,6 +71,13 @@ const (
|
|||||||
xlStorageFormatFile = "xl.meta"
|
xlStorageFormatFile = "xl.meta"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var alignedBuf []byte
|
||||||
|
|
||||||
|
func init() {
|
||||||
|
alignedBuf = disk.AlignedBlock(4096)
|
||||||
|
_, _ = rand.Read(alignedBuf)
|
||||||
|
}
|
||||||
|
|
||||||
// isValidVolname verifies a volname name in accordance with object
|
// isValidVolname verifies a volname name in accordance with object
|
||||||
// layer requirements.
|
// layer requirements.
|
||||||
func isValidVolname(volname string) bool {
|
func isValidVolname(volname string) bool {
|
||||||
@ -282,10 +289,17 @@ func newXLStorage(ep Endpoint) (*xlStorage, error) {
|
|||||||
var rnd [8]byte
|
var rnd [8]byte
|
||||||
_, _ = rand.Read(rnd[:])
|
_, _ = rand.Read(rnd[:])
|
||||||
tmpFile := ".writable-check-" + hex.EncodeToString(rnd[:]) + ".tmp"
|
tmpFile := ".writable-check-" + hex.EncodeToString(rnd[:]) + ".tmp"
|
||||||
if err = p.CreateFile(GlobalContext, minioMetaTmpBucket, tmpFile, 1, strings.NewReader("0")); err != nil {
|
filePath := pathJoin(p.diskPath, minioMetaTmpBucket, tmpFile)
|
||||||
|
w, err := disk.OpenFileDirectIO(filePath, os.O_CREATE|os.O_WRONLY|os.O_EXCL, 0666)
|
||||||
|
if err != nil {
|
||||||
return p, err
|
return p, err
|
||||||
}
|
}
|
||||||
defer os.Remove(pathJoin(p.diskPath, minioMetaTmpBucket, tmpFile))
|
if _, err = w.Write(alignedBuf[:]); err != nil {
|
||||||
|
w.Close()
|
||||||
|
return p, err
|
||||||
|
}
|
||||||
|
w.Close()
|
||||||
|
defer os.Remove(filePath)
|
||||||
|
|
||||||
volumeDir, err := p.getVolDir(minioMetaTmpBucket)
|
volumeDir, err := p.getVolDir(minioMetaTmpBucket)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -294,7 +308,7 @@ func newXLStorage(ep Endpoint) (*xlStorage, error) {
|
|||||||
|
|
||||||
// Check if backend is readable, and optionally supports O_DIRECT.
|
// Check if backend is readable, and optionally supports O_DIRECT.
|
||||||
if _, err = p.readAllData(volumeDir, pathJoin(volumeDir, tmpFile), true); err != nil {
|
if _, err = p.readAllData(volumeDir, pathJoin(volumeDir, tmpFile), true); err != nil {
|
||||||
if err != errUnsupportedDisk {
|
if !errors.Is(err, errUnsupportedDisk) {
|
||||||
return p, err
|
return p, err
|
||||||
}
|
}
|
||||||
// error is unsupported disk, turn-off directIO for reads
|
// error is unsupported disk, turn-off directIO for reads
|
||||||
@ -1432,7 +1446,7 @@ func (s *xlStorage) CreateFile(ctx context.Context, volume, path string, fileSiz
|
|||||||
return errInvalidArgument
|
return errInvalidArgument
|
||||||
}
|
}
|
||||||
|
|
||||||
if fileSize <= smallFileThreshold {
|
if fileSize >= 0 && fileSize <= smallFileThreshold {
|
||||||
// For streams smaller than 128KiB we simply write them as O_DSYNC (fdatasync)
|
// For streams smaller than 128KiB we simply write them as O_DSYNC (fdatasync)
|
||||||
// and not O_DIRECT to avoid the complexities of aligned I/O.
|
// and not O_DIRECT to avoid the complexities of aligned I/O.
|
||||||
w, err := s.openFile(volume, path, os.O_CREATE|os.O_WRONLY|os.O_TRUNC)
|
w, err := s.openFile(volume, path, os.O_CREATE|os.O_WRONLY|os.O_TRUNC)
|
||||||
@ -1483,9 +1497,9 @@ func (s *xlStorage) CreateFile(ctx context.Context, volume, path string, fileSiz
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
if written < fileSize {
|
if written < fileSize && fileSize >= 0 {
|
||||||
return errLessData
|
return errLessData
|
||||||
} else if written > fileSize {
|
} else if written > fileSize && fileSize >= 0 {
|
||||||
return errMoreData
|
return errMoreData
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1730,7 +1730,7 @@ func TestXLStorageVerifyFile(t *testing.T) {
|
|||||||
// 4) Streaming bitrot check on corrupted file
|
// 4) Streaming bitrot check on corrupted file
|
||||||
|
|
||||||
// create xlStorage test setup
|
// create xlStorage test setup
|
||||||
xlStorage, path, err := newXLStorageTestSetup()
|
storage, path, err := newXLStorageTestSetup()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("Unable to create xlStorage test setup, %s", err)
|
t.Fatalf("Unable to create xlStorage test setup, %s", err)
|
||||||
}
|
}
|
||||||
@ -1738,7 +1738,7 @@ func TestXLStorageVerifyFile(t *testing.T) {
|
|||||||
|
|
||||||
volName := "testvol"
|
volName := "testvol"
|
||||||
fileName := "testfile"
|
fileName := "testfile"
|
||||||
if err := xlStorage.MakeVol(context.Background(), volName); err != nil {
|
if err := storage.MakeVol(context.Background(), volName); err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1752,29 +1752,29 @@ func TestXLStorageVerifyFile(t *testing.T) {
|
|||||||
h := algo.New()
|
h := algo.New()
|
||||||
h.Write(data)
|
h.Write(data)
|
||||||
hashBytes := h.Sum(nil)
|
hashBytes := h.Sum(nil)
|
||||||
if err := xlStorage.WriteAll(context.Background(), volName, fileName, data); err != nil {
|
if err := storage.WriteAll(context.Background(), volName, fileName, data); err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
if err := xlStorage.storage.bitrotVerify(pathJoin(path, volName, fileName), size, algo, hashBytes, 0); err != nil {
|
if err := storage.storage.(*xlStorage).bitrotVerify(pathJoin(path, volName, fileName), size, algo, hashBytes, 0); err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// 2) Whole-file bitrot check on corrupted file
|
// 2) Whole-file bitrot check on corrupted file
|
||||||
if err := xlStorage.AppendFile(context.Background(), volName, fileName, []byte("a")); err != nil {
|
if err := storage.AppendFile(context.Background(), volName, fileName, []byte("a")); err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check if VerifyFile reports the incorrect file length (the correct length is `size+1`)
|
// Check if VerifyFile reports the incorrect file length (the correct length is `size+1`)
|
||||||
if err := xlStorage.storage.bitrotVerify(pathJoin(path, volName, fileName), size, algo, hashBytes, 0); err == nil {
|
if err := storage.storage.(*xlStorage).bitrotVerify(pathJoin(path, volName, fileName), size, algo, hashBytes, 0); err == nil {
|
||||||
t.Fatal("expected to fail bitrot check")
|
t.Fatal("expected to fail bitrot check")
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check if bitrot fails
|
// Check if bitrot fails
|
||||||
if err := xlStorage.storage.bitrotVerify(pathJoin(path, volName, fileName), size+1, algo, hashBytes, 0); err == nil {
|
if err := storage.storage.(*xlStorage).bitrotVerify(pathJoin(path, volName, fileName), size+1, algo, hashBytes, 0); err == nil {
|
||||||
t.Fatal("expected to fail bitrot check")
|
t.Fatal("expected to fail bitrot check")
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := xlStorage.Delete(context.Background(), volName, fileName, false); err != nil {
|
if err := storage.Delete(context.Background(), volName, fileName, false); err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1782,7 +1782,7 @@ func TestXLStorageVerifyFile(t *testing.T) {
|
|||||||
algo = HighwayHash256S
|
algo = HighwayHash256S
|
||||||
shardSize := int64(1024 * 1024)
|
shardSize := int64(1024 * 1024)
|
||||||
shard := make([]byte, shardSize)
|
shard := make([]byte, shardSize)
|
||||||
w := newStreamingBitrotWriter(xlStorage, volName, fileName, size, algo, shardSize)
|
w := newStreamingBitrotWriter(storage, volName, fileName, size, algo, shardSize, false)
|
||||||
reader := bytes.NewReader(data)
|
reader := bytes.NewReader(data)
|
||||||
for {
|
for {
|
||||||
// Using io.Copy instead of this loop will not work for us as io.Copy
|
// Using io.Copy instead of this loop will not work for us as io.Copy
|
||||||
@ -1798,13 +1798,13 @@ func TestXLStorageVerifyFile(t *testing.T) {
|
|||||||
}
|
}
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
w.Close()
|
w.(io.Closer).Close()
|
||||||
if err := xlStorage.storage.bitrotVerify(pathJoin(path, volName, fileName), size, algo, nil, shardSize); err != nil {
|
if err := storage.storage.(*xlStorage).bitrotVerify(pathJoin(path, volName, fileName), size, algo, nil, shardSize); err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// 4) Streaming bitrot check on corrupted file
|
// 4) Streaming bitrot check on corrupted file
|
||||||
filePath := pathJoin(xlStorage.String(), volName, fileName)
|
filePath := pathJoin(storage.String(), volName, fileName)
|
||||||
f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_SYNC, 0644)
|
f, err := os.OpenFile(filePath, os.O_WRONLY|os.O_SYNC, 0644)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
@ -1814,10 +1814,10 @@ func TestXLStorageVerifyFile(t *testing.T) {
|
|||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
}
|
}
|
||||||
f.Close()
|
f.Close()
|
||||||
if err := xlStorage.storage.bitrotVerify(pathJoin(path, volName, fileName), size, algo, nil, shardSize); err == nil {
|
if err := storage.storage.(*xlStorage).bitrotVerify(pathJoin(path, volName, fileName), size, algo, nil, shardSize); err == nil {
|
||||||
t.Fatal("expected to fail bitrot check")
|
t.Fatal("expected to fail bitrot check")
|
||||||
}
|
}
|
||||||
if err := xlStorage.storage.bitrotVerify(pathJoin(path, volName, fileName), size+1, algo, nil, shardSize); err == nil {
|
if err := storage.storage.(*xlStorage).bitrotVerify(pathJoin(path, volName, fileName), size+1, algo, nil, shardSize); err == nil {
|
||||||
t.Fatal("expected to fail bitrot check")
|
t.Fatal("expected to fail bitrot check")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -20,8 +20,10 @@ package ioutil
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"context"
|
||||||
"io"
|
"io"
|
||||||
"os"
|
"os"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/minio/minio/pkg/disk"
|
"github.com/minio/minio/pkg/disk"
|
||||||
)
|
)
|
||||||
@ -64,6 +66,56 @@ func WriteOnClose(w io.Writer) *WriteOnCloser {
|
|||||||
return &WriteOnCloser{w, false}
|
return &WriteOnCloser{w, false}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type ioret struct {
|
||||||
|
n int
|
||||||
|
err error
|
||||||
|
}
|
||||||
|
|
||||||
|
// DeadlineWriter deadline writer with context
|
||||||
|
type DeadlineWriter struct {
|
||||||
|
io.WriteCloser
|
||||||
|
timeout time.Duration
|
||||||
|
err error
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewDeadlineWriter wraps a writer to make it respect given deadline
|
||||||
|
// value per Write(). If there is a blocking write, the returned Writer
|
||||||
|
// will return whenever the timer hits (the return values are n=0
|
||||||
|
// and err=context.Canceled.)
|
||||||
|
func NewDeadlineWriter(w io.WriteCloser, timeout time.Duration) io.WriteCloser {
|
||||||
|
return &DeadlineWriter{WriteCloser: w, timeout: timeout}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (w *DeadlineWriter) Write(buf []byte) (int, error) {
|
||||||
|
if w.err != nil {
|
||||||
|
return 0, w.err
|
||||||
|
}
|
||||||
|
|
||||||
|
c := make(chan ioret, 1)
|
||||||
|
t := time.NewTimer(w.timeout)
|
||||||
|
defer t.Stop()
|
||||||
|
|
||||||
|
go func() {
|
||||||
|
n, err := w.WriteCloser.Write(buf)
|
||||||
|
c <- ioret{n, err}
|
||||||
|
close(c)
|
||||||
|
}()
|
||||||
|
|
||||||
|
select {
|
||||||
|
case r := <-c:
|
||||||
|
w.err = r.err
|
||||||
|
return r.n, r.err
|
||||||
|
case <-t.C:
|
||||||
|
w.err = context.Canceled
|
||||||
|
return 0, context.Canceled
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Close closer interface to close the underlying closer
|
||||||
|
func (w *DeadlineWriter) Close() error {
|
||||||
|
return w.WriteCloser.Close()
|
||||||
|
}
|
||||||
|
|
||||||
// LimitWriter implements io.WriteCloser.
|
// LimitWriter implements io.WriteCloser.
|
||||||
//
|
//
|
||||||
// This is implemented such that we want to restrict
|
// This is implemented such that we want to restrict
|
||||||
|
@ -18,12 +18,49 @@ package ioutil
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"context"
|
||||||
"io"
|
"io"
|
||||||
goioutil "io/ioutil"
|
goioutil "io/ioutil"
|
||||||
"os"
|
"os"
|
||||||
"testing"
|
"testing"
|
||||||
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
type sleepWriter struct {
|
||||||
|
timeout time.Duration
|
||||||
|
}
|
||||||
|
|
||||||
|
func (w *sleepWriter) Write(p []byte) (n int, err error) {
|
||||||
|
time.Sleep(w.timeout)
|
||||||
|
return len(p), nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (w *sleepWriter) Close() error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestDeadlineWriter(t *testing.T) {
|
||||||
|
w := NewDeadlineWriter(&sleepWriter{timeout: 500 * time.Millisecond}, 450*time.Millisecond)
|
||||||
|
_, err := w.Write([]byte("1"))
|
||||||
|
w.Close()
|
||||||
|
if err != context.Canceled {
|
||||||
|
t.Error("DeadlineWriter shouldn't be successful - should return context.Canceled")
|
||||||
|
}
|
||||||
|
_, err = w.Write([]byte("1"))
|
||||||
|
if err != context.Canceled {
|
||||||
|
t.Error("DeadlineWriter shouldn't be successful - should return context.Canceled")
|
||||||
|
}
|
||||||
|
w = NewDeadlineWriter(&sleepWriter{timeout: 500 * time.Millisecond}, 600*time.Millisecond)
|
||||||
|
n, err := w.Write([]byte("abcd"))
|
||||||
|
w.Close()
|
||||||
|
if err != nil {
|
||||||
|
t.Errorf("DeadlineWriter should succeed but failed with %s", err)
|
||||||
|
}
|
||||||
|
if n != 4 {
|
||||||
|
t.Errorf("DeadlineWriter should succeed but should have only written 0 bytes, but returned %d instead", n)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func TestCloseOnWriter(t *testing.T) {
|
func TestCloseOnWriter(t *testing.T) {
|
||||||
writer := WriteOnClose(goioutil.Discard)
|
writer := WriteOnClose(goioutil.Discard)
|
||||||
if writer.HasWritten() {
|
if writer.HasWritten() {
|
||||||
|
Loading…
x
Reference in New Issue
Block a user