From bee3231ed42c9219dbc9ee63a09187fe9dfe91de Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 17 Jun 2023 18:58:36 +0200 Subject: [PATCH 01/18] lock: try refreshing of stale locks A stale lock may be refreshed if it continues to exist until after a replacement lock has been created. This ensures that a repository was not unlocked in the meantime. --- cmd/restic/lock.go | 46 ++++++++++++++++++++++------- internal/restic/lock.go | 64 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 100 insertions(+), 10 deletions(-) diff --git a/cmd/restic/lock.go b/cmd/restic/lock.go index 336b56ad1..4bac4794f 100644 --- a/cmd/restic/lock.go +++ b/cmd/restic/lock.go @@ -108,11 +108,12 @@ retryLoop: } lockInfo.refreshWG.Add(2) refreshChan := make(chan struct{}) + forcedRefreshChan := make(chan struct{}) globalLocks.Lock() globalLocks.locks[lock] = lockInfo - go refreshLocks(ctx, lock, lockInfo, refreshChan) - go monitorLockRefresh(ctx, lockInfo, refreshChan) + go refreshLocks(ctx, lock, lockInfo, refreshChan, forcedRefreshChan) + go monitorLockRefresh(ctx, lock, lockInfo, refreshChan, forcedRefreshChan) globalLocks.Unlock() return lock, ctx, err @@ -124,7 +125,7 @@ var refreshInterval = 5 * time.Minute // the difference allows to compensate for a small time drift between clients. var refreshabilityTimeout = restic.StaleLockTimeout - refreshInterval*3/2 -func refreshLocks(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, refreshed chan<- struct{}) { +func refreshLocks(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, refreshed chan<- struct{}, forcedRefresh <-chan struct{}) { debug.Log("start") ticker := time.NewTicker(refreshInterval) lastRefresh := lock.Time @@ -149,6 +150,11 @@ func refreshLocks(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, case <-ctx.Done(): debug.Log("terminate") return + + case <-forcedRefresh: + // update lock refresh time + lastRefresh = lock.Time + case <-ticker.C: if time.Since(lastRefresh) > refreshabilityTimeout { // the lock is too old, wait until the expiry monitor cancels the context @@ -161,7 +167,7 @@ func refreshLocks(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, Warnf("unable to refresh lock: %v\n", err) } else { lastRefresh = lock.Time - // inform monitor gorountine about successful refresh + // inform monitor goroutine about successful refresh select { case <-ctx.Done(): case refreshed <- struct{}{}: @@ -171,7 +177,7 @@ func refreshLocks(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, } } -func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <-chan struct{}) { +func monitorLockRefresh(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, refreshed <-chan struct{}, forcedRefresh chan<- struct{}) { // time.Now() might use a monotonic timer which is paused during standby // convert to unix time to ensure we compare real time values lastRefresh := time.Now().UnixNano() @@ -183,9 +189,9 @@ func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <- // timers are paused during standby, which is a problem as the refresh timeout // _must_ expire if the host was too long in standby. Thus fall back to periodic checks // https://github.com/golang/go/issues/35012 - timer := time.NewTimer(pollDuration) + ticker := time.NewTicker(pollDuration) defer func() { - timer.Stop() + ticker.Stop() lockInfo.cancel() lockInfo.refreshWG.Done() }() @@ -197,10 +203,20 @@ func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <- return case <-refreshed: lastRefresh = time.Now().UnixNano() - case <-timer.C: + case <-ticker.C: if time.Now().UnixNano()-lastRefresh < refreshabilityTimeout.Nanoseconds() { - // restart timer - timer.Reset(pollDuration) + continue + } + + // keep on going if our current lock still exists + if tryRefreshStaleLock(ctx, lock) { + lastRefresh = time.Now().UnixNano() + + // inform refresh gorountine about forced refresh + select { + case <-ctx.Done(): + case forcedRefresh <- struct{}{}: + } continue } @@ -210,6 +226,16 @@ func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <- } } +func tryRefreshStaleLock(ctx context.Context, lock *restic.Lock) bool { + err := lock.RefreshStaleLock(ctx) + if err != nil { + Warnf("failed to refresh stale lock: %v\n", err) + return false + } + + return true +} + func unlockRepo(lock *restic.Lock) { if lock == nil { return diff --git a/internal/restic/lock.go b/internal/restic/lock.go index d500c019a..b1584aa19 100644 --- a/internal/restic/lock.go +++ b/internal/restic/lock.go @@ -81,6 +81,8 @@ func IsInvalidLock(err error) bool { return errors.As(err, &e) } +var ErrRemovedLock = errors.New("lock file was removed in the meantime") + // NewLock returns a new, non-exclusive lock for the repository. If an // exclusive lock is already held by another process, it returns an error // that satisfies IsAlreadyLocked. @@ -274,6 +276,68 @@ func (l *Lock) Refresh(ctx context.Context) error { return l.repo.Backend().Remove(context.TODO(), Handle{Type: LockFile, Name: oldLockID.String()}) } +// RefreshStaleLock is an extended variant of Refresh that can also refresh stale lock files. +func (l *Lock) RefreshStaleLock(ctx context.Context) error { + debug.Log("refreshing stale lock %v", l.lockID) + // refreshing a stale lock is possible if it still exists and continues to do + // so until after creating a new lock. The initial check avoids creating a new + // lock file if this lock was already removed in the meantime. + exists, err := l.checkExistence(ctx) + if err != nil { + return err + } else if !exists { + return ErrRemovedLock + } + + l.lock.Lock() + l.Time = time.Now() + l.lock.Unlock() + id, err := l.createLock(ctx) + if err != nil { + return err + } + + time.Sleep(waitBeforeLockCheck) + + exists, err = l.checkExistence(ctx) + if err != nil { + // cleanup replacement lock + _ = l.repo.Backend().Remove(context.TODO(), Handle{Type: LockFile, Name: id.String()}) + return err + } + + l.lock.Lock() + defer l.lock.Unlock() + + if !exists { + // cleanup replacement lock + _ = l.repo.Backend().Remove(context.TODO(), Handle{Type: LockFile, Name: id.String()}) + return ErrRemovedLock + } + + debug.Log("new lock ID %v", id) + oldLockID := l.lockID + l.lockID = &id + + return l.repo.Backend().Remove(context.TODO(), Handle{Type: LockFile, Name: oldLockID.String()}) +} + +func (l *Lock) checkExistence(ctx context.Context) (bool, error) { + l.lock.Lock() + defer l.lock.Unlock() + + exists := false + + err := l.repo.Backend().List(ctx, LockFile, func(fi FileInfo) error { + if fi.Name == l.lockID.String() { + exists = true + } + return nil + }) + + return exists, err +} + func (l *Lock) String() string { l.lock.Lock() defer l.lock.Unlock() From 11eb88a2ea6a871095db4a1d607ea292013e1ca7 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 17 Jun 2023 19:13:05 +0200 Subject: [PATCH 02/18] lock: cleanup --- cmd/restic/lock.go | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/cmd/restic/lock.go b/cmd/restic/lock.go index 4bac4794f..95fcd0c65 100644 --- a/cmd/restic/lock.go +++ b/cmd/restic/lock.go @@ -12,6 +12,7 @@ import ( ) type lockContext struct { + lock *restic.Lock cancel context.CancelFunc refreshWG sync.WaitGroup } @@ -104,6 +105,7 @@ retryLoop: ctx, cancel := context.WithCancel(ctx) lockInfo := &lockContext{ + lock: lock, cancel: cancel, } lockInfo.refreshWG.Add(2) @@ -112,8 +114,8 @@ retryLoop: globalLocks.Lock() globalLocks.locks[lock] = lockInfo - go refreshLocks(ctx, lock, lockInfo, refreshChan, forcedRefreshChan) - go monitorLockRefresh(ctx, lock, lockInfo, refreshChan, forcedRefreshChan) + go refreshLocks(ctx, lockInfo, refreshChan, forcedRefreshChan) + go monitorLockRefresh(ctx, lockInfo, refreshChan, forcedRefreshChan) globalLocks.Unlock() return lock, ctx, err @@ -125,8 +127,9 @@ var refreshInterval = 5 * time.Minute // the difference allows to compensate for a small time drift between clients. var refreshabilityTimeout = restic.StaleLockTimeout - refreshInterval*3/2 -func refreshLocks(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, refreshed chan<- struct{}, forcedRefresh <-chan struct{}) { +func refreshLocks(ctx context.Context, lockInfo *lockContext, refreshed chan<- struct{}, forcedRefresh <-chan struct{}) { debug.Log("start") + lock := lockInfo.lock ticker := time.NewTicker(refreshInterval) lastRefresh := lock.Time @@ -177,7 +180,7 @@ func refreshLocks(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, } } -func monitorLockRefresh(ctx context.Context, lock *restic.Lock, lockInfo *lockContext, refreshed <-chan struct{}, forcedRefresh chan<- struct{}) { +func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <-chan struct{}, forcedRefresh chan<- struct{}) { // time.Now() might use a monotonic timer which is paused during standby // convert to unix time to ensure we compare real time values lastRefresh := time.Now().UnixNano() @@ -209,7 +212,7 @@ func monitorLockRefresh(ctx context.Context, lock *restic.Lock, lockInfo *lockCo } // keep on going if our current lock still exists - if tryRefreshStaleLock(ctx, lock) { + if tryRefreshStaleLock(ctx, lockInfo.lock) { lastRefresh = time.Now().UnixNano() // inform refresh gorountine about forced refresh From 51718ec5612b773c8b0bf8c0a79a7a5bb73ca54c Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 17 Jun 2023 19:23:55 +0200 Subject: [PATCH 03/18] restic: generic implemention of AsBackend --- internal/migrations/s3_layout.go | 21 ++------------- internal/migrations/s3_layout_test.go | 27 ------------------- internal/restic/backend.go | 17 ++++++++++++ internal/restic/backend_test.go | 38 +++++++++++++++++++++++++++ 4 files changed, 57 insertions(+), 46 deletions(-) delete mode 100644 internal/migrations/s3_layout_test.go create mode 100644 internal/restic/backend_test.go diff --git a/internal/migrations/s3_layout.go b/internal/migrations/s3_layout.go index 78d2492d8..9effaee70 100644 --- a/internal/migrations/s3_layout.go +++ b/internal/migrations/s3_layout.go @@ -21,26 +21,9 @@ func init() { // "default" layout. type S3Layout struct{} -func toS3Backend(b restic.Backend) *s3.Backend { - for b != nil { - if be, ok := b.(*s3.Backend); ok { - return be - } - - if be, ok := b.(restic.BackendUnwrapper); ok { - b = be.Unwrap() - } else { - // not the backend we're looking for - break - } - } - debug.Log("backend is not s3") - return nil -} - // Check tests whether the migration can be applied. func (m *S3Layout) Check(_ context.Context, repo restic.Repository) (bool, string, error) { - be := toS3Backend(repo.Backend()) + be := restic.AsBackend[*s3.Backend](repo.Backend()) if be == nil { debug.Log("backend is not s3") return false, "backend is not s3", nil @@ -92,7 +75,7 @@ func (m *S3Layout) moveFiles(ctx context.Context, be *s3.Backend, l layout.Layou // Apply runs the migration. func (m *S3Layout) Apply(ctx context.Context, repo restic.Repository) error { - be := toS3Backend(repo.Backend()) + be := restic.AsBackend[*s3.Backend](repo.Backend()) if be == nil { debug.Log("backend is not s3") return errors.New("backend is not s3") diff --git a/internal/migrations/s3_layout_test.go b/internal/migrations/s3_layout_test.go deleted file mode 100644 index ad0eedea6..000000000 --- a/internal/migrations/s3_layout_test.go +++ /dev/null @@ -1,27 +0,0 @@ -package migrations - -import ( - "testing" - - "github.com/restic/restic/internal/backend/mock" - "github.com/restic/restic/internal/backend/s3" - "github.com/restic/restic/internal/cache" - "github.com/restic/restic/internal/test" -) - -func TestS3UnwrapBackend(t *testing.T) { - // toS3Backend(b restic.Backend) *s3.Backend - - m := mock.NewBackend() - test.Assert(t, toS3Backend(m) == nil, "mock backend is not an s3 backend") - - // uninitialized fake backend for testing - s3 := &s3.Backend{} - test.Assert(t, toS3Backend(s3) == s3, "s3 was not returned") - - c := &cache.Backend{Backend: s3} - test.Assert(t, toS3Backend(c) == s3, "failed to unwrap s3 backend") - - c.Backend = m - test.Assert(t, toS3Backend(c) == nil, "a wrapped mock backend is not an s3 backend") -} diff --git a/internal/restic/backend.go b/internal/restic/backend.go index 555b9d96e..58aab1f3e 100644 --- a/internal/restic/backend.go +++ b/internal/restic/backend.go @@ -75,6 +75,23 @@ type BackendUnwrapper interface { Unwrap() Backend } +func AsBackend[B Backend](b Backend) B { + for b != nil { + if be, ok := b.(B); ok { + return be + } + + if be, ok := b.(BackendUnwrapper); ok { + b = be.Unwrap() + } else { + // not the backend we're looking for + break + } + } + var be B + return be +} + // FileInfo is contains information about a file in the backend. type FileInfo struct { Size int64 diff --git a/internal/restic/backend_test.go b/internal/restic/backend_test.go new file mode 100644 index 000000000..a970eb5b3 --- /dev/null +++ b/internal/restic/backend_test.go @@ -0,0 +1,38 @@ +package restic_test + +import ( + "testing" + + "github.com/restic/restic/internal/restic" + "github.com/restic/restic/internal/test" +) + +type testBackend struct { + restic.Backend +} + +func (t *testBackend) Unwrap() restic.Backend { + return nil +} + +type otherTestBackend struct { + restic.Backend +} + +func (t *otherTestBackend) Unwrap() restic.Backend { + return t.Backend +} + +func TestAsBackend(t *testing.T) { + other := otherTestBackend{} + test.Assert(t, restic.AsBackend[*testBackend](other) == nil, "otherTestBackend is not a testBackend backend") + + testBe := &testBackend{} + test.Assert(t, restic.AsBackend[*testBackend](testBe) == testBe, "testBackend was not returned") + + wrapper := &otherTestBackend{Backend: testBe} + test.Assert(t, restic.AsBackend[*testBackend](wrapper) == testBe, "failed to unwrap testBackend backend") + + wrapper.Backend = other + test.Assert(t, restic.AsBackend[*testBackend](wrapper) == nil, "a wrapped otherTestBackend is not a testBackend") +} From f490288738e73b7bfa306f7ebe1644c8c386a48f Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 17 Jun 2023 19:36:56 +0200 Subject: [PATCH 04/18] lock: freeze backend operations while refreshing stale lock Freeze new backend operations while trying to refresh a stale lock. --- cmd/restic/lock.go | 17 ++++++++++++---- internal/backend/sema/backend.go | 34 +++++++++++++++++++++++++++++++- internal/restic/backend.go | 8 ++++++++ 3 files changed, 54 insertions(+), 5 deletions(-) diff --git a/cmd/restic/lock.go b/cmd/restic/lock.go index 95fcd0c65..a19014ae9 100644 --- a/cmd/restic/lock.go +++ b/cmd/restic/lock.go @@ -115,7 +115,7 @@ retryLoop: globalLocks.Lock() globalLocks.locks[lock] = lockInfo go refreshLocks(ctx, lockInfo, refreshChan, forcedRefreshChan) - go monitorLockRefresh(ctx, lockInfo, refreshChan, forcedRefreshChan) + go monitorLockRefresh(ctx, repo.Backend(), lockInfo, refreshChan, forcedRefreshChan) globalLocks.Unlock() return lock, ctx, err @@ -180,7 +180,7 @@ func refreshLocks(ctx context.Context, lockInfo *lockContext, refreshed chan<- s } } -func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <-chan struct{}, forcedRefresh chan<- struct{}) { +func monitorLockRefresh(ctx context.Context, backend restic.Backend, lockInfo *lockContext, refreshed <-chan struct{}, forcedRefresh chan<- struct{}) { // time.Now() might use a monotonic timer which is paused during standby // convert to unix time to ensure we compare real time values lastRefresh := time.Now().UnixNano() @@ -212,7 +212,7 @@ func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <- } // keep on going if our current lock still exists - if tryRefreshStaleLock(ctx, lockInfo.lock) { + if tryRefreshStaleLock(ctx, backend, lockInfo.lock, lockInfo.cancel) { lastRefresh = time.Now().UnixNano() // inform refresh gorountine about forced refresh @@ -229,10 +229,19 @@ func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <- } } -func tryRefreshStaleLock(ctx context.Context, lock *restic.Lock) bool { +func tryRefreshStaleLock(ctx context.Context, backend restic.Backend, lock *restic.Lock, cancel context.CancelFunc) bool { + freeze := restic.AsBackend[restic.FreezeBackend](backend) + if freeze != nil { + debug.Log("freezing backend") + freeze.Freeze() + defer freeze.Unfreeze() + } + err := lock.RefreshStaleLock(ctx) if err != nil { Warnf("failed to refresh stale lock: %v\n", err) + // cancel context while the backend is still frozen to prevent accidental modifications + cancel() return false } diff --git a/internal/backend/sema/backend.go b/internal/backend/sema/backend.go index dd4859ed1..d60788f26 100644 --- a/internal/backend/sema/backend.go +++ b/internal/backend/sema/backend.go @@ -3,6 +3,7 @@ package sema import ( "context" "io" + "sync" "github.com/cenkalti/backoff/v4" "github.com/restic/restic/internal/errors" @@ -15,7 +16,8 @@ var _ restic.Backend = &connectionLimitedBackend{} // connectionLimitedBackend limits the number of concurrent operations. type connectionLimitedBackend struct { restic.Backend - sem semaphore + sem semaphore + freezeLock sync.Mutex } // NewBackend creates a backend that limits the concurrent operations on the underlying backend @@ -39,9 +41,23 @@ func (be *connectionLimitedBackend) typeDependentLimit(t restic.FileType) func() return func() {} } be.sem.GetToken() + // prevent token usage while the backend is frozen + be.freezeLock.Lock() + defer be.freezeLock.Unlock() + return be.sem.ReleaseToken } +// Freeze blocks all backend operations except those on lock files +func (be *connectionLimitedBackend) Freeze() { + be.freezeLock.Lock() +} + +// Unfreeze allows all backend operations to continue +func (be *connectionLimitedBackend) Unfreeze() { + be.freezeLock.Unlock() +} + // Save adds new Data to the backend. func (be *connectionLimitedBackend) Save(ctx context.Context, h restic.Handle, rd restic.RewindReader) error { if err := h.Valid(); err != nil { @@ -50,6 +66,10 @@ func (be *connectionLimitedBackend) Save(ctx context.Context, h restic.Handle, r defer be.typeDependentLimit(h.Type)() + if ctx.Err() != nil { + return ctx.Err() + } + return be.Backend.Save(ctx, h, rd) } @@ -68,6 +88,10 @@ func (be *connectionLimitedBackend) Load(ctx context.Context, h restic.Handle, l defer be.typeDependentLimit(h.Type)() + if ctx.Err() != nil { + return ctx.Err() + } + return be.Backend.Load(ctx, h, length, offset, fn) } @@ -79,6 +103,10 @@ func (be *connectionLimitedBackend) Stat(ctx context.Context, h restic.Handle) ( defer be.typeDependentLimit(h.Type)() + if ctx.Err() != nil { + return restic.FileInfo{}, ctx.Err() + } + return be.Backend.Stat(ctx, h) } @@ -90,6 +118,10 @@ func (be *connectionLimitedBackend) Remove(ctx context.Context, h restic.Handle) defer be.typeDependentLimit(h.Type)() + if ctx.Err() != nil { + return ctx.Err() + } + return be.Backend.Remove(ctx, h) } diff --git a/internal/restic/backend.go b/internal/restic/backend.go index 58aab1f3e..df3281641 100644 --- a/internal/restic/backend.go +++ b/internal/restic/backend.go @@ -92,6 +92,14 @@ func AsBackend[B Backend](b Backend) B { return be } +type FreezeBackend interface { + Backend + // Freeze blocks all backend operations except those on lock files + Freeze() + // Unfreeze allows all backend operations to continue + Unfreeze() +} + // FileInfo is contains information about a file in the backend. type FileInfo struct { Size int64 From 05e5e29a8c848576e7d69576688ef81cdd3a9184 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 17 Jun 2023 20:06:42 +0200 Subject: [PATCH 05/18] backup: don't pretend to lock repo for dry run The dry run did not actually create lock files, but still ran all other parts of the lock processing. --- cmd/restic/cmd_backup.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/cmd/restic/cmd_backup.go b/cmd/restic/cmd_backup.go index d7e899eaf..0e678eb2f 100644 --- a/cmd/restic/cmd_backup.go +++ b/cmd/restic/cmd_backup.go @@ -506,10 +506,13 @@ func runBackup(ctx context.Context, opts BackupOptions, gopts GlobalOptions, ter if !gopts.JSON { progressPrinter.V("lock repository") } - lock, ctx, err := lockRepo(ctx, repo, gopts.RetryLock, gopts.JSON) - defer unlockRepo(lock) - if err != nil { - return err + if !opts.DryRun { + var lock *restic.Lock + lock, ctx, err = lockRepo(ctx, repo, gopts.RetryLock, gopts.JSON) + defer unlockRepo(lock) + if err != nil { + return err + } } // rejectByNameFuncs collect functions that can reject items from the backup based on path only From b2fcbc21cb232c2e9bab86b910466115d54b5491 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 17 Jun 2023 21:29:18 +0200 Subject: [PATCH 06/18] lock: rework stale lock refresh to avoid data race --- cmd/restic/lock.go | 52 +++++++++++++++++++++++++++++++++------------- 1 file changed, 37 insertions(+), 15 deletions(-) diff --git a/cmd/restic/lock.go b/cmd/restic/lock.go index a19014ae9..c7fb93a47 100644 --- a/cmd/restic/lock.go +++ b/cmd/restic/lock.go @@ -110,12 +110,12 @@ retryLoop: } lockInfo.refreshWG.Add(2) refreshChan := make(chan struct{}) - forcedRefreshChan := make(chan struct{}) + forceRefreshChan := make(chan refreshLockRequest) globalLocks.Lock() globalLocks.locks[lock] = lockInfo - go refreshLocks(ctx, lockInfo, refreshChan, forcedRefreshChan) - go monitorLockRefresh(ctx, repo.Backend(), lockInfo, refreshChan, forcedRefreshChan) + go refreshLocks(ctx, repo.Backend(), lockInfo, refreshChan, forceRefreshChan) + go monitorLockRefresh(ctx, lockInfo, refreshChan, forceRefreshChan) globalLocks.Unlock() return lock, ctx, err @@ -127,7 +127,11 @@ var refreshInterval = 5 * time.Minute // the difference allows to compensate for a small time drift between clients. var refreshabilityTimeout = restic.StaleLockTimeout - refreshInterval*3/2 -func refreshLocks(ctx context.Context, lockInfo *lockContext, refreshed chan<- struct{}, forcedRefresh <-chan struct{}) { +type refreshLockRequest struct { + result chan bool +} + +func refreshLocks(ctx context.Context, backend restic.Backend, lockInfo *lockContext, refreshed chan<- struct{}, forceRefresh <-chan refreshLockRequest) { debug.Log("start") lock := lockInfo.lock ticker := time.NewTicker(refreshInterval) @@ -154,9 +158,19 @@ func refreshLocks(ctx context.Context, lockInfo *lockContext, refreshed chan<- s debug.Log("terminate") return - case <-forcedRefresh: - // update lock refresh time - lastRefresh = lock.Time + case req := <-forceRefresh: + // keep on going if our current lock still exists + success := tryRefreshStaleLock(ctx, backend, lock, lockInfo.cancel) + // inform refresh goroutine about forced refresh + select { + case <-ctx.Done(): + case req.result <- success: + } + + if success { + // update lock refresh time + lastRefresh = lock.Time + } case <-ticker.C: if time.Since(lastRefresh) > refreshabilityTimeout { @@ -180,7 +194,7 @@ func refreshLocks(ctx context.Context, lockInfo *lockContext, refreshed chan<- s } } -func monitorLockRefresh(ctx context.Context, backend restic.Backend, lockInfo *lockContext, refreshed <-chan struct{}, forcedRefresh chan<- struct{}) { +func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <-chan struct{}, forceRefresh chan<- refreshLockRequest) { // time.Now() might use a monotonic timer which is paused during standby // convert to unix time to ensure we compare real time values lastRefresh := time.Now().UnixNano() @@ -212,14 +226,22 @@ func monitorLockRefresh(ctx context.Context, backend restic.Backend, lockInfo *l } // keep on going if our current lock still exists - if tryRefreshStaleLock(ctx, backend, lockInfo.lock, lockInfo.cancel) { - lastRefresh = time.Now().UnixNano() + refreshReq := refreshLockRequest{ + result: make(chan bool), + } + // inform refresh goroutine about forced refresh + select { + case <-ctx.Done(): + case forceRefresh <- refreshReq: + } + var success bool + select { + case <-ctx.Done(): + case success = <-refreshReq.result: + } - // inform refresh gorountine about forced refresh - select { - case <-ctx.Done(): - case forcedRefresh <- struct{}{}: - } + if success { + lastRefresh = time.Now().UnixNano() continue } From 6a436d731dc1f7726943b8d6885afa999522b198 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sun, 9 Jul 2023 15:15:43 +0200 Subject: [PATCH 07/18] lock: rename test helper function --- cmd/restic/lock_test.go | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/cmd/restic/lock_test.go b/cmd/restic/lock_test.go index 150bd8730..31911c027 100644 --- a/cmd/restic/lock_test.go +++ b/cmd/restic/lock_test.go @@ -13,7 +13,7 @@ import ( "github.com/restic/restic/internal/test" ) -func openTestRepo(t *testing.T, wrapper backendWrapper) (*repository.Repository, func(), *testEnvironment) { +func openLockTestRepo(t *testing.T, wrapper backendWrapper) (*repository.Repository, func(), *testEnvironment) { env, cleanup := withTestEnvironment(t) if wrapper != nil { env.gopts.backendTestHook = wrapper @@ -36,7 +36,7 @@ func checkedLockRepo(ctx context.Context, t *testing.T, repo restic.Repository, } func TestLock(t *testing.T) { - repo, cleanup, env := openTestRepo(t, nil) + repo, cleanup, env := openLockTestRepo(t, nil) defer cleanup() lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, env) @@ -47,7 +47,7 @@ func TestLock(t *testing.T) { } func TestLockCancel(t *testing.T) { - repo, cleanup, env := openTestRepo(t, nil) + repo, cleanup, env := openLockTestRepo(t, nil) defer cleanup() ctx, cancel := context.WithCancel(context.Background()) @@ -63,7 +63,7 @@ func TestLockCancel(t *testing.T) { } func TestLockUnlockAll(t *testing.T) { - repo, cleanup, env := openTestRepo(t, nil) + repo, cleanup, env := openLockTestRepo(t, nil) defer cleanup() lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, env) @@ -78,7 +78,7 @@ func TestLockUnlockAll(t *testing.T) { } func TestLockConflict(t *testing.T) { - repo, cleanup, env := openTestRepo(t, nil) + repo, cleanup, env := openLockTestRepo(t, nil) defer cleanup() repo2, err := OpenRepository(context.TODO(), env.gopts) test.OK(t, err) @@ -107,7 +107,7 @@ func (b *writeOnceBackend) Save(ctx context.Context, h restic.Handle, rd restic. } func TestLockFailedRefresh(t *testing.T) { - repo, cleanup, env := openTestRepo(t, func(r restic.Backend) (restic.Backend, error) { + repo, cleanup, env := openLockTestRepo(t, func(r restic.Backend) (restic.Backend, error) { return &writeOnceBackend{Backend: r}, nil }) defer cleanup() @@ -145,7 +145,7 @@ func (b *loggingBackend) Save(ctx context.Context, h restic.Handle, rd restic.Re } func TestLockSuccessfulRefresh(t *testing.T) { - repo, cleanup, env := openTestRepo(t, func(r restic.Backend) (restic.Backend, error) { + repo, cleanup, env := openLockTestRepo(t, func(r restic.Backend) (restic.Backend, error) { return &loggingBackend{ Backend: r, t: t, @@ -183,7 +183,7 @@ func TestLockSuccessfulRefresh(t *testing.T) { } func TestLockWaitTimeout(t *testing.T) { - repo, cleanup, env := openTestRepo(t, nil) + repo, cleanup, env := openLockTestRepo(t, nil) defer cleanup() elock, _, err := lockRepoExclusive(context.TODO(), repo, env.gopts.RetryLock, env.gopts.JSON) @@ -205,8 +205,9 @@ func TestLockWaitTimeout(t *testing.T) { test.OK(t, lock.Unlock()) test.OK(t, elock.Unlock()) } + func TestLockWaitCancel(t *testing.T) { - repo, cleanup, env := openTestRepo(t, nil) + repo, cleanup, env := openLockTestRepo(t, nil) defer cleanup() elock, _, err := lockRepoExclusive(context.TODO(), repo, env.gopts.RetryLock, env.gopts.JSON) @@ -234,7 +235,7 @@ func TestLockWaitCancel(t *testing.T) { } func TestLockWaitSuccess(t *testing.T) { - repo, cleanup, env := openTestRepo(t, nil) + repo, cleanup, env := openLockTestRepo(t, nil) defer cleanup() elock, _, err := lockRepoExclusive(context.TODO(), repo, env.gopts.RetryLock, env.gopts.JSON) From 399f8e84a11e12ff5b2c9840f8581048829d9a5f Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sun, 9 Jul 2023 15:18:16 +0200 Subject: [PATCH 08/18] lock: use mem backend for locking tests This hopefully fixes the timing related test failures which appear to be caused by very slow fsync calls. --- cmd/restic/lock_test.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/cmd/restic/lock_test.go b/cmd/restic/lock_test.go index 31911c027..91b90db4f 100644 --- a/cmd/restic/lock_test.go +++ b/cmd/restic/lock_test.go @@ -8,6 +8,8 @@ import ( "testing" "time" + "github.com/restic/restic/internal/backend/location" + "github.com/restic/restic/internal/backend/mem" "github.com/restic/restic/internal/repository" "github.com/restic/restic/internal/restic" "github.com/restic/restic/internal/test" @@ -15,6 +17,12 @@ import ( func openLockTestRepo(t *testing.T, wrapper backendWrapper) (*repository.Repository, func(), *testEnvironment) { env, cleanup := withTestEnvironment(t) + + reg := location.NewRegistry() + reg.Register(mem.NewFactory()) + env.gopts.backends = reg + env.gopts.Repo = "mem:" + if wrapper != nil { env.gopts.backendTestHook = wrapper } From 24c8a33da961ef33ac3dbb38ef073ee763580c7a Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sun, 9 Jul 2023 15:35:11 +0200 Subject: [PATCH 09/18] restic: reduce sleeps in lock tests --- internal/restic/lock_test.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/internal/restic/lock_test.go b/internal/restic/lock_test.go index 2d14499bd..9e6bd3083 100644 --- a/internal/restic/lock_test.go +++ b/internal/restic/lock_test.go @@ -16,6 +16,7 @@ import ( func TestLock(t *testing.T) { repo := repository.TestRepository(t) + restic.TestSetLockTimeout(t, 5*time.Millisecond) lock, err := restic.NewLock(context.TODO(), repo) rtest.OK(t, err) @@ -25,6 +26,7 @@ func TestLock(t *testing.T) { func TestDoubleUnlock(t *testing.T) { repo := repository.TestRepository(t) + restic.TestSetLockTimeout(t, 5*time.Millisecond) lock, err := restic.NewLock(context.TODO(), repo) rtest.OK(t, err) @@ -38,6 +40,7 @@ func TestDoubleUnlock(t *testing.T) { func TestMultipleLock(t *testing.T) { repo := repository.TestRepository(t) + restic.TestSetLockTimeout(t, 5*time.Millisecond) lock1, err := restic.NewLock(context.TODO(), repo) rtest.OK(t, err) @@ -63,6 +66,7 @@ func (be *failLockLoadingBackend) Load(ctx context.Context, h restic.Handle, len func TestMultipleLockFailure(t *testing.T) { be := &failLockLoadingBackend{Backend: mem.New()} repo := repository.TestRepositoryWithBackend(t, be, 0) + restic.TestSetLockTimeout(t, 5*time.Millisecond) lock1, err := restic.NewLock(context.TODO(), repo) rtest.OK(t, err) @@ -83,6 +87,7 @@ func TestLockExclusive(t *testing.T) { func TestLockOnExclusiveLockedRepo(t *testing.T) { repo := repository.TestRepository(t) + restic.TestSetLockTimeout(t, 5*time.Millisecond) elock, err := restic.NewExclusiveLock(context.TODO(), repo) rtest.OK(t, err) @@ -99,6 +104,7 @@ func TestLockOnExclusiveLockedRepo(t *testing.T) { func TestExclusiveLockOnLockedRepo(t *testing.T) { repo := repository.TestRepository(t) + restic.TestSetLockTimeout(t, 5*time.Millisecond) elock, err := restic.NewLock(context.TODO(), repo) rtest.OK(t, err) From d4bf7a3cb1cb170f7dd913a08699b9aacd73ad16 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sun, 9 Jul 2023 15:35:37 +0200 Subject: [PATCH 10/18] restic: cleanup lock test --- internal/restic/lock_test.go | 42 ++++++++++++++++++------------------ 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/internal/restic/lock_test.go b/internal/restic/lock_test.go index 9e6bd3083..839ebefc7 100644 --- a/internal/restic/lock_test.go +++ b/internal/restic/lock_test.go @@ -253,15 +253,10 @@ func TestRemoveAllLocks(t *testing.T) { 3, processed) } -func TestLockRefresh(t *testing.T) { - repo := repository.TestRepository(t) - - lock, err := restic.NewLock(context.TODO(), repo) - rtest.OK(t, err) - time0 := lock.Time - +func checkSingleLock(t *testing.T, repo restic.Repository) restic.ID { + t.Helper() var lockID *restic.ID - err = repo.List(context.TODO(), restic.LockFile, func(id restic.ID, size int64) error { + err := repo.List(context.TODO(), restic.LockFile, func(id restic.ID, size int64) error { if lockID != nil { t.Error("more than one lock found") } @@ -271,25 +266,30 @@ func TestLockRefresh(t *testing.T) { if err != nil { t.Fatal(err) } + if lockID == nil { + t.Fatal("no lock found") + } + return *lockID +} + +func TestLockRefresh(t *testing.T) { + repo := repository.TestRepository(t) + restic.TestSetLockTimeout(t, 5*time.Millisecond) + + lock, err := restic.NewLock(context.TODO(), repo) + rtest.OK(t, err) + time0 := lock.Time + + lockID := checkSingleLock(t, repo) time.Sleep(time.Millisecond) rtest.OK(t, lock.Refresh(context.TODO())) - var lockID2 *restic.ID - err = repo.List(context.TODO(), restic.LockFile, func(id restic.ID, size int64) error { - if lockID2 != nil { - t.Error("more than one lock found") - } - lockID2 = &id - return nil - }) - if err != nil { - t.Fatal(err) - } + lockID2 := checkSingleLock(t, repo) - rtest.Assert(t, !lockID.Equal(*lockID2), + rtest.Assert(t, !lockID.Equal(lockID2), "expected a new ID after lock refresh, got the same") - lock2, err := restic.LoadLock(context.TODO(), repo, *lockID2) + lock2, err := restic.LoadLock(context.TODO(), repo, lockID2) rtest.OK(t, err) rtest.Assert(t, lock2.Time.After(time0), "expected a later timestamp after lock refresh") From 5d9b0d894e9dda33b950e599433117678dcebb05 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sun, 9 Jul 2023 15:41:44 +0200 Subject: [PATCH 11/18] lock: add unit test for RefreshStaleLock --- internal/restic/lock_test.go | 31 +++++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/internal/restic/lock_test.go b/internal/restic/lock_test.go index 839ebefc7..f3c405c9c 100644 --- a/internal/restic/lock_test.go +++ b/internal/restic/lock_test.go @@ -272,7 +272,7 @@ func checkSingleLock(t *testing.T, repo restic.Repository) restic.ID { return *lockID } -func TestLockRefresh(t *testing.T) { +func testLockRefresh(t *testing.T, refresh func(lock *restic.Lock) error) { repo := repository.TestRepository(t) restic.TestSetLockTimeout(t, 5*time.Millisecond) @@ -283,7 +283,7 @@ func TestLockRefresh(t *testing.T) { lockID := checkSingleLock(t, repo) time.Sleep(time.Millisecond) - rtest.OK(t, lock.Refresh(context.TODO())) + rtest.OK(t, refresh(lock)) lockID2 := checkSingleLock(t, repo) @@ -295,3 +295,30 @@ func TestLockRefresh(t *testing.T) { "expected a later timestamp after lock refresh") rtest.OK(t, lock.Unlock()) } + +func TestLockRefresh(t *testing.T) { + testLockRefresh(t, func(lock *restic.Lock) error { + return lock.Refresh(context.TODO()) + }) +} + +func TestLockRefreshStale(t *testing.T) { + testLockRefresh(t, func(lock *restic.Lock) error { + return lock.RefreshStaleLock(context.TODO()) + }) +} + +func TestLockRefreshStaleMissing(t *testing.T) { + repo := repository.TestRepository(t) + restic.TestSetLockTimeout(t, 5*time.Millisecond) + + lock, err := restic.NewLock(context.TODO(), repo) + rtest.OK(t, err) + lockID := checkSingleLock(t, repo) + + // refresh must fail if lock was removed + rtest.OK(t, repo.Backend().Remove(context.TODO(), restic.Handle{Type: restic.LockFile, Name: lockID.String()})) + time.Sleep(time.Millisecond) + err = lock.RefreshStaleLock(context.TODO()) + rtest.Assert(t, err == restic.ErrRemovedLock, "unexpected error, expected %v, got %v", restic.ErrRemovedLock, err) +} From 2dd6769429534c21f62e3f68ecd16e39a73538a0 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 15 Jul 2023 15:28:02 +0200 Subject: [PATCH 12/18] lock: Fix possible deadlock during refresh of stale lock A delayed lock refresh could send a signal on the `refreshed` channel while the `monitorLockRefresh` goroutine waits for a reply to its `refreshLockRequest`. As the channels are unbuffered, this resulted in a deadlock. --- cmd/restic/lock.go | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/cmd/restic/lock.go b/cmd/restic/lock.go index c7fb93a47..e1466a902 100644 --- a/cmd/restic/lock.go +++ b/cmd/restic/lock.go @@ -213,15 +213,21 @@ func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <- lockInfo.refreshWG.Done() }() + var refreshStaleLockResult chan bool + for { select { case <-ctx.Done(): debug.Log("terminate expiry monitoring") return case <-refreshed: + if refreshStaleLockResult != nil { + // ignore delayed refresh notifications while the stale lock is refreshed + continue + } lastRefresh = time.Now().UnixNano() case <-ticker.C: - if time.Now().UnixNano()-lastRefresh < refreshabilityTimeout.Nanoseconds() { + if time.Now().UnixNano()-lastRefresh < refreshabilityTimeout.Nanoseconds() || refreshStaleLockResult != nil { continue } @@ -229,19 +235,17 @@ func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <- refreshReq := refreshLockRequest{ result: make(chan bool), } + refreshStaleLockResult = refreshReq.result + // inform refresh goroutine about forced refresh select { case <-ctx.Done(): case forceRefresh <- refreshReq: } - var success bool - select { - case <-ctx.Done(): - case success = <-refreshReq.result: - } - + case success := <-refreshStaleLockResult: if success { lastRefresh = time.Now().UnixNano() + refreshStaleLockResult = nil continue } From 1f6883a05c5a398d2e8abaee798fba2b61209c7c Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 15 Jul 2023 18:18:35 +0200 Subject: [PATCH 13/18] lock: test stale lock refresh --- cmd/restic/lock.go | 2 ++ cmd/restic/lock_test.go | 65 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 67 insertions(+) diff --git a/cmd/restic/lock.go b/cmd/restic/lock.go index e1466a902..11c1ed8f5 100644 --- a/cmd/restic/lock.go +++ b/cmd/restic/lock.go @@ -159,6 +159,7 @@ func refreshLocks(ctx context.Context, backend restic.Backend, lockInfo *lockCon return case req := <-forceRefresh: + debug.Log("trying to refresh stale lock") // keep on going if our current lock still exists success := tryRefreshStaleLock(ctx, backend, lock, lockInfo.cancel) // inform refresh goroutine about forced refresh @@ -231,6 +232,7 @@ func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <- continue } + debug.Log("trying to refreshStaleLock") // keep on going if our current lock still exists refreshReq := refreshLockRequest{ result: make(chan bool), diff --git a/cmd/restic/lock_test.go b/cmd/restic/lock_test.go index 91b90db4f..afad6cdab 100644 --- a/cmd/restic/lock_test.go +++ b/cmd/restic/lock_test.go @@ -5,11 +5,13 @@ import ( "fmt" "runtime" "strings" + "sync" "testing" "time" "github.com/restic/restic/internal/backend/location" "github.com/restic/restic/internal/backend/mem" + "github.com/restic/restic/internal/debug" "github.com/restic/restic/internal/repository" "github.com/restic/restic/internal/restic" "github.com/restic/restic/internal/test" @@ -190,6 +192,69 @@ func TestLockSuccessfulRefresh(t *testing.T) { unlockRepo(lock) } +type slowBackend struct { + restic.Backend + m sync.Mutex + sleep time.Duration +} + +func (b *slowBackend) Save(ctx context.Context, h restic.Handle, rd restic.RewindReader) error { + b.m.Lock() + sleep := b.sleep + b.m.Unlock() + time.Sleep(sleep) + return b.Backend.Save(ctx, h, rd) +} + +func TestLockSuccessfulStaleRefresh(t *testing.T) { + var sb *slowBackend + repo, cleanup, env := openLockTestRepo(t, func(r restic.Backend) (restic.Backend, error) { + sb = &slowBackend{Backend: r} + return sb, nil + }) + defer cleanup() + + t.Logf("test for successful lock refresh %v", time.Now()) + // reduce locking intervals to be suitable for testing + ri, rt := refreshInterval, refreshabilityTimeout + refreshInterval = 10 * time.Millisecond + refreshabilityTimeout = 50 * time.Millisecond + defer func() { + refreshInterval, refreshabilityTimeout = ri, rt + }() + + lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, env) + // delay lock refreshing long enough that the lock would expire + sb.m.Lock() + sb.sleep = refreshabilityTimeout + refreshInterval + sb.m.Unlock() + + select { + case <-wrappedCtx.Done(): + // don't call t.Fatal to allow the lock to be properly cleaned up + t.Error("lock refresh failed", time.Now()) + + case <-time.After(refreshabilityTimeout): + } + // reset slow backend + sb.m.Lock() + sb.sleep = 0 + sb.m.Unlock() + debug.Log("normal lock period has expired") + + select { + case <-wrappedCtx.Done(): + // don't call t.Fatal to allow the lock to be properly cleaned up + t.Error("lock refresh failed", time.Now()) + + case <-time.After(3 * refreshabilityTimeout): + // expected lock refresh to work + } + + // unlockRepo should not crash + unlockRepo(lock) +} + func TestLockWaitTimeout(t *testing.T) { repo, cleanup, env := openLockTestRepo(t, nil) defer cleanup() From b8f4267a361cc3a041ef2b1871c308e7314cf1ea Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 15 Jul 2023 18:31:52 +0200 Subject: [PATCH 14/18] lock: add more debug information to flaky test --- cmd/restic/lock_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/restic/lock_test.go b/cmd/restic/lock_test.go index afad6cdab..bf36df858 100644 --- a/cmd/restic/lock_test.go +++ b/cmd/restic/lock_test.go @@ -301,7 +301,7 @@ func TestLockWaitCancel(t *testing.T) { test.Assert(t, strings.Contains(err.Error(), "context canceled"), "create normal lock with exclusively locked repo didn't return the correct error") test.Assert(t, cancelAfter <= duration && duration < retryLock-10*time.Millisecond, - "create normal lock with exclusively locked repo didn't return in time") + "create normal lock with exclusively locked repo didn't return in time, duration %v", duration) test.OK(t, lock.Unlock()) test.OK(t, elock.Unlock()) From 96eada3d5fb9de11807788634a20df23f7a55e12 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 15 Jul 2023 19:01:32 +0200 Subject: [PATCH 15/18] backend/sema: add test for freeze functionality --- internal/backend/sema/backend_test.go | 36 +++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/internal/backend/sema/backend_test.go b/internal/backend/sema/backend_test.go index dc599b7f8..a1dd16187 100644 --- a/internal/backend/sema/backend_test.go +++ b/internal/backend/sema/backend_test.go @@ -3,6 +3,7 @@ package sema_test import ( "context" "io" + "sync" "sync/atomic" "testing" "time" @@ -197,3 +198,38 @@ func TestConcurrencyUnlimitedLockSave(t *testing.T) { } }, unblock, true) } + +func TestFreeze(t *testing.T) { + var counter int64 + m := mock.NewBackend() + m.SaveFn = func(ctx context.Context, h restic.Handle, rd restic.RewindReader) error { + atomic.AddInt64(&counter, 1) + return nil + } + m.ConnectionsFn = func() uint { return 2 } + be := sema.NewBackend(m) + fb := be.(restic.FreezeBackend) + + // Freeze backend + fb.Freeze() + + // Start Save call that should block + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + h := restic.Handle{Type: restic.PackFile, Name: "foobar"} + test.OK(t, be.Save(context.TODO(), h, nil)) + }() + + // check + time.Sleep(1 * time.Millisecond) + val := atomic.LoadInt64(&counter) + test.Assert(t, val == 0, "save call worked despite frozen backend") + + // unfreeze and check that save did complete + fb.Unfreeze() + wg.Wait() + val = atomic.LoadInt64(&counter) + test.Assert(t, val == 1, "save call should have completed") +} From 20e82d1fcfe5d89a25a99ebf4672dd4e49b74455 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sun, 16 Jul 2023 13:10:23 +0200 Subject: [PATCH 16/18] lock: add changelog for stale lock refresh --- changelog/unreleased/issue-4274 | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 changelog/unreleased/issue-4274 diff --git a/changelog/unreleased/issue-4274 b/changelog/unreleased/issue-4274 new file mode 100644 index 000000000..96cb0709d --- /dev/null +++ b/changelog/unreleased/issue-4274 @@ -0,0 +1,10 @@ +Bugfix: Improve lock refresh handling when using standby + +If the restic process was stopped or the host running restic entered standby +during a long running operation such as a backup, this resulted in the +operation failing with `Fatal: failed to refresh lock in time`. We've reworked +the lock refresh such that restic first checks whether it is safe to continue +the current operation and only throws an error if not. + +https://github.com/restic/restic/issues/4274 +https://github.com/restic/restic/pull/4374 From 76253b2a2018c252200528f11ca197afe77185fb Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sun, 16 Jul 2023 15:00:48 +0200 Subject: [PATCH 17/18] Hopefully fix flaky TestLockWaitCancel test --- cmd/restic/lock_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/restic/lock_test.go b/cmd/restic/lock_test.go index bf36df858..2f8420853 100644 --- a/cmd/restic/lock_test.go +++ b/cmd/restic/lock_test.go @@ -289,10 +289,10 @@ func TestLockWaitCancel(t *testing.T) { retryLock := 200 * time.Millisecond cancelAfter := 40 * time.Millisecond + start := time.Now() ctx, cancel := context.WithCancel(context.TODO()) time.AfterFunc(cancelAfter, cancel) - start := time.Now() lock, _, err := lockRepo(ctx, repo, retryLock, env.gopts.JSON) duration := time.Since(start) From 3f63b53090db85a5da27ad24286a8bdb1ab1f2f9 Mon Sep 17 00:00:00 2001 From: Michael Eischer Date: Sat, 22 Jul 2023 23:29:14 +0200 Subject: [PATCH 18/18] lock: Shrink critical section in RefreshStaleLock A cleaning up after a failed lock refresh attempt does not require a mutex. --- internal/restic/lock.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/internal/restic/lock.go b/internal/restic/lock.go index b1584aa19..a65ed6b5c 100644 --- a/internal/restic/lock.go +++ b/internal/restic/lock.go @@ -306,15 +306,15 @@ func (l *Lock) RefreshStaleLock(ctx context.Context) error { return err } - l.lock.Lock() - defer l.lock.Unlock() - if !exists { // cleanup replacement lock _ = l.repo.Backend().Remove(context.TODO(), Handle{Type: LockFile, Name: id.String()}) return ErrRemovedLock } + l.lock.Lock() + defer l.lock.Unlock() + debug.Log("new lock ID %v", id) oldLockID := l.lockID l.lockID = &id