rename all remaining packages to internal/ (#12418)

This is to ensure that there are no projects
that try to import `minio/minio/pkg` into
their own repo. Any such common packages should
go to `https://github.com/minio/pkg`
This commit is contained in:
Harshavardhana
2021-06-01 14:59:40 -07:00
committed by GitHub
parent bf87c4b1e4
commit 1f262daf6f
540 changed files with 757 additions and 778 deletions

3
internal/dsync/.gitignore vendored Normal file
View File

@@ -0,0 +1,3 @@
dsync.test
coverage.txt
*.out

635
internal/dsync/drwmutex.go Normal file
View File

@@ -0,0 +1,635 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package dsync
import (
"context"
"errors"
"math/rand"
"os"
"sync"
"time"
"github.com/minio/pkg/console"
)
// 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"
}
func log(format string, data ...interface{}) {
if dsyncLog {
console.Printf(format, data...)
}
}
// dRWMutexAcquireTimeout - tolerance limit to wait for lock acquisition before.
const drwMutexAcquireTimeout = 1 * time.Second // 1 second.
// dRWMutexRefreshTimeout - timeout for the refresh call
const drwMutexRefreshCallTimeout = 5 * time.Second
// dRWMutexUnlockTimeout - timeout for the unlock call
const drwMutexUnlockCallTimeout = 30 * time.Second
// dRWMutexRefreshInterval - the interval between two refresh calls
const drwMutexRefreshInterval = 10 * time.Second
const drwMutexInfinite = 1<<63 - 1
// A DRWMutex is a distributed mutual exclusion lock.
type DRWMutex struct {
Names []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
cancelRefresh context.CancelFunc
}
// 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(clnt *Dsync, names ...string) *DRWMutex {
restClnts, _ := clnt.GetLockers()
return &DRWMutex{
writeLocks: make([]string, len(restClnts)),
Names: names,
clnt: clnt,
}
}
// 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(context.Background(), nil, id, source, isReadLock, Options{
Timeout: drwMutexInfinite,
})
}
// Options lock options.
type Options struct {
Timeout time.Duration
}
// 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(ctx context.Context, cancel context.CancelFunc, id, source string, opts Options) (locked bool) {
isReadLock := false
return dm.lockBlocking(ctx, cancel, id, source, isReadLock, opts)
}
// 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(context.Background(), nil, id, source, isReadLock, Options{
Timeout: drwMutexInfinite,
})
}
// 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(ctx context.Context, cancel context.CancelFunc, id, source string, opts Options) (locked bool) {
isReadLock := true
return dm.lockBlocking(ctx, cancel, id, source, isReadLock, opts)
}
const (
lockRetryInterval = 1 * time.Second
)
// 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(ctx context.Context, lockLossCallback func(), id, source string, isReadLock bool, opts Options) (locked bool) {
restClnts, _ := dm.clnt.GetLockers()
r := rand.New(rand.NewSource(time.Now().UnixNano()))
// Create lock array to capture the successful lockers
locks := make([]string, len(restClnts))
log("lockBlocking %s/%s for %#v: lockType readLock(%t), additional opts: %#v\n", id, source, dm.Names, isReadLock, opts)
// Add total timeout
ctx, cancel := context.WithTimeout(ctx, opts.Timeout)
defer cancel()
// Tolerance is not set, defaults to half of the locker clients.
tolerance := len(restClnts) / 2
// Quorum is effectively = total clients subtracted with tolerance limit
quorum := len(restClnts) - tolerance
if !isReadLock {
// In situations for write locks, as a special case
// to avoid split brains we make sure to acquire
// quorum + 1 when tolerance is exactly half of the
// total locker clients.
if quorum == tolerance {
quorum++
}
}
tolerance = len(restClnts) - quorum
for {
select {
case <-ctx.Done():
return false
default:
// Try to acquire the lock.
if locked = lock(ctx, dm.clnt, &locks, id, source, isReadLock, tolerance, quorum, dm.Names...); locked {
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, len(restClnts)))
// and copy stack array into last spot
copy(dm.readersLocks[len(dm.readersLocks)-1], locks[:])
} else {
copy(dm.writeLocks, locks[:])
}
dm.m.Unlock()
log("lockBlocking %s/%s for %#v: granted\n", id, source, dm.Names)
// Refresh lock continuously and cancel if there is no quorum in the lock anymore
dm.startContinousLockRefresh(lockLossCallback, id, source, quorum)
return locked
}
time.Sleep(time.Duration(r.Float64() * float64(lockRetryInterval)))
}
}
}
func (dm *DRWMutex) startContinousLockRefresh(lockLossCallback func(), id, source string, quorum int) {
ctx, cancel := context.WithCancel(context.Background())
dm.m.Lock()
dm.cancelRefresh = cancel
dm.m.Unlock()
go func() {
defer cancel()
refreshTimer := time.NewTimer(drwMutexRefreshInterval)
defer refreshTimer.Stop()
for {
select {
case <-ctx.Done():
return
case <-refreshTimer.C:
refreshTimer.Reset(drwMutexRefreshInterval)
refreshed, err := refresh(ctx, dm.clnt, id, source, quorum, dm.Names...)
if err == nil && !refreshed {
if lockLossCallback != nil {
lockLossCallback()
}
return
}
}
}
}()
}
type refreshResult struct {
offline bool
succeeded bool
}
func refresh(ctx context.Context, ds *Dsync, id, source string, quorum int, lockNames ...string) (bool, error) {
restClnts, owner := ds.GetLockers()
// Create buffered channel of size equal to total number of nodes.
ch := make(chan refreshResult, len(restClnts))
var wg sync.WaitGroup
for index, c := range restClnts {
wg.Add(1)
// Send refresh request to all nodes
go func(index int, c NetLocker) {
defer wg.Done()
if c == nil {
ch <- refreshResult{offline: true}
return
}
args := LockArgs{
Owner: owner,
UID: id,
Resources: lockNames,
Source: source,
Quorum: quorum,
}
ctx, cancel := context.WithTimeout(ctx, drwMutexRefreshCallTimeout)
defer cancel()
refreshed, err := c.Refresh(ctx, args)
if refreshed && err == nil {
ch <- refreshResult{succeeded: true}
} else {
if err != nil {
ch <- refreshResult{offline: true}
log("dsync: Unable to call Refresh failed with %s for %#v at %s\n", err, args, c)
} else {
ch <- refreshResult{succeeded: false}
log("dsync: Refresh returned false for %#v at %s\n", args, c)
}
}
}(index, c)
}
// Wait until we have either
//
// a) received all refresh responses
// b) received too many refreshed for quorum to be still possible
// c) timed out
//
i, refreshFailed, refreshSucceeded := 0, 0, 0
done := false
for ; i < len(restClnts); i++ {
select {
case refresh := <-ch:
if refresh.offline {
continue
}
if refresh.succeeded {
refreshSucceeded++
} else {
refreshFailed++
}
if refreshFailed > quorum {
// We know that we are not going to succeed with refresh
done = true
}
case <-ctx.Done():
// Refreshing is canceled
return false, ctx.Err()
}
if done {
break
}
}
refreshQuorum := refreshSucceeded >= quorum
if !refreshQuorum {
refreshQuorum = refreshFailed < quorum
}
// We may have some unused results in ch, release them async.
go func() {
wg.Wait()
close(ch)
for range ch {
}
}()
return refreshQuorum, nil
}
// lock tries to acquire the distributed lock, returning true or false.
func lock(ctx context.Context, ds *Dsync, locks *[]string, id, source string, isReadLock bool, tolerance, quorum int, lockNames ...string) bool {
for i := range *locks {
(*locks)[i] = ""
}
restClnts, owner := ds.GetLockers()
// Create buffered channel of size equal to total number of nodes.
ch := make(chan Granted, len(restClnts))
var wg sync.WaitGroup
// Combined timeout for the lock attempt.
ctx, cancel := context.WithTimeout(ctx, drwMutexAcquireTimeout)
defer cancel()
for index, c := range restClnts {
wg.Add(1)
// broadcast lock request to all nodes
go func(index int, isReadLock bool, c NetLocker) {
defer wg.Done()
g := Granted{index: index}
if c == nil {
log("dsync: nil locker\n")
ch <- g
return
}
args := LockArgs{
Owner: owner,
UID: id,
Resources: lockNames,
Source: source,
Quorum: quorum,
}
var locked bool
var err error
if isReadLock {
if locked, err = c.RLock(context.Background(), args); err != nil {
log("dsync: Unable to call RLock failed with %s for %#v at %s\n", err, args, c)
}
} else {
if locked, err = c.Lock(context.Background(), args); err != nil {
log("dsync: Unable to call Lock failed with %s for %#v at %s\n", err, args, c)
}
}
if locked {
g.lockUID = args.UID
}
ch <- g
}(index, isReadLock, c)
}
// Wait until we have either
//
// a) received all lock responses
// b) received too many 'non-'locks for quorum to be still possible
// c) timed out
//
i, locksFailed := 0, 0
done := false
for ; i < len(restClnts); 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 locksFailed > tolerance {
// 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
}
}
case <-ctx.Done():
// Capture timedout locks as failed or took too long
locksFailed++
if locksFailed > tolerance {
// 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
}
}
if done {
break
}
}
quorumLocked := checkQuorumLocked(locks, quorum) && locksFailed <= tolerance
if !quorumLocked {
log("Releasing all acquired locks now abandoned after quorum was not met\n")
if !releaseAll(ds, tolerance, owner, locks, isReadLock, restClnts, lockNames...) {
log("Unable to release acquired locks, stale locks might be present\n")
}
}
// We may have some unused results in ch, release them async.
go func() {
wg.Wait()
close(ch)
for grantToBeReleased := range ch {
if grantToBeReleased.isLocked() {
// release abandoned lock
log("Releasing abandoned lock\n")
sendRelease(ds, restClnts[grantToBeReleased.index],
owner, grantToBeReleased.lockUID, isReadLock, lockNames...)
}
}
}()
return quorumLocked
}
// checkFailedUnlocks determines whether we have sufficiently unlocked all
// resources to ensure no deadlocks for future callers
func checkFailedUnlocks(locks []string, tolerance int) bool {
unlocksFailed := 0
for lockID := range locks {
if isLocked(locks[lockID]) {
unlocksFailed++
}
}
// Unlock failures are higher than tolerance limit
// for this instance of unlocker, we should let the
// caller know that lock is not successfully released
// yet.
if len(locks)-tolerance == tolerance {
// Incase of split brain scenarios where
// tolerance is exactly half of the len(*locks)
// then we need to make sure we have unlocked
// upto tolerance+1 - especially for RUnlock
// to ensure that we don't end up with active
// read locks on the resource after unlocking
// only half of the lockers.
return unlocksFailed >= tolerance
}
return unlocksFailed > tolerance
}
// checkQuorumLocked determines whether we have locked the required quorum of underlying locks or not
func checkQuorumLocked(locks *[]string, quorum int) bool {
count := 0
for _, uid := range *locks {
if isLocked(uid) {
count++
}
}
return count >= quorum
}
// releaseAll releases all locks that are marked as locked
func releaseAll(ds *Dsync, tolerance int, owner string, locks *[]string, isReadLock bool, restClnts []NetLocker, lockNames ...string) bool {
var wg sync.WaitGroup
for lockID := range restClnts {
wg.Add(1)
go func(lockID int) {
defer wg.Done()
if isLocked((*locks)[lockID]) {
if sendRelease(ds, restClnts[lockID], owner, (*locks)[lockID], isReadLock, lockNames...) {
(*locks)[lockID] = ""
}
}
}(lockID)
}
wg.Wait()
// Return true if releaseAll was successful, otherwise we return 'false'
// to indicate we haven't sufficiently unlocked lockers to avoid deadlocks.
//
// Caller may use this as an indication to call again.
return !checkFailedUnlocks(*locks, tolerance)
}
// Unlock unlocks the write lock.
//
// It is a run-time error if dm is not locked on entry to Unlock.
func (dm *DRWMutex) Unlock() {
dm.m.Lock()
dm.cancelRefresh()
dm.m.Unlock()
restClnts, owner := dm.clnt.GetLockers()
// create temp array on stack
locks := make([]string, len(restClnts))
{
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[:])
}
// Tolerance is not set, defaults to half of the locker clients.
tolerance := len(restClnts) / 2
isReadLock := false
r := rand.New(rand.NewSource(time.Now().UnixNano()))
for !releaseAll(dm.clnt, tolerance, owner, &locks, isReadLock, restClnts, dm.Names...) {
time.Sleep(time.Duration(r.Float64() * float64(lockRetryInterval)))
}
}
// 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() {
dm.m.Lock()
dm.cancelRefresh()
dm.m.Unlock()
// create temp array on stack
restClnts, owner := dm.clnt.GetLockers()
locks := make([]string, len(restClnts))
{
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:]
}
// Tolerance is not set, defaults to half of the locker clients.
tolerance := len(restClnts) / 2
isReadLock := true
r := rand.New(rand.NewSource(time.Now().UnixNano()))
for !releaseAll(dm.clnt, tolerance, owner, &locks, isReadLock, restClnts, dm.Names...) {
time.Sleep(time.Duration(r.Float64() * float64(lockRetryInterval)))
}
}
// sendRelease sends a release message to a node that previously granted a lock
func sendRelease(ds *Dsync, c NetLocker, owner string, uid string, isReadLock bool, names ...string) bool {
if c == nil {
log("Unable to call RUnlock failed with %s\n", errors.New("netLocker is offline"))
return false
}
args := LockArgs{
Owner: owner,
UID: uid,
Resources: names,
}
ctx, cancel := context.WithTimeout(context.Background(), drwMutexUnlockCallTimeout)
defer cancel()
if isReadLock {
if _, err := c.RUnlock(ctx, args); err != nil {
log("dsync: Unable to call RUnlock failed with %s for %#v at %s\n", err, args, c)
return false
}
} else {
if _, err := c.Unlock(ctx, args); err != nil {
log("dsync: Unable to call Unlock failed with %s for %#v at %s\n", err, args, c)
return false
}
}
return true
}

View File

@@ -0,0 +1,348 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package dsync
import (
"context"
"fmt"
"runtime"
"sync/atomic"
"testing"
"time"
)
const (
id = "1234-5678"
source = "main.go"
)
func testSimpleWriteLock(t *testing.T, duration time.Duration) (locked bool) {
drwm := NewDRWMutex(ds, "simplelock")
ctx1, cancel1 := context.WithCancel(context.Background())
if !drwm.GetRLock(ctx1, cancel1, id, source, Options{Timeout: time.Second}) {
panic("Failed to acquire read lock")
}
// fmt.Println("1st read lock acquired, waiting...")
ctx2, cancel2 := context.WithCancel(context.Background())
if !drwm.GetRLock(ctx2, cancel2, id, source, Options{Timeout: 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...")
ctx3, cancel3 := context.WithCancel(context.Background())
locked = drwm.GetLock(ctx3, cancel3, id, source, Options{Timeout: 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(ds, "duallock")
// fmt.Println("Getting initial write lock")
ctx1, cancel1 := context.WithCancel(context.Background())
if !drwm.GetLock(ctx1, cancel1, id, source, Options{Timeout: 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...")
ctx2, cancel2 := context.WithCancel(context.Background())
locked = drwm.GetLock(ctx2, cancel2, id, source, Options{Timeout: 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(ctx context.Context, m *DRWMutex, clocked, cunlock, cdone chan bool) {
if m.GetRLock(ctx, nil, id, source, Options{Timeout: time.Second}) {
clocked <- true
<-cunlock
m.RUnlock()
cdone <- true
}
}
// Borrowed from rwmutex_test.go
func doTestParallelReaders(numReaders, gomaxprocs int) {
runtime.GOMAXPROCS(gomaxprocs)
m := NewDRWMutex(ds, "test-parallel")
clocked := make(chan bool)
cunlock := make(chan bool)
cdone := make(chan bool)
for i := 0; i < numReaders; i++ {
go parallelReader(context.Background(), 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(context.Background(), nil, id, source, Options{Timeout: 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(context.Background(), nil, id, source, Options{Timeout: 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(ds, "test")
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 := 100
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 TestUnlockPanic(t *testing.T) {
defer func() {
if recover() == nil {
t.Fatalf("unlock of unlocked RWMutex did not panic")
}
}()
mu := NewDRWMutex(ds, "test")
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(ds, "test-unlock-panic-2")
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(ds, "test")
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(ds, "test-runlock-panic-2")
mu.Lock(id, source)
mu.RUnlock()
}
// Borrowed from rwmutex_test.go
func benchmarkRWMutex(b *testing.B, localWork, writeRatio int) {
rwm := NewDRWMutex(ds, "test")
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++ {
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)
}

View File

@@ -0,0 +1,151 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package dsync
import (
"fmt"
"sync"
"sync/atomic"
"time"
)
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
// Refresh returns lock not found if set to true
lockNotFound bool
// Set to true if you want peers servers to do not respond
responseDelay int64
}
func (l *lockServer) setRefreshReply(refreshed bool) {
l.mutex.Lock()
defer l.mutex.Unlock()
l.lockNotFound = !refreshed
}
func (l *lockServer) setResponseDelay(responseDelay time.Duration) {
atomic.StoreInt64(&l.responseDelay, int64(responseDelay))
}
func (l *lockServer) Lock(args *LockArgs, reply *bool) error {
if d := atomic.LoadInt64(&l.responseDelay); d != 0 {
time.Sleep(time.Duration(d))
}
l.mutex.Lock()
defer l.mutex.Unlock()
if _, *reply = l.lockMap[args.Resources[0]]; !*reply {
l.lockMap[args.Resources[0]] = 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 {
if d := atomic.LoadInt64(&l.responseDelay); d != 0 {
time.Sleep(time.Duration(d))
}
l.mutex.Lock()
defer l.mutex.Unlock()
var locksHeld int64
if locksHeld, *reply = l.lockMap[args.Resources[0]]; !*reply { // No lock is held on the given name
return fmt.Errorf("Unlock attempted on an unlocked entity: %s", args.Resources[0])
}
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.Resources[0], locksHeld)
}
delete(l.lockMap, args.Resources[0]) // Remove the write lock
return nil
}
const ReadLock = 1
func (l *lockServer) RLock(args *LockArgs, reply *bool) error {
if d := atomic.LoadInt64(&l.responseDelay); d != 0 {
time.Sleep(time.Duration(d))
}
l.mutex.Lock()
defer l.mutex.Unlock()
var locksHeld int64
if locksHeld, *reply = l.lockMap[args.Resources[0]]; !*reply {
l.lockMap[args.Resources[0]] = 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.Resources[0]] = locksHeld + ReadLock // Grant another read lock
}
}
return nil
}
func (l *lockServer) RUnlock(args *LockArgs, reply *bool) error {
if d := atomic.LoadInt64(&l.responseDelay); d != 0 {
time.Sleep(time.Duration(d))
}
l.mutex.Lock()
defer l.mutex.Unlock()
var locksHeld int64
if locksHeld, *reply = l.lockMap[args.Resources[0]]; !*reply { // No lock is held on the given name
return fmt.Errorf("RUnlock attempted on an unlocked entity: %s", args.Resources[0])
}
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.Resources[0])
}
if locksHeld > ReadLock {
l.lockMap[args.Resources[0]] = locksHeld - ReadLock // Remove one of the read locks held
} else {
delete(l.lockMap, args.Resources[0]) // Remove the (last) read lock
}
return nil
}
func (l *lockServer) Refresh(args *LockArgs, reply *bool) error {
if d := atomic.LoadInt64(&l.responseDelay); d != 0 {
time.Sleep(time.Duration(d))
}
l.mutex.Lock()
defer l.mutex.Unlock()
*reply = !l.lockNotFound
return nil
}
func (l *lockServer) ForceUnlock(args *LockArgs, reply *bool) error {
if d := atomic.LoadInt64(&l.responseDelay); d != 0 {
time.Sleep(time.Duration(d))
}
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.Resources[0]) // Remove the lock (irrespective of write or read lock)
*reply = true
return nil
}

25
internal/dsync/dsync.go Normal file
View File

@@ -0,0 +1,25 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package dsync
// Dsync represents dsync client object which is initialized with
// authenticated clients, used to initiate lock REST calls.
type Dsync struct {
// List of rest client objects, one per lock server.
GetLockers func() ([]NetLocker, string)
}

View File

@@ -0,0 +1,430 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package dsync
import (
"context"
"fmt"
golog "log"
"math/rand"
"net"
"net/http"
"net/rpc"
"os"
"strconv"
"sync"
"testing"
"time"
"github.com/google/uuid"
)
const numberOfNodes = 5
var ds *Dsync
var rpcPaths []string // list of rpc paths where lock server is serving.
var nodes = make([]string, numberOfNodes) // list of node IP addrs or hostname with ports.
var lockServers []*lockServer
func startRPCServers() {
for i := range nodes {
server := rpc.NewServer()
ls := &lockServer{
mutex: sync.Mutex{},
lockMap: make(map[string]int64),
}
server.RegisterName("Dsync", ls)
// 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 {
golog.Fatal("listen error:", e)
}
go http.Serve(l, nil)
lockServers = append(lockServers, ls)
}
// 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())
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]))
}
ds = &Dsync{
GetLockers: func() ([]NetLocker, string) { return clnts, uuid.New().String() },
}
startRPCServers()
os.Exit(m.Run())
}
func TestSimpleLock(t *testing.T) {
dm := NewDRWMutex(ds, "test")
dm.Lock(id, source)
// fmt.Println("Lock acquired, waiting...")
time.Sleep(2500 * time.Millisecond)
dm.Unlock()
}
func TestSimpleLockUnlockMultipleTimes(t *testing.T) {
dm := NewDRWMutex(ds, "test")
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(ds, "aap")
dm2nd := NewDRWMutex(ds, "aap")
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(ds, "aap")
dm2nd := NewDRWMutex(ds, "aap")
dm3rd := NewDRWMutex(ds, "aap")
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(ds, "aap")
dm2 := NewDRWMutex(ds, "noot")
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)
}
// Test refreshing lock
func TestFailedRefreshLock(t *testing.T) {
// Simulate Refresh RPC response to return no locking found
for i := range lockServers {
lockServers[i].setRefreshReply(false)
defer lockServers[i].setRefreshReply(true)
}
dm := NewDRWMutex(ds, "aap")
wg := sync.WaitGroup{}
wg.Add(1)
ctx, cl := context.WithCancel(context.Background())
cancel := func() {
cl()
wg.Done()
}
if !dm.GetLock(ctx, cancel, id, source, Options{Timeout: 5 * time.Minute}) {
t.Fatal("GetLock() should be successful")
}
// Wait until context is canceled
wg.Wait()
if ctx.Err() == nil {
t.Fatal("Unexpected error", ctx.Err())
}
// Should be safe operation in all cases
dm.Unlock()
}
// Test Unlock should not timeout
func TestUnlockShouldNotTimeout(t *testing.T) {
dm := NewDRWMutex(ds, "aap")
if !dm.GetLock(context.Background(), nil, id, source, Options{Timeout: 5 * time.Minute}) {
t.Fatal("GetLock() should be successful")
}
// Add delay to lock server responses to ensure that lock does not timeout
for i := range lockServers {
lockServers[i].setResponseDelay(2 * drwMutexUnlockCallTimeout)
defer lockServers[i].setResponseDelay(0)
}
unlockReturned := make(chan struct{}, 1)
go func() {
dm.Unlock()
unlockReturned <- struct{}{}
}()
timer := time.NewTimer(2 * drwMutexUnlockCallTimeout)
defer timer.Stop()
select {
case <-unlockReturned:
t.Fatal("Unlock timed out, which should not happen")
case <-timer.C:
}
}
// 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) {
loops := 200
if testing.Short() {
loops = 5
}
c := make(chan bool)
m := NewDRWMutex(ds, "test")
for i := 0; i < 10; i++ {
go HammerMutex(m, loops, c)
}
for i := 0; i < 10; i++ {
<-c
}
}
func BenchmarkMutexUncontended(b *testing.B) {
type PaddedMutex struct {
*DRWMutex
}
b.RunParallel(func(pb *testing.PB) {
var mu = PaddedMutex{NewDRWMutex(ds, "")}
for pb.Next() {
mu.Lock(id, source)
mu.Unlock()
}
})
}
func benchmarkMutex(b *testing.B, slack, work bool) {
mu := NewDRWMutex(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(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(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]++
}
}
})
}

View File

@@ -0,0 +1,128 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package dsync
import (
"context"
"net/rpc"
"sync"
)
// 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) IsOnline() bool {
rpcClient.mutex.Lock()
defer rpcClient.mutex.Unlock()
// If rpc client has not connected yet there is nothing to close.
return rpcClient.rpc != nil
}
func (rpcClient *ReconnectRPCClient) IsLocal() bool {
return false
}
// 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(ctx context.Context, args LockArgs) (status bool, err error) {
err = rpcClient.Call("Dsync.RLock", &args, &status)
return status, err
}
func (rpcClient *ReconnectRPCClient) Lock(ctx context.Context, args LockArgs) (status bool, err error) {
err = rpcClient.Call("Dsync.Lock", &args, &status)
return status, err
}
func (rpcClient *ReconnectRPCClient) RUnlock(ctx context.Context, args LockArgs) (status bool, err error) {
err = rpcClient.Call("Dsync.RUnlock", &args, &status)
return status, err
}
func (rpcClient *ReconnectRPCClient) Unlock(ctx context.Context, args LockArgs) (status bool, err error) {
err = rpcClient.Call("Dsync.Unlock", &args, &status)
return status, err
}
func (rpcClient *ReconnectRPCClient) Refresh(ctx context.Context, args LockArgs) (refreshed bool, err error) {
err = rpcClient.Call("Dsync.Refresh", &args, &refreshed)
return refreshed, err
}
func (rpcClient *ReconnectRPCClient) ForceUnlock(ctx context.Context, args LockArgs) (reply bool, err error) {
err = rpcClient.Call("Dsync.ForceUnlock", &args, &reply)
return reply, err
}
func (rpcClient *ReconnectRPCClient) String() string {
return "http://" + rpcClient.addr + "/" + rpcClient.endpoint
}

View File

@@ -0,0 +1,87 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package dsync
import "context"
// LockArgs is minimal required values for any dsync compatible lock operation.
type LockArgs struct {
// Unique ID of lock/unlock request.
UID string
// Resources contains single or multiple entries to be locked/unlocked.
Resources []string
// Source contains the line number, function and file name of the code
// on the client node that requested the lock.
Source string
// Owner represents unique ID for this instance, an owner who originally requested
// the locked resource, useful primarily in figuring our stale locks.
Owner string
// Quorum represents the expected quorum for this lock type.
Quorum int
}
// 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(ctx context.Context, 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(ctx context.Context, 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.
// Canceling the context will abort the remote call.
// In that case, the resource may or may not be unlocked.
RUnlock(ctx context.Context, 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.
// Canceling the context will abort the remote call.
// In that case, the resource may or may not be unlocked.
Unlock(ctx context.Context, args LockArgs) (bool, error)
// Refresh the given lock to prevent it from becoming stale
Refresh(ctx context.Context, args LockArgs) (bool, error)
// Unlock (read/write) forcefully for given LockArgs. It should return
// * a boolean to indicate success/failure of the operation
// * an error on failure of unlock request operation.
ForceUnlock(ctx context.Context, 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
// Is the underlying connection online? (is always true for any local lockers)
IsOnline() bool
// Is the underlying locker local to this server?
IsLocal() bool
}