lock: move code to repository package

This commit is contained in:
Michael Eischer 2024-02-24 16:26:29 +01:00
parent 118a69a84b
commit cbb5f89252
3 changed files with 370 additions and 363 deletions

View File

@ -2,26 +2,13 @@ package main
import (
"context"
"fmt"
"sync"
"time"
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/debug"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/repository"
"github.com/restic/restic/internal/restic"
)
type lockContext struct {
lock *restic.Lock
cancel context.CancelFunc
refreshWG sync.WaitGroup
}
var globalLocks struct {
locks map[*restic.Lock]*lockContext
sync.Mutex
sync.Once
}
@ -34,9 +21,20 @@ func internalOpenWithLocked(ctx context.Context, gopts GlobalOptions, dryRun boo
unlock := func() {}
if !dryRun {
var lock *restic.Lock
lock, ctx, err = lockRepository(ctx, repo, exclusive, gopts.RetryLock, gopts.JSON)
// make sure that a repository is unlocked properly and after cancel() was
// called by the cleanup handler in global.go
globalLocks.Do(func() {
AddCleanupHandler(repository.UnlockAll)
})
lock, ctx, err = repository.Lock(ctx, repo, exclusive, gopts.RetryLock, func(msg string) {
if !gopts.JSON {
Verbosef("%s", msg)
}
}, Warnf)
unlock = func() {
unlockRepo(lock)
repository.Unlock(lock)
}
if err != nil {
return nil, nil, nil, err
@ -61,287 +59,3 @@ func openWithAppendLock(ctx context.Context, gopts GlobalOptions, dryRun bool) (
func openWithExclusiveLock(ctx context.Context, gopts GlobalOptions, dryRun bool) (context.Context, *repository.Repository, func(), error) {
return internalOpenWithLocked(ctx, gopts, dryRun, true)
}
var (
retrySleepStart = 5 * time.Second
retrySleepMax = 60 * time.Second
)
func minDuration(a, b time.Duration) time.Duration {
if a <= b {
return a
}
return b
}
// lockRepository wraps the ctx such that it is cancelled when the repository is unlocked
// cancelling the original context also stops the lock refresh
func lockRepository(ctx context.Context, repo restic.Repository, exclusive bool, retryLock time.Duration, json bool) (*restic.Lock, context.Context, error) {
// make sure that a repository is unlocked properly and after cancel() was
// called by the cleanup handler in global.go
globalLocks.Do(func() {
AddCleanupHandler(unlockAll)
})
lockFn := restic.NewLock
if exclusive {
lockFn = restic.NewExclusiveLock
}
var lock *restic.Lock
var err error
retrySleep := minDuration(retrySleepStart, retryLock)
retryMessagePrinted := false
retryTimeout := time.After(retryLock)
retryLoop:
for {
lock, err = lockFn(ctx, repo)
if err != nil && restic.IsAlreadyLocked(err) {
if !retryMessagePrinted {
if !json {
Verbosef("repo already locked, waiting up to %s for the lock\n", retryLock)
}
retryMessagePrinted = true
}
debug.Log("repo already locked, retrying in %v", retrySleep)
retrySleepCh := time.After(retrySleep)
select {
case <-ctx.Done():
return nil, ctx, ctx.Err()
case <-retryTimeout:
debug.Log("repo already locked, timeout expired")
// Last lock attempt
lock, err = lockFn(ctx, repo)
break retryLoop
case <-retrySleepCh:
retrySleep = minDuration(retrySleep*2, retrySleepMax)
}
} else {
// anything else, either a successful lock or another error
break retryLoop
}
}
if restic.IsInvalidLock(err) {
return nil, ctx, errors.Fatalf("%v\n\nthe `unlock --remove-all` command can be used to remove invalid locks. Make sure that no other restic process is accessing the repository when running the command", err)
}
if err != nil {
return nil, ctx, fmt.Errorf("unable to create lock in backend: %w", err)
}
debug.Log("create lock %p (exclusive %v)", lock, exclusive)
ctx, cancel := context.WithCancel(ctx)
lockInfo := &lockContext{
lock: lock,
cancel: cancel,
}
lockInfo.refreshWG.Add(2)
refreshChan := make(chan struct{})
forceRefreshChan := make(chan refreshLockRequest)
globalLocks.Lock()
globalLocks.locks[lock] = lockInfo
go refreshLocks(ctx, repo.Backend(), lockInfo, refreshChan, forceRefreshChan)
go monitorLockRefresh(ctx, lockInfo, refreshChan, forceRefreshChan)
globalLocks.Unlock()
return lock, ctx, err
}
var refreshInterval = 5 * time.Minute
// consider a lock refresh failed a bit before the lock actually becomes stale
// the difference allows to compensate for a small time drift between clients.
var refreshabilityTimeout = restic.StaleLockTimeout - refreshInterval*3/2
type refreshLockRequest struct {
result chan bool
}
func refreshLocks(ctx context.Context, backend backend.Backend, lockInfo *lockContext, refreshed chan<- struct{}, forceRefresh <-chan refreshLockRequest) {
debug.Log("start")
lock := lockInfo.lock
ticker := time.NewTicker(refreshInterval)
lastRefresh := lock.Time
defer func() {
ticker.Stop()
// ensure that the context was cancelled before removing the lock
lockInfo.cancel()
// remove the lock from the repo
debug.Log("unlocking repository with lock %v", lock)
if err := lock.Unlock(); err != nil {
debug.Log("error while unlocking: %v", err)
Warnf("error while unlocking: %v", err)
}
lockInfo.refreshWG.Done()
}()
for {
select {
case <-ctx.Done():
debug.Log("terminate")
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
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 {
// the lock is too old, wait until the expiry monitor cancels the context
continue
}
debug.Log("refreshing locks")
err := lock.Refresh(context.TODO())
if err != nil {
Warnf("unable to refresh lock: %v\n", err)
} else {
lastRefresh = lock.Time
// inform monitor goroutine about successful refresh
select {
case <-ctx.Done():
case refreshed <- 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()
pollDuration := 1 * time.Second
if refreshInterval < pollDuration {
// require for TestLockFailedRefresh
pollDuration = refreshInterval / 5
}
// 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
ticker := time.NewTicker(pollDuration)
defer func() {
ticker.Stop()
lockInfo.cancel()
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() || refreshStaleLockResult != nil {
continue
}
debug.Log("trying to refreshStaleLock")
// keep on going if our current lock still exists
refreshReq := refreshLockRequest{
result: make(chan bool),
}
refreshStaleLockResult = refreshReq.result
// inform refresh goroutine about forced refresh
select {
case <-ctx.Done():
case forceRefresh <- refreshReq:
}
case success := <-refreshStaleLockResult:
if success {
lastRefresh = time.Now().UnixNano()
refreshStaleLockResult = nil
continue
}
Warnf("Fatal: failed to refresh lock in time\n")
return
}
}
}
func tryRefreshStaleLock(ctx context.Context, be backend.Backend, lock *restic.Lock, cancel context.CancelFunc) bool {
freeze := backend.AsBackend[backend.FreezeBackend](be)
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
}
return true
}
func unlockRepo(lock *restic.Lock) {
if lock == nil {
return
}
globalLocks.Lock()
lockInfo, exists := globalLocks.locks[lock]
delete(globalLocks.locks, lock)
globalLocks.Unlock()
if !exists {
debug.Log("unable to find lock %v in the global list of locks, ignoring", lock)
return
}
lockInfo.cancel()
lockInfo.refreshWG.Wait()
}
func unlockAll(code int) (int, error) {
globalLocks.Lock()
locks := globalLocks.locks
debug.Log("unlocking %d locks", len(globalLocks.locks))
for _, lockInfo := range globalLocks.locks {
lockInfo.cancel()
}
globalLocks.locks = make(map[*restic.Lock]*lockContext)
globalLocks.Unlock()
for _, lockInfo := range locks {
lockInfo.refreshWG.Wait()
}
return code, nil
}
func init() {
globalLocks.locks = make(map[*restic.Lock]*lockContext)
}

301
internal/repository/lock.go Normal file
View File

@ -0,0 +1,301 @@
package repository
import (
"context"
"fmt"
"sync"
"time"
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/debug"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/restic"
)
type lockContext struct {
lock *restic.Lock
cancel context.CancelFunc
refreshWG sync.WaitGroup
}
var globalLocks struct {
locks map[*restic.Lock]*lockContext
sync.Mutex
}
var (
retrySleepStart = 5 * time.Second
retrySleepMax = 60 * time.Second
)
func minDuration(a, b time.Duration) time.Duration {
if a <= b {
return a
}
return b
}
// Lock wraps the ctx such that it is cancelled when the repository is unlocked
// cancelling the original context also stops the lock refresh
func Lock(ctx context.Context, repo restic.Repository, exclusive bool, retryLock time.Duration, printRetry func(msg string), logger func(format string, args ...interface{})) (*restic.Lock, context.Context, error) {
lockFn := restic.NewLock
if exclusive {
lockFn = restic.NewExclusiveLock
}
var lock *restic.Lock
var err error
retrySleep := minDuration(retrySleepStart, retryLock)
retryMessagePrinted := false
retryTimeout := time.After(retryLock)
retryLoop:
for {
lock, err = lockFn(ctx, repo)
if err != nil && restic.IsAlreadyLocked(err) {
if !retryMessagePrinted {
printRetry(fmt.Sprintf("repo already locked, waiting up to %s for the lock\n", retryLock))
retryMessagePrinted = true
}
debug.Log("repo already locked, retrying in %v", retrySleep)
retrySleepCh := time.After(retrySleep)
select {
case <-ctx.Done():
return nil, ctx, ctx.Err()
case <-retryTimeout:
debug.Log("repo already locked, timeout expired")
// Last lock attempt
lock, err = lockFn(ctx, repo)
break retryLoop
case <-retrySleepCh:
retrySleep = minDuration(retrySleep*2, retrySleepMax)
}
} else {
// anything else, either a successful lock or another error
break retryLoop
}
}
if restic.IsInvalidLock(err) {
return nil, ctx, errors.Fatalf("%v\n\nthe `unlock --remove-all` command can be used to remove invalid locks. Make sure that no other restic process is accessing the repository when running the command", err)
}
if err != nil {
return nil, ctx, fmt.Errorf("unable to create lock in backend: %w", err)
}
debug.Log("create lock %p (exclusive %v)", lock, exclusive)
ctx, cancel := context.WithCancel(ctx)
lockInfo := &lockContext{
lock: lock,
cancel: cancel,
}
lockInfo.refreshWG.Add(2)
refreshChan := make(chan struct{})
forceRefreshChan := make(chan refreshLockRequest)
globalLocks.Lock()
globalLocks.locks[lock] = lockInfo
go refreshLocks(ctx, repo.Backend(), lockInfo, refreshChan, forceRefreshChan, logger)
go monitorLockRefresh(ctx, lockInfo, refreshChan, forceRefreshChan, logger)
globalLocks.Unlock()
return lock, ctx, err
}
var refreshInterval = 5 * time.Minute
// consider a lock refresh failed a bit before the lock actually becomes stale
// the difference allows to compensate for a small time drift between clients.
var refreshabilityTimeout = restic.StaleLockTimeout - refreshInterval*3/2
type refreshLockRequest struct {
result chan bool
}
func refreshLocks(ctx context.Context, backend backend.Backend, lockInfo *lockContext, refreshed chan<- struct{}, forceRefresh <-chan refreshLockRequest, logger func(format string, args ...interface{})) {
debug.Log("start")
lock := lockInfo.lock
ticker := time.NewTicker(refreshInterval)
lastRefresh := lock.Time
defer func() {
ticker.Stop()
// ensure that the context was cancelled before removing the lock
lockInfo.cancel()
// remove the lock from the repo
debug.Log("unlocking repository with lock %v", lock)
if err := lock.Unlock(); err != nil {
debug.Log("error while unlocking: %v", err)
logger("error while unlocking: %v", err)
}
lockInfo.refreshWG.Done()
}()
for {
select {
case <-ctx.Done():
debug.Log("terminate")
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, logger)
// 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 {
// the lock is too old, wait until the expiry monitor cancels the context
continue
}
debug.Log("refreshing locks")
err := lock.Refresh(context.TODO())
if err != nil {
logger("unable to refresh lock: %v\n", err)
} else {
lastRefresh = lock.Time
// inform monitor goroutine about successful refresh
select {
case <-ctx.Done():
case refreshed <- struct{}{}:
}
}
}
}
}
func monitorLockRefresh(ctx context.Context, lockInfo *lockContext, refreshed <-chan struct{}, forceRefresh chan<- refreshLockRequest, logger func(format string, args ...interface{})) {
// 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()
pollDuration := 1 * time.Second
if refreshInterval < pollDuration {
// required for TestLockFailedRefresh
pollDuration = refreshInterval / 5
}
// 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
ticker := time.NewTicker(pollDuration)
defer func() {
ticker.Stop()
lockInfo.cancel()
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() || refreshStaleLockResult != nil {
continue
}
debug.Log("trying to refreshStaleLock")
// keep on going if our current lock still exists
refreshReq := refreshLockRequest{
result: make(chan bool),
}
refreshStaleLockResult = refreshReq.result
// inform refresh goroutine about forced refresh
select {
case <-ctx.Done():
case forceRefresh <- refreshReq:
}
case success := <-refreshStaleLockResult:
if success {
lastRefresh = time.Now().UnixNano()
refreshStaleLockResult = nil
continue
}
logger("Fatal: failed to refresh lock in time\n")
return
}
}
}
func tryRefreshStaleLock(ctx context.Context, be backend.Backend, lock *restic.Lock, cancel context.CancelFunc, logger func(format string, args ...interface{})) bool {
freeze := backend.AsBackend[backend.FreezeBackend](be)
if freeze != nil {
debug.Log("freezing backend")
freeze.Freeze()
defer freeze.Unfreeze()
}
err := lock.RefreshStaleLock(ctx)
if err != nil {
logger("failed to refresh stale lock: %v\n", err)
// cancel context while the backend is still frozen to prevent accidental modifications
cancel()
return false
}
return true
}
func Unlock(lock *restic.Lock) {
if lock == nil {
return
}
globalLocks.Lock()
lockInfo, exists := globalLocks.locks[lock]
delete(globalLocks.locks, lock)
globalLocks.Unlock()
if !exists {
debug.Log("unable to find lock %v in the global list of locks, ignoring", lock)
return
}
lockInfo.cancel()
lockInfo.refreshWG.Wait()
}
func UnlockAll(code int) (int, error) {
globalLocks.Lock()
locks := globalLocks.locks
debug.Log("unlocking %d locks", len(globalLocks.locks))
for _, lockInfo := range globalLocks.locks {
lockInfo.cancel()
}
globalLocks.locks = make(map[*restic.Lock]*lockContext)
globalLocks.Unlock()
for _, lockInfo := range locks {
lockInfo.refreshWG.Wait()
}
return code, nil
}
func init() {
globalLocks.locks = make(map[*restic.Lock]*lockContext)
}

View File

@ -1,4 +1,4 @@
package main
package repository
import (
"context"
@ -10,34 +10,35 @@ import (
"time"
"github.com/restic/restic/internal/backend"
"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"
rtest "github.com/restic/restic/internal/test"
)
func openLockTestRepo(t *testing.T, wrapper backendWrapper) (*repository.Repository, func(), *testEnvironment) {
env, cleanup := withTestEnvironment(t)
type backendWrapper func(r backend.Backend) (backend.Backend, error)
reg := location.NewRegistry()
reg.Register(mem.NewFactory())
env.gopts.backends = reg
env.gopts.Repo = "mem:"
func openLockTestRepo(t *testing.T, wrapper backendWrapper) restic.Repository {
be := backend.Backend(mem.New())
// initialize repo
TestRepositoryWithBackend(t, be, 0, Options{})
// reopen repository to allow injecting a backend wrapper
if wrapper != nil {
env.gopts.backendTestHook = wrapper
var err error
be, err = wrapper(be)
rtest.OK(t, err)
}
testRunInit(t, env.gopts)
repo, err := OpenRepository(context.TODO(), env.gopts)
test.OK(t, err)
return repo, cleanup, env
repo, err := New(be, Options{})
rtest.OK(t, err)
rtest.OK(t, repo.SearchKey(context.TODO(), test.TestPassword, 1, ""))
return repo
}
func checkedLockRepo(ctx context.Context, t *testing.T, repo restic.Repository, env *testEnvironment) (*restic.Lock, context.Context) {
lock, wrappedCtx, err := lockRepository(ctx, repo, false, env.gopts.RetryLock, env.gopts.JSON)
func checkedLockRepo(ctx context.Context, t *testing.T, repo restic.Repository, retryLock time.Duration) (*restic.Lock, context.Context) {
lock, wrappedCtx, err := Lock(ctx, repo, false, retryLock, func(msg string) {}, func(format string, args ...interface{}) {})
test.OK(t, err)
test.OK(t, wrappedCtx.Err())
if lock.Stale() {
@ -47,57 +48,54 @@ func checkedLockRepo(ctx context.Context, t *testing.T, repo restic.Repository,
}
func TestLock(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, nil)
defer cleanup()
repo := openLockTestRepo(t, nil)
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, env)
unlockRepo(lock)
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, 0)
Unlock(lock)
if wrappedCtx.Err() == nil {
t.Fatal("unlock did not cancel context")
}
}
func TestLockCancel(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, nil)
defer cleanup()
repo := openLockTestRepo(t, nil)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
lock, wrappedCtx := checkedLockRepo(ctx, t, repo, env)
lock, wrappedCtx := checkedLockRepo(ctx, t, repo, 0)
cancel()
if wrappedCtx.Err() == nil {
t.Fatal("canceled parent context did not cancel context")
}
// unlockRepo should not crash
unlockRepo(lock)
// Unlock should not crash
Unlock(lock)
}
func TestLockUnlockAll(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, nil)
defer cleanup()
repo := openLockTestRepo(t, nil)
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, env)
_, err := unlockAll(0)
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, 0)
_, err := UnlockAll(0)
test.OK(t, err)
if wrappedCtx.Err() == nil {
t.Fatal("canceled parent context did not cancel context")
}
// unlockRepo should not crash
unlockRepo(lock)
// Unlock should not crash
Unlock(lock)
}
func TestLockConflict(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, nil)
defer cleanup()
repo2, err := OpenRepository(context.TODO(), env.gopts)
repo := openLockTestRepo(t, nil)
repo2, err := New(repo.Backend(), Options{})
test.OK(t, err)
test.OK(t, repo2.SearchKey(context.TODO(), test.TestPassword, 1, ""))
lock, _, err := lockRepository(context.Background(), repo, true, env.gopts.RetryLock, env.gopts.JSON)
lock, _, err := Lock(context.Background(), repo, true, 0, func(msg string) {}, func(format string, args ...interface{}) {})
test.OK(t, err)
defer unlockRepo(lock)
_, _, err = lockRepository(context.Background(), repo2, false, env.gopts.RetryLock, env.gopts.JSON)
defer Unlock(lock)
_, _, err = Lock(context.Background(), repo2, false, 0, func(msg string) {}, func(format string, args ...interface{}) {})
if err == nil {
t.Fatal("second lock should have failed")
}
@ -118,10 +116,9 @@ func (b *writeOnceBackend) Save(ctx context.Context, h backend.Handle, rd backen
}
func TestLockFailedRefresh(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, func(r backend.Backend) (backend.Backend, error) {
repo := openLockTestRepo(t, func(r backend.Backend) (backend.Backend, error) {
return &writeOnceBackend{Backend: r}, nil
})
defer cleanup()
// reduce locking intervals to be suitable for testing
ri, rt := refreshInterval, refreshabilityTimeout
@ -131,7 +128,7 @@ func TestLockFailedRefresh(t *testing.T) {
refreshInterval, refreshabilityTimeout = ri, rt
}()
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, env)
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, 0)
select {
case <-wrappedCtx.Done():
@ -139,8 +136,8 @@ func TestLockFailedRefresh(t *testing.T) {
case <-time.After(time.Second):
t.Fatal("failed lock refresh did not cause context cancellation")
}
// unlockRepo should not crash
unlockRepo(lock)
// Unlock should not crash
Unlock(lock)
}
type loggingBackend struct {
@ -156,13 +153,12 @@ func (b *loggingBackend) Save(ctx context.Context, h backend.Handle, rd backend.
}
func TestLockSuccessfulRefresh(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, func(r backend.Backend) (backend.Backend, error) {
repo := openLockTestRepo(t, func(r backend.Backend) (backend.Backend, error) {
return &loggingBackend{
Backend: r,
t: t,
}, nil
})
defer cleanup()
t.Logf("test for successful lock refresh %v", time.Now())
// reduce locking intervals to be suitable for testing
@ -173,7 +169,7 @@ func TestLockSuccessfulRefresh(t *testing.T) {
refreshInterval, refreshabilityTimeout = ri, rt
}()
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, env)
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, 0)
select {
case <-wrappedCtx.Done():
@ -189,8 +185,8 @@ func TestLockSuccessfulRefresh(t *testing.T) {
case <-time.After(2 * refreshabilityTimeout):
// expected lock refresh to work
}
// unlockRepo should not crash
unlockRepo(lock)
// Unlock should not crash
Unlock(lock)
}
type slowBackend struct {
@ -209,11 +205,10 @@ func (b *slowBackend) Save(ctx context.Context, h backend.Handle, rd backend.Rew
func TestLockSuccessfulStaleRefresh(t *testing.T) {
var sb *slowBackend
repo, cleanup, env := openLockTestRepo(t, func(r backend.Backend) (backend.Backend, error) {
repo := openLockTestRepo(t, func(r backend.Backend) (backend.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
@ -224,7 +219,7 @@ func TestLockSuccessfulStaleRefresh(t *testing.T) {
refreshInterval, refreshabilityTimeout = ri, rt
}()
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, env)
lock, wrappedCtx := checkedLockRepo(context.Background(), t, repo, 0)
// delay lock refreshing long enough that the lock would expire
sb.m.Lock()
sb.sleep = refreshabilityTimeout + refreshInterval
@ -252,21 +247,20 @@ func TestLockSuccessfulStaleRefresh(t *testing.T) {
// expected lock refresh to work
}
// unlockRepo should not crash
unlockRepo(lock)
// Unlock should not crash
Unlock(lock)
}
func TestLockWaitTimeout(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, nil)
defer cleanup()
repo := openLockTestRepo(t, nil)
elock, _, err := lockRepository(context.TODO(), repo, true, env.gopts.RetryLock, env.gopts.JSON)
elock, _, err := Lock(context.TODO(), repo, true, 0, func(msg string) {}, func(format string, args ...interface{}) {})
test.OK(t, err)
retryLock := 200 * time.Millisecond
start := time.Now()
lock, _, err := lockRepository(context.TODO(), repo, false, retryLock, env.gopts.JSON)
lock, _, err := Lock(context.TODO(), repo, false, retryLock, func(msg string) {}, func(format string, args ...interface{}) {})
duration := time.Since(start)
test.Assert(t, err != nil,
@ -281,10 +275,9 @@ func TestLockWaitTimeout(t *testing.T) {
}
func TestLockWaitCancel(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, nil)
defer cleanup()
repo := openLockTestRepo(t, nil)
elock, _, err := lockRepository(context.TODO(), repo, true, env.gopts.RetryLock, env.gopts.JSON)
elock, _, err := Lock(context.TODO(), repo, true, 0, func(msg string) {}, func(format string, args ...interface{}) {})
test.OK(t, err)
retryLock := 200 * time.Millisecond
@ -294,7 +287,7 @@ func TestLockWaitCancel(t *testing.T) {
ctx, cancel := context.WithCancel(context.TODO())
time.AfterFunc(cancelAfter, cancel)
lock, _, err := lockRepository(ctx, repo, false, retryLock, env.gopts.JSON)
lock, _, err := Lock(ctx, repo, false, retryLock, func(msg string) {}, func(format string, args ...interface{}) {})
duration := time.Since(start)
test.Assert(t, err != nil,
@ -309,10 +302,9 @@ func TestLockWaitCancel(t *testing.T) {
}
func TestLockWaitSuccess(t *testing.T) {
repo, cleanup, env := openLockTestRepo(t, nil)
defer cleanup()
repo := openLockTestRepo(t, nil)
elock, _, err := lockRepository(context.TODO(), repo, true, env.gopts.RetryLock, env.gopts.JSON)
elock, _, err := Lock(context.TODO(), repo, true, 0, func(msg string) {}, func(format string, args ...interface{}) {})
test.OK(t, err)
retryLock := 200 * time.Millisecond
@ -322,7 +314,7 @@ func TestLockWaitSuccess(t *testing.T) {
test.OK(t, elock.Unlock())
})
lock, _, err := lockRepository(context.TODO(), repo, false, retryLock, env.gopts.JSON)
lock, _, err := Lock(context.TODO(), repo, false, retryLock, func(msg string) {}, func(format string, args ...interface{}) {})
test.OK(t, err)
test.OK(t, lock.Unlock())