Various improvements in replication (#11949)

- collect real time replication metrics for prometheus.
- add pending_count, failed_count metric for total pending/failed replication operations.

- add API to get replication metrics

- add MRF worker to handle spill-over replication operations

- multiple issues found with replication
- fixes an issue when client sends a bucket
 name with `/` at the end from SetRemoteTarget
 API call make sure to trim the bucket name to 
 avoid any extra `/`.

- hold write locks in GetObjectNInfo during replication
  to ensure that object version stack is not overwritten
  while reading the content.

- add additional protection during WriteMetadata() to
  ensure that we always write a valid FileInfo{} and avoid
  ever writing empty FileInfo{} to the lowest layers.

Co-authored-by: Poorna Krishnamoorthy <poorna@minio.io>
Co-authored-by: Harshavardhana <harsha@minio.io>
This commit is contained in:
Poorna Krishnamoorthy 2021-04-03 09:03:42 -07:00 committed by GitHub
parent dca7cf7200
commit 47c09a1e6f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
36 changed files with 1914 additions and 496 deletions

View File

@ -21,6 +21,7 @@ import (
"io" "io"
"io/ioutil" "io/ioutil"
"net/http" "net/http"
"path"
"github.com/gorilla/mux" "github.com/gorilla/mux"
"github.com/minio/minio/cmd/logger" "github.com/minio/minio/cmd/logger"
@ -50,7 +51,7 @@ func (a adminAPIHandlers) PutBucketQuotaConfigHandler(w http.ResponseWriter, r *
} }
vars := mux.Vars(r) vars := mux.Vars(r)
bucket := vars["bucket"] bucket := path.Clean(vars["bucket"])
if _, err := objectAPI.GetBucketInfo(ctx, bucket); err != nil { if _, err := objectAPI.GetBucketInfo(ctx, bucket); err != nil {
writeErrorResponseJSON(ctx, w, toAPIError(ctx, err), r.URL) writeErrorResponseJSON(ctx, w, toAPIError(ctx, err), r.URL)
@ -90,7 +91,8 @@ func (a adminAPIHandlers) GetBucketQuotaConfigHandler(w http.ResponseWriter, r *
} }
vars := mux.Vars(r) vars := mux.Vars(r)
bucket := vars["bucket"] bucket := path.Clean(vars["bucket"])
if _, err := objectAPI.GetBucketInfo(ctx, bucket); err != nil { if _, err := objectAPI.GetBucketInfo(ctx, bucket); err != nil {
writeErrorResponseJSON(ctx, w, toAPIError(ctx, err), r.URL) writeErrorResponseJSON(ctx, w, toAPIError(ctx, err), r.URL)
return return
@ -118,7 +120,7 @@ func (a adminAPIHandlers) SetRemoteTargetHandler(w http.ResponseWriter, r *http.
defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r)) defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r))
vars := mux.Vars(r) vars := mux.Vars(r)
bucket := vars["bucket"] bucket := path.Clean(vars["bucket"])
update := r.URL.Query().Get("update") == "true" update := r.URL.Query().Get("update") == "true"
if !globalIsErasure { if !globalIsErasure {
@ -211,7 +213,7 @@ func (a adminAPIHandlers) ListRemoteTargetsHandler(w http.ResponseWriter, r *htt
defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r)) defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r))
vars := mux.Vars(r) vars := mux.Vars(r)
bucket := vars["bucket"] bucket := path.Clean(vars["bucket"])
arnType := vars["type"] arnType := vars["type"]
if !globalIsErasure { if !globalIsErasure {
writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErr(ErrNotImplemented), r.URL) writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErr(ErrNotImplemented), r.URL)
@ -250,7 +252,7 @@ func (a adminAPIHandlers) RemoveRemoteTargetHandler(w http.ResponseWriter, r *ht
defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r)) defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r))
vars := mux.Vars(r) vars := mux.Vars(r)
bucket := vars["bucket"] bucket := path.Clean(vars["bucket"])
arn := vars["arn"] arn := vars["arn"]
if !globalIsErasure { if !globalIsErasure {

View File

@ -775,7 +775,7 @@ func (a adminAPIHandlers) AccountInfoHandler(w http.ResponseWriter, r *http.Requ
if !dataUsageInfo.LastUpdate.IsZero() { if !dataUsageInfo.LastUpdate.IsZero() {
size = dataUsageInfo.BucketsUsage[bucket.Name].Size size = dataUsageInfo.BucketsUsage[bucket.Name].Size
} }
acctInfo.Buckets = append(acctInfo.Buckets, madmin.BucketUsageInfo{ acctInfo.Buckets = append(acctInfo.Buckets, madmin.BucketAccessInfo{
Name: bucket.Name, Name: bucket.Name,
Created: bucket.Created, Created: bucket.Created,
Size: size, Size: size,

View File

@ -311,7 +311,6 @@ func registerAPIRouter(router *mux.Router) {
// GetBucketReplicationConfig // GetBucketReplicationConfig
router.Methods(http.MethodGet).HandlerFunc( router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("getbucketreplicationconfiguration", maxClients(httpTraceAll(api.GetBucketReplicationConfigHandler)))).Queries("replication", "") collectAPIStats("getbucketreplicationconfiguration", maxClients(httpTraceAll(api.GetBucketReplicationConfigHandler)))).Queries("replication", "")
// GetBucketVersioning // GetBucketVersioning
router.Methods(http.MethodGet).HandlerFunc( router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("getbucketversioning", maxClients(httpTraceAll(api.GetBucketVersioningHandler)))).Queries("versioning", "") collectAPIStats("getbucketversioning", maxClients(httpTraceAll(api.GetBucketVersioningHandler)))).Queries("versioning", "")
@ -378,8 +377,6 @@ func registerAPIRouter(router *mux.Router) {
// PutBucketReplicationConfig // PutBucketReplicationConfig
router.Methods(http.MethodPut).HandlerFunc( router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("putbucketreplicationconfiguration", maxClients(httpTraceAll(api.PutBucketReplicationConfigHandler)))).Queries("replication", "") collectAPIStats("putbucketreplicationconfiguration", maxClients(httpTraceAll(api.PutBucketReplicationConfigHandler)))).Queries("replication", "")
// GetObjectRetention
// PutBucketEncryption // PutBucketEncryption
router.Methods(http.MethodPut).HandlerFunc( router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("putbucketencryption", maxClients(httpTraceAll(api.PutBucketEncryptionHandler)))).Queries("encryption", "") collectAPIStats("putbucketencryption", maxClients(httpTraceAll(api.PutBucketEncryptionHandler)))).Queries("encryption", "")
@ -430,6 +427,12 @@ func registerAPIRouter(router *mux.Router) {
// ListObjectsV1 (Legacy) // ListObjectsV1 (Legacy)
router.Methods(http.MethodGet).HandlerFunc( router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("listobjectsv1", maxClients(httpTraceAll(api.ListObjectsV1Handler)))) collectAPIStats("listobjectsv1", maxClients(httpTraceAll(api.ListObjectsV1Handler))))
// MinIO extension API for replication.
//
// GetBucketReplicationMetrics
router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("getbucketreplicationmetrics", maxClients(httpTraceAll(api.GetBucketReplicationMetricsHandler)))).Queries("replication-metrics", "")
} }
/// Root operation /// Root operation

View File

@ -20,6 +20,7 @@ import (
"bytes" "bytes"
"crypto/subtle" "crypto/subtle"
"encoding/base64" "encoding/base64"
"encoding/json"
"encoding/xml" "encoding/xml"
"fmt" "fmt"
"io" "io"
@ -1243,7 +1244,7 @@ func (api objectAPIHandlers) DeleteBucketHandler(w http.ResponseWriter, r *http.
return return
} }
} }
globalReplicationStats.Delete(ctx, bucket)
// Write success response. // Write success response.
writeSuccessNoContent(w) writeSuccessNoContent(w)
@ -1603,3 +1604,40 @@ func (api objectAPIHandlers) DeleteBucketReplicationConfigHandler(w http.Respons
// Write success response. // Write success response.
writeSuccessResponseHeadersOnly(w) writeSuccessResponseHeadersOnly(w)
} }
// GetBucketReplicationMetricsHandler - GET Bucket replication metrics.
// ----------
// Gets the replication metrics for a bucket.
func (api objectAPIHandlers) GetBucketReplicationMetricsHandler(w http.ResponseWriter, r *http.Request) {
ctx := newContext(r, w, "GetBucketReplicationMetrics")
defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r))
vars := mux.Vars(r)
bucket := vars["bucket"]
objectAPI := api.ObjectAPI()
if objectAPI == nil {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(ErrServerNotInitialized), r.URL, guessIsBrowserReq(r))
return
}
// check if user has permissions to perform this operation
if s3Error := checkRequestAuthType(ctx, r, policy.GetReplicationConfigurationAction, bucket, ""); s3Error != ErrNone {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL, guessIsBrowserReq(r))
return
}
// Check if bucket exists.
if _, err := objectAPI.GetBucketInfo(ctx, bucket); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return
}
metrics := globalReplicationStats.Get(bucket)
if err := json.NewEncoder(w).Encode(&metrics); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return
}
w.(http.Flusher).Flush()
}

View File

@ -169,7 +169,10 @@ func (sys *BucketMetadataSys) Update(bucket string, configFile string, configDat
} }
meta.ReplicationConfigXML = configData meta.ReplicationConfigXML = configData
case bucketTargetsFile: case bucketTargetsFile:
meta.BucketTargetsConfigJSON, meta.BucketTargetsConfigMetaJSON, err = encryptBucketMetadata(meta.Name, configData, crypto.Context{bucket: meta.Name, bucketTargetsFile: bucketTargetsFile}) meta.BucketTargetsConfigJSON, meta.BucketTargetsConfigMetaJSON, err = encryptBucketMetadata(meta.Name, configData, crypto.Context{
bucket: meta.Name,
bucketTargetsFile: bucketTargetsFile,
})
if err != nil { if err != nil {
return fmt.Errorf("Error encrypting bucket target metadata %w", err) return fmt.Errorf("Error encrypting bucket target metadata %w", err)
} }

View File

@ -416,6 +416,7 @@ func encryptBucketMetadata(bucket string, input []byte, kmsContext crypto.Contex
if err != nil { if err != nil {
return return
} }
outbuf := bytes.NewBuffer(nil) outbuf := bytes.NewBuffer(nil)
objectKey := crypto.GenerateKey(key, rand.Reader) objectKey := crypto.GenerateKey(key, rand.Reader)
sealedKey = objectKey.Seal(key, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, "") sealedKey = objectKey.Seal(key, crypto.GenerateIV(rand.Reader), crypto.S3.String(), bucket, "")
@ -437,7 +438,6 @@ func decryptBucketMetadata(input []byte, bucket string, meta map[string]string,
return nil, errKMSNotConfigured return nil, errKMSNotConfigured
} }
keyID, kmsKey, sealedKey, err := crypto.S3.ParseMetadata(meta) keyID, kmsKey, sealedKey, err := crypto.S3.ParseMetadata(meta)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -449,8 +449,8 @@ func decryptBucketMetadata(input []byte, bucket string, meta map[string]string,
if err = objectKey.Unseal(extKey, sealedKey, crypto.S3.String(), bucket, ""); err != nil { if err = objectKey.Unseal(extKey, sealedKey, crypto.S3.String(), bucket, ""); err != nil {
return nil, err return nil, err
} }
outbuf := bytes.NewBuffer(nil) outbuf := bytes.NewBuffer(nil)
_, err = sio.Decrypt(outbuf, bytes.NewBuffer(input), sio.Config{Key: objectKey[:], MinVersion: sio.Version20}) _, err = sio.Decrypt(outbuf, bytes.NewBuffer(input), sio.Config{Key: objectKey[:], MinVersion: sio.Version20})
return outbuf.Bytes(), err return outbuf.Bytes(), err
} }

View File

@ -88,7 +88,7 @@ func (sys *BucketQuotaSys) check(ctx context.Context, bucket string, size int64)
return err return err
} }
dui := v.(DataUsageInfo) dui := v.(madmin.DataUsageInfo)
bui, ok := dui.BucketsUsage[bucket] bui, ok := dui.BucketsUsage[bucket]
if !ok { if !ok {
@ -115,7 +115,7 @@ func enforceBucketQuota(ctx context.Context, bucket string, size int64) error {
// enforceFIFOQuota deletes objects in FIFO order until sufficient objects // enforceFIFOQuota deletes objects in FIFO order until sufficient objects
// have been deleted so as to bring bucket usage within quota. // have been deleted so as to bring bucket usage within quota.
func enforceFIFOQuotaBucket(ctx context.Context, objectAPI ObjectLayer, bucket string, bui BucketUsageInfo) { func enforceFIFOQuotaBucket(ctx context.Context, objectAPI ObjectLayer, bucket string, bui madmin.BucketUsageInfo) {
// Check if the current bucket has quota restrictions, if not skip it // Check if the current bucket has quota restrictions, if not skip it
cfg, err := globalBucketQuotaSys.Get(bucket) cfg, err := globalBucketQuotaSys.Get(bucket)
if err != nil { if err != nil {

View File

@ -0,0 +1,173 @@
/*
* MinIO Cloud Storage, (C) 2021 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 cmd
import (
"context"
"sync"
"sync/atomic"
"github.com/minio/minio/pkg/bucket/replication"
)
// BucketReplicationStats represents inline replication statistics
// such as pending, failed and completed bytes in total for a bucket
type BucketReplicationStats struct {
// Pending size in bytes
PendingSize uint64 `json:"pendingReplicationSize"`
// Completed size in bytes
ReplicatedSize uint64 `json:"completedReplicationSize"`
// Total Replica size in bytes
ReplicaSize uint64 `json:"replicaSize"`
// Failed size in bytes
FailedSize uint64 `json:"failedReplicationSize"`
// Total number of pending operations including metadata updates
PendingCount uint64 `json:"pendingReplicationCount"`
// Total number of failed operations including metadata updates
FailedCount uint64 `json:"failedReplicationCount"`
}
func (b *BucketReplicationStats) hasReplicationUsage() bool {
return b.PendingSize > 0 ||
b.FailedSize > 0 ||
b.ReplicatedSize > 0 ||
b.ReplicaSize > 0 ||
b.PendingCount > 0 ||
b.FailedCount > 0
}
// ReplicationStats holds the global in-memory replication stats
type ReplicationStats struct {
sync.RWMutex
Cache map[string]*BucketReplicationStats
}
// Delete deletes in-memory replication statistics for a bucket.
func (r *ReplicationStats) Delete(ctx context.Context, bucket string) {
if r == nil {
return
}
r.Lock()
defer r.Unlock()
delete(r.Cache, bucket)
}
// Update updates in-memory replication statistics with new values.
func (r *ReplicationStats) Update(ctx context.Context, bucket string, n int64, status, prevStatus replication.StatusType, opType replication.Type) {
if r == nil {
return
}
r.RLock()
b, ok := r.Cache[bucket]
if !ok {
b = &BucketReplicationStats{}
}
r.RUnlock()
switch status {
case replication.Pending:
if opType == replication.ObjectReplicationType {
atomic.AddUint64(&b.PendingSize, uint64(n))
}
atomic.AddUint64(&b.PendingCount, 1)
case replication.Completed:
switch prevStatus { // adjust counters based on previous state
case replication.Pending:
atomic.AddUint64(&b.PendingCount, ^uint64(0))
case replication.Failed:
atomic.AddUint64(&b.FailedCount, ^uint64(0))
}
if opType == replication.ObjectReplicationType {
atomic.AddUint64(&b.ReplicatedSize, uint64(n))
switch prevStatus {
case replication.Pending:
atomic.AddUint64(&b.PendingSize, ^uint64(n-1))
case replication.Failed:
atomic.AddUint64(&b.FailedSize, ^uint64(n-1))
}
}
case replication.Failed:
// count failures only once - not on every retry
if opType == replication.ObjectReplicationType {
if prevStatus == replication.Pending {
atomic.AddUint64(&b.FailedSize, uint64(n))
atomic.AddUint64(&b.FailedCount, 1)
}
}
case replication.Replica:
if opType == replication.ObjectReplicationType {
atomic.AddUint64(&b.ReplicaSize, uint64(n))
}
}
}
// Get total bytes pending replication for a bucket
func (r *ReplicationStats) Get(bucket string) BucketReplicationStats {
if r == nil {
return BucketReplicationStats{}
}
r.RLock()
defer r.RUnlock()
st, ok := r.Cache[bucket]
if !ok {
return BucketReplicationStats{}
}
return BucketReplicationStats{
PendingSize: atomic.LoadUint64(&st.PendingSize),
FailedSize: atomic.LoadUint64(&st.FailedSize),
ReplicatedSize: atomic.LoadUint64(&st.ReplicatedSize),
ReplicaSize: atomic.LoadUint64(&st.ReplicaSize),
PendingCount: atomic.LoadUint64(&st.PendingCount),
FailedCount: atomic.LoadUint64(&st.FailedCount),
}
}
// NewReplicationStats initialize in-memory replication statistics
func NewReplicationStats(ctx context.Context, objectAPI ObjectLayer) *ReplicationStats {
st := &ReplicationStats{
Cache: make(map[string]*BucketReplicationStats),
}
dataUsageInfo, err := loadDataUsageFromBackend(ctx, objectAPI)
if err != nil {
return st
}
// data usage has not captured any data yet.
if dataUsageInfo.LastUpdate.IsZero() {
return st
}
for bucket, usage := range dataUsageInfo.BucketsUsage {
b := &BucketReplicationStats{
PendingSize: usage.ReplicationPendingSize,
FailedSize: usage.ReplicationFailedSize,
ReplicatedSize: usage.ReplicatedSize,
ReplicaSize: usage.ReplicaSize,
PendingCount: usage.ReplicationPendingCount,
FailedCount: usage.ReplicationFailedCount,
}
if b.hasReplicationUsage() {
st.Cache[bucket] = b
}
}
return st
}

View File

@ -291,6 +291,13 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectVersionInfo, objectA
versionPurgeStatus = Complete versionPurgeStatus = Complete
} }
} }
prevStatus := dobj.DeleteMarkerReplicationStatus
currStatus := replicationStatus
if dobj.VersionID != "" {
prevStatus = string(dobj.VersionPurgeStatus)
currStatus = string(versionPurgeStatus)
}
globalReplicationStats.Update(ctx, dobj.Bucket, 0, replication.StatusType(currStatus), replication.StatusType(prevStatus), replication.DeleteReplicationType) // to decrement pending count
var eventName = event.ObjectReplicationComplete var eventName = event.ObjectReplicationComplete
if replicationStatus == string(replication.Failed) || versionPurgeStatus == Failed { if replicationStatus == string(replication.Failed) || versionPurgeStatus == Failed {
@ -594,7 +601,7 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
}) })
return return
} }
gr, err := objectAPI.GetObjectNInfo(ctx, bucket, object, nil, http.Header{}, readLock, ObjectOptions{ gr, err := objectAPI.GetObjectNInfo(ctx, bucket, object, nil, http.Header{}, writeLock, ObjectOptions{
VersionID: objInfo.VersionID, VersionID: objInfo.VersionID,
}) })
if err != nil { if err != nil {
@ -604,10 +611,10 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
Object: objInfo, Object: objInfo,
Host: "Internal: [Replication]", Host: "Internal: [Replication]",
}) })
logger.LogIf(ctx, err) logger.LogIf(ctx, fmt.Errorf("Unable to update replicate for %s/%s(%s): %w", bucket, object, objInfo.VersionID, err))
return return
} }
defer gr.Close() // hold read lock for entire transaction defer gr.Close() // hold write lock for entire transaction
objInfo = gr.ObjInfo objInfo = gr.ObjInfo
size, err := objInfo.GetActualSize() size, err := objInfo.GetActualSize()
@ -644,7 +651,7 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
rtype = getReplicationAction(objInfo, oi) rtype = getReplicationAction(objInfo, oi)
if rtype == replicateNone { if rtype == replicateNone {
// object with same VersionID already exists, replication kicked off by // object with same VersionID already exists, replication kicked off by
// PutObject might have completed. // PutObject might have completed
return return
} }
} }
@ -656,7 +663,8 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
srcOpts := miniogo.CopySrcOptions{ srcOpts := miniogo.CopySrcOptions{
Bucket: dest.Bucket, Bucket: dest.Bucket,
Object: object, Object: object,
VersionID: objInfo.VersionID} VersionID: objInfo.VersionID,
}
dstOpts := miniogo.PutObjectOptions{ dstOpts := miniogo.PutObjectOptions{
Internal: miniogo.AdvancedPutOptions{ Internal: miniogo.AdvancedPutOptions{
SourceVersionID: objInfo.VersionID, SourceVersionID: objInfo.VersionID,
@ -712,6 +720,7 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
defer r.Close() defer r.Close()
} }
prevReplStatus := objInfo.ReplicationStatus
objInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replicationStatus.String() objInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replicationStatus.String()
if objInfo.UserTags != "" { if objInfo.UserTags != "" {
objInfo.UserDefined[xhttp.AmzObjectTagging] = objInfo.UserTags objInfo.UserDefined[xhttp.AmzObjectTagging] = objInfo.UserTags
@ -736,6 +745,11 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
logger.LogIf(ctx, fmt.Errorf("Unable to update replication metadata for %s/%s(%s): %w", bucket, objInfo.Name, objInfo.VersionID, err)) logger.LogIf(ctx, fmt.Errorf("Unable to update replication metadata for %s/%s(%s): %w", bucket, objInfo.Name, objInfo.VersionID, err))
} }
} }
opType := replication.MetadataReplicationType
if rtype == replicateAll {
opType = replication.ObjectReplicationType
}
globalReplicationStats.Update(ctx, bucket, size, replicationStatus, prevReplStatus, opType)
sendEvent(eventArgs{ sendEvent(eventArgs{
EventName: eventName, EventName: eventName,
BucketName: bucket, BucketName: bucket,
@ -775,6 +789,7 @@ type DeletedObjectVersionInfo struct {
var ( var (
globalReplicationPool *ReplicationPool globalReplicationPool *ReplicationPool
globalReplicationStats *ReplicationStats
) )
// ReplicationPool describes replication pool // ReplicationPool describes replication pool
@ -783,6 +798,8 @@ type ReplicationPool struct {
size int size int
replicaCh chan ObjectInfo replicaCh chan ObjectInfo
replicaDeleteCh chan DeletedObjectVersionInfo replicaDeleteCh chan DeletedObjectVersionInfo
mrfReplicaCh chan ObjectInfo
mrfReplicaDeleteCh chan DeletedObjectVersionInfo
killCh chan struct{} killCh chan struct{}
wg sync.WaitGroup wg sync.WaitGroup
ctx context.Context ctx context.Context
@ -792,20 +809,40 @@ type ReplicationPool struct {
// NewReplicationPool creates a pool of replication workers of specified size // NewReplicationPool creates a pool of replication workers of specified size
func NewReplicationPool(ctx context.Context, o ObjectLayer, sz int) *ReplicationPool { func NewReplicationPool(ctx context.Context, o ObjectLayer, sz int) *ReplicationPool {
pool := &ReplicationPool{ pool := &ReplicationPool{
replicaCh: make(chan ObjectInfo, 10000), replicaCh: make(chan ObjectInfo, 1000),
replicaDeleteCh: make(chan DeletedObjectVersionInfo, 10000), replicaDeleteCh: make(chan DeletedObjectVersionInfo, 1000),
mrfReplicaCh: make(chan ObjectInfo, 100000),
mrfReplicaDeleteCh: make(chan DeletedObjectVersionInfo, 100000),
ctx: ctx, ctx: ctx,
objLayer: o, objLayer: o,
} }
go func() {
<-ctx.Done()
close(pool.replicaCh)
close(pool.replicaDeleteCh)
}()
pool.Resize(sz) pool.Resize(sz)
// add long running worker for handling most recent failures/pending replications
go pool.AddMRFWorker()
return pool return pool
} }
// AddMRFWorker adds a pending/failed replication worker to handle requests that could not be queued
// to the other workers
func (p *ReplicationPool) AddMRFWorker() {
for {
select {
case <-p.ctx.Done():
return
case oi, ok := <-p.mrfReplicaCh:
if !ok {
return
}
replicateObject(p.ctx, oi, p.objLayer)
case doi, ok := <-p.mrfReplicaDeleteCh:
if !ok {
return
}
replicateDelete(p.ctx, doi, p.objLayer)
}
}
}
// AddWorker adds a replication worker to the pool // AddWorker adds a replication worker to the pool
func (p *ReplicationPool) AddWorker() { func (p *ReplicationPool) AddWorker() {
defer p.wg.Done() defer p.wg.Done()
@ -846,28 +883,39 @@ func (p *ReplicationPool) Resize(n int) {
} }
} }
func (p *ReplicationPool) queueReplicaTask(oi ObjectInfo) { func (p *ReplicationPool) queueReplicaTask(ctx context.Context, oi ObjectInfo) {
if p == nil { if p == nil {
return return
} }
select { select {
case <-ctx.Done():
close(p.replicaCh)
close(p.mrfReplicaCh)
case p.replicaCh <- oi: case p.replicaCh <- oi:
case p.mrfReplicaCh <- oi:
// queue all overflows into the mrfReplicaCh to handle incoming pending/failed operations
default: default:
} }
} }
func (p *ReplicationPool) queueReplicaDeleteTask(doi DeletedObjectVersionInfo) { func (p *ReplicationPool) queueReplicaDeleteTask(ctx context.Context, doi DeletedObjectVersionInfo) {
if p == nil { if p == nil {
return return
} }
select { select {
case <-ctx.Done():
close(p.replicaDeleteCh)
close(p.mrfReplicaDeleteCh)
case p.replicaDeleteCh <- doi: case p.replicaDeleteCh <- doi:
case p.mrfReplicaDeleteCh <- doi:
// queue all overflows into the mrfReplicaDeleteCh to handle incoming pending/failed operations
default: default:
} }
} }
func initBackgroundReplication(ctx context.Context, objectAPI ObjectLayer) { func initBackgroundReplication(ctx context.Context, objectAPI ObjectLayer) {
globalReplicationPool = NewReplicationPool(ctx, objectAPI, globalAPIConfig.getReplicationWorkers()) globalReplicationPool = NewReplicationPool(ctx, objectAPI, globalAPIConfig.getReplicationWorkers())
globalReplicationStats = NewReplicationStats(ctx, objectAPI)
} }
// get Reader from replication target if active-active replication is in place and // get Reader from replication target if active-active replication is in place and
@ -1003,11 +1051,14 @@ func proxyHeadToReplicationTarget(ctx context.Context, bucket, object string, op
return oi, proxy, err return oi, proxy, err
} }
func scheduleReplication(ctx context.Context, objInfo ObjectInfo, o ObjectLayer, sync bool) { func scheduleReplication(ctx context.Context, objInfo ObjectInfo, o ObjectLayer, sync bool, opType replication.Type) {
if sync { if sync {
replicateObject(ctx, objInfo, o) replicateObject(ctx, objInfo, o)
} else { } else {
globalReplicationPool.queueReplicaTask(objInfo) globalReplicationPool.queueReplicaTask(GlobalContext, objInfo)
}
if sz, err := objInfo.GetActualSize(); err == nil {
globalReplicationStats.Update(ctx, objInfo.Bucket, sz, objInfo.ReplicationStatus, replication.StatusType(""), opType)
} }
} }
@ -1015,6 +1066,7 @@ func scheduleReplicationDelete(ctx context.Context, dv DeletedObjectVersionInfo,
if sync { if sync {
replicateDelete(ctx, dv, o) replicateDelete(ctx, dv, o)
} else { } else {
globalReplicationPool.queueReplicaDeleteTask(dv) globalReplicationPool.queueReplicaDeleteTask(GlobalContext, dv)
} }
globalReplicationStats.Update(ctx, dv.Bucket, 0, replication.Pending, replication.StatusType(""), replication.DeleteReplicationType)
} }

View File

@ -31,6 +31,7 @@ import (
miniogo "github.com/minio/minio-go/v7" miniogo "github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials" "github.com/minio/minio-go/v7/pkg/credentials"
"github.com/minio/minio/cmd/crypto" "github.com/minio/minio/cmd/crypto"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/bucket/versioning" "github.com/minio/minio/pkg/bucket/versioning"
"github.com/minio/minio/pkg/madmin" "github.com/minio/minio/pkg/madmin"
) )
@ -328,6 +329,7 @@ func (sys *BucketTargetSys) load(ctx context.Context, buckets []BucketInfo, objA
for _, bucket := range buckets { for _, bucket := range buckets {
cfg, err := globalBucketMetadataSys.GetBucketTargetsConfig(bucket.Name) cfg, err := globalBucketMetadataSys.GetBucketTargetsConfig(bucket.Name)
if err != nil { if err != nil {
logger.LogIf(ctx, err)
continue continue
} }
if cfg == nil || cfg.Empty() { if cfg == nil || cfg.Empty() {
@ -339,6 +341,7 @@ func (sys *BucketTargetSys) load(ctx context.Context, buckets []BucketInfo, objA
for _, tgt := range cfg.Targets { for _, tgt := range cfg.Targets {
tgtClient, err := sys.getRemoteTargetClient(&tgt) tgtClient, err := sys.getRemoteTargetClient(&tgt)
if err != nil { if err != nil {
logger.LogIf(ctx, err)
continue continue
} }
sys.arnRemotesMap[tgt.Arn] = tgtClient sys.arnRemotesMap[tgt.Arn] = tgtClient
@ -432,7 +435,10 @@ func parseBucketTargetConfig(bucket string, cdata, cmetadata []byte) (*madmin.Bu
return nil, err return nil, err
} }
if crypto.S3.IsEncrypted(meta) { if crypto.S3.IsEncrypted(meta) {
if data, err = decryptBucketMetadata(cdata, bucket, meta, crypto.Context{bucket: bucket, bucketTargetsFile: bucketTargetsFile}); err != nil { if data, err = decryptBucketMetadata(cdata, bucket, meta, crypto.Context{
bucket: bucket,
bucketTargetsFile: bucketTargetsFile,
}); err != nil {
return nil, err return nil, err
} }
} }

View File

@ -37,6 +37,7 @@ const (
apiListQuorum = "list_quorum" apiListQuorum = "list_quorum"
apiExtendListCacheLife = "extend_list_cache_life" apiExtendListCacheLife = "extend_list_cache_life"
apiReplicationWorkers = "replication_workers" apiReplicationWorkers = "replication_workers"
EnvAPIRequestsMax = "MINIO_API_REQUESTS_MAX" EnvAPIRequestsMax = "MINIO_API_REQUESTS_MAX"
EnvAPIRequestsDeadline = "MINIO_API_REQUESTS_DEADLINE" EnvAPIRequestsDeadline = "MINIO_API_REQUESTS_DEADLINE"
EnvAPIClusterDeadline = "MINIO_API_CLUSTER_DEADLINE" EnvAPIClusterDeadline = "MINIO_API_CLUSTER_DEADLINE"
@ -87,7 +88,7 @@ var (
}, },
config.KV{ config.KV{
Key: apiReplicationWorkers, Key: apiReplicationWorkers,
Value: "100", Value: "500",
}, },
} }
) )

View File

@ -134,7 +134,7 @@ func runDataScanner(ctx context.Context, objAPI ObjectLayer) {
} }
// Wait before starting next cycle and wait on startup. // Wait before starting next cycle and wait on startup.
results := make(chan DataUsageInfo, 1) results := make(chan madmin.DataUsageInfo, 1)
go storeDataUsageInBackend(ctx, objAPI, results) go storeDataUsageInBackend(ctx, objAPI, results)
bf, err := globalNotificationSys.updateBloomFilter(ctx, nextBloomCycle) bf, err := globalNotificationSys.updateBloomFilter(ctx, nextBloomCycle)
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
@ -790,6 +790,8 @@ type sizeSummary struct {
pendingSize int64 pendingSize int64
failedSize int64 failedSize int64
replicaSize int64 replicaSize int64
pendingCount uint64
failedCount uint64
} }
type getSizeFn func(item scannerItem) (sizeSummary, error) type getSizeFn func(item scannerItem) (sizeSummary, error)
@ -1105,11 +1107,13 @@ func (i *scannerItem) healReplication(ctx context.Context, o ObjectLayer, oi Obj
} }
switch oi.ReplicationStatus { switch oi.ReplicationStatus {
case replication.Pending: case replication.Pending:
sizeS.pendingCount++
sizeS.pendingSize += oi.Size sizeS.pendingSize += oi.Size
globalReplicationPool.queueReplicaTask(oi) globalReplicationPool.queueReplicaTask(ctx, oi)
case replication.Failed: case replication.Failed:
sizeS.failedSize += oi.Size sizeS.failedSize += oi.Size
globalReplicationPool.queueReplicaTask(oi) sizeS.failedCount++
globalReplicationPool.queueReplicaTask(ctx, oi)
case replication.Completed, "COMPLETE": case replication.Completed, "COMPLETE":
sizeS.replicatedSize += oi.Size sizeS.replicatedSize += oi.Size
case replication.Replica: case replication.Replica:
@ -1128,7 +1132,7 @@ func (i *scannerItem) healReplicationDeletes(ctx context.Context, o ObjectLayer,
} else { } else {
versionID = oi.VersionID versionID = oi.VersionID
} }
globalReplicationPool.queueReplicaDeleteTask(DeletedObjectVersionInfo{ globalReplicationPool.queueReplicaDeleteTask(ctx, DeletedObjectVersionInfo{
DeletedObject: DeletedObject{ DeletedObject: DeletedObject{
ObjectName: oi.Name, ObjectName: oi.Name,
DeleteMarkerVersionID: dmVersionID, DeleteMarkerVersionID: dmVersionID,

View File

@ -31,11 +31,9 @@ import (
"sync" "sync"
"time" "time"
"github.com/minio/minio/cmd/config"
"github.com/minio/minio/cmd/logger" "github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/color" "github.com/minio/minio/pkg/color"
"github.com/minio/minio/pkg/console" "github.com/minio/minio/pkg/console"
"github.com/minio/minio/pkg/env"
"github.com/willf/bloom" "github.com/willf/bloom"
) )
@ -80,7 +78,7 @@ func newDataUpdateTracker() *dataUpdateTracker {
Current: dataUpdateFilter{ Current: dataUpdateFilter{
idx: 1, idx: 1,
}, },
debug: env.Get(envDataUsageScannerDebug, config.EnableOff) == config.EnableOn || serverDebugLog, debug: serverDebugLog,
input: make(chan string, dataUpdateTrackerQueueSize), input: make(chan string, dataUpdateTrackerQueueSize),
save: make(chan struct{}, 1), save: make(chan struct{}, 1),
saveExited: make(chan struct{}), saveExited: make(chan struct{}),

View File

@ -32,6 +32,7 @@ import (
"github.com/minio/minio/cmd/logger" "github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/bucket/lifecycle" "github.com/minio/minio/pkg/bucket/lifecycle"
"github.com/minio/minio/pkg/hash" "github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/madmin"
"github.com/tinylib/msgp/msgp" "github.com/tinylib/msgp/msgp"
) )
@ -45,15 +46,26 @@ type sizeHistogram [dataUsageBucketLen]uint64
//msgp:tuple dataUsageEntry //msgp:tuple dataUsageEntry
type dataUsageEntry struct { type dataUsageEntry struct {
Children dataUsageHashMap
// These fields do no include any children. // These fields do no include any children.
Size int64 Size int64
ReplicatedSize uint64
ReplicationPendingSize uint64
ReplicationFailedSize uint64
ReplicaSize uint64
Objects uint64 Objects uint64
ObjSizes sizeHistogram ObjSizes sizeHistogram
Children dataUsageHashMap ReplicationStats replicationStats
}
//msgp:tuple replicationStats
type replicationStats struct {
PendingSize uint64
ReplicatedSize uint64
FailedSize uint64
ReplicaSize uint64
FailedCount uint64
PendingCount uint64
MissedThresholdSize uint64
AfterThresholdSize uint64
MissedThresholdCount uint64
AfterThresholdCount uint64
} }
//msgp:tuple dataUsageEntryV2 //msgp:tuple dataUsageEntryV2
@ -65,20 +77,40 @@ type dataUsageEntryV2 struct {
Children dataUsageHashMap Children dataUsageHashMap
} }
// dataUsageCache contains a cache of data usage entries latest version 3. //msgp:tuple dataUsageEntryV3
type dataUsageCache struct { type dataUsageEntryV3 struct {
Info dataUsageCacheInfo // These fields do no include any children.
Disks []string Size int64
Cache map[string]dataUsageEntry ReplicatedSize uint64
ReplicationPendingSize uint64
ReplicationFailedSize uint64
ReplicaSize uint64
Objects uint64
ObjSizes sizeHistogram
Children dataUsageHashMap
} }
// dataUsageCache contains a cache of data usage entries version 2. // dataUsageCache contains a cache of data usage entries latest version 4.
type dataUsageCache struct {
Info dataUsageCacheInfo
Cache map[string]dataUsageEntry
Disks []string
}
// dataUsageCacheV2 contains a cache of data usage entries version 2.
type dataUsageCacheV2 struct { type dataUsageCacheV2 struct {
Info dataUsageCacheInfo Info dataUsageCacheInfo
Disks []string Disks []string
Cache map[string]dataUsageEntryV2 Cache map[string]dataUsageEntryV2
} }
// dataUsageCache contains a cache of data usage entries version 3.
type dataUsageCacheV3 struct {
Info dataUsageCacheInfo
Disks []string
Cache map[string]dataUsageEntryV3
}
//msgp:ignore dataUsageEntryInfo //msgp:ignore dataUsageEntryInfo
type dataUsageEntryInfo struct { type dataUsageEntryInfo struct {
Name string Name string
@ -89,8 +121,8 @@ type dataUsageEntryInfo struct {
type dataUsageCacheInfo struct { type dataUsageCacheInfo struct {
// Name of the bucket. Also root element. // Name of the bucket. Also root element.
Name string Name string
LastUpdate time.Time
NextCycle uint32 NextCycle uint32
LastUpdate time.Time
// indicates if the disk is being healed and scanner // indicates if the disk is being healed and scanner
// should skip healing the disk // should skip healing the disk
SkipHealing bool SkipHealing bool
@ -100,20 +132,25 @@ type dataUsageCacheInfo struct {
func (e *dataUsageEntry) addSizes(summary sizeSummary) { func (e *dataUsageEntry) addSizes(summary sizeSummary) {
e.Size += summary.totalSize e.Size += summary.totalSize
e.ReplicatedSize += uint64(summary.replicatedSize) e.ReplicationStats.ReplicatedSize += uint64(summary.replicatedSize)
e.ReplicationFailedSize += uint64(summary.failedSize) e.ReplicationStats.FailedSize += uint64(summary.failedSize)
e.ReplicationPendingSize += uint64(summary.pendingSize) e.ReplicationStats.PendingSize += uint64(summary.pendingSize)
e.ReplicaSize += uint64(summary.replicaSize) e.ReplicationStats.ReplicaSize += uint64(summary.replicaSize)
e.ReplicationStats.PendingCount += uint64(summary.pendingCount)
e.ReplicationStats.FailedCount += uint64(summary.failedCount)
} }
// merge other data usage entry into this, excluding children. // merge other data usage entry into this, excluding children.
func (e *dataUsageEntry) merge(other dataUsageEntry) { func (e *dataUsageEntry) merge(other dataUsageEntry) {
e.Objects += other.Objects e.Objects += other.Objects
e.Size += other.Size e.Size += other.Size
e.ReplicationPendingSize += other.ReplicationPendingSize e.ReplicationStats.PendingSize += other.ReplicationStats.PendingSize
e.ReplicationFailedSize += other.ReplicationFailedSize e.ReplicationStats.FailedSize += other.ReplicationStats.FailedSize
e.ReplicatedSize += other.ReplicatedSize e.ReplicationStats.ReplicatedSize += other.ReplicationStats.ReplicatedSize
e.ReplicaSize += other.ReplicaSize e.ReplicationStats.ReplicaSize += other.ReplicationStats.ReplicaSize
e.ReplicationStats.PendingCount += other.ReplicationStats.PendingCount
e.ReplicationStats.FailedCount += other.ReplicationStats.FailedCount
for i, v := range other.ObjSizes[:] { for i, v := range other.ObjSizes[:] {
e.ObjSizes[i] += v e.ObjSizes[i] += v
@ -238,23 +275,25 @@ func (d *dataUsageCache) keepRootChildren(list map[dataUsageHash]struct{}) {
} }
} }
// dui converts the flattened version of the path to DataUsageInfo. // dui converts the flattened version of the path to madmin.DataUsageInfo.
// As a side effect d will be flattened, use a clone if this is not ok. // As a side effect d will be flattened, use a clone if this is not ok.
func (d *dataUsageCache) dui(path string, buckets []BucketInfo) DataUsageInfo { func (d *dataUsageCache) dui(path string, buckets []BucketInfo) madmin.DataUsageInfo {
e := d.find(path) e := d.find(path)
if e == nil { if e == nil {
// No entry found, return empty. // No entry found, return empty.
return DataUsageInfo{} return madmin.DataUsageInfo{}
} }
flat := d.flatten(*e) flat := d.flatten(*e)
return DataUsageInfo{ return madmin.DataUsageInfo{
LastUpdate: d.Info.LastUpdate, LastUpdate: d.Info.LastUpdate,
ObjectsTotalCount: flat.Objects, ObjectsTotalCount: flat.Objects,
ObjectsTotalSize: uint64(flat.Size), ObjectsTotalSize: uint64(flat.Size),
ReplicatedSize: flat.ReplicatedSize, ReplicatedSize: flat.ReplicationStats.ReplicatedSize,
ReplicationFailedSize: flat.ReplicationFailedSize, ReplicationFailedSize: flat.ReplicationStats.FailedSize,
ReplicationPendingSize: flat.ReplicationPendingSize, ReplicationPendingSize: flat.ReplicationStats.PendingSize,
ReplicaSize: flat.ReplicaSize, ReplicaSize: flat.ReplicationStats.ReplicaSize,
ReplicationPendingCount: flat.ReplicationStats.PendingCount,
ReplicationFailedCount: flat.ReplicationStats.FailedCount,
BucketsCount: uint64(len(e.Children)), BucketsCount: uint64(len(e.Children)),
BucketsUsage: d.bucketsUsageInfo(buckets), BucketsUsage: d.bucketsUsageInfo(buckets),
} }
@ -373,21 +412,23 @@ func (h *sizeHistogram) toMap() map[string]uint64 {
// bucketsUsageInfo returns the buckets usage info as a map, with // bucketsUsageInfo returns the buckets usage info as a map, with
// key as bucket name // key as bucket name
func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]BucketUsageInfo { func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]madmin.BucketUsageInfo {
var dst = make(map[string]BucketUsageInfo, len(buckets)) var dst = make(map[string]madmin.BucketUsageInfo, len(buckets))
for _, bucket := range buckets { for _, bucket := range buckets {
e := d.find(bucket.Name) e := d.find(bucket.Name)
if e == nil { if e == nil {
continue continue
} }
flat := d.flatten(*e) flat := d.flatten(*e)
dst[bucket.Name] = BucketUsageInfo{ dst[bucket.Name] = madmin.BucketUsageInfo{
Size: uint64(flat.Size), Size: uint64(flat.Size),
ObjectsCount: flat.Objects, ObjectsCount: flat.Objects,
ReplicationPendingSize: flat.ReplicationPendingSize, ReplicationPendingSize: flat.ReplicationStats.PendingSize,
ReplicatedSize: flat.ReplicatedSize, ReplicatedSize: flat.ReplicationStats.ReplicatedSize,
ReplicationFailedSize: flat.ReplicationFailedSize, ReplicationFailedSize: flat.ReplicationStats.FailedSize,
ReplicaSize: flat.ReplicaSize, ReplicationPendingCount: flat.ReplicationStats.PendingCount,
ReplicationFailedCount: flat.ReplicationStats.FailedCount,
ReplicaSize: flat.ReplicationStats.ReplicaSize,
ObjectSizesHistogram: flat.ObjSizes.toMap(), ObjectSizesHistogram: flat.ObjSizes.toMap(),
} }
} }
@ -396,19 +437,21 @@ func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]Bucke
// bucketUsageInfo returns the buckets usage info. // bucketUsageInfo returns the buckets usage info.
// If not found all values returned are zero values. // If not found all values returned are zero values.
func (d *dataUsageCache) bucketUsageInfo(bucket string) BucketUsageInfo { func (d *dataUsageCache) bucketUsageInfo(bucket string) madmin.BucketUsageInfo {
e := d.find(bucket) e := d.find(bucket)
if e == nil { if e == nil {
return BucketUsageInfo{} return madmin.BucketUsageInfo{}
} }
flat := d.flatten(*e) flat := d.flatten(*e)
return BucketUsageInfo{ return madmin.BucketUsageInfo{
Size: uint64(flat.Size), Size: uint64(flat.Size),
ObjectsCount: flat.Objects, ObjectsCount: flat.Objects,
ReplicationPendingSize: flat.ReplicationPendingSize, ReplicationPendingSize: flat.ReplicationStats.PendingSize,
ReplicatedSize: flat.ReplicatedSize, ReplicationPendingCount: flat.ReplicationStats.PendingCount,
ReplicationFailedSize: flat.ReplicationFailedSize, ReplicatedSize: flat.ReplicationStats.ReplicatedSize,
ReplicaSize: flat.ReplicaSize, ReplicationFailedSize: flat.ReplicationStats.FailedSize,
ReplicationFailedCount: flat.ReplicationStats.FailedCount,
ReplicaSize: flat.ReplicationStats.ReplicaSize,
ObjectSizesHistogram: flat.ObjSizes.toMap(), ObjectSizesHistogram: flat.ObjSizes.toMap(),
} }
} }
@ -533,6 +576,7 @@ func (d *dataUsageCache) save(ctx context.Context, store objectIO, name string)
// Bumping the cache version will drop data from previous versions // Bumping the cache version will drop data from previous versions
// and write new data with the new version. // and write new data with the new version.
const ( const (
dataUsageCacheVerV4 = 4
dataUsageCacheVerV3 = 3 dataUsageCacheVerV3 = 3
dataUsageCacheVerV2 = 2 dataUsageCacheVerV2 = 2
dataUsageCacheVerV1 = 1 dataUsageCacheVerV1 = 1
@ -541,7 +585,7 @@ const (
// serialize the contents of the cache. // serialize the contents of the cache.
func (d *dataUsageCache) serializeTo(dst io.Writer) error { func (d *dataUsageCache) serializeTo(dst io.Writer) error {
// Add version and compress. // Add version and compress.
_, err := dst.Write([]byte{dataUsageCacheVerV3}) _, err := dst.Write([]byte{dataUsageCacheVerV4})
if err != nil { if err != nil {
return err return err
} }
@ -609,6 +653,35 @@ func (d *dataUsageCache) deserialize(r io.Reader) error {
return err return err
} }
defer dec.Close() defer dec.Close()
dold := &dataUsageCacheV3{}
if err = dold.DecodeMsg(msgp.NewReader(dec)); err != nil {
return err
}
d.Info = dold.Info
d.Disks = dold.Disks
d.Cache = make(map[string]dataUsageEntry, len(dold.Cache))
for k, v := range dold.Cache {
d.Cache[k] = dataUsageEntry{
Size: v.Size,
Objects: v.Objects,
ObjSizes: v.ObjSizes,
Children: v.Children,
ReplicationStats: replicationStats{
ReplicatedSize: v.ReplicatedSize,
ReplicaSize: v.ReplicaSize,
FailedSize: v.ReplicationFailedSize,
PendingSize: v.ReplicationPendingSize,
},
}
}
return nil
case dataUsageCacheVerV4:
// Zstd compressed.
dec, err := zstd.NewReader(r, zstd.WithDecoderConcurrency(2))
if err != nil {
return err
}
defer dec.Close()
return d.DecodeMsg(msgp.NewReader(dec)) return d.DecodeMsg(msgp.NewReader(dec))
} }

File diff suppressed because it is too large Load Diff

View File

@ -348,6 +348,119 @@ func BenchmarkDecodedataUsageCacheV2(b *testing.B) {
} }
} }
func TestMarshalUnmarshaldataUsageCacheV3(t *testing.T) {
v := dataUsageCacheV3{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgdataUsageCacheV3(b *testing.B) {
v := dataUsageCacheV3{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgdataUsageCacheV3(b *testing.B) {
v := dataUsageCacheV3{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshaldataUsageCacheV3(b *testing.B) {
v := dataUsageCacheV3{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodedataUsageCacheV3(t *testing.T) {
v := dataUsageCacheV3{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodedataUsageCacheV3 Msgsize() is inaccurate")
}
vn := dataUsageCacheV3{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodedataUsageCacheV3(b *testing.B) {
v := dataUsageCacheV3{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodedataUsageCacheV3(b *testing.B) {
v := dataUsageCacheV3{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshaldataUsageEntry(t *testing.T) { func TestMarshalUnmarshaldataUsageEntry(t *testing.T) {
v := dataUsageEntry{} v := dataUsageEntry{}
bts, err := v.MarshalMsg(nil) bts, err := v.MarshalMsg(nil)
@ -574,6 +687,232 @@ func BenchmarkDecodedataUsageEntryV2(b *testing.B) {
} }
} }
func TestMarshalUnmarshaldataUsageEntryV3(t *testing.T) {
v := dataUsageEntryV3{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgdataUsageEntryV3(b *testing.B) {
v := dataUsageEntryV3{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgdataUsageEntryV3(b *testing.B) {
v := dataUsageEntryV3{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshaldataUsageEntryV3(b *testing.B) {
v := dataUsageEntryV3{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodedataUsageEntryV3(t *testing.T) {
v := dataUsageEntryV3{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodedataUsageEntryV3 Msgsize() is inaccurate")
}
vn := dataUsageEntryV3{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodedataUsageEntryV3(b *testing.B) {
v := dataUsageEntryV3{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodedataUsageEntryV3(b *testing.B) {
v := dataUsageEntryV3{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalreplicationStats(t *testing.T) {
v := replicationStats{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgreplicationStats(b *testing.B) {
v := replicationStats{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgreplicationStats(b *testing.B) {
v := replicationStats{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalreplicationStats(b *testing.B) {
v := replicationStats{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodereplicationStats(t *testing.T) {
v := replicationStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodereplicationStats Msgsize() is inaccurate")
}
vn := replicationStats{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodereplicationStats(b *testing.B) {
v := replicationStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodereplicationStats(b *testing.B) {
v := replicationStats{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalsizeHistogram(t *testing.T) { func TestMarshalUnmarshalsizeHistogram(t *testing.T) {
v := sizeHistogram{} v := sizeHistogram{}
bts, err := v.MarshalMsg(nil) bts, err := v.MarshalMsg(nil)

View File

@ -25,11 +25,10 @@ import (
jsoniter "github.com/json-iterator/go" jsoniter "github.com/json-iterator/go"
"github.com/minio/minio/cmd/logger" "github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/hash" "github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/madmin"
) )
const ( const (
envDataUsageScannerDebug = "MINIO_DISK_USAGE_SCANNER_DEBUG"
dataUsageRoot = SlashSeparator dataUsageRoot = SlashSeparator
dataUsageBucket = minioMetaBucket + SlashSeparator + bucketMetaPrefix dataUsageBucket = minioMetaBucket + SlashSeparator + bucketMetaPrefix
@ -39,7 +38,7 @@ const (
) )
// storeDataUsageInBackend will store all objects sent on the gui channel until closed. // storeDataUsageInBackend will store all objects sent on the gui channel until closed.
func storeDataUsageInBackend(ctx context.Context, objAPI ObjectLayer, dui <-chan DataUsageInfo) { func storeDataUsageInBackend(ctx context.Context, objAPI ObjectLayer, dui <-chan madmin.DataUsageInfo) {
for dataUsageInfo := range dui { for dataUsageInfo := range dui {
dataUsageJSON, err := json.Marshal(dataUsageInfo) dataUsageJSON, err := json.Marshal(dataUsageInfo)
if err != nil { if err != nil {
@ -59,27 +58,27 @@ func storeDataUsageInBackend(ctx context.Context, objAPI ObjectLayer, dui <-chan
} }
} }
func loadDataUsageFromBackend(ctx context.Context, objAPI ObjectLayer) (DataUsageInfo, error) { func loadDataUsageFromBackend(ctx context.Context, objAPI ObjectLayer) (madmin.DataUsageInfo, error) {
r, err := objAPI.GetObjectNInfo(ctx, dataUsageBucket, dataUsageObjName, nil, http.Header{}, readLock, ObjectOptions{}) r, err := objAPI.GetObjectNInfo(ctx, dataUsageBucket, dataUsageObjName, nil, http.Header{}, readLock, ObjectOptions{})
if err != nil { if err != nil {
if isErrObjectNotFound(err) || isErrBucketNotFound(err) { if isErrObjectNotFound(err) || isErrBucketNotFound(err) {
return DataUsageInfo{}, nil return madmin.DataUsageInfo{}, nil
} }
return DataUsageInfo{}, toObjectErr(err, dataUsageBucket, dataUsageObjName) return madmin.DataUsageInfo{}, toObjectErr(err, dataUsageBucket, dataUsageObjName)
} }
defer r.Close() defer r.Close()
var dataUsageInfo DataUsageInfo var dataUsageInfo madmin.DataUsageInfo
var json = jsoniter.ConfigCompatibleWithStandardLibrary var json = jsoniter.ConfigCompatibleWithStandardLibrary
if err = json.NewDecoder(r).Decode(&dataUsageInfo); err != nil { if err = json.NewDecoder(r).Decode(&dataUsageInfo); err != nil {
return DataUsageInfo{}, err return madmin.DataUsageInfo{}, err
} }
// For forward compatibility reasons, we need to add this code. // For forward compatibility reasons, we need to add this code.
if len(dataUsageInfo.BucketsUsage) == 0 { if len(dataUsageInfo.BucketsUsage) == 0 {
dataUsageInfo.BucketsUsage = make(map[string]BucketUsageInfo, len(dataUsageInfo.BucketSizes)) dataUsageInfo.BucketsUsage = make(map[string]madmin.BucketUsageInfo, len(dataUsageInfo.BucketSizes))
for bucket, size := range dataUsageInfo.BucketSizes { for bucket, size := range dataUsageInfo.BucketSizes {
dataUsageInfo.BucketsUsage[bucket] = BucketUsageInfo{Size: size} dataUsageInfo.BucketsUsage[bucket] = madmin.BucketUsageInfo{Size: size}
} }
} }

View File

@ -135,7 +135,8 @@ func readAllFileInfo(ctx context.Context, disks []StorageAPI, bucket, object, ve
errFileVersionNotFound, errFileVersionNotFound,
errDiskNotFound, errDiskNotFound,
}...) { }...) {
logger.LogOnceIf(ctx, fmt.Errorf("Drive %s returned an error (%w)", disks[index], err), logger.LogOnceIf(ctx, fmt.Errorf("Drive %s, path (%s/%s) returned an error (%w)",
disks[index], bucket, object, err),
disks[index].String()) disks[index].String())
} }
} }

View File

@ -322,8 +322,12 @@ func writeUniqueFileInfo(ctx context.Context, disks []StorageAPI, bucket, prefix
return errDiskNotFound return errDiskNotFound
} }
// Pick one FileInfo for a disk at index. // Pick one FileInfo for a disk at index.
files[index].Erasure.Index = index + 1 fi := files[index]
return disks[index].WriteMetadata(ctx, bucket, prefix, files[index]) fi.Erasure.Index = index + 1
if fi.IsValid() {
return disks[index].WriteMetadata(ctx, bucket, prefix, fi)
}
return errCorruptedFormat
}, index) }, index)
} }

View File

@ -72,7 +72,7 @@ func (er erasureObjects) CopyObject(ctx context.Context, srcBucket, srcObject, d
// Read metadata associated with the object from all disks. // Read metadata associated with the object from all disks.
storageDisks := er.getDisks() storageDisks := er.getDisks()
metaArr, errs := readAllFileInfo(ctx, storageDisks, srcBucket, srcObject, srcOpts.VersionID, false) metaArr, errs := readAllFileInfo(ctx, storageDisks, srcBucket, srcObject, srcOpts.VersionID, true)
// get Quorum for this object // get Quorum for this object
readQuorum, writeQuorum, err := objectQuorumFromMeta(ctx, metaArr, errs, er.defaultParityCount) readQuorum, writeQuorum, err := objectQuorumFromMeta(ctx, metaArr, errs, er.defaultParityCount)
@ -1215,7 +1215,7 @@ func (er erasureObjects) PutObjectTags(ctx context.Context, bucket, object strin
disks := er.getDisks() disks := er.getDisks()
// Read metadata associated with the object from all disks. // Read metadata associated with the object from all disks.
metaArr, errs := readAllFileInfo(ctx, disks, bucket, object, opts.VersionID, false) metaArr, errs := readAllFileInfo(ctx, disks, bucket, object, opts.VersionID, true)
readQuorum, writeQuorum, err := objectQuorumFromMeta(ctx, metaArr, errs, er.defaultParityCount) readQuorum, writeQuorum, err := objectQuorumFromMeta(ctx, metaArr, errs, er.defaultParityCount)
if err != nil { if err != nil {
@ -1289,7 +1289,7 @@ func (er erasureObjects) updateObjectMeta(ctx context.Context, bucket, object st
disks := er.getDisks() disks := er.getDisks()
// Read metadata associated with the object from all disks. // Read metadata associated with the object from all disks.
metaArr, errs := readAllFileInfo(ctx, disks, bucket, object, opts.VersionID, false) metaArr, errs := readAllFileInfo(ctx, disks, bucket, object, opts.VersionID, true)
readQuorum, writeQuorum, err := objectQuorumFromMeta(ctx, metaArr, errs, er.defaultParityCount) readQuorum, writeQuorum, err := objectQuorumFromMeta(ctx, metaArr, errs, er.defaultParityCount)
if err != nil { if err != nil {

View File

@ -433,7 +433,7 @@ func (z *erasureServerPools) StorageInfo(ctx context.Context) (StorageInfo, []er
return storageInfo, errs return storageInfo, errs
} }
func (z *erasureServerPools) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- DataUsageInfo) error { func (z *erasureServerPools) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- madmin.DataUsageInfo) error {
ctx, cancel := context.WithCancel(ctx) ctx, cancel := context.WithCancel(ctx)
defer cancel() defer cancel()
@ -448,7 +448,7 @@ func (z *erasureServerPools) NSScanner(ctx context.Context, bf *bloomFilter, upd
} }
if len(allBuckets) == 0 { if len(allBuckets) == 0 {
updates <- DataUsageInfo{} // no buckets found update data usage to reflect latest state updates <- madmin.DataUsageInfo{} // no buckets found update data usage to reflect latest state
return nil return nil
} }

View File

@ -237,7 +237,7 @@ func (fs *FSObjects) StorageInfo(ctx context.Context) (StorageInfo, []error) {
} }
// NSScanner returns data usage stats of the current FS deployment // NSScanner returns data usage stats of the current FS deployment
func (fs *FSObjects) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- DataUsageInfo) error { func (fs *FSObjects) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- madmin.DataUsageInfo) error {
// Load bucket totals // Load bucket totals
var totalCache dataUsageCache var totalCache dataUsageCache
err := totalCache.load(ctx, fs, dataUsageCacheName) err := totalCache.load(ctx, fs, dataUsageCacheName)

View File

@ -47,7 +47,7 @@ func (a GatewayUnsupported) LocalStorageInfo(ctx context.Context) (StorageInfo,
} }
// NSScanner - scanner is not implemented for gateway // NSScanner - scanner is not implemented for gateway
func (a GatewayUnsupported) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- DataUsageInfo) error { func (a GatewayUnsupported) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- madmin.DataUsageInfo) error {
logger.CriticalIf(ctx, errors.New("not implemented")) logger.CriticalIf(ctx, errors.New("not implemented"))
return NotImplemented{} return NotImplemented{}
} }

View File

@ -44,7 +44,7 @@ const (
healMetricNamespace MetricNamespace = "minio_heal" healMetricNamespace MetricNamespace = "minio_heal"
interNodeMetricNamespace MetricNamespace = "minio_inter_node" interNodeMetricNamespace MetricNamespace = "minio_inter_node"
nodeMetricNamespace MetricNamespace = "minio_node" nodeMetricNamespace MetricNamespace = "minio_node"
minIOMetricNamespace MetricNamespace = "minio" minioMetricNamespace MetricNamespace = "minio"
s3MetricNamespace MetricNamespace = "minio_s3" s3MetricNamespace MetricNamespace = "minio_s3"
) )
@ -93,9 +93,11 @@ const (
writeTotal MetricName = "write_total" writeTotal MetricName = "write_total"
total MetricName = "total" total MetricName = "total"
failedCount MetricName = "failed_count"
failedBytes MetricName = "failed_bytes" failedBytes MetricName = "failed_bytes"
freeBytes MetricName = "free_bytes" freeBytes MetricName = "free_bytes"
pendingBytes MetricName = "pending_bytes" pendingBytes MetricName = "pending_bytes"
pendingCount MetricName = "pending_count"
readBytes MetricName = "read_bytes" readBytes MetricName = "read_bytes"
rcharBytes MetricName = "rchar_bytes" rcharBytes MetricName = "rchar_bytes"
receivedBytes MetricName = "received_bytes" receivedBytes MetricName = "received_bytes"
@ -356,6 +358,16 @@ func getNodeDiskTotalBytesMD() MetricDescription {
Type: gaugeMetric, Type: gaugeMetric,
} }
} }
func getUsageLastScanActivityMD() MetricDescription {
return MetricDescription{
Namespace: minioMetricNamespace,
Subsystem: usageSubsystem,
Name: lastActivityTime,
Help: "Time elapsed (in nano seconds) since last scan activity. This is set to 0 until first scan cycle",
Type: gaugeMetric,
}
}
func getBucketUsageTotalBytesMD() MetricDescription { func getBucketUsageTotalBytesMD() MetricDescription {
return MetricDescription{ return MetricDescription{
Namespace: bucketMetricNamespace, Namespace: bucketMetricNamespace,
@ -410,6 +422,24 @@ func getBucketRepReceivedBytesMD() MetricDescription {
Type: gaugeMetric, Type: gaugeMetric,
} }
} }
func getBucketRepPendingOperationsMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
Subsystem: replicationSubsystem,
Name: pendingCount,
Help: "Total number of objects pending replication",
Type: gaugeMetric,
}
}
func getBucketRepFailedOperationsMD() MetricDescription {
return MetricDescription{
Namespace: bucketMetricNamespace,
Subsystem: replicationSubsystem,
Name: failedCount,
Help: "Total number of objects which failed replication",
Type: gaugeMetric,
}
}
func getBucketObjectDistributionMD() MetricDescription { func getBucketObjectDistributionMD() MetricDescription {
return MetricDescription{ return MetricDescription{
Namespace: bucketMetricNamespace, Namespace: bucketMetricNamespace,
@ -666,7 +696,7 @@ func getNodeOfflineTotalMD() MetricDescription {
} }
func getMinIOVersionMD() MetricDescription { func getMinIOVersionMD() MetricDescription {
return MetricDescription{ return MetricDescription{
Namespace: minIOMetricNamespace, Namespace: minioMetricNamespace,
Subsystem: softwareSubsystem, Subsystem: softwareSubsystem,
Name: versionInfo, Name: versionInfo,
Help: "MinIO Release tag for the server", Help: "MinIO Release tag for the server",
@ -675,7 +705,7 @@ func getMinIOVersionMD() MetricDescription {
} }
func getMinIOCommitMD() MetricDescription { func getMinIOCommitMD() MetricDescription {
return MetricDescription{ return MetricDescription{
Namespace: minIOMetricNamespace, Namespace: minioMetricNamespace,
Subsystem: softwareSubsystem, Subsystem: softwareSubsystem,
Name: commitInfo, Name: commitInfo,
Help: "Git commit hash for the MinIO release.", Help: "Git commit hash for the MinIO release.",
@ -996,13 +1026,14 @@ func getMinioHealingMetrics() MetricsGroup {
if !exists { if !exists {
return return
} }
var dur time.Duration
if !bgSeq.lastHealActivity.IsZero() { if bgSeq.lastHealActivity.IsZero() {
dur = time.Since(bgSeq.lastHealActivity) return
} }
metrics = append(metrics, Metric{ metrics = append(metrics, Metric{
Description: getHealLastActivityTimeMD(), Description: getHealLastActivityTimeMD(),
Value: float64(dur), Value: float64(time.Since(bgSeq.lastHealActivity)),
}) })
metrics = append(metrics, getObjectsScanned(bgSeq)...) metrics = append(metrics, getObjectsScanned(bgSeq)...)
metrics = append(metrics, getScannedItems(bgSeq)...) metrics = append(metrics, getScannedItems(bgSeq)...)
@ -1224,7 +1255,14 @@ func getBucketUsageMetrics() MetricsGroup {
return return
} }
metrics = append(metrics, Metric{
Description: getUsageLastScanActivityMD(),
Value: float64(time.Since(dataUsageInfo.LastUpdate)),
})
for bucket, usage := range dataUsageInfo.BucketsUsage { for bucket, usage := range dataUsageInfo.BucketsUsage {
stat := getLatestReplicationStats(bucket, usage)
metrics = append(metrics, Metric{ metrics = append(metrics, Metric{
Description: getBucketUsageTotalBytesMD(), Description: getBucketUsageTotalBytesMD(),
Value: float64(usage.Size), Value: float64(usage.Size),
@ -1237,25 +1275,35 @@ func getBucketUsageMetrics() MetricsGroup {
VariableLabels: map[string]string{"bucket": bucket}, VariableLabels: map[string]string{"bucket": bucket},
}) })
if usage.hasReplicationUsage() { if stat.hasReplicationUsage() {
metrics = append(metrics, Metric{ metrics = append(metrics, Metric{
Description: getBucketRepPendingBytesMD(), Description: getBucketRepPendingBytesMD(),
Value: float64(usage.ReplicationPendingSize), Value: float64(stat.PendingSize),
VariableLabels: map[string]string{"bucket": bucket}, VariableLabels: map[string]string{"bucket": bucket},
}) })
metrics = append(metrics, Metric{ metrics = append(metrics, Metric{
Description: getBucketRepFailedBytesMD(), Description: getBucketRepFailedBytesMD(),
Value: float64(usage.ReplicationFailedSize), Value: float64(stat.FailedSize),
VariableLabels: map[string]string{"bucket": bucket}, VariableLabels: map[string]string{"bucket": bucket},
}) })
metrics = append(metrics, Metric{ metrics = append(metrics, Metric{
Description: getBucketRepSentBytesMD(), Description: getBucketRepSentBytesMD(),
Value: float64(usage.ReplicatedSize), Value: float64(stat.ReplicatedSize),
VariableLabels: map[string]string{"bucket": bucket}, VariableLabels: map[string]string{"bucket": bucket},
}) })
metrics = append(metrics, Metric{ metrics = append(metrics, Metric{
Description: getBucketRepReceivedBytesMD(), Description: getBucketRepReceivedBytesMD(),
Value: float64(usage.ReplicaSize), Value: float64(stat.ReplicaSize),
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
Description: getBucketRepPendingOperationsMD(),
Value: float64(stat.PendingCount),
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
Description: getBucketRepFailedOperationsMD(),
Value: float64(stat.FailedCount),
VariableLabels: map[string]string{"bucket": bucket}, VariableLabels: map[string]string{"bucket": bucket},
}) })
} }
@ -1372,13 +1420,6 @@ func getClusterStorageMetrics() MetricsGroup {
} }
} }
func (b *BucketUsageInfo) hasReplicationUsage() bool {
return b.ReplicationPendingSize > 0 ||
b.ReplicationFailedSize > 0 ||
b.ReplicatedSize > 0 ||
b.ReplicaSize > 0
}
type minioClusterCollector struct { type minioClusterCollector struct {
desc *prometheus.Desc desc *prometheus.Desc
} }

View File

@ -23,6 +23,7 @@ import (
"time" "time"
"github.com/minio/minio/cmd/logger" "github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/madmin"
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promhttp" "github.com/prometheus/client_golang/prometheus/promhttp"
) )
@ -430,6 +431,39 @@ func networkMetricsPrometheus(ch chan<- prometheus.Metric) {
) )
} }
// get the most current of in-memory replication stats and data usage info from crawler.
func getLatestReplicationStats(bucket string, u madmin.BucketUsageInfo) BucketReplicationStats {
s := BucketReplicationStats{
PendingSize: u.ReplicationPendingSize,
FailedSize: u.ReplicationFailedSize,
ReplicatedSize: u.ReplicatedSize,
ReplicaSize: u.ReplicaSize,
PendingCount: u.ReplicationPendingCount,
FailedCount: u.ReplicationFailedCount,
}
rStat := globalReplicationStats.Get(bucket)
// use in memory replication stats if it is ahead of usage info.
if rStat.ReplicatedSize > u.ReplicatedSize {
s.ReplicatedSize = rStat.ReplicatedSize
}
if rStat.PendingSize > u.ReplicationPendingSize {
s.PendingSize = rStat.PendingSize
}
if rStat.FailedSize > u.ReplicationFailedSize {
s.FailedSize = rStat.FailedSize
}
if rStat.ReplicaSize > u.ReplicaSize {
s.ReplicaSize = rStat.ReplicaSize
}
if rStat.PendingCount > u.ReplicationPendingCount {
s.PendingCount = rStat.PendingCount
}
if rStat.FailedCount > u.ReplicationFailedCount {
s.FailedCount = rStat.FailedCount
}
return s
}
// Populates prometheus with bucket usage metrics, this metrics // Populates prometheus with bucket usage metrics, this metrics
// is only enabled if scanner is enabled. // is only enabled if scanner is enabled.
func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) { func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
@ -447,13 +481,13 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
if err != nil { if err != nil {
return return
} }
// data usage has not captured any data yet. // data usage has not captured any data yet.
if dataUsageInfo.LastUpdate.IsZero() { if dataUsageInfo.LastUpdate.IsZero() {
return return
} }
for bucket, usageInfo := range dataUsageInfo.BucketsUsage { for bucket, usageInfo := range dataUsageInfo.BucketsUsage {
stat := getLatestReplicationStats(bucket, usageInfo)
// Total space used by bucket // Total space used by bucket
ch <- prometheus.MustNewConstMetric( ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc( prometheus.NewDesc(
@ -479,7 +513,7 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity pending to be replicated", "Total capacity pending to be replicated",
[]string{"bucket"}, nil), []string{"bucket"}, nil),
prometheus.GaugeValue, prometheus.GaugeValue,
float64(usageInfo.ReplicationPendingSize), float64(stat.PendingSize),
bucket, bucket,
) )
ch <- prometheus.MustNewConstMetric( ch <- prometheus.MustNewConstMetric(
@ -488,7 +522,7 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity failed to replicate at least once", "Total capacity failed to replicate at least once",
[]string{"bucket"}, nil), []string{"bucket"}, nil),
prometheus.GaugeValue, prometheus.GaugeValue,
float64(usageInfo.ReplicationFailedSize), float64(stat.FailedSize),
bucket, bucket,
) )
ch <- prometheus.MustNewConstMetric( ch <- prometheus.MustNewConstMetric(
@ -497,7 +531,7 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity replicated to destination", "Total capacity replicated to destination",
[]string{"bucket"}, nil), []string{"bucket"}, nil),
prometheus.GaugeValue, prometheus.GaugeValue,
float64(usageInfo.ReplicatedSize), float64(stat.ReplicatedSize),
bucket, bucket,
) )
ch <- prometheus.MustNewConstMetric( ch <- prometheus.MustNewConstMetric(
@ -506,7 +540,25 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity replicated to this instance", "Total capacity replicated to this instance",
[]string{"bucket"}, nil), []string{"bucket"}, nil),
prometheus.GaugeValue, prometheus.GaugeValue,
float64(usageInfo.ReplicaSize), float64(stat.ReplicaSize),
bucket,
)
ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc(
prometheus.BuildFQName("bucket", "replication", "pending_count"),
"Total replication operations pending",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(stat.PendingCount),
bucket,
)
ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc(
prometheus.BuildFQName("bucket", "replication", "failed_count"),
"Total replication operations failed",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(stat.FailedCount),
bucket, bucket,
) )
for k, v := range usageInfo.ObjectSizesHistogram { for k, v := range usageInfo.ObjectSizesHistogram {

View File

@ -69,57 +69,6 @@ var ObjectsHistogramIntervals = []objectHistogramInterval{
{"GREATER_THAN_512_MB", humanize.MiByte * 512, math.MaxInt64}, {"GREATER_THAN_512_MB", humanize.MiByte * 512, math.MaxInt64},
} }
// BucketUsageInfo - bucket usage info provides
// - total size of the bucket
// - total objects in a bucket
// - object size histogram per bucket
type BucketUsageInfo struct {
Size uint64 `json:"size"`
ReplicationPendingSize uint64 `json:"objectsPendingReplicationTotalSize"`
ReplicationFailedSize uint64 `json:"objectsFailedReplicationTotalSize"`
ReplicatedSize uint64 `json:"objectsReplicatedTotalSize"`
ReplicaSize uint64 `json:"objectReplicaTotalSize"`
ObjectsCount uint64 `json:"objectsCount"`
ObjectSizesHistogram map[string]uint64 `json:"objectsSizesHistogram"`
}
// DataUsageInfo represents data usage stats of the underlying Object API
type DataUsageInfo struct {
// LastUpdate is the timestamp of when the data usage info was last updated.
// This does not indicate a full scan.
LastUpdate time.Time `json:"lastUpdate"`
// Objects total count across all buckets
ObjectsTotalCount uint64 `json:"objectsCount"`
// Objects total size across all buckets
ObjectsTotalSize uint64 `json:"objectsTotalSize"`
// Total Size for objects that have not yet been replicated
ReplicationPendingSize uint64 `json:"objectsPendingReplicationTotalSize"`
// Total size for objects that have witness one or more failures and will be retried
ReplicationFailedSize uint64 `json:"objectsFailedReplicationTotalSize"`
// Total size for objects that have been replicated to destination
ReplicatedSize uint64 `json:"objectsReplicatedTotalSize"`
// Total size for objects that are replicas
ReplicaSize uint64 `json:"objectsReplicaTotalSize"`
// Total number of buckets in this cluster
BucketsCount uint64 `json:"bucketsCount"`
// Buckets usage info provides following information across all buckets
// - total size of the bucket
// - total objects in a bucket
// - object size histogram per bucket
BucketsUsage map[string]BucketUsageInfo `json:"bucketsUsageInfo"`
// Deprecated kept here for backward compatibility reasons.
BucketSizes map[string]uint64 `json:"bucketsSizes"`
}
// BucketInfo - represents bucket metadata. // BucketInfo - represents bucket metadata.
type BucketInfo struct { type BucketInfo struct {
// Name of the bucket. // Name of the bucket.

View File

@ -91,7 +91,7 @@ type ObjectLayer interface {
// Storage operations. // Storage operations.
Shutdown(context.Context) error Shutdown(context.Context) error
NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- DataUsageInfo) error NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- madmin.DataUsageInfo) error
BackendInfo() madmin.BackendInfo BackendInfo() madmin.BackendInfo
StorageInfo(ctx context.Context) (StorageInfo, []error) StorageInfo(ctx context.Context) (StorageInfo, []error)

View File

@ -1341,7 +1341,7 @@ func (api objectAPIHandlers) CopyObjectHandler(w http.ResponseWriter, r *http.Re
response := generateCopyObjectResponse(objInfo.ETag, objInfo.ModTime) response := generateCopyObjectResponse(objInfo.ETag, objInfo.ModTime)
encodedSuccessResponse := encodeResponse(response) encodedSuccessResponse := encodeResponse(response)
if replicate, sync := mustReplicate(ctx, r, dstBucket, dstObject, objInfo.UserDefined, objInfo.ReplicationStatus.String()); replicate { if replicate, sync := mustReplicate(ctx, r, dstBucket, dstObject, objInfo.UserDefined, objInfo.ReplicationStatus.String()); replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync) scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
} }
setPutObjHeaders(w, objInfo, false) setPutObjHeaders(w, objInfo, false)
@ -1656,7 +1656,7 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req
} }
} }
if replicate, sync := mustReplicate(ctx, r, bucket, object, metadata, ""); replicate { if replicate, sync := mustReplicate(ctx, r, bucket, object, metadata, ""); replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync) scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
} }
setPutObjHeaders(w, objInfo, false) setPutObjHeaders(w, objInfo, false)
@ -1938,7 +1938,7 @@ func (api objectAPIHandlers) PutObjectExtractHandler(w http.ResponseWriter, r *h
} }
if replicate, sync := mustReplicate(ctx, r, bucket, object, metadata, ""); replicate { if replicate, sync := mustReplicate(ctx, r, bucket, object, metadata, ""); replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync) scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
} }
} }
@ -3014,7 +3014,7 @@ func (api objectAPIHandlers) CompleteMultipartUploadHandler(w http.ResponseWrite
setPutObjHeaders(w, objInfo, false) setPutObjHeaders(w, objInfo, false)
if replicate, sync := mustReplicate(ctx, r, bucket, object, objInfo.UserDefined, objInfo.ReplicationStatus.String()); replicate { if replicate, sync := mustReplicate(ctx, r, bucket, object, objInfo.UserDefined, objInfo.ReplicationStatus.String()); replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync) scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
} }
// Write success response. // Write success response.
@ -3294,7 +3294,7 @@ func (api objectAPIHandlers) PutObjectLegalHoldHandler(w http.ResponseWriter, r
return return
} }
if replicate { if replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync) scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.MetadataReplicationType)
} }
writeSuccessResponseHeadersOnly(w) writeSuccessResponseHeadersOnly(w)
@ -3467,7 +3467,7 @@ func (api objectAPIHandlers) PutObjectRetentionHandler(w http.ResponseWriter, r
return return
} }
if replicate { if replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync) scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.MetadataReplicationType)
} }
writeSuccessNoContent(w) writeSuccessNoContent(w)
@ -3650,7 +3650,7 @@ func (api objectAPIHandlers) PutObjectTaggingHandler(w http.ResponseWriter, r *h
} }
if replicate { if replicate {
scheduleReplication(ctx, objInfo.Clone(), objAPI, sync) scheduleReplication(ctx, objInfo.Clone(), objAPI, sync, replication.MetadataReplicationType)
} }
if objInfo.VersionID != "" { if objInfo.VersionID != "" {
@ -3724,7 +3724,7 @@ func (api objectAPIHandlers) DeleteObjectTaggingHandler(w http.ResponseWriter, r
} }
if replicate { if replicate {
scheduleReplication(ctx, oi.Clone(), objAPI, sync) scheduleReplication(ctx, oi.Clone(), objAPI, sync, replication.MetadataReplicationType)
} }
if oi.VersionID != "" { if oi.VersionID != "" {

View File

@ -1336,7 +1336,7 @@ func (web *webAPIHandlers) Upload(w http.ResponseWriter, r *http.Request) {
} }
} }
if mustReplicate { if mustReplicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync) scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
} }
reqParams := extractReqParams(r) reqParams := extractReqParams(r)

View File

@ -231,7 +231,7 @@ type xlMetaV2 struct {
Versions []xlMetaV2Version `json:"Versions" msg:"Versions"` Versions []xlMetaV2Version `json:"Versions" msg:"Versions"`
// data will contain raw data if any. // data will contain raw data if any.
// data will be one or more versions indexed by storage dir. // data will be one or more versions indexed by versionID.
// To remove all data set to nil. // To remove all data set to nil.
data xlMetaInlineData `msg:"-"` data xlMetaInlineData `msg:"-"`
} }
@ -295,28 +295,31 @@ func (x xlMetaInlineData) validate() error {
if len(x) == 0 { if len(x) == 0 {
return nil return nil
} }
if !x.versionOK() { if !x.versionOK() {
return fmt.Errorf("xlMetaInlineData: unknown version 0x%x", x[0]) return fmt.Errorf("xlMetaInlineData: unknown version 0x%x", x[0])
} }
sz, buf, err := msgp.ReadMapHeaderBytes(x.afterVersion()) sz, buf, err := msgp.ReadMapHeaderBytes(x.afterVersion())
if err != nil { if err != nil {
return err return fmt.Errorf("xlMetaInlineData: %w", err)
} }
for i := uint32(0); i < sz; i++ { for i := uint32(0); i < sz; i++ {
var key []byte var key []byte
key, buf, err = msgp.ReadMapKeyZC(buf) key, buf, err = msgp.ReadMapKeyZC(buf)
if err != nil { if err != nil {
return err return fmt.Errorf("xlMetaInlineData: %w", err)
} }
if len(key) == 0 { if len(key) == 0 {
return fmt.Errorf("xlMetaInlineData: key %d is length 0", i) return fmt.Errorf("xlMetaInlineData: key %d is length 0", i)
} }
_, buf, err = msgp.ReadBytesZC(buf) _, buf, err = msgp.ReadBytesZC(buf)
if err != nil { if err != nil {
return err return fmt.Errorf("xlMetaInlineData: %w", err)
} }
} }
return nil return nil
} }
@ -564,31 +567,27 @@ func (z *xlMetaV2) AddLegacy(m *xlMetaV1Object) error {
func (z *xlMetaV2) Load(buf []byte) error { func (z *xlMetaV2) Load(buf []byte) error {
buf, _, minor, err := checkXL2V1(buf) buf, _, minor, err := checkXL2V1(buf)
if err != nil { if err != nil {
return errFileCorrupt return fmt.Errorf("z.Load %w", err)
} }
switch minor { switch minor {
case 0: case 0:
_, err = z.UnmarshalMsg(buf) _, err = z.UnmarshalMsg(buf)
if err != nil { if err != nil {
return errFileCorrupt return fmt.Errorf("z.Load %w", err)
} }
return nil return nil
case 1: case 1:
v, buf, err := msgp.ReadBytesZC(buf) v, buf, err := msgp.ReadBytesZC(buf)
if err != nil { if err != nil {
return errFileCorrupt return fmt.Errorf("z.Load version(%d), bufLen(%d) %w", minor, len(buf), err)
} }
_, err = z.UnmarshalMsg(v) if _, err = z.UnmarshalMsg(v); err != nil {
if err != nil { return fmt.Errorf("z.Load version(%d), vLen(%d), %w", minor, len(v), err)
return errFileCorrupt
} }
// Add remaining data. // Add remaining data.
z.data = nil
if len(buf) > 0 {
z.data = buf z.data = buf
if err := z.data.validate(); err != nil { if err = z.data.validate(); err != nil {
return errFileCorrupt return fmt.Errorf("z.Load version(%d), bufLen(%d) %w", minor, len(buf), err)
}
} }
default: default:
return errors.New("unknown metadata version") return errors.New("unknown metadata version")

View File

@ -911,29 +911,28 @@ func (s *xlStorage) WriteMetadata(ctx context.Context, volume, path string, fi F
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
return err return err
} }
buf, err = xlMeta.AppendTo(nil) buf, err = xlMeta.AppendTo(nil)
if err != nil { if err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
return err return err
} }
if err := xlMeta.Load(buf); err != nil {
panic(err)
}
} else { } else {
if err = xlMeta.Load(buf); err != nil { if err = xlMeta.Load(buf); err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
return err return err
} }
if err = xlMeta.AddVersion(fi); err != nil { if err = xlMeta.AddVersion(fi); err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
return err return err
} }
buf, err = xlMeta.AppendTo(nil) buf, err = xlMeta.AppendTo(nil)
if err != nil { if err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
return err return err
} }
} }
return s.WriteAll(ctx, volume, pathJoin(path, xlStorageFormatFile), buf) return s.WriteAll(ctx, volume, pathJoin(path, xlStorageFormatFile), buf)
@ -1042,11 +1041,11 @@ func (s *xlStorage) ReadVersion(ctx context.Context, volume, path, versionID str
if len(fi.Data) > 0 || fi.Size == 0 { if len(fi.Data) > 0 || fi.Size == 0 {
return fi, nil return fi, nil
} }
// Reading data for small objects when // Reading data for small objects when
// - object has not yet transitioned // - object has not yet transitioned
// - object size lesser than 32KiB // - object size lesser than 32KiB
// - object has maximum of 1 parts // - object has maximum of 1 parts
if fi.TransitionStatus == "" && fi.DataDir != "" && fi.Size <= smallFileThreshold && len(fi.Parts) == 1 { if fi.TransitionStatus == "" && fi.DataDir != "" && fi.Size <= smallFileThreshold && len(fi.Parts) == 1 {
// Enable O_DIRECT optionally only if drive supports it. // Enable O_DIRECT optionally only if drive supports it.
requireDirectIO := globalStorageClass.GetDMA() == storageclass.DMAReadWrite requireDirectIO := globalStorageClass.GetDMA() == storageclass.DMAReadWrite

View File

@ -12,6 +12,7 @@ These metrics can be from any MinIO server once per collection.
|`minio_bucket_replication_pending_bytes` |Total bytes pending to replicate. | |`minio_bucket_replication_pending_bytes` |Total bytes pending to replicate. |
|`minio_bucket_replication_received_bytes` |Total number of bytes replicated to this bucket from another source bucket. | |`minio_bucket_replication_received_bytes` |Total number of bytes replicated to this bucket from another source bucket. |
|`minio_bucket_replication_sent_bytes` |Total number of bytes replicated to the target bucket. | |`minio_bucket_replication_sent_bytes` |Total number of bytes replicated to the target bucket. |
|`minio_bucket_replication_pending_count` |Total number of replication operations pending for this bucket. |
|`minio_bucket_usage_object_total` |Total number of objects | |`minio_bucket_usage_object_total` |Total number of objects |
|`minio_bucket_usage_total_bytes` |Total bucket size in bytes | |`minio_bucket_usage_total_bytes` |Total bucket size in bytes |
|`minio_cache_hits_total` |Total number of disk cache hits | |`minio_cache_hits_total` |Total number of disk cache hits |

View File

@ -258,15 +258,15 @@ Fetches accounting usage information for the current authenticated user
| Param | Type | Description | | Param | Type | Description |
|--------------------------------|----------------------|-------------------------| |--------------------------------|----------------------|-------------------------|
| `AccountInfo.AccountName` | _string_ | Account name. | | `AccountInfo.AccountName` | _string_ | Account name. |
| `AccountInfo.Buckets` | _[]BucketUsageInfo_ | Bucket usage info. | | `AccountInfo.Buckets` | _[]BucketAccessInfo_ | Bucket usage info. |
| Param | Type | Description | | Param | Type | Description |
|----------------------------|-----------------|-----------------------------------------| |----------------------------|-----------------|-----------------------------------------|
| `BucketUsageInfo.Name` | _string_ | The name of the current bucket | `BucketAccessInfo.Name` | _string_ | The name of the current bucket
| `BucketUsageInfo.Size` | _uint64_ | The total size of the current bucket | `BucketAccessInfo.Size` | _uint64_ | The total size of the current bucket
| `BucketUsageInfo.Created` | _time.Time_ | Bucket creation time | `BucketAccessInfo.Created` | _time.Time_ | Bucket creation time
| `BucketUsageInfo.Access` | _AccountAccess_ | Type of access of the current account | `BucketAccessInfo.Access` | _AccountAccess_ | Type of access of the current account
| Param | Type | Description | | Param | Type | Description |

View File

@ -20,7 +20,6 @@ package madmin
import ( import (
"context" "context"
"encoding/json" "encoding/json"
"io/ioutil"
"net/http" "net/http"
"time" "time"
) )
@ -124,36 +123,71 @@ func (adm *AdminClient) StorageInfo(ctx context.Context) (StorageInfo, error) {
// Unmarshal the server's json response // Unmarshal the server's json response
var storageInfo StorageInfo var storageInfo StorageInfo
if err = json.NewDecoder(resp.Body).Decode(&storageInfo); err != nil {
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return StorageInfo{}, err
}
err = json.Unmarshal(respBytes, &storageInfo)
if err != nil {
return StorageInfo{}, err return StorageInfo{}, err
} }
return storageInfo, nil return storageInfo, nil
} }
// DataUsageInfo represents data usage of an Object API // BucketUsageInfo - bucket usage info provides
// - total size of the bucket
// - total objects in a bucket
// - object size histogram per bucket
type BucketUsageInfo struct {
Size uint64 `json:"size"`
ReplicationPendingSize uint64 `json:"objectsPendingReplicationTotalSize"`
ReplicationFailedSize uint64 `json:"objectsFailedReplicationTotalSize"`
ReplicatedSize uint64 `json:"objectsReplicatedTotalSize"`
ReplicaSize uint64 `json:"objectReplicaTotalSize"`
ReplicationPendingCount uint64 `json:"objectsPendingReplicationCount"`
ReplicationFailedCount uint64 `json:"objectsFailedReplicationCount"`
ObjectsCount uint64 `json:"objectsCount"`
ObjectSizesHistogram map[string]uint64 `json:"objectsSizesHistogram"`
}
// DataUsageInfo represents data usage stats of the underlying Object API
type DataUsageInfo struct { type DataUsageInfo struct {
// LastUpdate is the timestamp of when the data usage info was last updated. // LastUpdate is the timestamp of when the data usage info was last updated.
// This does not indicate a full scan. // This does not indicate a full scan.
LastUpdate time.Time `json:"lastUpdate"` LastUpdate time.Time `json:"lastUpdate"`
ObjectsCount uint64 `json:"objectsCount"`
// Objects total count across all buckets
ObjectsTotalCount uint64 `json:"objectsCount"`
// Objects total size across all buckets
ObjectsTotalSize uint64 `json:"objectsTotalSize"` ObjectsTotalSize uint64 `json:"objectsTotalSize"`
// ObjectsSizesHistogram contains information on objects across all buckets. // Total Size for objects that have not yet been replicated
// See ObjectsHistogramIntervals. ReplicationPendingSize uint64 `json:"objectsPendingReplicationTotalSize"`
ObjectsSizesHistogram map[string]uint64 `json:"objectsSizesHistogram"`
// Total size for objects that have witness one or more failures and will be retried
ReplicationFailedSize uint64 `json:"objectsFailedReplicationTotalSize"`
// Total size for objects that have been replicated to destination
ReplicatedSize uint64 `json:"objectsReplicatedTotalSize"`
// Total size for objects that are replicas
ReplicaSize uint64 `json:"objectsReplicaTotalSize"`
// Total number of objects pending replication
ReplicationPendingCount uint64 `json:"objectsPendingReplicationCount"`
// Total number of objects that failed replication
ReplicationFailedCount uint64 `json:"objectsFailedReplicationCount"`
// Total number of buckets in this cluster
BucketsCount uint64 `json:"bucketsCount"` BucketsCount uint64 `json:"bucketsCount"`
// BucketsSizes is "bucket name" -> size. // Buckets usage info provides following information across all buckets
BucketsSizes map[string]uint64 `json:"bucketsSizes"` // - total size of the bucket
// - total objects in a bucket
// - object size histogram per bucket
BucketsUsage map[string]BucketUsageInfo `json:"bucketsUsageInfo"`
// Deprecated kept here for backward compatibility reasons.
BucketSizes map[string]uint64 `json:"bucketsSizes"`
} }
// DataUsageInfo - returns data usage of the current object API // DataUsageInfo - returns data usage of the current object API
@ -171,14 +205,7 @@ func (adm *AdminClient) DataUsageInfo(ctx context.Context) (DataUsageInfo, error
// Unmarshal the server's json response // Unmarshal the server's json response
var dataUsageInfo DataUsageInfo var dataUsageInfo DataUsageInfo
if err = json.NewDecoder(resp.Body).Decode(&dataUsageInfo); err != nil {
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return DataUsageInfo{}, err
}
err = json.Unmarshal(respBytes, &dataUsageInfo)
if err != nil {
return DataUsageInfo{}, err return DataUsageInfo{}, err
} }
@ -344,14 +371,7 @@ func (adm *AdminClient) ServerInfo(ctx context.Context) (InfoMessage, error) {
// Unmarshal the server's json response // Unmarshal the server's json response
var message InfoMessage var message InfoMessage
if err = json.NewDecoder(resp.Body).Decode(&message); err != nil {
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return InfoMessage{}, err
}
err = json.Unmarshal(respBytes, &message)
if err != nil {
return InfoMessage{}, err return InfoMessage{}, err
} }

View File

@ -20,7 +20,6 @@ package madmin
import ( import (
"context" "context"
"encoding/json" "encoding/json"
"io/ioutil"
"net/http" "net/http"
"net/url" "net/url"
) )
@ -53,10 +52,9 @@ func (adm *AdminClient) ServerUpdate(ctx context.Context, updateURL string) (us
return us, httpRespToErrorResponse(resp) return us, httpRespToErrorResponse(resp)
} }
buf, err := ioutil.ReadAll(resp.Body) if err = json.NewDecoder(resp.Body).Decode(&us); err != nil {
if err != nil {
return us, err return us, err
} }
err = json.Unmarshal(buf, &us)
return us, err return us, nil
} }

View File

@ -35,9 +35,9 @@ type AccountAccess struct {
Write bool `json:"write"` Write bool `json:"write"`
} }
// BucketUsageInfo represents bucket usage of a bucket, and its relevant // BucketAccessInfo represents bucket usage of a bucket, and its relevant
// access type for an account // access type for an account
type BucketUsageInfo struct { type BucketAccessInfo struct {
Name string `json:"name"` Name string `json:"name"`
Size uint64 `json:"size"` Size uint64 `json:"size"`
Created time.Time `json:"created"` Created time.Time `json:"created"`
@ -49,7 +49,7 @@ type BucketUsageInfo struct {
type AccountInfo struct { type AccountInfo struct {
AccountName string AccountName string
Policy iampolicy.Policy Policy iampolicy.Policy
Buckets []BucketUsageInfo Buckets []BucketAccessInfo
} }
// AccountInfo returns the usage info for the authenticating account. // AccountInfo returns the usage info for the authenticating account.