mirror of
https://github.com/minio/minio.git
synced 2025-01-11 15:03:22 -05:00
fix: handle concurrent lockers with multiple optimizations (#10640)
- select lockers which are non-local and online to have affinity towards remote servers for lock contention - optimize lock retry interval to avoid sending too many messages during lock contention, reduces average CPU usage as well - if bucket is not set, when deleteObject fails make sure setPutObjHeaders() honors lifecycle only if bucket name is set. - fix top locks to list out always the oldest lockers always, avoid getting bogged down into map's unordered nature.
This commit is contained in:
parent
907a171edd
commit
736e58dd68
@ -368,7 +368,7 @@ func lriToLockEntry(l lockRequesterInfo, resource, server string, rquorum, wquor
|
|||||||
return entry
|
return entry
|
||||||
}
|
}
|
||||||
|
|
||||||
func topLockEntries(peerLocks []*PeerLocks, count int, rquorum, wquorum int, stale bool) madmin.LockEntries {
|
func topLockEntries(peerLocks []*PeerLocks, rquorum, wquorum int, stale bool) madmin.LockEntries {
|
||||||
entryMap := make(map[string]*madmin.LockEntry)
|
entryMap := make(map[string]*madmin.LockEntry)
|
||||||
for _, peerLock := range peerLocks {
|
for _, peerLock := range peerLocks {
|
||||||
if peerLock == nil {
|
if peerLock == nil {
|
||||||
@ -388,9 +388,6 @@ func topLockEntries(peerLocks []*PeerLocks, count int, rquorum, wquorum int, sta
|
|||||||
}
|
}
|
||||||
var lockEntries madmin.LockEntries
|
var lockEntries madmin.LockEntries
|
||||||
for _, v := range entryMap {
|
for _, v := range entryMap {
|
||||||
if len(lockEntries) == count {
|
|
||||||
break
|
|
||||||
}
|
|
||||||
if stale {
|
if stale {
|
||||||
lockEntries = append(lockEntries, *v)
|
lockEntries = append(lockEntries, *v)
|
||||||
continue
|
continue
|
||||||
@ -436,9 +433,13 @@ func (a adminAPIHandlers) TopLocksHandler(w http.ResponseWriter, r *http.Request
|
|||||||
rquorum := getReadQuorum(objectAPI.SetDriveCount())
|
rquorum := getReadQuorum(objectAPI.SetDriveCount())
|
||||||
wquorum := getWriteQuorum(objectAPI.SetDriveCount())
|
wquorum := getWriteQuorum(objectAPI.SetDriveCount())
|
||||||
|
|
||||||
topLocks := topLockEntries(peerLocks, count, rquorum, wquorum, stale)
|
topLocks := topLockEntries(peerLocks, rquorum, wquorum, stale)
|
||||||
|
|
||||||
|
// Marshal API response upto requested count.
|
||||||
|
if len(topLocks) > count && count > 0 {
|
||||||
|
topLocks = topLocks[:count]
|
||||||
|
}
|
||||||
|
|
||||||
// Marshal API response
|
|
||||||
jsonBytes, err := json.Marshal(topLocks)
|
jsonBytes, err := json.Marshal(topLocks)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
|
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
|
||||||
|
@ -107,6 +107,10 @@ func newBucketMetadata(name string) BucketMetadata {
|
|||||||
// Load - loads the metadata of bucket by name from ObjectLayer api.
|
// Load - loads the metadata of bucket by name from ObjectLayer api.
|
||||||
// If an error is returned the returned metadata will be default initialized.
|
// If an error is returned the returned metadata will be default initialized.
|
||||||
func (b *BucketMetadata) Load(ctx context.Context, api ObjectLayer, name string) error {
|
func (b *BucketMetadata) Load(ctx context.Context, api ObjectLayer, name string) error {
|
||||||
|
if name == "" {
|
||||||
|
logger.LogIf(ctx, errors.New("bucket name cannot be empty"))
|
||||||
|
return errors.New("bucket name cannot be empty")
|
||||||
|
}
|
||||||
configFile := path.Join(bucketConfigPrefix, name, bucketMetadataFile)
|
configFile := path.Join(bucketConfigPrefix, name, bucketMetadataFile)
|
||||||
data, err := readConfig(ctx, api, configFile)
|
data, err := readConfig(ctx, api, configFile)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -128,20 +132,22 @@ func (b *BucketMetadata) Load(ctx context.Context, api ObjectLayer, name string)
|
|||||||
}
|
}
|
||||||
// OK, parse data.
|
// OK, parse data.
|
||||||
_, err = b.UnmarshalMsg(data[4:])
|
_, err = b.UnmarshalMsg(data[4:])
|
||||||
|
b.Name = name // in-case parsing failed for some reason, make sure bucket name is not empty.
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// loadBucketMetadata loads and migrates to bucket metadata.
|
// loadBucketMetadata loads and migrates to bucket metadata.
|
||||||
func loadBucketMetadata(ctx context.Context, objectAPI ObjectLayer, bucket string) (BucketMetadata, error) {
|
func loadBucketMetadata(ctx context.Context, objectAPI ObjectLayer, bucket string) (BucketMetadata, error) {
|
||||||
b := newBucketMetadata(bucket)
|
b := newBucketMetadata(bucket)
|
||||||
err := b.Load(ctx, objectAPI, bucket)
|
err := b.Load(ctx, objectAPI, b.Name)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
return b, b.convertLegacyConfigs(ctx, objectAPI)
|
return b, b.convertLegacyConfigs(ctx, objectAPI)
|
||||||
}
|
}
|
||||||
|
|
||||||
if err != errConfigNotFound {
|
if !errors.Is(err, errConfigNotFound) {
|
||||||
return b, err
|
return b, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Old bucket without bucket metadata. Hence we migrate existing settings.
|
// Old bucket without bucket metadata. Hence we migrate existing settings.
|
||||||
return b, b.convertLegacyConfigs(ctx, objectAPI)
|
return b, b.convertLegacyConfigs(ctx, objectAPI)
|
||||||
}
|
}
|
||||||
|
@ -54,7 +54,7 @@ const (
|
|||||||
|
|
||||||
var (
|
var (
|
||||||
globalCrawlerConfig crawler.Config
|
globalCrawlerConfig crawler.Config
|
||||||
dataCrawlerLeaderLockTimeout = newDynamicTimeout(1*time.Minute, 30*time.Second)
|
dataCrawlerLeaderLockTimeout = newDynamicTimeout(30*time.Second, 10*time.Second)
|
||||||
)
|
)
|
||||||
|
|
||||||
// initDataCrawler will start the crawler unless disabled.
|
// initDataCrawler will start the crawler unless disabled.
|
||||||
|
@ -770,10 +770,11 @@ func GetProxyEndpoints(endpointZones EndpointZones) ([]ProxyEndpoint, error) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
tr := newCustomHTTPTransport(tlsConfig, rest.DefaultTimeout)()
|
// allow transport to be HTTP/1.1 for proxying.
|
||||||
|
tr := newCustomHTTP11Transport(tlsConfig, rest.DefaultTimeout)()
|
||||||
|
|
||||||
// Allow more requests to be in flight with higher response header timeout.
|
// Allow more requests to be in flight with higher response header timeout.
|
||||||
tr.ResponseHeaderTimeout = 30 * time.Minute
|
tr.ResponseHeaderTimeout = 30 * time.Minute
|
||||||
tr.MaxIdleConns = 64
|
|
||||||
tr.MaxIdleConnsPerHost = 64
|
tr.MaxIdleConnsPerHost = 64
|
||||||
|
|
||||||
proxyEps = append(proxyEps, ProxyEndpoint{
|
proxyEps = append(proxyEps, ProxyEndpoint{
|
||||||
|
@ -211,13 +211,6 @@ func (er erasureObjects) GetObject(ctx context.Context, bucket, object string, s
|
|||||||
return errUnexpected
|
return errUnexpected
|
||||||
}
|
}
|
||||||
|
|
||||||
// If its a directory request, we return an empty body.
|
|
||||||
if HasSuffix(object, SlashSeparator) {
|
|
||||||
_, err := writer.Write([]byte(""))
|
|
||||||
logger.LogIf(ctx, err)
|
|
||||||
return toObjectErr(err, bucket, object)
|
|
||||||
}
|
|
||||||
|
|
||||||
return er.getObject(ctx, bucket, object, startOffset, length, writer, etag, opts)
|
return er.getObject(ctx, bucket, object, startOffset, length, writer, etag, opts)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -862,11 +855,11 @@ func (er erasureObjects) DeleteObjects(ctx context.Context, bucket string, objec
|
|||||||
dobjects[objIndex] = DeletedObject{
|
dobjects[objIndex] = DeletedObject{
|
||||||
DeleteMarker: versions[objIndex].Deleted,
|
DeleteMarker: versions[objIndex].Deleted,
|
||||||
DeleteMarkerVersionID: versions[objIndex].VersionID,
|
DeleteMarkerVersionID: versions[objIndex].VersionID,
|
||||||
ObjectName: versions[objIndex].Name,
|
ObjectName: decodeDirObject(versions[objIndex].Name),
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
dobjects[objIndex] = DeletedObject{
|
dobjects[objIndex] = DeletedObject{
|
||||||
ObjectName: versions[objIndex].Name,
|
ObjectName: decodeDirObject(versions[objIndex].Name),
|
||||||
VersionID: versions[objIndex].VersionID,
|
VersionID: versions[objIndex].VersionID,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -899,7 +892,7 @@ func (er erasureObjects) DeleteObjects(ctx context.Context, bucket string, objec
|
|||||||
func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string, opts ObjectOptions) (objInfo ObjectInfo, err error) {
|
func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string, opts ObjectOptions) (objInfo ObjectInfo, err error) {
|
||||||
// Acquire a write lock before deleting the object.
|
// Acquire a write lock before deleting the object.
|
||||||
lk := er.NewNSLock(ctx, bucket, object)
|
lk := er.NewNSLock(ctx, bucket, object)
|
||||||
if err = lk.GetLock(globalOperationTimeout); err != nil {
|
if err = lk.GetLock(globalDeleteOperationTimeout); err != nil {
|
||||||
return ObjectInfo{}, err
|
return ObjectInfo{}, err
|
||||||
}
|
}
|
||||||
defer lk.Unlock()
|
defer lk.Unlock()
|
||||||
@ -945,7 +938,7 @@ func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return ObjectInfo{Bucket: bucket, Name: object, VersionID: opts.VersionID}, nil
|
return ObjectInfo{Bucket: bucket, Name: decodeDirObject(object), VersionID: opts.VersionID}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Send the successful but partial upload/delete, however ignore
|
// Send the successful but partial upload/delete, however ignore
|
||||||
|
@ -273,6 +273,13 @@ func (s *erasureSets) GetLockers(setIndex int) func() ([]dsync.NetLocker, string
|
|||||||
return func() ([]dsync.NetLocker, string) {
|
return func() ([]dsync.NetLocker, string) {
|
||||||
lockers := make([]dsync.NetLocker, s.setDriveCount)
|
lockers := make([]dsync.NetLocker, s.setDriveCount)
|
||||||
copy(lockers, s.erasureLockers[setIndex])
|
copy(lockers, s.erasureLockers[setIndex])
|
||||||
|
sort.Slice(lockers, func(i, j int) bool {
|
||||||
|
// re-order lockers with affinity for
|
||||||
|
// - non-local lockers
|
||||||
|
// - online lockers
|
||||||
|
// are used first
|
||||||
|
return !lockers[i].IsLocal() && lockers[i].IsOnline()
|
||||||
|
})
|
||||||
return lockers, s.erasureLockOwner
|
return lockers, s.erasureLockOwner
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -568,6 +568,7 @@ func (z *erasureZones) DeleteObject(ctx context.Context, bucket string, object s
|
|||||||
break
|
break
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return objInfo, err
|
return objInfo, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -210,7 +210,8 @@ var (
|
|||||||
globalDomainNames []string // Root domains for virtual host style requests
|
globalDomainNames []string // Root domains for virtual host style requests
|
||||||
globalDomainIPs set.StringSet // Root domain IP address(s) for a distributed MinIO deployment
|
globalDomainIPs set.StringSet // Root domain IP address(s) for a distributed MinIO deployment
|
||||||
|
|
||||||
globalOperationTimeout = newDynamicTimeout(10*time.Minute, 5*time.Minute) // default timeout for general ops
|
globalOperationTimeout = newDynamicTimeout(10*time.Minute, 5*time.Minute) // default timeout for general ops
|
||||||
|
globalDeleteOperationTimeout = newDynamicTimeout(5*time.Minute, 1*time.Minute) // default time for delete ops
|
||||||
|
|
||||||
globalBucketObjectLockSys *BucketObjectLockSys
|
globalBucketObjectLockSys *BucketObjectLockSys
|
||||||
globalBucketQuotaSys *BucketQuotaSys
|
globalBucketQuotaSys *BucketQuotaSys
|
||||||
|
@ -203,11 +203,16 @@ func (l *localLocker) Close() error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Local locker is always online.
|
// IsOnline - local locker is always online.
|
||||||
func (l *localLocker) IsOnline() bool {
|
func (l *localLocker) IsOnline() bool {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// IsLocal - local locker returns true.
|
||||||
|
func (l *localLocker) IsLocal() bool {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
func (l *localLocker) Expired(ctx context.Context, args dsync.LockArgs) (expired bool, err error) {
|
func (l *localLocker) Expired(ctx context.Context, args dsync.LockArgs) (expired bool, err error) {
|
||||||
select {
|
select {
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
|
@ -76,6 +76,11 @@ func (client *lockRESTClient) IsOnline() bool {
|
|||||||
return client.restClient.IsOnline()
|
return client.restClient.IsOnline()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Not a local locker
|
||||||
|
func (client *lockRESTClient) IsLocal() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
// Close - marks the client as closed.
|
// Close - marks the client as closed.
|
||||||
func (client *lockRESTClient) Close() error {
|
func (client *lockRESTClient) Close() error {
|
||||||
client.restClient.Close()
|
client.restClient.Close()
|
||||||
|
@ -60,16 +60,16 @@ func NewResponseWriter(w http.ResponseWriter) *ResponseWriter {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (lrw *ResponseWriter) Write(p []byte) (int, error) {
|
func (lrw *ResponseWriter) Write(p []byte) (int, error) {
|
||||||
n, err := lrw.ResponseWriter.Write(p)
|
|
||||||
lrw.bytesWritten += n
|
|
||||||
if lrw.TimeToFirstByte == 0 {
|
|
||||||
lrw.TimeToFirstByte = time.Now().UTC().Sub(lrw.StartTime)
|
|
||||||
}
|
|
||||||
if !lrw.headersLogged {
|
if !lrw.headersLogged {
|
||||||
// We assume the response code to be '200 OK' when WriteHeader() is not called,
|
// We assume the response code to be '200 OK' when WriteHeader() is not called,
|
||||||
// that way following Golang HTTP response behavior.
|
// that way following Golang HTTP response behavior.
|
||||||
lrw.WriteHeader(http.StatusOK)
|
lrw.WriteHeader(http.StatusOK)
|
||||||
}
|
}
|
||||||
|
n, err := lrw.ResponseWriter.Write(p)
|
||||||
|
lrw.bytesWritten += n
|
||||||
|
if lrw.TimeToFirstByte == 0 {
|
||||||
|
lrw.TimeToFirstByte = time.Now().UTC().Sub(lrw.StartTime)
|
||||||
|
}
|
||||||
if (lrw.LogErrBody && lrw.StatusCode >= http.StatusBadRequest) || lrw.LogAllBody {
|
if (lrw.LogErrBody && lrw.StatusCode >= http.StatusBadRequest) || lrw.LogAllBody {
|
||||||
// Always logging error responses.
|
// Always logging error responses.
|
||||||
lrw.body.Write(p)
|
lrw.body.Write(p)
|
||||||
|
@ -534,13 +534,13 @@ func (sys *NotificationSys) GetLocks(ctx context.Context, r *http.Request) []*Pe
|
|||||||
}
|
}
|
||||||
// Once we have received all the locks currently used from peers
|
// Once we have received all the locks currently used from peers
|
||||||
// add the local peer locks list as well.
|
// add the local peer locks list as well.
|
||||||
var getRespLocks GetLocksResp
|
llockers := make(GetLocksResp, 0, len(globalLockServers))
|
||||||
for _, llocker := range globalLockServers {
|
for _, llocker := range globalLockServers {
|
||||||
getRespLocks = append(getRespLocks, llocker.DupLockMap())
|
llockers = append(llockers, llocker.DupLockMap())
|
||||||
}
|
}
|
||||||
locksResp = append(locksResp, &PeerLocks{
|
locksResp = append(locksResp, &PeerLocks{
|
||||||
Addr: getHostName(r),
|
Addr: getHostName(r),
|
||||||
Locks: getRespLocks,
|
Locks: llockers,
|
||||||
})
|
})
|
||||||
return locksResp
|
return locksResp
|
||||||
}
|
}
|
||||||
|
@ -322,6 +322,10 @@ func isStringEqual(s1 string, s2 string) bool {
|
|||||||
|
|
||||||
// Ignores all reserved bucket names or invalid bucket names.
|
// Ignores all reserved bucket names or invalid bucket names.
|
||||||
func isReservedOrInvalidBucket(bucketEntry string, strict bool) bool {
|
func isReservedOrInvalidBucket(bucketEntry string, strict bool) bool {
|
||||||
|
if bucketEntry == "" {
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
bucketEntry = strings.TrimSuffix(bucketEntry, SlashSeparator)
|
bucketEntry = strings.TrimSuffix(bucketEntry, SlashSeparator)
|
||||||
if strict {
|
if strict {
|
||||||
if err := s3utils.CheckValidBucketNameStrict(bucketEntry); err != nil {
|
if err := s3utils.CheckValidBucketNameStrict(bucketEntry); err != nil {
|
||||||
|
@ -261,18 +261,20 @@ func setPutObjHeaders(w http.ResponseWriter, objInfo ObjectInfo, delete bool) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if lc, err := globalLifecycleSys.Get(objInfo.Bucket); err == nil && !delete {
|
if objInfo.Bucket != "" {
|
||||||
ruleID, expiryTime := lc.PredictExpiryTime(lifecycle.ObjectOpts{
|
if lc, err := globalLifecycleSys.Get(objInfo.Bucket); err == nil && !delete {
|
||||||
Name: objInfo.Name,
|
ruleID, expiryTime := lc.PredictExpiryTime(lifecycle.ObjectOpts{
|
||||||
UserTags: objInfo.UserTags,
|
Name: objInfo.Name,
|
||||||
VersionID: objInfo.VersionID,
|
UserTags: objInfo.UserTags,
|
||||||
ModTime: objInfo.ModTime,
|
VersionID: objInfo.VersionID,
|
||||||
IsLatest: objInfo.IsLatest,
|
ModTime: objInfo.ModTime,
|
||||||
DeleteMarker: objInfo.DeleteMarker,
|
IsLatest: objInfo.IsLatest,
|
||||||
})
|
DeleteMarker: objInfo.DeleteMarker,
|
||||||
if !expiryTime.IsZero() {
|
})
|
||||||
w.Header()[xhttp.AmzExpiration] = []string{
|
if !expiryTime.IsZero() {
|
||||||
fmt.Sprintf(`expiry-date="%s", rule-id="%s"`, expiryTime.Format(http.TimeFormat), ruleID),
|
w.Header()[xhttp.AmzExpiration] = []string{
|
||||||
|
fmt.Sprintf(`expiry-date="%s", rule-id="%s"`, expiryTime.Format(http.TimeFormat), ruleID),
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -47,7 +47,7 @@ func (s *peerRESTServer) GetLocksHandler(w http.ResponseWriter, r *http.Request)
|
|||||||
|
|
||||||
ctx := newContext(r, w, "GetLocks")
|
ctx := newContext(r, w, "GetLocks")
|
||||||
|
|
||||||
llockers := make([]map[string][]lockRequesterInfo, 0, len(globalLockServers))
|
llockers := make(GetLocksResp, 0, len(globalLockServers))
|
||||||
for _, llocker := range globalLockServers {
|
for _, llocker := range globalLockServers {
|
||||||
llockers = append(llockers, llocker.DupLockMap())
|
llockers = append(llockers, llocker.DupLockMap())
|
||||||
}
|
}
|
||||||
|
24
cmd/utils.go
24
cmd/utils.go
@ -490,6 +490,29 @@ func newInternodeHTTPTransport(tlsConfig *tls.Config, dialTimeout time.Duration)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func newCustomHTTP11Transport(tlsConfig *tls.Config, dialTimeout time.Duration) func() *http.Transport {
|
||||||
|
// For more details about various values used here refer
|
||||||
|
// https://golang.org/pkg/net/http/#Transport documentation
|
||||||
|
tr := &http.Transport{
|
||||||
|
Proxy: http.ProxyFromEnvironment,
|
||||||
|
DialContext: xhttp.NewCustomDialContext(dialTimeout),
|
||||||
|
MaxIdleConnsPerHost: 16,
|
||||||
|
IdleConnTimeout: 1 * time.Minute,
|
||||||
|
ResponseHeaderTimeout: 3 * time.Minute, // Set conservative timeouts for MinIO internode.
|
||||||
|
TLSHandshakeTimeout: 10 * time.Second,
|
||||||
|
ExpectContinueTimeout: 10 * time.Second,
|
||||||
|
TLSClientConfig: tlsConfig,
|
||||||
|
// Go net/http automatically unzip if content-type is
|
||||||
|
// gzip disable this feature, as we are always interested
|
||||||
|
// in raw stream.
|
||||||
|
DisableCompression: true,
|
||||||
|
}
|
||||||
|
|
||||||
|
return func() *http.Transport {
|
||||||
|
return tr
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func newCustomHTTPTransport(tlsConfig *tls.Config, dialTimeout time.Duration) func() *http.Transport {
|
func newCustomHTTPTransport(tlsConfig *tls.Config, dialTimeout time.Duration) func() *http.Transport {
|
||||||
// For more details about various values used here refer
|
// For more details about various values used here refer
|
||||||
// https://golang.org/pkg/net/http/#Transport documentation
|
// https://golang.org/pkg/net/http/#Transport documentation
|
||||||
@ -532,7 +555,6 @@ func newGatewayHTTPTransport(timeout time.Duration) *http.Transport {
|
|||||||
|
|
||||||
// Allow more requests to be in flight.
|
// Allow more requests to be in flight.
|
||||||
tr.ResponseHeaderTimeout = timeout
|
tr.ResponseHeaderTimeout = timeout
|
||||||
tr.MaxIdleConns = 256
|
|
||||||
tr.MaxIdleConnsPerHost = 16
|
tr.MaxIdleConnsPerHost = 16
|
||||||
return tr
|
return tr
|
||||||
}
|
}
|
||||||
|
@ -661,21 +661,7 @@ func (s *xlStorage) ListVols(context.Context) (volsInfo []VolInfo, err error) {
|
|||||||
atomic.AddInt32(&s.activeIOCount, -1)
|
atomic.AddInt32(&s.activeIOCount, -1)
|
||||||
}()
|
}()
|
||||||
|
|
||||||
volsInfo, err = listVols(s.diskPath)
|
return listVols(s.diskPath)
|
||||||
if err != nil {
|
|
||||||
if isSysErrIO(err) {
|
|
||||||
return nil, errFaultyDisk
|
|
||||||
}
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
for i, vol := range volsInfo {
|
|
||||||
volInfo := VolInfo{
|
|
||||||
Name: vol.Name,
|
|
||||||
Created: vol.Created,
|
|
||||||
}
|
|
||||||
volsInfo[i] = volInfo
|
|
||||||
}
|
|
||||||
return volsInfo, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// List all the volumes from diskPath.
|
// List all the volumes from diskPath.
|
||||||
|
@ -133,7 +133,7 @@ func (dm *DRWMutex) GetRLock(ctx context.Context, id, source string, opts Option
|
|||||||
}
|
}
|
||||||
|
|
||||||
const (
|
const (
|
||||||
lockRetryInterval = 100 * time.Millisecond
|
lockRetryInterval = 1 * time.Second
|
||||||
)
|
)
|
||||||
|
|
||||||
// lockBlocking will try to acquire either a read or a write lock
|
// lockBlocking will try to acquire either a read or a write lock
|
||||||
@ -204,6 +204,7 @@ func (dm *DRWMutex) lockBlocking(ctx context.Context, id, source string, isReadL
|
|||||||
dm.m.Unlock()
|
dm.m.Unlock()
|
||||||
return locked
|
return locked
|
||||||
}
|
}
|
||||||
|
|
||||||
time.Sleep(time.Duration(r.Float64() * float64(lockRetryInterval)))
|
time.Sleep(time.Duration(r.Float64() * float64(lockRetryInterval)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -268,6 +269,7 @@ func lock(ctx context.Context, ds *Dsync, locks *[]string, id, source string, is
|
|||||||
|
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
go func(isReadLock bool) {
|
go func(isReadLock bool) {
|
||||||
|
defer wg.Done()
|
||||||
|
|
||||||
// Wait until we have either
|
// Wait until we have either
|
||||||
//
|
//
|
||||||
@ -317,9 +319,6 @@ func lock(ctx context.Context, ds *Dsync, locks *[]string, id, source string, is
|
|||||||
// Count locks in order to determine whether we have quorum or not
|
// Count locks in order to determine whether we have quorum or not
|
||||||
quorumLocked = checkQuorumLocked(locks, quorum)
|
quorumLocked = checkQuorumLocked(locks, quorum)
|
||||||
|
|
||||||
// Signal that we have the quorum
|
|
||||||
wg.Done()
|
|
||||||
|
|
||||||
// Wait for the other responses and immediately release the locks
|
// Wait for the other responses and immediately release the locks
|
||||||
// (do not add them to the locks array because the DRWMutex could
|
// (do not add them to the locks array because the DRWMutex could
|
||||||
// already has been unlocked again by the original calling thread)
|
// already has been unlocked again by the original calling thread)
|
||||||
|
@ -50,6 +50,10 @@ func (rpcClient *ReconnectRPCClient) IsOnline() bool {
|
|||||||
return rpcClient.rpc != nil
|
return rpcClient.rpc != nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (rpcClient *ReconnectRPCClient) IsLocal() bool {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
// Close closes the underlying socket file descriptor.
|
// Close closes the underlying socket file descriptor.
|
||||||
func (rpcClient *ReconnectRPCClient) Close() error {
|
func (rpcClient *ReconnectRPCClient) Close() error {
|
||||||
rpcClient.mutex.Lock()
|
rpcClient.mutex.Lock()
|
||||||
|
@ -68,4 +68,7 @@ type NetLocker interface {
|
|||||||
|
|
||||||
// Is the underlying connection online? (is always true for any local lockers)
|
// Is the underlying connection online? (is always true for any local lockers)
|
||||||
IsOnline() bool
|
IsOnline() bool
|
||||||
|
|
||||||
|
// Is the underlying locker local to this server?
|
||||||
|
IsLocal() bool
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user