fix: unexpected logging with bucket metadata conversions (#9519)

master
Harshavardhana 4 years ago committed by GitHub
parent 7b58dcb28c
commit b768645fde
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 1
      cmd/admin-heal-ops.go
  2. 6
      cmd/bucket-handlers.go
  3. 6
      cmd/bucket-meta.go
  4. 2
      cmd/bucket-quota.go
  5. 70
      cmd/config-encrypted.go
  6. 4
      cmd/data-update-tracker.go
  7. 13
      cmd/format-fs.go
  8. 11
      cmd/retry.go
  9. 128
      cmd/server-main.go
  10. 9
      pkg/dsync/drwmutex.go
  11. 15
      pkg/dsync/retry.go
  12. 29
      pkg/dsync/retry_test.go

@ -715,7 +715,6 @@ func (h *healSequence) healItemsFromSourceCh() error {
func (h *healSequence) healFromSourceCh() { func (h *healSequence) healFromSourceCh() {
h.healItemsFromSourceCh() h.healItemsFromSourceCh()
close(h.traverseAndHealDoneCh)
} }
func (h *healSequence) healItems(bucketsOnly bool) error { func (h *healSequence) healItems(bucketsOnly bool) error {

@ -293,10 +293,12 @@ func (api objectAPIHandlers) ListBucketsHandler(w http.ResponseWriter, r *http.R
for i := range bucketsInfo { for i := range bucketsInfo {
meta, err := loadBucketMetadata(ctx, objectAPI, bucketsInfo[i].Name) meta, err := loadBucketMetadata(ctx, objectAPI, bucketsInfo[i].Name)
logger.LogIf(ctx, err)
if err == nil { if err == nil {
bucketsInfo[i].Created = meta.Created bucketsInfo[i].Created = meta.Created
} }
if err != errMetaDataConverted {
logger.LogIf(ctx, err)
}
} }
} }
@ -1103,7 +1105,7 @@ func (api objectAPIHandlers) GetBucketObjectLockConfigHandler(w http.ResponseWri
} }
meta, err := loadBucketMetadata(ctx, objectAPI, bucket) meta, err := loadBucketMetadata(ctx, objectAPI, bucket)
if err != nil { if err != nil && err != errMetaDataConverted {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r)) writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return return
} }

@ -115,13 +115,15 @@ func (b *bucketMetadata) convertLegacyLockconfig(ctx context.Context, objectAPI
return err return err
} }
logger.LogIf(ctx, deleteConfig(ctx, objectAPI, configFile)) if err := deleteConfig(ctx, objectAPI, configFile); err != nil && !errors.Is(err, errConfigNotFound) {
logger.LogIf(ctx, err)
}
return nil return nil
} }
configData, err := readConfig(ctx, objectAPI, configFile) configData, err := readConfig(ctx, objectAPI, configFile)
if err != nil { if err != nil {
if err != errConfigNotFound { if !errors.Is(err, errConfigNotFound) {
return err return err
} }

@ -77,7 +77,7 @@ func (sys *BucketQuotaSys) Init(buckets []BucketInfo, objAPI ObjectLayer) error
return errServerNotInitialized return errServerNotInitialized
} }
// In gateway mode, we always fetch the bucket lifecycle configuration from the gateway backend. // In gateway mode, we do not support bucket quota.
// So, this is a no-op for gateway servers. // So, this is a no-op for gateway servers.
if globalIsGateway { if globalIsGateway {
return nil return nil

@ -36,42 +36,10 @@ func handleEncryptedConfigBackend(objAPI ObjectLayer, server bool) error {
return nil return nil
} }
// If its server mode or nas gateway, migrate the backend. encrypted, err := checkBackendEncrypted(objAPI)
doneCh := make(chan struct{}) if err != nil {
return fmt.Errorf("Unable to encrypt config %w", err)
var encrypted bool
var err error
// Migrating Config backend needs a retry mechanism for
// the following reasons:
// - Read quorum is lost just after the initialization
// of the object layer.
retryTimerCh := newRetryTimerSimple(doneCh)
var stop bool
rquorum := InsufficientReadQuorum{}
wquorum := InsufficientWriteQuorum{}
for !stop {
select {
case <-retryTimerCh:
if encrypted, err = checkBackendEncrypted(objAPI); err != nil {
if errors.Is(err, errDiskNotFound) ||
errors.As(err, &rquorum) ||
isErrBucketNotFound(err) {
logger.Info("Waiting for config backend to be encrypted..")
continue
}
close(doneCh)
return err
}
stop = true
case <-globalOSSignalCh:
close(doneCh)
return fmt.Errorf("Config encryption process stopped gracefully")
}
} }
close(doneCh)
if encrypted { if encrypted {
// backend is encrypted, but credentials are not specified // backend is encrypted, but credentials are not specified
@ -91,34 +59,12 @@ func handleEncryptedConfigBackend(objAPI ObjectLayer, server bool) error {
} }
} }
doneCh = make(chan struct{}) // Migrate IAM configuration
defer close(doneCh) if err = migrateConfigPrefixToEncrypted(objAPI, globalOldCred, encrypted); err != nil {
return fmt.Errorf("Unable to migrate all config at .minio.sys/config/: %w", err)
retryTimerCh = newRetryTimerSimple(doneCh)
// Migrating Config backend needs a retry mechanism for
// the following reasons:
// - Read quorum is lost just after the initialization
// of the object layer.
for {
select {
case <-retryTimerCh:
// Migrate IAM configuration
if err = migrateConfigPrefixToEncrypted(objAPI, globalOldCred, encrypted); err != nil {
if errors.Is(err, errDiskNotFound) ||
errors.As(err, &rquorum) ||
errors.As(err, &wquorum) ||
isErrBucketNotFound(err) {
logger.Info("Waiting for config backend to be encrypted..")
continue
}
return err
}
return nil
case <-globalOSSignalCh:
return fmt.Errorf("Config encryption process stopped gracefully")
}
} }
return nil
} }
const ( const (

@ -352,7 +352,9 @@ func (d *dataUpdateTracker) deserialize(src io.Reader, newerThan time.Time) erro
// Version // Version
if _, err := io.ReadFull(src, tmp[:1]); err != nil { if _, err := io.ReadFull(src, tmp[:1]); err != nil {
if d.debug { if d.debug {
logger.LogIf(ctx, err) if err != io.EOF {
logger.LogIf(ctx, err)
}
} }
return err return err
} }

@ -190,7 +190,7 @@ func initFormatFS(ctx context.Context, fsPath string) (rlk *lock.RLockedFile, er
fsFormatPath := pathJoin(fsPath, minioMetaBucket, formatConfigFile) fsFormatPath := pathJoin(fsPath, minioMetaBucket, formatConfigFile)
// Add a deployment ID, if it does not exist. // Add a deployment ID, if it does not exist.
if err := formatFSFixDeploymentID(fsFormatPath); err != nil { if err := formatFSFixDeploymentID(ctx, fsFormatPath); err != nil {
return nil, err return nil, err
} }
@ -288,7 +288,7 @@ func formatFSGetDeploymentID(rlk *lock.RLockedFile) (id string, err error) {
} }
// Generate a deployment ID if one does not exist already. // Generate a deployment ID if one does not exist already.
func formatFSFixDeploymentID(fsFormatPath string) error { func formatFSFixDeploymentID(ctx context.Context, fsFormatPath string) error {
rlk, err := lock.RLockedOpenFile(fsFormatPath) rlk, err := lock.RLockedOpenFile(fsFormatPath)
if err == nil { if err == nil {
// format.json can be empty in a rare condition when another // format.json can be empty in a rare condition when another
@ -339,11 +339,12 @@ func formatFSFixDeploymentID(fsFormatPath string) error {
return time.Now().Round(time.Second).Sub(formatStartTime).String() return time.Now().Round(time.Second).Sub(formatStartTime).String()
} }
doneCh := make(chan struct{}) retryCtx, cancel := context.WithCancel(ctx)
defer close(doneCh) // Indicate to our routine to exit cleanly upon return.
defer cancel()
var wlk *lock.LockedFile var wlk *lock.LockedFile
retryCh := newRetryTimerSimple(doneCh) retryCh := newRetryTimerSimple(retryCtx)
var stop bool var stop bool
for !stop { for !stop {
select { select {
@ -358,7 +359,7 @@ func formatFSFixDeploymentID(fsFormatPath string) error {
return err return err
} }
stop = true stop = true
case <-globalOSSignalCh: case <-ctx.Done():
return fmt.Errorf("Initializing FS format stopped gracefully") return fmt.Errorf("Initializing FS format stopped gracefully")
} }
} }

@ -17,6 +17,7 @@
package cmd package cmd
import ( import (
"context"
"math/rand" "math/rand"
"sync" "sync"
"time" "time"
@ -61,7 +62,7 @@ var globalRandomSource = rand.New(&lockedRandSource{
// until the maximum retry attempts are reached. - this function is a fully // until the maximum retry attempts are reached. - this function is a fully
// configurable version, meant for only advanced use cases. For the most part // configurable version, meant for only advanced use cases. For the most part
// one should use newRetryTimerSimple and newRetryTimer. // one should use newRetryTimerSimple and newRetryTimer.
func newRetryTimerWithJitter(unit time.Duration, cap time.Duration, jitter float64, doneCh chan struct{}) <-chan int { func newRetryTimerWithJitter(ctx context.Context, unit time.Duration, cap time.Duration, jitter float64) <-chan int {
attemptCh := make(chan int) attemptCh := make(chan int)
// normalize jitter to the range [0, 1.0] // normalize jitter to the range [0, 1.0]
@ -100,7 +101,7 @@ func newRetryTimerWithJitter(unit time.Duration, cap time.Duration, jitter float
select { // Attempts starts. select { // Attempts starts.
case attemptCh <- nextBackoff: case attemptCh <- nextBackoff:
nextBackoff++ nextBackoff++
case <-doneCh: case <-ctx.Done():
// Stop the routine. // Stop the routine.
return return
} }
@ -108,7 +109,7 @@ func newRetryTimerWithJitter(unit time.Duration, cap time.Duration, jitter float
// wait till next backoff time or till doneCh gets a message. // wait till next backoff time or till doneCh gets a message.
select { select {
case <-timer.C: case <-timer.C:
case <-doneCh: case <-ctx.Done():
// stop the timer and return. // stop the timer and return.
timer.Stop() timer.Stop()
return return
@ -130,6 +131,6 @@ const (
// newRetryTimerSimple creates a timer with exponentially increasing delays // newRetryTimerSimple creates a timer with exponentially increasing delays
// until the maximum retry attempts are reached. - this function is a // until the maximum retry attempts are reached. - this function is a
// simpler version with all default values. // simpler version with all default values.
func newRetryTimerSimple(doneCh chan struct{}) <-chan int { func newRetryTimerSimple(ctx context.Context) <-chan int {
return newRetryTimerWithJitter(defaultRetryUnit, defaultRetryCap, MaxJitter, doneCh) return newRetryTimerWithJitter(ctx, defaultRetryUnit, defaultRetryCap, MaxJitter)
} }

@ -160,53 +160,45 @@ func newAllSubsystems() {
globalBucketQuotaSys = NewBucketQuotaSys() globalBucketQuotaSys = NewBucketQuotaSys()
} }
func initSafeMode(buckets []BucketInfo) (err error) { func initSafeMode() (err error) {
newObject := newObjectLayerWithoutSafeModeFn() newObject := newObjectLayerWithoutSafeModeFn()
// Construct path to config/transaction.lock for locking
transactionConfigPrefix := minioConfigPrefix + "/transaction.lock"
// Make sure to hold lock for entire migration to avoid // Make sure to hold lock for entire migration to avoid
// such that only one server should migrate the entire config // such that only one server should migrate the entire config
// at a given time, this big transaction lock ensures this // at a given time, this big transaction lock ensures this
// appropriately. This is also true for rotation of encrypted // appropriately. This is also true for rotation of encrypted
// content. // content.
objLock := newObject.NewNSLock(GlobalContext, minioMetaBucket, transactionConfigPrefix) txnLk := newObject.NewNSLock(GlobalContext, minioMetaBucket, minioConfigPrefix+"/transaction.lock")
if err = objLock.GetLock(globalOperationTimeout); err != nil { defer func(txnLk RWLocker) {
return err txnLk.Unlock()
}
defer func(objLock RWLocker) {
objLock.Unlock()
if err != nil { if err != nil {
var cerr config.Err var cerr config.Err
// For any config error, we don't need to drop into safe-mode
// instead its a user error and should be fixed by user.
if errors.As(err, &cerr) { if errors.As(err, &cerr) {
return return
} }
// Prints the formatted startup message in safe mode operation. // Prints the formatted startup message in safe mode operation.
// Drops-into safe mode where users need to now manually recover
// the server.
printStartupSafeModeMessage(getAPIEndpoints(), err) printStartupSafeModeMessage(getAPIEndpoints(), err)
// Initialization returned error reaching safe mode and // Initialization returned error reaching safe mode and
// not proceeding waiting for admin action. // not proceeding waiting for admin action.
handleSignals() handleSignals()
} }
}(objLock) }(txnLk)
// Migrate all backend configs to encrypted backend configs, optionally
// handles rotating keys for encryption.
if err = handleEncryptedConfigBackend(newObject, true); err != nil {
return fmt.Errorf("Unable to handle encrypted backend for config, iam and policies: %w", err)
}
// **** WARNING **** // **** WARNING ****
// 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.
// Create cancel context to control 'newRetryTimer' go routine.
retryCtx, cancel := context.WithCancel(GlobalContext)
// Validate and initialize all subsystems. // Indicate to our routine to exit cleanly upon return.
doneCh := make(chan struct{}) defer cancel()
defer close(doneCh)
// Initializing sub-systems needs a retry mechanism for // Initializing sub-systems needs a retry mechanism for
// the following reasons: // the following reasons:
@ -214,43 +206,67 @@ func initSafeMode(buckets []BucketInfo) (err error) {
// of the object layer. // of the object layer.
// - Write quorum not met when upgrading configuration // - Write quorum not met when upgrading configuration
// version is needed, migration is needed etc. // version is needed, migration is needed etc.
retryTimerCh := newRetryTimerSimple(doneCh) rquorum := InsufficientReadQuorum{}
for { wquorum := InsufficientWriteQuorum{}
rquorum := InsufficientReadQuorum{} optimeout := OperationTimedOut{}
wquorum := InsufficientWriteQuorum{} for n := range newRetryTimerSimple(retryCtx) {
bucketNotFound := BucketNotFound{} // let one of the server acquire the lock, if not let them timeout.
var err error // which shall be retried again by this loop.
select { if err = txnLk.GetLock(leaderLockTimeout); err == nil {
case n := <-retryTimerCh: // Migrate all backend configs to encrypted backend configs, optionally
if err = initAllSubsystems(buckets, newObject); err != nil { // handles rotating keys for encryption, if there is any retriable failure
if errors.Is(err, errDiskNotFound) || // that shall be retried if there is an error.
errors.As(err, &rquorum) || if err = handleEncryptedConfigBackend(newObject, true); err == nil {
errors.As(err, &wquorum) || // Upon success migrating the config, initialize all sub-systems
errors.As(err, &bucketNotFound) { // if all sub-systems initialized successfully return right away
if n < 5 { if err = initAllSubsystems(newObject); err == nil {
logger.Info("Waiting for all sub-systems to be initialized..") return nil
} else {
logger.Info("Waiting for all sub-systems to be initialized.. %v", err)
}
continue
} }
return err
} }
return nil }
case <-globalOSSignalCh:
if err == nil { // One of these retriable errors shall be retried.
return errors.New("Initializing sub-systems stopped gracefully") if errors.Is(err, errDiskNotFound) ||
errors.Is(err, errConfigNotFound) ||
errors.Is(err, context.Canceled) ||
errors.Is(err, context.DeadlineExceeded) ||
errors.As(err, &optimeout) ||
errors.As(err, &rquorum) ||
errors.As(err, &wquorum) ||
isErrBucketNotFound(err) {
if n < 5 {
logger.Info("Waiting for all MinIO sub-systems to be initialized..")
} else {
logger.Info("Waiting for all MinIO sub-systems to be initialized.. possible cause (%v)", err)
} }
return fmt.Errorf("Unable to initialize sub-systems: %w", err) continue
} }
// Any other unhandled return right here.
return fmt.Errorf("Unable to initialize sub-systems: %w", err)
}
// Return an error when retry is canceled or deadlined
if err = retryCtx.Err(); err != nil {
return fmt.Errorf("Unable to initialize sub-systems: %w", err)
} }
// Retry was canceled successfully.
return errors.New("Initializing sub-systems stopped gracefully")
} }
func initAllSubsystems(buckets []BucketInfo, newObject ObjectLayer) (err error) { func initAllSubsystems(newObject ObjectLayer) (err error) {
// List buckets to be re-used for loading configs.
buckets, err := newObject.ListBuckets(GlobalContext)
if err != nil {
return fmt.Errorf("Unable to list buckets: %w", err)
}
// Initialize config system. // Initialize config system.
if err = globalConfigSys.Init(newObject); err != nil { if err = globalConfigSys.Init(newObject); err != nil {
return fmt.Errorf("Unable to initialize config system: %w", err) return fmt.Errorf("Unable to initialize config system: %w", err)
} }
if globalEtcdClient != nil { if globalEtcdClient != nil {
// **** WARNING **** // **** WARNING ****
// Migrating to encrypted backend on etcd should happen before initialization of // Migrating to encrypted backend on etcd should happen before initialization of
@ -294,6 +310,11 @@ func initAllSubsystems(buckets []BucketInfo, newObject ObjectLayer) (err error)
return fmt.Errorf("Unable to initialize bucket quota system: %w", err) return fmt.Errorf("Unable to initialize bucket quota system: %w", err)
} }
// Populate existing buckets to the etcd backend
if globalDNSConfig != nil {
initFederatorBackend(buckets, newObject)
}
return nil return nil
} }
@ -458,13 +479,7 @@ func serverMain(ctx *cli.Context) {
go startBackgroundOps(GlobalContext, newObject) go startBackgroundOps(GlobalContext, newObject)
// Calls New() and initializes all sub-systems. logger.FatalIf(initSafeMode(), "Unable to initialize server switching into safe-mode")
buckets, err := newObject.ListBuckets(GlobalContext)
if err != nil {
logger.Fatal(err, "Unable to list buckets")
}
logger.FatalIf(initSafeMode(buckets), "Unable to initialize server switching into safe-mode")
if globalCacheConfig.Enabled { if globalCacheConfig.Enabled {
// initialize the new disk cache objects. // initialize the new disk cache objects.
@ -477,11 +492,6 @@ func serverMain(ctx *cli.Context) {
globalObjLayerMutex.Unlock() globalObjLayerMutex.Unlock()
} }
// Populate existing buckets to the etcd backend
if globalDNSConfig != nil {
initFederatorBackend(buckets, newObject)
}
// Disable safe mode operation, after all initialization is over. // Disable safe mode operation, after all initialization is over.
globalObjLayerMutex.Lock() globalObjLayerMutex.Lock()
globalSafeMode = false globalSafeMode = false

@ -128,13 +128,16 @@ func (dm *DRWMutex) GetRLock(id, source string, timeout time.Duration) (locked b
// algorithm until either the lock is acquired successfully or more // algorithm until either the lock is acquired successfully or more
// time has elapsed than the timeout value. // time has elapsed than the timeout value.
func (dm *DRWMutex) lockBlocking(timeout time.Duration, id, source string, isReadLock bool) (locked bool) { func (dm *DRWMutex) lockBlocking(timeout time.Duration, id, source string, isReadLock bool) (locked bool) {
doneCh, start := make(chan struct{}), time.Now().UTC() start := time.Now().UTC()
defer close(doneCh)
restClnts := dm.clnt.GetLockersFn() restClnts := dm.clnt.GetLockersFn()
retryCtx, cancel := context.WithCancel(dm.ctx)
defer cancel()
// Use incremental back-off algorithm for repeated attempts to acquire the lock // Use incremental back-off algorithm for repeated attempts to acquire the lock
for range newRetryTimerSimple(doneCh) { for range newRetryTimerSimple(retryCtx) {
select { select {
case <-dm.ctx.Done(): case <-dm.ctx.Done():
return return

@ -17,6 +17,7 @@
package dsync package dsync
import ( import (
"context"
"math/rand" "math/rand"
"sync" "sync"
"time" "time"
@ -61,7 +62,7 @@ var globalRandomSource = rand.New(&lockedRandSource{
// until the maximum retry attempts are reached. - this function is a fully // until the maximum retry attempts are reached. - this function is a fully
// configurable version, meant for only advanced use cases. For the most part // configurable version, meant for only advanced use cases. For the most part
// one should use newRetryTimerSimple and newRetryTimer. // one should use newRetryTimerSimple and newRetryTimer.
func newRetryTimerWithJitter(unit time.Duration, cap time.Duration, jitter float64, doneCh <-chan struct{}) <-chan int { func newRetryTimerWithJitter(ctx context.Context, unit time.Duration, cap time.Duration, jitter float64) <-chan int {
attemptCh := make(chan int) attemptCh := make(chan int)
// normalize jitter to the range [0, 1.0] // normalize jitter to the range [0, 1.0]
@ -100,7 +101,7 @@ func newRetryTimerWithJitter(unit time.Duration, cap time.Duration, jitter float
select { // Attempts starts. select { // Attempts starts.
case attemptCh <- nextBackoff: case attemptCh <- nextBackoff:
nextBackoff++ nextBackoff++
case <-doneCh: case <-ctx.Done():
// Stop the routine. // Stop the routine.
return return
} }
@ -108,7 +109,7 @@ func newRetryTimerWithJitter(unit time.Duration, cap time.Duration, jitter float
// wait till next backoff time or till doneCh gets a message. // wait till next backoff time or till doneCh gets a message.
select { select {
case <-timer.C: case <-timer.C:
case <-doneCh: case <-ctx.Done():
// stop the timer and return. // stop the timer and return.
timer.Stop() timer.Stop()
return return
@ -130,13 +131,13 @@ const (
// newRetryTimer creates a timer with exponentially increasing delays // newRetryTimer creates a timer with exponentially increasing delays
// until the maximum retry attempts are reached. - this function provides // until the maximum retry attempts are reached. - this function provides
// resulting retry values to be of maximum jitter. // resulting retry values to be of maximum jitter.
func newRetryTimer(unit time.Duration, cap time.Duration, doneCh <-chan struct{}) <-chan int { func newRetryTimer(ctx context.Context, unit time.Duration, cap time.Duration) <-chan int {
return newRetryTimerWithJitter(unit, cap, MaxJitter, doneCh) return newRetryTimerWithJitter(ctx, unit, cap, MaxJitter)
} }
// newRetryTimerSimple creates a timer with exponentially increasing delays // newRetryTimerSimple creates a timer with exponentially increasing delays
// until the maximum retry attempts are reached. - this function is a // until the maximum retry attempts are reached. - this function is a
// simpler version with all default values. // simpler version with all default values.
func newRetryTimerSimple(doneCh <-chan struct{}) <-chan int { func newRetryTimerSimple(ctx context.Context) <-chan int {
return newRetryTimerWithJitter(defaultRetryUnit, defaultRetryCap, MaxJitter, doneCh) return newRetryTimerWithJitter(ctx, defaultRetryUnit, defaultRetryCap, MaxJitter)
} }

@ -17,25 +17,26 @@
package dsync package dsync
import ( import (
"context"
"testing" "testing"
"time" "time"
) )
// Tests for retry timer. // Tests for retry timer.
func TestRetryTimerSimple(t *testing.T) { func TestRetryTimerSimple(t *testing.T) {
doneCh := make(chan struct{}) rctx, cancel := context.WithCancel(context.Background())
attemptCh := newRetryTimerSimple(doneCh) attemptCh := newRetryTimerSimple(rctx)
i := <-attemptCh i := <-attemptCh
if i != 0 { if i != 0 {
close(doneCh) cancel()
t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i) t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i)
} }
i = <-attemptCh i = <-attemptCh
if i <= 0 { if i <= 0 {
close(doneCh) cancel()
t.Fatalf("Invalid attempt counter returned should be greater than 0, found %d instead", i) t.Fatalf("Invalid attempt counter returned should be greater than 0, found %d instead", i)
} }
close(doneCh) cancel()
_, ok := <-attemptCh _, ok := <-attemptCh
if ok { if ok {
t.Fatal("Attempt counter should be closed") t.Fatal("Attempt counter should be closed")
@ -44,18 +45,19 @@ func TestRetryTimerSimple(t *testing.T) {
// Test retry time with no jitter. // Test retry time with no jitter.
func TestRetryTimerWithNoJitter(t *testing.T) { func TestRetryTimerWithNoJitter(t *testing.T) {
doneCh := make(chan struct{}) rctx, cancel := context.WithCancel(context.Background())
// No jitter // No jitter
attemptCh := newRetryTimerWithJitter(time.Millisecond, 5*time.Millisecond, NoJitter, doneCh) attemptCh := newRetryTimerWithJitter(rctx, time.Millisecond, 5*time.Millisecond, NoJitter)
i := <-attemptCh i := <-attemptCh
if i != 0 { if i != 0 {
close(doneCh) cancel()
t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i) t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i)
} }
// Loop through the maximum possible attempt. // Loop through the maximum possible attempt.
for i = range attemptCh { for i = range attemptCh {
if i == 30 { if i == 30 {
close(doneCh) cancel()
} }
} }
_, ok := <-attemptCh _, ok := <-attemptCh
@ -66,15 +68,16 @@ func TestRetryTimerWithNoJitter(t *testing.T) {
// Test retry time with Jitter greater than MaxJitter. // Test retry time with Jitter greater than MaxJitter.
func TestRetryTimerWithJitter(t *testing.T) { func TestRetryTimerWithJitter(t *testing.T) {
doneCh := make(chan struct{}) rctx, cancel := context.WithCancel(context.Background())
// Jitter will be set back to 1.0 // Jitter will be set back to 1.0
attemptCh := newRetryTimerWithJitter(time.Second, 30*time.Second, 2.0, doneCh) attemptCh := newRetryTimerWithJitter(rctx, time.Second, 30*time.Second, 2.0)
i := <-attemptCh i := <-attemptCh
if i != 0 { if i != 0 {
close(doneCh) cancel()
t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i) t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i)
} }
close(doneCh) cancel()
_, ok := <-attemptCh _, ok := <-attemptCh
if ok { if ok {
t.Fatal("Attempt counter should be closed") t.Fatal("Attempt counter should be closed")

Loading…
Cancel
Save