Support MinIO to be deployed on more than 32 nodes (#8492)
This PR implements locking from a global entity into a more localized set level entity, allowing for locks to be held only on the resources which are writing to a collection of disks rather than a global level. In this process this PR also removes the top-level limit of 32 nodes to an unlimited number of nodes. This is a precursor change before bring in bucket expansion.master
parent
069b8ee8ff
commit
e9b2bf00ad
@ -0,0 +1,3 @@ |
|||||||
|
dsync.test |
||||||
|
coverage.txt |
||||||
|
*.out |
@ -0,0 +1,440 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package dsync |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"fmt" |
||||||
|
golog "log" |
||||||
|
"math/rand" |
||||||
|
"os" |
||||||
|
"path" |
||||||
|
"runtime" |
||||||
|
"sync" |
||||||
|
"time" |
||||||
|
) |
||||||
|
|
||||||
|
// Indicator if logging is enabled.
|
||||||
|
var dsyncLog bool |
||||||
|
|
||||||
|
func init() { |
||||||
|
// Check for MINIO_DSYNC_TRACE env variable, if set logging will be enabled for failed REST operations.
|
||||||
|
dsyncLog = os.Getenv("MINIO_DSYNC_TRACE") == "1" |
||||||
|
rand.Seed(time.Now().UnixNano()) |
||||||
|
} |
||||||
|
|
||||||
|
func log(msg ...interface{}) { |
||||||
|
if dsyncLog { |
||||||
|
golog.Println(msg...) |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// DRWMutexAcquireTimeout - tolerance limit to wait for lock acquisition before.
|
||||||
|
const DRWMutexAcquireTimeout = 1 * time.Second // 1 second.
|
||||||
|
const drwMutexInfinite = time.Duration(1<<63 - 1) |
||||||
|
|
||||||
|
// A DRWMutex is a distributed mutual exclusion lock.
|
||||||
|
type DRWMutex struct { |
||||||
|
Name string |
||||||
|
writeLocks []string // Array of nodes that granted a write lock
|
||||||
|
readersLocks [][]string // Array of array of nodes that granted reader locks
|
||||||
|
m sync.Mutex // Mutex to prevent multiple simultaneous locks from this node
|
||||||
|
clnt *Dsync |
||||||
|
ctx context.Context |
||||||
|
} |
||||||
|
|
||||||
|
// Granted - represents a structure of a granted lock.
|
||||||
|
type Granted struct { |
||||||
|
index int |
||||||
|
lockUID string // Locked if set with UID string, unlocked if empty
|
||||||
|
} |
||||||
|
|
||||||
|
func (g *Granted) isLocked() bool { |
||||||
|
return isLocked(g.lockUID) |
||||||
|
} |
||||||
|
|
||||||
|
func isLocked(uid string) bool { |
||||||
|
return len(uid) > 0 |
||||||
|
} |
||||||
|
|
||||||
|
// NewDRWMutex - initializes a new dsync RW mutex.
|
||||||
|
func NewDRWMutex(ctx context.Context, name string, clnt *Dsync) *DRWMutex { |
||||||
|
return &DRWMutex{ |
||||||
|
Name: name, |
||||||
|
writeLocks: make([]string, clnt.dNodeCount), |
||||||
|
clnt: clnt, |
||||||
|
ctx: ctx, |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Lock holds a write lock on dm.
|
||||||
|
//
|
||||||
|
// If the lock is already in use, the calling go routine
|
||||||
|
// blocks until the mutex is available.
|
||||||
|
func (dm *DRWMutex) Lock(id, source string) { |
||||||
|
|
||||||
|
isReadLock := false |
||||||
|
dm.lockBlocking(drwMutexInfinite, id, source, isReadLock) |
||||||
|
} |
||||||
|
|
||||||
|
// GetLock tries to get a write lock on dm before the timeout elapses.
|
||||||
|
//
|
||||||
|
// If the lock is already in use, the calling go routine
|
||||||
|
// blocks until either the mutex becomes available and return success or
|
||||||
|
// more time has passed than the timeout value and return false.
|
||||||
|
func (dm *DRWMutex) GetLock(id, source string, timeout time.Duration) (locked bool) { |
||||||
|
|
||||||
|
isReadLock := false |
||||||
|
return dm.lockBlocking(timeout, id, source, isReadLock) |
||||||
|
} |
||||||
|
|
||||||
|
// RLock holds a read lock on dm.
|
||||||
|
//
|
||||||
|
// If one or more read locks are already in use, it will grant another lock.
|
||||||
|
// Otherwise the calling go routine blocks until the mutex is available.
|
||||||
|
func (dm *DRWMutex) RLock(id, source string) { |
||||||
|
|
||||||
|
isReadLock := true |
||||||
|
dm.lockBlocking(drwMutexInfinite, id, source, isReadLock) |
||||||
|
} |
||||||
|
|
||||||
|
// GetRLock tries to get a read lock on dm before the timeout elapses.
|
||||||
|
//
|
||||||
|
// If one or more read locks are already in use, it will grant another lock.
|
||||||
|
// Otherwise the calling go routine blocks until either the mutex becomes
|
||||||
|
// available and return success or more time has passed than the timeout
|
||||||
|
// value and return false.
|
||||||
|
func (dm *DRWMutex) GetRLock(id, source string, timeout time.Duration) (locked bool) { |
||||||
|
|
||||||
|
isReadLock := true |
||||||
|
return dm.lockBlocking(timeout, id, source, isReadLock) |
||||||
|
} |
||||||
|
|
||||||
|
// lockBlocking will try to acquire either a read or a write lock
|
||||||
|
//
|
||||||
|
// The function will loop using a built-in timing randomized back-off
|
||||||
|
// algorithm until either the lock is acquired successfully or more
|
||||||
|
// time has elapsed than the timeout value.
|
||||||
|
func (dm *DRWMutex) lockBlocking(timeout time.Duration, id, source string, isReadLock bool) (locked bool) { |
||||||
|
doneCh, start := make(chan struct{}), time.Now().UTC() |
||||||
|
defer close(doneCh) |
||||||
|
|
||||||
|
// Use incremental back-off algorithm for repeated attempts to acquire the lock
|
||||||
|
for range newRetryTimerSimple(doneCh) { |
||||||
|
select { |
||||||
|
case <-dm.ctx.Done(): |
||||||
|
return |
||||||
|
default: |
||||||
|
} |
||||||
|
|
||||||
|
// Create temp array on stack.
|
||||||
|
locks := make([]string, dm.clnt.dNodeCount) |
||||||
|
|
||||||
|
// Try to acquire the lock.
|
||||||
|
success := lock(dm.clnt, &locks, dm.Name, id, source, isReadLock) |
||||||
|
if success { |
||||||
|
dm.m.Lock() |
||||||
|
|
||||||
|
// If success, copy array to object
|
||||||
|
if isReadLock { |
||||||
|
// Append new array of strings at the end
|
||||||
|
dm.readersLocks = append(dm.readersLocks, make([]string, dm.clnt.dNodeCount)) |
||||||
|
// and copy stack array into last spot
|
||||||
|
copy(dm.readersLocks[len(dm.readersLocks)-1], locks[:]) |
||||||
|
} else { |
||||||
|
copy(dm.writeLocks, locks[:]) |
||||||
|
} |
||||||
|
|
||||||
|
dm.m.Unlock() |
||||||
|
return true |
||||||
|
} |
||||||
|
if time.Now().UTC().Sub(start) >= timeout { // Are we past the timeout?
|
||||||
|
break |
||||||
|
} |
||||||
|
// Failed to acquire the lock on this attempt, incrementally wait
|
||||||
|
// for a longer back-off time and try again afterwards.
|
||||||
|
} |
||||||
|
return false |
||||||
|
} |
||||||
|
|
||||||
|
// lock tries to acquire the distributed lock, returning true or false.
|
||||||
|
func lock(ds *Dsync, locks *[]string, lockName, id, source string, isReadLock bool) bool { |
||||||
|
|
||||||
|
// Create buffered channel of size equal to total number of nodes.
|
||||||
|
ch := make(chan Granted, ds.dNodeCount) |
||||||
|
defer close(ch) |
||||||
|
|
||||||
|
var wg sync.WaitGroup |
||||||
|
for index, c := range ds.restClnts { |
||||||
|
|
||||||
|
wg.Add(1) |
||||||
|
// broadcast lock request to all nodes
|
||||||
|
go func(index int, isReadLock bool, c NetLocker) { |
||||||
|
defer wg.Done() |
||||||
|
|
||||||
|
args := LockArgs{ |
||||||
|
UID: id, |
||||||
|
Resource: lockName, |
||||||
|
Source: source, |
||||||
|
} |
||||||
|
|
||||||
|
var locked bool |
||||||
|
var err error |
||||||
|
if isReadLock { |
||||||
|
if locked, err = c.RLock(args); err != nil { |
||||||
|
log("Unable to call RLock", err) |
||||||
|
} |
||||||
|
} else { |
||||||
|
if locked, err = c.Lock(args); err != nil { |
||||||
|
log("Unable to call Lock", err) |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
g := Granted{index: index} |
||||||
|
if locked { |
||||||
|
g.lockUID = args.UID |
||||||
|
} |
||||||
|
|
||||||
|
ch <- g |
||||||
|
|
||||||
|
}(index, isReadLock, c) |
||||||
|
} |
||||||
|
|
||||||
|
quorum := false |
||||||
|
|
||||||
|
wg.Add(1) |
||||||
|
go func(isReadLock bool) { |
||||||
|
|
||||||
|
// Wait until we have either
|
||||||
|
//
|
||||||
|
// a) received all lock responses
|
||||||
|
// b) received too many 'non-'locks for quorum to be still possible
|
||||||
|
// c) time out
|
||||||
|
//
|
||||||
|
i, locksFailed := 0, 0 |
||||||
|
done := false |
||||||
|
timeout := time.After(DRWMutexAcquireTimeout) |
||||||
|
|
||||||
|
for ; i < ds.dNodeCount; i++ { // Loop until we acquired all locks
|
||||||
|
|
||||||
|
select { |
||||||
|
case grant := <-ch: |
||||||
|
if grant.isLocked() { |
||||||
|
// Mark that this node has acquired the lock
|
||||||
|
(*locks)[grant.index] = grant.lockUID |
||||||
|
} else { |
||||||
|
locksFailed++ |
||||||
|
if !isReadLock && locksFailed > ds.dNodeCount-ds.dquorum || |
||||||
|
isReadLock && locksFailed > ds.dNodeCount-ds.dquorumReads { |
||||||
|
// We know that we are not going to get the lock anymore,
|
||||||
|
// so exit out and release any locks that did get acquired
|
||||||
|
done = true |
||||||
|
// Increment the number of grants received from the buffered channel.
|
||||||
|
i++ |
||||||
|
releaseAll(ds, locks, lockName, isReadLock) |
||||||
|
} |
||||||
|
} |
||||||
|
case <-timeout: |
||||||
|
done = true |
||||||
|
// timeout happened, maybe one of the nodes is slow, count
|
||||||
|
// number of locks to check whether we have quorum or not
|
||||||
|
if !quorumMet(locks, isReadLock, ds.dquorum, ds.dquorumReads) { |
||||||
|
releaseAll(ds, locks, lockName, isReadLock) |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
if done { |
||||||
|
break |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Count locks in order to determine whether we have quorum or not
|
||||||
|
quorum = quorumMet(locks, isReadLock, ds.dquorum, ds.dquorumReads) |
||||||
|
|
||||||
|
// Signal that we have the quorum
|
||||||
|
wg.Done() |
||||||
|
|
||||||
|
// Wait for the other responses and immediately release the locks
|
||||||
|
// (do not add them to the locks array because the DRWMutex could
|
||||||
|
// already has been unlocked again by the original calling thread)
|
||||||
|
for ; i < ds.dNodeCount; i++ { |
||||||
|
grantToBeReleased := <-ch |
||||||
|
if grantToBeReleased.isLocked() { |
||||||
|
// release lock
|
||||||
|
sendRelease(ds, ds.restClnts[grantToBeReleased.index], lockName, grantToBeReleased.lockUID, isReadLock) |
||||||
|
} |
||||||
|
} |
||||||
|
}(isReadLock) |
||||||
|
|
||||||
|
wg.Wait() |
||||||
|
|
||||||
|
return quorum |
||||||
|
} |
||||||
|
|
||||||
|
// quorumMet determines whether we have acquired the required quorum of underlying locks or not
|
||||||
|
func quorumMet(locks *[]string, isReadLock bool, quorum, quorumReads int) bool { |
||||||
|
|
||||||
|
count := 0 |
||||||
|
for _, uid := range *locks { |
||||||
|
if isLocked(uid) { |
||||||
|
count++ |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
var metQuorum bool |
||||||
|
if isReadLock { |
||||||
|
metQuorum = count >= quorumReads |
||||||
|
} else { |
||||||
|
metQuorum = count >= quorum |
||||||
|
} |
||||||
|
|
||||||
|
return metQuorum |
||||||
|
} |
||||||
|
|
||||||
|
// releaseAll releases all locks that are marked as locked
|
||||||
|
func releaseAll(ds *Dsync, locks *[]string, lockName string, isReadLock bool) { |
||||||
|
for lock := 0; lock < ds.dNodeCount; lock++ { |
||||||
|
if isLocked((*locks)[lock]) { |
||||||
|
sendRelease(ds, ds.restClnts[lock], lockName, (*locks)[lock], isReadLock) |
||||||
|
(*locks)[lock] = "" |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Unlock unlocks the write lock.
|
||||||
|
//
|
||||||
|
// It is a run-time error if dm is not locked on entry to Unlock.
|
||||||
|
func (dm *DRWMutex) Unlock() { |
||||||
|
|
||||||
|
// create temp array on stack
|
||||||
|
locks := make([]string, dm.clnt.dNodeCount) |
||||||
|
|
||||||
|
{ |
||||||
|
dm.m.Lock() |
||||||
|
defer dm.m.Unlock() |
||||||
|
|
||||||
|
// Check if minimally a single bool is set in the writeLocks array
|
||||||
|
lockFound := false |
||||||
|
for _, uid := range dm.writeLocks { |
||||||
|
if isLocked(uid) { |
||||||
|
lockFound = true |
||||||
|
break |
||||||
|
} |
||||||
|
} |
||||||
|
if !lockFound { |
||||||
|
panic("Trying to Unlock() while no Lock() is active") |
||||||
|
} |
||||||
|
|
||||||
|
// Copy write locks to stack array
|
||||||
|
copy(locks, dm.writeLocks[:]) |
||||||
|
// Clear write locks array
|
||||||
|
dm.writeLocks = make([]string, dm.clnt.dNodeCount) |
||||||
|
} |
||||||
|
|
||||||
|
isReadLock := false |
||||||
|
unlock(dm.clnt, locks, dm.Name, isReadLock) |
||||||
|
} |
||||||
|
|
||||||
|
// RUnlock releases a read lock held on dm.
|
||||||
|
//
|
||||||
|
// It is a run-time error if dm is not locked on entry to RUnlock.
|
||||||
|
func (dm *DRWMutex) RUnlock() { |
||||||
|
|
||||||
|
// create temp array on stack
|
||||||
|
locks := make([]string, dm.clnt.dNodeCount) |
||||||
|
|
||||||
|
{ |
||||||
|
dm.m.Lock() |
||||||
|
defer dm.m.Unlock() |
||||||
|
if len(dm.readersLocks) == 0 { |
||||||
|
panic("Trying to RUnlock() while no RLock() is active") |
||||||
|
} |
||||||
|
// Copy out first element to release it first (FIFO)
|
||||||
|
copy(locks, dm.readersLocks[0][:]) |
||||||
|
// Drop first element from array
|
||||||
|
dm.readersLocks = dm.readersLocks[1:] |
||||||
|
} |
||||||
|
|
||||||
|
isReadLock := true |
||||||
|
unlock(dm.clnt, locks, dm.Name, isReadLock) |
||||||
|
} |
||||||
|
|
||||||
|
func unlock(ds *Dsync, locks []string, name string, isReadLock bool) { |
||||||
|
|
||||||
|
// We don't need to synchronously wait until we have released all the locks (or the quorum)
|
||||||
|
// (a subsequent lock will retry automatically in case it would fail to get quorum)
|
||||||
|
|
||||||
|
for index, c := range ds.restClnts { |
||||||
|
|
||||||
|
if isLocked(locks[index]) { |
||||||
|
// broadcast lock release to all nodes that granted the lock
|
||||||
|
sendRelease(ds, c, name, locks[index], isReadLock) |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// sendRelease sends a release message to a node that previously granted a lock
|
||||||
|
func sendRelease(ds *Dsync, c NetLocker, name, uid string, isReadLock bool) { |
||||||
|
args := LockArgs{ |
||||||
|
UID: uid, |
||||||
|
Resource: name, |
||||||
|
} |
||||||
|
if isReadLock { |
||||||
|
if _, err := c.RUnlock(args); err != nil { |
||||||
|
log("Unable to call RUnlock", err) |
||||||
|
} |
||||||
|
} else { |
||||||
|
if _, err := c.Unlock(args); err != nil { |
||||||
|
log("Unable to call Unlock", err) |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// DRLocker returns a sync.Locker interface that implements
|
||||||
|
// the Lock and Unlock methods by calling drw.RLock and drw.RUnlock.
|
||||||
|
func (dm *DRWMutex) DRLocker() sync.Locker { |
||||||
|
return (*drlocker)(dm) |
||||||
|
} |
||||||
|
|
||||||
|
type drlocker DRWMutex |
||||||
|
|
||||||
|
var letterRunes = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ") |
||||||
|
|
||||||
|
func randString(n int) string { |
||||||
|
b := make([]rune, n) |
||||||
|
for i := range b { |
||||||
|
b[i] = letterRunes[rand.Intn(len(letterRunes))] |
||||||
|
} |
||||||
|
return string(b) |
||||||
|
} |
||||||
|
|
||||||
|
func getSource() string { |
||||||
|
var funcName string |
||||||
|
pc, filename, lineNum, ok := runtime.Caller(2) |
||||||
|
if ok { |
||||||
|
filename = path.Base(filename) |
||||||
|
funcName = runtime.FuncForPC(pc).Name() |
||||||
|
} else { |
||||||
|
filename = "<unknown>" |
||||||
|
lineNum = 0 |
||||||
|
} |
||||||
|
|
||||||
|
return fmt.Sprintf("[%s:%d:%s()]", filename, lineNum, funcName) |
||||||
|
} |
||||||
|
|
||||||
|
func (dr *drlocker) Lock() { (*DRWMutex)(dr).RLock(randString(16), getSource()) } |
||||||
|
func (dr *drlocker) Unlock() { (*DRWMutex)(dr).RUnlock() } |
@ -0,0 +1,383 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
// GOMAXPROCS=10 go test
|
||||||
|
|
||||||
|
package dsync_test |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"fmt" |
||||||
|
"runtime" |
||||||
|
"sync" |
||||||
|
"sync/atomic" |
||||||
|
"testing" |
||||||
|
"time" |
||||||
|
|
||||||
|
. "github.com/minio/minio/pkg/dsync" |
||||||
|
) |
||||||
|
|
||||||
|
const ( |
||||||
|
id = "1234-5678" |
||||||
|
source = "main.go" |
||||||
|
) |
||||||
|
|
||||||
|
func testSimpleWriteLock(t *testing.T, duration time.Duration) (locked bool) { |
||||||
|
|
||||||
|
drwm := NewDRWMutex(context.Background(), "simplelock", ds) |
||||||
|
|
||||||
|
if !drwm.GetRLock(id, source, time.Second) { |
||||||
|
panic("Failed to acquire read lock") |
||||||
|
} |
||||||
|
// fmt.Println("1st read lock acquired, waiting...")
|
||||||
|
|
||||||
|
if !drwm.GetRLock(id, source, time.Second) { |
||||||
|
panic("Failed to acquire read lock") |
||||||
|
} |
||||||
|
// fmt.Println("2nd read lock acquired, waiting...")
|
||||||
|
|
||||||
|
go func() { |
||||||
|
time.Sleep(2 * time.Second) |
||||||
|
drwm.RUnlock() |
||||||
|
// fmt.Println("1st read lock released, waiting...")
|
||||||
|
}() |
||||||
|
|
||||||
|
go func() { |
||||||
|
time.Sleep(3 * time.Second) |
||||||
|
drwm.RUnlock() |
||||||
|
// fmt.Println("2nd read lock released, waiting...")
|
||||||
|
}() |
||||||
|
|
||||||
|
// fmt.Println("Trying to acquire write lock, waiting...")
|
||||||
|
locked = drwm.GetLock(id, source, duration) |
||||||
|
if locked { |
||||||
|
// fmt.Println("Write lock acquired, waiting...")
|
||||||
|
time.Sleep(time.Second) |
||||||
|
|
||||||
|
drwm.Unlock() |
||||||
|
} |
||||||
|
// fmt.Println("Write lock failed due to timeout")
|
||||||
|
return |
||||||
|
} |
||||||
|
|
||||||
|
func TestSimpleWriteLockAcquired(t *testing.T) { |
||||||
|
locked := testSimpleWriteLock(t, 5*time.Second) |
||||||
|
|
||||||
|
expected := true |
||||||
|
if locked != expected { |
||||||
|
t.Errorf("TestSimpleWriteLockAcquired(): \nexpected %#v\ngot %#v", expected, locked) |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func TestSimpleWriteLockTimedOut(t *testing.T) { |
||||||
|
locked := testSimpleWriteLock(t, time.Second) |
||||||
|
|
||||||
|
expected := false |
||||||
|
if locked != expected { |
||||||
|
t.Errorf("TestSimpleWriteLockTimedOut(): \nexpected %#v\ngot %#v", expected, locked) |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func testDualWriteLock(t *testing.T, duration time.Duration) (locked bool) { |
||||||
|
|
||||||
|
drwm := NewDRWMutex(context.Background(), "duallock", ds) |
||||||
|
|
||||||
|
// fmt.Println("Getting initial write lock")
|
||||||
|
if !drwm.GetLock(id, source, time.Second) { |
||||||
|
panic("Failed to acquire initial write lock") |
||||||
|
} |
||||||
|
|
||||||
|
go func() { |
||||||
|
time.Sleep(2 * time.Second) |
||||||
|
drwm.Unlock() |
||||||
|
// fmt.Println("Initial write lock released, waiting...")
|
||||||
|
}() |
||||||
|
|
||||||
|
// fmt.Println("Trying to acquire 2nd write lock, waiting...")
|
||||||
|
locked = drwm.GetLock(id, source, duration) |
||||||
|
if locked { |
||||||
|
// fmt.Println("2nd write lock acquired, waiting...")
|
||||||
|
time.Sleep(time.Second) |
||||||
|
|
||||||
|
drwm.Unlock() |
||||||
|
} |
||||||
|
// fmt.Println("2nd write lock failed due to timeout")
|
||||||
|
return |
||||||
|
} |
||||||
|
|
||||||
|
func TestDualWriteLockAcquired(t *testing.T) { |
||||||
|
locked := testDualWriteLock(t, 5*time.Second) |
||||||
|
|
||||||
|
expected := true |
||||||
|
if locked != expected { |
||||||
|
t.Errorf("TestDualWriteLockAcquired(): \nexpected %#v\ngot %#v", expected, locked) |
||||||
|
} |
||||||
|
|
||||||
|
} |
||||||
|
|
||||||
|
func TestDualWriteLockTimedOut(t *testing.T) { |
||||||
|
locked := testDualWriteLock(t, time.Second) |
||||||
|
|
||||||
|
expected := false |
||||||
|
if locked != expected { |
||||||
|
t.Errorf("TestDualWriteLockTimedOut(): \nexpected %#v\ngot %#v", expected, locked) |
||||||
|
} |
||||||
|
|
||||||
|
} |
||||||
|
|
||||||
|
// Test cases below are copied 1 to 1 from sync/rwmutex_test.go (adapted to use DRWMutex)
|
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func parallelReader(m *DRWMutex, clocked, cunlock, cdone chan bool) { |
||||||
|
if m.GetRLock(id, source, time.Second) { |
||||||
|
clocked <- true |
||||||
|
<-cunlock |
||||||
|
m.RUnlock() |
||||||
|
cdone <- true |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func doTestParallelReaders(numReaders, gomaxprocs int) { |
||||||
|
runtime.GOMAXPROCS(gomaxprocs) |
||||||
|
m := NewDRWMutex(context.Background(), "test-parallel", ds) |
||||||
|
|
||||||
|
clocked := make(chan bool) |
||||||
|
cunlock := make(chan bool) |
||||||
|
cdone := make(chan bool) |
||||||
|
for i := 0; i < numReaders; i++ { |
||||||
|
go parallelReader(m, clocked, cunlock, cdone) |
||||||
|
} |
||||||
|
// Wait for all parallel RLock()s to succeed.
|
||||||
|
for i := 0; i < numReaders; i++ { |
||||||
|
<-clocked |
||||||
|
} |
||||||
|
for i := 0; i < numReaders; i++ { |
||||||
|
cunlock <- true |
||||||
|
} |
||||||
|
// Wait for the goroutines to finish.
|
||||||
|
for i := 0; i < numReaders; i++ { |
||||||
|
<-cdone |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func TestParallelReaders(t *testing.T) { |
||||||
|
defer runtime.GOMAXPROCS(runtime.GOMAXPROCS(-1)) |
||||||
|
doTestParallelReaders(1, 4) |
||||||
|
doTestParallelReaders(3, 4) |
||||||
|
doTestParallelReaders(4, 2) |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func reader(rwm *DRWMutex, numIterations int, activity *int32, cdone chan bool) { |
||||||
|
for i := 0; i < numIterations; i++ { |
||||||
|
if rwm.GetRLock(id, source, time.Second) { |
||||||
|
n := atomic.AddInt32(activity, 1) |
||||||
|
if n < 1 || n >= 10000 { |
||||||
|
panic(fmt.Sprintf("wlock(%d)\n", n)) |
||||||
|
} |
||||||
|
for i := 0; i < 100; i++ { |
||||||
|
} |
||||||
|
atomic.AddInt32(activity, -1) |
||||||
|
rwm.RUnlock() |
||||||
|
} |
||||||
|
} |
||||||
|
cdone <- true |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func writer(rwm *DRWMutex, numIterations int, activity *int32, cdone chan bool) { |
||||||
|
for i := 0; i < numIterations; i++ { |
||||||
|
if rwm.GetLock(id, source, time.Second) { |
||||||
|
n := atomic.AddInt32(activity, 10000) |
||||||
|
if n != 10000 { |
||||||
|
panic(fmt.Sprintf("wlock(%d)\n", n)) |
||||||
|
} |
||||||
|
for i := 0; i < 100; i++ { |
||||||
|
} |
||||||
|
atomic.AddInt32(activity, -10000) |
||||||
|
rwm.Unlock() |
||||||
|
} |
||||||
|
} |
||||||
|
cdone <- true |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func HammerRWMutex(gomaxprocs, numReaders, numIterations int) { |
||||||
|
runtime.GOMAXPROCS(gomaxprocs) |
||||||
|
// Number of active readers + 10000 * number of active writers.
|
||||||
|
var activity int32 |
||||||
|
rwm := NewDRWMutex(context.Background(), "test", ds) |
||||||
|
cdone := make(chan bool) |
||||||
|
go writer(rwm, numIterations, &activity, cdone) |
||||||
|
var i int |
||||||
|
for i = 0; i < numReaders/2; i++ { |
||||||
|
go reader(rwm, numIterations, &activity, cdone) |
||||||
|
} |
||||||
|
go writer(rwm, numIterations, &activity, cdone) |
||||||
|
for ; i < numReaders; i++ { |
||||||
|
go reader(rwm, numIterations, &activity, cdone) |
||||||
|
} |
||||||
|
// Wait for the 2 writers and all readers to finish.
|
||||||
|
for i := 0; i < 2+numReaders; i++ { |
||||||
|
<-cdone |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func TestRWMutex(t *testing.T) { |
||||||
|
defer runtime.GOMAXPROCS(runtime.GOMAXPROCS(-1)) |
||||||
|
n := 1000 |
||||||
|
if testing.Short() { |
||||||
|
n = 5 |
||||||
|
} |
||||||
|
HammerRWMutex(1, 1, n) |
||||||
|
HammerRWMutex(1, 3, n) |
||||||
|
HammerRWMutex(1, 10, n) |
||||||
|
HammerRWMutex(4, 1, n) |
||||||
|
HammerRWMutex(4, 3, n) |
||||||
|
HammerRWMutex(4, 10, n) |
||||||
|
HammerRWMutex(10, 1, n) |
||||||
|
HammerRWMutex(10, 3, n) |
||||||
|
HammerRWMutex(10, 10, n) |
||||||
|
HammerRWMutex(10, 5, n) |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func TestDRLocker(t *testing.T) { |
||||||
|
wl := NewDRWMutex(context.Background(), "test", ds) |
||||||
|
var rl sync.Locker |
||||||
|
wlocked := make(chan bool, 1) |
||||||
|
rlocked := make(chan bool, 1) |
||||||
|
rl = wl.DRLocker() |
||||||
|
n := 10 |
||||||
|
go func() { |
||||||
|
for i := 0; i < n; i++ { |
||||||
|
rl.Lock() |
||||||
|
rl.Lock() |
||||||
|
rlocked <- true |
||||||
|
wl.Lock(id, source) |
||||||
|
wlocked <- true |
||||||
|
} |
||||||
|
}() |
||||||
|
for i := 0; i < n; i++ { |
||||||
|
<-rlocked |
||||||
|
rl.Unlock() |
||||||
|
select { |
||||||
|
case <-wlocked: |
||||||
|
t.Fatal("RLocker() didn't read-lock it") |
||||||
|
default: |
||||||
|
} |
||||||
|
rl.Unlock() |
||||||
|
<-wlocked |
||||||
|
select { |
||||||
|
case <-rlocked: |
||||||
|
t.Fatal("RLocker() didn't respect the write lock") |
||||||
|
default: |
||||||
|
} |
||||||
|
wl.Unlock() |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func TestUnlockPanic(t *testing.T) { |
||||||
|
defer func() { |
||||||
|
if recover() == nil { |
||||||
|
t.Fatalf("unlock of unlocked RWMutex did not panic") |
||||||
|
} |
||||||
|
}() |
||||||
|
mu := NewDRWMutex(context.Background(), "test", ds) |
||||||
|
mu.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func TestUnlockPanic2(t *testing.T) { |
||||||
|
defer func() { |
||||||
|
if recover() == nil { |
||||||
|
t.Fatalf("unlock of unlocked RWMutex did not panic") |
||||||
|
} |
||||||
|
}() |
||||||
|
mu := NewDRWMutex(context.Background(), "test-unlock-panic-2", ds) |
||||||
|
mu.RLock(id, source) |
||||||
|
mu.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func TestRUnlockPanic(t *testing.T) { |
||||||
|
defer func() { |
||||||
|
if recover() == nil { |
||||||
|
t.Fatalf("read unlock of unlocked RWMutex did not panic") |
||||||
|
} |
||||||
|
}() |
||||||
|
mu := NewDRWMutex(context.Background(), "test", ds) |
||||||
|
mu.RUnlock() |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func TestRUnlockPanic2(t *testing.T) { |
||||||
|
defer func() { |
||||||
|
if recover() == nil { |
||||||
|
t.Fatalf("read unlock of unlocked RWMutex did not panic") |
||||||
|
} |
||||||
|
}() |
||||||
|
mu := NewDRWMutex(context.Background(), "test-runlock-panic-2", ds) |
||||||
|
mu.Lock(id, source) |
||||||
|
mu.RUnlock() |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func benchmarkRWMutex(b *testing.B, localWork, writeRatio int) { |
||||||
|
rwm := NewDRWMutex(context.Background(), "test", ds) |
||||||
|
b.RunParallel(func(pb *testing.PB) { |
||||||
|
foo := 0 |
||||||
|
for pb.Next() { |
||||||
|
foo++ |
||||||
|
if foo%writeRatio == 0 { |
||||||
|
rwm.Lock(id, source) |
||||||
|
rwm.Unlock() |
||||||
|
} else { |
||||||
|
rwm.RLock(id, source) |
||||||
|
for i := 0; i != localWork; i += 1 { |
||||||
|
foo *= 2 |
||||||
|
foo /= 2 |
||||||
|
} |
||||||
|
rwm.RUnlock() |
||||||
|
} |
||||||
|
} |
||||||
|
_ = foo |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func BenchmarkRWMutexWrite100(b *testing.B) { |
||||||
|
benchmarkRWMutex(b, 0, 100) |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func BenchmarkRWMutexWrite10(b *testing.B) { |
||||||
|
benchmarkRWMutex(b, 0, 10) |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func BenchmarkRWMutexWorkWrite100(b *testing.B) { |
||||||
|
benchmarkRWMutex(b, 100, 100) |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from rwmutex_test.go
|
||||||
|
func BenchmarkRWMutexWorkWrite10(b *testing.B) { |
||||||
|
benchmarkRWMutex(b, 100, 10) |
||||||
|
} |
@ -0,0 +1,103 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package dsync_test |
||||||
|
|
||||||
|
import ( |
||||||
|
"fmt" |
||||||
|
"sync" |
||||||
|
|
||||||
|
. "github.com/minio/minio/pkg/dsync" |
||||||
|
) |
||||||
|
|
||||||
|
const WriteLock = -1 |
||||||
|
|
||||||
|
type lockServer struct { |
||||||
|
mutex sync.Mutex |
||||||
|
// Map of locks, with negative value indicating (exclusive) write lock
|
||||||
|
// and positive values indicating number of read locks
|
||||||
|
lockMap map[string]int64 |
||||||
|
} |
||||||
|
|
||||||
|
func (l *lockServer) Lock(args *LockArgs, reply *bool) error { |
||||||
|
l.mutex.Lock() |
||||||
|
defer l.mutex.Unlock() |
||||||
|
if _, *reply = l.lockMap[args.Resource]; !*reply { |
||||||
|
l.lockMap[args.Resource] = WriteLock // No locks held on the given name, so claim write lock
|
||||||
|
} |
||||||
|
*reply = !*reply // Negate *reply to return true when lock is granted or false otherwise
|
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
func (l *lockServer) Unlock(args *LockArgs, reply *bool) error { |
||||||
|
l.mutex.Lock() |
||||||
|
defer l.mutex.Unlock() |
||||||
|
var locksHeld int64 |
||||||
|
if locksHeld, *reply = l.lockMap[args.Resource]; !*reply { // No lock is held on the given name
|
||||||
|
return fmt.Errorf("Unlock attempted on an unlocked entity: %s", args.Resource) |
||||||
|
} |
||||||
|
if *reply = locksHeld == WriteLock; !*reply { // Unless it is a write lock
|
||||||
|
return fmt.Errorf("Unlock attempted on a read locked entity: %s (%d read locks active)", args.Resource, locksHeld) |
||||||
|
} |
||||||
|
delete(l.lockMap, args.Resource) // Remove the write lock
|
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
const ReadLock = 1 |
||||||
|
|
||||||
|
func (l *lockServer) RLock(args *LockArgs, reply *bool) error { |
||||||
|
l.mutex.Lock() |
||||||
|
defer l.mutex.Unlock() |
||||||
|
var locksHeld int64 |
||||||
|
if locksHeld, *reply = l.lockMap[args.Resource]; !*reply { |
||||||
|
l.lockMap[args.Resource] = ReadLock // No locks held on the given name, so claim (first) read lock
|
||||||
|
*reply = true |
||||||
|
} else { |
||||||
|
if *reply = locksHeld != WriteLock; *reply { // Unless there is a write lock
|
||||||
|
l.lockMap[args.Resource] = locksHeld + ReadLock // Grant another read lock
|
||||||
|
} |
||||||
|
} |
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
func (l *lockServer) RUnlock(args *LockArgs, reply *bool) error { |
||||||
|
l.mutex.Lock() |
||||||
|
defer l.mutex.Unlock() |
||||||
|
var locksHeld int64 |
||||||
|
if locksHeld, *reply = l.lockMap[args.Resource]; !*reply { // No lock is held on the given name
|
||||||
|
return fmt.Errorf("RUnlock attempted on an unlocked entity: %s", args.Resource) |
||||||
|
} |
||||||
|
if *reply = locksHeld != WriteLock; !*reply { // A write-lock is held, cannot release a read lock
|
||||||
|
return fmt.Errorf("RUnlock attempted on a write locked entity: %s", args.Resource) |
||||||
|
} |
||||||
|
if locksHeld > ReadLock { |
||||||
|
l.lockMap[args.Resource] = locksHeld - ReadLock // Remove one of the read locks held
|
||||||
|
} else { |
||||||
|
delete(l.lockMap, args.Resource) // Remove the (last) read lock
|
||||||
|
} |
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
func (l *lockServer) ForceUnlock(args *LockArgs, reply *bool) error { |
||||||
|
l.mutex.Lock() |
||||||
|
defer l.mutex.Unlock() |
||||||
|
if len(args.UID) != 0 { |
||||||
|
return fmt.Errorf("ForceUnlock called with non-empty UID: %s", args.UID) |
||||||
|
} |
||||||
|
delete(l.lockMap, args.Resource) // Remove the lock (irrespective of write or read lock)
|
||||||
|
*reply = true |
||||||
|
return nil |
||||||
|
} |
@ -0,0 +1,60 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package dsync |
||||||
|
|
||||||
|
import ( |
||||||
|
"errors" |
||||||
|
"math" |
||||||
|
) |
||||||
|
|
||||||
|
// Dsync represents dsync client object which is initialized with
|
||||||
|
// authenticated clients, used to initiate lock REST calls.
|
||||||
|
type Dsync struct { |
||||||
|
// Number of nodes participating in the distributed locking.
|
||||||
|
dNodeCount int |
||||||
|
|
||||||
|
// List of rest client objects, one per lock server.
|
||||||
|
restClnts []NetLocker |
||||||
|
|
||||||
|
// Simple majority based quorum, set to dNodeCount/2+1
|
||||||
|
dquorum int |
||||||
|
|
||||||
|
// Simple quorum for read operations, set to dNodeCount/2
|
||||||
|
dquorumReads int |
||||||
|
} |
||||||
|
|
||||||
|
// New - initializes a new dsync object with input restClnts.
|
||||||
|
func New(restClnts []NetLocker) (*Dsync, error) { |
||||||
|
if len(restClnts) < 2 { |
||||||
|
return nil, errors.New("Dsync is not designed for less than 2 nodes") |
||||||
|
} else if len(restClnts) > 32 { |
||||||
|
return nil, errors.New("Dsync is not designed for more than 32 nodes") |
||||||
|
} |
||||||
|
|
||||||
|
ds := &Dsync{} |
||||||
|
ds.dNodeCount = len(restClnts) |
||||||
|
|
||||||
|
// With odd number of nodes, write and read quorum is basically the same
|
||||||
|
ds.dquorum = int(ds.dNodeCount/2) + 1 |
||||||
|
ds.dquorumReads = int(math.Ceil(float64(ds.dNodeCount) / 2.0)) |
||||||
|
|
||||||
|
// Initialize node name and rest path for each NetLocker object.
|
||||||
|
ds.restClnts = make([]NetLocker, ds.dNodeCount) |
||||||
|
copy(ds.restClnts, restClnts) |
||||||
|
|
||||||
|
return ds, nil |
||||||
|
} |
@ -0,0 +1,58 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2018 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
// GOMAXPROCS=10 go test
|
||||||
|
|
||||||
|
package dsync |
||||||
|
|
||||||
|
import "testing" |
||||||
|
|
||||||
|
// Tests dsync.New
|
||||||
|
func TestNew(t *testing.T) { |
||||||
|
nclnts := make([]NetLocker, 33) |
||||||
|
if _, err := New(nclnts); err == nil { |
||||||
|
t.Fatal("Should have failed") |
||||||
|
} |
||||||
|
|
||||||
|
nclnts = make([]NetLocker, 1) |
||||||
|
if _, err := New(nclnts); err == nil { |
||||||
|
t.Fatal("Should have failed") |
||||||
|
} |
||||||
|
|
||||||
|
nclnts = make([]NetLocker, 2) |
||||||
|
nds, err := New(nclnts) |
||||||
|
if err != nil { |
||||||
|
t.Fatal("Should pass", err) |
||||||
|
} |
||||||
|
|
||||||
|
if nds.dquorumReads != 1 { |
||||||
|
t.Fatalf("Unexpected read quorum values expected 1, got %d", nds.dquorumReads) |
||||||
|
} |
||||||
|
|
||||||
|
if nds.dquorum != 2 { |
||||||
|
t.Fatalf("Unexpected quorum values expected 2, got %d", nds.dquorum) |
||||||
|
} |
||||||
|
|
||||||
|
nclnts = make([]NetLocker, 3) |
||||||
|
nds, err = New(nclnts) |
||||||
|
if err != nil { |
||||||
|
t.Fatal("Should pass", err) |
||||||
|
} |
||||||
|
|
||||||
|
if nds.dquorumReads != nds.dquorum { |
||||||
|
t.Fatalf("Unexpected quorum values for odd nodes we expect read %d and write %d quorum to be same", nds.dquorumReads, nds.dquorum) |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,361 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
// GOMAXPROCS=10 go test
|
||||||
|
|
||||||
|
package dsync_test |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"fmt" |
||||||
|
"log" |
||||||
|
"math/rand" |
||||||
|
"net" |
||||||
|
"net/http" |
||||||
|
"net/rpc" |
||||||
|
"os" |
||||||
|
"strconv" |
||||||
|
"sync" |
||||||
|
"testing" |
||||||
|
"time" |
||||||
|
|
||||||
|
. "github.com/minio/minio/pkg/dsync" |
||||||
|
) |
||||||
|
|
||||||
|
var ds *Dsync |
||||||
|
var rpcPaths []string // list of rpc paths where lock server is serving.
|
||||||
|
|
||||||
|
func startRPCServers(nodes []string) { |
||||||
|
for i := range nodes { |
||||||
|
server := rpc.NewServer() |
||||||
|
server.RegisterName("Dsync", &lockServer{ |
||||||
|
mutex: sync.Mutex{}, |
||||||
|
lockMap: make(map[string]int64), |
||||||
|
}) |
||||||
|
// For some reason the registration paths need to be different (even for different server objs)
|
||||||
|
server.HandleHTTP(rpcPaths[i], fmt.Sprintf("%s-debug", rpcPaths[i])) |
||||||
|
l, e := net.Listen("tcp", ":"+strconv.Itoa(i+12345)) |
||||||
|
if e != nil { |
||||||
|
log.Fatal("listen error:", e) |
||||||
|
} |
||||||
|
go http.Serve(l, nil) |
||||||
|
} |
||||||
|
|
||||||
|
// Let servers start
|
||||||
|
time.Sleep(10 * time.Millisecond) |
||||||
|
} |
||||||
|
|
||||||
|
// TestMain initializes the testing framework
|
||||||
|
func TestMain(m *testing.M) { |
||||||
|
const rpcPath = "/dsync" |
||||||
|
|
||||||
|
rand.Seed(time.Now().UTC().UnixNano()) |
||||||
|
|
||||||
|
nodes := make([]string, 4) // list of node IP addrs or hostname with ports.
|
||||||
|
for i := range nodes { |
||||||
|
nodes[i] = fmt.Sprintf("127.0.0.1:%d", i+12345) |
||||||
|
} |
||||||
|
for i := range nodes { |
||||||
|
rpcPaths = append(rpcPaths, rpcPath+"-"+strconv.Itoa(i)) |
||||||
|
} |
||||||
|
|
||||||
|
// Initialize net/rpc clients for dsync.
|
||||||
|
var clnts []NetLocker |
||||||
|
for i := 0; i < len(nodes); i++ { |
||||||
|
clnts = append(clnts, newClient(nodes[i], rpcPaths[i])) |
||||||
|
} |
||||||
|
|
||||||
|
var err error |
||||||
|
ds, err = New(clnts) |
||||||
|
if err != nil { |
||||||
|
log.Fatalf("set nodes failed with %v", err) |
||||||
|
} |
||||||
|
|
||||||
|
startRPCServers(nodes) |
||||||
|
|
||||||
|
os.Exit(m.Run()) |
||||||
|
} |
||||||
|
|
||||||
|
func TestSimpleLock(t *testing.T) { |
||||||
|
|
||||||
|
dm := NewDRWMutex(context.Background(), "test", ds) |
||||||
|
|
||||||
|
dm.Lock(id, source) |
||||||
|
|
||||||
|
// fmt.Println("Lock acquired, waiting...")
|
||||||
|
time.Sleep(2500 * time.Millisecond) |
||||||
|
|
||||||
|
dm.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
func TestSimpleLockUnlockMultipleTimes(t *testing.T) { |
||||||
|
|
||||||
|
dm := NewDRWMutex(context.Background(), "test", ds) |
||||||
|
|
||||||
|
dm.Lock(id, source) |
||||||
|
time.Sleep(time.Duration(10+(rand.Float32()*50)) * time.Millisecond) |
||||||
|
dm.Unlock() |
||||||
|
|
||||||
|
dm.Lock(id, source) |
||||||
|
time.Sleep(time.Duration(10+(rand.Float32()*50)) * time.Millisecond) |
||||||
|
dm.Unlock() |
||||||
|
|
||||||
|
dm.Lock(id, source) |
||||||
|
time.Sleep(time.Duration(10+(rand.Float32()*50)) * time.Millisecond) |
||||||
|
dm.Unlock() |
||||||
|
|
||||||
|
dm.Lock(id, source) |
||||||
|
time.Sleep(time.Duration(10+(rand.Float32()*50)) * time.Millisecond) |
||||||
|
dm.Unlock() |
||||||
|
|
||||||
|
dm.Lock(id, source) |
||||||
|
time.Sleep(time.Duration(10+(rand.Float32()*50)) * time.Millisecond) |
||||||
|
dm.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
// Test two locks for same resource, one succeeds, one fails (after timeout)
|
||||||
|
func TestTwoSimultaneousLocksForSameResource(t *testing.T) { |
||||||
|
|
||||||
|
dm1st := NewDRWMutex(context.Background(), "aap", ds) |
||||||
|
dm2nd := NewDRWMutex(context.Background(), "aap", ds) |
||||||
|
|
||||||
|
dm1st.Lock(id, source) |
||||||
|
|
||||||
|
// Release lock after 10 seconds
|
||||||
|
go func() { |
||||||
|
time.Sleep(10 * time.Second) |
||||||
|
// fmt.Println("Unlocking dm1")
|
||||||
|
|
||||||
|
dm1st.Unlock() |
||||||
|
}() |
||||||
|
|
||||||
|
dm2nd.Lock(id, source) |
||||||
|
|
||||||
|
// fmt.Printf("2nd lock obtained after 1st lock is released\n")
|
||||||
|
time.Sleep(2500 * time.Millisecond) |
||||||
|
|
||||||
|
dm2nd.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
// Test three locks for same resource, one succeeds, one fails (after timeout)
|
||||||
|
func TestThreeSimultaneousLocksForSameResource(t *testing.T) { |
||||||
|
|
||||||
|
dm1st := NewDRWMutex(context.Background(), "aap", ds) |
||||||
|
dm2nd := NewDRWMutex(context.Background(), "aap", ds) |
||||||
|
dm3rd := NewDRWMutex(context.Background(), "aap", ds) |
||||||
|
|
||||||
|
dm1st.Lock(id, source) |
||||||
|
|
||||||
|
// Release lock after 10 seconds
|
||||||
|
go func() { |
||||||
|
time.Sleep(10 * time.Second) |
||||||
|
// fmt.Println("Unlocking dm1")
|
||||||
|
|
||||||
|
dm1st.Unlock() |
||||||
|
}() |
||||||
|
|
||||||
|
var wg sync.WaitGroup |
||||||
|
wg.Add(2) |
||||||
|
|
||||||
|
go func() { |
||||||
|
defer wg.Done() |
||||||
|
|
||||||
|
dm2nd.Lock(id, source) |
||||||
|
|
||||||
|
// Release lock after 10 seconds
|
||||||
|
go func() { |
||||||
|
time.Sleep(2500 * time.Millisecond) |
||||||
|
// fmt.Println("Unlocking dm2")
|
||||||
|
|
||||||
|
dm2nd.Unlock() |
||||||
|
}() |
||||||
|
|
||||||
|
dm3rd.Lock(id, source) |
||||||
|
|
||||||
|
// fmt.Printf("3rd lock obtained after 1st & 2nd locks are released\n")
|
||||||
|
time.Sleep(2500 * time.Millisecond) |
||||||
|
|
||||||
|
dm3rd.Unlock() |
||||||
|
}() |
||||||
|
|
||||||
|
go func() { |
||||||
|
defer wg.Done() |
||||||
|
|
||||||
|
dm3rd.Lock(id, source) |
||||||
|
|
||||||
|
// Release lock after 10 seconds
|
||||||
|
go func() { |
||||||
|
time.Sleep(2500 * time.Millisecond) |
||||||
|
// fmt.Println("Unlocking dm3")
|
||||||
|
|
||||||
|
dm3rd.Unlock() |
||||||
|
}() |
||||||
|
|
||||||
|
dm2nd.Lock(id, source) |
||||||
|
|
||||||
|
// fmt.Printf("2nd lock obtained after 1st & 3rd locks are released\n")
|
||||||
|
time.Sleep(2500 * time.Millisecond) |
||||||
|
|
||||||
|
dm2nd.Unlock() |
||||||
|
}() |
||||||
|
|
||||||
|
wg.Wait() |
||||||
|
} |
||||||
|
|
||||||
|
// Test two locks for different resources, both succeed
|
||||||
|
func TestTwoSimultaneousLocksForDifferentResources(t *testing.T) { |
||||||
|
|
||||||
|
dm1 := NewDRWMutex(context.Background(), "aap", ds) |
||||||
|
dm2 := NewDRWMutex(context.Background(), "noot", ds) |
||||||
|
|
||||||
|
dm1.Lock(id, source) |
||||||
|
dm2.Lock(id, source) |
||||||
|
|
||||||
|
// fmt.Println("Both locks acquired, waiting...")
|
||||||
|
time.Sleep(2500 * time.Millisecond) |
||||||
|
|
||||||
|
dm1.Unlock() |
||||||
|
dm2.Unlock() |
||||||
|
|
||||||
|
time.Sleep(10 * time.Millisecond) |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from mutex_test.go
|
||||||
|
func HammerMutex(m *DRWMutex, loops int, cdone chan bool) { |
||||||
|
for i := 0; i < loops; i++ { |
||||||
|
m.Lock(id, source) |
||||||
|
m.Unlock() |
||||||
|
} |
||||||
|
cdone <- true |
||||||
|
} |
||||||
|
|
||||||
|
// Borrowed from mutex_test.go
|
||||||
|
func TestMutex(t *testing.T) { |
||||||
|
c := make(chan bool) |
||||||
|
m := NewDRWMutex(context.Background(), "test", ds) |
||||||
|
for i := 0; i < 10; i++ { |
||||||
|
go HammerMutex(m, 1000, c) |
||||||
|
} |
||||||
|
for i := 0; i < 10; i++ { |
||||||
|
<-c |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func BenchmarkMutexUncontended(b *testing.B) { |
||||||
|
type PaddedMutex struct { |
||||||
|
DRWMutex |
||||||
|
pad [128]uint8 |
||||||
|
} |
||||||
|
b.RunParallel(func(pb *testing.PB) { |
||||||
|
var mu PaddedMutex |
||||||
|
for pb.Next() { |
||||||
|
mu.Lock(id, source) |
||||||
|
mu.Unlock() |
||||||
|
} |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func benchmarkMutex(b *testing.B, slack, work bool) { |
||||||
|
mu := NewDRWMutex(context.Background(), "", ds) |
||||||
|
if slack { |
||||||
|
b.SetParallelism(10) |
||||||
|
} |
||||||
|
b.RunParallel(func(pb *testing.PB) { |
||||||
|
foo := 0 |
||||||
|
for pb.Next() { |
||||||
|
mu.Lock(id, source) |
||||||
|
mu.Unlock() |
||||||
|
if work { |
||||||
|
for i := 0; i < 100; i++ { |
||||||
|
foo *= 2 |
||||||
|
foo /= 2 |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
_ = foo |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func BenchmarkMutex(b *testing.B) { |
||||||
|
benchmarkMutex(b, false, false) |
||||||
|
} |
||||||
|
|
||||||
|
func BenchmarkMutexSlack(b *testing.B) { |
||||||
|
benchmarkMutex(b, true, false) |
||||||
|
} |
||||||
|
|
||||||
|
func BenchmarkMutexWork(b *testing.B) { |
||||||
|
benchmarkMutex(b, false, true) |
||||||
|
} |
||||||
|
|
||||||
|
func BenchmarkMutexWorkSlack(b *testing.B) { |
||||||
|
benchmarkMutex(b, true, true) |
||||||
|
} |
||||||
|
|
||||||
|
func BenchmarkMutexNoSpin(b *testing.B) { |
||||||
|
// This benchmark models a situation where spinning in the mutex should be
|
||||||
|
// non-profitable and allows to confirm that spinning does not do harm.
|
||||||
|
// To achieve this we create excess of goroutines most of which do local work.
|
||||||
|
// These goroutines yield during local work, so that switching from
|
||||||
|
// a blocked goroutine to other goroutines is profitable.
|
||||||
|
// As a matter of fact, this benchmark still triggers some spinning in the mutex.
|
||||||
|
m := NewDRWMutex(context.Background(), "", ds) |
||||||
|
var acc0, acc1 uint64 |
||||||
|
b.SetParallelism(4) |
||||||
|
b.RunParallel(func(pb *testing.PB) { |
||||||
|
c := make(chan bool) |
||||||
|
var data [4 << 10]uint64 |
||||||
|
for i := 0; pb.Next(); i++ { |
||||||
|
if i%4 == 0 { |
||||||
|
m.Lock(id, source) |
||||||
|
acc0 -= 100 |
||||||
|
acc1 += 100 |
||||||
|
m.Unlock() |
||||||
|
} else { |
||||||
|
for i := 0; i < len(data); i += 4 { |
||||||
|
data[i]++ |
||||||
|
} |
||||||
|
// Elaborate way to say runtime.Gosched
|
||||||
|
// that does not put the goroutine onto global runq.
|
||||||
|
go func() { |
||||||
|
c <- true |
||||||
|
}() |
||||||
|
<-c |
||||||
|
} |
||||||
|
} |
||||||
|
}) |
||||||
|
} |
||||||
|
|
||||||
|
func BenchmarkMutexSpin(b *testing.B) { |
||||||
|
// This benchmark models a situation where spinning in the mutex should be
|
||||||
|
// profitable. To achieve this we create a goroutine per-proc.
|
||||||
|
// These goroutines access considerable amount of local data so that
|
||||||
|
// unnecessary rescheduling is penalized by cache misses.
|
||||||
|
m := NewDRWMutex(context.Background(), "", ds) |
||||||
|
var acc0, acc1 uint64 |
||||||
|
b.RunParallel(func(pb *testing.PB) { |
||||||
|
var data [16 << 10]uint64 |
||||||
|
for i := 0; pb.Next(); i++ { |
||||||
|
m.Lock(id, source) |
||||||
|
acc0 -= 100 |
||||||
|
acc1 += 100 |
||||||
|
m.Unlock() |
||||||
|
for i := 0; i < len(data); i += 4 { |
||||||
|
data[i]++ |
||||||
|
} |
||||||
|
} |
||||||
|
}) |
||||||
|
} |
@ -0,0 +1,142 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2017 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package dsync |
||||||
|
|
||||||
|
import ( |
||||||
|
"math/rand" |
||||||
|
"sync" |
||||||
|
"time" |
||||||
|
) |
||||||
|
|
||||||
|
// lockedRandSource provides protected rand source, implements rand.Source interface.
|
||||||
|
type lockedRandSource struct { |
||||||
|
lk sync.Mutex |
||||||
|
src rand.Source |
||||||
|
} |
||||||
|
|
||||||
|
// Int63 returns a non-negative pseudo-random 63-bit integer as an
|
||||||
|
// int64.
|
||||||
|
func (r *lockedRandSource) Int63() (n int64) { |
||||||
|
r.lk.Lock() |
||||||
|
n = r.src.Int63() |
||||||
|
r.lk.Unlock() |
||||||
|
return |
||||||
|
} |
||||||
|
|
||||||
|
// Seed uses the provided seed value to initialize the generator to a
|
||||||
|
// deterministic state.
|
||||||
|
func (r *lockedRandSource) Seed(seed int64) { |
||||||
|
r.lk.Lock() |
||||||
|
r.src.Seed(seed) |
||||||
|
r.lk.Unlock() |
||||||
|
} |
||||||
|
|
||||||
|
// MaxJitter will randomize over the full exponential backoff time
|
||||||
|
const MaxJitter = 1.0 |
||||||
|
|
||||||
|
// NoJitter disables the use of jitter for randomizing the
|
||||||
|
// exponential backoff time
|
||||||
|
const NoJitter = 0.0 |
||||||
|
|
||||||
|
// Global random source for fetching random values.
|
||||||
|
var globalRandomSource = rand.New(&lockedRandSource{ |
||||||
|
src: rand.NewSource(time.Now().UTC().UnixNano()), |
||||||
|
}) |
||||||
|
|
||||||
|
// newRetryTimerJitter creates a timer with exponentially increasing delays
|
||||||
|
// until the maximum retry attempts are reached. - this function is a fully
|
||||||
|
// configurable version, meant for only advanced use cases. For the most part
|
||||||
|
// one should use newRetryTimerSimple and newRetryTimer.
|
||||||
|
func newRetryTimerWithJitter(unit time.Duration, cap time.Duration, jitter float64, doneCh <-chan struct{}) <-chan int { |
||||||
|
attemptCh := make(chan int) |
||||||
|
|
||||||
|
// normalize jitter to the range [0, 1.0]
|
||||||
|
if jitter < NoJitter { |
||||||
|
jitter = NoJitter |
||||||
|
} |
||||||
|
if jitter > MaxJitter { |
||||||
|
jitter = MaxJitter |
||||||
|
} |
||||||
|
|
||||||
|
// computes the exponential backoff duration according to
|
||||||
|
// https://www.awsarchitectureblog.com/2015/03/backoff.html
|
||||||
|
exponentialBackoffWait := func(attempt int) time.Duration { |
||||||
|
// 1<<uint(attempt) below could overflow, so limit the value of attempt
|
||||||
|
maxAttempt := 30 |
||||||
|
if attempt > maxAttempt { |
||||||
|
attempt = maxAttempt |
||||||
|
} |
||||||
|
//sleep = random_between(0, min(cap, base * 2 ** attempt))
|
||||||
|
sleep := unit * time.Duration(1<<uint(attempt)) |
||||||
|
if sleep > cap { |
||||||
|
sleep = cap |
||||||
|
} |
||||||
|
if jitter != NoJitter { |
||||||
|
sleep -= time.Duration(globalRandomSource.Float64() * float64(sleep) * jitter) |
||||||
|
} |
||||||
|
return sleep |
||||||
|
} |
||||||
|
|
||||||
|
go func() { |
||||||
|
defer close(attemptCh) |
||||||
|
nextBackoff := 0 |
||||||
|
// Channel used to signal after the expiry of backoff wait seconds.
|
||||||
|
var timer *time.Timer |
||||||
|
for { |
||||||
|
select { // Attempts starts.
|
||||||
|
case attemptCh <- nextBackoff: |
||||||
|
nextBackoff++ |
||||||
|
case <-doneCh: |
||||||
|
// Stop the routine.
|
||||||
|
return |
||||||
|
} |
||||||
|
timer = time.NewTimer(exponentialBackoffWait(nextBackoff)) |
||||||
|
// wait till next backoff time or till doneCh gets a message.
|
||||||
|
select { |
||||||
|
case <-timer.C: |
||||||
|
case <-doneCh: |
||||||
|
// stop the timer and return.
|
||||||
|
timer.Stop() |
||||||
|
return |
||||||
|
} |
||||||
|
|
||||||
|
} |
||||||
|
}() |
||||||
|
|
||||||
|
// Start reading..
|
||||||
|
return attemptCh |
||||||
|
} |
||||||
|
|
||||||
|
// Default retry constants.
|
||||||
|
const ( |
||||||
|
defaultRetryUnit = time.Second // 1 second.
|
||||||
|
defaultRetryCap = 1 * time.Second // 1 second.
|
||||||
|
) |
||||||
|
|
||||||
|
// newRetryTimer creates a timer with exponentially increasing delays
|
||||||
|
// until the maximum retry attempts are reached. - this function provides
|
||||||
|
// resulting retry values to be of maximum jitter.
|
||||||
|
func newRetryTimer(unit time.Duration, cap time.Duration, doneCh <-chan struct{}) <-chan int { |
||||||
|
return newRetryTimerWithJitter(unit, cap, MaxJitter, doneCh) |
||||||
|
} |
||||||
|
|
||||||
|
// newRetryTimerSimple creates a timer with exponentially increasing delays
|
||||||
|
// until the maximum retry attempts are reached. - this function is a
|
||||||
|
// simpler version with all default values.
|
||||||
|
func newRetryTimerSimple(doneCh <-chan struct{}) <-chan int { |
||||||
|
return newRetryTimerWithJitter(defaultRetryUnit, defaultRetryCap, MaxJitter, doneCh) |
||||||
|
} |
@ -0,0 +1,82 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2017 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package dsync |
||||||
|
|
||||||
|
import ( |
||||||
|
"testing" |
||||||
|
"time" |
||||||
|
) |
||||||
|
|
||||||
|
// Tests for retry timer.
|
||||||
|
func TestRetryTimerSimple(t *testing.T) { |
||||||
|
doneCh := make(chan struct{}) |
||||||
|
attemptCh := newRetryTimerSimple(doneCh) |
||||||
|
i := <-attemptCh |
||||||
|
if i != 0 { |
||||||
|
close(doneCh) |
||||||
|
t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i) |
||||||
|
} |
||||||
|
i = <-attemptCh |
||||||
|
if i <= 0 { |
||||||
|
close(doneCh) |
||||||
|
t.Fatalf("Invalid attempt counter returned should be greater than 0, found %d instead", i) |
||||||
|
} |
||||||
|
close(doneCh) |
||||||
|
_, ok := <-attemptCh |
||||||
|
if ok { |
||||||
|
t.Fatal("Attempt counter should be closed") |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Test retry time with no jitter.
|
||||||
|
func TestRetryTimerWithNoJitter(t *testing.T) { |
||||||
|
doneCh := make(chan struct{}) |
||||||
|
// No jitter
|
||||||
|
attemptCh := newRetryTimerWithJitter(time.Millisecond, 5*time.Millisecond, NoJitter, doneCh) |
||||||
|
i := <-attemptCh |
||||||
|
if i != 0 { |
||||||
|
close(doneCh) |
||||||
|
t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i) |
||||||
|
} |
||||||
|
// Loop through the maximum possible attempt.
|
||||||
|
for i = range attemptCh { |
||||||
|
if i == 30 { |
||||||
|
close(doneCh) |
||||||
|
} |
||||||
|
} |
||||||
|
_, ok := <-attemptCh |
||||||
|
if ok { |
||||||
|
t.Fatal("Attempt counter should be closed") |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Test retry time with Jitter greater than MaxJitter.
|
||||||
|
func TestRetryTimerWithJitter(t *testing.T) { |
||||||
|
doneCh := make(chan struct{}) |
||||||
|
// Jitter will be set back to 1.0
|
||||||
|
attemptCh := newRetryTimerWithJitter(time.Second, 30*time.Second, 2.0, doneCh) |
||||||
|
i := <-attemptCh |
||||||
|
if i != 0 { |
||||||
|
close(doneCh) |
||||||
|
t.Fatalf("Invalid attempt counter returned should be 0, found %d instead", i) |
||||||
|
} |
||||||
|
close(doneCh) |
||||||
|
_, ok := <-attemptCh |
||||||
|
if ok { |
||||||
|
t.Fatal("Attempt counter should be closed") |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,111 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package dsync_test |
||||||
|
|
||||||
|
import ( |
||||||
|
"net/rpc" |
||||||
|
"sync" |
||||||
|
|
||||||
|
. "github.com/minio/minio/pkg/dsync" |
||||||
|
) |
||||||
|
|
||||||
|
// ReconnectRPCClient is a wrapper type for rpc.Client which provides reconnect on first failure.
|
||||||
|
type ReconnectRPCClient struct { |
||||||
|
mutex sync.Mutex |
||||||
|
rpc *rpc.Client |
||||||
|
addr string |
||||||
|
endpoint string |
||||||
|
} |
||||||
|
|
||||||
|
// newClient constructs a ReconnectRPCClient object with addr and endpoint initialized.
|
||||||
|
// It _doesn't_ connect to the remote endpoint. See Call method to see when the
|
||||||
|
// connect happens.
|
||||||
|
func newClient(addr, endpoint string) NetLocker { |
||||||
|
return &ReconnectRPCClient{ |
||||||
|
addr: addr, |
||||||
|
endpoint: endpoint, |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Close closes the underlying socket file descriptor.
|
||||||
|
func (rpcClient *ReconnectRPCClient) Close() error { |
||||||
|
rpcClient.mutex.Lock() |
||||||
|
defer rpcClient.mutex.Unlock() |
||||||
|
// If rpc client has not connected yet there is nothing to close.
|
||||||
|
if rpcClient.rpc == nil { |
||||||
|
return nil |
||||||
|
} |
||||||
|
// Reset rpcClient.rpc to allow for subsequent calls to use a new
|
||||||
|
// (socket) connection.
|
||||||
|
clnt := rpcClient.rpc |
||||||
|
rpcClient.rpc = nil |
||||||
|
return clnt.Close() |
||||||
|
} |
||||||
|
|
||||||
|
// Call makes a RPC call to the remote endpoint using the default codec, namely encoding/gob.
|
||||||
|
func (rpcClient *ReconnectRPCClient) Call(serviceMethod string, args interface{}, reply interface{}) (err error) { |
||||||
|
rpcClient.mutex.Lock() |
||||||
|
defer rpcClient.mutex.Unlock() |
||||||
|
dialCall := func() error { |
||||||
|
// If the rpc.Client is nil, we attempt to (re)connect with the remote endpoint.
|
||||||
|
if rpcClient.rpc == nil { |
||||||
|
clnt, derr := rpc.DialHTTPPath("tcp", rpcClient.addr, rpcClient.endpoint) |
||||||
|
if derr != nil { |
||||||
|
return derr |
||||||
|
} |
||||||
|
rpcClient.rpc = clnt |
||||||
|
} |
||||||
|
// If the RPC fails due to a network-related error, then we reset
|
||||||
|
// rpc.Client for a subsequent reconnect.
|
||||||
|
return rpcClient.rpc.Call(serviceMethod, args, reply) |
||||||
|
} |
||||||
|
if err = dialCall(); err == rpc.ErrShutdown { |
||||||
|
rpcClient.rpc.Close() |
||||||
|
rpcClient.rpc = nil |
||||||
|
err = dialCall() |
||||||
|
} |
||||||
|
return err |
||||||
|
} |
||||||
|
|
||||||
|
func (rpcClient *ReconnectRPCClient) RLock(args LockArgs) (status bool, err error) { |
||||||
|
err = rpcClient.Call("Dsync.RLock", &args, &status) |
||||||
|
return status, err |
||||||
|
} |
||||||
|
|
||||||
|
func (rpcClient *ReconnectRPCClient) Lock(args LockArgs) (status bool, err error) { |
||||||
|
err = rpcClient.Call("Dsync.Lock", &args, &status) |
||||||
|
return status, err |
||||||
|
} |
||||||
|
|
||||||
|
func (rpcClient *ReconnectRPCClient) RUnlock(args LockArgs) (status bool, err error) { |
||||||
|
err = rpcClient.Call("Dsync.RUnlock", &args, &status) |
||||||
|
return status, err |
||||||
|
} |
||||||
|
|
||||||
|
func (rpcClient *ReconnectRPCClient) Unlock(args LockArgs) (status bool, err error) { |
||||||
|
err = rpcClient.Call("Dsync.Unlock", &args, &status) |
||||||
|
return status, err |
||||||
|
} |
||||||
|
|
||||||
|
func (rpcClient *ReconnectRPCClient) ForceUnlock(args LockArgs) (status bool, err error) { |
||||||
|
err = rpcClient.Call("Dsync.ForceUnlock", &args, &status) |
||||||
|
return status, err |
||||||
|
} |
||||||
|
|
||||||
|
func (rpcClient *ReconnectRPCClient) String() string { |
||||||
|
return "http://" + rpcClient.addr + "/" + rpcClient.endpoint |
||||||
|
} |
@ -0,0 +1,59 @@ |
|||||||
|
/* |
||||||
|
* Minio Cloud Storage, (C) 2016 Minio, Inc. |
||||||
|
* |
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License"); |
||||||
|
* you may not use this file except in compliance with the License. |
||||||
|
* You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package dsync |
||||||
|
|
||||||
|
// LockArgs is minimal required values for any dsync compatible lock operation.
|
||||||
|
type LockArgs struct { |
||||||
|
// Unique ID of lock/unlock request.
|
||||||
|
UID string |
||||||
|
|
||||||
|
// Resource contains a entity to be locked/unlocked.
|
||||||
|
Resource string |
||||||
|
|
||||||
|
// Source contains the line number, function and file name of the code
|
||||||
|
// on the client node that requested the lock.
|
||||||
|
Source string |
||||||
|
} |
||||||
|
|
||||||
|
// NetLocker is dsync compatible locker interface.
|
||||||
|
type NetLocker interface { |
||||||
|
// Do read lock for given LockArgs. It should return
|
||||||
|
// * a boolean to indicate success/failure of the operation
|
||||||
|
// * an error on failure of lock request operation.
|
||||||
|
RLock(args LockArgs) (bool, error) |
||||||
|
|
||||||
|
// Do write lock for given LockArgs. It should return
|
||||||
|
// * a boolean to indicate success/failure of the operation
|
||||||
|
// * an error on failure of lock request operation.
|
||||||
|
Lock(args LockArgs) (bool, error) |
||||||
|
|
||||||
|
// Do read unlock for given LockArgs. It should return
|
||||||
|
// * a boolean to indicate success/failure of the operation
|
||||||
|
// * an error on failure of unlock request operation.
|
||||||
|
RUnlock(args LockArgs) (bool, error) |
||||||
|
|
||||||
|
// Do write unlock for given LockArgs. It should return
|
||||||
|
// * a boolean to indicate success/failure of the operation
|
||||||
|
// * an error on failure of unlock request operation.
|
||||||
|
Unlock(args LockArgs) (bool, error) |
||||||
|
|
||||||
|
// Returns underlying endpoint of this lock client instance.
|
||||||
|
String() string |
||||||
|
|
||||||
|
// Close closes any underlying connection to the service endpoint
|
||||||
|
Close() error |
||||||
|
} |
Loading…
Reference in new issue