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
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()