fix: add lock ownership to expire locks (#10571)

- Add owner information for expiry, locking, unlocking a resource
- TopLocks returns now locks in quorum by default, provides
  a way to capture stale locks as well with `?stale=true`
- Simplify the quorum handling for locks to avoid from storage
  class, because there were challenges to make it consistent
  across all situations.
- And other tiny simplifications to reset locks.
This commit is contained in:
Harshavardhana
2020-09-25 19:21:52 -07:00
committed by GitHub
parent 66b4a862e0
commit eafa775952
16 changed files with 207 additions and 174 deletions

View File

@@ -336,23 +336,26 @@ func (a adminAPIHandlers) DataUsageInfoHandler(w http.ResponseWriter, r *http.Re
writeSuccessResponseJSON(w, dataUsageInfoJSON)
}
func lriToLockEntry(l lockRequesterInfo, resource, server string) *madmin.LockEntry {
func lriToLockEntry(l lockRequesterInfo, resource, server string, rquorum, wquorum int) *madmin.LockEntry {
entry := &madmin.LockEntry{
Timestamp: l.Timestamp,
Resource: resource,
ServerList: []string{server},
Source: l.Source,
Owner: l.Owner,
ID: l.UID,
}
if l.Writer {
entry.Type = "WRITE"
entry.Quorum = wquorum
} else {
entry.Type = "READ"
entry.Quorum = rquorum
}
return entry
}
func topLockEntries(peerLocks []*PeerLocks, count int) madmin.LockEntries {
func topLockEntries(peerLocks []*PeerLocks, count int, rquorum, wquorum int, stale bool) madmin.LockEntries {
entryMap := make(map[string]*madmin.LockEntry)
for _, peerLock := range peerLocks {
if peerLock == nil {
@@ -364,20 +367,26 @@ func topLockEntries(peerLocks []*PeerLocks, count int) madmin.LockEntries {
if val, ok := entryMap[lockReqInfo.UID]; ok {
val.ServerList = append(val.ServerList, peerLock.Addr)
} else {
entryMap[lockReqInfo.UID] = lriToLockEntry(lockReqInfo, k, peerLock.Addr)
entryMap[lockReqInfo.UID] = lriToLockEntry(lockReqInfo, k, peerLock.Addr, rquorum, wquorum)
}
}
}
}
}
var lockEntries = make(madmin.LockEntries, 0, len(entryMap))
var lockEntries madmin.LockEntries
for _, v := range entryMap {
lockEntries = append(lockEntries, *v)
if len(lockEntries) == count {
break
}
if stale {
lockEntries = append(lockEntries, *v)
continue
}
if len(v.ServerList) >= v.Quorum {
lockEntries = append(lockEntries, *v)
}
}
sort.Sort(lockEntries)
if len(lockEntries) > count {
lockEntries = lockEntries[:count]
}
return lockEntries
}
@@ -407,21 +416,14 @@ func (a adminAPIHandlers) TopLocksHandler(w http.ResponseWriter, r *http.Request
return
}
}
stale := r.URL.Query().Get("stale") == "true" // list also stale locks
peerLocks := globalNotificationSys.GetLocks(ctx)
// Once we have received all the locks currently used from peers
// add the local peer locks list as well.
var getRespLocks GetLocksResp
for _, llocker := range globalLockServers {
getRespLocks = append(getRespLocks, llocker.DupLockMap())
}
peerLocks := globalNotificationSys.GetLocks(ctx, r)
peerLocks = append(peerLocks, &PeerLocks{
Addr: getHostName(r),
Locks: getRespLocks,
})
rquorum := getReadQuorum(objectAPI.SetDriveCount())
wquorum := getWriteQuorum(objectAPI.SetDriveCount())
topLocks := topLockEntries(peerLocks, count)
topLocks := topLockEntries(peerLocks, count, rquorum, wquorum, stale)
// Marshal API response
jsonBytes, err := json.Marshal(topLocks)

View File

@@ -21,6 +21,7 @@ import (
"context"
"encoding/binary"
"errors"
"math/rand"
"os"
"path"
"strconv"
@@ -49,10 +50,6 @@ const (
healDeleteDangling = true
healFolderIncludeProb = 32 // Include a clean folder one in n cycles.
healObjectSelectProb = 512 // Overall probability of a file being scanned; one in n.
// sleep for an hour after a lock timeout
// before retrying to acquire lock again.
dataCrawlerLeaderLockTimeoutSleepInterval = time.Hour
)
var (
@@ -73,10 +70,11 @@ func initDataCrawler(ctx context.Context, objAPI ObjectLayer) {
func runDataCrawler(ctx context.Context, objAPI ObjectLayer) {
// Make sure only 1 crawler is running on the cluster.
locker := objAPI.NewNSLock(ctx, minioMetaBucket, "runDataCrawler.lock")
r := rand.New(rand.NewSource(time.Now().UnixNano()))
for {
err := locker.GetLock(dataCrawlerLeaderLockTimeout)
if err != nil {
time.Sleep(dataCrawlerLeaderLockTimeoutSleepInterval)
time.Sleep(time.Duration(r.Float64() * float64(dataCrawlStartDelay)))
continue
}
break

View File

@@ -66,6 +66,9 @@ type erasureSets struct {
// Distributed locker clients.
erasureLockers setsDsyncLockers
// Distributed lock owner (constant per running instance).
erasureLockOwner string
// List of endpoints provided on the command line.
endpoints Endpoints
@@ -261,11 +264,11 @@ func (s *erasureSets) monitorAndConnectEndpoints(ctx context.Context, monitorInt
}
}
func (s *erasureSets) GetLockers(setIndex int) func() []dsync.NetLocker {
return func() []dsync.NetLocker {
func (s *erasureSets) GetLockers(setIndex int) func() ([]dsync.NetLocker, string) {
return func() ([]dsync.NetLocker, string) {
lockers := make([]dsync.NetLocker, s.setDriveCount)
copy(lockers, s.erasureLockers[setIndex])
return lockers
return lockers, s.erasureLockOwner
}
}
@@ -308,6 +311,7 @@ func newErasureSets(ctx context.Context, endpoints Endpoints, storageDisks []Sto
sets: make([]*erasureObjects, setCount),
erasureDisks: make([][]StorageAPI, setCount),
erasureLockers: make([][]dsync.NetLocker, setCount),
erasureLockOwner: mustGetUUID(),
endpoints: endpoints,
endpointStrings: endpointStrings,
setCount: setCount,

View File

@@ -52,7 +52,7 @@ type erasureObjects struct {
getDisks func() []StorageAPI
// getLockers returns list of remote and local lockers.
getLockers func() []dsync.NetLocker
getLockers func() ([]dsync.NetLocker, string)
// getEndpoints returns list of endpoint strings belonging this set.
// some may be local and some remote.

View File

@@ -32,6 +32,9 @@ type lockRequesterInfo struct {
Timestamp time.Time // Timestamp set at the time of initialization.
TimeLastCheck time.Time // Timestamp for last check of validity of lock.
Source string // Contains line, function and filename reqesting the lock.
// Owner represents the UUID of the owner who originally requested the lock
// useful in expiry.
Owner string
}
// isWriteLock returns whether the lock is a write or read lock.
@@ -73,34 +76,30 @@ func (l *localLocker) canTakeLock(resources ...string) bool {
}
func (l *localLocker) Lock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
select {
case <-ctx.Done():
return false, ctx.Err()
default:
l.mutex.Lock()
defer l.mutex.Unlock()
l.mutex.Lock()
defer l.mutex.Unlock()
if !l.canTakeLock(args.Resources...) {
// Not all locks can be taken on resources,
// reject it completely.
return false, nil
}
// No locks held on the all resources, so claim write
// lock on all resources at once.
for _, resource := range args.Resources {
l.lockMap[resource] = []lockRequesterInfo{
{
Writer: true,
Source: args.Source,
UID: args.UID,
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
},
}
}
return true, nil
if !l.canTakeLock(args.Resources...) {
// Not all locks can be taken on resources,
// reject it completely.
return false, nil
}
// No locks held on the all resources, so claim write
// lock on all resources at once.
for _, resource := range args.Resources {
l.lockMap[resource] = []lockRequesterInfo{
{
Writer: true,
Source: args.Source,
Owner: args.Owner,
UID: args.UID,
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
},
}
}
return true, nil
}
func (l *localLocker) Unlock(args dsync.LockArgs) (reply bool, err error) {
@@ -113,7 +112,7 @@ func (l *localLocker) Unlock(args dsync.LockArgs) (reply bool, err error) {
}
for _, resource := range args.Resources {
lri := l.lockMap[resource]
if !l.removeEntry(resource, args.UID, &lri) {
if !l.removeEntry(resource, args, &lri) {
return false, fmt.Errorf("Unlock unable to find corresponding lock for uid: %s on resource %s", args.UID, resource)
}
}
@@ -124,10 +123,10 @@ func (l *localLocker) Unlock(args dsync.LockArgs) (reply bool, err error) {
// removeEntry based on the uid of the lock message, removes a single entry from the
// lockRequesterInfo array or the whole array from the map (in case of a write lock
// or last read lock)
func (l *localLocker) removeEntry(name, uid string, lri *[]lockRequesterInfo) bool {
func (l *localLocker) removeEntry(name string, args dsync.LockArgs, lri *[]lockRequesterInfo) bool {
// Find correct entry to remove based on uid.
for index, entry := range *lri {
if entry.UID == uid {
if entry.UID == args.UID && entry.Owner == args.Owner {
if len(*lri) == 1 {
// Remove the write lock.
delete(l.lockMap, name)
@@ -145,32 +144,28 @@ func (l *localLocker) removeEntry(name, uid string, lri *[]lockRequesterInfo) bo
}
func (l *localLocker) RLock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
select {
case <-ctx.Done():
return false, ctx.Err()
default:
l.mutex.Lock()
defer l.mutex.Unlock()
lrInfo := lockRequesterInfo{
Writer: false,
Source: args.Source,
UID: args.UID,
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
}
resource := args.Resources[0]
if lri, ok := l.lockMap[resource]; ok {
if reply = !isWriteLock(lri); reply {
// Unless there is a write lock
l.lockMap[resource] = append(l.lockMap[resource], lrInfo)
}
} else {
// No locks held on the given name, so claim (first) read lock
l.lockMap[resource] = []lockRequesterInfo{lrInfo}
reply = true
}
return reply, nil
l.mutex.Lock()
defer l.mutex.Unlock()
lrInfo := lockRequesterInfo{
Writer: false,
Source: args.Source,
Owner: args.Owner,
UID: args.UID,
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
}
resource := args.Resources[0]
if lri, ok := l.lockMap[resource]; ok {
if reply = !isWriteLock(lri); reply {
// Unless there is a write lock
l.lockMap[resource] = append(l.lockMap[resource], lrInfo)
}
} else {
// No locks held on the given name, so claim (first) read lock
l.lockMap[resource] = []lockRequesterInfo{lrInfo}
reply = true
}
return reply, nil
}
func (l *localLocker) RUnlock(args dsync.LockArgs) (reply bool, err error) {
@@ -187,7 +182,7 @@ func (l *localLocker) RUnlock(args dsync.LockArgs) (reply bool, err error) {
// A write-lock is held, cannot release a read lock
return reply, fmt.Errorf("RUnlock attempted on a write locked entity: %s", resource)
}
if !l.removeEntry(resource, args.UID, &lri) {
if !l.removeEntry(resource, args, &lri) {
return false, fmt.Errorf("RUnlock unable to find corresponding read lock for uid: %s", args.UID)
}
return reply, nil
@@ -226,7 +221,7 @@ func (l *localLocker) Expired(ctx context.Context, args dsync.LockArgs) (expired
if lri, ok := l.lockMap[resource]; ok {
// Check whether uid is still active
for _, entry := range lri {
if entry.UID == args.UID {
if entry.UID == args.UID && entry.Owner == args.Owner {
return false, nil
}
}
@@ -244,7 +239,7 @@ func (l *localLocker) removeEntryIfExists(nlrip nameLockRequesterInfoPair) {
// Even if the entry exists, it may not be the same entry which was
// considered as expired, so we simply an attempt to remove it if its
// not possible there is nothing we need to do.
l.removeEntry(nlrip.name, nlrip.lri.UID, &lri)
l.removeEntry(nlrip.name, dsync.LockArgs{Owner: nlrip.lri.Owner, UID: nlrip.lri.UID}, &lri)
}
}

View File

@@ -87,6 +87,7 @@ func (client *lockRESTClient) Close() error {
func (client *lockRESTClient) restCall(ctx context.Context, call string, args dsync.LockArgs) (reply bool, err error) {
values := url.Values{}
values.Set(lockRESTUID, args.UID)
values.Set(lockRESTOwner, args.Owner)
values.Set(lockRESTSource, args.Source)
var buffer bytes.Buffer
for _, resource := range args.Resources {

View File

@@ -34,8 +34,12 @@ const (
lockRESTMethodRUnlock = "/runlock"
lockRESTMethodExpired = "/expired"
// lockRESTOwner represents owner UUID
lockRESTOwner = "owner"
// Unique ID of lock/unlock request.
lockRESTUID = "uid"
// Source contains the line number, function and file name of the code
// on the client node that requested the lock.
lockRESTSource = "source"

View File

@@ -21,6 +21,8 @@ import (
"reflect"
"sync"
"testing"
"github.com/minio/minio/pkg/dsync"
)
// Helper function to create a lock server for testing
@@ -53,12 +55,14 @@ func TestLockRpcServerRemoveEntry(t *testing.T) {
defer os.RemoveAll(testPath)
lockRequesterInfo1 := lockRequesterInfo{
Owner: "owner",
Writer: true,
UID: "0123-4567",
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
}
lockRequesterInfo2 := lockRequesterInfo{
Owner: "owner",
Writer: true,
UID: "89ab-cdef",
Timestamp: UTCNow(),
@@ -73,11 +77,17 @@ func TestLockRpcServerRemoveEntry(t *testing.T) {
lri := locker.ll.lockMap["name"]
// test unknown uid
if locker.ll.removeEntry("name", "unknown-uid", &lri) {
if locker.ll.removeEntry("name", dsync.LockArgs{
Owner: "owner",
UID: "unknown-uid",
}, &lri) {
t.Errorf("Expected %#v, got %#v", false, true)
}
if !locker.ll.removeEntry("name", "0123-4567", &lri) {
if !locker.ll.removeEntry("name", dsync.LockArgs{
Owner: "owner",
UID: "0123-4567",
}, &lri) {
t.Errorf("Expected %#v, got %#v", true, false)
} else {
gotLri := locker.ll.lockMap["name"]
@@ -87,7 +97,10 @@ func TestLockRpcServerRemoveEntry(t *testing.T) {
}
}
if !locker.ll.removeEntry("name", "89ab-cdef", &lri) {
if !locker.ll.removeEntry("name", dsync.LockArgs{
Owner: "owner",
UID: "89ab-cdef",
}, &lri) {
t.Errorf("Expected %#v, got %#v", true, false)
} else {
gotLri := locker.ll.lockMap["name"]

View File

@@ -32,7 +32,7 @@ import (
const (
// Lock maintenance interval.
lockMaintenanceInterval = 1 * time.Minute
lockMaintenanceInterval = 30 * time.Second
// Lock validity check interval.
lockValidityCheckInterval = 2 * time.Minute
@@ -64,6 +64,7 @@ func (l *lockRESTServer) IsValid(w http.ResponseWriter, r *http.Request) bool {
func getLockArgs(r *http.Request) (args dsync.LockArgs, err error) {
args = dsync.LockArgs{
Owner: r.URL.Query().Get(lockRESTOwner),
UID: r.URL.Query().Get(lockRESTUID),
Source: r.URL.Query().Get(lockRESTSource),
}
@@ -246,10 +247,28 @@ func lockMaintenance(ctx context.Context, interval time.Duration) error {
return nil
}
type nlock struct {
locks int
writer bool
}
updateNlocks := func(nlripsMap map[string]nlock, name string, writer bool) {
nlk, ok := nlripsMap[name]
if !ok {
nlripsMap[name] = nlock{
locks: 1,
writer: writer,
}
} else {
nlk.locks++
nlripsMap[name] = nlk
}
}
// Validate if long lived locks are indeed clean.
// Get list of long lived locks to check for staleness.
for lendpoint, nlrips := range getLongLivedLocks(interval) {
nlripsMap := make(map[string]int, len(nlrips))
nlripsMap := make(map[string]nlock, len(nlrips))
for _, nlrip := range nlrips {
// Locks are only held on first zone, make sure that
// we only look for ownership of locks from endpoints
@@ -257,7 +276,7 @@ func lockMaintenance(ctx context.Context, interval time.Duration) error {
for _, endpoint := range globalEndpoints[0].Endpoints {
c := newLockAPI(endpoint)
if !c.IsOnline() {
nlripsMap[nlrip.name]++
updateNlocks(nlripsMap, nlrip.name, nlrip.lri.Writer)
continue
}
@@ -266,18 +285,19 @@ func lockMaintenance(ctx context.Context, interval time.Duration) error {
// Call back to original server verify whether the lock is
// still active (based on name & uid)
expired, err := c.Expired(ctx, dsync.LockArgs{
Owner: nlrip.lri.Owner,
UID: nlrip.lri.UID,
Resources: []string{nlrip.name},
})
cancel()
if err != nil {
nlripsMap[nlrip.name]++
updateNlocks(nlripsMap, nlrip.name, nlrip.lri.Writer)
c.Close()
continue
}
if !expired {
nlripsMap[nlrip.name]++
updateNlocks(nlripsMap, nlrip.name, nlrip.lri.Writer)
}
// Close the connection regardless of the call response.
@@ -285,14 +305,13 @@ func lockMaintenance(ctx context.Context, interval time.Duration) error {
}
// Read locks we assume quorum for be N/2 success
quorum := objAPI.SetDriveCount() / 2
quorum := getReadQuorum(objAPI.SetDriveCount())
if nlrip.lri.Writer {
// For write locks we need N/2+1 success
quorum = objAPI.SetDriveCount()/2 + 1
quorum = getWriteQuorum(objAPI.SetDriveCount())
}
// less than the quorum, we have locks expired.
if nlripsMap[nlrip.name] < quorum {
if nlripsMap[nlrip.name].locks < quorum {
// The lock is no longer active at server that originated
// the lock, attempt to remove the lock.
globalLockServers[lendpoint].mutex.Lock()
@@ -348,7 +367,6 @@ func startLockMaintenance(ctx context.Context) {
// registerLockRESTHandlers - register lock rest router.
func registerLockRESTHandlers(router *mux.Router, endpointZones EndpointZones) {
queries := restQueries(lockRESTUID, lockRESTSource)
for _, ep := range endpointZones {
for _, endpoint := range ep.Endpoints {
if !endpoint.IsLocal {
@@ -361,11 +379,11 @@ func registerLockRESTHandlers(router *mux.Router, endpointZones EndpointZones) {
subrouter := router.PathPrefix(path.Join(lockRESTPrefix, endpoint.Path)).Subrouter()
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodHealth).HandlerFunc(httpTraceHdrs(lockServer.HealthHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodLock).HandlerFunc(httpTraceHdrs(lockServer.LockHandler)).Queries(queries...)
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodRLock).HandlerFunc(httpTraceHdrs(lockServer.RLockHandler)).Queries(queries...)
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodUnlock).HandlerFunc(httpTraceHdrs(lockServer.UnlockHandler)).Queries(queries...)
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodRUnlock).HandlerFunc(httpTraceHdrs(lockServer.RUnlockHandler)).Queries(queries...)
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodExpired).HandlerFunc(httpTraceAll(lockServer.ExpiredHandler)).Queries(queries...)
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodLock).HandlerFunc(httpTraceHdrs(lockServer.LockHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodRLock).HandlerFunc(httpTraceHdrs(lockServer.RLockHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodUnlock).HandlerFunc(httpTraceHdrs(lockServer.UnlockHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodRUnlock).HandlerFunc(httpTraceHdrs(lockServer.RUnlockHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodExpired).HandlerFunc(httpTraceAll(lockServer.ExpiredHandler))
globalLockServers[endpoint] = lockServer.ll
}

View File

@@ -28,7 +28,6 @@ import (
"fmt"
"time"
"github.com/minio/minio/cmd/config/storageclass"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/dsync"
"github.com/minio/minio/pkg/lsync"
@@ -148,17 +147,8 @@ func (di *distLockInstance) GetLock(timeout *dynamicTimeout) (timedOutErr error)
lockSource := getSource(2)
start := UTCNow()
// Lockers default to standard storage class always, why because
// we always dictate storage tolerance in terms of standard
// storage class be it number of drives or a multiplicative
// of number of nodes, defaulting lockers to this value
// simply means that locking is always similar in behavior
// and effect with erasure coded drive tolerance.
tolerance := globalStorageClass.GetParityForSC(storageclass.STANDARD)
if !di.rwMutex.GetLock(di.ctx, di.opsID, lockSource, dsync.Options{
Timeout: timeout.Timeout(),
Tolerance: tolerance,
Timeout: timeout.Timeout(),
}) {
timeout.LogFailure()
return OperationTimedOut{}
@@ -177,12 +167,8 @@ func (di *distLockInstance) GetRLock(timeout *dynamicTimeout) (timedOutErr error
lockSource := getSource(2)
start := UTCNow()
// Lockers default to standard storage class always.
tolerance := globalStorageClass.GetParityForSC(storageclass.STANDARD)
if !di.rwMutex.GetRLock(di.ctx, di.opsID, lockSource, dsync.Options{
Timeout: timeout.Timeout(),
Tolerance: tolerance,
Timeout: timeout.Timeout(),
}) {
timeout.LogFailure()
return OperationTimedOut{}
@@ -208,11 +194,11 @@ type localLockInstance struct {
// NewNSLock - returns a lock instance for a given volume and
// path. The returned lockInstance object encapsulates the nsLockMap,
// volume, path and operation ID.
func (n *nsLockMap) NewNSLock(ctx context.Context, lockersFn func() []dsync.NetLocker, volume string, paths ...string) RWLocker {
func (n *nsLockMap) NewNSLock(ctx context.Context, lockers func() ([]dsync.NetLocker, string), volume string, paths ...string) RWLocker {
opsID := mustGetUUID()
if n.isDistErasure {
drwmutex := dsync.NewDRWMutex(&dsync.Dsync{
GetLockersFn: lockersFn,
GetLockers: lockers,
}, pathsJoinPrefix(volume, paths...)...)
return &distLockInstance{drwmutex, opsID, ctx}
}

View File

@@ -22,6 +22,7 @@ import (
"encoding/json"
"fmt"
"io"
"net/http"
"net/url"
"sort"
"strings"
@@ -505,34 +506,22 @@ func (sys *NotificationSys) updateBloomFilter(ctx context.Context, current uint6
}
// GetLocks - makes GetLocks RPC call on all peers.
func (sys *NotificationSys) GetLocks(ctx context.Context) []*PeerLocks {
func (sys *NotificationSys) GetLocks(ctx context.Context, r *http.Request) []*PeerLocks {
locksResp := make([]*PeerLocks, len(sys.peerClients))
g := errgroup.WithNErrs(len(sys.peerClients))
for index, client := range sys.peerClients {
if client == nil {
continue
}
index := index
g.Go(func() error {
// Try to fetch serverInfo remotely in three attempts.
for i := 0; i < 3; i++ {
serverLocksResp, err := sys.peerClients[index].GetLocks()
if err == nil {
locksResp[index] = &PeerLocks{
Addr: sys.peerClients[index].host.String(),
Locks: serverLocksResp,
}
return nil
}
// Last iteration log the error.
if i == 2 {
return err
}
// Wait for one second and no need wait after last attempt.
if i < 2 {
time.Sleep(1 * time.Second)
}
if client == nil {
return nil
}
serverLocksResp, err := sys.peerClients[index].GetLocks()
if err != nil {
return err
}
locksResp[index] = &PeerLocks{
Addr: sys.peerClients[index].host.String(),
Locks: serverLocksResp,
}
return nil
}, index)
@@ -543,6 +532,16 @@ func (sys *NotificationSys) GetLocks(ctx context.Context) []*PeerLocks {
ctx := logger.SetReqInfo(ctx, reqInfo)
logger.LogOnceIf(ctx, err, sys.peerClients[index].host.String())
}
// Once we have received all the locks currently used from peers
// add the local peer locks list as well.
var getRespLocks GetLocksResp
for _, llocker := range globalLockServers {
getRespLocks = append(getRespLocks, llocker.DupLockMap())
}
locksResp = append(locksResp, &PeerLocks{
Addr: getHostName(r),
Locks: getRespLocks,
})
return locksResp
}