1
0
Fork 0
mirror of https://github.com/chrislusf/seaweedfs synced 2024-06-01 16:22:43 +02:00

avoid too large expiration time

This commit is contained in:
Chris Lu 2024-02-04 09:20:21 -08:00
parent 1b5ba4190c
commit 0a12301b3d
4 changed files with 118 additions and 84 deletions

View file

@ -35,10 +35,10 @@ type LiveLock struct {
filer pb.ServerAddress filer pb.ServerAddress
cancelCh chan struct{} cancelCh chan struct{}
grpcDialOption grpc.DialOption grpcDialOption grpc.DialOption
isLocked bool isLocked bool
self string self string
lc *LockClient lc *LockClient
owner string owner string
} }
// NewShortLivedLock creates a lock with a 5-second duration // NewShortLivedLock creates a lock with a 5-second duration
@ -47,12 +47,12 @@ func (lc *LockClient) NewShortLivedLock(key string, owner string) (lock *LiveLoc
key: key, key: key,
filer: lc.seedFiler, filer: lc.seedFiler,
cancelCh: make(chan struct{}), cancelCh: make(chan struct{}),
expireAtNs: time.Now().Add(5*time.Second).UnixNano(), expireAtNs: time.Now().Add(5 * time.Second).UnixNano(),
grpcDialOption: lc.grpcDialOption, grpcDialOption: lc.grpcDialOption,
self: owner, self: owner,
lc: lc, lc: lc,
} }
lock.retryUntilLocked(5*time.Second) lock.retryUntilLocked(5 * time.Second)
return return
} }
@ -62,7 +62,7 @@ func (lc *LockClient) StartLongLivedLock(key string, owner string, onLockOwnerCh
key: key, key: key,
filer: lc.seedFiler, filer: lc.seedFiler,
cancelCh: make(chan struct{}), cancelCh: make(chan struct{}),
expireAtNs: time.Now().Add(lock_manager.MaxDuration).UnixNano(), expireAtNs: time.Now().Add(lock_manager.LiveLockTTL).UnixNano(),
grpcDialOption: lc.grpcDialOption, grpcDialOption: lc.grpcDialOption,
self: owner, self: owner,
lc: lc, lc: lc,
@ -72,12 +72,12 @@ func (lc *LockClient) StartLongLivedLock(key string, owner string, onLockOwnerCh
lockOwner := "" lockOwner := ""
for { for {
if isLocked { if isLocked {
if err := lock.AttemptToLock(lock_manager.MaxDuration); err != nil { if err := lock.AttemptToLock(lock_manager.LiveLockTTL); err != nil {
glog.V(0).Infof("Lost lock %s: %v", key, err) glog.V(0).Infof("Lost lock %s: %v", key, err)
isLocked = false isLocked = false
} }
} else { } else {
if err := lock.AttemptToLock(lock_manager.MaxDuration); err == nil { if err := lock.AttemptToLock(lock_manager.LiveLockTTL); err == nil {
isLocked = true isLocked = true
} }
} }
@ -90,7 +90,7 @@ func (lc *LockClient) StartLongLivedLock(key string, owner string, onLockOwnerCh
case <-lock.cancelCh: case <-lock.cancelCh:
return return
default: default:
time.Sleep(5*time.Second) time.Sleep(lock_manager.RenewInterval)
} }
} }
}() }()
@ -111,10 +111,12 @@ func (lock *LiveLock) retryUntilLocked(lockDuration time.Duration) {
func (lock *LiveLock) AttemptToLock(lockDuration time.Duration) error { func (lock *LiveLock) AttemptToLock(lockDuration time.Duration) error {
errorMessage, err := lock.doLock(lockDuration) errorMessage, err := lock.doLock(lockDuration)
if err != nil { if err != nil {
glog.Warningf("lock1 %s: %v", lock.key, err)
time.Sleep(time.Second) time.Sleep(time.Second)
return err return err
} }
if errorMessage != "" { if errorMessage != "" {
glog.Warningf("lock2 %s: %v", lock.key, errorMessage)
time.Sleep(time.Second) time.Sleep(time.Second)
return fmt.Errorf("%v", errorMessage) return fmt.Errorf("%v", errorMessage)
} }
@ -123,7 +125,7 @@ func (lock *LiveLock) AttemptToLock(lockDuration time.Duration) error {
} }
func (lock *LiveLock) IsLocked() bool { func (lock *LiveLock) IsLocked() bool {
return lock!=nil && lock.isLocked return lock != nil && lock.isLocked
} }
func (lock *LiveLock) StopShortLivedLock() error { func (lock *LiveLock) StopShortLivedLock() error {
@ -154,8 +156,8 @@ func (lock *LiveLock) doLock(lockDuration time.Duration) (errorMessage string, e
if err == nil && resp != nil { if err == nil && resp != nil {
lock.renewToken = resp.RenewToken lock.renewToken = resp.RenewToken
} else { } else {
// this can be retried. Need to remember the last valid renewToken //this can be retried. Need to remember the last valid renewToken
// lock.renewToken = "" lock.renewToken = ""
} }
if resp != nil { if resp != nil {
errorMessage = resp.Error errorMessage = resp.Error

View file

@ -7,7 +7,8 @@ import (
"time" "time"
) )
const MaxDuration = time.Hour * 24 * 365 * 100 const RenewInterval = time.Second * 3
const LiveLockTTL = time.Second * 7
var NoLockServerError = fmt.Errorf("no lock server found") var NoLockServerError = fmt.Errorf("no lock server found")

View file

@ -3,8 +3,8 @@ package lock_manager
import ( import (
"fmt" "fmt"
"github.com/google/uuid" "github.com/google/uuid"
"github.com/puzpuzpuz/xsync/v2"
"github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/glog"
"sync"
"time" "time"
) )
@ -16,7 +16,8 @@ var LockNotFound = fmt.Errorf("lock not found")
// LockManager local lock manager, used by distributed lock manager // LockManager local lock manager, used by distributed lock manager
type LockManager struct { type LockManager struct {
locks *xsync.MapOf[string, *Lock] locks map[string]*Lock
accessLock sync.RWMutex
} }
type Lock struct { type Lock struct {
Token string Token string
@ -27,125 +28,155 @@ type Lock struct {
func NewLockManager() *LockManager { func NewLockManager() *LockManager {
t := &LockManager{ t := &LockManager{
locks: xsync.NewMapOf[*Lock](), locks: make(map[string]*Lock),
} }
go t.CleanUp() go t.CleanUp()
return t return t
} }
func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner string) (lockOwner, renewToken string, err error) { func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner string) (lockOwner, renewToken string, err error) {
lm.locks.Compute(path, func(oldValue *Lock, loaded bool) (newValue *Lock, delete bool) { lm.accessLock.Lock()
if oldValue != nil { defer lm.accessLock.Unlock()
if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < time.Now().UnixNano() {
// lock is expired, set to a new lock glog.V(4).Infof("lock %s %v %v %v", path, time.Unix(0, expiredAtNs), token, owner)
if token != "" {
err = LockErrorNonEmptyTokenOnExpiredLock if oldValue, found := lm.locks[path]; found {
return nil, false if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < time.Now().UnixNano() {
} else { // lock is expired, set to a new lock
// new lock if token != "" {
renewToken = uuid.New().String() glog.V(4).Infof("lock expired key %s non-empty token %v owner %v ts %s", path, token, owner, time.Unix(0, oldValue.ExpiredAtNs))
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}, false err = LockErrorNonEmptyTokenOnExpiredLock
} return
}
// not expired
lockOwner = oldValue.Owner
if oldValue.Token == token {
// token matches, renew the lock
renewToken = uuid.New().String()
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}, false
} else { } else {
err = LockErrorTokenMismatch // new lock
return oldValue, false renewToken = uuid.New().String()
glog.V(4).Infof("key %s new token %v owner %v", path, renewToken, owner)
lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}
return
} }
}
// not expired
lockOwner = oldValue.Owner
if oldValue.Token == token {
// token matches, renew the lock
renewToken = uuid.New().String()
glog.V(4).Infof("key %s old token %v owner %v => %v owner %v", path, oldValue.Token, oldValue.Owner, renewToken, owner)
lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}
return
} else { } else {
if token == "" { if token == "" {
// new lock // new lock
renewToken = uuid.New().String() glog.V(4).Infof("key %s locked by %v", path, oldValue.Owner)
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}, false err = fmt.Errorf("lock already owned by %v", oldValue.Owner)
} else { return
err = LockErrorNonEmptyTokenOnNewLock
return nil, false
} }
glog.V(4).Infof("key %s expected token %v owner %v received %v from %v", path, oldValue.Token, oldValue.Owner, token, owner)
err = fmt.Errorf("lock: token mismatch")
return
} }
}) } else {
return glog.V(4).Infof("key %s no lock owner %v", path, owner)
if token == "" {
// new lock
glog.V(4).Infof("key %s new token %v owner %v", path, token, owner)
renewToken = uuid.New().String()
lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}
return
} else {
glog.V(4).Infof("key %s non-empty token %v owner %v", path, token, owner)
err = LockErrorNonEmptyTokenOnNewLock
return
}
}
} }
func (lm *LockManager) Unlock(path string, token string) (isUnlocked bool, err error) { func (lm *LockManager) Unlock(path string, token string) (isUnlocked bool, err error) {
lm.locks.Compute(path, func(oldValue *Lock, loaded bool) (newValue *Lock, delete bool) { lm.accessLock.Lock()
if oldValue != nil { defer lm.accessLock.Unlock()
now := time.Now()
if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < now.UnixNano() { if oldValue, found := lm.locks[path]; found {
// lock is expired, delete it now := time.Now()
isUnlocked = true if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < now.UnixNano() {
return nil, true // lock is expired, delete it
}
if oldValue.Token == token {
if oldValue.ExpiredAtNs <= now.UnixNano() {
isUnlocked = true
return nil, true
}
return oldValue, false
} else {
isUnlocked = false
err = UnlockErrorTokenMismatch
return oldValue, false
}
} else {
isUnlocked = true isUnlocked = true
return nil, true glog.V(4).Infof("key %s expired at %v", path, time.Unix(0, oldValue.ExpiredAtNs))
delete(lm.locks, path)
return
} }
}) if oldValue.Token == token {
isUnlocked = true
glog.V(4).Infof("key %s unlocked with %v", path, token)
delete(lm.locks, path)
return
} else {
isUnlocked = false
err = UnlockErrorTokenMismatch
return
}
}
err = LockNotFound
return return
} }
func (lm *LockManager) CleanUp() { func (lm *LockManager) CleanUp() {
for { for {
time.Sleep(1 * time.Minute) time.Sleep(1 * time.Minute)
now := time.Now().UnixNano() now := time.Now().UnixNano()
lm.locks.Range(func(key string, value *Lock) bool {
lm.accessLock.Lock()
for key, value := range lm.locks {
if value == nil { if value == nil {
return true continue
} }
if now > value.ExpiredAtNs { if now > value.ExpiredAtNs {
lm.locks.Delete(key) glog.V(4).Infof("key %s expired at %v", key, time.Unix(0, value.ExpiredAtNs))
return true delete(lm.locks, key)
} }
return true }
}) lm.accessLock.Unlock()
} }
} }
// SelectLocks takes out locks by key // SelectLocks takes out locks by key
// if keyFn return true, the lock will be taken out // if keyFn return true, the lock will be taken out
func (lm *LockManager) SelectLocks(selectFn func(key string) bool) (locks []*Lock) { func (lm *LockManager) SelectLocks(selectFn func(key string) bool) (locks []*Lock) {
lm.accessLock.RLock()
defer lm.accessLock.RUnlock()
now := time.Now().UnixNano() now := time.Now().UnixNano()
lm.locks.Range(func(key string, lock *Lock) bool {
for key, lock := range lm.locks {
if now > lock.ExpiredAtNs { if now > lock.ExpiredAtNs {
lm.locks.Delete(key) glog.V(4).Infof("key %s expired at %v", key, time.Unix(0, lock.ExpiredAtNs))
return true delete(lm.locks, key)
continue
} }
if selectFn(key) { if selectFn(key) {
lm.locks.Delete(key) glog.V(4).Infof("key %s selected and deleted", key)
delete(lm.locks, key)
lock.Key = key lock.Key = key
locks = append(locks, lock) locks = append(locks, lock)
} }
return true }
})
return return
} }
// InsertLock inserts a lock unconditionally // InsertLock inserts a lock unconditionally
func (lm *LockManager) InsertLock(path string, expiredAtNs int64, token string, owner string) { func (lm *LockManager) InsertLock(path string, expiredAtNs int64, token string, owner string) {
lm.locks.Store(path, &Lock{Token: token, ExpiredAtNs: expiredAtNs, Owner: owner}) lm.accessLock.Lock()
defer lm.accessLock.Unlock()
lm.locks[path] = &Lock{Token: token, ExpiredAtNs: expiredAtNs, Owner: owner}
} }
func (lm *LockManager) GetLockOwner(key string) (owner string, err error) { func (lm *LockManager) GetLockOwner(key string) (owner string, err error) {
lock, _ := lm.locks.Load(key) lm.accessLock.RLock()
if lock != nil { defer lm.accessLock.RUnlock()
if lock, found := lm.locks[key]; found {
return lock.Owner, nil return lock.Owner, nil
} }
glog.V(0).Infof("get lock %s %+v", key, lock)
err = LockNotFound err = LockNotFound
return return
} }

View file

@ -95,7 +95,7 @@ func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.Dial
for { for {
time.Sleep(time.Second) time.Sleep(time.Second)
if err := mqBroker.lockAsBalancer.AttemptToLock(lock_manager.MaxDuration); err != nil { if err := mqBroker.lockAsBalancer.AttemptToLock(lock_manager.RenewInterval); err != nil {
glog.V(0).Infof("AttemptToLock: %v", err) glog.V(0).Infof("AttemptToLock: %v", err)
} }
} }