Top Locks command implementation (#7052)

API to list locks used in distributed XL mode
This commit is contained in:
kannappanr 2019-01-24 07:22:14 -08:00 committed by GitHub
parent 964e354d06
commit ce870466ff
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 512 additions and 151 deletions

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2016, 2017, 2018 Minio, Inc.
* Minio Cloud Storage, (C) 2016, 2017, 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -25,6 +25,7 @@ import (
"io"
"net/http"
"os"
"sort"
"strconv"
"strings"
"time"
@ -299,7 +300,7 @@ func (a adminAPIHandlers) ServerInfoHandler(w http.ResponseWriter, r *http.Reque
writeSuccessResponseJSON(w, jsonBytes)
}
// ServerDrivesPerfInfo holds informantion about address, performance
// ServerDrivesPerfInfo holds information about address, performance
// of all drives on one server. It also reports any errors if encountered
// while trying to reach this server.
type ServerDrivesPerfInfo struct {
@ -415,6 +416,103 @@ func (a adminAPIHandlers) PerfInfoHandler(w http.ResponseWriter, r *http.Request
return
}
func newLockEntry(l lockRequesterInfo, resource, server string) *madmin.LockEntry {
entry := &madmin.LockEntry{Timestamp: l.Timestamp, Resource: resource, ServerList: []string{server}, Owner: l.Node, Source: l.Source, ID: l.UID}
if l.Writer {
entry.Type = "Write"
} else {
entry.Type = "Read"
}
return entry
}
func topLockEntries(peerLocks []*PeerLocks) madmin.LockEntries {
const listCount int = 10
entryMap := make(map[string]*madmin.LockEntry)
for _, peerLock := range peerLocks {
if peerLock == nil {
continue
}
for k, v := range peerLock.Locks {
for _, lockReqInfo := range v {
if val, ok := entryMap[lockReqInfo.UID]; ok {
val.ServerList = append(val.ServerList, peerLock.Addr)
} else {
entryMap[lockReqInfo.UID] = newLockEntry(lockReqInfo, k, peerLock.Addr)
}
}
}
}
var lockEntries = make(madmin.LockEntries, 0)
for _, v := range entryMap {
lockEntries = append(lockEntries, *v)
}
sort.Sort(lockEntries)
if len(lockEntries) > listCount {
lockEntries = lockEntries[:listCount]
}
return lockEntries
}
// PeerLocks holds server information result of one node
type PeerLocks struct {
Addr string
Locks GetLocksResp
}
// TopLocksHandler Get list of locks in use
func (a adminAPIHandlers) TopLocksHandler(w http.ResponseWriter, r *http.Request) {
ctx := newContext(r, w, "TopLocks")
objectAPI := newObjectLayerFn()
if objectAPI == nil || globalNotificationSys == nil {
writeErrorResponseJSON(w, ErrServerNotInitialized, r.URL)
return
}
// Method only allowed in Distributed XL mode.
if globalIsDistXL == false {
writeErrorResponseJSON(w, ErrMethodNotAllowed, r.URL)
return
}
// Authenticate request
// Setting the region as empty so as the mc server info command is irrespective to the region.
adminAPIErr := checkAdminRequestAuthType(ctx, r, "")
if adminAPIErr != ErrNone {
writeErrorResponseJSON(w, adminAPIErr, r.URL)
return
}
thisAddr, err := xnet.ParseHost(GetLocalPeer(globalEndpoints))
if err != nil {
writeErrorResponseJSON(w, toAdminAPIErrCode(ctx, err), r.URL)
return
}
peerLocks := globalNotificationSys.GetLocks(ctx)
// Once we have received all the locks currently used from peers
// add the local peer locks list as well.
localLocks := globalLockServer.ll.DupLockMap()
peerLocks = append(peerLocks, &PeerLocks{
Addr: thisAddr.String(),
Locks: localLocks,
})
topLocks := topLockEntries(peerLocks)
// Marshal API response
jsonBytes, err := json.Marshal(topLocks)
if err != nil {
writeErrorResponseJSON(w, toAdminAPIErrCode(ctx, err), r.URL)
return
}
// Reply with storage information (across nodes in a
// distributed setup) as json.
writeSuccessResponseJSON(w, jsonBytes)
}
// StartProfilingResult contains the status of the starting
// profiling action in a given server
type StartProfilingResult struct {

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2016 Minio, Inc.
* Minio Cloud Storage, (C) 2016, 2017, 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -114,6 +114,10 @@ func registerAdminRouter(router *mux.Router, enableConfigOps, enableIAMOps bool)
adminV1Router.Methods(http.MethodGet).Path("/list-canned-policies").HandlerFunc(httpTraceHdrs(adminAPI.ListCannedPolicies))
}
// -- Top APIs --
// Top locks
adminV1Router.Methods(http.MethodGet).Path("/top/locks").HandlerFunc(httpTraceHdrs(adminAPI.TopLocksHandler))
// If none of the routes match, return error.
adminV1Router.NotFoundHandler = http.HandlerFunc(httpTraceHdrs(notFoundHandlerJSON))
}

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2018 Minio, Inc.
* Minio Cloud Storage, (C) 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -26,17 +26,18 @@ import (
// lockRequesterInfo stores various info from the client for each lock that is requested.
type lockRequesterInfo struct {
writer bool // Bool whether write or read lock.
node string // Network address of client claiming lock.
serviceEndpoint string // RPC path of client claiming lock.
uid string // UID to uniquely identify request of client.
timestamp time.Time // Timestamp set at the time of initialization.
timeLastCheck time.Time // Timestamp for last check of validity of lock.
Writer bool // Bool whether write or read lock.
Node string // Network address of client claiming lock.
ServiceEndpoint string // RPC path of client claiming lock.
UID string // UID to uniquely identify request of client.
Timestamp time.Time // Timestamp set at the time of initialization.
TimeLastCheck time.Time // Timestamp for last check of validity of lock.
Source string // Contains line, function and filename reqesting the lock.
}
// isWriteLock returns whether the lock is a write or read lock.
func isWriteLock(lri []lockRequesterInfo) bool {
return len(lri) == 1 && lri[0].writer
return len(lri) == 1 && lri[0].Writer
}
// localLocker implements Dsync.NetLocker
@ -62,12 +63,13 @@ func (l *localLocker) Lock(args dsync.LockArgs) (reply bool, err error) {
if !isLockTaken { // No locks held on the given name, so claim write lock
l.lockMap[args.Resource] = []lockRequesterInfo{
{
writer: true,
node: args.ServerAddr,
serviceEndpoint: args.ServiceEndpoint,
uid: args.UID,
timestamp: UTCNow(),
timeLastCheck: UTCNow(),
Writer: true,
Node: args.ServerAddr,
ServiceEndpoint: args.ServiceEndpoint,
Source: args.Source,
UID: args.UID,
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
},
}
}
@ -98,12 +100,13 @@ func (l *localLocker) RLock(args dsync.LockArgs) (reply bool, err error) {
l.mutex.Lock()
defer l.mutex.Unlock()
lrInfo := lockRequesterInfo{
writer: false,
node: args.ServerAddr,
serviceEndpoint: args.ServiceEndpoint,
uid: args.UID,
timestamp: UTCNow(),
timeLastCheck: UTCNow(),
Writer: false,
Node: args.ServerAddr,
ServiceEndpoint: args.ServiceEndpoint,
Source: args.Source,
UID: args.UID,
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
}
if lri, ok := l.lockMap[args.Resource]; ok {
if reply = !isWriteLock(lri); reply {
@ -149,3 +152,18 @@ func (l *localLocker) ForceUnlock(args dsync.LockArgs) (reply bool, err error) {
}
return true, nil
}
func (l *localLocker) DupLockMap() map[string][]lockRequesterInfo {
l.mutex.Lock()
defer l.mutex.Unlock()
lockCopy := make(map[string][]lockRequesterInfo)
for k, v := range l.lockMap {
var lockSlice []lockRequesterInfo
for _, lockInfo := range v {
lockSlice = append(lockSlice, lockInfo)
}
lockCopy[k] = lockSlice
}
return lockCopy
}

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2016, 2017 Minio, Inc.
* Minio Cloud Storage, (C) 2016, 2017, 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -34,12 +34,12 @@ type nameLockRequesterInfoPair struct {
func (l *localLocker) removeEntryIfExists(nlrip nameLockRequesterInfoPair) {
// Check if entry is still in map (could have been removed altogether by 'concurrent' (R)Unlock of last entry)
if lri, ok := l.lockMap[nlrip.name]; ok {
if !l.removeEntry(nlrip.name, nlrip.lri.uid, &lri) {
if !l.removeEntry(nlrip.name, nlrip.lri.UID, &lri) {
// Remove failed, in case it is a:
if nlrip.lri.writer {
if nlrip.lri.Writer {
// Writer: this should never happen as the whole (mapped) entry should have been deleted
reqInfo := (&logger.ReqInfo{}).AppendTags("name", nlrip.name)
reqInfo.AppendTags("uid", nlrip.lri.uid)
reqInfo.AppendTags("uid", nlrip.lri.UID)
ctx := logger.SetReqInfo(context.Background(), reqInfo)
logger.LogIf(ctx, errors.New("Lock maintenance failed to remove entry for write lock (should never happen)"))
} // Reader: this can happen if multiple read locks were active and
@ -53,7 +53,7 @@ func (l *localLocker) removeEntryIfExists(nlrip nameLockRequesterInfoPair) {
func (l *localLocker) removeEntry(name, uid string, lri *[]lockRequesterInfo) bool {
// Find correct entry to remove based on uid.
for index, entry := range *lri {
if entry.uid == uid {
if entry.UID == uid {
if len(*lri) == 1 {
// Remove the (last) lock.
delete(l.lockMap, name)
@ -76,9 +76,9 @@ func getLongLivedLocks(m map[string][]lockRequesterInfo, interval time.Duration)
for name, lriArray := range m {
for idx := range lriArray {
// Check whether enough time has gone by since last check
if time.Since(lriArray[idx].timeLastCheck) >= interval {
if time.Since(lriArray[idx].TimeLastCheck) >= interval {
rslt = append(rslt, nameLockRequesterInfoPair{name: name, lri: lriArray[idx]})
lriArray[idx].timeLastCheck = UTCNow()
lriArray[idx].TimeLastCheck = UTCNow()
}
}
}

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2016, 2017 Minio, Inc.
* Minio Cloud Storage, (C) 2016, 2017, 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -29,12 +29,12 @@ func TestLockRpcServerRemoveEntryIfExists(t *testing.T) {
defer os.RemoveAll(testPath)
lri := lockRequesterInfo{
writer: false,
node: "host",
serviceEndpoint: "rpc-path",
uid: "0123-4567",
timestamp: UTCNow(),
timeLastCheck: UTCNow(),
Writer: false,
Node: "host",
ServiceEndpoint: "rpc-path",
UID: "0123-4567",
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
}
nlrip := nameLockRequesterInfoPair{name: "name", lri: lri}
@ -66,20 +66,20 @@ func TestLockRpcServerRemoveEntry(t *testing.T) {
defer os.RemoveAll(testPath)
lockRequesterInfo1 := lockRequesterInfo{
writer: true,
node: "host",
serviceEndpoint: "rpc-path",
uid: "0123-4567",
timestamp: UTCNow(),
timeLastCheck: UTCNow(),
Writer: true,
Node: "host",
ServiceEndpoint: "rpc-path",
UID: "0123-4567",
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
}
lockRequesterInfo2 := lockRequesterInfo{
writer: true,
node: "host",
serviceEndpoint: "rpc-path",
uid: "89ab-cdef",
timestamp: UTCNow(),
timeLastCheck: UTCNow(),
Writer: true,
Node: "host",
ServiceEndpoint: "rpc-path",
UID: "89ab-cdef",
Timestamp: UTCNow(),
TimeLastCheck: UTCNow(),
}
locker.ll.lockMap["name"] = []lockRequesterInfo{
@ -128,12 +128,12 @@ func TestLockRpcServerGetLongLivedLocks(t *testing.T) {
{
lockMap: map[string][]lockRequesterInfo{
"test": {{
writer: true,
node: "10.1.10.21",
serviceEndpoint: "/lock/mnt/disk1",
uid: "10000112",
timestamp: ut,
timeLastCheck: ut,
Writer: true,
Node: "10.1.10.21",
ServiceEndpoint: "/lock/mnt/disk1",
UID: "10000112",
Timestamp: ut,
TimeLastCheck: ut,
}},
},
lockInterval: 1 * time.Minute,
@ -143,12 +143,12 @@ func TestLockRpcServerGetLongLivedLocks(t *testing.T) {
{
lockMap: map[string][]lockRequesterInfo{
"test": {{
writer: true,
node: "10.1.10.21",
serviceEndpoint: "/lock/mnt/disk1",
uid: "10000112",
timestamp: ut,
timeLastCheck: ut.Add(-2 * time.Minute),
Writer: true,
Node: "10.1.10.21",
ServiceEndpoint: "/lock/mnt/disk1",
UID: "10000112",
Timestamp: ut,
TimeLastCheck: ut.Add(-2 * time.Minute),
}},
},
lockInterval: 1 * time.Minute,
@ -156,12 +156,12 @@ func TestLockRpcServerGetLongLivedLocks(t *testing.T) {
{
name: "test",
lri: lockRequesterInfo{
writer: true,
node: "10.1.10.21",
serviceEndpoint: "/lock/mnt/disk1",
uid: "10000112",
timestamp: ut,
timeLastCheck: ut.Add(-2 * time.Minute),
Writer: true,
Node: "10.1.10.21",
ServiceEndpoint: "/lock/mnt/disk1",
UID: "10000112",
Timestamp: ut,
TimeLastCheck: ut.Add(-2 * time.Minute),
},
},
},

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2016, 2017 Minio, Inc.
* Minio Cloud Storage, (C) 2016, 2017, 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -94,7 +94,7 @@ func (l *lockRPCReceiver) Expired(args *LockArgs, reply *bool) error {
if lri, ok := l.ll.lockMap[args.LockArgs.Resource]; ok {
// Check whether uid is still active
for _, entry := range lri {
if entry.uid == args.LockArgs.UID {
if entry.UID == args.LockArgs.UID {
*reply = false // When uid found, lock is still active so return not expired.
return nil // When uid found *reply is set to true.
}
@ -123,7 +123,7 @@ func (l *lockRPCReceiver) lockMaintenance(interval time.Duration) {
// Validate if long lived locks are indeed clean.
for _, nlrip := range nlripLongLived {
// Initialize client based on the long live locks.
host, err := xnet.ParseHost(nlrip.lri.node)
host, err := xnet.ParseHost(nlrip.lri.Node)
if err != nil {
logger.LogIf(context.Background(), err)
continue
@ -136,7 +136,7 @@ func (l *lockRPCReceiver) lockMaintenance(interval time.Duration) {
// Call back to original server verify whether the lock is still active (based on name & uid)
expired, _ := c.Expired(dsync.LockArgs{
UID: nlrip.lri.uid,
UID: nlrip.lri.UID,
Resource: nlrip.name,
})

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2016, 2017 Minio, Inc.
* Minio Cloud Storage, (C) 2016, 2017, 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -32,10 +32,10 @@ func testLockEquality(lriLeft, lriRight []lockRequesterInfo) bool {
}
for i := 0; i < len(lriLeft); i++ {
if lriLeft[i].writer != lriRight[i].writer ||
lriLeft[i].node != lriRight[i].node ||
lriLeft[i].serviceEndpoint != lriRight[i].serviceEndpoint ||
lriLeft[i].uid != lriRight[i].uid {
if lriLeft[i].Writer != lriRight[i].Writer ||
lriLeft[i].Node != lriRight[i].Node ||
lriLeft[i].ServiceEndpoint != lriRight[i].ServiceEndpoint ||
lriLeft[i].UID != lriRight[i].UID {
return false
}
}
@ -97,10 +97,10 @@ func TestLockRpcServerLock(t *testing.T) {
gotLri, _ := locker.ll.lockMap["name"]
expectedLri := []lockRequesterInfo{
{
writer: true,
node: "node",
serviceEndpoint: "rpc-path",
uid: "0123-4567",
Writer: true,
Node: "node",
ServiceEndpoint: "rpc-path",
UID: "0123-4567",
},
}
if !testLockEquality(expectedLri, gotLri) {
@ -213,10 +213,10 @@ func TestLockRpcServerRLock(t *testing.T) {
gotLri, _ := locker.ll.lockMap["name"]
expectedLri := []lockRequesterInfo{
{
writer: false,
node: "node",
serviceEndpoint: "rpc-path",
uid: "0123-4567",
Writer: false,
Node: "node",
ServiceEndpoint: "rpc-path",
UID: "0123-4567",
},
}
if !testLockEquality(expectedLri, gotLri) {
@ -315,10 +315,10 @@ func TestLockRpcServerRUnlock(t *testing.T) {
gotLri, _ := locker.ll.lockMap["name"]
expectedLri := []lockRequesterInfo{
{
writer: false,
node: "node",
serviceEndpoint: "rpc-path",
uid: "89ab-cdef",
Writer: false,
Node: "node",
ServiceEndpoint: "rpc-path",
UID: "89ab-cdef",
},
}
if !testLockEquality(expectedLri, gotLri) {

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2016 Minio, Inc.
* Minio Cloud Storage, (C) 2016, 2017, 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -53,9 +53,9 @@ type RWLocker interface {
// RWLockerSync - internal locker interface.
type RWLockerSync interface {
GetLock(timeout time.Duration) bool
GetLock(id, source string, timeout time.Duration) bool
Unlock()
GetRLock(timeout time.Duration) bool
GetRLock(id, source string, timeout time.Duration) bool
RUnlock()
}
@ -162,9 +162,9 @@ func (n *nsLockMap) lock(volume, path string, lockSource, opsID string, readLock
// Locking here will block (until timeout).
if readLock {
locked = nsLk.GetRLock(timeout)
locked = nsLk.GetRLock(opsID, lockSource, timeout)
} else {
locked = nsLk.GetLock(timeout)
locked = nsLk.GetLock(opsID, lockSource, timeout)
}
if !locked { // We failed to get the lock

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2018 Minio, Inc.
* Minio Cloud Storage, (C) 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -340,6 +340,44 @@ func (sys *NotificationSys) ServerInfo(ctx context.Context) []ServerInfo {
return serverInfo
}
// GetLocks - makes GetLocks RPC call on all peers.
func (sys *NotificationSys) GetLocks(ctx context.Context) []*PeerLocks {
var idx = 0
locksResp := make([]*PeerLocks, len(sys.peerRPCClientMap))
var wg sync.WaitGroup
for addr, client := range sys.peerRPCClientMap {
wg.Add(1)
go func(idx int, addr xnet.Host, client *PeerRPCClient) {
defer wg.Done()
// Try to fetch serverInfo remotely in three attempts.
for i := 0; i < 3; i++ {
serverLocksResp, err := client.GetLocks()
if err == nil {
locksResp[idx] = &PeerLocks{
Addr: addr.String(),
Locks: serverLocksResp,
}
return
}
// Last iteration log the error.
if i == 2 {
reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", addr.String())
ctx := logger.SetReqInfo(ctx, reqInfo)
logger.LogOnceIf(ctx, err, addr.String())
}
// Wait for one second and no need wait after last attempt.
if i < 2 {
time.Sleep(1 * time.Second)
}
}
}(idx, addr, client)
idx++
}
wg.Wait()
return locksResp
}
// SetBucketPolicy - calls SetBucketPolicy RPC call on all peers.
func (sys *NotificationSys) SetBucketPolicy(ctx context.Context, bucketName string, bucketPolicy *policy.Policy) {
go func() {

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2018 Minio, Inc.
* Minio Cloud Storage, (C) 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -200,6 +200,15 @@ func (rpcClient *PeerRPCClient) ServerInfo() (ServerInfoData, error) {
return reply, err
}
// GetLocksResp stores various info from the client for each lock that is requested.
type GetLocksResp map[string][]lockRequesterInfo
// GetLocks - returns the lock information on the server to which the RPC call is made.
func (rpcClient *PeerRPCClient) GetLocks() (resp GetLocksResp, err error) {
err = rpcClient.Call(peerServiceName+".GetLocks", &AuthArgs{}, &resp)
return resp, err
}
// NewPeerRPCClient - returns new peer RPC client.
func NewPeerRPCClient(host *xnet.Host) (*PeerRPCClient, error) {
scheme := "http"

View File

@ -1,5 +1,5 @@
/*
* Minio Cloud Storage, (C) 2018 Minio, Inc.
* Minio Cloud Storage, (C) 2018, 2019 Minio, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@ -383,6 +383,24 @@ func (receiver *peerRPCReceiver) ServerInfo(args *AuthArgs, reply *ServerInfoDat
return nil
}
// GetLocks - Get Locks receiver.
func (receiver *peerRPCReceiver) GetLocks(args *AuthArgs, reply *GetLocksResp) error {
if globalBootTime.IsZero() {
return errServerNotInitialized
}
// Build storage info
objLayer := newObjectLayerFn()
if objLayer == nil {
return errServerNotInitialized
}
// Locks data.
*reply = globalLockServer.ll.DupLockMap()
return nil
}
// NewPeerRPCServer - returns new peer RPC server.
func NewPeerRPCServer() (*xrpc.Server, error) {
rpcServer := xrpc.NewServer()

View File

@ -36,12 +36,12 @@ func main() {
```
| Service operations | Info operations | Healing operations | Config operations | IAM operations | Misc |
|:----------------------------|:----------------------------|:--------------------------------------|:--------------------------|:------------------------------------|:------------------------------------|
| [`ServiceStatus`](#ServiceStatus) | [`ServerInfo`](#ServerInfo) | [`Heal`](#Heal) | [`GetConfig`](#GetConfig) | [`AddUser`](#AddUser) | [`SetAdminCredentials`](#SetAdminCredentials) |
| [`ServiceSendAction`](#ServiceSendAction) | [`ServerDrivesPerfInfo`](#ServerDrivesPerfInfo) | [`ServerCPULoadInfo`](#ServerCPULoadInfo) | [`ServerMemUsageInfo`](#ServerMemUsageInfo) | [`SetConfig`](#SetConfig) | [`SetUserPolicy`](#SetUserPolicy) | [`StartProfiling`](#StartProfiling) |
| | | | [`GetConfigKeys`](#GetConfigKeys) | [`ListUsers`](#ListUsers) | [`DownloadProfilingData`](#DownloadProfilingData) |
| | | | [`SetConfigKeys`](#SetConfigKeys) | [`AddCannedPolicy`](#AddCannedPolicy) | |
| Service operations | Info operations | Healing operations | Config operations | Top operations | IAM operations | Misc |
|:----------------------------|:----------------------------|:--------------------------------------|:--------------------------|:--------------------------|:------------------------------------|:------------------------------------|
| [`ServiceStatus`](#ServiceStatus) | [`ServerInfo`](#ServerInfo) | [`Heal`](#Heal) | [`GetConfig`](#GetConfig) | [`TopLocks`](#TopLocks) | [`AddUser`](#AddUser) | [`SetAdminCredentials`](#SetAdminCredentials) |
| [`ServiceSendAction`](#ServiceSendAction) | [`ServerCPULoadInfo`](#ServerCPULoadInfo) | | [`SetConfig`](#SetConfig) | | [`SetUserPolicy`](#SetUserPolicy) | [`StartProfiling`](#StartProfiling) |
| |[`ServerMemUsageInfo`](#ServerMemUsageInfo) | | [`GetConfigKeys`](#GetConfigKeys) | | [`ListUsers`](#ListUsers) | [`DownloadProfilingData`](#DownloadProfilingData) |
| | | | [`SetConfigKeys`](#SetConfigKeys) | | [`AddCannedPolicy`](#AddCannedPolicy) | |
## 1. Constructor
@ -399,7 +399,29 @@ __Example__
log.Println("New configuration successfully set")
```
## 8. IAM operations
## 8. Top operations
<a name="TopLocks"></a>
### TopLocks() (LockEntries, error)
Get the oldest locks from Minio server.
__Example__
``` go
locks, err := madmClnt.TopLocks()
if err != nil {
log.Fatalf("failed due to: %v", err)
}
out, err := json.Marshal(locks)
if err != nil {
log.Fatalf("Marshal failed due to: %v", err)
}
log.Println("TopLocks received successfully: ", string(out))
```
## 9. IAM operations
<a name="AddCannedPolicy"></a>
### AddCannedPolicy(policyName string, policy string) error
@ -455,7 +477,7 @@ __Example__
}
```
## 9. Misc operations
## 10. Misc operations
<a name="SetAdminCredentials"></a>
### SetAdminCredentials() error

View File

@ -0,0 +1,50 @@
// +build ignore
/*
* Minio Cloud Storage, (C) 2019 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 main
import (
"encoding/json"
"log"
"github.com/minio/minio/pkg/madmin"
)
func main() {
// Note: YOUR-ACCESSKEYID, YOUR-SECRETACCESSKEY are
// dummy values, please replace them with original values.
// API requests are secure (HTTPS) if secure=true and insecure (HTTPS) otherwise.
// New returns an Minio Admin client object.
madmClnt, err := madmin.New("your-minio.example.com:9000", "YOUR-ACCESSKEYID", "YOUR-SECRETACCESSKEY", true)
if err != nil {
log.Fatalln(err)
}
locks, err := madmClnt.TopLocks()
if err != nil {
log.Fatalf("failed due to: %v", err)
}
out, err := json.Marshal(locks)
if err != nil {
log.Fatalf("Marshal failed due to: %v", err)
}
log.Println("Top Locks received successfully: ", string(out))
}

View File

@ -0,0 +1,78 @@
/*
* Minio Cloud Storage, (C) 2019 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 madmin
import (
"encoding/json"
"io/ioutil"
"net/http"
"time"
)
// LockEntry holds information about client requesting the lock,
// servers holding the lock, source on the client machine,
// ID, type(read or write) and time stamp.
type LockEntry struct {
Timestamp time.Time `json:"time"` // Timestamp set at the time of initialization.
Resource string `json:"resource"` // Resource contains info like bucket, object etc
Type string `json:"type"` // Bool whether write or read lock.
Source string `json:"source"` // Source which created the lock
ServerList []string `json:"serverlist"` // RPC path of servers issuing the lock.
Owner string `json:"owner"` // RPC path of client claiming lock.
ID string `json:"id"` // UID to uniquely identify request of client.
}
// LockEntries - To sort the locks
type LockEntries []LockEntry
func (l LockEntries) Len() int {
return len(l)
}
func (l LockEntries) Less(i, j int) bool {
return l[i].Timestamp.Before(l[j].Timestamp)
}
func (l LockEntries) Swap(i, j int) {
l[i], l[j] = l[j], l[i]
}
// TopLocks - returns the oldest locks in a minio setup.
func (adm *AdminClient) TopLocks() (LockEntries, error) {
// Execute GET on /minio/admin/v1/top/locks
// to get the oldest locks in a minio setup.
resp, err := adm.executeMethod("GET",
requestData{relPath: "/v1/top/locks"})
defer closeResponse(resp)
if err != nil {
return nil, err
}
if resp.StatusCode != http.StatusOK {
return nil, httpRespToErrorResponse(resp)
}
response, err := ioutil.ReadAll(resp.Body)
if err != nil {
return LockEntries{}, err
}
var lockEntries LockEntries
err = json.Unmarshal(response, &lockEntries)
return lockEntries, err
}

View File

@ -5,20 +5,20 @@ A distributed locking and syncing package for Go.
Introduction
------------
`dsync` is a package for doing distributed locks over a network of `n` nodes. It is designed with simplicity in mind and hence offers limited scalability (`n <= 16`). Each node will be connected to all other nodes and lock requests from any node will be broadcast to all connected nodes. A node will succeed in getting the lock if `n/2 + 1` nodes (whether or not including itself) respond positively. If the lock is acquired it can be held for as long as the client desires and needs to be released afterwards. This will cause the release to be broadcast to all nodes after which the lock becomes available again.
`dsync` is a package for doing distributed locks over a network of `n` nodes. It is designed with simplicity in mind and hence offers limited scalability (`n <= 32`). Each node will be connected to all other nodes and lock requests from any node will be broadcast to all connected nodes. A node will succeed in getting the lock if `n/2 + 1` nodes (whether or not including itself) respond positively. If the lock is acquired it can be held for as long as the client desires and needs to be released afterwards. This will cause the release to be broadcast to all nodes after which the lock becomes available again.
Motivation
----------
This package was developed for the distributed server version of [Minio Object Storage](https://minio.io/). For this we needed a distributed locking mechanism for up to 16 servers that each would be running `minio server`. The locking mechanism itself should be a reader/writer mutual exclusion lock meaning that it can be held by a single writer or an arbitrary number of readers.
This package was developed for the distributed server version of [Minio Object Storage](https://minio.io/). For this we needed a distributed locking mechanism for up to 32 servers that each would be running `minio server`. The locking mechanism itself should be a reader/writer mutual exclusion lock meaning that it can be held by a single writer or an arbitrary number of readers.
For [minio](https://minio.io/) the distributed version is started as follows (for a 6-server system):
```
$ minio server http://server1/disk http://server2/disk http://server3/disk http://server4/disk http://server5/disk http://server6/disk
$ minio server http://server1/disk http://server2/disk http://server3/disk http://server4/disk http://server5/disk http://server6/disk
```
_(note that the same identical command should be run on servers `server1` through to `server6`)_
Design goals
@ -33,7 +33,7 @@ Design goals
Restrictions
------------
* Limited scalability: up to 16 nodes.
* Limited scalability: up to 32 nodes.
* Fixed configuration: changes in the number and/or network names/IP addresses need a restart of all nodes in order to take effect.
* If a down node comes up, it will not try to (re)acquire any locks that it may have held.
* Not designed for high performance applications such as key/value stores.
@ -41,10 +41,10 @@ Restrictions
Performance
-----------
* Support up to a total of 7500 locks/second for maximum size of 16 nodes (consuming 10% CPU usage per server) on moderately powerful server hardware.
* Support up to a total of 7500 locks/second for a size of 16 nodes (consuming 10% CPU usage per server) on moderately powerful server hardware.
* Lock requests (successful) should not take longer than 1ms (provided decent network connection of 1 Gbit or more between the nodes).
The tables below show detailed performance numbers.
The tables below show detailed performance numbers.
### Performance with varying number of nodes
@ -91,7 +91,7 @@ Usage
been changed to `dsync.New([]NetLocker, nodeIndex)` which returns a `*Dsync` object to be used in
every instance of `NewDRWMutex("test", *Dsync)`
### Exclusive lock
### Exclusive lock
Here is a simple example showing how to protect a single resource (drop-in replacement for `sync.Mutex`):
@ -105,7 +105,7 @@ func lockSameResource() {
// Create distributed mutex to protect resource 'test'
dm := dsync.NewDRWMutex("test", ds)
dm.Lock()
dm.Lock("lock-1", "example.go:505:lockSameResource()")
log.Println("first lock granted")
// Release 1st lock after 5 seconds
@ -117,7 +117,7 @@ func lockSameResource() {
// Try to acquire lock again, will block until initial lock is released
log.Println("about to lock same resource again...")
dm.Lock()
dm.Lock("lock-1", "example.go:515:lockSameResource()")
log.Println("second lock granted")
time.Sleep(2 * time.Second)
@ -143,10 +143,10 @@ func twoReadLocksAndSingleWriteLock() {
drwm := dsync.NewDRWMutex("resource", ds)
drwm.RLock()
drwm.RLock("RLock-1", "example.go:416:twoReadLocksAndSingleWriteLock()")
log.Println("1st read lock acquired, waiting...")
drwm.RLock()
drwm.RLock("RLock-2", "example.go:420:twoReadLocksAndSingleWriteLock()")
log.Println("2nd read lock acquired, waiting...")
go func() {
@ -162,7 +162,7 @@ func twoReadLocksAndSingleWriteLock() {
}()
log.Println("Trying to acquire write lock, waiting...")
drwm.Lock()
drwm.Lock("Lock-1", "example.go:445:twoReadLocksAndSingleWriteLock()")
log.Println("Write lock acquired, waiting...")
time.Sleep(3 * time.Second)
@ -190,7 +190,7 @@ Basic architecture
The basic steps in the lock process are as follows:
- broadcast lock message to all `n` nodes
- collect all responses within certain time-out window
- if quorum met (minimally `n/2 + 1` responded positively) then grant lock
- if quorum met (minimally `n/2 + 1` responded positively) then grant lock
- otherwise release all underlying locks and try again after a (semi-)random delay
- release any locks that (still) came in after time time-out window
@ -236,7 +236,7 @@ This table summarizes the conditions for different configurations during which t
| 16 | 7 | 2 | 9 |
(for more info see `testMultipleServersOverQuorumDownDuringLockKnownError` in [chaos.go](https://github.com/minio/dsync/blob/master/chaos/chaos.go))
### Lock not available anymore
This would be due to too many stale locks and/or too many servers down (total over `n/2 - 1`). The following table shows the maximum toterable number for different node sizes:
@ -284,7 +284,7 @@ func (l *lockServer) Unlock(args *LockArgs, reply *bool) error {
defer l.mutex.Unlock()
var locksHeld int64
if locksHeld, *reply = l.lockMap[args.Name]; !*reply { // No lock is held on the given name
return fmt.Errorf("Unlock attempted on an unlocked entity: %s", args.Name)
return fmt.Errorf("Unlock attempted on an unlocked entity: %s", args.Name)
}
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.Name, locksHeld)
@ -357,11 +357,11 @@ For this case it is possible to reduce the number of nodes to be contacted to fo
You do however want to make sure that you have some sort of 'random' selection of which 12 out of the 16 nodes will participate in every lock. See [here](https://gist.github.com/fwessels/dbbafd537c13ec8f88b360b3a0091ac0) for some sample code that could help with this.
### Scale beyond 16 nodes?
### Scale beyond 32 nodes?
Building on the previous example and depending on how resilient you want to be for outages of nodes, you can also go the other way, namely to increase the total number of nodes while keeping the number of nodes contacted per lock the same.
For instance you could imagine a system of 32 nodes where only a quorom majority of `9` would be needed out of `12` nodes. Again this requires some sort of pseudo-random 'deterministic' selection of 12 nodes out of the total of 32 servers (same [example](https://gist.github.com/fwessels/dbbafd537c13ec8f88b360b3a0091ac0) as above).
For instance you could imagine a system of 64 nodes where only a quorum majority of `17` would be needed out of `28` nodes. Again this requires some sort of pseudo-random 'deterministic' selection of 28 nodes out of the total of 64 servers (same [example](https://gist.github.com/harshavardhana/44614a69650c9111defe3470941cdd16) as above).
Other techniques
----------------

View File

@ -17,10 +17,12 @@
package dsync
import (
cryptorand "crypto/rand"
"fmt"
golog "log"
"math/rand"
"os"
"path"
"runtime"
"sync"
"time"
)
@ -31,6 +33,7 @@ var dsyncLog bool
func init() {
// Check for DSYNC_LOG env variable, if set logging will be enabled for failed RPC operations.
dsyncLog = os.Getenv("DSYNC_LOG") == "1"
rand.Seed(time.Now().UnixNano())
}
func log(msg ...interface{}) {
@ -79,10 +82,10 @@ func NewDRWMutex(name string, clnt *Dsync) *DRWMutex {
//
// If the lock is already in use, the calling go routine
// blocks until the mutex is available.
func (dm *DRWMutex) Lock() {
func (dm *DRWMutex) Lock(id, source string) {
isReadLock := false
dm.lockBlocking(drwMutexInfinite, isReadLock)
dm.lockBlocking(drwMutexInfinite, id, source, isReadLock)
}
// GetLock tries to get a write lock on dm before the timeout elapses.
@ -90,20 +93,20 @@ func (dm *DRWMutex) Lock() {
// 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(timeout time.Duration) (locked bool) {
func (dm *DRWMutex) GetLock(id, source string, timeout time.Duration) (locked bool) {
isReadLock := false
return dm.lockBlocking(timeout, isReadLock)
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() {
func (dm *DRWMutex) RLock(id, source string) {
isReadLock := true
dm.lockBlocking(drwMutexInfinite, isReadLock)
dm.lockBlocking(drwMutexInfinite, id, source, isReadLock)
}
// GetRLock tries to get a read lock on dm before the timeout elapses.
@ -112,10 +115,10 @@ func (dm *DRWMutex) RLock() {
// 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(timeout time.Duration) (locked bool) {
func (dm *DRWMutex) GetRLock(id, source string, timeout time.Duration) (locked bool) {
isReadLock := true
return dm.lockBlocking(timeout, isReadLock)
return dm.lockBlocking(timeout, id, source, isReadLock)
}
// lockBlocking will try to acquire either a read or a write lock
@ -123,7 +126,7 @@ func (dm *DRWMutex) GetRLock(timeout time.Duration) (locked bool) {
// 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, isReadLock bool) (locked bool) {
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)
@ -133,7 +136,7 @@ func (dm *DRWMutex) lockBlocking(timeout time.Duration, isReadLock bool) (locked
locks := make([]string, dm.clnt.dNodeCount)
// Try to acquire the lock.
success := lock(dm.clnt, &locks, dm.Name, isReadLock)
success := lock(dm.clnt, &locks, dm.Name, id, source, isReadLock)
if success {
dm.m.Lock()
defer dm.m.Unlock()
@ -160,7 +163,7 @@ func (dm *DRWMutex) lockBlocking(timeout time.Duration, isReadLock bool) (locked
}
// lock tries to acquire the distributed lock, returning true or false.
func lock(ds *Dsync, locks *[]string, lockName string, isReadLock bool) bool {
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)
@ -174,17 +177,12 @@ func lock(ds *Dsync, locks *[]string, lockName string, isReadLock bool) bool {
go func(index int, isReadLock bool, c NetLocker) {
defer wg.Done()
// All client methods issuing RPCs are thread-safe and goroutine-safe,
// i.e. it is safe to call them from multiple concurrently running go routines.
bytesUID := [16]byte{}
cryptorand.Read(bytesUID[:])
uid := fmt.Sprintf("%X", bytesUID[:])
args := LockArgs{
UID: uid,
UID: id,
Resource: lockName,
ServerAddr: ds.rpcClnts[ds.ownNode].ServerAddr(),
ServiceEndpoint: ds.rpcClnts[ds.ownNode].ServiceEndpoint(),
Source: source,
}
var locked bool
@ -438,5 +436,29 @@ func (dm *DRWMutex) DRLocker() sync.Locker {
type drlocker DRWMutex
func (dr *drlocker) Lock() { (*DRWMutex)(dr).RLock() }
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() }

View File

@ -29,6 +29,10 @@ type LockArgs struct {
// ServiceEndpoint contains the network path of above server to do lock/unlock.
ServiceEndpoint 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.

View File

@ -22,9 +22,9 @@ import (
)
const (
WRITELOCK = -1 + iota
NOLOCKS
READLOCKS
WRITELOCK = -1 + iota
NOLOCKS
READLOCKS
)
// A LRWMutex is a mutual exclusion lock with timeouts.
@ -49,7 +49,7 @@ func (lm *LRWMutex) Lock() {
}
// GetLock tries to get a write lock on lm before the timeout occurs.
func (lm *LRWMutex) GetLock(timeout time.Duration) (locked bool) {
func (lm *LRWMutex) GetLock(id, source string, timeout time.Duration) (locked bool) {
isWriteLock := true
return lm.lockLoop(timeout, isWriteLock)
@ -66,7 +66,7 @@ func (lm *LRWMutex) RLock() {
}
// GetRLock tries to get a read lock on lm before the timeout occurs.
func (lm *LRWMutex) GetRLock(timeout time.Duration) (locked bool) {
func (lm *LRWMutex) GetRLock(id, source string, timeout time.Duration) (locked bool) {
isWriteLock := false
return lm.lockLoop(timeout, isWriteLock)

6
vendor/vendor.json vendored
View File

@ -579,10 +579,10 @@
"revisionTime": "2017-02-27T07:32:28Z"
},
{
"checksumSHA1": "1AQVDkFvVxn1RMTLyjeDSESBhrc=",
"checksumSHA1": "CwWXALTRt30txpLPfuazncqZnJ0=",
"path": "github.com/minio/dsync",
"revision": "439a0961af700f80db84cc180fe324a89070fa65",
"revisionTime": "2018-01-23T12:12:34Z"
"revision": "61c41ffdeea2cd0b58edea078ba30c9b6d7411e9",
"revisionTime": "2019-01-04T00:30:57Z"
},
{
"checksumSHA1": "CD2MtlgA8h0z6hYJHURS5eOmZ1k=",