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

View File

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

View File

@ -311,7 +311,6 @@ func registerAPIRouter(router *mux.Router) {
// GetBucketReplicationConfig
router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("getbucketreplicationconfiguration", maxClients(httpTraceAll(api.GetBucketReplicationConfigHandler)))).Queries("replication", "")
// GetBucketVersioning
router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("getbucketversioning", maxClients(httpTraceAll(api.GetBucketVersioningHandler)))).Queries("versioning", "")
@ -378,8 +377,6 @@ func registerAPIRouter(router *mux.Router) {
// PutBucketReplicationConfig
router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("putbucketreplicationconfiguration", maxClients(httpTraceAll(api.PutBucketReplicationConfigHandler)))).Queries("replication", "")
// GetObjectRetention
// PutBucketEncryption
router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("putbucketencryption", maxClients(httpTraceAll(api.PutBucketEncryptionHandler)))).Queries("encryption", "")
@ -430,6 +427,12 @@ func registerAPIRouter(router *mux.Router) {
// ListObjectsV1 (Legacy)
router.Methods(http.MethodGet).HandlerFunc(
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

View File

@ -20,6 +20,7 @@ import (
"bytes"
"crypto/subtle"
"encoding/base64"
"encoding/json"
"encoding/xml"
"fmt"
"io"
@ -1243,7 +1244,7 @@ func (api objectAPIHandlers) DeleteBucketHandler(w http.ResponseWriter, r *http.
return
}
}
globalReplicationStats.Delete(ctx, bucket)
// Write success response.
writeSuccessNoContent(w)
@ -1603,3 +1604,40 @@ func (api objectAPIHandlers) DeleteBucketReplicationConfigHandler(w http.Respons
// Write success response.
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
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 {
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 {
return
}
outbuf := bytes.NewBuffer(nil)
objectKey := crypto.GenerateKey(key, rand.Reader)
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
}
keyID, kmsKey, sealedKey, err := crypto.S3.ParseMetadata(meta)
if err != nil {
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 {
return nil, err
}
outbuf := bytes.NewBuffer(nil)
_, err = sio.Decrypt(outbuf, bytes.NewBuffer(input), sio.Config{Key: objectKey[:], MinVersion: sio.Version20})
return outbuf.Bytes(), err
}

View File

@ -88,7 +88,7 @@ func (sys *BucketQuotaSys) check(ctx context.Context, bucket string, size int64)
return err
}
dui := v.(DataUsageInfo)
dui := v.(madmin.DataUsageInfo)
bui, ok := dui.BucketsUsage[bucket]
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
// 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
cfg, err := globalBucketQuotaSys.Get(bucket)
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
}
}
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
if replicationStatus == string(replication.Failed) || versionPurgeStatus == Failed {
@ -594,7 +601,7 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
})
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,
})
if err != nil {
@ -604,10 +611,10 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
Object: objInfo,
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
}
defer gr.Close() // hold read lock for entire transaction
defer gr.Close() // hold write lock for entire transaction
objInfo = gr.ObjInfo
size, err := objInfo.GetActualSize()
@ -644,7 +651,7 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
rtype = getReplicationAction(objInfo, oi)
if rtype == replicateNone {
// object with same VersionID already exists, replication kicked off by
// PutObject might have completed.
// PutObject might have completed
return
}
}
@ -656,7 +663,8 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
srcOpts := miniogo.CopySrcOptions{
Bucket: dest.Bucket,
Object: object,
VersionID: objInfo.VersionID}
VersionID: objInfo.VersionID,
}
dstOpts := miniogo.PutObjectOptions{
Internal: miniogo.AdvancedPutOptions{
SourceVersionID: objInfo.VersionID,
@ -712,6 +720,7 @@ func replicateObject(ctx context.Context, objInfo ObjectInfo, objectAPI ObjectLa
defer r.Close()
}
prevReplStatus := objInfo.ReplicationStatus
objInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replicationStatus.String()
if 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))
}
}
opType := replication.MetadataReplicationType
if rtype == replicateAll {
opType = replication.ObjectReplicationType
}
globalReplicationStats.Update(ctx, bucket, size, replicationStatus, prevReplStatus, opType)
sendEvent(eventArgs{
EventName: eventName,
BucketName: bucket,
@ -775,6 +789,7 @@ type DeletedObjectVersionInfo struct {
var (
globalReplicationPool *ReplicationPool
globalReplicationStats *ReplicationStats
)
// ReplicationPool describes replication pool
@ -783,6 +798,8 @@ type ReplicationPool struct {
size int
replicaCh chan ObjectInfo
replicaDeleteCh chan DeletedObjectVersionInfo
mrfReplicaCh chan ObjectInfo
mrfReplicaDeleteCh chan DeletedObjectVersionInfo
killCh chan struct{}
wg sync.WaitGroup
ctx context.Context
@ -792,20 +809,40 @@ type ReplicationPool struct {
// NewReplicationPool creates a pool of replication workers of specified size
func NewReplicationPool(ctx context.Context, o ObjectLayer, sz int) *ReplicationPool {
pool := &ReplicationPool{
replicaCh: make(chan ObjectInfo, 10000),
replicaDeleteCh: make(chan DeletedObjectVersionInfo, 10000),
replicaCh: make(chan ObjectInfo, 1000),
replicaDeleteCh: make(chan DeletedObjectVersionInfo, 1000),
mrfReplicaCh: make(chan ObjectInfo, 100000),
mrfReplicaDeleteCh: make(chan DeletedObjectVersionInfo, 100000),
ctx: ctx,
objLayer: o,
}
go func() {
<-ctx.Done()
close(pool.replicaCh)
close(pool.replicaDeleteCh)
}()
pool.Resize(sz)
// add long running worker for handling most recent failures/pending replications
go pool.AddMRFWorker()
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
func (p *ReplicationPool) AddWorker() {
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 {
return
}
select {
case <-ctx.Done():
close(p.replicaCh)
close(p.mrfReplicaCh)
case p.replicaCh <- oi:
case p.mrfReplicaCh <- oi:
// queue all overflows into the mrfReplicaCh to handle incoming pending/failed operations
default:
}
}
func (p *ReplicationPool) queueReplicaDeleteTask(doi DeletedObjectVersionInfo) {
func (p *ReplicationPool) queueReplicaDeleteTask(ctx context.Context, doi DeletedObjectVersionInfo) {
if p == nil {
return
}
select {
case <-ctx.Done():
close(p.replicaDeleteCh)
close(p.mrfReplicaDeleteCh)
case p.replicaDeleteCh <- doi:
case p.mrfReplicaDeleteCh <- doi:
// queue all overflows into the mrfReplicaDeleteCh to handle incoming pending/failed operations
default:
}
}
func initBackgroundReplication(ctx context.Context, objectAPI ObjectLayer) {
globalReplicationPool = NewReplicationPool(ctx, objectAPI, globalAPIConfig.getReplicationWorkers())
globalReplicationStats = NewReplicationStats(ctx, objectAPI)
}
// 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
}
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 {
replicateObject(ctx, objInfo, o)
} 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 {
replicateDelete(ctx, dv, o)
} 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"
"github.com/minio/minio-go/v7/pkg/credentials"
"github.com/minio/minio/cmd/crypto"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/bucket/versioning"
"github.com/minio/minio/pkg/madmin"
)
@ -328,6 +329,7 @@ func (sys *BucketTargetSys) load(ctx context.Context, buckets []BucketInfo, objA
for _, bucket := range buckets {
cfg, err := globalBucketMetadataSys.GetBucketTargetsConfig(bucket.Name)
if err != nil {
logger.LogIf(ctx, err)
continue
}
if cfg == nil || cfg.Empty() {
@ -339,6 +341,7 @@ func (sys *BucketTargetSys) load(ctx context.Context, buckets []BucketInfo, objA
for _, tgt := range cfg.Targets {
tgtClient, err := sys.getRemoteTargetClient(&tgt)
if err != nil {
logger.LogIf(ctx, err)
continue
}
sys.arnRemotesMap[tgt.Arn] = tgtClient
@ -432,7 +435,10 @@ func parseBucketTargetConfig(bucket string, cdata, cmetadata []byte) (*madmin.Bu
return nil, err
}
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
}
}

View File

@ -37,6 +37,7 @@ const (
apiListQuorum = "list_quorum"
apiExtendListCacheLife = "extend_list_cache_life"
apiReplicationWorkers = "replication_workers"
EnvAPIRequestsMax = "MINIO_API_REQUESTS_MAX"
EnvAPIRequestsDeadline = "MINIO_API_REQUESTS_DEADLINE"
EnvAPIClusterDeadline = "MINIO_API_CLUSTER_DEADLINE"
@ -87,7 +88,7 @@ var (
},
config.KV{
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.
results := make(chan DataUsageInfo, 1)
results := make(chan madmin.DataUsageInfo, 1)
go storeDataUsageInBackend(ctx, objAPI, results)
bf, err := globalNotificationSys.updateBloomFilter(ctx, nextBloomCycle)
logger.LogIf(ctx, err)
@ -790,6 +790,8 @@ type sizeSummary struct {
pendingSize int64
failedSize int64
replicaSize int64
pendingCount uint64
failedCount uint64
}
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 {
case replication.Pending:
sizeS.pendingCount++
sizeS.pendingSize += oi.Size
globalReplicationPool.queueReplicaTask(oi)
globalReplicationPool.queueReplicaTask(ctx, oi)
case replication.Failed:
sizeS.failedSize += oi.Size
globalReplicationPool.queueReplicaTask(oi)
sizeS.failedCount++
globalReplicationPool.queueReplicaTask(ctx, oi)
case replication.Completed, "COMPLETE":
sizeS.replicatedSize += oi.Size
case replication.Replica:
@ -1128,7 +1132,7 @@ func (i *scannerItem) healReplicationDeletes(ctx context.Context, o ObjectLayer,
} else {
versionID = oi.VersionID
}
globalReplicationPool.queueReplicaDeleteTask(DeletedObjectVersionInfo{
globalReplicationPool.queueReplicaDeleteTask(ctx, DeletedObjectVersionInfo{
DeletedObject: DeletedObject{
ObjectName: oi.Name,
DeleteMarkerVersionID: dmVersionID,

View File

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

View File

@ -32,6 +32,7 @@ import (
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/bucket/lifecycle"
"github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/madmin"
"github.com/tinylib/msgp/msgp"
)
@ -45,15 +46,26 @@ type sizeHistogram [dataUsageBucketLen]uint64
//msgp:tuple dataUsageEntry
type dataUsageEntry struct {
Children dataUsageHashMap
// These fields do no include any children.
Size int64
ReplicatedSize uint64
ReplicationPendingSize uint64
ReplicationFailedSize uint64
ReplicaSize uint64
Objects uint64
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
@ -65,20 +77,40 @@ type dataUsageEntryV2 struct {
Children dataUsageHashMap
}
// dataUsageCache contains a cache of data usage entries latest version 3.
type dataUsageCache struct {
Info dataUsageCacheInfo
Disks []string
Cache map[string]dataUsageEntry
//msgp:tuple dataUsageEntryV3
type dataUsageEntryV3 struct {
// These fields do no include any children.
Size int64
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 {
Info dataUsageCacheInfo
Disks []string
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
type dataUsageEntryInfo struct {
Name string
@ -89,8 +121,8 @@ type dataUsageEntryInfo struct {
type dataUsageCacheInfo struct {
// Name of the bucket. Also root element.
Name string
LastUpdate time.Time
NextCycle uint32
LastUpdate time.Time
// indicates if the disk is being healed and scanner
// should skip healing the disk
SkipHealing bool
@ -100,20 +132,25 @@ type dataUsageCacheInfo struct {
func (e *dataUsageEntry) addSizes(summary sizeSummary) {
e.Size += summary.totalSize
e.ReplicatedSize += uint64(summary.replicatedSize)
e.ReplicationFailedSize += uint64(summary.failedSize)
e.ReplicationPendingSize += uint64(summary.pendingSize)
e.ReplicaSize += uint64(summary.replicaSize)
e.ReplicationStats.ReplicatedSize += uint64(summary.replicatedSize)
e.ReplicationStats.FailedSize += uint64(summary.failedSize)
e.ReplicationStats.PendingSize += uint64(summary.pendingSize)
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.
func (e *dataUsageEntry) merge(other dataUsageEntry) {
e.Objects += other.Objects
e.Size += other.Size
e.ReplicationPendingSize += other.ReplicationPendingSize
e.ReplicationFailedSize += other.ReplicationFailedSize
e.ReplicatedSize += other.ReplicatedSize
e.ReplicaSize += other.ReplicaSize
e.ReplicationStats.PendingSize += other.ReplicationStats.PendingSize
e.ReplicationStats.FailedSize += other.ReplicationStats.FailedSize
e.ReplicationStats.ReplicatedSize += other.ReplicationStats.ReplicatedSize
e.ReplicationStats.ReplicaSize += other.ReplicationStats.ReplicaSize
e.ReplicationStats.PendingCount += other.ReplicationStats.PendingCount
e.ReplicationStats.FailedCount += other.ReplicationStats.FailedCount
for i, v := range other.ObjSizes[:] {
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.
func (d *dataUsageCache) dui(path string, buckets []BucketInfo) DataUsageInfo {
func (d *dataUsageCache) dui(path string, buckets []BucketInfo) madmin.DataUsageInfo {
e := d.find(path)
if e == nil {
// No entry found, return empty.
return DataUsageInfo{}
return madmin.DataUsageInfo{}
}
flat := d.flatten(*e)
return DataUsageInfo{
return madmin.DataUsageInfo{
LastUpdate: d.Info.LastUpdate,
ObjectsTotalCount: flat.Objects,
ObjectsTotalSize: uint64(flat.Size),
ReplicatedSize: flat.ReplicatedSize,
ReplicationFailedSize: flat.ReplicationFailedSize,
ReplicationPendingSize: flat.ReplicationPendingSize,
ReplicaSize: flat.ReplicaSize,
ReplicatedSize: flat.ReplicationStats.ReplicatedSize,
ReplicationFailedSize: flat.ReplicationStats.FailedSize,
ReplicationPendingSize: flat.ReplicationStats.PendingSize,
ReplicaSize: flat.ReplicationStats.ReplicaSize,
ReplicationPendingCount: flat.ReplicationStats.PendingCount,
ReplicationFailedCount: flat.ReplicationStats.FailedCount,
BucketsCount: uint64(len(e.Children)),
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
// key as bucket name
func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]BucketUsageInfo {
var dst = make(map[string]BucketUsageInfo, len(buckets))
func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]madmin.BucketUsageInfo {
var dst = make(map[string]madmin.BucketUsageInfo, len(buckets))
for _, bucket := range buckets {
e := d.find(bucket.Name)
if e == nil {
continue
}
flat := d.flatten(*e)
dst[bucket.Name] = BucketUsageInfo{
dst[bucket.Name] = madmin.BucketUsageInfo{
Size: uint64(flat.Size),
ObjectsCount: flat.Objects,
ReplicationPendingSize: flat.ReplicationPendingSize,
ReplicatedSize: flat.ReplicatedSize,
ReplicationFailedSize: flat.ReplicationFailedSize,
ReplicaSize: flat.ReplicaSize,
ReplicationPendingSize: flat.ReplicationStats.PendingSize,
ReplicatedSize: flat.ReplicationStats.ReplicatedSize,
ReplicationFailedSize: flat.ReplicationStats.FailedSize,
ReplicationPendingCount: flat.ReplicationStats.PendingCount,
ReplicationFailedCount: flat.ReplicationStats.FailedCount,
ReplicaSize: flat.ReplicationStats.ReplicaSize,
ObjectSizesHistogram: flat.ObjSizes.toMap(),
}
}
@ -396,19 +437,21 @@ func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]Bucke
// bucketUsageInfo returns the buckets usage info.
// 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)
if e == nil {
return BucketUsageInfo{}
return madmin.BucketUsageInfo{}
}
flat := d.flatten(*e)
return BucketUsageInfo{
return madmin.BucketUsageInfo{
Size: uint64(flat.Size),
ObjectsCount: flat.Objects,
ReplicationPendingSize: flat.ReplicationPendingSize,
ReplicatedSize: flat.ReplicatedSize,
ReplicationFailedSize: flat.ReplicationFailedSize,
ReplicaSize: flat.ReplicaSize,
ReplicationPendingSize: flat.ReplicationStats.PendingSize,
ReplicationPendingCount: flat.ReplicationStats.PendingCount,
ReplicatedSize: flat.ReplicationStats.ReplicatedSize,
ReplicationFailedSize: flat.ReplicationStats.FailedSize,
ReplicationFailedCount: flat.ReplicationStats.FailedCount,
ReplicaSize: flat.ReplicationStats.ReplicaSize,
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
// and write new data with the new version.
const (
dataUsageCacheVerV4 = 4
dataUsageCacheVerV3 = 3
dataUsageCacheVerV2 = 2
dataUsageCacheVerV1 = 1
@ -541,7 +585,7 @@ const (
// serialize the contents of the cache.
func (d *dataUsageCache) serializeTo(dst io.Writer) error {
// Add version and compress.
_, err := dst.Write([]byte{dataUsageCacheVerV3})
_, err := dst.Write([]byte{dataUsageCacheVerV4})
if err != nil {
return err
}
@ -609,6 +653,35 @@ func (d *dataUsageCache) deserialize(r io.Reader) error {
return err
}
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))
}

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) {
v := dataUsageEntry{}
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) {
v := sizeHistogram{}
bts, err := v.MarshalMsg(nil)

View File

@ -25,11 +25,10 @@ import (
jsoniter "github.com/json-iterator/go"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/hash"
"github.com/minio/minio/pkg/madmin"
)
const (
envDataUsageScannerDebug = "MINIO_DISK_USAGE_SCANNER_DEBUG"
dataUsageRoot = SlashSeparator
dataUsageBucket = minioMetaBucket + SlashSeparator + bucketMetaPrefix
@ -39,7 +38,7 @@ const (
)
// 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 {
dataUsageJSON, err := json.Marshal(dataUsageInfo)
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{})
if err != nil {
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()
var dataUsageInfo DataUsageInfo
var dataUsageInfo madmin.DataUsageInfo
var json = jsoniter.ConfigCompatibleWithStandardLibrary
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.
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 {
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,
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())
}
}

View File

@ -322,8 +322,12 @@ func writeUniqueFileInfo(ctx context.Context, disks []StorageAPI, bucket, prefix
return errDiskNotFound
}
// Pick one FileInfo for a disk at index.
files[index].Erasure.Index = index + 1
return disks[index].WriteMetadata(ctx, bucket, prefix, files[index])
fi := files[index]
fi.Erasure.Index = index + 1
if fi.IsValid() {
return disks[index].WriteMetadata(ctx, bucket, prefix, fi)
}
return errCorruptedFormat
}, 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.
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
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()
// 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)
if err != nil {
@ -1289,7 +1289,7 @@ func (er erasureObjects) updateObjectMeta(ctx context.Context, bucket, object st
disks := er.getDisks()
// 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)
if err != nil {

View File

@ -433,7 +433,7 @@ func (z *erasureServerPools) StorageInfo(ctx context.Context) (StorageInfo, []er
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)
defer cancel()
@ -448,7 +448,7 @@ func (z *erasureServerPools) NSScanner(ctx context.Context, bf *bloomFilter, upd
}
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
}

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
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
var totalCache dataUsageCache
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
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"))
return NotImplemented{}
}

View File

@ -44,7 +44,7 @@ const (
healMetricNamespace MetricNamespace = "minio_heal"
interNodeMetricNamespace MetricNamespace = "minio_inter_node"
nodeMetricNamespace MetricNamespace = "minio_node"
minIOMetricNamespace MetricNamespace = "minio"
minioMetricNamespace MetricNamespace = "minio"
s3MetricNamespace MetricNamespace = "minio_s3"
)
@ -93,9 +93,11 @@ const (
writeTotal MetricName = "write_total"
total MetricName = "total"
failedCount MetricName = "failed_count"
failedBytes MetricName = "failed_bytes"
freeBytes MetricName = "free_bytes"
pendingBytes MetricName = "pending_bytes"
pendingCount MetricName = "pending_count"
readBytes MetricName = "read_bytes"
rcharBytes MetricName = "rchar_bytes"
receivedBytes MetricName = "received_bytes"
@ -356,6 +358,16 @@ func getNodeDiskTotalBytesMD() MetricDescription {
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 {
return MetricDescription{
Namespace: bucketMetricNamespace,
@ -410,6 +422,24 @@ func getBucketRepReceivedBytesMD() MetricDescription {
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 {
return MetricDescription{
Namespace: bucketMetricNamespace,
@ -666,7 +696,7 @@ func getNodeOfflineTotalMD() MetricDescription {
}
func getMinIOVersionMD() MetricDescription {
return MetricDescription{
Namespace: minIOMetricNamespace,
Namespace: minioMetricNamespace,
Subsystem: softwareSubsystem,
Name: versionInfo,
Help: "MinIO Release tag for the server",
@ -675,7 +705,7 @@ func getMinIOVersionMD() MetricDescription {
}
func getMinIOCommitMD() MetricDescription {
return MetricDescription{
Namespace: minIOMetricNamespace,
Namespace: minioMetricNamespace,
Subsystem: softwareSubsystem,
Name: commitInfo,
Help: "Git commit hash for the MinIO release.",
@ -996,13 +1026,14 @@ func getMinioHealingMetrics() MetricsGroup {
if !exists {
return
}
var dur time.Duration
if !bgSeq.lastHealActivity.IsZero() {
dur = time.Since(bgSeq.lastHealActivity)
if bgSeq.lastHealActivity.IsZero() {
return
}
metrics = append(metrics, Metric{
Description: getHealLastActivityTimeMD(),
Value: float64(dur),
Value: float64(time.Since(bgSeq.lastHealActivity)),
})
metrics = append(metrics, getObjectsScanned(bgSeq)...)
metrics = append(metrics, getScannedItems(bgSeq)...)
@ -1224,7 +1255,14 @@ func getBucketUsageMetrics() MetricsGroup {
return
}
metrics = append(metrics, Metric{
Description: getUsageLastScanActivityMD(),
Value: float64(time.Since(dataUsageInfo.LastUpdate)),
})
for bucket, usage := range dataUsageInfo.BucketsUsage {
stat := getLatestReplicationStats(bucket, usage)
metrics = append(metrics, Metric{
Description: getBucketUsageTotalBytesMD(),
Value: float64(usage.Size),
@ -1237,25 +1275,35 @@ func getBucketUsageMetrics() MetricsGroup {
VariableLabels: map[string]string{"bucket": bucket},
})
if usage.hasReplicationUsage() {
if stat.hasReplicationUsage() {
metrics = append(metrics, Metric{
Description: getBucketRepPendingBytesMD(),
Value: float64(usage.ReplicationPendingSize),
Value: float64(stat.PendingSize),
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
Description: getBucketRepFailedBytesMD(),
Value: float64(usage.ReplicationFailedSize),
Value: float64(stat.FailedSize),
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
Description: getBucketRepSentBytesMD(),
Value: float64(usage.ReplicatedSize),
Value: float64(stat.ReplicatedSize),
VariableLabels: map[string]string{"bucket": bucket},
})
metrics = append(metrics, Metric{
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},
})
}
@ -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 {
desc *prometheus.Desc
}

View File

@ -23,6 +23,7 @@ import (
"time"
"github.com/minio/minio/cmd/logger"
"github.com/minio/minio/pkg/madmin"
"github.com/prometheus/client_golang/prometheus"
"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
// is only enabled if scanner is enabled.
func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
@ -447,13 +481,13 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
if err != nil {
return
}
// data usage has not captured any data yet.
if dataUsageInfo.LastUpdate.IsZero() {
return
}
for bucket, usageInfo := range dataUsageInfo.BucketsUsage {
stat := getLatestReplicationStats(bucket, usageInfo)
// Total space used by bucket
ch <- prometheus.MustNewConstMetric(
prometheus.NewDesc(
@ -479,7 +513,7 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity pending to be replicated",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(usageInfo.ReplicationPendingSize),
float64(stat.PendingSize),
bucket,
)
ch <- prometheus.MustNewConstMetric(
@ -488,7 +522,7 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity failed to replicate at least once",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(usageInfo.ReplicationFailedSize),
float64(stat.FailedSize),
bucket,
)
ch <- prometheus.MustNewConstMetric(
@ -497,7 +531,7 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity replicated to destination",
[]string{"bucket"}, nil),
prometheus.GaugeValue,
float64(usageInfo.ReplicatedSize),
float64(stat.ReplicatedSize),
bucket,
)
ch <- prometheus.MustNewConstMetric(
@ -506,7 +540,25 @@ func bucketUsageMetricsPrometheus(ch chan<- prometheus.Metric) {
"Total capacity replicated to this instance",
[]string{"bucket"}, nil),
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,
)
for k, v := range usageInfo.ObjectSizesHistogram {

View File

@ -69,57 +69,6 @@ var ObjectsHistogramIntervals = []objectHistogramInterval{
{"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.
type BucketInfo struct {
// Name of the bucket.

View File

@ -91,7 +91,7 @@ type ObjectLayer interface {
// Storage operations.
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
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)
encodedSuccessResponse := encodeResponse(response)
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)
@ -1656,7 +1656,7 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req
}
}
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)
@ -1938,7 +1938,7 @@ func (api objectAPIHandlers) PutObjectExtractHandler(w http.ResponseWriter, r *h
}
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)
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.
@ -3294,7 +3294,7 @@ func (api objectAPIHandlers) PutObjectLegalHoldHandler(w http.ResponseWriter, r
return
}
if replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync)
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.MetadataReplicationType)
}
writeSuccessResponseHeadersOnly(w)
@ -3467,7 +3467,7 @@ func (api objectAPIHandlers) PutObjectRetentionHandler(w http.ResponseWriter, r
return
}
if replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync)
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.MetadataReplicationType)
}
writeSuccessNoContent(w)
@ -3650,7 +3650,7 @@ func (api objectAPIHandlers) PutObjectTaggingHandler(w http.ResponseWriter, r *h
}
if replicate {
scheduleReplication(ctx, objInfo.Clone(), objAPI, sync)
scheduleReplication(ctx, objInfo.Clone(), objAPI, sync, replication.MetadataReplicationType)
}
if objInfo.VersionID != "" {
@ -3724,7 +3724,7 @@ func (api objectAPIHandlers) DeleteObjectTaggingHandler(w http.ResponseWriter, r
}
if replicate {
scheduleReplication(ctx, oi.Clone(), objAPI, sync)
scheduleReplication(ctx, oi.Clone(), objAPI, sync, replication.MetadataReplicationType)
}
if oi.VersionID != "" {

View File

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

View File

@ -231,7 +231,7 @@ type xlMetaV2 struct {
Versions []xlMetaV2Version `json:"Versions" msg:"Versions"`
// 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.
data xlMetaInlineData `msg:"-"`
}
@ -295,28 +295,31 @@ func (x xlMetaInlineData) validate() error {
if len(x) == 0 {
return nil
}
if !x.versionOK() {
return fmt.Errorf("xlMetaInlineData: unknown version 0x%x", x[0])
}
sz, buf, err := msgp.ReadMapHeaderBytes(x.afterVersion())
if err != nil {
return err
return fmt.Errorf("xlMetaInlineData: %w", err)
}
for i := uint32(0); i < sz; i++ {
var key []byte
key, buf, err = msgp.ReadMapKeyZC(buf)
if err != nil {
return err
return fmt.Errorf("xlMetaInlineData: %w", err)
}
if len(key) == 0 {
return fmt.Errorf("xlMetaInlineData: key %d is length 0", i)
}
_, buf, err = msgp.ReadBytesZC(buf)
if err != nil {
return err
return fmt.Errorf("xlMetaInlineData: %w", err)
}
}
return nil
}
@ -564,31 +567,27 @@ func (z *xlMetaV2) AddLegacy(m *xlMetaV1Object) error {
func (z *xlMetaV2) Load(buf []byte) error {
buf, _, minor, err := checkXL2V1(buf)
if err != nil {
return errFileCorrupt
return fmt.Errorf("z.Load %w", err)
}
switch minor {
case 0:
_, err = z.UnmarshalMsg(buf)
if err != nil {
return errFileCorrupt
return fmt.Errorf("z.Load %w", err)
}
return nil
case 1:
v, buf, err := msgp.ReadBytesZC(buf)
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 != nil {
return errFileCorrupt
if _, err = z.UnmarshalMsg(v); err != nil {
return fmt.Errorf("z.Load version(%d), vLen(%d), %w", minor, len(v), err)
}
// Add remaining data.
z.data = nil
if len(buf) > 0 {
z.data = buf
if err := z.data.validate(); err != nil {
return errFileCorrupt
}
if err = z.data.validate(); err != nil {
return fmt.Errorf("z.Load version(%d), bufLen(%d) %w", minor, len(buf), err)
}
default:
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)
return err
}
buf, err = xlMeta.AppendTo(nil)
if err != nil {
logger.LogIf(ctx, err)
return err
}
if err := xlMeta.Load(buf); err != nil {
panic(err)
}
} else {
if err = xlMeta.Load(buf); err != nil {
logger.LogIf(ctx, err)
return err
}
if err = xlMeta.AddVersion(fi); err != nil {
logger.LogIf(ctx, err)
return err
}
buf, err = xlMeta.AppendTo(nil)
if err != nil {
logger.LogIf(ctx, err)
return err
}
}
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 {
return fi, nil
}
// Reading data for small objects when
// - object has not yet transitioned
// - object size lesser than 32KiB
// - object has maximum of 1 parts
if fi.TransitionStatus == "" && fi.DataDir != "" && fi.Size <= smallFileThreshold && len(fi.Parts) == 1 {
// Enable O_DIRECT optionally only if drive supports it.
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_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_pending_count` |Total number of replication operations pending for this bucket. |
|`minio_bucket_usage_object_total` |Total number of objects |
|`minio_bucket_usage_total_bytes` |Total bucket size in bytes |
|`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 |
|--------------------------------|----------------------|-------------------------|
| `AccountInfo.AccountName` | _string_ | Account name. |
| `AccountInfo.Buckets` | _[]BucketUsageInfo_ | Bucket usage info. |
| `AccountInfo.Buckets` | _[]BucketAccessInfo_ | Bucket usage info. |
| Param | Type | Description |
|----------------------------|-----------------|-----------------------------------------|
| `BucketUsageInfo.Name` | _string_ | The name of the current bucket
| `BucketUsageInfo.Size` | _uint64_ | The total size of the current bucket
| `BucketUsageInfo.Created` | _time.Time_ | Bucket creation time
| `BucketUsageInfo.Access` | _AccountAccess_ | Type of access of the current account
| `BucketAccessInfo.Name` | _string_ | The name of the current bucket
| `BucketAccessInfo.Size` | _uint64_ | The total size of the current bucket
| `BucketAccessInfo.Created` | _time.Time_ | Bucket creation time
| `BucketAccessInfo.Access` | _AccountAccess_ | Type of access of the current account
| Param | Type | Description |

View File

@ -20,7 +20,6 @@ package madmin
import (
"context"
"encoding/json"
"io/ioutil"
"net/http"
"time"
)
@ -124,36 +123,71 @@ func (adm *AdminClient) StorageInfo(ctx context.Context) (StorageInfo, error) {
// Unmarshal the server's json response
var storageInfo StorageInfo
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return StorageInfo{}, err
}
err = json.Unmarshal(respBytes, &storageInfo)
if err != nil {
if err = json.NewDecoder(resp.Body).Decode(&storageInfo); err != nil {
return StorageInfo{}, err
}
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 {
// 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"`
ObjectsCount uint64 `json:"objectsCount"`
// Objects total count across all buckets
ObjectsTotalCount uint64 `json:"objectsCount"`
// Objects total size across all buckets
ObjectsTotalSize uint64 `json:"objectsTotalSize"`
// ObjectsSizesHistogram contains information on objects across all buckets.
// See ObjectsHistogramIntervals.
ObjectsSizesHistogram map[string]uint64 `json:"objectsSizesHistogram"`
// 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 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"`
// BucketsSizes is "bucket name" -> size.
BucketsSizes map[string]uint64 `json:"bucketsSizes"`
// 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"`
}
// 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
var dataUsageInfo DataUsageInfo
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return DataUsageInfo{}, err
}
err = json.Unmarshal(respBytes, &dataUsageInfo)
if err != nil {
if err = json.NewDecoder(resp.Body).Decode(&dataUsageInfo); err != nil {
return DataUsageInfo{}, err
}
@ -344,14 +371,7 @@ func (adm *AdminClient) ServerInfo(ctx context.Context) (InfoMessage, error) {
// Unmarshal the server's json response
var message InfoMessage
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
return InfoMessage{}, err
}
err = json.Unmarshal(respBytes, &message)
if err != nil {
if err = json.NewDecoder(resp.Body).Decode(&message); err != nil {
return InfoMessage{}, err
}

View File

@ -20,7 +20,6 @@ package madmin
import (
"context"
"encoding/json"
"io/ioutil"
"net/http"
"net/url"
)
@ -53,10 +52,9 @@ func (adm *AdminClient) ServerUpdate(ctx context.Context, updateURL string) (us
return us, httpRespToErrorResponse(resp)
}
buf, err := ioutil.ReadAll(resp.Body)
if err != nil {
if err = json.NewDecoder(resp.Body).Decode(&us); err != nil {
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"`
}
// 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
type BucketUsageInfo struct {
type BucketAccessInfo struct {
Name string `json:"name"`
Size uint64 `json:"size"`
Created time.Time `json:"created"`
@ -49,7 +49,7 @@ type BucketUsageInfo struct {
type AccountInfo struct {
AccountName string
Policy iampolicy.Policy
Buckets []BucketUsageInfo
Buckets []BucketAccessInfo
}
// AccountInfo returns the usage info for the authenticating account.