mirror of
https://github.com/octoleo/restic.git
synced 2024-11-11 07:41:03 +00:00
lock: move code to repository package
This commit is contained in:
parent
118a69a84b
commit
cbb5f89252
@ -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
301
internal/repository/lock.go
Normal 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)
|
||||
}
|
@ -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())
|
Loading…
Reference in New Issue
Block a user