From c4373ef290344fd01981864cb9feca9410cf5b21 Mon Sep 17 00:00:00 2001 From: Poorna Krishnamoorthy Date: Sat, 18 Sep 2021 16:31:35 -0400 Subject: [PATCH] Add support for multi site replication (#12880) --- cmd/admin-handlers-users.go | 2 +- cmd/api-datatypes.go | 16 +- cmd/bucket-handlers.go | 98 +- cmd/bucket-quota.go | 2 +- cmd/bucket-replication-stats.go | 123 +- cmd/bucket-replication-utils.go | 602 ++++++++ cmd/bucket-replication-utils_gen.go | 1285 +++++++++++++++++ cmd/bucket-replication-utils_gen_test.go | 688 +++++++++ cmd/bucket-replication-utils_test.go | 247 ++++ cmd/bucket-replication.go | 1045 +++++++++----- cmd/bucket-replication_test.go | 138 +- cmd/bucket-stats.go | 98 +- cmd/bucket-stats_gen.go | 427 +++++- cmd/bucket-stats_gen_test.go | 113 ++ cmd/bucket-targets.go | 44 +- cmd/data-scanner.go | 88 +- cmd/data-usage-cache.go | 223 ++- cmd/data-usage-cache_gen.go | 634 +++++++- cmd/data-usage-cache_gen_test.go | 226 +++ cmd/data-usage-utils.go | 87 ++ cmd/data-usage.go | 37 +- cmd/data-usage_test.go | 21 +- cmd/erasure-metadata.go | 75 +- cmd/erasure-object.go | 87 +- cmd/erasure-server-pool.go | 4 +- cmd/fs-v1.go | 2 +- cmd/gateway-unsupported.go | 2 +- cmd/metrics-v2.go | 46 +- cmd/metrics.go | 87 +- cmd/object-api-datatypes.go | 88 +- cmd/object-api-interface.go | 73 +- cmd/object-api-options.go | 61 +- cmd/object-api-utils.go | 2 +- cmd/object-handlers.go | 245 ++-- cmd/storage-datatypes.go | 31 +- cmd/storage-datatypes_gen.go | 112 +- cmd/storage-rest-common.go | 2 +- cmd/xl-storage-format-v2.go | 75 +- cmd/xl-storage-format-v2_test.go | 20 +- cmd/xl-storage-free-version_test.go | 10 +- cmd/xl-storage.go | 17 +- docs/bucket/replication/DESIGN.md | 93 +- docs/bucket/replication/README.md | 19 +- go.mod | 2 +- go.sum | 3 +- internal/bucket/replication/datatypes.go | 47 + internal/bucket/replication/datatypes_gen.go | 59 + .../bucket/replication/datatypes_gen_test.go | 3 + internal/bucket/replication/destination.go | 22 +- internal/bucket/replication/replication.go | 112 +- .../bucket/replication/replication_test.go | 73 +- internal/http/headers.go | 6 + 52 files changed, 6492 insertions(+), 1230 deletions(-) create mode 100644 cmd/bucket-replication-utils.go create mode 100644 cmd/bucket-replication-utils_gen.go create mode 100644 cmd/bucket-replication-utils_gen_test.go create mode 100644 cmd/bucket-replication-utils_test.go create mode 100644 cmd/data-usage-utils.go create mode 100644 internal/bucket/replication/datatypes.go create mode 100644 internal/bucket/replication/datatypes_gen.go create mode 100644 internal/bucket/replication/datatypes_gen_test.go diff --git a/cmd/admin-handlers-users.go b/cmd/admin-handlers-users.go index 2ea469dbf..4269fcfb8 100644 --- a/cmd/admin-handlers-users.go +++ b/cmd/admin-handlers-users.go @@ -1006,7 +1006,7 @@ func (a adminAPIHandlers) AccountInfoHandler(w http.ResponseWriter, r *http.Requ return rd, wr } - var dataUsageInfo madmin.DataUsageInfo + var dataUsageInfo DataUsageInfo var err error if !globalIsGateway { // Load the latest calculated data usage diff --git a/cmd/api-datatypes.go b/cmd/api-datatypes.go index dd99041fb..3d4350e7a 100644 --- a/cmd/api-datatypes.go +++ b/cmd/api-datatypes.go @@ -28,14 +28,10 @@ type DeletedObject struct { DeleteMarkerVersionID string `xml:"DeleteMarkerVersionId,omitempty"` ObjectName string `xml:"Key,omitempty"` VersionID string `xml:"VersionId,omitempty"` - - // MinIO extensions to support delete marker replication - // Replication status of DeleteMarker - DeleteMarkerReplicationStatus string `xml:"DeleteMarkerReplicationStatus,omitempty"` // MTime of DeleteMarker on source that needs to be propagated to replica - DeleteMarkerMTime DeleteMarkerMTime `xml:"DeleteMarkerMTime,omitempty"` - // Status of versioned delete (of object or DeleteMarker) - VersionPurgeStatus VersionPurgeStatusType `xml:"VersionPurgeStatus,omitempty"` + DeleteMarkerMTime DeleteMarkerMTime `xml:"-"` + // MinIO extensions to support delete marker replication + ReplicationState ReplicationState `xml:"-"` } // DeleteMarkerMTime is an embedded type containing time.Time for XML marshal @@ -60,8 +56,10 @@ type ObjectToDelete struct { DeleteMarkerReplicationStatus string `xml:"DeleteMarkerReplicationStatus"` // Status of versioned delete (of object or DeleteMarker) VersionPurgeStatus VersionPurgeStatusType `xml:"VersionPurgeStatus"` - // Version ID of delete marker - DeleteMarkerVersionID string `xml:"DeleteMarkerVersionId"` + // VersionPurgeStatuses holds the internal + VersionPurgeStatuses string `xml:"VersionPurgeStatuses"` + // ReplicateDecisionStr stringified representation of replication decision + ReplicateDecisionStr string `xml:"-"` } // createBucketConfiguration container for bucket configuration request from client. diff --git a/cmd/bucket-handlers.go b/cmd/bucket-handlers.go index b0ef77c6f..594921f4a 100644 --- a/cmd/bucket-handlers.go +++ b/cmd/bucket-handlers.go @@ -452,9 +452,10 @@ func (api objectAPIHandlers) DeleteMultipleObjectsHandler(w http.ResponseWriter, } var ( - hasLockEnabled, replicateSync bool - goi ObjectInfo - gerr error + hasLockEnabled bool + dsc ReplicateDecision + goi ObjectInfo + gerr error ) replicateDeletes := hasReplicationRules(ctx, bucket, deleteObjects.Objects) if rcfg, _ := globalBucketObjectLockSys.Get(bucket); rcfg.LockEnabled { @@ -514,17 +515,18 @@ func (api objectAPIHandlers) DeleteMultipleObjectsHandler(w http.ResponseWriter, } if replicateDeletes { - replicate, repsync := checkReplicateDelete(ctx, bucket, ObjectToDelete{ + dsc = checkReplicateDelete(ctx, bucket, ObjectToDelete{ ObjectName: object.ObjectName, VersionID: object.VersionID, - }, goi, gerr) - replicateSync = repsync - if replicate { + }, goi, opts, gerr) + if dsc.ReplicateAny() { if object.VersionID != "" { object.VersionPurgeStatus = Pending + object.VersionPurgeStatuses = dsc.PendingStatus() } else { - object.DeleteMarkerReplicationStatus = string(replication.Pending) + object.DeleteMarkerReplicationStatus = dsc.PendingStatus() } + object.ReplicateDecisionStr = dsc.String() } } if object.VersionID != "" && hasLockEnabled { @@ -570,14 +572,15 @@ func (api objectAPIHandlers) DeleteMultipleObjectsHandler(w http.ResponseWriter, objToDel := ObjectToDelete{ ObjectName: dObjects[i].ObjectName, VersionID: dObjects[i].VersionID, - VersionPurgeStatus: dObjects[i].VersionPurgeStatus, - DeleteMarkerReplicationStatus: dObjects[i].DeleteMarkerReplicationStatus, + VersionPurgeStatus: dObjects[i].VersionPurgeStatus(), + VersionPurgeStatuses: dObjects[i].ReplicationState.VersionPurgeStatusInternal, + DeleteMarkerReplicationStatus: dObjects[i].ReplicationState.ReplicationStatusInternal, + ReplicateDecisionStr: dObjects[i].ReplicationState.ReplicateDecisionStr, } dindex := objectsToDelete[objToDel] if errs[i] == nil || isErrObjectNotFound(errs[i]) || isErrVersionNotFound(errs[i]) { if replicateDeletes { - dObjects[i].DeleteMarkerReplicationStatus = deleteList[i].DeleteMarkerReplicationStatus - dObjects[i].VersionPurgeStatus = deleteList[i].VersionPurgeStatus + dObjects[i].ReplicationState = deleteList[i].ReplicationState() } deletedObjects[dindex] = dObjects[i] continue @@ -610,12 +613,12 @@ func (api objectAPIHandlers) DeleteMultipleObjectsHandler(w http.ResponseWriter, } if replicateDeletes { - if dobj.DeleteMarkerReplicationStatus == string(replication.Pending) || dobj.VersionPurgeStatus == Pending { + if dobj.DeleteMarkerReplicationStatus() == replication.Pending || dobj.VersionPurgeStatus() == Pending { dv := DeletedObjectReplicationInfo{ DeletedObject: dobj, Bucket: bucket, } - scheduleReplicationDelete(ctx, dv, objectAPI, replicateSync) + scheduleReplicationDelete(ctx, dv, objectAPI) } } @@ -1639,32 +1642,19 @@ func (api objectAPIHandlers) GetBucketReplicationMetricsHandler(w http.ResponseW writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) return } - - bucketStats := globalNotificationSys.GetClusterBucketStats(r.Context(), bucket) - bucketReplStats := BucketReplicationStats{} - // sum up metrics from each node in the cluster - for _, bucketStat := range bucketStats { - bucketReplStats.FailedCount += bucketStat.ReplicationStats.FailedCount - bucketReplStats.FailedSize += bucketStat.ReplicationStats.FailedSize - bucketReplStats.PendingCount += bucketStat.ReplicationStats.PendingCount - bucketReplStats.PendingSize += bucketStat.ReplicationStats.PendingSize - bucketReplStats.ReplicaSize += bucketStat.ReplicationStats.ReplicaSize - bucketReplStats.ReplicatedSize += bucketStat.ReplicationStats.ReplicatedSize + var usageInfo BucketUsageInfo + dataUsageInfo, err := loadDataUsageFromBackend(ctx, objectAPI) + if err == nil && !dataUsageInfo.LastUpdate.IsZero() { + usageInfo = dataUsageInfo.BucketsUsage[bucket] } - // add initial usage from the time of cluster up - usageStat := globalReplicationStats.GetInitialUsage(bucket) - bucketReplStats.FailedCount += usageStat.FailedCount - bucketReplStats.FailedSize += usageStat.FailedSize - bucketReplStats.PendingCount += usageStat.PendingCount - bucketReplStats.PendingSize += usageStat.PendingSize - bucketReplStats.ReplicaSize += usageStat.ReplicaSize - bucketReplStats.ReplicatedSize += usageStat.ReplicatedSize - if err := json.NewEncoder(w).Encode(&bucketReplStats); err != nil { - writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) + bucketReplStats := getLatestReplicationStats(bucket, usageInfo) + jsonData, err := json.Marshal(bucketReplStats) + if err != nil { + writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL) return } - w.(http.Flusher).Flush() + writeSuccessResponseJSON(w, jsonData) } // ResetBucketReplicationStateHandler - starts a replication reset for all objects in a bucket which @@ -1673,12 +1663,16 @@ func (api objectAPIHandlers) GetBucketReplicationMetricsHandler(w http.ResponseW // remote target is entirely lost,and previously replicated objects need to be re-synced. func (api objectAPIHandlers) ResetBucketReplicationStateHandler(w http.ResponseWriter, r *http.Request) { ctx := newContext(r, w, "ResetBucketReplicationState") - defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r)) vars := mux.Vars(r) bucket := vars["bucket"] - durationStr := r.Form.Get("older-than") + durationStr := r.URL.Query().Get("older-than") + arn := r.URL.Query().Get("arn") + resetID := r.URL.Query().Get("reset-id") + if resetID == "" { + resetID = mustGetUUID() + } var ( days time.Duration err error @@ -1719,9 +1713,31 @@ func (api objectAPIHandlers) ResetBucketReplicationStateHandler(w http.ResponseW writeErrorResponse(ctx, w, errorCodes.ToAPIErr(ErrReplicationNoMatchingRuleError), r.URL) return } - target := globalBucketTargetSys.GetRemoteBucketTargetByArn(ctx, bucket, config.RoleArn) + tgtArns := config.FilterTargetArns( + replication.ObjectOpts{ + OpType: replication.ResyncReplicationType, + TargetArn: arn}) + + if len(tgtArns) == 0 { + writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErrWithErr(ErrBadRequest, InvalidArgument{ + Bucket: bucket, + Err: fmt.Errorf("Remote target ARN %s missing/not eligible for replication resync", arn), + }), r.URL) + return + } + + if len(tgtArns) > 1 && arn == "" { + writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErrWithErr(ErrBadRequest, InvalidArgument{ + Bucket: bucket, + Err: fmt.Errorf("ARN should be specified for replication reset"), + }), r.URL) + return + } + var rinfo ResyncTargetsInfo + target := globalBucketTargetSys.GetRemoteBucketTargetByArn(ctx, bucket, tgtArns[0]) target.ResetBeforeDate = UTCNow().AddDate(0, 0, -1*int(days/24)) - target.ResetID = mustGetUUID() + target.ResetID = resetID + rinfo.Targets = append(rinfo.Targets, ResyncTarget{Arn: tgtArns[0], ResetID: target.ResetID}) if err = globalBucketTargetSys.SetTarget(ctx, bucket, &target, true); err != nil { switch err.(type) { case BucketRemoteConnectionErr: @@ -1745,7 +1761,7 @@ func (api objectAPIHandlers) ResetBucketReplicationStateHandler(w http.ResponseW writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) return } - data, err := json.Marshal(target.ResetID) + data, err := json.Marshal(rinfo) if err != nil { writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL) return diff --git a/cmd/bucket-quota.go b/cmd/bucket-quota.go index d0e80fb51..e2555b790 100644 --- a/cmd/bucket-quota.go +++ b/cmd/bucket-quota.go @@ -116,7 +116,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 madmin.BucketUsageInfo) { +func enforceFIFOQuotaBucket(ctx context.Context, objectAPI ObjectLayer, bucket string, bui BucketUsageInfo) { // Check if the current bucket has quota restrictions, if not skip it cfg, err := globalBucketQuotaSys.Get(bucket) if err != nil { diff --git a/cmd/bucket-replication-stats.go b/cmd/bucket-replication-stats.go index 0e03ecfd1..8cebd402a 100644 --- a/cmd/bucket-replication-stats.go +++ b/cmd/bucket-replication-stats.go @@ -26,10 +26,12 @@ import ( ) func (b *BucketReplicationStats) hasReplicationUsage() bool { - return b.FailedSize > 0 || - b.ReplicatedSize > 0 || - b.ReplicaSize > 0 || - b.FailedCount > 0 + for _, s := range b.Stats { + if s.hasReplicationUsage() { + return true + } + } + return false } // ReplicationStats holds the global in-memory replication stats @@ -37,6 +39,7 @@ type ReplicationStats struct { Cache map[string]*BucketReplicationStats UsageCache map[string]*BucketReplicationStats sync.RWMutex + ulock sync.RWMutex } // Delete deletes in-memory replication statistics for a bucket. @@ -52,45 +55,65 @@ func (r *ReplicationStats) Delete(bucket string) { } -// Update updates in-memory replication statistics with new values. -func (r *ReplicationStats) Update(bucket string, n int64, status, prevStatus replication.StatusType, opType replication.Type) { +// UpdateReplicaStat updates in-memory replica statistics with new values. +func (r *ReplicationStats) UpdateReplicaStat(bucket string, n int64) { if r == nil { return } - r.RLock() - b, ok := r.Cache[bucket] + r.Lock() + defer r.Unlock() + bs, ok := r.Cache[bucket] if !ok { - b = &BucketReplicationStats{} + bs = &BucketReplicationStats{Stats: make(map[string]*BucketReplicationStat)} + } + atomic.StoreInt64(&bs.ReplicaSize, n) + r.Cache[bucket] = bs +} + +// Update updates in-memory replication statistics with new values. +func (r *ReplicationStats) Update(bucket string, arn string, n int64, status, prevStatus replication.StatusType, opType replication.Type) { + if r == nil { + return + } + r.RLock() + bs, ok := r.Cache[bucket] + if !ok { + bs = &BucketReplicationStats{Stats: make(map[string]*BucketReplicationStat)} + } + b, ok := bs.Stats[arn] + if !ok { + b = &BucketReplicationStat{} } r.RUnlock() switch status { case replication.Completed: switch prevStatus { // adjust counters based on previous state case replication.Failed: - atomic.AddUint64(&b.FailedCount, ^uint64(0)) + atomic.AddInt64(&b.FailedCount, -1) } if opType == replication.ObjectReplicationType { - atomic.AddUint64(&b.ReplicatedSize, uint64(n)) + atomic.AddInt64(&b.ReplicatedSize, n) switch prevStatus { case replication.Failed: - atomic.AddUint64(&b.FailedSize, ^uint64(n-1)) + atomic.AddInt64(&b.FailedSize, -1*n) } } case replication.Failed: if opType == replication.ObjectReplicationType { if prevStatus == replication.Pending { - atomic.AddUint64(&b.FailedSize, uint64(n)) - atomic.AddUint64(&b.FailedCount, 1) + atomic.AddInt64(&b.FailedSize, n) + atomic.AddInt64(&b.FailedCount, 1) } } case replication.Replica: if opType == replication.ObjectReplicationType { - atomic.AddUint64(&b.ReplicaSize, uint64(n)) + atomic.AddInt64(&b.ReplicaSize, n) } } r.Lock() - r.Cache[bucket] = b + bs.Stats[arn] = b + r.Cache[bucket] = bs r.Unlock() } @@ -100,25 +123,49 @@ func (r *ReplicationStats) GetInitialUsage(bucket string) BucketReplicationStats return BucketReplicationStats{} } - r.RLock() - defer r.RUnlock() + r.ulock.RLock() + + brs := BucketReplicationStats{Stats: make(map[string]*BucketReplicationStat)} st, ok := r.UsageCache[bucket] - if !ok { - return BucketReplicationStats{} + if ok { + return st.Clone() } - return BucketReplicationStats{ - FailedSize: atomic.LoadUint64(&st.FailedSize), - ReplicatedSize: atomic.LoadUint64(&st.ReplicatedSize), - ReplicaSize: atomic.LoadUint64(&st.ReplicaSize), - FailedCount: atomic.LoadUint64(&st.FailedCount), + r.ulock.RUnlock() + + dataUsageInfo, err := loadDataUsageFromBackend(GlobalContext, newObjectLayerFn()) + if err != nil { + return brs } + // data usage has not captured any data yet. + if dataUsageInfo.LastUpdate.IsZero() { + return brs + } + usage, ok := dataUsageInfo.BucketsUsage[bucket] + if ok && usage.ReplicationInfo != nil { + brs.ReplicaSize = int64(usage.ReplicaSize) + for arn, uinfo := range usage.ReplicationInfo { + brs.Stats[arn] = &BucketReplicationStat{ + FailedSize: int64(uinfo.ReplicationFailedSize), + ReplicatedSize: int64(uinfo.ReplicatedSize), + ReplicaSize: int64(uinfo.ReplicaSize), + FailedCount: int64(uinfo.ReplicationFailedCount), + } + } + if brs.hasReplicationUsage() { + r.ulock.Lock() + defer r.ulock.Unlock() + r.UsageCache[bucket] = &brs + } + + } + return brs } // Get replication metrics for a bucket from this node since this node came up. func (r *ReplicationStats) Get(bucket string) BucketReplicationStats { if r == nil { - return BucketReplicationStats{} + return BucketReplicationStats{Stats: make(map[string]*BucketReplicationStat)} } r.RLock() @@ -126,15 +173,9 @@ func (r *ReplicationStats) Get(bucket string) BucketReplicationStats { st, ok := r.Cache[bucket] if !ok { - return BucketReplicationStats{} - } - - return BucketReplicationStats{ - FailedSize: atomic.LoadUint64(&st.FailedSize), - ReplicatedSize: atomic.LoadUint64(&st.ReplicatedSize), - ReplicaSize: atomic.LoadUint64(&st.ReplicaSize), - FailedCount: atomic.LoadUint64(&st.FailedCount), + return BucketReplicationStats{Stats: make(map[string]*BucketReplicationStat)} } + return st.Clone() } // NewReplicationStats initialize in-memory replication statistics @@ -156,11 +197,17 @@ func NewReplicationStats(ctx context.Context, objectAPI ObjectLayer) *Replicatio for bucket, usage := range dataUsageInfo.BucketsUsage { b := &BucketReplicationStats{ - FailedSize: usage.ReplicationFailedSize, - ReplicatedSize: usage.ReplicatedSize, - ReplicaSize: usage.ReplicaSize, - FailedCount: usage.ReplicationFailedCount, + Stats: make(map[string]*BucketReplicationStat, len(usage.ReplicationInfo)), } + for arn, uinfo := range usage.ReplicationInfo { + b.Stats[arn] = &BucketReplicationStat{ + FailedSize: int64(uinfo.ReplicationFailedSize), + ReplicatedSize: int64(uinfo.ReplicatedSize), + ReplicaSize: int64(uinfo.ReplicaSize), + FailedCount: int64(uinfo.ReplicationFailedCount), + } + } + b.ReplicaSize += int64(usage.ReplicaSize) if b.hasReplicationUsage() { st.UsageCache[bucket] = b } diff --git a/cmd/bucket-replication-utils.go b/cmd/bucket-replication-utils.go new file mode 100644 index 000000000..c645fcbe2 --- /dev/null +++ b/cmd/bucket-replication-utils.go @@ -0,0 +1,602 @@ +// Copyright (c) 2015-2021 MinIO, Inc. +// +// This file is part of MinIO Object Storage stack +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// This program is distributed in the hope that it will be useful +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +package cmd + +import ( + "bytes" + "fmt" + "regexp" + "strconv" + "strings" + "time" + + "github.com/minio/minio/internal/bucket/replication" +) + +//go:generate msgp -file=$GOFILE + +// replicatedTargetInfo struct represents replication info on a target +type replicatedTargetInfo struct { + Arn string + Size int64 + ReplicationAction replicationAction // full or metadata only + OpType replication.Type // whether incoming replication, existing object, healing etc.. + ReplicationStatus replication.StatusType + PrevReplicationStatus replication.StatusType + VersionPurgeStatus VersionPurgeStatusType + ResyncTimestamp string + ReplicationResynced bool // true only if resync attempted for this target +} + +// Empty returns true for a target if arn is empty +func (rt replicatedTargetInfo) Empty() bool { + return rt.Arn == "" +} + +type replicatedInfos struct { + ReplicationTimeStamp time.Time + Targets []replicatedTargetInfo +} + +func (ri replicatedInfos) CompletedSize() (sz int64) { + for _, t := range ri.Targets { + if t.Empty() { + continue + } + if t.ReplicationStatus == replication.Completed && t.PrevReplicationStatus != replication.Completed { + sz += t.Size + } + } + return sz +} + +// ReplicationAttempted returns true if replication was attempted on any of the targets for the object version +// queued +func (ri replicatedInfos) ReplicationResynced() bool { + for _, t := range ri.Targets { + if t.Empty() || !t.ReplicationResynced { + continue + } + return true + } + return false +} + +func (ri replicatedInfos) ReplicationStatusInternal() string { + b := new(bytes.Buffer) + for _, t := range ri.Targets { + if t.Empty() { + continue + } + fmt.Fprintf(b, "%s=%s;", t.Arn, t.ReplicationStatus.String()) + } + return b.String() +} + +func (ri replicatedInfos) ReplicationStatus() replication.StatusType { + if len(ri.Targets) == 0 { + return replication.StatusType("") + } + completed := 0 + for _, v := range ri.Targets { + switch v.ReplicationStatus { + case replication.Failed: + return replication.Failed + case replication.Completed: + completed++ + } + } + if completed == len(ri.Targets) { + return replication.Completed + } + return replication.Pending +} + +func (ri replicatedInfos) VersionPurgeStatus() VersionPurgeStatusType { + if len(ri.Targets) == 0 { + return VersionPurgeStatusType("") + } + completed := 0 + for _, v := range ri.Targets { + switch v.VersionPurgeStatus { + case Failed: + return Failed + case Complete: + completed++ + } + } + if completed == len(ri.Targets) { + return Complete + } + return Pending +} + +func (ri replicatedInfos) VersionPurgeStatusInternal() string { + b := new(bytes.Buffer) + for _, t := range ri.Targets { + if t.Empty() { + continue + } + if t.VersionPurgeStatus.Empty() { + continue + } + fmt.Fprintf(b, "%s=%s;", t.Arn, t.VersionPurgeStatus) + } + return b.String() +} + +func (ri replicatedInfos) Action() replicationAction { + for _, t := range ri.Targets { + if t.Empty() { + continue + } + // rely on replication action from target that actually performed replication now. + if t.PrevReplicationStatus != replication.Completed { + return t.ReplicationAction + } + } + return replicateNone +} + +var replStatusRegex = regexp.MustCompile(`([^=].*?)=([^,].*?);`) + +// TargetReplicationStatus - returns replication status of a target +func (o *ObjectInfo) TargetReplicationStatus(arn string) (status replication.StatusType) { + repStatMatches := replStatusRegex.FindAllStringSubmatch(o.ReplicationStatusInternal, -1) + for _, repStatMatch := range repStatMatches { + if len(repStatMatch) != 3 { + return + } + if repStatMatch[1] == arn { + return replication.StatusType(repStatMatch[2]) + } + } + return +} + +type replicateTargetDecision struct { + Replicate bool // Replicate to this target + Synchronous bool // Synchronous replication configured. + Arn string //ARN of replication target + ID string +} + +func (t *replicateTargetDecision) String() string { + return fmt.Sprintf("%t;%t;%s;%s", t.Replicate, t.Synchronous, t.Arn, t.ID) +} + +func newReplicateTargetDecision(arn string, replicate bool, sync bool) replicateTargetDecision { + d := replicateTargetDecision{ + Replicate: replicate, + Synchronous: sync, + Arn: arn, + } + return d +} + +// ReplicateDecision represents replication decision for each target +type ReplicateDecision struct { + targetsMap map[string]replicateTargetDecision +} + +// ReplicateAny returns true if atleast one target qualifies for replication +func (d *ReplicateDecision) ReplicateAny() bool { + for _, t := range d.targetsMap { + if t.Replicate { + return true + } + } + return false +} + +// Synchronous returns true if atleast one target qualifies for synchronous replication +func (d *ReplicateDecision) Synchronous() bool { + for _, t := range d.targetsMap { + if t.Synchronous { + return true + } + } + return false +} + +func (d *ReplicateDecision) String() string { + b := new(bytes.Buffer) + for key, value := range d.targetsMap { + fmt.Fprintf(b, "%s=%s,", key, value.String()) + } + return strings.TrimSuffix(b.String(), ",") +} + +// Set updates ReplicateDecision with target's replication decision +func (d *ReplicateDecision) Set(t replicateTargetDecision) { + if d.targetsMap == nil { + d.targetsMap = make(map[string]replicateTargetDecision) + } + d.targetsMap[t.Arn] = t +} + +// PendingStatus returns a stringified representation of internal replication status with all targets marked as `PENDING` +func (d *ReplicateDecision) PendingStatus() string { + b := new(bytes.Buffer) + for _, k := range d.targetsMap { + if k.Replicate { + fmt.Fprintf(b, "%s=%s;", k.Arn, replication.Pending.String()) + } + } + return b.String() +} + +// ResyncDecision is a struct representing a map with target's individual resync decisions +type ResyncDecision struct { + targets map[string]ResyncTargetDecision +} + +// Empty returns true if no targets with resync decision present +func (r *ResyncDecision) Empty() bool { + return r.targets == nil +} + +func (r *ResyncDecision) mustResync() bool { + for _, v := range r.targets { + if v.Replicate { + return true + } + } + return false +} + +func (r *ResyncDecision) mustResyncTarget(tgtArn string) bool { + if r.targets == nil { + return false + } + v, ok := r.targets[tgtArn] + if ok && v.Replicate { + return true + } + return false +} + +// ResyncTargetDecision is struct that represents resync decision for this target +type ResyncTargetDecision struct { + Replicate bool + ResetID string + ResetBeforeDate time.Time +} + +var errInvalidReplicateDecisionFormat = fmt.Errorf("ReplicateDecision has invalid format") + +// parse k-v pairs of target ARN to stringified ReplicateTargetDecision delimited by ',' into a +// ReplicateDecision struct +func parseReplicateDecision(s string) (r ReplicateDecision, err error) { + r = ReplicateDecision{ + targetsMap: make(map[string]replicateTargetDecision), + } + if len(s) == 0 { + return + } + pairs := strings.Split(s, ",") + for _, p := range pairs { + slc := strings.Split(p, "=") + if len(slc) != 2 { + return r, errInvalidReplicateDecisionFormat + } + tgtStr := strings.TrimPrefix(slc[1], "\"") + tgtStr = strings.TrimSuffix(tgtStr, "\"") + tgt := strings.Split(tgtStr, ";") + if len(tgt) != 4 { + return r, errInvalidReplicateDecisionFormat + } + var replicate, sync bool + var err error + replicate, err = strconv.ParseBool(tgt[0]) + if err != nil { + return r, err + } + sync, err = strconv.ParseBool(tgt[1]) + if err != nil { + return r, err + } + r.targetsMap[slc[0]] = replicateTargetDecision{Replicate: replicate, Synchronous: sync, Arn: tgt[2], ID: tgt[3]} + } + return +} + +// ReplicationState represents internal replication state +type ReplicationState struct { + ReplicaTimeStamp time.Time // timestamp when last replica update was received + ReplicaStatus replication.StatusType // replica statusstringis + DeleteMarker bool // represents DeleteMarker replication state + ReplicationTimeStamp time.Time // timestamp when last replication activity happened + ReplicationStatusInternal string // stringified representation of all replication activity + // VersionPurgeStatusInternal is internally in the format "arn1=PENDING;arn2=COMMPLETED;" + VersionPurgeStatusInternal string // stringified representation of all version purge statuses + ReplicateDecisionStr string // stringified representation of replication decision for each target + Targets map[string]replication.StatusType // map of ARN->replication status for ongoing replication activity + PurgeTargets map[string]VersionPurgeStatusType // map of ARN->VersionPurgeStatus for all the targets + ResetStatusesMap map[string]string // map of ARN-> stringified reset id and timestamp for all the targets +} + +// Equal returns true if replication state is identical for version purge statuses and (replica)tion statuses. +func (rs *ReplicationState) Equal(o ReplicationState) bool { + return rs.ReplicaStatus == o.ReplicaStatus && + rs.ReplicaTimeStamp.Equal(o.ReplicaTimeStamp) && + rs.ReplicationTimeStamp.Equal(o.ReplicationTimeStamp) && + rs.ReplicationStatusInternal == o.ReplicationStatusInternal && + rs.VersionPurgeStatusInternal == o.VersionPurgeStatusInternal +} + +// CompositeReplicationStatus returns overall replication status for the object version being replicated. +func (rs *ReplicationState) CompositeReplicationStatus() (st replication.StatusType) { + switch { + case rs.ReplicationStatusInternal != "": + switch replication.StatusType(rs.ReplicationStatusInternal) { + case replication.Pending, replication.Completed, replication.Failed, replication.Replica: // for backward compatibility + return replication.StatusType(rs.ReplicationStatusInternal) + default: + replStatus := getCompositeReplicationStatus(rs.Targets) + // return REPLICA status if replica received timestamp is later than replication timestamp + // provided object replication completed for all targets. + if !rs.ReplicaTimeStamp.Equal(timeSentinel) && replStatus == replication.Completed && rs.ReplicaTimeStamp.After(rs.ReplicationTimeStamp) { + return rs.ReplicaStatus + } + return replStatus + } + case !rs.ReplicaStatus.Empty(): + return rs.ReplicaStatus + default: + return + } +} + +// CompositeVersionPurgeStatus returns overall replication purge status for the permanent delete being replicated. +func (rs *ReplicationState) CompositeVersionPurgeStatus() VersionPurgeStatusType { + switch VersionPurgeStatusType(rs.VersionPurgeStatusInternal) { + case Pending, Complete, Failed: // for backward compatibility + return VersionPurgeStatusType(rs.VersionPurgeStatusInternal) + default: + return getCompositeVersionPurgeStatus(rs.PurgeTargets) + } +} + +// TargetState returns replicatedInfos struct initialized with the previous state of replication +func (rs *ReplicationState) targetState(arn string) (r replicatedTargetInfo) { + return replicatedTargetInfo{ + Arn: arn, + PrevReplicationStatus: rs.Targets[arn], + VersionPurgeStatus: rs.PurgeTargets[arn], + ResyncTimestamp: rs.ResetStatusesMap[arn], + } +} + +// getReplicationState returns replication state using target replicated info for the targets +func getReplicationState(rinfos replicatedInfos, prevState ReplicationState, vID string) ReplicationState { + rs := ReplicationState{ + ReplicateDecisionStr: prevState.ReplicateDecisionStr, + ResetStatusesMap: prevState.ResetStatusesMap, + ReplicaTimeStamp: prevState.ReplicaTimeStamp, + ReplicaStatus: prevState.ReplicaStatus, + } + var replStatuses, vpurgeStatuses string + replStatuses = rinfos.ReplicationStatusInternal() + rs.Targets = replicationStatusesMap(replStatuses) + rs.ReplicationStatusInternal = replStatuses + rs.ReplicationTimeStamp = rinfos.ReplicationTimeStamp + + vpurgeStatuses = rinfos.VersionPurgeStatusInternal() + rs.VersionPurgeStatusInternal = vpurgeStatuses + rs.PurgeTargets = versionPurgeStatusesMap(vpurgeStatuses) + + for _, rinfo := range rinfos.Targets { + if rinfo.ResyncTimestamp != "" { + rs.ResetStatusesMap[targetResetHeader(rinfo.Arn)] = rinfo.ResyncTimestamp + } + } + return rs +} + +// constructs a replication status map from string representation +func replicationStatusesMap(s string) map[string]replication.StatusType { + targets := make(map[string]replication.StatusType) + repStatMatches := replStatusRegex.FindAllStringSubmatch(s, -1) + for _, repStatMatch := range repStatMatches { + if len(repStatMatch) != 3 { + continue + } + status := replication.StatusType(repStatMatch[2]) + targets[repStatMatch[1]] = status + } + return targets +} + +// constructs a version purge status map from string representation +func versionPurgeStatusesMap(s string) map[string]VersionPurgeStatusType { + targets := make(map[string]VersionPurgeStatusType) + purgeStatusMatches := replStatusRegex.FindAllStringSubmatch(s, -1) + for _, purgeStatusMatch := range purgeStatusMatches { + if len(purgeStatusMatch) != 3 { + continue + } + targets[purgeStatusMatch[1]] = VersionPurgeStatusType(purgeStatusMatch[2]) + } + return targets +} + +// return the overall replication status for all the targets +func getCompositeReplicationStatus(m map[string]replication.StatusType) replication.StatusType { + if len(m) == 0 { + return replication.StatusType("") + } + completed := 0 + for _, v := range m { + switch v { + case replication.Failed: + return replication.Failed + case replication.Completed: + completed++ + } + } + if completed == len(m) { + return replication.Completed + } + return replication.Pending +} + +// return the overall version purge status for all the targets +func getCompositeVersionPurgeStatus(m map[string]VersionPurgeStatusType) VersionPurgeStatusType { + if len(m) == 0 { + return VersionPurgeStatusType("") + } + completed := 0 + for _, v := range m { + switch v { + case Failed: + return Failed + case Complete: + completed++ + } + } + if completed == len(m) { + return Complete + } + return Pending +} + +// getHealReplicateObjectInfo returns info needed by heal replication in ReplicateObjectInfo +func getHealReplicateObjectInfo(objInfo ObjectInfo, rcfg replicationConfig) ReplicateObjectInfo { + oi := objInfo.Clone() + if rcfg.Config != nil && rcfg.Config.RoleArn != "" { + // For backward compatibility of objects pending/failed replication. + // Save replication related statuses in the new internal representation for + // compatible behavior. + if !oi.ReplicationStatus.Empty() { + oi.ReplicationStatusInternal = fmt.Sprintf("%s=%s;", rcfg.Config.RoleArn, oi.ReplicationStatus) + } + if !oi.VersionPurgeStatus.Empty() { + oi.VersionPurgeStatusInternal = fmt.Sprintf("%s=%s;", rcfg.Config.RoleArn, oi.VersionPurgeStatus) + } + for k, v := range oi.UserDefined { + switch { + case strings.EqualFold(k, ReservedMetadataPrefixLower+ReplicationReset): + delete(oi.UserDefined, k) + oi.UserDefined[targetResetHeader(rcfg.Config.RoleArn)] = v + } + } + } + var dsc ReplicateDecision + var tgtStatuses map[string]replication.StatusType + if oi.DeleteMarker { + dsc = checkReplicateDelete(GlobalContext, oi.Bucket, ObjectToDelete{ + ObjectName: oi.Name, + VersionID: oi.VersionID, + }, oi, ObjectOptions{}, nil) + } else { + dsc = mustReplicate(GlobalContext, oi.Bucket, oi.Name, getMustReplicateOptions(ObjectInfo{ + UserDefined: oi.UserDefined, + }, replication.HealReplicationType, ObjectOptions{})) + } + tgtStatuses = replicationStatusesMap(oi.ReplicationStatusInternal) + + existingObjResync := rcfg.Resync(GlobalContext, oi, &dsc, tgtStatuses) + + return ReplicateObjectInfo{ + ObjectInfo: oi, + OpType: replication.HealReplicationType, + Dsc: dsc, + ExistingObjResync: existingObjResync, + TargetStatuses: tgtStatuses, + } +} + +// vID here represents the versionID client specified in request - need to distinguish between delete marker and delete marker deletion +func (o *ObjectInfo) getReplicationState(dsc string, vID string, heal bool) ReplicationState { + rs := ReplicationState{ + ReplicationStatusInternal: o.ReplicationStatusInternal, + VersionPurgeStatusInternal: o.VersionPurgeStatusInternal, + ReplicateDecisionStr: dsc, + Targets: make(map[string]replication.StatusType), + PurgeTargets: make(map[string]VersionPurgeStatusType), + ResetStatusesMap: make(map[string]string), + } + rs.Targets = replicationStatusesMap(o.ReplicationStatusInternal) + rs.PurgeTargets = versionPurgeStatusesMap(o.VersionPurgeStatusInternal) + for k, v := range o.UserDefined { + if strings.HasPrefix(k, ReservedMetadataPrefixLower+ReplicationReset) { + arn := strings.TrimPrefix(k, fmt.Sprintf("%s-", ReservedMetadataPrefixLower+ReplicationReset)) + rs.ResetStatusesMap[arn] = v + } + } + return rs +} + +// ReplicationState returns replication state using other internal replication metadata in ObjectToDelete +func (o *ObjectToDelete) ReplicationState() ReplicationState { + r := ReplicationState{ + ReplicationStatusInternal: o.DeleteMarkerReplicationStatus, + VersionPurgeStatusInternal: o.VersionPurgeStatuses, + ReplicateDecisionStr: o.ReplicateDecisionStr, + } + + r.Targets = replicationStatusesMap(o.DeleteMarkerReplicationStatus) + r.PurgeTargets = versionPurgeStatusesMap(o.VersionPurgeStatuses) + return r +} + +// VersionPurgeStatus returns a composite version purge status across targets +func (d *DeletedObject) VersionPurgeStatus() VersionPurgeStatusType { + return d.ReplicationState.CompositeVersionPurgeStatus() +} + +// DeleteMarkerReplicationStatus return composite replication status of delete marker across targets +func (d *DeletedObject) DeleteMarkerReplicationStatus() replication.StatusType { + return d.ReplicationState.CompositeReplicationStatus() +} + +// ResyncTargetsInfo holds a slice of targets with resync info per target +type ResyncTargetsInfo struct { + Targets []ResyncTarget `json:"target,omitempty"` +} + +// ResyncTarget is a struct representing the Target reset ID where target is identified by its Arn +type ResyncTarget struct { + Arn string `json:"arn"` + ResetID string `json:"resetid"` +} + +// VersionPurgeStatusType represents status of a versioned delete or permanent delete w.r.t bucket replication +type VersionPurgeStatusType string + +const ( + // Pending - versioned delete replication is pending. + Pending VersionPurgeStatusType = "PENDING" + + // Complete - versioned delete replication is now complete, erase version on disk. + Complete VersionPurgeStatusType = "COMPLETE" + + // Failed - versioned delete replication failed. + Failed VersionPurgeStatusType = "FAILED" +) + +// Empty returns true if purge status was not set. +func (v VersionPurgeStatusType) Empty() bool { + return string(v) == "" +} + +// Pending returns true if the version is pending purge. +func (v VersionPurgeStatusType) Pending() bool { + return v == Pending || v == Failed +} diff --git a/cmd/bucket-replication-utils_gen.go b/cmd/bucket-replication-utils_gen.go new file mode 100644 index 000000000..c91b75772 --- /dev/null +++ b/cmd/bucket-replication-utils_gen.go @@ -0,0 +1,1285 @@ +package cmd + +// Code generated by github.com/tinylib/msgp DO NOT EDIT. + +import ( + "github.com/minio/minio/internal/bucket/replication" + "github.com/tinylib/msgp/msgp" +) + +// DecodeMsg implements msgp.Decodable +func (z *ReplicateDecision) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z ReplicateDecision) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 0 + err = en.Append(0x80) + if err != nil { + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z ReplicateDecision) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 0 + o = append(o, 0x80) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ReplicateDecision) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z ReplicateDecision) Msgsize() (s int) { + s = 1 + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ReplicationState) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "ReplicaTimeStamp": + z.ReplicaTimeStamp, err = dc.ReadTime() + if err != nil { + err = msgp.WrapError(err, "ReplicaTimeStamp") + return + } + case "ReplicaStatus": + err = z.ReplicaStatus.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "ReplicaStatus") + return + } + case "DeleteMarker": + z.DeleteMarker, err = dc.ReadBool() + if err != nil { + err = msgp.WrapError(err, "DeleteMarker") + return + } + case "ReplicationTimeStamp": + z.ReplicationTimeStamp, err = dc.ReadTime() + if err != nil { + err = msgp.WrapError(err, "ReplicationTimeStamp") + return + } + case "ReplicationStatusInternal": + z.ReplicationStatusInternal, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "ReplicationStatusInternal") + return + } + case "VersionPurgeStatusInternal": + z.VersionPurgeStatusInternal, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "VersionPurgeStatusInternal") + return + } + case "ReplicateDecisionStr": + z.ReplicateDecisionStr, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "ReplicateDecisionStr") + return + } + case "Targets": + var zb0002 uint32 + zb0002, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + if z.Targets == nil { + z.Targets = make(map[string]replication.StatusType, zb0002) + } else if len(z.Targets) > 0 { + for key := range z.Targets { + delete(z.Targets, key) + } + } + for zb0002 > 0 { + zb0002-- + var za0001 string + var za0002 replication.StatusType + za0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + err = za0002.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + z.Targets[za0001] = za0002 + } + case "PurgeTargets": + var zb0003 uint32 + zb0003, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "PurgeTargets") + return + } + if z.PurgeTargets == nil { + z.PurgeTargets = make(map[string]VersionPurgeStatusType, zb0003) + } else if len(z.PurgeTargets) > 0 { + for key := range z.PurgeTargets { + delete(z.PurgeTargets, key) + } + } + for zb0003 > 0 { + zb0003-- + var za0003 string + var za0004 VersionPurgeStatusType + za0003, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "PurgeTargets") + return + } + { + var zb0004 string + zb0004, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "PurgeTargets", za0003) + return + } + za0004 = VersionPurgeStatusType(zb0004) + } + z.PurgeTargets[za0003] = za0004 + } + case "ResetStatusesMap": + var zb0005 uint32 + zb0005, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap") + return + } + if z.ResetStatusesMap == nil { + z.ResetStatusesMap = make(map[string]string, zb0005) + } else if len(z.ResetStatusesMap) > 0 { + for key := range z.ResetStatusesMap { + delete(z.ResetStatusesMap, key) + } + } + for zb0005 > 0 { + zb0005-- + var za0005 string + var za0006 string + za0005, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap") + return + } + za0006, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap", za0005) + return + } + z.ResetStatusesMap[za0005] = za0006 + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *ReplicationState) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 10 + // write "ReplicaTimeStamp" + err = en.Append(0x8a, 0xb0, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x74, 0x61, 0x6d, 0x70) + if err != nil { + return + } + err = en.WriteTime(z.ReplicaTimeStamp) + if err != nil { + err = msgp.WrapError(err, "ReplicaTimeStamp") + return + } + // write "ReplicaStatus" + err = en.Append(0xad, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73) + if err != nil { + return + } + err = z.ReplicaStatus.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "ReplicaStatus") + return + } + // write "DeleteMarker" + err = en.Append(0xac, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72) + if err != nil { + return + } + err = en.WriteBool(z.DeleteMarker) + if err != nil { + err = msgp.WrapError(err, "DeleteMarker") + return + } + // write "ReplicationTimeStamp" + err = en.Append(0xb4, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x74, 0x61, 0x6d, 0x70) + if err != nil { + return + } + err = en.WriteTime(z.ReplicationTimeStamp) + if err != nil { + err = msgp.WrapError(err, "ReplicationTimeStamp") + return + } + // write "ReplicationStatusInternal" + err = en.Append(0xb9, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c) + if err != nil { + return + } + err = en.WriteString(z.ReplicationStatusInternal) + if err != nil { + err = msgp.WrapError(err, "ReplicationStatusInternal") + return + } + // write "VersionPurgeStatusInternal" + err = en.Append(0xba, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x75, 0x72, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c) + if err != nil { + return + } + err = en.WriteString(z.VersionPurgeStatusInternal) + if err != nil { + err = msgp.WrapError(err, "VersionPurgeStatusInternal") + return + } + // write "ReplicateDecisionStr" + err = en.Append(0xb4, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x44, 0x65, 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x72) + if err != nil { + return + } + err = en.WriteString(z.ReplicateDecisionStr) + if err != nil { + err = msgp.WrapError(err, "ReplicateDecisionStr") + return + } + // write "Targets" + err = en.Append(0xa7, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73) + if err != nil { + return + } + err = en.WriteMapHeader(uint32(len(z.Targets))) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + for za0001, za0002 := range z.Targets { + err = en.WriteString(za0001) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + err = za0002.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + } + // write "PurgeTargets" + err = en.Append(0xac, 0x50, 0x75, 0x72, 0x67, 0x65, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73) + if err != nil { + return + } + err = en.WriteMapHeader(uint32(len(z.PurgeTargets))) + if err != nil { + err = msgp.WrapError(err, "PurgeTargets") + return + } + for za0003, za0004 := range z.PurgeTargets { + err = en.WriteString(za0003) + if err != nil { + err = msgp.WrapError(err, "PurgeTargets") + return + } + err = en.WriteString(string(za0004)) + if err != nil { + err = msgp.WrapError(err, "PurgeTargets", za0003) + return + } + } + // write "ResetStatusesMap" + err = en.Append(0xb0, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x4d, 0x61, 0x70) + if err != nil { + return + } + err = en.WriteMapHeader(uint32(len(z.ResetStatusesMap))) + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap") + return + } + for za0005, za0006 := range z.ResetStatusesMap { + err = en.WriteString(za0005) + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap") + return + } + err = en.WriteString(za0006) + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap", za0005) + return + } + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *ReplicationState) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 10 + // string "ReplicaTimeStamp" + o = append(o, 0x8a, 0xb0, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x74, 0x61, 0x6d, 0x70) + o = msgp.AppendTime(o, z.ReplicaTimeStamp) + // string "ReplicaStatus" + o = append(o, 0xad, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73) + o, err = z.ReplicaStatus.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "ReplicaStatus") + return + } + // string "DeleteMarker" + o = append(o, 0xac, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72) + o = msgp.AppendBool(o, z.DeleteMarker) + // string "ReplicationTimeStamp" + o = append(o, 0xb4, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x74, 0x61, 0x6d, 0x70) + o = msgp.AppendTime(o, z.ReplicationTimeStamp) + // string "ReplicationStatusInternal" + o = append(o, 0xb9, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c) + o = msgp.AppendString(o, z.ReplicationStatusInternal) + // string "VersionPurgeStatusInternal" + o = append(o, 0xba, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x50, 0x75, 0x72, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c) + o = msgp.AppendString(o, z.VersionPurgeStatusInternal) + // string "ReplicateDecisionStr" + o = append(o, 0xb4, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x44, 0x65, 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x72) + o = msgp.AppendString(o, z.ReplicateDecisionStr) + // string "Targets" + o = append(o, 0xa7, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73) + o = msgp.AppendMapHeader(o, uint32(len(z.Targets))) + for za0001, za0002 := range z.Targets { + o = msgp.AppendString(o, za0001) + o, err = za0002.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + } + // string "PurgeTargets" + o = append(o, 0xac, 0x50, 0x75, 0x72, 0x67, 0x65, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73) + o = msgp.AppendMapHeader(o, uint32(len(z.PurgeTargets))) + for za0003, za0004 := range z.PurgeTargets { + o = msgp.AppendString(o, za0003) + o = msgp.AppendString(o, string(za0004)) + } + // string "ResetStatusesMap" + o = append(o, 0xb0, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x4d, 0x61, 0x70) + o = msgp.AppendMapHeader(o, uint32(len(z.ResetStatusesMap))) + for za0005, za0006 := range z.ResetStatusesMap { + o = msgp.AppendString(o, za0005) + o = msgp.AppendString(o, za0006) + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ReplicationState) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "ReplicaTimeStamp": + z.ReplicaTimeStamp, bts, err = msgp.ReadTimeBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaTimeStamp") + return + } + case "ReplicaStatus": + bts, err = z.ReplicaStatus.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaStatus") + return + } + case "DeleteMarker": + z.DeleteMarker, bts, err = msgp.ReadBoolBytes(bts) + if err != nil { + err = msgp.WrapError(err, "DeleteMarker") + return + } + case "ReplicationTimeStamp": + z.ReplicationTimeStamp, bts, err = msgp.ReadTimeBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicationTimeStamp") + return + } + case "ReplicationStatusInternal": + z.ReplicationStatusInternal, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicationStatusInternal") + return + } + case "VersionPurgeStatusInternal": + z.VersionPurgeStatusInternal, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "VersionPurgeStatusInternal") + return + } + case "ReplicateDecisionStr": + z.ReplicateDecisionStr, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicateDecisionStr") + return + } + case "Targets": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + if z.Targets == nil { + z.Targets = make(map[string]replication.StatusType, zb0002) + } else if len(z.Targets) > 0 { + for key := range z.Targets { + delete(z.Targets, key) + } + } + for zb0002 > 0 { + var za0001 string + var za0002 replication.StatusType + zb0002-- + za0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + bts, err = za0002.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + z.Targets[za0001] = za0002 + } + case "PurgeTargets": + var zb0003 uint32 + zb0003, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "PurgeTargets") + return + } + if z.PurgeTargets == nil { + z.PurgeTargets = make(map[string]VersionPurgeStatusType, zb0003) + } else if len(z.PurgeTargets) > 0 { + for key := range z.PurgeTargets { + delete(z.PurgeTargets, key) + } + } + for zb0003 > 0 { + var za0003 string + var za0004 VersionPurgeStatusType + zb0003-- + za0003, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "PurgeTargets") + return + } + { + var zb0004 string + zb0004, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "PurgeTargets", za0003) + return + } + za0004 = VersionPurgeStatusType(zb0004) + } + z.PurgeTargets[za0003] = za0004 + } + case "ResetStatusesMap": + var zb0005 uint32 + zb0005, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap") + return + } + if z.ResetStatusesMap == nil { + z.ResetStatusesMap = make(map[string]string, zb0005) + } else if len(z.ResetStatusesMap) > 0 { + for key := range z.ResetStatusesMap { + delete(z.ResetStatusesMap, key) + } + } + for zb0005 > 0 { + var za0005 string + var za0006 string + zb0005-- + za0005, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap") + return + } + za0006, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ResetStatusesMap", za0005) + return + } + z.ResetStatusesMap[za0005] = za0006 + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *ReplicationState) Msgsize() (s int) { + s = 1 + 17 + msgp.TimeSize + 14 + z.ReplicaStatus.Msgsize() + 13 + msgp.BoolSize + 21 + msgp.TimeSize + 26 + msgp.StringPrefixSize + len(z.ReplicationStatusInternal) + 27 + msgp.StringPrefixSize + len(z.VersionPurgeStatusInternal) + 21 + msgp.StringPrefixSize + len(z.ReplicateDecisionStr) + 8 + msgp.MapHeaderSize + if z.Targets != nil { + for za0001, za0002 := range z.Targets { + _ = za0002 + s += msgp.StringPrefixSize + len(za0001) + za0002.Msgsize() + } + } + s += 13 + msgp.MapHeaderSize + if z.PurgeTargets != nil { + for za0003, za0004 := range z.PurgeTargets { + _ = za0004 + s += msgp.StringPrefixSize + len(za0003) + msgp.StringPrefixSize + len(string(za0004)) + } + } + s += 17 + msgp.MapHeaderSize + if z.ResetStatusesMap != nil { + for za0005, za0006 := range z.ResetStatusesMap { + _ = za0006 + s += msgp.StringPrefixSize + len(za0005) + msgp.StringPrefixSize + len(za0006) + } + } + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ResyncDecision) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z ResyncDecision) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 0 + err = en.Append(0x80) + if err != nil { + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z ResyncDecision) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 0 + o = append(o, 0x80) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ResyncDecision) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z ResyncDecision) Msgsize() (s int) { + s = 1 + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ResyncTarget) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Arn": + z.Arn, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Arn") + return + } + case "ResetID": + z.ResetID, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "ResetID") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z ResyncTarget) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 2 + // write "Arn" + err = en.Append(0x82, 0xa3, 0x41, 0x72, 0x6e) + if err != nil { + return + } + err = en.WriteString(z.Arn) + if err != nil { + err = msgp.WrapError(err, "Arn") + return + } + // write "ResetID" + err = en.Append(0xa7, 0x52, 0x65, 0x73, 0x65, 0x74, 0x49, 0x44) + if err != nil { + return + } + err = en.WriteString(z.ResetID) + if err != nil { + err = msgp.WrapError(err, "ResetID") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z ResyncTarget) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 2 + // string "Arn" + o = append(o, 0x82, 0xa3, 0x41, 0x72, 0x6e) + o = msgp.AppendString(o, z.Arn) + // string "ResetID" + o = append(o, 0xa7, 0x52, 0x65, 0x73, 0x65, 0x74, 0x49, 0x44) + o = msgp.AppendString(o, z.ResetID) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ResyncTarget) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Arn": + z.Arn, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Arn") + return + } + case "ResetID": + z.ResetID, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ResetID") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z ResyncTarget) Msgsize() (s int) { + s = 1 + 4 + msgp.StringPrefixSize + len(z.Arn) + 8 + msgp.StringPrefixSize + len(z.ResetID) + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ResyncTargetDecision) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Replicate": + z.Replicate, err = dc.ReadBool() + if err != nil { + err = msgp.WrapError(err, "Replicate") + return + } + case "ResetID": + z.ResetID, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "ResetID") + return + } + case "ResetBeforeDate": + z.ResetBeforeDate, err = dc.ReadTime() + if err != nil { + err = msgp.WrapError(err, "ResetBeforeDate") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z ResyncTargetDecision) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 3 + // write "Replicate" + err = en.Append(0x83, 0xa9, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65) + if err != nil { + return + } + err = en.WriteBool(z.Replicate) + if err != nil { + err = msgp.WrapError(err, "Replicate") + return + } + // write "ResetID" + err = en.Append(0xa7, 0x52, 0x65, 0x73, 0x65, 0x74, 0x49, 0x44) + if err != nil { + return + } + err = en.WriteString(z.ResetID) + if err != nil { + err = msgp.WrapError(err, "ResetID") + return + } + // write "ResetBeforeDate" + err = en.Append(0xaf, 0x52, 0x65, 0x73, 0x65, 0x74, 0x42, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x44, 0x61, 0x74, 0x65) + if err != nil { + return + } + err = en.WriteTime(z.ResetBeforeDate) + if err != nil { + err = msgp.WrapError(err, "ResetBeforeDate") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z ResyncTargetDecision) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 3 + // string "Replicate" + o = append(o, 0x83, 0xa9, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65) + o = msgp.AppendBool(o, z.Replicate) + // string "ResetID" + o = append(o, 0xa7, 0x52, 0x65, 0x73, 0x65, 0x74, 0x49, 0x44) + o = msgp.AppendString(o, z.ResetID) + // string "ResetBeforeDate" + o = append(o, 0xaf, 0x52, 0x65, 0x73, 0x65, 0x74, 0x42, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x44, 0x61, 0x74, 0x65) + o = msgp.AppendTime(o, z.ResetBeforeDate) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ResyncTargetDecision) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Replicate": + z.Replicate, bts, err = msgp.ReadBoolBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Replicate") + return + } + case "ResetID": + z.ResetID, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ResetID") + return + } + case "ResetBeforeDate": + z.ResetBeforeDate, bts, err = msgp.ReadTimeBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ResetBeforeDate") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z ResyncTargetDecision) Msgsize() (s int) { + s = 1 + 10 + msgp.BoolSize + 8 + msgp.StringPrefixSize + len(z.ResetID) + 16 + msgp.TimeSize + return +} + +// DecodeMsg implements msgp.Decodable +func (z *ResyncTargetsInfo) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Targets": + var zb0002 uint32 + zb0002, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + if cap(z.Targets) >= int(zb0002) { + z.Targets = (z.Targets)[:zb0002] + } else { + z.Targets = make([]ResyncTarget, zb0002) + } + for za0001 := range z.Targets { + var zb0003 uint32 + zb0003, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + for zb0003 > 0 { + zb0003-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + switch msgp.UnsafeString(field) { + case "Arn": + z.Targets[za0001].Arn, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Targets", za0001, "Arn") + return + } + case "ResetID": + z.Targets[za0001].ResetID, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Targets", za0001, "ResetID") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + } + } + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *ResyncTargetsInfo) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 1 + // write "Targets" + err = en.Append(0x81, 0xa7, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73) + if err != nil { + return + } + err = en.WriteArrayHeader(uint32(len(z.Targets))) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + for za0001 := range z.Targets { + // map header, size 2 + // write "Arn" + err = en.Append(0x82, 0xa3, 0x41, 0x72, 0x6e) + if err != nil { + return + } + err = en.WriteString(z.Targets[za0001].Arn) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001, "Arn") + return + } + // write "ResetID" + err = en.Append(0xa7, 0x52, 0x65, 0x73, 0x65, 0x74, 0x49, 0x44) + if err != nil { + return + } + err = en.WriteString(z.Targets[za0001].ResetID) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001, "ResetID") + return + } + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *ResyncTargetsInfo) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 1 + // string "Targets" + o = append(o, 0x81, 0xa7, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73) + o = msgp.AppendArrayHeader(o, uint32(len(z.Targets))) + for za0001 := range z.Targets { + // map header, size 2 + // string "Arn" + o = append(o, 0x82, 0xa3, 0x41, 0x72, 0x6e) + o = msgp.AppendString(o, z.Targets[za0001].Arn) + // string "ResetID" + o = append(o, 0xa7, 0x52, 0x65, 0x73, 0x65, 0x74, 0x49, 0x44) + o = msgp.AppendString(o, z.Targets[za0001].ResetID) + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *ResyncTargetsInfo) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Targets": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + if cap(z.Targets) >= int(zb0002) { + z.Targets = (z.Targets)[:zb0002] + } else { + z.Targets = make([]ResyncTarget, zb0002) + } + for za0001 := range z.Targets { + var zb0003 uint32 + zb0003, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + for zb0003 > 0 { + zb0003-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + switch msgp.UnsafeString(field) { + case "Arn": + z.Targets[za0001].Arn, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001, "Arn") + return + } + case "ResetID": + z.Targets[za0001].ResetID, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001, "ResetID") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + } + } + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *ResyncTargetsInfo) Msgsize() (s int) { + s = 1 + 8 + msgp.ArrayHeaderSize + for za0001 := range z.Targets { + s += 1 + 4 + msgp.StringPrefixSize + len(z.Targets[za0001].Arn) + 8 + msgp.StringPrefixSize + len(z.Targets[za0001].ResetID) + } + return +} + +// DecodeMsg implements msgp.Decodable +func (z *VersionPurgeStatusType) DecodeMsg(dc *msgp.Reader) (err error) { + { + var zb0001 string + zb0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err) + return + } + (*z) = VersionPurgeStatusType(zb0001) + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z VersionPurgeStatusType) EncodeMsg(en *msgp.Writer) (err error) { + err = en.WriteString(string(z)) + if err != nil { + err = msgp.WrapError(err) + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z VersionPurgeStatusType) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + o = msgp.AppendString(o, string(z)) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *VersionPurgeStatusType) UnmarshalMsg(bts []byte) (o []byte, err error) { + { + var zb0001 string + zb0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + (*z) = VersionPurgeStatusType(zb0001) + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z VersionPurgeStatusType) Msgsize() (s int) { + s = msgp.StringPrefixSize + len(string(z)) + return +} diff --git a/cmd/bucket-replication-utils_gen_test.go b/cmd/bucket-replication-utils_gen_test.go new file mode 100644 index 000000000..51390211f --- /dev/null +++ b/cmd/bucket-replication-utils_gen_test.go @@ -0,0 +1,688 @@ +package cmd + +// Code generated by github.com/tinylib/msgp DO NOT EDIT. + +import ( + "bytes" + "testing" + + "github.com/tinylib/msgp/msgp" +) + +func TestMarshalUnmarshalReplicateDecision(t *testing.T) { + v := ReplicateDecision{} + 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 BenchmarkMarshalMsgReplicateDecision(b *testing.B) { + v := ReplicateDecision{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgReplicateDecision(b *testing.B) { + v := ReplicateDecision{} + 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 BenchmarkUnmarshalReplicateDecision(b *testing.B) { + v := ReplicateDecision{} + 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 TestEncodeDecodeReplicateDecision(t *testing.T) { + v := ReplicateDecision{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeReplicateDecision Msgsize() is inaccurate") + } + + vn := ReplicateDecision{} + 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 BenchmarkEncodeReplicateDecision(b *testing.B) { + v := ReplicateDecision{} + 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 BenchmarkDecodeReplicateDecision(b *testing.B) { + v := ReplicateDecision{} + 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 TestMarshalUnmarshalReplicationState(t *testing.T) { + v := ReplicationState{} + 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 BenchmarkMarshalMsgReplicationState(b *testing.B) { + v := ReplicationState{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgReplicationState(b *testing.B) { + v := ReplicationState{} + 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 BenchmarkUnmarshalReplicationState(b *testing.B) { + v := ReplicationState{} + 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 TestEncodeDecodeReplicationState(t *testing.T) { + v := ReplicationState{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeReplicationState Msgsize() is inaccurate") + } + + vn := ReplicationState{} + 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 BenchmarkEncodeReplicationState(b *testing.B) { + v := ReplicationState{} + 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 BenchmarkDecodeReplicationState(b *testing.B) { + v := ReplicationState{} + 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 TestMarshalUnmarshalResyncDecision(t *testing.T) { + v := ResyncDecision{} + 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 BenchmarkMarshalMsgResyncDecision(b *testing.B) { + v := ResyncDecision{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgResyncDecision(b *testing.B) { + v := ResyncDecision{} + 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 BenchmarkUnmarshalResyncDecision(b *testing.B) { + v := ResyncDecision{} + 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 TestEncodeDecodeResyncDecision(t *testing.T) { + v := ResyncDecision{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeResyncDecision Msgsize() is inaccurate") + } + + vn := ResyncDecision{} + 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 BenchmarkEncodeResyncDecision(b *testing.B) { + v := ResyncDecision{} + 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 BenchmarkDecodeResyncDecision(b *testing.B) { + v := ResyncDecision{} + 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 TestMarshalUnmarshalResyncTarget(t *testing.T) { + v := ResyncTarget{} + 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 BenchmarkMarshalMsgResyncTarget(b *testing.B) { + v := ResyncTarget{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgResyncTarget(b *testing.B) { + v := ResyncTarget{} + 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 BenchmarkUnmarshalResyncTarget(b *testing.B) { + v := ResyncTarget{} + 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 TestEncodeDecodeResyncTarget(t *testing.T) { + v := ResyncTarget{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeResyncTarget Msgsize() is inaccurate") + } + + vn := ResyncTarget{} + 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 BenchmarkEncodeResyncTarget(b *testing.B) { + v := ResyncTarget{} + 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 BenchmarkDecodeResyncTarget(b *testing.B) { + v := ResyncTarget{} + 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 TestMarshalUnmarshalResyncTargetDecision(t *testing.T) { + v := ResyncTargetDecision{} + 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 BenchmarkMarshalMsgResyncTargetDecision(b *testing.B) { + v := ResyncTargetDecision{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgResyncTargetDecision(b *testing.B) { + v := ResyncTargetDecision{} + 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 BenchmarkUnmarshalResyncTargetDecision(b *testing.B) { + v := ResyncTargetDecision{} + 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 TestEncodeDecodeResyncTargetDecision(t *testing.T) { + v := ResyncTargetDecision{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeResyncTargetDecision Msgsize() is inaccurate") + } + + vn := ResyncTargetDecision{} + 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 BenchmarkEncodeResyncTargetDecision(b *testing.B) { + v := ResyncTargetDecision{} + 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 BenchmarkDecodeResyncTargetDecision(b *testing.B) { + v := ResyncTargetDecision{} + 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 TestMarshalUnmarshalResyncTargetsInfo(t *testing.T) { + v := ResyncTargetsInfo{} + 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 BenchmarkMarshalMsgResyncTargetsInfo(b *testing.B) { + v := ResyncTargetsInfo{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgResyncTargetsInfo(b *testing.B) { + v := ResyncTargetsInfo{} + 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 BenchmarkUnmarshalResyncTargetsInfo(b *testing.B) { + v := ResyncTargetsInfo{} + 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 TestEncodeDecodeResyncTargetsInfo(t *testing.T) { + v := ResyncTargetsInfo{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeResyncTargetsInfo Msgsize() is inaccurate") + } + + vn := ResyncTargetsInfo{} + 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 BenchmarkEncodeResyncTargetsInfo(b *testing.B) { + v := ResyncTargetsInfo{} + 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 BenchmarkDecodeResyncTargetsInfo(b *testing.B) { + v := ResyncTargetsInfo{} + 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) + } + } +} diff --git a/cmd/bucket-replication-utils_test.go b/cmd/bucket-replication-utils_test.go new file mode 100644 index 000000000..8737c5c23 --- /dev/null +++ b/cmd/bucket-replication-utils_test.go @@ -0,0 +1,247 @@ +// Copyright (c) 2015-2021 MinIO, Inc. +// +// This file is part of MinIO Object Storage stack +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// This program is distributed in the hope that it will be useful +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +package cmd + +import ( + "testing" + + "github.com/minio/minio/internal/bucket/replication" +) + +var replicatedInfosTests = []struct { + name string + tgtInfos []replicatedTargetInfo + expectedCompletedSize int64 + expectedReplicationStatusInternal string + expectedReplicationStatus replication.StatusType + expectedOpType replication.Type + expectedAction replicationAction +}{ + { //1. empty tgtInfos slice + name: "no replicated targets", + tgtInfos: []replicatedTargetInfo{}, + expectedCompletedSize: 0, + expectedReplicationStatusInternal: "", + expectedReplicationStatus: replication.StatusType(""), + expectedOpType: replication.UnsetReplicationType, + expectedAction: replicateNone, + }, + { //2. replication completed to single target + name: "replication completed to single target", + tgtInfos: []replicatedTargetInfo{ + { + Arn: "arn1", + Size: 249, + PrevReplicationStatus: replication.Pending, + ReplicationStatus: replication.Completed, + OpType: replication.ObjectReplicationType, + ReplicationAction: replicateAll, + }, + }, + expectedCompletedSize: 249, + expectedReplicationStatusInternal: "arn1=COMPLETED;", + expectedReplicationStatus: replication.Completed, + expectedOpType: replication.ObjectReplicationType, + expectedAction: replicateAll, + }, + { //3. replication completed to single target; failed to another + name: "replication completed to single target", + tgtInfos: []replicatedTargetInfo{ + { + Arn: "arn1", + Size: 249, + PrevReplicationStatus: replication.Pending, + ReplicationStatus: replication.Completed, + OpType: replication.ObjectReplicationType, + ReplicationAction: replicateAll, + }, + { + Arn: "arn2", + Size: 249, + PrevReplicationStatus: replication.Pending, + ReplicationStatus: replication.Failed, + OpType: replication.ObjectReplicationType, + ReplicationAction: replicateAll, + }}, + expectedCompletedSize: 249, + expectedReplicationStatusInternal: "arn1=COMPLETED;arn2=FAILED;", + expectedReplicationStatus: replication.Failed, + expectedOpType: replication.ObjectReplicationType, + expectedAction: replicateAll, + }, + { //4. replication pending on one target; failed to another + name: "replication completed to single target", + tgtInfos: []replicatedTargetInfo{ + { + Arn: "arn1", + Size: 249, + PrevReplicationStatus: replication.Pending, + ReplicationStatus: replication.Pending, + OpType: replication.ObjectReplicationType, + ReplicationAction: replicateAll, + }, + { + Arn: "arn2", + Size: 249, + PrevReplicationStatus: replication.Pending, + ReplicationStatus: replication.Failed, + OpType: replication.ObjectReplicationType, + ReplicationAction: replicateAll, + }}, + expectedCompletedSize: 0, + expectedReplicationStatusInternal: "arn1=PENDING;arn2=FAILED;", + expectedReplicationStatus: replication.Failed, + expectedOpType: replication.ObjectReplicationType, + expectedAction: replicateAll, + }, +} + +func TestReplicatedInfos(t *testing.T) { + for i, test := range replicatedInfosTests { + rinfos := replicatedInfos{ + Targets: test.tgtInfos, + } + if actualSize := rinfos.CompletedSize(); actualSize != test.expectedCompletedSize { + t.Errorf("Test%d (%s): Size got %d , want %d", i+1, test.name, actualSize, test.expectedCompletedSize) + } + if repStatusStr := rinfos.ReplicationStatusInternal(); repStatusStr != test.expectedReplicationStatusInternal { + t.Errorf("Test%d (%s): Internal replication status got %s , want %s", i+1, test.name, repStatusStr, test.expectedReplicationStatusInternal) + } + if repStatus := rinfos.ReplicationStatus(); repStatus != test.expectedReplicationStatus { + t.Errorf("Test%d (%s): ReplicationStatus got %s , want %s", i+1, test.name, repStatus, test.expectedReplicationStatus) + } + if action := rinfos.Action(); action != test.expectedAction { + t.Errorf("Test%d (%s): Action got %s , want %s", i+1, test.name, action, test.expectedAction) + } + } +} + +var parseReplicationDecisionTest = []struct { + name string + dsc string + expDsc ReplicateDecision + expErr error +}{ + { //1. + name: "empty string", + dsc: "", + expDsc: ReplicateDecision{ + targetsMap: map[string]replicateTargetDecision{}, + }, + expErr: nil, + }, + + { //2. + name: "replicate decision for one target", + dsc: "arn:minio:replication::id:bucket=true;false;arn:minio:replication::id:bucket;id", + expErr: nil, + expDsc: ReplicateDecision{ + targetsMap: map[string]replicateTargetDecision{ + "arn:minio:replication::id:bucket": newReplicateTargetDecision("arn:minio:replication::id:bucket", true, false), + }, + }, + }, + { //3. + name: "replicate decision for multiple targets", + dsc: "arn:minio:replication::id:bucket=true;false;arn:minio:replication::id:bucket;id,arn:minio:replication::id2:bucket=false;true;arn:minio:replication::id2:bucket;id2", + expErr: nil, + expDsc: ReplicateDecision{ + targetsMap: map[string]replicateTargetDecision{ + "arn:minio:replication::id:bucket": newReplicateTargetDecision("arn:minio:replication::id:bucket", true, false), + "arn:minio:replication::id2:bucket": newReplicateTargetDecision("arn:minio:replication::id2:bucket", false, true), + }, + }, + }, + { //4. + name: "invalid format replicate decision for one target", + dsc: "arn:minio:replication::id:bucket:true;false;arn:minio:replication::id:bucket;id", + expErr: errInvalidReplicateDecisionFormat, + expDsc: ReplicateDecision{ + targetsMap: map[string]replicateTargetDecision{ + "arn:minio:replication::id:bucket": newReplicateTargetDecision("arn:minio:replication::id:bucket", true, false), + }, + }, + }, +} + +func TestParseReplicateDecision(t *testing.T) { + for i, test := range parseReplicationDecisionTest { + //dsc, err := parseReplicateDecision(test.dsc) + dsc, err := parseReplicateDecision(test.expDsc.String()) + + if err != nil { + if test.expErr != err { + t.Errorf("Test%d (%s): Expected parse error got %t , want %t", i+1, test.name, err, test.expErr) + } + continue + } + if len(dsc.targetsMap) != len(test.expDsc.targetsMap) { + t.Errorf("Test%d (%s): Invalid number of entries in targetsMap got %d , want %d", i+1, test.name, len(dsc.targetsMap), len(test.expDsc.targetsMap)) + } + for arn, tdsc := range dsc.targetsMap { + expDsc, ok := test.expDsc.targetsMap[arn] + if !ok || expDsc != tdsc { + t.Errorf("Test%d (%s): Invalid target replicate decision: got %+v, want %+v", i+1, test.name, tdsc, expDsc) + } + } + } +} + +var replicationStateTest = []struct { + name string + rs ReplicationState + arn string + expStatus replication.StatusType +}{ + { //1. no replication status header + name: "no replicated targets", + rs: ReplicationState{}, + expStatus: replication.StatusType(""), + }, + { //2. replication status for one target + name: "replication status for one target", + rs: ReplicationState{ReplicationStatusInternal: "arn1=PENDING;", Targets: map[string]replication.StatusType{"arn1": "PENDING"}}, + expStatus: replication.Pending, + }, + { //3. replication status for one target - incorrect format + name: "replication status for one target", + rs: ReplicationState{ReplicationStatusInternal: "arn1=PENDING"}, + expStatus: replication.StatusType(""), + }, + { //4. replication status for 3 targets, one of them failed + name: "replication status for 3 targets - one failed", + rs: ReplicationState{ + ReplicationStatusInternal: "arn1=COMPLETED;arn2=COMPLETED;arn3=FAILED;", + Targets: map[string]replication.StatusType{"arn1": "COMPLETED", "arn2": "COMPLETED", "arn3": "FAILED"}, + }, + expStatus: replication.Failed, + }, + { //5. replication status for replica version + name: "replication status for replica version", + rs: ReplicationState{ReplicationStatusInternal: string(replication.Replica)}, + expStatus: replication.Replica, + }, +} + +func TestCompositeReplicationStatus(t *testing.T) { + for i, test := range replicationStateTest { + if rstatus := test.rs.CompositeReplicationStatus(); rstatus != test.expStatus { + t.Errorf("Test%d (%s): Overall replication status got %s , want %s", i+1, test.name, rstatus, test.expStatus) + } + } +} diff --git a/cmd/bucket-replication.go b/cmd/bucket-replication.go index 979c3924b..7d50b41f0 100644 --- a/cmd/bucket-replication.go +++ b/cmd/bucket-replication.go @@ -42,17 +42,36 @@ import ( "github.com/minio/minio/internal/logger" ) -const throttleDeadline = 1 * time.Hour +const ( + throttleDeadline = 1 * time.Hour + // ReplicationReset has reset id and timestamp of last reset operation + ReplicationReset = "replication-reset" + // ReplicationStatus has internal replication status - stringified representation of target's replication status for all replication + // activity initiated from this cluster + ReplicationStatus = "replication-status" + // ReplicationTimestamp - the last time replication was initiated on this cluster for this object version + ReplicationTimestamp = "replication-timestamp" + // ReplicaStatus - this header is present if a replica was received by this cluster for this object version + ReplicaStatus = "replica-status" + // ReplicaTimestamp - the last time a replica was received by this cluster for this object version + ReplicaTimestamp = "replica-timestamp" + // TaggingTimestamp - the last time a tag metadata modification happened on this cluster for this object version + TaggingTimestamp = "tagging-timestamp" + // ObjectLockRetentionTimestamp - the last time a object lock metadata modification happened on this cluster for this object version + ObjectLockRetentionTimestamp = "objectlock-retention-timestamp" + // ObjectLockLegalHoldTimestamp - the last time a legal hold metadata modification happened on this cluster for this object version + ObjectLockLegalHoldTimestamp = "objectlock-legalhold-timestamp" +) // gets replication config associated to a given bucket name. func getReplicationConfig(ctx context.Context, bucketName string) (rc *replication.Config, err error) { if globalIsGateway { objAPI := newObjectLayerFn() if objAPI == nil { - return nil, errServerNotInitialized + return rc, errServerNotInitialized } - return nil, BucketReplicationConfigNotFound{Bucket: bucketName} + return rc, BucketReplicationConfigNotFound{Bucket: bucketName} } return globalBucketMetadataSys.GetReplicationConfig(ctx, bucketName) @@ -61,43 +80,54 @@ func getReplicationConfig(ctx context.Context, bucketName string) (rc *replicati // validateReplicationDestination returns error if replication destination bucket missing or not configured // It also returns true if replication destination is same as this server. func validateReplicationDestination(ctx context.Context, bucket string, rCfg *replication.Config) (bool, error) { - arn, err := madmin.ParseARN(rCfg.RoleArn) - if err != nil { - return false, BucketRemoteArnInvalid{} - } - if arn.Type != madmin.ReplicationService { - return false, BucketRemoteArnTypeInvalid{} - } - clnt := globalBucketTargetSys.GetRemoteTargetClient(ctx, rCfg.RoleArn) - if clnt == nil { - return false, BucketRemoteTargetNotFound{Bucket: bucket} - } - if found, _ := clnt.BucketExists(ctx, rCfg.GetDestination().Bucket); !found { - return false, BucketRemoteDestinationNotFound{Bucket: rCfg.GetDestination().Bucket} - } - if ret, err := globalBucketObjectLockSys.Get(bucket); err == nil { - if ret.LockEnabled { - lock, _, _, _, err := clnt.GetObjectLockConfig(ctx, rCfg.GetDestination().Bucket) - if err != nil || lock != "Enabled" { - return false, BucketReplicationDestinationMissingLock{Bucket: rCfg.GetDestination().Bucket} - } + var arns []string + if rCfg.RoleArn != "" { + arns = append(arns, rCfg.RoleArn) + } else { + for _, rule := range rCfg.Rules { + arns = append(arns, rule.Destination.String()) } } - // validate replication ARN against target endpoint - c, ok := globalBucketTargetSys.arnRemotesMap[rCfg.RoleArn] - if ok { - if c.EndpointURL().String() == clnt.EndpointURL().String() { - sameTarget, _ := isLocalHost(clnt.EndpointURL().Hostname(), clnt.EndpointURL().Port(), globalMinioPort) - return sameTarget, nil + for _, arnStr := range arns { + arn, err := madmin.ParseARN(arnStr) + if err != nil { + return false, BucketRemoteArnInvalid{} + } + if arn.Type != madmin.ReplicationService { + return false, BucketRemoteArnTypeInvalid{} + } + clnt := globalBucketTargetSys.GetRemoteTargetClient(ctx, arnStr) + if clnt == nil { + return false, BucketRemoteTargetNotFound{Bucket: bucket} + } + if found, _ := clnt.BucketExists(ctx, arn.Bucket); !found { + return false, BucketRemoteDestinationNotFound{Bucket: arn.Bucket} + } + if ret, err := globalBucketObjectLockSys.Get(bucket); err == nil { + if ret.LockEnabled { + lock, _, _, _, err := clnt.GetObjectLockConfig(ctx, arn.Bucket) + if err != nil || lock != "Enabled" { + return false, BucketReplicationDestinationMissingLock{Bucket: arn.Bucket} + } + } + } + // validate replication ARN against target endpoint + c, ok := globalBucketTargetSys.arnRemotesMap[arnStr] + if ok { + if c.EndpointURL().String() == clnt.EndpointURL().String() { + sameTarget, _ := isLocalHost(clnt.EndpointURL().Hostname(), clnt.EndpointURL().Port(), globalMinioPort) + return sameTarget, nil + } } } return false, BucketRemoteTargetNotFound{Bucket: bucket} } type mustReplicateOptions struct { - meta map[string]string - status replication.StatusType - opType replication.Type + meta map[string]string + status replication.StatusType + opType replication.Type + replicationRequest bool // incoming request is a replication request } func (o mustReplicateOptions) ReplicationStatus() (s replication.StatusType) { @@ -113,7 +143,7 @@ func (o mustReplicateOptions) isExistingObjectReplication() bool { func (o mustReplicateOptions) isMetadataReplication() bool { return o.opType == replication.MetadataReplicationType } -func getMustReplicateOptions(o ObjectInfo, op replication.Type) mustReplicateOptions { +func getMustReplicateOptions(o ObjectInfo, op replication.Type, opts ObjectOptions) mustReplicateOptions { if !op.Valid() { op = replication.ObjectReplicationType if o.metadataOnly { @@ -124,26 +154,33 @@ func getMustReplicateOptions(o ObjectInfo, op replication.Type) mustReplicateOpt if o.UserTags != "" { meta[xhttp.AmzObjectTagging] = o.UserTags } + return mustReplicateOptions{ - meta: meta, - status: o.ReplicationStatus, - opType: op, + meta: meta, + status: o.ReplicationStatus, + opType: op, + replicationRequest: opts.ReplicationRequest, } } // mustReplicate returns 2 booleans - true if object meets replication criteria and true if replication is to be done in // a synchronous manner. -func mustReplicate(ctx context.Context, bucket, object string, mopts mustReplicateOptions) (replicate bool, sync bool) { +func mustReplicate(ctx context.Context, bucket, object string, mopts mustReplicateOptions) (dsc ReplicateDecision) { if globalIsGateway { - return replicate, sync + return } + replStatus := mopts.ReplicationStatus() if replStatus == replication.Replica && !mopts.isMetadataReplication() { - return replicate, sync + return + } + + if mopts.replicationRequest { // incoming replication request on target cluster + return } cfg, err := getReplicationConfig(ctx, bucket) if err != nil { - return replicate, sync + return } opts := replication.ObjectOpts{ Name: object, @@ -155,13 +192,20 @@ func mustReplicate(ctx context.Context, bucket, object string, mopts mustReplica if ok { opts.UserTags = tagStr } - tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, cfg.RoleArn) - // the target online status should not be used here while deciding - // whether to replicate as the target could be temporarily down - if tgt != nil { - return cfg.Replicate(opts), tgt.replicateSync + tgtArns := cfg.FilterTargetArns(opts) + for _, tgtArn := range tgtArns { + tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, tgtArn) + // the target online status should not be used here while deciding + // whether to replicate as the target could be temporarily down + opts.TargetArn = tgtArn + replicate := cfg.Replicate(opts) + var synchronous bool + if tgt != nil { + synchronous = tgt.replicateSync + } + dsc.Set(newReplicateTargetDecision(tgtArn, replicate, synchronous)) } - return cfg.Replicate(opts), false + return dsc } // Standard headers that needs to be extracted from User metadata. @@ -201,10 +245,14 @@ func isStandardHeader(matchHeaderKey string) bool { } // returns whether object version is a deletemarker and if object qualifies for replication -func checkReplicateDelete(ctx context.Context, bucket string, dobj ObjectToDelete, oi ObjectInfo, gerr error) (replicate, sync bool) { +func checkReplicateDelete(ctx context.Context, bucket string, dobj ObjectToDelete, oi ObjectInfo, delOpts ObjectOptions, gerr error) (dsc ReplicateDecision) { rcfg, err := getReplicationConfig(ctx, bucket) if err != nil || rcfg == nil { - return false, sync + return + } + // If incoming request is a replication request, it does not need to be re-replicated. + if delOpts.ReplicationRequest { + return } opts := replication.ObjectOpts{ Name: dobj.ObjectName, @@ -214,29 +262,43 @@ func checkReplicateDelete(ctx context.Context, bucket string, dobj ObjectToDelet VersionID: dobj.VersionID, OpType: replication.DeleteReplicationType, } - replicate = rcfg.Replicate(opts) - // when incoming delete is removal of a delete marker( a.k.a versioned delete), - // GetObjectInfo returns extra information even though it returns errFileNotFound - if gerr != nil { - validReplStatus := false - switch oi.ReplicationStatus { - case replication.Pending, replication.Completed, replication.Failed: - validReplStatus = true + tgtArns := rcfg.FilterTargetArns(opts) + if len(tgtArns) > 0 { + dsc.targetsMap = make(map[string]replicateTargetDecision, len(tgtArns)) + var sync, replicate bool + for _, tgtArn := range tgtArns { + opts.TargetArn = tgtArn + replicate = rcfg.Replicate(opts) + // when incoming delete is removal of a delete marker( a.k.a versioned delete), + // GetObjectInfo returns extra information even though it returns errFileNotFound + if gerr != nil { + validReplStatus := false + switch oi.TargetReplicationStatus(tgtArn) { + case replication.Pending, replication.Completed, replication.Failed: + validReplStatus = true + } + if oi.DeleteMarker && (validReplStatus || replicate) { + dsc.Set(newReplicateTargetDecision(tgtArn, replicate, sync)) + continue + } else { + // can be the case that other cluster is down and duplicate `mc rm --vid` + // is issued - this still needs to be replicated back to the other target + replicate = oi.VersionPurgeStatus == Pending || oi.VersionPurgeStatus == Failed + dsc.Set(newReplicateTargetDecision(tgtArn, replicate, sync)) + continue + } + } + tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, tgtArn) + // the target online status should not be used here while deciding + // whether to replicate deletes as the target could be temporarily down + tgtDsc := newReplicateTargetDecision(tgtArn, false, false) + if tgt != nil { + tgtDsc = newReplicateTargetDecision(tgtArn, replicate, tgt.replicateSync) + } + dsc.Set(tgtDsc) } - if oi.DeleteMarker && (validReplStatus || replicate) { - return true, sync - } - // can be the case that other cluster is down and duplicate `mc rm --vid` - // is issued - this still needs to be replicated back to the other target - return oi.VersionPurgeStatus == Pending || oi.VersionPurgeStatus == Failed, sync } - tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, rcfg.RoleArn) - // the target online status should not be used here while deciding - // whether to replicate deletes as the target could be temporarily down - if tgt == nil { - return false, false - } - return replicate, tgt.replicateSync + return dsc } // replicate deletes to the designated replication target if replication configuration @@ -250,9 +312,7 @@ func checkReplicateDelete(ctx context.Context, bucket string, dobj ObjectToDelet // deleted from the source when the VersionPurgeStatus changes to "Complete", i.e after replication succeeds // on target. func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, objectAPI ObjectLayer, trigger string) { - var versionPurgeStatus VersionPurgeStatusType - var replicationStatus string - + var replicationStatus replication.StatusType bucket := dobj.Bucket versionID := dobj.DeleteMarkerVersionID if versionID == "" { @@ -260,10 +320,7 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, obj } defer func() { - replStatus := replicationStatus - if !versionPurgeStatus.Empty() { - replStatus = string(versionPurgeStatus) - } + replStatus := string(replicationStatus) auditLogInternal(context.Background(), bucket, dobj.ObjectName, AuditLogOptions{ Trigger: trigger, APIName: ReplicateDeleteAPI, @@ -288,10 +345,9 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, obj }) return } - - tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, rcfg.RoleArn) - if tgt == nil { - logger.LogIf(ctx, fmt.Errorf("failed to get target for bucket:%s arn:%s", bucket, rcfg.RoleArn)) + dsc, err := parseReplicateDecision(dobj.ReplicationState.ReplicateDecisionStr) + if err != nil { + logger.LogIf(ctx, err) sendEvent(eventArgs{ BucketName: bucket, Object: ObjectInfo{ @@ -306,21 +362,6 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, obj return } - if tgt.IsOffline() { - logger.LogIf(ctx, fmt.Errorf("remote target is offline for bucket:%s arn:%s", bucket, rcfg.RoleArn)) - sendEvent(eventArgs{ - BucketName: bucket, - Object: ObjectInfo{ - Bucket: bucket, - Name: dobj.ObjectName, - VersionID: versionID, - DeleteMarker: dobj.DeleteMarker, - }, - Host: "Internal: [Replication]", - EventName: event.ObjectReplicationNotTracked, - }) - return - } // Lock the object name before starting replication operation. // Use separate lock that doesn't collide with regular objects. lk := objectAPI.NewNSLock(bucket, "/[replicate]/"+dobj.ObjectName) @@ -342,67 +383,72 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, obj } ctx = lkctx.Context() defer lk.Unlock(lkctx.Cancel) - // early return if already replicated delete marker for existing object replication - if dobj.DeleteMarkerVersionID != "" && dobj.OpType == replication.ExistingObjectReplicationType { - _, err := tgt.StatObject(ctx, rcfg.GetDestination().Bucket, dobj.ObjectName, miniogo.StatObjectOptions{ - VersionID: versionID, - Internal: miniogo.AdvancedGetOptions{ - ReplicationProxyRequest: "false", - }}) - if isErrMethodNotAllowed(ErrorRespToObjectError(err, dobj.Bucket, dobj.ObjectName)) { - return + + var wg sync.WaitGroup + var rinfos replicatedInfos + rinfos.Targets = make([]replicatedTargetInfo, len(dsc.targetsMap)) + idx := -1 + for tgtArn := range dsc.targetsMap { + idx++ + tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, tgtArn) + if tgt == nil { + logger.LogIf(ctx, fmt.Errorf("failed to get target for bucket:%s arn:%s", bucket, tgtArn)) + sendEvent(eventArgs{ + BucketName: bucket, + Object: ObjectInfo{ + Bucket: bucket, + Name: dobj.ObjectName, + VersionID: versionID, + DeleteMarker: dobj.DeleteMarker, + }, + Host: "Internal: [Replication]", + EventName: event.ObjectReplicationNotTracked, + }) + continue } + if tgt := dsc.targetsMap[tgtArn]; !tgt.Replicate { + continue + } + // if dobj.TargetArn is not empty string, this is a case of specific target being re-synced. + if dobj.TargetArn != "" && dobj.TargetArn != tgt.ARN { + continue + } + wg.Add(1) + go func(index int, tgt *TargetClient) { + defer wg.Done() + rinfo := replicateDeleteToTarget(ctx, dobj, objectAPI, tgt) + rinfos.Targets[index] = rinfo + }(idx, tgt) } + wg.Wait() - rmErr := tgt.RemoveObject(ctx, rcfg.GetDestination().Bucket, dobj.ObjectName, miniogo.RemoveObjectOptions{ - VersionID: versionID, - Internal: miniogo.AdvancedRemoveOptions{ - ReplicationDeleteMarker: dobj.DeleteMarkerVersionID != "", - ReplicationMTime: dobj.DeleteMarkerMTime.Time, - ReplicationStatus: miniogo.ReplicationStatusReplica, - ReplicationRequest: true, // always set this to distinguish between `mc mirror` replication and serverside - }, - }) + replicationStatus = rinfos.ReplicationStatus() + prevStatus := dobj.DeleteMarkerReplicationStatus() - replicationStatus = dobj.DeleteMarkerReplicationStatus - versionPurgeStatus = dobj.VersionPurgeStatus - - if rmErr != nil { - if dobj.VersionID == "" { - replicationStatus = string(replication.Failed) - } else { - versionPurgeStatus = Failed - } - logger.LogIf(ctx, fmt.Errorf("Unable to replicate delete marker to %s/%s(%s): %s", rcfg.GetDestination().Bucket, dobj.ObjectName, versionID, rmErr)) - } else { - if dobj.VersionID == "" { - replicationStatus = string(replication.Completed) - } else { - versionPurgeStatus = Complete - } - } - prevStatus := dobj.DeleteMarkerReplicationStatus - currStatus := replicationStatus if dobj.VersionID != "" { - prevStatus = string(dobj.VersionPurgeStatus) - currStatus = string(versionPurgeStatus) + prevStatus = replication.StatusType(dobj.VersionPurgeStatus()) + replicationStatus = replication.StatusType(rinfos.VersionPurgeStatus()) } + // to decrement pending count later. - globalReplicationStats.Update(dobj.Bucket, 0, replication.StatusType(currStatus), - replication.StatusType(prevStatus), replication.DeleteReplicationType) + for _, rinfo := range rinfos.Targets { + if rinfo.ReplicationStatus != rinfo.PrevReplicationStatus { + globalReplicationStats.Update(dobj.Bucket, rinfo.Arn, 0, replicationStatus, + prevStatus, replication.DeleteReplicationType) + } + } var eventName = event.ObjectReplicationComplete - if replicationStatus == string(replication.Failed) || versionPurgeStatus == Failed { + if replicationStatus == replication.Failed { eventName = event.ObjectReplicationFailed } - - // Update metadata on the delete marker or purge permanent delete if replication success. + drs := getReplicationState(rinfos, dobj.ReplicationState, dobj.VersionID) dobjInfo, err := objectAPI.DeleteObject(ctx, bucket, dobj.ObjectName, ObjectOptions{ - VersionID: versionID, - DeleteMarkerReplicationStatus: replicationStatus, - VersionPurgeStatus: versionPurgeStatus, - Versioned: globalBucketVersioningSys.Enabled(bucket), - VersionSuspended: globalBucketVersioningSys.Suspended(bucket), + VersionID: versionID, + MTime: dobj.DeleteMarkerMTime.Time, + DeleteReplication: drs, + Versioned: globalBucketVersioningSys.Enabled(bucket), + VersionSuspended: globalBucketVersioningSys.Suspended(bucket), }) if err != nil && !isErrVersionNotFound(err) { // VersionNotFound would be reported by pool that object version is missing on. logger.LogIf(ctx, fmt.Errorf("Unable to update replication metadata for %s/%s(%s): %s", bucket, dobj.ObjectName, versionID, err)) @@ -427,7 +473,90 @@ func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, obj } } -func getCopyObjMetadata(oi ObjectInfo, dest replication.Destination) map[string]string { +func replicateDeleteToTarget(ctx context.Context, dobj DeletedObjectReplicationInfo, objectAPI ObjectLayer, tgt *TargetClient) (rinfo replicatedTargetInfo) { + versionID := dobj.DeleteMarkerVersionID + if versionID == "" { + versionID = dobj.VersionID + } + + rinfo = dobj.ReplicationState.targetState(tgt.ARN) + rinfo.OpType = dobj.OpType + defer func() { + if rinfo.ReplicationStatus == replication.Completed && tgt.ResetID != "" && dobj.OpType == replication.ExistingObjectReplicationType { + rinfo.ResyncTimestamp = fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), tgt.ResetID) + } + }() + + if dobj.VersionID == "" && rinfo.PrevReplicationStatus == replication.Completed && dobj.OpType != replication.ExistingObjectReplicationType { + rinfo.ReplicationStatus = rinfo.PrevReplicationStatus + return + } + if dobj.VersionID != "" && rinfo.VersionPurgeStatus == Complete { + return + } + if tgt.IsOffline() { + logger.LogIf(ctx, fmt.Errorf("remote target is offline for bucket:%s arn:%s", dobj.Bucket, tgt.ARN)) + sendEvent(eventArgs{ + BucketName: dobj.Bucket, + Object: ObjectInfo{ + Bucket: dobj.Bucket, + Name: dobj.ObjectName, + VersionID: dobj.VersionID, + DeleteMarker: dobj.DeleteMarker, + }, + Host: "Internal: [Replication]", + EventName: event.ObjectReplicationNotTracked, + }) + if dobj.VersionID == "" { + rinfo.ReplicationStatus = replication.Failed + } else { + rinfo.VersionPurgeStatus = Failed + } + return + } + // early return if already replicated delete marker for existing object replication + if dobj.DeleteMarkerVersionID != "" && dobj.OpType == replication.ExistingObjectReplicationType { + if _, err := tgt.StatObject(ctx, tgt.Bucket, dobj.ObjectName, miniogo.StatObjectOptions{ + VersionID: versionID, + Internal: miniogo.AdvancedGetOptions{ + ReplicationProxyRequest: "false", + }}); isErrMethodNotAllowed(ErrorRespToObjectError(err, dobj.Bucket, dobj.ObjectName)) { + if dobj.VersionID == "" { + rinfo.ReplicationStatus = replication.Completed + } else { + rinfo.VersionPurgeStatus = Complete + } + return + } + } + + rmErr := tgt.RemoveObject(ctx, tgt.Bucket, dobj.ObjectName, miniogo.RemoveObjectOptions{ + VersionID: versionID, + Internal: miniogo.AdvancedRemoveOptions{ + ReplicationDeleteMarker: dobj.DeleteMarkerVersionID != "", + ReplicationMTime: dobj.DeleteMarkerMTime.Time, + ReplicationStatus: miniogo.ReplicationStatusReplica, + ReplicationRequest: true, // always set this to distinguish between `mc mirror` replication and serverside + }, + }) + if rmErr != nil { + if dobj.VersionID == "" { + rinfo.ReplicationStatus = replication.Failed + } else { + rinfo.VersionPurgeStatus = Failed + } + logger.LogIf(ctx, fmt.Errorf("Unable to replicate delete marker to %s/%s(%s): %s", tgt.Bucket, dobj.ObjectName, versionID, rmErr)) + } else { + if dobj.VersionID == "" { + rinfo.ReplicationStatus = replication.Completed + } else { + rinfo.VersionPurgeStatus = Complete + } + } + return +} + +func getCopyObjMetadata(oi ObjectInfo, sc string) map[string]string { meta := make(map[string]string, len(oi.UserDefined)) for k, v := range oi.UserDefined { if strings.HasPrefix(strings.ToLower(k), ReservedMetadataPrefixLower) { @@ -458,7 +587,6 @@ func getCopyObjMetadata(oi ObjectInfo, dest replication.Destination) map[string] meta[xhttp.AmzTagDirective] = "REPLACE" } - sc := dest.StorageClass if sc == "" { sc = oi.StorageClass } @@ -493,7 +621,7 @@ func (m caseInsensitiveMap) Lookup(key string) (string, bool) { return "", false } -func putReplicationOpts(ctx context.Context, dest replication.Destination, objInfo ObjectInfo) (putOpts miniogo.PutObjectOptions, err error) { +func putReplicationOpts(ctx context.Context, sc string, objInfo ObjectInfo) (putOpts miniogo.PutObjectOptions, err error) { meta := make(map[string]string) for k, v := range objInfo.UserDefined { if strings.HasPrefix(strings.ToLower(k), ReservedMetadataPrefixLower) { @@ -505,7 +633,6 @@ func putReplicationOpts(ctx context.Context, dest replication.Destination, objIn meta[k] = v } - sc := dest.StorageClass if sc == "" && (objInfo.StorageClass == storageclass.STANDARD || objInfo.StorageClass == storageclass.RRS) { sc = objInfo.StorageClass } @@ -526,6 +653,15 @@ func putReplicationOpts(ctx context.Context, dest replication.Destination, objIn tag, _ := tags.ParseObjectTags(objInfo.UserTags) if tag != nil { putOpts.UserTags = tag.ToMap() + // set tag timestamp in opts + tagTimestamp := objInfo.ModTime + if tagTmstampStr, ok := objInfo.UserDefined[ReservedMetadataPrefixLower+TaggingTimestamp]; ok { + tagTimestamp, err = time.Parse(time.RFC3339Nano, tagTmstampStr) + if err != nil { + return putOpts, err + } + } + putOpts.Internal.TaggingTimestamp = tagTimestamp } } @@ -549,9 +685,27 @@ func putReplicationOpts(ctx context.Context, dest replication.Destination, objIn return putOpts, err } putOpts.RetainUntilDate = rdate + // set retention timestamp in opts + retTimestamp := objInfo.ModTime + if retainTmstampStr, ok := objInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockRetentionTimestamp]; ok { + retTimestamp, err = time.Parse(time.RFC3339Nano, retainTmstampStr) + if err != nil { + return putOpts, err + } + } + putOpts.Internal.RetentionTimestamp = retTimestamp } if lhold, ok := lkMap.Lookup(xhttp.AmzObjectLockLegalHold); ok { putOpts.LegalHold = miniogo.LegalHoldStatus(lhold) + // set legalhold timestamp in opts + lholdTimestamp := objInfo.ModTime + if lholdTmstampStr, ok := objInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockLegalHoldTimestamp]; ok { + lholdTimestamp, err = time.Parse(time.RFC3339Nano, lholdTmstampStr) + if err != nil { + return putOpts, err + } + } + putOpts.Internal.LegalholdTimestamp = lholdTimestamp } if crypto.S3.IsEncrypted(objInfo.UserDefined) { putOpts.ServerSideEncryption = encrypt.NewSSE() @@ -570,7 +724,7 @@ const ( // matches k1 with all keys, returns 'true' if one of them matches func equals(k1 string, keys ...string) bool { for _, k2 := range keys { - if strings.ToLower(k1) == strings.ToLower(k2) { + if strings.EqualFold(strings.ToLower(k1), strings.ToLower(k2)) { return true } } @@ -696,27 +850,10 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje }) return } - tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, cfg.RoleArn) - if tgt == nil { - logger.LogIf(ctx, fmt.Errorf("failed to get target for bucket:%s arn:%s", bucket, cfg.RoleArn)) - sendEvent(eventArgs{ - EventName: event.ObjectReplicationNotTracked, - BucketName: bucket, - Object: objInfo, - Host: "Internal: [Replication]", - }) - return - } - if tgt.IsOffline() { - logger.LogIf(ctx, fmt.Errorf("remote target is offline for bucket:%s arn:%s", bucket, cfg.RoleArn)) - sendEvent(eventArgs{ - EventName: event.ObjectReplicationNotTracked, - BucketName: bucket, - Object: objInfo, - Host: "Internal: [Replication]", - }) - return - } + tgtArns := cfg.FilterTargetArns(replication.ObjectOpts{ + Name: object, + SSEC: crypto.SSEC.IsEncrypted(objInfo.UserDefined), + }) // Lock the object name before starting replication. // Use separate lock that doesn't collide with regular objects. lk := objectAPI.NewNSLock(bucket, "/[replicate]/"+object) @@ -734,8 +871,142 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje ctx = lkctx.Context() defer lk.Unlock(lkctx.Cancel) - var closeOnDefer bool - gr, err := objectAPI.GetObjectNInfo(ctx, bucket, object, nil, http.Header{}, readLock, ObjectOptions{ + var wg sync.WaitGroup + var rinfos replicatedInfos + rinfos.Targets = make([]replicatedTargetInfo, len(tgtArns)) + for i, tgtArn := range tgtArns { + tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, tgtArn) + if tgt == nil { + logger.LogIf(ctx, fmt.Errorf("failed to get target for bucket:%s arn:%s", bucket, tgtArn)) + sendEvent(eventArgs{ + EventName: event.ObjectReplicationNotTracked, + BucketName: bucket, + Object: objInfo, + Host: "Internal: [Replication]", + }) + continue + } + wg.Add(1) + go func(index int, tgt *TargetClient) { + defer wg.Done() + rinfos.Targets[index] = replicateObjectToTarget(ctx, ri, objectAPI, tgt) + }(i, tgt) + } + wg.Wait() + // metadata update once.. + if objInfo.UserTags != "" { + objInfo.UserDefined[xhttp.AmzObjectTagging] = objInfo.UserTags + } + // FIXME: add support for missing replication events + // - event.ObjectReplicationMissedThreshold + // - event.ObjectReplicationReplicatedAfterThreshold + var eventName = event.ObjectReplicationComplete + if rinfos.ReplicationStatus() == replication.Failed { + eventName = event.ObjectReplicationFailed + } + newReplStatusInternal := rinfos.ReplicationStatusInternal() + // Note that internal replication status(es) may match for previously replicated objects - in such cases + // metadata should be updated with last resync timestamp. + if objInfo.ReplicationStatusInternal != newReplStatusInternal || rinfos.ReplicationResynced() { + popts := ObjectOptions{ + MTime: objInfo.ModTime, + VersionID: objInfo.VersionID, + UserDefined: make(map[string]string, len(objInfo.UserDefined)), + } + for k, v := range objInfo.UserDefined { + popts.UserDefined[k] = v + } + popts.UserDefined[ReservedMetadataPrefixLower+ReplicationStatus] = newReplStatusInternal + popts.UserDefined[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) + popts.UserDefined[xhttp.AmzBucketReplicationStatus] = string(rinfos.ReplicationStatus()) + for _, rinfo := range rinfos.Targets { + if rinfo.ResyncTimestamp != "" { + popts.UserDefined[targetResetHeader(rinfo.Arn)] = rinfo.ResyncTimestamp + } + } + if objInfo.UserTags != "" { + popts.UserDefined[xhttp.AmzObjectTagging] = objInfo.UserTags + } + if _, err = objectAPI.PutObjectMetadata(ctx, bucket, object, popts); err != nil { + 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 rinfos.Action() == replicateAll { + opType = replication.ObjectReplicationType + } + for _, rinfo := range rinfos.Targets { + if rinfo.ReplicationStatus != rinfo.PrevReplicationStatus { + globalReplicationStats.Update(bucket, rinfo.Arn, rinfo.Size, rinfo.ReplicationStatus, rinfo.PrevReplicationStatus, opType) + } + } + } + + sendEvent(eventArgs{ + EventName: eventName, + BucketName: bucket, + Object: objInfo, + Host: "Internal: [Replication]", + }) + + // re-queue failures once more - keep a retry count to avoid flooding the queue if + // the target site is down. Leave it to scanner to catch up instead. + if rinfos.ReplicationStatus() != replication.Completed && ri.RetryCount < 1 { + ri.OpType = replication.HealReplicationType + ri.ReplicationStatusInternal = rinfos.ReplicationStatusInternal() + ri.RetryCount++ + globalReplicationPool.queueReplicaFailedTask(ri) + } +} + +// replicateObjectToTarget replicates the specified version of the object to destination bucket +// The source object is then updated to reflect the replication status. +func replicateObjectToTarget(ctx context.Context, ri ReplicateObjectInfo, objectAPI ObjectLayer, tgt *TargetClient) (rinfo replicatedTargetInfo) { + objInfo := ri.ObjectInfo + bucket := objInfo.Bucket + object := objInfo.Name + var ( + closeOnDefer bool + gr *GetObjectReader + size int64 + err error + ) + sz, _ := objInfo.GetActualSize() + // set defaults for replication action based on operation being performed - actual + // replication action can only be determined after stat on remote. This default is + // needed for updating replication metrics correctly when target is offline. + var rAction replicationAction + switch ri.OpType { + case replication.MetadataReplicationType: + rAction = replicateMetadata + default: + rAction = replicateAll + } + rinfo = replicatedTargetInfo{ + Size: sz, + Arn: tgt.ARN, + PrevReplicationStatus: objInfo.TargetReplicationStatus(tgt.ARN), + ReplicationStatus: replication.Failed, + OpType: ri.OpType, + ReplicationAction: rAction, + } + if ri.ObjectInfo.TargetReplicationStatus(tgt.ARN) == replication.Completed && !ri.ExistingObjResync.Empty() && !ri.ExistingObjResync.mustResyncTarget(tgt.ARN) { + rinfo.ReplicationStatus = replication.Completed + rinfo.ReplicationResynced = true + return + } + if tgt.IsOffline() { + logger.LogIf(ctx, fmt.Errorf("remote target is offline for bucket:%s arn:%s", bucket, tgt.ARN)) + sendEvent(eventArgs{ + EventName: event.ObjectReplicationNotTracked, + BucketName: bucket, + Object: objInfo, + Host: "Internal: [Replication]", + }) + return + } + + gr, err = objectAPI.GetObjectNInfo(ctx, bucket, object, nil, http.Header{}, readLock, ObjectOptions{ VersionID: objInfo.VersionID, }) if err != nil { @@ -756,7 +1027,7 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje closeOnDefer = true objInfo = gr.ObjInfo - size, err := objInfo.GetActualSize() + size, err = objInfo.GetActualSize() if err != nil { logger.LogIf(ctx, err) sendEvent(eventArgs{ @@ -768,8 +1039,7 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje return } - dest := cfg.GetDestination() - if dest.Bucket == "" { + if tgt.Bucket == "" { logger.LogIf(ctx, fmt.Errorf("Unable to replicate object %s(%s), bucket is empty", objInfo.Name, objInfo.VersionID)) sendEvent(eventArgs{ EventName: event.ObjectReplicationNotTracked, @@ -777,21 +1047,22 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje Object: objInfo, Host: "Internal: [Replication]", }) - return + return rinfo } - rtype := replicateAll - oi, err := tgt.StatObject(ctx, dest.Bucket, object, miniogo.StatObjectOptions{ + rAction = replicateAll + oi, cerr := tgt.StatObject(ctx, tgt.Bucket, object, miniogo.StatObjectOptions{ VersionID: objInfo.VersionID, Internal: miniogo.AdvancedGetOptions{ ReplicationProxyRequest: "false", }}) - if err == nil { - rtype = getReplicationAction(objInfo, oi) - if rtype == replicateNone { + if cerr == nil { + rAction = getReplicationAction(objInfo, oi) + rinfo.ReplicationStatus = replication.Completed + if rAction == replicateNone { // object with same VersionID already exists, replication kicked off by // PutObject might have completed - if objInfo.ReplicationStatus == replication.Pending || objInfo.ReplicationStatus == replication.Failed || ri.OpType == replication.ExistingObjectReplicationType { + if objInfo.TargetReplicationStatus(tgt.ARN) == replication.Pending || objInfo.TargetReplicationStatus(tgt.ARN) == replication.Failed || ri.OpType == replication.ExistingObjectReplicationType { // if metadata is not updated for some reason after replication, such as // 503 encountered while updating metadata - make sure to set ReplicationStatus // as Completed. @@ -799,36 +1070,32 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje // Note: Replication Stats would have been updated despite metadata update failure. gr.Close() closeOnDefer = false - popts := ObjectOptions{ - MTime: objInfo.ModTime, - VersionID: objInfo.VersionID, - UserDefined: make(map[string]string, len(objInfo.UserDefined)), - } - for k, v := range objInfo.UserDefined { - popts.UserDefined[k] = v - } - if ri.OpType == replication.ExistingObjectReplicationType { - popts.UserDefined[xhttp.MinIOReplicationResetStatus] = fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), ri.ResetID) - } - popts.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Completed.String() - if objInfo.UserTags != "" { - popts.UserDefined[xhttp.AmzObjectTagging] = objInfo.UserTags - } - if _, err = objectAPI.PutObjectMetadata(ctx, bucket, object, popts); err != nil { - logger.LogIf(ctx, fmt.Errorf("Unable to update replication metadata for %s/%s(%s): %w", bucket, objInfo.Name, objInfo.VersionID, err)) + return replicatedTargetInfo{ + ReplicationStatus: replication.Completed, + Size: sz, + Arn: tgt.ARN, + ReplicationAction: rAction, + PrevReplicationStatus: objInfo.TargetReplicationStatus(tgt.ARN), } } return } } - - replicationStatus = replication.Completed + rinfo.ReplicationStatus = replication.Completed + rinfo.Size = size + rinfo.ReplicationAction = rAction + defer func() { + if rinfo.ReplicationStatus == replication.Completed && ri.OpType == replication.ExistingObjectReplicationType && tgt.ResetID != "" { + rinfo.ResyncTimestamp = fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), tgt.ResetID) + rinfo.ReplicationResynced = true + } + }() // use core client to avoid doing multipart on PUT c := &miniogo.Core{Client: tgt.Client} - if rtype != replicateAll { + if rAction != replicateAll { // replicate metadata for object tagging/copy with metadata replacement srcOpts := miniogo.CopySrcOptions{ - Bucket: dest.Bucket, + Bucket: tgt.Bucket, Object: object, VersionID: objInfo.VersionID, } @@ -837,14 +1104,15 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje SourceVersionID: objInfo.VersionID, ReplicationRequest: true, // always set this to distinguish between `mc mirror` replication and serverside }} - if _, err = c.CopyObject(ctx, dest.Bucket, object, dest.Bucket, object, getCopyObjMetadata(objInfo, dest), srcOpts, dstOpts); err != nil { - replicationStatus = replication.Failed + if _, err = c.CopyObject(ctx, tgt.Bucket, object, tgt.Bucket, object, getCopyObjMetadata(objInfo, tgt.StorageClass), srcOpts, dstOpts); err != nil { + rinfo.ReplicationStatus = replication.Failed logger.LogIf(ctx, fmt.Errorf("Unable to replicate metadata for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, err)) } } else { - putOpts, err := putReplicationOpts(ctx, dest, objInfo) + var putOpts minio.PutObjectOptions + putOpts, err = putReplicationOpts(ctx, tgt.StorageClass, objInfo) if err != nil { - logger.LogIf(ctx, fmt.Errorf("failed to get target for replication bucket:%s cfg:%s err:%w", bucket, cfg.RoleArn, err)) + logger.LogIf(ctx, fmt.Errorf("failed to get target for replication bucket:%s err:%w", bucket, err)) sendEvent(eventArgs{ EventName: event.ObjectReplicationNotTracked, BucketName: bucket, @@ -870,75 +1138,21 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje } r := bandwidth.NewMonitoredReader(newCtx, globalBucketMonitor, gr, opts) if objInfo.isMultipart() { - if err := replicateObjectWithMultipart(ctx, c, dest.Bucket, object, + if err := replicateObjectWithMultipart(ctx, c, tgt.Bucket, object, r, objInfo, putOpts); err != nil { - replicationStatus = replication.Failed + rinfo.ReplicationStatus = replication.Failed logger.LogIf(ctx, fmt.Errorf("Unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, err)) } } else { - if _, err = c.PutObject(ctx, dest.Bucket, object, r, size, "", "", putOpts); err != nil { - replicationStatus = replication.Failed + if _, err = c.PutObject(ctx, tgt.Bucket, object, r, size, "", "", putOpts); err != nil { + rinfo.ReplicationStatus = replication.Failed logger.LogIf(ctx, fmt.Errorf("Unable to replicate for object %s/%s(%s): %s", bucket, objInfo.Name, objInfo.VersionID, err)) } } } gr.Close() closeOnDefer = false - - prevReplStatus := objInfo.ReplicationStatus - objInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replicationStatus.String() - if objInfo.UserTags != "" { - objInfo.UserDefined[xhttp.AmzObjectTagging] = objInfo.UserTags - } - if ri.OpType == replication.ExistingObjectReplicationType { - objInfo.UserDefined[xhttp.MinIOReplicationResetStatus] = fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), ri.ResetID) - } - // FIXME: add support for missing replication events - // - event.ObjectReplicationMissedThreshold - // - event.ObjectReplicationReplicatedAfterThreshold - var eventName = event.ObjectReplicationComplete - if replicationStatus == replication.Failed { - eventName = event.ObjectReplicationFailed - } - - // Leave metadata in `PENDING` state if inline replication fails to save iops - if ri.OpType == replication.HealReplicationType || - replicationStatus == replication.Completed { - popts := ObjectOptions{ - MTime: objInfo.ModTime, - VersionID: objInfo.VersionID, - UserDefined: make(map[string]string, len(objInfo.UserDefined)), - } - for k, v := range objInfo.UserDefined { - popts.UserDefined[k] = v - } - popts.UserDefined[xhttp.AmzBucketReplicationStatus] = replicationStatus.String() - if objInfo.UserTags != "" { - popts.UserDefined[xhttp.AmzObjectTagging] = objInfo.UserTags - } - if _, err = objectAPI.PutObjectMetadata(ctx, bucket, object, popts); err != nil { - 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(bucket, size, replicationStatus, prevReplStatus, opType) - sendEvent(eventArgs{ - EventName: eventName, - BucketName: bucket, - Object: objInfo, - Host: "Internal: [Replication]", - }) - } - // re-queue failures once more - keep a retry count to avoid flooding the queue if - // the target site is down. Leave it to scanner to catch up instead. - if replicationStatus != replication.Completed && ri.RetryCount < 1 { - ri.OpType = replication.HealReplicationType - ri.RetryCount++ - globalReplicationPool.queueReplicaFailedTask(ri) - } + return } func replicateObjectWithMultipart(ctx context.Context, c *miniogo.Core, bucket, object string, r io.Reader, objInfo ObjectInfo, opts miniogo.PutObjectOptions) (err error) { @@ -1015,9 +1229,10 @@ func filterReplicationStatusMetadata(metadata map[string]string) map[string]stri // DeletedObjectReplicationInfo has info on deleted object type DeletedObjectReplicationInfo struct { DeletedObject - Bucket string - OpType replication.Type - ResetID string + Bucket string + OpType replication.Type + ResetID string + TargetArn string } // Replication specific APIName @@ -1224,11 +1439,21 @@ func (p *ReplicationPool) queueReplicaTask(ri ReplicateObjectInfo) { } } +func queueReplicateDeletesWrapper(doi DeletedObjectReplicationInfo, existingObjectResync ResyncDecision) { + for k, v := range existingObjectResync.targets { + if v.Replicate { + doi.ResetID = v.ResetID + doi.TargetArn = k + + globalReplicationPool.queueReplicaDeleteTask(doi) + } + } +} + func (p *ReplicationPool) queueReplicaDeleteTask(doi DeletedObjectReplicationInfo) { if p == nil { return } - var ch chan DeletedObjectReplicationInfo switch doi.OpType { case replication.ExistingObjectReplicationType: @@ -1265,9 +1490,9 @@ func initBackgroundReplication(ctx context.Context, objectAPI ObjectLayer) { // get Reader from replication target if active-active replication is in place and // this node returns a 404 -func proxyGetToReplicationTarget(ctx context.Context, bucket, object string, rs *HTTPRangeSpec, h http.Header, opts ObjectOptions) (gr *GetObjectReader, proxy bool) { - tgt, oi, proxy, err := proxyHeadToRepTarget(ctx, bucket, object, opts) - if !proxy || err != nil { +func proxyGetToReplicationTarget(ctx context.Context, bucket, object string, rs *HTTPRangeSpec, h http.Header, opts ObjectOptions, proxyTargets *madmin.BucketTargets) (gr *GetObjectReader, proxy bool) { + tgt, oi, proxy := proxyHeadToRepTarget(ctx, bucket, object, opts, proxyTargets) + if !proxy { return nil, false } fn, off, length, err := NewGetObjectReader(rs, oi, opts) @@ -1306,119 +1531,115 @@ func proxyGetToReplicationTarget(ctx context.Context, bucket, object string, rs return reader, true } -// isProxyable returns true if replication config found for this bucket -func isProxyable(ctx context.Context, bucket string) bool { +func getproxyTargets(ctx context.Context, bucket, object string, opts ObjectOptions) (tgts *madmin.BucketTargets) { cfg, err := getReplicationConfig(ctx, bucket) - if err != nil { - return false + if err != nil || cfg == nil { + return &madmin.BucketTargets{} } - dest := cfg.GetDestination() - return dest.Bucket == bucket + topts := replication.ObjectOpts{Name: object} + tgtArns := cfg.FilterTargetArns(topts) + tgts = &madmin.BucketTargets{Targets: make([]madmin.BucketTarget, len(tgtArns))} + for i, tgtArn := range tgtArns { + tgt := globalBucketTargetSys.GetRemoteBucketTargetByArn(ctx, bucket, tgtArn) + tgts.Targets[i] = tgt + } + + return tgts } -func proxyHeadToRepTarget(ctx context.Context, bucket, object string, opts ObjectOptions) (tgt *TargetClient, oi ObjectInfo, proxy bool, err error) { +func proxyHeadToRepTarget(ctx context.Context, bucket, object string, opts ObjectOptions, proxyTargets *madmin.BucketTargets) (tgt *TargetClient, oi ObjectInfo, proxy bool) { // this option is set when active-active replication is in place between site A -> B, // and site B does not have the object yet. if opts.ProxyRequest || (opts.ProxyHeaderSet && !opts.ProxyRequest) { // true only when site B sets MinIOSourceProxyRequest header - return nil, oi, false, nil - } - cfg, err := getReplicationConfig(ctx, bucket) - if err != nil { - return nil, oi, false, err - } - dest := cfg.GetDestination() - if dest.Bucket != bucket { // not active-active - return nil, oi, false, err - } - ssec := false - if opts.ServerSideEncryption != nil { - ssec = opts.ServerSideEncryption.Type() == encrypt.SSEC - } - ropts := replication.ObjectOpts{ - Name: object, - SSEC: ssec, - } - if !cfg.Replicate(ropts) { // no matching rule for object prefix - return nil, oi, false, nil - } - tgt = globalBucketTargetSys.GetRemoteTargetClient(ctx, cfg.RoleArn) - if tgt == nil || tgt.IsOffline() { - return nil, oi, false, fmt.Errorf("target is offline or not configured") - } - // if proxying explicitly disabled on remote target - if tgt.disableProxy { - return nil, oi, false, nil - } - gopts := miniogo.GetObjectOptions{ - VersionID: opts.VersionID, - ServerSideEncryption: opts.ServerSideEncryption, - Internal: miniogo.AdvancedGetOptions{ - ReplicationProxyRequest: "true", - }, + return nil, oi, false } + for _, t := range proxyTargets.Targets { + tgt = globalBucketTargetSys.GetRemoteTargetClient(ctx, t.Arn) + if tgt == nil || tgt.IsOffline() { + continue + } + // if proxying explicitly disabled on remote target + if tgt.disableProxy { + continue + } - objInfo, err := tgt.StatObject(ctx, dest.Bucket, object, gopts) - if err != nil { - return nil, oi, false, err - } + gopts := miniogo.GetObjectOptions{ + VersionID: opts.VersionID, + ServerSideEncryption: opts.ServerSideEncryption, + Internal: miniogo.AdvancedGetOptions{ + ReplicationProxyRequest: "true", + }, + } + objInfo, err := tgt.StatObject(ctx, t.TargetBucket, object, gopts) + if err != nil { + continue + } - tags, _ := tags.MapToObjectTags(objInfo.UserTags) - oi = ObjectInfo{ - Bucket: bucket, - Name: object, - ModTime: objInfo.LastModified, - Size: objInfo.Size, - ETag: objInfo.ETag, - VersionID: objInfo.VersionID, - IsLatest: objInfo.IsLatest, - DeleteMarker: objInfo.IsDeleteMarker, - ContentType: objInfo.ContentType, - Expires: objInfo.Expires, - StorageClass: objInfo.StorageClass, - ReplicationStatus: replication.StatusType(objInfo.ReplicationStatus), - UserTags: tags.String(), + tags, _ := tags.MapToObjectTags(objInfo.UserTags) + oi = ObjectInfo{ + Bucket: bucket, + Name: object, + ModTime: objInfo.LastModified, + Size: objInfo.Size, + ETag: objInfo.ETag, + VersionID: objInfo.VersionID, + IsLatest: objInfo.IsLatest, + DeleteMarker: objInfo.IsDeleteMarker, + ContentType: objInfo.ContentType, + Expires: objInfo.Expires, + StorageClass: objInfo.StorageClass, + ReplicationStatusInternal: objInfo.ReplicationStatus, + UserTags: tags.String(), + } + oi.UserDefined = make(map[string]string, len(objInfo.Metadata)) + for k, v := range objInfo.Metadata { + oi.UserDefined[k] = v[0] + } + ce, ok := oi.UserDefined[xhttp.ContentEncoding] + if !ok { + ce, ok = oi.UserDefined[strings.ToLower(xhttp.ContentEncoding)] + } + if ok { + oi.ContentEncoding = ce + } + return tgt, oi, true } - oi.UserDefined = make(map[string]string, len(objInfo.Metadata)) - for k, v := range objInfo.Metadata { - oi.UserDefined[k] = v[0] - } - ce, ok := oi.UserDefined[xhttp.ContentEncoding] - if !ok { - ce, ok = oi.UserDefined[strings.ToLower(xhttp.ContentEncoding)] - } - if ok { - oi.ContentEncoding = ce - } - return tgt, oi, true, nil + return nil, oi, false } // get object info from replication target if active-active replication is in place and // this node returns a 404 -func proxyHeadToReplicationTarget(ctx context.Context, bucket, object string, opts ObjectOptions) (oi ObjectInfo, proxy bool, err error) { - _, oi, proxy, err = proxyHeadToRepTarget(ctx, bucket, object, opts) - return oi, proxy, err +func proxyHeadToReplicationTarget(ctx context.Context, bucket, object string, opts ObjectOptions, proxyTargets *madmin.BucketTargets) (oi ObjectInfo, proxy bool) { + _, oi, proxy = proxyHeadToRepTarget(ctx, bucket, object, opts, proxyTargets) + return oi, proxy } -func scheduleReplication(ctx context.Context, objInfo ObjectInfo, o ObjectLayer, sync bool, opType replication.Type) { - if sync { - replicateObject(ctx, ReplicateObjectInfo{ObjectInfo: objInfo, OpType: opType}, o, ReplicateIncoming) +func scheduleReplication(ctx context.Context, objInfo ObjectInfo, o ObjectLayer, dsc ReplicateDecision, opType replication.Type) { + if dsc.Synchronous() { + replicateObject(ctx, ReplicateObjectInfo{ObjectInfo: objInfo, OpType: opType, Dsc: dsc}, o, ReplicateIncoming) } else { - globalReplicationPool.queueReplicaTask(ReplicateObjectInfo{ObjectInfo: objInfo, OpType: opType}) + globalReplicationPool.queueReplicaTask(ReplicateObjectInfo{ObjectInfo: objInfo, OpType: opType, Dsc: dsc}) } if sz, err := objInfo.GetActualSize(); err == nil { - globalReplicationStats.Update(objInfo.Bucket, sz, objInfo.ReplicationStatus, replication.StatusType(""), opType) + for arn := range dsc.targetsMap { + globalReplicationStats.Update(objInfo.Bucket, arn, sz, objInfo.ReplicationStatus, replication.StatusType(""), opType) + } } } -func scheduleReplicationDelete(ctx context.Context, dv DeletedObjectReplicationInfo, o ObjectLayer, sync bool) { +func scheduleReplicationDelete(ctx context.Context, dv DeletedObjectReplicationInfo, o ObjectLayer) { globalReplicationPool.queueReplicaDeleteTask(dv) - globalReplicationStats.Update(dv.Bucket, 0, replication.Pending, replication.StatusType(""), replication.DeleteReplicationType) + for arn := range dv.ReplicationState.Targets { + globalReplicationStats.Update(dv.Bucket, arn, 0, replication.Pending, replication.StatusType(""), replication.DeleteReplicationType) + } + for arn := range dv.ReplicationState.PurgeTargets { + globalReplicationStats.Update(dv.Bucket, arn, 0, replication.Pending, replication.StatusType(""), replication.DeleteReplicationType) + } } type replicationConfig struct { - Config *replication.Config - ResetID string - ResetBeforeDate time.Time + Config *replication.Config + remotes *madmin.BucketTargets } func (c replicationConfig) Empty() bool { @@ -1429,59 +1650,109 @@ func (c replicationConfig) Replicate(opts replication.ObjectOpts) bool { } // Resync returns true if replication reset is requested -func (c replicationConfig) Resync(ctx context.Context, oi ObjectInfo) bool { +func (c replicationConfig) Resync(ctx context.Context, oi ObjectInfo, dsc *ReplicateDecision, tgtStatuses map[string]replication.StatusType) (r ResyncDecision) { if c.Empty() { - return false + return } // existing object replication does not apply to un-versioned objects if oi.VersionID == "" || oi.VersionID == nullVersionID { - return false + return } - var replicate bool + // Now overlay existing object replication choices for target if oi.DeleteMarker { - if c.Replicate(replication.ObjectOpts{ + opts := replication.ObjectOpts{ Name: oi.Name, SSEC: crypto.SSEC.IsEncrypted(oi.UserDefined), UserTags: oi.UserTags, DeleteMarker: oi.DeleteMarker, VersionID: oi.VersionID, OpType: replication.DeleteReplicationType, - ExistingObject: true}) { - replicate = true + ExistingObject: true} + + tgtArns := c.Config.FilterTargetArns(opts) + // indicates no matching target with Existing object replication enabled. + if len(tgtArns) == 0 { + return } - } else { - // Ignore previous replication status when deciding if object can be re-replicated - objInfo := oi.Clone() - objInfo.ReplicationStatus = replication.StatusType("") - replicate, _ = mustReplicate(ctx, oi.Bucket, oi.Name, getMustReplicateOptions(objInfo, replication.ExistingObjectReplicationType)) + for _, t := range tgtArns { + opts.TargetArn = t + // Update replication decision for target based on existing object replciation rule. + dsc.Set(newReplicateTargetDecision(t, c.Replicate(opts), false)) + } + return c.resync(oi, dsc, tgtStatuses) } - return c.resync(oi, replicate) + + // Ignore previous replication status when deciding if object can be re-replicated + objInfo := oi.Clone() + objInfo.ReplicationStatusInternal = "" + objInfo.VersionPurgeStatusInternal = "" + objInfo.ReplicationStatus = "" + objInfo.VersionPurgeStatus = "" + resyncdsc := mustReplicate(ctx, oi.Bucket, oi.Name, getMustReplicateOptions(objInfo, replication.ExistingObjectReplicationType, ObjectOptions{})) + dsc = &resyncdsc + return c.resync(oi, dsc, tgtStatuses) } // wrapper function for testability. Returns true if a new reset is requested on // already replicated objects OR object qualifies for existing object replication // and no reset requested. -func (c replicationConfig) resync(oi ObjectInfo, replicate bool) bool { - if !replicate { - return false +func (c replicationConfig) resync(oi ObjectInfo, dsc *ReplicateDecision, tgtStatuses map[string]replication.StatusType) (r ResyncDecision) { + r = ResyncDecision{ + targets: make(map[string]ResyncTargetDecision), } - rs, ok := oi.UserDefined[xhttp.MinIOReplicationResetStatus] - if !ok { // existing object replication is enabled and object version is unreplicated so far. - if c.ResetID != "" && oi.ModTime.Before(c.ResetBeforeDate) { // trigger replication if `mc replicate reset` requested - return true + if c.remotes == nil { + return + } + for _, tgt := range c.remotes.Targets { + d, ok := dsc.targetsMap[tgt.Arn] + if !ok { + continue } - return oi.ReplicationStatus != replication.Completed + if !d.Replicate { + continue + } + r.targets[d.Arn] = resyncTarget(oi, tgt.Arn, tgt.ResetID, tgt.ResetBeforeDate, tgtStatuses[tgt.Arn]) } - if c.ResetID == "" || c.ResetBeforeDate.Equal(timeSentinel) { // no reset in progress - return false + return +} + +func targetResetHeader(arn string) string { + return fmt.Sprintf("%s-%s", ReservedMetadataPrefixLower+ReplicationReset, arn) +} + +func resyncTarget(oi ObjectInfo, arn string, resetID string, resetBeforeDate time.Time, tgtStatus replication.StatusType) (rd ResyncTargetDecision) { + rd = ResyncTargetDecision{ + ResetID: resetID, + ResetBeforeDate: resetBeforeDate, } + rs, ok := oi.UserDefined[targetResetHeader(arn)] + if !ok { + rs, ok = oi.UserDefined[xhttp.MinIOReplicationResetStatus] //for backward compatibility + } + if !ok { // existing object replication is enabled and object version is unreplicated so far. + if resetID != "" && oi.ModTime.Before(resetBeforeDate) { // trigger replication if `mc replicate reset` requested + rd.Replicate = true + return + } + // For existing object reset - this condition is needed + rd.Replicate = tgtStatus == "" + return + } + if resetID == "" || resetBeforeDate.Equal(timeSentinel) { // no reset in progress + return + } + // if already replicated, return true if a new reset was requested. splits := strings.SplitN(rs, ";", 2) - newReset := splits[1] != c.ResetID - if !newReset && oi.ReplicationStatus == replication.Completed { - // already replicated and no reset requested - return false + if len(splits) != 2 { + return } - return newReset && oi.ModTime.Before(c.ResetBeforeDate) + newReset := splits[1] != resetID + if !newReset && tgtStatus == replication.Completed { + // already replicated and no reset requested + return + } + rd.Replicate = newReset && oi.ModTime.Before(resetBeforeDate) + return } diff --git a/cmd/bucket-replication_test.go b/cmd/bucket-replication_test.go index f307c8389..aee42ad98 100644 --- a/cmd/bucket-replication_test.go +++ b/cmd/bucket-replication_test.go @@ -24,6 +24,7 @@ import ( "testing" "time" + "github.com/minio/madmin-go" "github.com/minio/minio/internal/bucket/replication" xhttp "github.com/minio/minio/internal/http" ) @@ -50,6 +51,8 @@ var replicationConfigTests = []struct { info ObjectInfo name string rcfg replicationConfig + dsc ReplicateDecision + tgtStatuses map[string]replication.StatusType expectedSync bool }{ { //1. no replication config @@ -84,8 +87,8 @@ var replicationConfigTests = []struct { func TestReplicationResync(t *testing.T) { ctx := context.Background() for i, test := range replicationConfigTests { - if sync := test.rcfg.Resync(ctx, test.info); sync != test.expectedSync { - t.Errorf("Test%d (%s): Resync got %t , want %t", i+1, test.name, sync, test.expectedSync) + if sync := test.rcfg.Resync(ctx, test.info, &test.dsc, test.tgtStatuses); sync.mustResync() != test.expectedSync { + t.Errorf("Test%d (%s): Resync got %t , want %t", i+1, test.name, sync.mustResync(), test.expectedSync) } } } @@ -94,27 +97,37 @@ var start = UTCNow().AddDate(0, 0, -1) var replicationConfigTests2 = []struct { info ObjectInfo name string - replicate bool rcfg replicationConfig + dsc ReplicateDecision + tgtStatuses map[string]replication.StatusType expectedSync bool }{ { // Cases 1-4: existing object replication enabled, versioning enabled, no reset - replication status varies // 1: Pending replication name: "existing object replication on object in Pending replication status", info: ObjectInfo{Size: 100, - ReplicationStatus: replication.Pending, - VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + ReplicationStatusInternal: "arn1:PENDING;", + ReplicationStatus: replication.Pending, + VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", }, - replicate: true, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + }}}}, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, expectedSync: true, }, + { // 2. replication status Failed name: "existing object replication on object in Failed replication status", info: ObjectInfo{Size: 100, - ReplicationStatus: replication.Failed, - VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + ReplicationStatusInternal: "arn1:FAILED", + ReplicationStatus: replication.Failed, + VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", }, - replicate: true, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + }}}}, expectedSync: true, }, { //3. replication status unset @@ -123,87 +136,136 @@ var replicationConfigTests2 = []struct { ReplicationStatus: replication.StatusType(""), VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", }, - replicate: true, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + }}}}, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, expectedSync: true, }, { //4. replication status Complete name: "existing object replication on object in Completed replication status", info: ObjectInfo{Size: 100, - ReplicationStatus: replication.Completed, - VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + ReplicationStatusInternal: "arn1:COMPLETED", + ReplicationStatus: replication.Completed, + VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", }, - replicate: true, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", false, false)}}, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + }}}}, expectedSync: false, }, { //5. existing object replication enabled, versioning enabled, replication status Pending & reset ID present name: "existing object replication with reset in progress and object in Pending status", info: ObjectInfo{Size: 100, - ReplicationStatus: replication.Pending, - VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + ReplicationStatusInternal: "arn1:PENDING;", + ReplicationStatus: replication.Pending, + VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + UserDefined: map[string]string{xhttp.MinIOReplicationResetStatus: fmt.Sprintf("%s;abc", UTCNow().AddDate(0, -1, 0).String())}, }, - replicate: true, expectedSync: true, - rcfg: replicationConfig{ResetID: "xyz", ResetBeforeDate: UTCNow()}, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + ResetID: "xyz", + ResetBeforeDate: UTCNow(), + }}}, + }, }, { //6. existing object replication enabled, versioning enabled, replication status Failed & reset ID present name: "existing object replication with reset in progress and object in Failed status", info: ObjectInfo{Size: 100, - ReplicationStatus: replication.Failed, - VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + ReplicationStatusInternal: "arn1:FAILED;", + ReplicationStatus: replication.Failed, + VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + UserDefined: map[string]string{xhttp.MinIOReplicationResetStatus: fmt.Sprintf("%s;abc", UTCNow().AddDate(0, -1, 0).String())}, + }, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + ResetID: "xyz", + ResetBeforeDate: UTCNow(), + }}}, }, - replicate: true, expectedSync: true, - rcfg: replicationConfig{ResetID: "xyz", ResetBeforeDate: UTCNow()}, }, { //7. existing object replication enabled, versioning enabled, replication status unset & reset ID present name: "existing object replication with reset in progress and object never replicated before", info: ObjectInfo{Size: 100, ReplicationStatus: replication.StatusType(""), VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + UserDefined: map[string]string{xhttp.MinIOReplicationResetStatus: fmt.Sprintf("%s;abc", UTCNow().AddDate(0, -1, 0).String())}, }, - replicate: true, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + ResetID: "xyz", + ResetBeforeDate: UTCNow(), + }}}, + }, + expectedSync: true, - rcfg: replicationConfig{ResetID: "xyz", ResetBeforeDate: UTCNow()}, }, + { //8. existing object replication enabled, versioning enabled, replication status Complete & reset ID present name: "existing object replication enabled - reset in progress for an object in Completed status", info: ObjectInfo{Size: 100, - ReplicationStatus: replication.Completed, - VersionID: "a3348c34-c352-4498-82f0-1098e8b34df8", + ReplicationStatusInternal: "arn1:COMPLETED;", + ReplicationStatus: replication.Completed, + VersionID: "a3348c34-c352-4498-82f0-1098e8b34df8", + UserDefined: map[string]string{xhttp.MinIOReplicationResetStatus: fmt.Sprintf("%s;abc", UTCNow().AddDate(0, -1, 0).String())}, }, - replicate: true, expectedSync: true, - rcfg: replicationConfig{ResetID: "xyz", ResetBeforeDate: UTCNow()}, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + ResetID: "xyz", + ResetBeforeDate: UTCNow(), + }}}, + }, }, { //9. existing object replication enabled, versioning enabled, replication status Pending & reset ID different name: "existing object replication enabled, newer reset in progress on object in Pending replication status", info: ObjectInfo{Size: 100, + ReplicationStatusInternal: "arn1:PENDING;", + ReplicationStatus: replication.Pending, VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", - UserDefined: map[string]string{xhttp.MinIOReplicationResetStatus: fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), "xyz")}, - ModTime: UTCNow().AddDate(0, 0, -1), + UserDefined: map[string]string{xhttp.MinIOReplicationResetStatus: fmt.Sprintf("%s;%s", UTCNow().AddDate(0, 0, -1).Format(http.TimeFormat), "abc")}, + ModTime: UTCNow().AddDate(0, 0, -2), }, - replicate: true, expectedSync: true, - rcfg: replicationConfig{ResetID: "abc", ResetBeforeDate: UTCNow()}, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + ResetID: "xyz", + ResetBeforeDate: UTCNow(), + }}}, + }, }, { //10. existing object replication enabled, versioning enabled, replication status Complete & reset done name: "reset done on object in Completed Status - ineligbile for re-replication", info: ObjectInfo{Size: 100, - ReplicationStatus: replication.Completed, - VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", - UserDefined: map[string]string{xhttp.MinIOReplicationResetStatus: fmt.Sprintf("%s;%s", start.Format(http.TimeFormat), "xyz")}, + ReplicationStatusInternal: "arn1:COMPLETED;", + ReplicationStatus: replication.Completed, + VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9", + UserDefined: map[string]string{xhttp.MinIOReplicationResetStatus: fmt.Sprintf("%s;%s", start.Format(http.TimeFormat), "xyz")}, }, - replicate: true, expectedSync: false, - rcfg: replicationConfig{ResetID: "xyz", ResetBeforeDate: UTCNow()}, + dsc: ReplicateDecision{targetsMap: map[string]replicateTargetDecision{"arn1": newReplicateTargetDecision("arn1", true, false)}}, + rcfg: replicationConfig{remotes: &madmin.BucketTargets{Targets: []madmin.BucketTarget{{ + Arn: "arn1", + ResetID: "xyz", + ResetBeforeDate: start, + }}}, + }, }, } func TestReplicationResyncwrapper(t *testing.T) { for i, test := range replicationConfigTests2 { - if sync := test.rcfg.resync(test.info, test.replicate); sync != test.expectedSync { - t.Errorf("%s (%s): Replicationresync got %t , want %t", fmt.Sprintf("Test%d - %s", i+1, time.Now().Format(http.TimeFormat)), test.name, sync, test.expectedSync) + if sync := test.rcfg.resync(test.info, &test.dsc, test.tgtStatuses); sync.mustResync() != test.expectedSync { + t.Errorf("%s (%s): Replicationresync got %t , want %t", fmt.Sprintf("Test%d - %s", i+1, time.Now().Format(http.TimeFormat)), test.name, sync.mustResync(), test.expectedSync) } } } diff --git a/cmd/bucket-stats.go b/cmd/bucket-stats.go index ee069589a..ab3dcbf6d 100644 --- a/cmd/bucket-stats.go +++ b/cmd/bucket-stats.go @@ -17,6 +17,10 @@ package cmd +import ( + "sync/atomic" +) + //go:generate msgp -file $GOFILE // BucketStats bucket statistics @@ -27,16 +31,98 @@ type BucketStats struct { // BucketReplicationStats represents inline replication statistics // such as pending, failed and completed bytes in total for a bucket type BucketReplicationStats struct { + Stats map[string]*BucketReplicationStat // Pending size in bytes - PendingSize uint64 `json:"pendingReplicationSize"` + PendingSize int64 `json:"pendingReplicationSize"` // Completed size in bytes - ReplicatedSize uint64 `json:"completedReplicationSize"` + ReplicatedSize int64 `json:"completedReplicationSize"` // Total Replica size in bytes - ReplicaSize uint64 `json:"replicaSize"` + ReplicaSize int64 `json:"replicaSize"` // Failed size in bytes - FailedSize uint64 `json:"failedReplicationSize"` + FailedSize int64 `json:"failedReplicationSize"` // Total number of pending operations including metadata updates - PendingCount uint64 `json:"pendingReplicationCount"` + PendingCount int64 `json:"pendingReplicationCount"` // Total number of failed operations including metadata updates - FailedCount uint64 `json:"failedReplicationCount"` + FailedCount int64 `json:"failedReplicationCount"` +} + +// Empty returns true if there are no target stats +func (brs *BucketReplicationStats) Empty() bool { + return len(brs.Stats) == 0 && brs.ReplicaSize == 0 +} + +// UpdateStat updates replication stats for the target arn +func (brs *BucketReplicationStats) UpdateStat(arn string, stat *BucketReplicationStat) { + var s BucketReplicationStat + if st, ok := brs.Stats[arn]; ok { + s = *st + } + // update target metric + atomic.AddInt64(&s.FailedSize, stat.FailedSize) + atomic.AddInt64(&s.FailedCount, stat.FailedCount) + atomic.AddInt64(&s.PendingCount, stat.PendingCount) + atomic.AddInt64(&s.PendingSize, stat.PendingSize) + atomic.AddInt64(&s.ReplicaSize, stat.ReplicaSize) + atomic.AddInt64(&s.ReplicatedSize, stat.ReplicatedSize) + // update total counts across targets + atomic.AddInt64(&brs.FailedSize, stat.FailedSize) + atomic.AddInt64(&brs.FailedCount, stat.FailedCount) + atomic.AddInt64(&brs.PendingCount, stat.PendingCount) + atomic.AddInt64(&brs.PendingSize, stat.PendingSize) + atomic.AddInt64(&brs.ReplicaSize, stat.ReplicaSize) + atomic.AddInt64(&brs.ReplicatedSize, stat.ReplicatedSize) + brs.Stats[arn] = &s +} + +// Clone creates a new BucketReplicationStats copy +func (brs BucketReplicationStats) Clone() BucketReplicationStats { + c := BucketReplicationStats{ + Stats: make(map[string]*BucketReplicationStat, len(brs.Stats)), + } + //this is called only by replicationStats cache and already holds a read lock before calling Clone() + for arn, st := range brs.Stats { + c.Stats[arn] = &BucketReplicationStat{ + FailedSize: atomic.LoadInt64(&st.FailedSize), + ReplicatedSize: atomic.LoadInt64(&st.ReplicatedSize), + ReplicaSize: atomic.LoadInt64(&st.ReplicaSize), + FailedCount: atomic.LoadInt64(&st.FailedCount), + PendingSize: atomic.LoadInt64(&st.PendingSize), + PendingCount: atomic.LoadInt64(&st.PendingCount), + } + } + // update total counts across targets + c.FailedSize = atomic.LoadInt64(&brs.FailedSize) + c.FailedCount = atomic.LoadInt64(&brs.FailedCount) + c.PendingCount = atomic.LoadInt64(&brs.PendingCount) + c.PendingSize = atomic.LoadInt64(&brs.PendingSize) + c.ReplicaSize = atomic.LoadInt64(&brs.ReplicaSize) + c.ReplicatedSize = atomic.LoadInt64(&brs.ReplicatedSize) + return c +} + +// BucketReplicationStat represents inline replication statistics +// such as pending, failed and completed bytes in total for a bucket +// remote target +type BucketReplicationStat struct { + // Pending size in bytes + PendingSize int64 `json:"pendingReplicationSize"` + // Completed size in bytes + ReplicatedSize int64 `json:"completedReplicationSize"` + // Total Replica size in bytes + ReplicaSize int64 `json:"replicaSize"` + // Failed size in bytes + FailedSize int64 `json:"failedReplicationSize"` + // Total number of pending operations including metadata updates + PendingCount int64 `json:"pendingReplicationCount"` + // Total number of failed operations including metadata updates + FailedCount int64 `json:"failedReplicationCount"` +} + +func (bs *BucketReplicationStat) hasReplicationUsage() bool { + return bs.FailedSize > 0 || + bs.ReplicatedSize > 0 || + bs.ReplicaSize > 0 || + bs.FailedCount > 0 || + bs.PendingCount > 0 || + bs.PendingSize > 0 } diff --git a/cmd/bucket-stats_gen.go b/cmd/bucket-stats_gen.go index 278798104..c0bec1059 100644 --- a/cmd/bucket-stats_gen.go +++ b/cmd/bucket-stats_gen.go @@ -6,6 +6,234 @@ import ( "github.com/tinylib/msgp/msgp" ) +// DecodeMsg implements msgp.Decodable +func (z *BucketReplicationStat) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "PendingSize": + z.PendingSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } + case "ReplicatedSize": + z.ReplicatedSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + case "ReplicaSize": + z.ReplicaSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + case "FailedSize": + z.FailedSize, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "FailedSize") + return + } + case "PendingCount": + z.PendingCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "PendingCount") + return + } + case "FailedCount": + z.FailedCount, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "FailedCount") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *BucketReplicationStat) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 6 + // write "PendingSize" + err = en.Append(0x86, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.PendingSize) + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } + // write "ReplicatedSize" + err = en.Append(0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicatedSize) + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + // write "ReplicaSize" + err = en.Append(0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.ReplicaSize) + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + // write "FailedSize" + err = en.Append(0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.FailedSize) + if err != nil { + err = msgp.WrapError(err, "FailedSize") + return + } + // write "PendingCount" + err = en.Append(0xac, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.PendingCount) + if err != nil { + err = msgp.WrapError(err, "PendingCount") + return + } + // write "FailedCount" + err = en.Append(0xab, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + if err != nil { + return + } + err = en.WriteInt64(z.FailedCount) + if err != nil { + err = msgp.WrapError(err, "FailedCount") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *BucketReplicationStat) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 6 + // string "PendingSize" + o = append(o, 0x86, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.PendingSize) + // string "ReplicatedSize" + o = append(o, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.ReplicatedSize) + // string "ReplicaSize" + o = append(o, 0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.ReplicaSize) + // string "FailedSize" + o = append(o, 0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.FailedSize) + // string "PendingCount" + o = append(o, 0xac, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.PendingCount) + // string "FailedCount" + o = append(o, 0xab, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) + o = msgp.AppendInt64(o, z.FailedCount) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *BucketReplicationStat) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "PendingSize": + z.PendingSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "PendingSize") + return + } + case "ReplicatedSize": + z.ReplicatedSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicatedSize") + return + } + case "ReplicaSize": + z.ReplicaSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + case "FailedSize": + z.FailedSize, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "FailedSize") + return + } + case "PendingCount": + z.PendingCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "PendingCount") + return + } + case "FailedCount": + z.FailedCount, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "FailedCount") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *BucketReplicationStat) Msgsize() (s int) { + s = 1 + 12 + msgp.Int64Size + 15 + msgp.Int64Size + 12 + msgp.Int64Size + 11 + msgp.Int64Size + 13 + msgp.Int64Size + 12 + msgp.Int64Size + return +} + // DecodeMsg implements msgp.Decodable func (z *BucketReplicationStats) DecodeMsg(dc *msgp.Reader) (err error) { var field []byte @@ -24,38 +252,80 @@ func (z *BucketReplicationStats) DecodeMsg(dc *msgp.Reader) (err error) { return } switch msgp.UnsafeString(field) { + case "Stats": + var zb0002 uint32 + zb0002, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Stats") + return + } + if z.Stats == nil { + z.Stats = make(map[string]*BucketReplicationStat, zb0002) + } else if len(z.Stats) > 0 { + for key := range z.Stats { + delete(z.Stats, key) + } + } + for zb0002 > 0 { + zb0002-- + var za0001 string + var za0002 *BucketReplicationStat + za0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Stats") + return + } + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "Stats", za0001) + return + } + za0002 = nil + } else { + if za0002 == nil { + za0002 = new(BucketReplicationStat) + } + err = za0002.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Stats", za0001) + return + } + } + z.Stats[za0001] = za0002 + } case "PendingSize": - z.PendingSize, err = dc.ReadUint64() + z.PendingSize, err = dc.ReadInt64() if err != nil { err = msgp.WrapError(err, "PendingSize") return } case "ReplicatedSize": - z.ReplicatedSize, err = dc.ReadUint64() + z.ReplicatedSize, err = dc.ReadInt64() if err != nil { err = msgp.WrapError(err, "ReplicatedSize") return } case "ReplicaSize": - z.ReplicaSize, err = dc.ReadUint64() + z.ReplicaSize, err = dc.ReadInt64() if err != nil { err = msgp.WrapError(err, "ReplicaSize") return } case "FailedSize": - z.FailedSize, err = dc.ReadUint64() + z.FailedSize, err = dc.ReadInt64() if err != nil { err = msgp.WrapError(err, "FailedSize") return } case "PendingCount": - z.PendingCount, err = dc.ReadUint64() + z.PendingCount, err = dc.ReadInt64() if err != nil { err = msgp.WrapError(err, "PendingCount") return } case "FailedCount": - z.FailedCount, err = dc.ReadUint64() + z.FailedCount, err = dc.ReadInt64() if err != nil { err = msgp.WrapError(err, "FailedCount") return @@ -73,13 +343,42 @@ func (z *BucketReplicationStats) DecodeMsg(dc *msgp.Reader) (err error) { // EncodeMsg implements msgp.Encodable func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { - // map header, size 6 - // write "PendingSize" - err = en.Append(0x86, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + // map header, size 7 + // write "Stats" + err = en.Append(0x87, 0xa5, 0x53, 0x74, 0x61, 0x74, 0x73) if err != nil { return } - err = en.WriteUint64(z.PendingSize) + err = en.WriteMapHeader(uint32(len(z.Stats))) + if err != nil { + err = msgp.WrapError(err, "Stats") + return + } + for za0001, za0002 := range z.Stats { + err = en.WriteString(za0001) + if err != nil { + err = msgp.WrapError(err, "Stats") + return + } + if za0002 == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = za0002.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Stats", za0001) + return + } + } + } + // write "PendingSize" + err = en.Append(0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + if err != nil { + return + } + err = en.WriteInt64(z.PendingSize) if err != nil { err = msgp.WrapError(err, "PendingSize") return @@ -89,7 +388,7 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { if err != nil { return } - err = en.WriteUint64(z.ReplicatedSize) + err = en.WriteInt64(z.ReplicatedSize) if err != nil { err = msgp.WrapError(err, "ReplicatedSize") return @@ -99,7 +398,7 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { if err != nil { return } - err = en.WriteUint64(z.ReplicaSize) + err = en.WriteInt64(z.ReplicaSize) if err != nil { err = msgp.WrapError(err, "ReplicaSize") return @@ -109,7 +408,7 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { if err != nil { return } - err = en.WriteUint64(z.FailedSize) + err = en.WriteInt64(z.FailedSize) if err != nil { err = msgp.WrapError(err, "FailedSize") return @@ -119,7 +418,7 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { if err != nil { return } - err = en.WriteUint64(z.PendingCount) + err = en.WriteInt64(z.PendingCount) if err != nil { err = msgp.WrapError(err, "PendingCount") return @@ -129,7 +428,7 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { if err != nil { return } - err = en.WriteUint64(z.FailedCount) + err = en.WriteInt64(z.FailedCount) if err != nil { err = msgp.WrapError(err, "FailedCount") return @@ -140,25 +439,40 @@ func (z *BucketReplicationStats) EncodeMsg(en *msgp.Writer) (err error) { // MarshalMsg implements msgp.Marshaler func (z *BucketReplicationStats) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // map header, size 6 + // map header, size 7 + // string "Stats" + o = append(o, 0x87, 0xa5, 0x53, 0x74, 0x61, 0x74, 0x73) + o = msgp.AppendMapHeader(o, uint32(len(z.Stats))) + for za0001, za0002 := range z.Stats { + o = msgp.AppendString(o, za0001) + if za0002 == nil { + o = msgp.AppendNil(o) + } else { + o, err = za0002.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Stats", za0001) + return + } + } + } // string "PendingSize" - o = append(o, 0x86, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) - o = msgp.AppendUint64(o, z.PendingSize) + o = append(o, 0xab, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65) + o = msgp.AppendInt64(o, z.PendingSize) // string "ReplicatedSize" o = append(o, 0xae, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) - o = msgp.AppendUint64(o, z.ReplicatedSize) + o = msgp.AppendInt64(o, z.ReplicatedSize) // string "ReplicaSize" o = append(o, 0xab, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x69, 0x7a, 0x65) - o = msgp.AppendUint64(o, z.ReplicaSize) + o = msgp.AppendInt64(o, z.ReplicaSize) // string "FailedSize" o = append(o, 0xaa, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x53, 0x69, 0x7a, 0x65) - o = msgp.AppendUint64(o, z.FailedSize) + o = msgp.AppendInt64(o, z.FailedSize) // string "PendingCount" o = append(o, 0xac, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74) - o = msgp.AppendUint64(o, z.PendingCount) + o = msgp.AppendInt64(o, z.PendingCount) // string "FailedCount" o = append(o, 0xab, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x43, 0x6f, 0x75, 0x6e, 0x74) - o = msgp.AppendUint64(o, z.FailedCount) + o = msgp.AppendInt64(o, z.FailedCount) return } @@ -180,38 +494,79 @@ func (z *BucketReplicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) return } switch msgp.UnsafeString(field) { + case "Stats": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Stats") + return + } + if z.Stats == nil { + z.Stats = make(map[string]*BucketReplicationStat, zb0002) + } else if len(z.Stats) > 0 { + for key := range z.Stats { + delete(z.Stats, key) + } + } + for zb0002 > 0 { + var za0001 string + var za0002 *BucketReplicationStat + zb0002-- + za0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Stats") + return + } + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + za0002 = nil + } else { + if za0002 == nil { + za0002 = new(BucketReplicationStat) + } + bts, err = za0002.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Stats", za0001) + return + } + } + z.Stats[za0001] = za0002 + } case "PendingSize": - z.PendingSize, bts, err = msgp.ReadUint64Bytes(bts) + z.PendingSize, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { err = msgp.WrapError(err, "PendingSize") return } case "ReplicatedSize": - z.ReplicatedSize, bts, err = msgp.ReadUint64Bytes(bts) + z.ReplicatedSize, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { err = msgp.WrapError(err, "ReplicatedSize") return } case "ReplicaSize": - z.ReplicaSize, bts, err = msgp.ReadUint64Bytes(bts) + z.ReplicaSize, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { err = msgp.WrapError(err, "ReplicaSize") return } case "FailedSize": - z.FailedSize, bts, err = msgp.ReadUint64Bytes(bts) + z.FailedSize, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { err = msgp.WrapError(err, "FailedSize") return } case "PendingCount": - z.PendingCount, bts, err = msgp.ReadUint64Bytes(bts) + z.PendingCount, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { err = msgp.WrapError(err, "PendingCount") return } case "FailedCount": - z.FailedCount, bts, err = msgp.ReadUint64Bytes(bts) + z.FailedCount, bts, err = msgp.ReadInt64Bytes(bts) if err != nil { err = msgp.WrapError(err, "FailedCount") return @@ -230,7 +585,19 @@ func (z *BucketReplicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) // Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message func (z *BucketReplicationStats) Msgsize() (s int) { - s = 1 + 12 + msgp.Uint64Size + 15 + msgp.Uint64Size + 12 + msgp.Uint64Size + 11 + msgp.Uint64Size + 13 + msgp.Uint64Size + 12 + msgp.Uint64Size + s = 1 + 6 + msgp.MapHeaderSize + if z.Stats != nil { + for za0001, za0002 := range z.Stats { + _ = za0002 + s += msgp.StringPrefixSize + len(za0001) + if za0002 == nil { + s += msgp.NilSize + } else { + s += za0002.Msgsize() + } + } + } + s += 12 + msgp.Int64Size + 15 + msgp.Int64Size + 12 + msgp.Int64Size + 11 + msgp.Int64Size + 13 + msgp.Int64Size + 12 + msgp.Int64Size return } diff --git a/cmd/bucket-stats_gen_test.go b/cmd/bucket-stats_gen_test.go index 6c435d5ec..33328d4f4 100644 --- a/cmd/bucket-stats_gen_test.go +++ b/cmd/bucket-stats_gen_test.go @@ -9,6 +9,119 @@ import ( "github.com/tinylib/msgp/msgp" ) +func TestMarshalUnmarshalBucketReplicationStat(t *testing.T) { + v := BucketReplicationStat{} + 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 BenchmarkMarshalMsgBucketReplicationStat(b *testing.B) { + v := BucketReplicationStat{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgBucketReplicationStat(b *testing.B) { + v := BucketReplicationStat{} + 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 BenchmarkUnmarshalBucketReplicationStat(b *testing.B) { + v := BucketReplicationStat{} + 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 TestEncodeDecodeBucketReplicationStat(t *testing.T) { + v := BucketReplicationStat{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeBucketReplicationStat Msgsize() is inaccurate") + } + + vn := BucketReplicationStat{} + 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 BenchmarkEncodeBucketReplicationStat(b *testing.B) { + v := BucketReplicationStat{} + 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 BenchmarkDecodeBucketReplicationStat(b *testing.B) { + v := BucketReplicationStat{} + 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 TestMarshalUnmarshalBucketReplicationStats(t *testing.T) { v := BucketReplicationStats{} bts, err := v.MarshalMsg(nil) diff --git a/cmd/bucket-targets.go b/cmd/bucket-targets.go index 8699c1add..b7e3297d7 100644 --- a/cmd/bucket-targets.go +++ b/cmd/bucket-targets.go @@ -19,8 +19,6 @@ package cmd import ( "context" - "crypto/sha256" - "encoding/hex" "net/http" "sync" "time" @@ -30,6 +28,7 @@ import ( minio "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/internal/bucket/replication" "github.com/minio/minio/internal/bucket/versioning" "github.com/minio/minio/internal/crypto" "github.com/minio/minio/internal/kms" @@ -139,17 +138,18 @@ func (sys *BucketTargetSys) SetTarget(ctx context.Context, bucket string, tgt *m defer sys.Unlock() tgts := sys.targetsMap[bucket] - newtgts := make([]madmin.BucketTarget, len(tgts)) found := false for idx, t := range tgts { if t.Type == tgt.Type { - if t.Arn == tgt.Arn && !update { - return BucketRemoteAlreadyExists{Bucket: t.TargetBucket} + if t.Arn == tgt.Arn { + if !update { + return BucketRemoteAlreadyExists{Bucket: t.TargetBucket} + } + newtgts[idx] = *tgt + found = true + continue } - newtgts[idx] = *tgt - found = true - continue } newtgts[idx] = t } @@ -160,7 +160,6 @@ func (sys *BucketTargetSys) SetTarget(ctx context.Context, bucket string, tgt *m if prevClnt, ok := sys.arnRemotesMap[tgt.Arn]; ok && prevClnt.healthCancelFn != nil { prevClnt.healthCancelFn() } - sys.targetsMap[bucket] = newtgts sys.arnRemotesMap[tgt.Arn] = clnt sys.updateBandwidthLimit(bucket, tgt.BandwidthLimit) @@ -201,12 +200,16 @@ func (sys *BucketTargetSys) RemoveTarget(ctx context.Context, bucket, arnStr str if arn.Type == madmin.ReplicationService { // reject removal of remote target if replication configuration is present rcfg, err := getReplicationConfig(ctx, bucket) - if err == nil && rcfg.RoleArn == arnStr { - sys.RLock() - _, ok := sys.arnRemotesMap[arnStr] - sys.RUnlock() - if ok { - return BucketRemoteRemoveDisallowed{Bucket: bucket} + if err == nil { + for _, tgtArn := range rcfg.FilterTargetArns(replication.ObjectOpts{}) { + if err == nil && (tgtArn == arnStr || rcfg.RoleArn == arnStr) { + sys.RLock() + _, ok := sys.arnRemotesMap[arnStr] + sys.RUnlock() + if ok { + return BucketRemoteRemoveDisallowed{Bucket: bucket} + } + } } } } @@ -385,6 +388,8 @@ func (sys *BucketTargetSys) getRemoteTargetClient(tcfg *madmin.BucketTarget) (*T StorageClass: tcfg.StorageClass, disableProxy: tcfg.DisableProxy, healthCancelFn: cancelFn, + ARN: tcfg.Arn, + ResetID: tcfg.ResetID, } return tc, nil } @@ -408,14 +413,9 @@ func (sys *BucketTargetSys) getRemoteARN(bucket string, target *madmin.BucketTar // generate ARN that is unique to this target type func generateARN(t *madmin.BucketTarget) string { - hash := sha256.New() - hash.Write([]byte(t.Type)) - hash.Write([]byte(t.Region)) - hash.Write([]byte(t.TargetBucket)) - hashSum := hex.EncodeToString(hash.Sum(nil)) arn := madmin.ARN{ Type: t.Type, - ID: hashSum, + ID: mustGetUUID(), Region: t.Region, Bucket: t.TargetBucket, } @@ -464,4 +464,6 @@ type TargetClient struct { StorageClass string // storage class on remote disableProxy bool healthCancelFn context.CancelFunc // cancellation function for client healthcheck + ARN string //ARN to uniquely identify remote target + ResetID string } diff --git a/cmd/data-scanner.go b/cmd/data-scanner.go index 8b785da27..99dfe9f38 100644 --- a/cmd/data-scanner.go +++ b/cmd/data-scanner.go @@ -142,7 +142,7 @@ func runDataScanner(pctx context.Context, objAPI ObjectLayer) { } // Wait before starting next cycle and wait on startup. - results := make(chan madmin.DataUsageInfo, 1) + results := make(chan DataUsageInfo, 1) go storeDataUsageInBackend(ctx, objAPI, results) bf, err := globalNotificationSys.updateBloomFilter(ctx, nextBloomCycle) logger.LogIf(ctx, err) @@ -834,12 +834,22 @@ type scannerItem struct { } type sizeSummary struct { - totalSize int64 - versions uint64 + totalSize int64 + versions uint64 + replicatedSize int64 + pendingSize int64 + failedSize int64 + replicaSize int64 + pendingCount uint64 + failedCount uint64 + replTargetStats map[string]replTargetSizeSummary +} + +// replTargetSizeSummary holds summary of replication stats by target +type replTargetSizeSummary struct { replicatedSize int64 pendingSize int64 failedSize int64 - replicaSize int64 pendingCount uint64 failedCount uint64 } @@ -1109,27 +1119,50 @@ func (i *scannerItem) objectPath() string { // healReplication will heal a scanned item that has failed replication. func (i *scannerItem) healReplication(ctx context.Context, o ObjectLayer, oi ObjectInfo, sizeS *sizeSummary) { - existingObjResync := i.replication.Resync(ctx, oi) + roi := getHealReplicateObjectInfo(oi, i.replication) + if oi.DeleteMarker || !oi.VersionPurgeStatus.Empty() { // heal delete marker replication failure or versioned delete replication failure if oi.ReplicationStatus == replication.Pending || oi.ReplicationStatus == replication.Failed || oi.VersionPurgeStatus == Failed || oi.VersionPurgeStatus == Pending { - i.healReplicationDeletes(ctx, o, oi, existingObjResync) + i.healReplicationDeletes(ctx, o, roi) return } // if replication status is Complete on DeleteMarker and existing object resync required - if existingObjResync && (oi.ReplicationStatus == replication.Completed) { - i.healReplicationDeletes(ctx, o, oi, existingObjResync) + if roi.ExistingObjResync.mustResync() && (oi.ReplicationStatus == replication.Completed || oi.ReplicationStatus.Empty()) { + i.healReplicationDeletes(ctx, o, roi) return } return } - roi := ReplicateObjectInfo{ObjectInfo: oi, OpType: replication.HealReplicationType} - if existingObjResync { + if roi.ExistingObjResync.mustResync() { roi.OpType = replication.ExistingObjectReplicationType - roi.ResetID = i.replication.ResetID } + + if roi.TargetStatuses != nil { + if sizeS.replTargetStats == nil { + sizeS.replTargetStats = make(map[string]replTargetSizeSummary) + } + for arn, tgtStatus := range roi.TargetStatuses { + tgtSizeS, ok := sizeS.replTargetStats[arn] + if !ok { + tgtSizeS = replTargetSizeSummary{} + } + switch tgtStatus { + case replication.Pending: + tgtSizeS.pendingCount++ + tgtSizeS.pendingSize += oi.Size + case replication.Failed: + tgtSizeS.failedSize += oi.Size + tgtSizeS.failedCount++ + case replication.Completed, "COMPLETE": + tgtSizeS.replicatedSize += oi.Size + } + sizeS.replTargetStats[arn] = tgtSizeS + } + } + switch oi.ReplicationStatus { case replication.Pending: sizeS.pendingCount++ @@ -1146,37 +1179,38 @@ func (i *scannerItem) healReplication(ctx context.Context, o ObjectLayer, oi Obj case replication.Replica: sizeS.replicaSize += oi.Size } - if existingObjResync { + if roi.ExistingObjResync.mustResync() { globalReplicationPool.queueReplicaTask(roi) } } // healReplicationDeletes will heal a scanned deleted item that failed to replicate deletes. -func (i *scannerItem) healReplicationDeletes(ctx context.Context, o ObjectLayer, oi ObjectInfo, existingObject bool) { +func (i *scannerItem) healReplicationDeletes(ctx context.Context, o ObjectLayer, roi ReplicateObjectInfo) { // handle soft delete and permanent delete failures here. - if oi.DeleteMarker || !oi.VersionPurgeStatus.Empty() { + if roi.DeleteMarker || !roi.VersionPurgeStatus.Empty() { versionID := "" dmVersionID := "" - if oi.VersionPurgeStatus.Empty() { - dmVersionID = oi.VersionID + if roi.VersionPurgeStatus.Empty() { + dmVersionID = roi.VersionID } else { - versionID = oi.VersionID + versionID = roi.VersionID } + doi := DeletedObjectReplicationInfo{ DeletedObject: DeletedObject{ - ObjectName: oi.Name, - DeleteMarkerVersionID: dmVersionID, - VersionID: versionID, - DeleteMarkerReplicationStatus: string(oi.ReplicationStatus), - DeleteMarkerMTime: DeleteMarkerMTime{oi.ModTime}, - DeleteMarker: oi.DeleteMarker, - VersionPurgeStatus: oi.VersionPurgeStatus, + ObjectName: roi.Name, + DeleteMarkerVersionID: dmVersionID, + VersionID: versionID, + ReplicationState: roi.getReplicationState(roi.Dsc.String(), versionID, true), + DeleteMarkerMTime: DeleteMarkerMTime{roi.ModTime}, + DeleteMarker: roi.DeleteMarker, }, - Bucket: oi.Bucket, + Bucket: roi.Bucket, } - if existingObject { + if roi.ExistingObjResync.mustResync() { doi.OpType = replication.ExistingObjectReplicationType - doi.ResetID = i.replication.ResetID + queueReplicateDeletesWrapper(doi, roi.ExistingObjResync) + return } globalReplicationPool.queueReplicaDeleteTask(doi) } diff --git a/cmd/data-usage-cache.go b/cmd/data-usage-cache.go index 3b0856c93..fb7adc9d9 100644 --- a/cmd/data-usage-cache.go +++ b/cmd/data-usage-cache.go @@ -31,7 +31,6 @@ import ( "github.com/cespare/xxhash/v2" "github.com/klauspost/compress/zstd" - "github.com/minio/madmin-go" "github.com/minio/minio/internal/bucket/lifecycle" "github.com/minio/minio/internal/hash" "github.com/minio/minio/internal/logger" @@ -54,7 +53,7 @@ type dataUsageEntry struct { Objects uint64 Versions uint64 // Versions that are not delete markers. ObjSizes sizeHistogram - ReplicationStats *replicationStats + ReplicationStats *replicationAllStats Compacted bool } @@ -63,7 +62,6 @@ type replicationStats struct { PendingSize uint64 ReplicatedSize uint64 FailedSize uint64 - ReplicaSize uint64 FailedCount uint64 PendingCount uint64 MissedThresholdSize uint64 @@ -72,6 +70,18 @@ type replicationStats struct { AfterThresholdCount uint64 } +func (rs replicationStats) Empty() bool { + return rs.ReplicatedSize == 0 && + rs.FailedSize == 0 && + rs.FailedCount == 0 +} + +//msgp:tuple replicationAllStats +type replicationAllStats struct { + Targets map[string]replicationStats + ReplicaSize uint64 +} + //msgp:encode ignore dataUsageEntryV2 dataUsageEntryV3 dataUsageEntryV4 //msgp:marshal ignore dataUsageEntryV2 dataUsageEntryV3 dataUsageEntryV4 @@ -107,6 +117,18 @@ type dataUsageEntryV4 struct { ReplicationStats replicationStats } +//msgp:tuple dataUsageEntryV5 +type dataUsageEntryV5 struct { + Children dataUsageHashMap + // These fields do no include any children. + Size int64 + Objects uint64 + Versions uint64 // Versions that are not delete markers. + ObjSizes sizeHistogram + ReplicationStats *replicationStats + Compacted bool +} + // dataUsageCache contains a cache of data usage entries latest version. type dataUsageCache struct { Info dataUsageCacheInfo @@ -114,8 +136,8 @@ type dataUsageCache struct { Disks []string } -//msgp:encode ignore dataUsageCacheV2 dataUsageCacheV3 dataUsageCacheV4 -//msgp:marshal ignore dataUsageCacheV2 dataUsageCacheV3 dataUsageCacheV4 +//msgp:encode ignore dataUsageCacheV2 dataUsageCacheV3 dataUsageCacheV4 dataUsageCacheV5 +//msgp:marshal ignore dataUsageCacheV2 dataUsageCacheV3 dataUsageCacheV4 dataUsageCacheV5 // dataUsageCacheV2 contains a cache of data usage entries version 2. type dataUsageCacheV2 struct { @@ -138,6 +160,13 @@ type dataUsageCacheV4 struct { Cache map[string]dataUsageEntryV4 } +// dataUsageCache contains a cache of data usage entries version 5. +type dataUsageCacheV5 struct { + Info dataUsageCacheInfo + Disks []string + Cache map[string]dataUsageEntryV5 +} + //msgp:ignore dataUsageEntryInfo type dataUsageEntryInfo struct { Name string @@ -170,17 +199,22 @@ func (e *dataUsageEntry) addSizes(summary sizeSummary) { e.Versions += summary.versions e.ObjSizes.add(summary.totalSize) - if summary.replicaSize > 0 || summary.pendingSize > 0 || summary.replicatedSize > 0 || - summary.failedCount > 0 || summary.pendingCount > 0 || summary.failedSize > 0 { + if summary.replTargetStats != nil { if e.ReplicationStats == nil { - e.ReplicationStats = &replicationStats{} + e.ReplicationStats = &replicationAllStats{Targets: make(map[string]replicationStats)} + } + for arn, st := range summary.replTargetStats { + tgtStat, ok := e.ReplicationStats.Targets[arn] + if !ok { + tgtStat = replicationStats{} + } + tgtStat.PendingSize = tgtStat.PendingSize + uint64(st.pendingSize) + tgtStat.FailedSize = tgtStat.FailedSize + uint64(st.failedSize) + tgtStat.ReplicatedSize = tgtStat.ReplicatedSize + uint64(st.replicatedSize) + tgtStat.FailedCount = tgtStat.FailedCount + st.failedCount + tgtStat.PendingCount = tgtStat.PendingCount + st.pendingCount + e.ReplicationStats.Targets[arn] = tgtStat } - 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 += summary.pendingCount - e.ReplicationStats.FailedCount += summary.failedCount } } @@ -190,18 +224,22 @@ func (e *dataUsageEntry) merge(other dataUsageEntry) { e.Versions += other.Versions e.Size += other.Size ors := other.ReplicationStats - empty := replicationStats{} - if ors != nil && *ors != empty { + if ors != nil && len(ors.Targets) > 0 { if e.ReplicationStats == nil { - e.ReplicationStats = &replicationStats{} + e.ReplicationStats = &replicationAllStats{Targets: make(map[string]replicationStats)} + } + if other.ReplicationStats != nil { + for arn, stat := range other.ReplicationStats.Targets { + st := e.ReplicationStats.Targets[arn] + e.ReplicationStats.Targets[arn] = replicationStats{ + PendingSize: stat.PendingSize + st.PendingSize, + FailedSize: stat.FailedSize + st.FailedSize, + ReplicatedSize: stat.ReplicatedSize + st.ReplicatedSize, + PendingCount: stat.PendingCount + st.PendingCount, + FailedCount: stat.FailedCount + st.FailedCount, + } + } } - 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[:] { @@ -361,28 +399,20 @@ func (d *dataUsageCache) keepRootChildren(list map[dataUsageHash]struct{}) { // 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) madmin.DataUsageInfo { +func (d *dataUsageCache) dui(path string, buckets []BucketInfo) DataUsageInfo { e := d.find(path) if e == nil { // No entry found, return empty. - return madmin.DataUsageInfo{} + return DataUsageInfo{} } flat := d.flatten(*e) - dui := madmin.DataUsageInfo{ + dui := DataUsageInfo{ LastUpdate: d.Info.LastUpdate, ObjectsTotalCount: flat.Objects, ObjectsTotalSize: uint64(flat.Size), BucketsCount: uint64(len(e.Children)), BucketsUsage: d.bucketsUsageInfo(buckets), } - if flat.ReplicationStats != nil { - dui.ReplicationPendingSize = flat.ReplicationStats.PendingSize - dui.ReplicatedSize = flat.ReplicationStats.ReplicatedSize - dui.ReplicationFailedSize = flat.ReplicationStats.FailedSize - dui.ReplicationPendingCount = flat.ReplicationStats.PendingCount - dui.ReplicationFailedCount = flat.ReplicationStats.FailedCount - dui.ReplicaSize = flat.ReplicationStats.ReplicaSize - } return dui } @@ -600,26 +630,32 @@ 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]madmin.BucketUsageInfo { - var dst = make(map[string]madmin.BucketUsageInfo, len(buckets)) +func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]BucketUsageInfo { + var dst = make(map[string]BucketUsageInfo, len(buckets)) for _, bucket := range buckets { e := d.find(bucket.Name) if e == nil { continue } flat := d.flatten(*e) - bui := madmin.BucketUsageInfo{ + bui := BucketUsageInfo{ Size: uint64(flat.Size), ObjectsCount: flat.Objects, ObjectSizesHistogram: flat.ObjSizes.toMap(), } if flat.ReplicationStats != nil { - bui.ReplicationPendingSize = flat.ReplicationStats.PendingSize - bui.ReplicatedSize = flat.ReplicationStats.ReplicatedSize - bui.ReplicationFailedSize = flat.ReplicationStats.FailedSize - bui.ReplicationPendingCount = flat.ReplicationStats.PendingCount - bui.ReplicationFailedCount = flat.ReplicationStats.FailedCount bui.ReplicaSize = flat.ReplicationStats.ReplicaSize + bui.ReplicationInfo = make(map[string]BucketTargetUsageInfo, len(flat.ReplicationStats.Targets)) + for arn, stat := range flat.ReplicationStats.Targets { + bui.ReplicationInfo[arn] = BucketTargetUsageInfo{ + ReplicationPendingSize: stat.PendingSize, + ReplicatedSize: stat.ReplicatedSize, + ReplicationFailedSize: stat.FailedSize, + ReplicationPendingCount: stat.PendingCount, + ReplicationFailedCount: stat.FailedCount, + } + + } } dst[bucket.Name] = bui } @@ -628,24 +664,29 @@ func (d *dataUsageCache) bucketsUsageInfo(buckets []BucketInfo) map[string]madmi // bucketUsageInfo returns the buckets usage info. // If not found all values returned are zero values. -func (d *dataUsageCache) bucketUsageInfo(bucket string) madmin.BucketUsageInfo { +func (d *dataUsageCache) bucketUsageInfo(bucket string) BucketUsageInfo { e := d.find(bucket) if e == nil { - return madmin.BucketUsageInfo{} + return BucketUsageInfo{} } flat := d.flatten(*e) - bui := madmin.BucketUsageInfo{ + bui := BucketUsageInfo{ Size: uint64(flat.Size), ObjectsCount: flat.Objects, ObjectSizesHistogram: flat.ObjSizes.toMap(), } if flat.ReplicationStats != nil { - bui.ReplicationPendingSize = flat.ReplicationStats.PendingSize - bui.ReplicatedSize = flat.ReplicationStats.ReplicatedSize - bui.ReplicationFailedSize = flat.ReplicationStats.FailedSize - bui.ReplicationPendingCount = flat.ReplicationStats.PendingCount - bui.ReplicationFailedCount = flat.ReplicationStats.FailedCount bui.ReplicaSize = flat.ReplicationStats.ReplicaSize + bui.ReplicationInfo = make(map[string]BucketTargetUsageInfo, len(flat.ReplicationStats.Targets)) + for arn, stat := range flat.ReplicationStats.Targets { + bui.ReplicationInfo[arn] = BucketTargetUsageInfo{ + ReplicationPendingSize: stat.PendingSize, + ReplicatedSize: stat.ReplicatedSize, + ReplicationFailedSize: stat.FailedSize, + ReplicationPendingCount: stat.PendingCount, + ReplicationFailedCount: stat.FailedCount, + } + } } return bui } @@ -790,7 +831,8 @@ 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 ( - dataUsageCacheVerCurrent = 5 + dataUsageCacheVerCurrent = 6 + dataUsageCacheVerV5 = 5 dataUsageCacheVerV4 = 4 dataUsageCacheVerV3 = 3 dataUsageCacheVerV2 = 2 @@ -885,9 +927,17 @@ func (d *dataUsageCache) deserialize(r io.Reader) error { Children: v.Children, } if v.ReplicatedSize > 0 || v.ReplicaSize > 0 || v.ReplicationFailedSize > 0 || v.ReplicationPendingSize > 0 { - due.ReplicationStats = &replicationStats{ + due.ReplicationStats = &replicationAllStats{ + Targets: make(map[string]replicationStats), + } + cfg, err := getReplicationConfig(GlobalContext, d.Info.Name) + if err != nil { + return err + } + due.ReplicationStats.ReplicaSize = v.ReplicaSize + + due.ReplicationStats.Targets[cfg.RoleArn] = replicationStats{ ReplicatedSize: v.ReplicatedSize, - ReplicaSize: v.ReplicaSize, FailedSize: v.ReplicationFailedSize, PendingSize: v.ReplicationPendingSize, } @@ -919,8 +969,16 @@ func (d *dataUsageCache) deserialize(r io.Reader) error { Children: v.Children, } empty := replicationStats{} + if v.ReplicationStats != empty { - due.ReplicationStats = &v.ReplicationStats + due.ReplicationStats = &replicationAllStats{ + Targets: make(map[string]replicationStats), + } + cfg, err := getReplicationConfig(GlobalContext, d.Info.Name) + if err != nil { + return err + } + due.ReplicationStats.Targets[cfg.RoleArn] = v.ReplicationStats } due.Compacted = len(due.Children) == 0 && k != d.Info.Name @@ -928,12 +986,63 @@ func (d *dataUsageCache) deserialize(r io.Reader) error { } // Populate compacted value and remove unneeded replica stats. - empty := replicationStats{} for k, e := range d.Cache { - if e.ReplicationStats != nil && *e.ReplicationStats == empty { + if e.ReplicationStats != nil && len(e.ReplicationStats.Targets) == 0 { e.ReplicationStats = nil } + d.Cache[k] = e + } + return nil + case dataUsageCacheVerV5: + // Zstd compressed. + dec, err := zstd.NewReader(r, zstd.WithDecoderConcurrency(2)) + if err != nil { + return err + } + defer dec.Close() + dold := &dataUsageCacheV5{} + 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)) + var arn string + for k, v := range dold.Cache { + due := dataUsageEntry{ + Size: v.Size, + Objects: v.Objects, + ObjSizes: v.ObjSizes, + Children: v.Children, + } + if v.ReplicationStats != nil && !v.ReplicationStats.Empty() { + if arn == "" { + cfg, err := getReplicationConfig(GlobalContext, d.Info.Name) + if err != nil { + return err + } + d.Info.replication = replicationConfig{Config: cfg} + arn = d.Info.replication.Config.RoleArn + } + due.ReplicationStats = &replicationAllStats{ + Targets: make(map[string]replicationStats), + } + + if arn != "" { + due.ReplicationStats.Targets[arn] = *v.ReplicationStats + } + } + due.Compacted = len(due.Children) == 0 && k != d.Info.Name + + d.Cache[k] = due + } + + // Populate compacted value and remove unneeded replica stats. + for k, e := range d.Cache { + if e.ReplicationStats != nil && len(e.ReplicationStats.Targets) == 0 { + e.ReplicationStats = nil + } d.Cache[k] = e } return nil diff --git a/cmd/data-usage-cache_gen.go b/cmd/data-usage-cache_gen.go index 41b46a98a..9ddd836c1 100644 --- a/cmd/data-usage-cache_gen.go +++ b/cmd/data-usage-cache_gen.go @@ -1064,6 +1064,191 @@ func (z *dataUsageCacheV4) Msgsize() (s int) { return } +// DecodeMsg implements msgp.Decodable +func (z *dataUsageCacheV5) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Info": + err = z.Info.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Info") + return + } + case "Disks": + var zb0002 uint32 + zb0002, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err, "Disks") + return + } + if cap(z.Disks) >= int(zb0002) { + z.Disks = (z.Disks)[:zb0002] + } else { + z.Disks = make([]string, zb0002) + } + for za0001 := range z.Disks { + z.Disks[za0001], err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Disks", za0001) + return + } + } + case "Cache": + var zb0003 uint32 + zb0003, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Cache") + return + } + if z.Cache == nil { + z.Cache = make(map[string]dataUsageEntryV5, zb0003) + } else if len(z.Cache) > 0 { + for key := range z.Cache { + delete(z.Cache, key) + } + } + for zb0003 > 0 { + zb0003-- + var za0002 string + var za0003 dataUsageEntryV5 + za0002, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Cache") + return + } + err = za0003.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Cache", za0002) + return + } + z.Cache[za0002] = za0003 + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *dataUsageCacheV5) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Info": + bts, err = z.Info.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Info") + return + } + case "Disks": + var zb0002 uint32 + zb0002, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Disks") + return + } + if cap(z.Disks) >= int(zb0002) { + z.Disks = (z.Disks)[:zb0002] + } else { + z.Disks = make([]string, zb0002) + } + for za0001 := range z.Disks { + z.Disks[za0001], bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Disks", za0001) + return + } + } + case "Cache": + var zb0003 uint32 + zb0003, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Cache") + return + } + if z.Cache == nil { + z.Cache = make(map[string]dataUsageEntryV5, zb0003) + } else if len(z.Cache) > 0 { + for key := range z.Cache { + delete(z.Cache, key) + } + } + for zb0003 > 0 { + var za0002 string + var za0003 dataUsageEntryV5 + zb0003-- + za0002, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Cache") + return + } + bts, err = za0003.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Cache", za0002) + return + } + z.Cache[za0002] = za0003 + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *dataUsageCacheV5) Msgsize() (s int) { + s = 1 + 5 + z.Info.Msgsize() + 6 + msgp.ArrayHeaderSize + for za0001 := range z.Disks { + s += msgp.StringPrefixSize + len(z.Disks[za0001]) + } + s += 6 + msgp.MapHeaderSize + if z.Cache != nil { + for za0002, za0003 := range z.Cache { + _ = za0003 + s += msgp.StringPrefixSize + len(za0002) + za0003.Msgsize() + } + } + return +} + // DecodeMsg implements msgp.Decodable func (z *dataUsageEntry) DecodeMsg(dc *msgp.Reader) (err error) { var zb0001 uint32 @@ -1122,7 +1307,7 @@ func (z *dataUsageEntry) DecodeMsg(dc *msgp.Reader) (err error) { z.ReplicationStats = nil } else { if z.ReplicationStats == nil { - z.ReplicationStats = new(replicationStats) + z.ReplicationStats = new(replicationAllStats) } err = z.ReplicationStats.DecodeMsg(dc) if err != nil { @@ -1284,7 +1469,7 @@ func (z *dataUsageEntry) UnmarshalMsg(bts []byte) (o []byte, err error) { z.ReplicationStats = nil } else { if z.ReplicationStats == nil { - z.ReplicationStats = new(replicationStats) + z.ReplicationStats = new(replicationAllStats) } bts, err = z.ReplicationStats.UnmarshalMsg(bts) if err != nil { @@ -1666,6 +1851,255 @@ func (z *dataUsageEntryV4) Msgsize() (s int) { return } +// DecodeMsg implements msgp.Decodable +func (z *dataUsageEntryV5) DecodeMsg(dc *msgp.Reader) (err error) { + var zb0001 uint32 + zb0001, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + if zb0001 != 7 { + err = msgp.ArrayError{Wanted: 7, Got: zb0001} + return + } + err = z.Children.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Children") + return + } + z.Size, err = dc.ReadInt64() + if err != nil { + err = msgp.WrapError(err, "Size") + return + } + z.Objects, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "Objects") + return + } + z.Versions, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "Versions") + return + } + var zb0002 uint32 + zb0002, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err, "ObjSizes") + return + } + if zb0002 != uint32(dataUsageBucketLen) { + err = msgp.ArrayError{Wanted: uint32(dataUsageBucketLen), Got: zb0002} + return + } + for za0001 := range z.ObjSizes { + z.ObjSizes[za0001], err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "ObjSizes", za0001) + return + } + } + if dc.IsNil() { + err = dc.ReadNil() + if err != nil { + err = msgp.WrapError(err, "ReplicationStats") + return + } + z.ReplicationStats = nil + } else { + if z.ReplicationStats == nil { + z.ReplicationStats = new(replicationStats) + } + err = z.ReplicationStats.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "ReplicationStats") + return + } + } + z.Compacted, err = dc.ReadBool() + if err != nil { + err = msgp.WrapError(err, "Compacted") + return + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *dataUsageEntryV5) EncodeMsg(en *msgp.Writer) (err error) { + // array header, size 7 + err = en.Append(0x97) + if err != nil { + return + } + err = z.Children.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Children") + return + } + err = en.WriteInt64(z.Size) + if err != nil { + err = msgp.WrapError(err, "Size") + return + } + err = en.WriteUint64(z.Objects) + if err != nil { + err = msgp.WrapError(err, "Objects") + return + } + err = en.WriteUint64(z.Versions) + if err != nil { + err = msgp.WrapError(err, "Versions") + return + } + err = en.WriteArrayHeader(uint32(dataUsageBucketLen)) + if err != nil { + err = msgp.WrapError(err, "ObjSizes") + return + } + for za0001 := range z.ObjSizes { + err = en.WriteUint64(z.ObjSizes[za0001]) + if err != nil { + err = msgp.WrapError(err, "ObjSizes", za0001) + return + } + } + if z.ReplicationStats == nil { + err = en.WriteNil() + if err != nil { + return + } + } else { + err = z.ReplicationStats.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "ReplicationStats") + return + } + } + err = en.WriteBool(z.Compacted) + if err != nil { + err = msgp.WrapError(err, "Compacted") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *dataUsageEntryV5) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // array header, size 7 + o = append(o, 0x97) + o, err = z.Children.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Children") + return + } + o = msgp.AppendInt64(o, z.Size) + o = msgp.AppendUint64(o, z.Objects) + o = msgp.AppendUint64(o, z.Versions) + o = msgp.AppendArrayHeader(o, uint32(dataUsageBucketLen)) + for za0001 := range z.ObjSizes { + o = msgp.AppendUint64(o, z.ObjSizes[za0001]) + } + if z.ReplicationStats == nil { + o = msgp.AppendNil(o) + } else { + o, err = z.ReplicationStats.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "ReplicationStats") + return + } + } + o = msgp.AppendBool(o, z.Compacted) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *dataUsageEntryV5) UnmarshalMsg(bts []byte) (o []byte, err error) { + var zb0001 uint32 + zb0001, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + if zb0001 != 7 { + err = msgp.ArrayError{Wanted: 7, Got: zb0001} + return + } + bts, err = z.Children.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Children") + return + } + z.Size, bts, err = msgp.ReadInt64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Size") + return + } + z.Objects, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Objects") + return + } + z.Versions, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Versions") + return + } + var zb0002 uint32 + zb0002, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "ObjSizes") + return + } + if zb0002 != uint32(dataUsageBucketLen) { + err = msgp.ArrayError{Wanted: uint32(dataUsageBucketLen), Got: zb0002} + return + } + for za0001 := range z.ObjSizes { + z.ObjSizes[za0001], bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ObjSizes", za0001) + return + } + } + if msgp.IsNil(bts) { + bts, err = msgp.ReadNilBytes(bts) + if err != nil { + return + } + z.ReplicationStats = nil + } else { + if z.ReplicationStats == nil { + z.ReplicationStats = new(replicationStats) + } + bts, err = z.ReplicationStats.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicationStats") + return + } + } + z.Compacted, bts, err = msgp.ReadBoolBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Compacted") + return + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *dataUsageEntryV5) Msgsize() (s int) { + s = 1 + z.Children.Msgsize() + msgp.Int64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.ArrayHeaderSize + (dataUsageBucketLen * (msgp.Uint64Size)) + if z.ReplicationStats == nil { + s += msgp.NilSize + } else { + s += z.ReplicationStats.Msgsize() + } + s += msgp.BoolSize + return +} + // DecodeMsg implements msgp.Decodable func (z *dataUsageHash) DecodeMsg(dc *msgp.Reader) (err error) { { @@ -1718,6 +2152,168 @@ func (z dataUsageHash) Msgsize() (s int) { return } +// DecodeMsg implements msgp.Decodable +func (z *replicationAllStats) DecodeMsg(dc *msgp.Reader) (err error) { + var zb0001 uint32 + zb0001, err = dc.ReadArrayHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + if zb0001 != 2 { + err = msgp.ArrayError{Wanted: 2, Got: zb0001} + return + } + var zb0002 uint32 + zb0002, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + if z.Targets == nil { + z.Targets = make(map[string]replicationStats, zb0002) + } else if len(z.Targets) > 0 { + for key := range z.Targets { + delete(z.Targets, key) + } + } + for zb0002 > 0 { + zb0002-- + var za0001 string + var za0002 replicationStats + za0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + err = za0002.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + z.Targets[za0001] = za0002 + } + z.ReplicaSize, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *replicationAllStats) EncodeMsg(en *msgp.Writer) (err error) { + // array header, size 2 + err = en.Append(0x92) + if err != nil { + return + } + err = en.WriteMapHeader(uint32(len(z.Targets))) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + for za0001, za0002 := range z.Targets { + err = en.WriteString(za0001) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + err = za0002.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + } + err = en.WriteUint64(z.ReplicaSize) + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *replicationAllStats) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // array header, size 2 + o = append(o, 0x92) + o = msgp.AppendMapHeader(o, uint32(len(z.Targets))) + for za0001, za0002 := range z.Targets { + o = msgp.AppendString(o, za0001) + o, err = za0002.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + } + o = msgp.AppendUint64(o, z.ReplicaSize) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *replicationAllStats) UnmarshalMsg(bts []byte) (o []byte, err error) { + var zb0001 uint32 + zb0001, bts, err = msgp.ReadArrayHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + if zb0001 != 2 { + err = msgp.ArrayError{Wanted: 2, Got: zb0001} + return + } + var zb0002 uint32 + zb0002, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + if z.Targets == nil { + z.Targets = make(map[string]replicationStats, zb0002) + } else if len(z.Targets) > 0 { + for key := range z.Targets { + delete(z.Targets, key) + } + } + for zb0002 > 0 { + var za0001 string + var za0002 replicationStats + zb0002-- + za0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Targets") + return + } + bts, err = za0002.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "Targets", za0001) + return + } + z.Targets[za0001] = za0002 + } + z.ReplicaSize, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "ReplicaSize") + return + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *replicationAllStats) Msgsize() (s int) { + s = 1 + msgp.MapHeaderSize + if z.Targets != nil { + for za0001, za0002 := range z.Targets { + _ = za0002 + s += msgp.StringPrefixSize + len(za0001) + za0002.Msgsize() + } + } + s += msgp.Uint64Size + return +} + // DecodeMsg implements msgp.Decodable func (z *replicationStats) DecodeMsg(dc *msgp.Reader) (err error) { var zb0001 uint32 @@ -1726,8 +2322,8 @@ func (z *replicationStats) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err) return } - if zb0001 != 10 { - err = msgp.ArrayError{Wanted: 10, Got: zb0001} + if zb0001 != 9 { + err = msgp.ArrayError{Wanted: 9, Got: zb0001} return } z.PendingSize, err = dc.ReadUint64() @@ -1745,11 +2341,6 @@ func (z *replicationStats) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "FailedSize") return } - z.ReplicaSize, err = dc.ReadUint64() - if err != nil { - err = msgp.WrapError(err, "ReplicaSize") - return - } z.FailedCount, err = dc.ReadUint64() if err != nil { err = msgp.WrapError(err, "FailedCount") @@ -1785,8 +2376,8 @@ func (z *replicationStats) DecodeMsg(dc *msgp.Reader) (err error) { // EncodeMsg implements msgp.Encodable func (z *replicationStats) EncodeMsg(en *msgp.Writer) (err error) { - // array header, size 10 - err = en.Append(0x9a) + // array header, size 9 + err = en.Append(0x99) if err != nil { return } @@ -1805,11 +2396,6 @@ func (z *replicationStats) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "FailedSize") return } - err = en.WriteUint64(z.ReplicaSize) - if err != nil { - err = msgp.WrapError(err, "ReplicaSize") - return - } err = en.WriteUint64(z.FailedCount) if err != nil { err = msgp.WrapError(err, "FailedCount") @@ -1846,12 +2432,11 @@ func (z *replicationStats) EncodeMsg(en *msgp.Writer) (err error) { // MarshalMsg implements msgp.Marshaler func (z *replicationStats) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // array header, size 10 - o = append(o, 0x9a) + // array header, size 9 + o = append(o, 0x99) o = msgp.AppendUint64(o, z.PendingSize) o = msgp.AppendUint64(o, z.ReplicatedSize) o = msgp.AppendUint64(o, z.FailedSize) - o = msgp.AppendUint64(o, z.ReplicaSize) o = msgp.AppendUint64(o, z.FailedCount) o = msgp.AppendUint64(o, z.PendingCount) o = msgp.AppendUint64(o, z.MissedThresholdSize) @@ -1869,8 +2454,8 @@ func (z *replicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err) return } - if zb0001 != 10 { - err = msgp.ArrayError{Wanted: 10, Got: zb0001} + if zb0001 != 9 { + err = msgp.ArrayError{Wanted: 9, Got: zb0001} return } z.PendingSize, bts, err = msgp.ReadUint64Bytes(bts) @@ -1888,11 +2473,6 @@ func (z *replicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err, "FailedSize") return } - z.ReplicaSize, bts, err = msgp.ReadUint64Bytes(bts) - if err != nil { - err = msgp.WrapError(err, "ReplicaSize") - return - } z.FailedCount, bts, err = msgp.ReadUint64Bytes(bts) if err != nil { err = msgp.WrapError(err, "FailedCount") @@ -1929,7 +2509,7 @@ func (z *replicationStats) UnmarshalMsg(bts []byte) (o []byte, err error) { // Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message func (z *replicationStats) Msgsize() (s int) { - s = 1 + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + s = 1 + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size + msgp.Uint64Size return } diff --git a/cmd/data-usage-cache_gen_test.go b/cmd/data-usage-cache_gen_test.go index b765880d5..e3b65a01e 100644 --- a/cmd/data-usage-cache_gen_test.go +++ b/cmd/data-usage-cache_gen_test.go @@ -348,6 +348,232 @@ func BenchmarkDecodedataUsageEntry(b *testing.B) { } } +func TestMarshalUnmarshaldataUsageEntryV5(t *testing.T) { + v := dataUsageEntryV5{} + 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 BenchmarkMarshalMsgdataUsageEntryV5(b *testing.B) { + v := dataUsageEntryV5{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgdataUsageEntryV5(b *testing.B) { + v := dataUsageEntryV5{} + 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 BenchmarkUnmarshaldataUsageEntryV5(b *testing.B) { + v := dataUsageEntryV5{} + 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 TestEncodeDecodedataUsageEntryV5(t *testing.T) { + v := dataUsageEntryV5{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodedataUsageEntryV5 Msgsize() is inaccurate") + } + + vn := dataUsageEntryV5{} + 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 BenchmarkEncodedataUsageEntryV5(b *testing.B) { + v := dataUsageEntryV5{} + 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 BenchmarkDecodedataUsageEntryV5(b *testing.B) { + v := dataUsageEntryV5{} + 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 TestMarshalUnmarshalreplicationAllStats(t *testing.T) { + v := replicationAllStats{} + 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 BenchmarkMarshalMsgreplicationAllStats(b *testing.B) { + v := replicationAllStats{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgreplicationAllStats(b *testing.B) { + v := replicationAllStats{} + 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 BenchmarkUnmarshalreplicationAllStats(b *testing.B) { + v := replicationAllStats{} + 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 TestEncodeDecodereplicationAllStats(t *testing.T) { + v := replicationAllStats{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodereplicationAllStats Msgsize() is inaccurate") + } + + vn := replicationAllStats{} + 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 BenchmarkEncodereplicationAllStats(b *testing.B) { + v := replicationAllStats{} + 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 BenchmarkDecodereplicationAllStats(b *testing.B) { + v := replicationAllStats{} + 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) diff --git a/cmd/data-usage-utils.go b/cmd/data-usage-utils.go new file mode 100644 index 000000000..e093ae49f --- /dev/null +++ b/cmd/data-usage-utils.go @@ -0,0 +1,87 @@ +// Copyright (c) 2015-2021 MinIO, Inc. +// +// This file is part of MinIO Object Storage stack +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// This program is distributed in the hope that it will be useful +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +package cmd + +import ( + "time" +) + +// BucketTargetUsageInfo - bucket target usage info provides +// - replicated size for all objects sent to this target +// - replica size for all objects received from this target +// - replication pending size for all objects pending replication to this target +// - replication failed size for all objects failed replication to this target +// - replica pending count +// - replica failed count +type BucketTargetUsageInfo struct { + ReplicationPendingSize uint64 `json:"objectsPendingReplicationTotalSize"` + ReplicationFailedSize uint64 `json:"objectsFailedReplicationTotalSize"` + ReplicatedSize uint64 `json:"objectsReplicatedTotalSize"` + ReplicaSize uint64 `json:"objectReplicaTotalSize"` + ReplicationPendingCount uint64 `json:"objectsPendingReplicationCount"` + ReplicationFailedCount uint64 `json:"objectsFailedReplicationCount"` +} + +// 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"` + // Following five fields suffixed with V1 are here for backward compatibility + // Total Size for objects that have not yet been replicated + ReplicationPendingSizeV1 uint64 `json:"objectsPendingReplicationTotalSize"` + // Total size for objects that have witness one or more failures and will be retried + ReplicationFailedSizeV1 uint64 `json:"objectsFailedReplicationTotalSize"` + // Total size for objects that have been replicated to destination + ReplicatedSizeV1 uint64 `json:"objectsReplicatedTotalSize"` + // Total number of objects pending replication + ReplicationPendingCountV1 uint64 `json:"objectsPendingReplicationCount"` + // Total number of objects that failed replication + ReplicationFailedCountV1 uint64 `json:"objectsFailedReplicationCount"` + + ObjectsCount uint64 `json:"objectsCount"` + ObjectSizesHistogram map[string]uint64 `json:"objectsSizesHistogram"` + ReplicaSize uint64 `json:"objectReplicaTotalSize"` + ReplicationInfo map[string]BucketTargetUsageInfo `json:"objectsReplicationInfo"` +} + +// 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"` + ReplicationInfo map[string]BucketTargetUsageInfo `json:"objectsReplicationInfo"` + // 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"` +} diff --git a/cmd/data-usage.go b/cmd/data-usage.go index 735879714..c804d2841 100644 --- a/cmd/data-usage.go +++ b/cmd/data-usage.go @@ -24,7 +24,6 @@ import ( "strings" jsoniter "github.com/json-iterator/go" - "github.com/minio/madmin-go" "github.com/minio/minio/internal/hash" "github.com/minio/minio/internal/logger" ) @@ -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 madmin.DataUsageInfo) { +func storeDataUsageInBackend(ctx context.Context, objAPI ObjectLayer, dui <-chan DataUsageInfo) { for dataUsageInfo := range dui { var json = jsoniter.ConfigCompatibleWithStandardLibrary dataUsageJSON, err := json.Marshal(dataUsageInfo) @@ -95,27 +94,26 @@ func loadPrefixUsageFromBackend(ctx context.Context, objAPI ObjectLayer, bucket return m, nil } -func loadDataUsageFromBackend(ctx context.Context, objAPI ObjectLayer) (madmin.DataUsageInfo, error) { +func loadDataUsageFromBackend(ctx context.Context, objAPI ObjectLayer) (DataUsageInfo, error) { r, err := objAPI.GetObjectNInfo(ctx, dataUsageBucket, dataUsageObjName, nil, http.Header{}, readLock, ObjectOptions{}) if err != nil { if isErrObjectNotFound(err) || isErrBucketNotFound(err) { - return madmin.DataUsageInfo{}, nil + return DataUsageInfo{}, nil } - return madmin.DataUsageInfo{}, toObjectErr(err, dataUsageBucket, dataUsageObjName) + return DataUsageInfo{}, toObjectErr(err, dataUsageBucket, dataUsageObjName) } defer r.Close() - var dataUsageInfo madmin.DataUsageInfo + var dataUsageInfo DataUsageInfo var json = jsoniter.ConfigCompatibleWithStandardLibrary if err = json.NewDecoder(r).Decode(&dataUsageInfo); err != nil { - return madmin.DataUsageInfo{}, err + return DataUsageInfo{}, err } - // For forward compatibility reasons, we need to add this code. if len(dataUsageInfo.BucketsUsage) == 0 { - dataUsageInfo.BucketsUsage = make(map[string]madmin.BucketUsageInfo, len(dataUsageInfo.BucketSizes)) + dataUsageInfo.BucketsUsage = make(map[string]BucketUsageInfo, len(dataUsageInfo.BucketSizes)) for bucket, size := range dataUsageInfo.BucketSizes { - dataUsageInfo.BucketsUsage[bucket] = madmin.BucketUsageInfo{Size: size} + dataUsageInfo.BucketsUsage[bucket] = BucketUsageInfo{Size: size} } } @@ -126,6 +124,23 @@ func loadDataUsageFromBackend(ctx context.Context, objAPI ObjectLayer) (madmin.D dataUsageInfo.BucketSizes[bucket] = bui.Size } } - + // For forward compatibility reasons, we need to add this code. + for bucket, bui := range dataUsageInfo.BucketsUsage { + if bui.ReplicatedSizeV1 > 0 || bui.ReplicationFailedCountV1 > 0 || + bui.ReplicationFailedSizeV1 > 0 || bui.ReplicationPendingCountV1 > 0 { + dataUsageInfo.ReplicationInfo = make(map[string]BucketTargetUsageInfo) + cfg, err := getReplicationConfig(GlobalContext, bucket) + if err != nil { + return DataUsageInfo{}, err + } + dataUsageInfo.ReplicationInfo[cfg.RoleArn] = BucketTargetUsageInfo{ + ReplicationFailedSize: bui.ReplicationFailedSizeV1, + ReplicationFailedCount: bui.ReplicationFailedCountV1, + ReplicatedSize: bui.ReplicatedSizeV1, + ReplicationPendingCount: bui.ReplicationPendingCountV1, + ReplicationPendingSize: bui.ReplicationPendingSizeV1, + } + } + } return dataUsageInfo, nil } diff --git a/cmd/data-usage_test.go b/cmd/data-usage_test.go index 32fb487d3..843b9c6e6 100644 --- a/cmd/data-usage_test.go +++ b/cmd/data-usage_test.go @@ -580,17 +580,16 @@ func TestDataUsageCacheSerialize(t *testing.T) { t.Fatal(err) } e := want.find("abucket/dir2") - e.ReplicationStats = &replicationStats{ - PendingSize: 1, - ReplicatedSize: 2, - FailedSize: 3, - ReplicaSize: 4, - FailedCount: 5, - PendingCount: 6, - MissedThresholdSize: 7, - AfterThresholdSize: 8, - MissedThresholdCount: 9, - AfterThresholdCount: 10, + e.ReplicationStats = &replicationAllStats{ + Targets: map[string]replicationStats{ + "arn": { + PendingSize: 1, + ReplicatedSize: 2, + FailedSize: 3, + FailedCount: 5, + PendingCount: 6, + }, + }, } want.replace("abucket/dir2", "", *e) var buf bytes.Buffer diff --git a/cmd/erasure-metadata.go b/cmd/erasure-metadata.go index 964530cfd..70b5ae018 100644 --- a/cmd/erasure-metadata.go +++ b/cmd/erasure-metadata.go @@ -24,6 +24,7 @@ import ( "fmt" "net/http" "sort" + "strings" "time" "github.com/minio/minio/internal/bucket/replication" @@ -148,11 +149,11 @@ func (fi FileInfo) ToObjectInfo(bucket, object string) ObjectInfo { } // Add replication status to the object info - objInfo.ReplicationStatus = replication.StatusType(fi.Metadata[xhttp.AmzBucketReplicationStatus]) - if fi.Deleted { - objInfo.ReplicationStatus = replication.StatusType(fi.DeleteMarkerReplicationStatus) - } + objInfo.ReplicationStatusInternal = fi.ReplicationState.ReplicationStatusInternal + objInfo.VersionPurgeStatusInternal = fi.ReplicationState.VersionPurgeStatusInternal + objInfo.ReplicationStatus = fi.ReplicationState.CompositeReplicationStatus() + objInfo.VersionPurgeStatus = fi.ReplicationState.CompositeVersionPurgeStatus() objInfo.TransitionedObject = TransitionedObject{ Name: fi.TransitionedObjName, VersionID: fi.TransitionVersionID, @@ -177,7 +178,7 @@ func (fi FileInfo) ToObjectInfo(bucket, object string) ObjectInfo { objInfo.StorageClass = globalMinioDefaultStorageClass } - objInfo.VersionPurgeStatus = fi.VersionPurgeStatus + objInfo.VersionPurgeStatus = fi.VersionPurgeStatus() // set restore status for transitioned object restoreHdr, ok := fi.Metadata[xhttp.AmzRestore] if ok { @@ -219,9 +220,7 @@ func (fi FileInfo) MetadataEquals(ofi FileInfo) bool { func (fi FileInfo) ReplicationInfoEquals(ofi FileInfo) bool { switch { case fi.MarkDeleted != ofi.MarkDeleted, - fi.DeleteMarkerReplicationStatus != ofi.DeleteMarkerReplicationStatus, - fi.VersionPurgeStatus != ofi.VersionPurgeStatus, - fi.Metadata[xhttp.AmzBucketReplicationStatus] != ofi.Metadata[xhttp.AmzBucketReplicationStatus]: + !fi.ReplicationState.Equal(ofi.ReplicationState): return false } return true @@ -307,9 +306,12 @@ func findFileInfoInQuorum(ctx context.Context, metaArr []FileInfo, modTime time. // Server-side replication fields h.Write([]byte(fmt.Sprintf("%v", meta.MarkDeleted))) - h.Write([]byte(meta.DeleteMarkerReplicationStatus)) - h.Write([]byte(meta.VersionPurgeStatus)) - h.Write([]byte(meta.Metadata[xhttp.AmzBucketReplicationStatus])) + h.Write([]byte(meta.Metadata[string(meta.ReplicationState.ReplicaStatus)])) + h.Write([]byte(meta.Metadata[meta.ReplicationState.ReplicationTimeStamp.Format(http.TimeFormat)])) + h.Write([]byte(meta.Metadata[meta.ReplicationState.ReplicaTimeStamp.Format(http.TimeFormat)])) + h.Write([]byte(meta.Metadata[meta.ReplicationState.ReplicationStatusInternal])) + h.Write([]byte(meta.Metadata[meta.ReplicationState.VersionPurgeStatusInternal])) + metaHashes[i] = hex.EncodeToString(h.Sum(nil)) h.Reset() } @@ -449,3 +451,54 @@ func (fi *FileInfo) TierFreeVersion() bool { _, ok := fi.Metadata[ReservedMetadataPrefixLower+tierFVMarker] return ok } + +// VersionPurgeStatus returns overall version purge status for this object version across targets +func (fi *FileInfo) VersionPurgeStatus() VersionPurgeStatusType { + return fi.ReplicationState.CompositeVersionPurgeStatus() +} + +// DeleteMarkerReplicationStatus returns overall replication status for this delete marker version across targets +func (fi *FileInfo) DeleteMarkerReplicationStatus() replication.StatusType { + if fi.Deleted { + return fi.ReplicationState.CompositeReplicationStatus() + } + return replication.StatusType("") +} + +// GetInternalReplicationState is a wrapper method to fetch internal replication state from the map m +func GetInternalReplicationState(m map[string][]byte) ReplicationState { + m1 := make(map[string]string, len(m)) + for k, v := range m { + m1[k] = string(v) + } + return getInternalReplicationState(m1) +} + +// getInternalReplicationState fetches internal replication state from the map m +func getInternalReplicationState(m map[string]string) ReplicationState { + d := ReplicationState{ + ResetStatusesMap: make(map[string]string), + } + for k, v := range m { + switch { + case equals(k, ReservedMetadataPrefixLower+ReplicationTimestamp): + tm, _ := time.Parse(http.TimeFormat, v) + d.ReplicationTimeStamp = tm + case equals(k, ReservedMetadataPrefixLower+ReplicaTimestamp): + tm, _ := time.Parse(http.TimeFormat, v) + d.ReplicaTimeStamp = tm + case equals(k, ReservedMetadataPrefixLower+ReplicaStatus): + d.ReplicaStatus = replication.StatusType(v) + case equals(k, ReservedMetadataPrefixLower+ReplicationStatus): + d.ReplicationStatusInternal = v + d.Targets = replicationStatusesMap(v) + case equals(k, VersionPurgeStatusKey): + d.VersionPurgeStatusInternal = v + d.PurgeTargets = versionPurgeStatusesMap(v) + case strings.HasPrefix(k, ReservedMetadataPrefixLower+ReplicationReset): + arn := strings.TrimPrefix(k, fmt.Sprintf("%s-", ReservedMetadataPrefixLower+ReplicationReset)) + d.ResetStatusesMap[arn] = v + } + } + return d +} diff --git a/cmd/erasure-object.go b/cmd/erasure-object.go index 233fef93a..cf5ba8c37 100644 --- a/cmd/erasure-object.go +++ b/cmd/erasure-object.go @@ -441,7 +441,7 @@ func (er erasureObjects) getObjectInfo(ctx context.Context, bucket, object strin } objInfo = fi.ToObjectInfo(bucket, object) - if !fi.VersionPurgeStatus.Empty() && opts.VersionID != "" { + if !fi.VersionPurgeStatus().Empty() && opts.VersionID != "" { // Make sure to return object info to provide extra information. return objInfo, toObjectErr(errMethodNotAllowed, bucket, object) } @@ -470,7 +470,7 @@ func (er erasureObjects) getObjectInfoAndQuorum(ctx context.Context, bucket, obj } objInfo = fi.ToObjectInfo(bucket, object) - if !fi.VersionPurgeStatus.Empty() && opts.VersionID != "" { + if !fi.VersionPurgeStatus().Empty() && opts.VersionID != "" { // Make sure to return object info to provide extra information. return objInfo, wquorum, toObjectErr(errMethodNotAllowed, bucket, object) } @@ -998,6 +998,7 @@ func (er erasureObjects) putObject(ctx context.Context, bucket string, object st break } + fi.ReplicationState = opts.PutReplicationState() online = countOnlineDisks(onlineDisks) return fi.ToObjectInfo(bucket, object), nil @@ -1088,11 +1089,10 @@ func (er erasureObjects) DeleteObjects(ctx context.Context, bucket string, objec } if opts.Versioned || opts.VersionSuspended { versions[i] = FileInfo{ - Name: objects[i].ObjectName, - ModTime: modTime, - Deleted: true, // delete marker - DeleteMarkerReplicationStatus: objects[i].DeleteMarkerReplicationStatus, - VersionPurgeStatus: objects[i].VersionPurgeStatus, + Name: objects[i].ObjectName, + ModTime: modTime, + Deleted: true, // delete marker + ReplicationState: objects[i].ReplicationState(), } versions[i].SetTierFreeVersionID(mustGetUUID()) if opts.Versioned { @@ -1102,10 +1102,9 @@ func (er erasureObjects) DeleteObjects(ctx context.Context, bucket string, objec } } versions[i] = FileInfo{ - Name: objects[i].ObjectName, - VersionID: objects[i].VersionID, - DeleteMarkerReplicationStatus: objects[i].DeleteMarkerReplicationStatus, - VersionPurgeStatus: objects[i].VersionPurgeStatus, + Name: objects[i].ObjectName, + VersionID: objects[i].VersionID, + ReplicationState: objects[i].ReplicationState(), } versions[i].SetTierFreeVersionID(mustGetUUID()) } @@ -1156,19 +1155,17 @@ func (er erasureObjects) DeleteObjects(ctx context.Context, bucket string, objec if versions[objIndex].Deleted { dobjects[objIndex] = DeletedObject{ - DeleteMarker: versions[objIndex].Deleted, - DeleteMarkerVersionID: versions[objIndex].VersionID, - DeleteMarkerMTime: DeleteMarkerMTime{versions[objIndex].ModTime}, - DeleteMarkerReplicationStatus: versions[objIndex].DeleteMarkerReplicationStatus, - ObjectName: versions[objIndex].Name, - VersionPurgeStatus: versions[objIndex].VersionPurgeStatus, + DeleteMarker: versions[objIndex].Deleted, + DeleteMarkerVersionID: versions[objIndex].VersionID, + DeleteMarkerMTime: DeleteMarkerMTime{versions[objIndex].ModTime}, + ObjectName: versions[objIndex].Name, + ReplicationState: versions[objIndex].ReplicationState, } } else { dobjects[objIndex] = DeletedObject{ - ObjectName: versions[objIndex].Name, - VersionID: versions[objIndex].VersionID, - VersionPurgeStatus: versions[objIndex].VersionPurgeStatus, - DeleteMarkerReplicationStatus: versions[objIndex].DeleteMarkerReplicationStatus, + ObjectName: versions[objIndex].Name, + VersionID: versions[objIndex].VersionID, + ReplicationState: versions[objIndex].ReplicationState, } } } @@ -1306,13 +1303,13 @@ func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string if opts.VersionID != "" { // case where replica version needs to be deleted on target cluster - if versionFound && opts.DeleteMarkerReplicationStatus == replication.Replica.String() { + if versionFound && opts.DeleteMarkerReplicationStatus() == replication.Replica { markDelete = false } - if opts.VersionPurgeStatus.Empty() && opts.DeleteMarkerReplicationStatus == "" { + if opts.VersionPurgeStatus().Empty() && opts.DeleteMarkerReplicationStatus().Empty() { markDelete = false } - if opts.VersionPurgeStatus == Complete { + if opts.VersionPurgeStatus() == Complete { markDelete = false } // determine if the version represents an object delete @@ -1330,14 +1327,13 @@ func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string if markDelete { if opts.Versioned || opts.VersionSuspended { fi := FileInfo{ - Name: object, - Deleted: deleteMarker, - MarkDeleted: markDelete, - ModTime: modTime, - DeleteMarkerReplicationStatus: opts.DeleteMarkerReplicationStatus, - VersionPurgeStatus: opts.VersionPurgeStatus, - TransitionStatus: opts.Transition.Status, - ExpireRestored: opts.Transition.ExpireRestored, + Name: object, + Deleted: deleteMarker, + MarkDeleted: markDelete, + ModTime: modTime, + ReplicationState: opts.DeleteReplication, + TransitionStatus: opts.Transition.Status, + ExpireRestored: opts.Transition.ExpireRestored, } fi.SetTierFreeVersionID(fvID) if opts.Versioned { @@ -1359,15 +1355,14 @@ func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string // Delete the object version on all disks. dfi := FileInfo{ - Name: object, - VersionID: opts.VersionID, - MarkDeleted: markDelete, - Deleted: deleteMarker, - ModTime: modTime, - DeleteMarkerReplicationStatus: opts.DeleteMarkerReplicationStatus, - VersionPurgeStatus: opts.VersionPurgeStatus, - TransitionStatus: opts.Transition.Status, - ExpireRestored: opts.Transition.ExpireRestored, + Name: object, + VersionID: opts.VersionID, + MarkDeleted: markDelete, + Deleted: deleteMarker, + ModTime: modTime, + ReplicationState: opts.DeleteReplication, + TransitionStatus: opts.Transition.Status, + ExpireRestored: opts.Transition.ExpireRestored, } dfi.SetTierFreeVersionID(fvID) if err = er.deleteObjectVersion(ctx, bucket, object, writeQuorum, dfi, opts.DeleteMarker); err != nil { @@ -1383,11 +1378,11 @@ func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string } return ObjectInfo{ - Bucket: bucket, - Name: object, - VersionID: opts.VersionID, - VersionPurgeStatus: opts.VersionPurgeStatus, - ReplicationStatus: replication.StatusType(opts.DeleteMarkerReplicationStatus), + Bucket: bucket, + Name: object, + VersionID: opts.VersionID, + VersionPurgeStatusInternal: opts.DeleteReplication.VersionPurgeStatusInternal, + ReplicationStatusInternal: opts.DeleteReplication.ReplicationStatusInternal, }, nil } diff --git a/cmd/erasure-server-pool.go b/cmd/erasure-server-pool.go index ba4cbbdb2..bd72184b0 100644 --- a/cmd/erasure-server-pool.go +++ b/cmd/erasure-server-pool.go @@ -456,7 +456,7 @@ func (z *erasureServerPools) StorageInfo(ctx context.Context) (StorageInfo, []er return storageInfo, errs } -func (z *erasureServerPools) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- madmin.DataUsageInfo, wantCycle uint32) error { +func (z *erasureServerPools) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- DataUsageInfo, wantCycle uint32) error { // Updates must be closed before we return. defer close(updates) @@ -474,7 +474,7 @@ func (z *erasureServerPools) NSScanner(ctx context.Context, bf *bloomFilter, upd } if len(allBuckets) == 0 { - updates <- madmin.DataUsageInfo{} // no buckets found update data usage to reflect latest state + updates <- DataUsageInfo{} // no buckets found update data usage to reflect latest state return nil } diff --git a/cmd/fs-v1.go b/cmd/fs-v1.go index 1cda0e1e0..44b45038c 100644 --- a/cmd/fs-v1.go +++ b/cmd/fs-v1.go @@ -240,7 +240,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<- madmin.DataUsageInfo, wantCycle uint32) error { +func (fs *FSObjects) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- DataUsageInfo, wantCycle uint32) error { defer close(updates) // Load bucket totals var totalCache dataUsageCache diff --git a/cmd/gateway-unsupported.go b/cmd/gateway-unsupported.go index 939a4c368..2ca959fb2 100644 --- a/cmd/gateway-unsupported.go +++ b/cmd/gateway-unsupported.go @@ -48,7 +48,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<- madmin.DataUsageInfo, wantCycle uint32) error { +func (a GatewayUnsupported) NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- DataUsageInfo, wantCycle uint32) error { logger.CriticalIf(ctx, errors.New("not implemented")) return NotImplemented{} } diff --git a/cmd/metrics-v2.go b/cmd/metrics-v2.go index 3247d07dd..68e4e806f 100644 --- a/cmd/metrics-v2.go +++ b/cmd/metrics-v2.go @@ -1352,7 +1352,7 @@ func getBucketUsageMetrics() MetricsGroup { }) for bucket, usage := range dataUsageInfo.BucketsUsage { - stat := getLatestReplicationStats(bucket, usage) + stats := getLatestReplicationStats(bucket, usage) metrics = append(metrics, Metric{ Description: getBucketUsageTotalBytesMD(), @@ -1366,27 +1366,29 @@ func getBucketUsageMetrics() MetricsGroup { VariableLabels: map[string]string{"bucket": bucket}, }) - if stat.hasReplicationUsage() { - metrics = append(metrics, Metric{ - Description: getBucketRepFailedBytesMD(), - Value: float64(stat.FailedSize), - VariableLabels: map[string]string{"bucket": bucket}, - }) - metrics = append(metrics, Metric{ - Description: getBucketRepSentBytesMD(), - Value: float64(stat.ReplicatedSize), - VariableLabels: map[string]string{"bucket": bucket}, - }) - metrics = append(metrics, Metric{ - Description: getBucketRepReceivedBytesMD(), - Value: float64(stat.ReplicaSize), - VariableLabels: map[string]string{"bucket": bucket}, - }) - metrics = append(metrics, Metric{ - Description: getBucketRepFailedOperationsMD(), - Value: float64(stat.FailedCount), - VariableLabels: map[string]string{"bucket": bucket}, - }) + if stats.hasReplicationUsage() { + for arn, stat := range stats.Stats { + metrics = append(metrics, Metric{ + Description: getBucketRepFailedBytesMD(), + Value: float64(stat.FailedSize), + VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn}, + }) + metrics = append(metrics, Metric{ + Description: getBucketRepSentBytesMD(), + Value: float64(stat.ReplicatedSize), + VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn}, + }) + metrics = append(metrics, Metric{ + Description: getBucketRepReceivedBytesMD(), + Value: float64(stat.ReplicaSize), + VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn}, + }) + metrics = append(metrics, Metric{ + Description: getBucketRepFailedOperationsMD(), + Value: float64(stat.FailedCount), + VariableLabels: map[string]string{"bucket": bucket, "targetArn": arn}, + }) + } } metrics = append(metrics, Metric{ diff --git a/cmd/metrics.go b/cmd/metrics.go index 51a8338ef..25f78c439 100644 --- a/cmd/metrics.go +++ b/cmd/metrics.go @@ -24,7 +24,6 @@ import ( "sync/atomic" "time" - "github.com/minio/madmin-go" "github.com/minio/minio/internal/logger" iampolicy "github.com/minio/pkg/iam/policy" "github.com/prometheus/client_golang/prometheus" @@ -435,30 +434,78 @@ 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) (s BucketReplicationStats) { +func getLatestReplicationStats(bucket string, u BucketUsageInfo) (s BucketReplicationStats) { + s = BucketReplicationStats{} + bucketStats := globalNotificationSys.GetClusterBucketStats(GlobalContext, bucket) - - replStats := BucketReplicationStats{} + // accumulate cluster bucket stats + stats := make(map[string]*BucketReplicationStat) for _, bucketStat := range bucketStats { - replStats.FailedCount += bucketStat.ReplicationStats.FailedCount - replStats.FailedSize += bucketStat.ReplicationStats.FailedSize - replStats.ReplicaSize += bucketStat.ReplicationStats.ReplicaSize - replStats.ReplicatedSize += bucketStat.ReplicationStats.ReplicatedSize + s.ReplicaSize += bucketStat.ReplicationStats.ReplicaSize + for arn, stat := range bucketStat.ReplicationStats.Stats { + oldst, ok := stats[arn] + if !ok { + oldst = &BucketReplicationStat{} + } + stats[arn] = &BucketReplicationStat{ + FailedCount: stat.FailedCount + oldst.FailedCount, + FailedSize: stat.FailedSize + oldst.FailedSize, + ReplicatedSize: stat.ReplicatedSize + oldst.ReplicatedSize, + } + s.FailedCount += stats[arn].FailedCount + s.FailedSize += stats[arn].FailedSize + s.ReplicatedSize += stats[arn].ReplicatedSize + } } - usageStat := globalReplicationStats.GetInitialUsage(bucket) - replStats.ReplicaSize += usageStat.ReplicaSize - replStats.ReplicatedSize += usageStat.ReplicatedSize + s.Stats = make(map[string]*BucketReplicationStat, len(stats)) - // use in memory replication stats if it is ahead of usage info. - s.ReplicatedSize = u.ReplicatedSize - if replStats.ReplicatedSize >= u.ReplicatedSize { - s.ReplicatedSize = replStats.ReplicatedSize + // add initial usage stat to cluster stats + usageStat := globalReplicationStats.GetInitialUsage(bucket) + if usageStat.Stats != nil { + for arn, stat := range usageStat.Stats { + st, ok := stats[arn] + if !ok { + st = &BucketReplicationStat{ + ReplicatedSize: stat.ReplicatedSize, + FailedSize: stat.FailedSize, + FailedCount: stat.FailedCount, + } + } else { + st.ReplicatedSize += stat.ReplicatedSize + st.FailedSize += stat.FailedSize + st.FailedCount += stat.FailedCount + } + stats[arn] = st + } } - // Reset FailedSize and FailedCount to 0 for negative overflows which can - // happen since data usage picture can lag behind actual usage state at the time of cluster start - s.FailedSize = uint64(math.Max(float64(replStats.FailedSize), 0)) - s.FailedCount = uint64(math.Max(float64(replStats.FailedCount), 0)) - s.ReplicaSize = uint64(math.Max(float64(replStats.ReplicaSize), float64(u.ReplicaSize))) + s.ReplicaSize += usageStat.ReplicaSize + // normalize computed real time stats with latest usage stat + var usgReplicatedSize int64 + for arn, tgtstat := range stats { + st := BucketReplicationStat{} + bu, ok := usageStat.Stats[arn] + if !ok { + bu = &BucketReplicationStat{} + } + usgReplicatedSize += bu.ReplicatedSize + // use in memory replication stats if it is ahead of usage info. + st.ReplicatedSize = bu.ReplicatedSize + if tgtstat.ReplicatedSize >= bu.ReplicatedSize { + st.ReplicatedSize = tgtstat.ReplicatedSize + } + // Reset FailedSize and FailedCount to 0 for negative overflows which can + // happen since data usage picture can lag behind actual usage state at the time of cluster start + st.FailedSize = int64(math.Max(float64(tgtstat.FailedSize), 0)) + st.FailedCount = int64(math.Max(float64(tgtstat.FailedCount), 0)) + st.ReplicaSize = int64(math.Max(float64(tgtstat.ReplicaSize), float64(u.ReplicaSize))) + s.Stats[arn] = &st + } + // normalize overall stats + s.FailedSize = int64(math.Max(float64(s.FailedSize), 0)) + s.FailedCount = int64(math.Max(float64(s.FailedCount), 0)) + s.ReplicaSize = int64(math.Max(float64(s.ReplicaSize), float64(u.ReplicaSize))) + s.ReplicatedSize = int64(math.Max(float64(s.ReplicatedSize), float64(usgReplicatedSize))) + return s } diff --git a/cmd/object-api-datatypes.go b/cmd/object-api-datatypes.go index 8aef0c8b9..9132ff6c2 100644 --- a/cmd/object-api-datatypes.go +++ b/cmd/object-api-datatypes.go @@ -141,7 +141,8 @@ type ObjectInfo struct { // Specify object storage class StorageClass string - ReplicationStatus replication.StatusType + ReplicationStatusInternal string + ReplicationStatus replication.StatusType // User-Defined metadata UserDefined map[string]string @@ -167,8 +168,9 @@ type ObjectInfo struct { // backendType indicates which backend filled this structure backendType BackendType - - VersionPurgeStatus VersionPurgeStatusType + // internal representation of version purge status + VersionPurgeStatusInternal string + VersionPurgeStatus VersionPurgeStatusType // The total count of all versions of this object NumVersions int @@ -179,40 +181,42 @@ type ObjectInfo struct { // Clone - Returns a cloned copy of current objectInfo func (o ObjectInfo) Clone() (cinfo ObjectInfo) { cinfo = ObjectInfo{ - Bucket: o.Bucket, - Name: o.Name, - ModTime: o.ModTime, - Size: o.Size, - IsDir: o.IsDir, - ETag: o.ETag, - InnerETag: o.InnerETag, - VersionID: o.VersionID, - IsLatest: o.IsLatest, - DeleteMarker: o.DeleteMarker, - TransitionedObject: o.TransitionedObject, - RestoreExpires: o.RestoreExpires, - RestoreOngoing: o.RestoreOngoing, - ContentType: o.ContentType, - ContentEncoding: o.ContentEncoding, - Expires: o.Expires, - CacheStatus: o.CacheStatus, - CacheLookupStatus: o.CacheLookupStatus, - StorageClass: o.StorageClass, - ReplicationStatus: o.ReplicationStatus, - UserTags: o.UserTags, - Parts: o.Parts, - Writer: o.Writer, - Reader: o.Reader, - PutObjReader: o.PutObjReader, - metadataOnly: o.metadataOnly, - versionOnly: o.versionOnly, - keyRotation: o.keyRotation, - backendType: o.backendType, - AccTime: o.AccTime, - Legacy: o.Legacy, - VersionPurgeStatus: o.VersionPurgeStatus, - NumVersions: o.NumVersions, - SuccessorModTime: o.SuccessorModTime, + Bucket: o.Bucket, + Name: o.Name, + ModTime: o.ModTime, + Size: o.Size, + IsDir: o.IsDir, + ETag: o.ETag, + InnerETag: o.InnerETag, + VersionID: o.VersionID, + IsLatest: o.IsLatest, + DeleteMarker: o.DeleteMarker, + TransitionedObject: o.TransitionedObject, + RestoreExpires: o.RestoreExpires, + RestoreOngoing: o.RestoreOngoing, + ContentType: o.ContentType, + ContentEncoding: o.ContentEncoding, + Expires: o.Expires, + CacheStatus: o.CacheStatus, + CacheLookupStatus: o.CacheLookupStatus, + StorageClass: o.StorageClass, + ReplicationStatus: o.ReplicationStatus, + UserTags: o.UserTags, + Parts: o.Parts, + Writer: o.Writer, + Reader: o.Reader, + PutObjReader: o.PutObjReader, + metadataOnly: o.metadataOnly, + versionOnly: o.versionOnly, + keyRotation: o.keyRotation, + backendType: o.backendType, + AccTime: o.AccTime, + Legacy: o.Legacy, + VersionPurgeStatus: o.VersionPurgeStatus, + NumVersions: o.NumVersions, + SuccessorModTime: o.SuccessorModTime, + ReplicationStatusInternal: o.ReplicationStatusInternal, + VersionPurgeStatusInternal: o.VersionPurgeStatusInternal, } cinfo.UserDefined = make(map[string]string, len(o.UserDefined)) for k, v := range o.UserDefined { @@ -224,9 +228,13 @@ func (o ObjectInfo) Clone() (cinfo ObjectInfo) { // ReplicateObjectInfo represents object info to be replicated type ReplicateObjectInfo struct { ObjectInfo - OpType replication.Type - RetryCount uint32 - ResetID string + OpType replication.Type + RetryCount uint32 + ResetID string + Dsc ReplicateDecision + ExistingObjResync ResyncDecision + TargetArn string + TargetStatuses map[string]replication.StatusType } // MultipartInfo captures metadata information about the uploadId diff --git a/cmd/object-api-interface.go b/cmd/object-api-interface.go index 237987d7f..5424ff7da 100644 --- a/cmd/object-api-interface.go +++ b/cmd/object-api-interface.go @@ -26,6 +26,7 @@ import ( "github.com/minio/madmin-go" "github.com/minio/minio-go/v7/pkg/encrypt" "github.com/minio/minio-go/v7/pkg/tags" + "github.com/minio/minio/internal/bucket/replication" "github.com/minio/pkg/bucket/policy" ) @@ -45,20 +46,22 @@ type ObjectOptions struct { MTime time.Time // Is only set in POST/PUT operations Expires time.Time // Is only used in POST/PUT operations - DeleteMarker bool // Is only set in DELETE operations for delete marker replication - UserDefined map[string]string // only set in case of POST/PUT operations - PartNumber int // only useful in case of GetObject/HeadObject - CheckPrecondFn CheckPreconditionFn // only set during GetObject/HeadObject/CopyObjectPart preconditional valuation - DeleteMarkerReplicationStatus string // Is only set in DELETE operations - VersionPurgeStatus VersionPurgeStatusType // Is only set in DELETE operations for delete marker version to be permanently deleted. - Transition TransitionOptions - Expiration ExpirationOptions + DeleteMarker bool // Is only set in DELETE operations for delete marker replication + UserDefined map[string]string // only set in case of POST/PUT operations + PartNumber int // only useful in case of GetObject/HeadObject + CheckPrecondFn CheckPreconditionFn // only set during GetObject/HeadObject/CopyObjectPart preconditional valuation + DeleteReplication ReplicationState // Represents internal replication state needed for Delete replication + Transition TransitionOptions + Expiration ExpirationOptions - NoLock bool // indicates to lower layers if the caller is expecting to hold locks. - ProxyRequest bool // only set for GET/HEAD in active-active replication scenario - ProxyHeaderSet bool // only set for GET/HEAD in active-active replication scenario - - DeletePrefix bool // set true to enforce a prefix deletion, only application for DeleteObject API, + NoLock bool // indicates to lower layers if the caller is expecting to hold locks. + ProxyRequest bool // only set for GET/HEAD in active-active replication scenario + ProxyHeaderSet bool // only set for GET/HEAD in active-active replication scenario + ReplicationRequest bool // true only if replication request + ReplicationSourceTaggingTimestamp time.Time // set if MinIOSourceTaggingTimestamp received + ReplicationSourceLegalholdTimestamp time.Time // set if MinIOSourceObjectLegalholdTimestamp received + ReplicationSourceRetentionTimestamp time.Time // set if MinIOSourceObjectRetentionTimestamp received + DeletePrefix bool // set true to enforce a prefix deletion, only application for DeleteObject API, // Use the maximum parity (N/2), used when saving server configuration files MaxParity bool @@ -86,6 +89,48 @@ type BucketOptions struct { VersioningEnabled bool } +// SetReplicaStatus sets replica status and timestamp for delete operations in ObjectOptions +func (o *ObjectOptions) SetReplicaStatus(st replication.StatusType) { + o.DeleteReplication.ReplicaStatus = st + o.DeleteReplication.ReplicaTimeStamp = UTCNow() +} + +// DeleteMarkerReplicationStatus - returns replication status of delete marker from DeleteReplication state in ObjectOptions +func (o *ObjectOptions) DeleteMarkerReplicationStatus() replication.StatusType { + return o.DeleteReplication.CompositeReplicationStatus() +} + +// VersionPurgeStatus - returns version purge status from DeleteReplication state in ObjectOptions +func (o *ObjectOptions) VersionPurgeStatus() VersionPurgeStatusType { + return o.DeleteReplication.CompositeVersionPurgeStatus() +} + +// SetDeleteReplicationState sets the delete replication options. +func (o *ObjectOptions) SetDeleteReplicationState(dsc ReplicateDecision, vID string) { + o.DeleteReplication = ReplicationState{ + ReplicateDecisionStr: dsc.String(), + } + switch { + case o.VersionID == "": + o.DeleteReplication.ReplicationStatusInternal = dsc.PendingStatus() + o.DeleteReplication.Targets = replicationStatusesMap(o.DeleteReplication.ReplicationStatusInternal) + default: + o.DeleteReplication.VersionPurgeStatusInternal = dsc.PendingStatus() + o.DeleteReplication.PurgeTargets = versionPurgeStatusesMap(o.DeleteReplication.VersionPurgeStatusInternal) + } +} + +// PutReplicationState gets ReplicationState for PUT operation from ObjectOptions +func (o *ObjectOptions) PutReplicationState() (r ReplicationState) { + rstatus, ok := o.UserDefined[ReservedMetadataPrefixLower+ReplicationStatus] + if !ok { + return + } + r.ReplicationStatusInternal = rstatus + r.Targets = replicationStatusesMap(rstatus) + return +} + // LockType represents required locking for ObjectLayer operations type LockType int @@ -109,7 +154,7 @@ type ObjectLayer interface { // Storage operations. Shutdown(context.Context) error - NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- madmin.DataUsageInfo, wantCycle uint32) error + NSScanner(ctx context.Context, bf *bloomFilter, updates chan<- DataUsageInfo, wantCycle uint32) error BackendInfo() madmin.BackendInfo StorageInfo(ctx context.Context) (StorageInfo, []error) diff --git a/cmd/object-api-options.go b/cmd/object-api-options.go index 7162f0c39..49f29351f 100644 --- a/cmd/object-api-options.go +++ b/cmd/object-api-options.go @@ -71,6 +71,9 @@ func getDefaultOpts(header http.Header, copySource bool, metadata map[string]str opts.ProxyHeaderSet = true opts.ProxyRequest = strings.Join(v, "") == "true" } + if _, ok := header[xhttp.MinIOSourceReplicationRequest]; ok { + opts.ReplicationRequest = true + } return } @@ -183,23 +186,6 @@ func delOpts(ctx context.Context, r *http.Request, bucket, object string) (opts } } - purgeVersion := strings.TrimSpace(r.Header.Get(xhttp.MinIOSourceDeleteMarkerDelete)) - if purgeVersion != "" { - switch purgeVersion { - case "true": - opts.VersionPurgeStatus = Complete - case "false": - default: - err = fmt.Errorf("Unable to parse %s, failed with %w", xhttp.MinIOSourceDeleteMarkerDelete, fmt.Errorf("DeleteMarkerPurge should be true or false")) - logger.LogIf(ctx, err) - return opts, InvalidArgument{ - Bucket: bucket, - Object: object, - Err: err, - } - } - } - mtime := strings.TrimSpace(r.Header.Get(xhttp.MinIOSourceMTime)) if mtime != "" { opts.MTime, err = time.Parse(time.RFC3339Nano, mtime) @@ -251,6 +237,44 @@ func putOpts(ctx context.Context, r *http.Request, bucket, object string, metada } } } + retaintimeStr := strings.TrimSpace(r.Header.Get(xhttp.MinIOSourceObjectRetentionTimestamp)) + retaintimestmp := mtime + if retaintimeStr != "" { + retaintimestmp, err = time.Parse(time.RFC3339, retaintimeStr) + if err != nil { + return opts, InvalidArgument{ + Bucket: bucket, + Object: object, + Err: fmt.Errorf("Unable to parse %s, failed with %w", xhttp.MinIOSourceObjectRetentionTimestamp, err), + } + } + } + + lholdtimeStr := strings.TrimSpace(r.Header.Get(xhttp.MinIOSourceObjectLegalHoldTimestamp)) + lholdtimestmp := mtime + if lholdtimeStr != "" { + lholdtimestmp, err = time.Parse(time.RFC3339, lholdtimeStr) + if err != nil { + return opts, InvalidArgument{ + Bucket: bucket, + Object: object, + Err: fmt.Errorf("Unable to parse %s, failed with %w", xhttp.MinIOSourceObjectLegalHoldTimestamp, err), + } + } + } + tagtimeStr := strings.TrimSpace(r.Header.Get(xhttp.MinIOSourceTaggingTimestamp)) + taggingtimestmp := mtime + if tagtimeStr != "" { + taggingtimestmp, err = time.Parse(time.RFC3339, tagtimeStr) + if err != nil { + return opts, InvalidArgument{ + Bucket: bucket, + Object: object, + Err: fmt.Errorf("Unable to parse %s, failed with %w", xhttp.MinIOSourceTaggingTimestamp, err), + } + } + } + etag := strings.TrimSpace(r.Header.Get(xhttp.MinIOSourceETag)) if etag != "" { if metadata == nil { @@ -306,6 +330,9 @@ func putOpts(ctx context.Context, r *http.Request, bucket, object string, metada opts.Versioned = versioned opts.VersionSuspended = versionSuspended opts.MTime = mtime + opts.ReplicationSourceLegalholdTimestamp = lholdtimestmp + opts.ReplicationSourceRetentionTimestamp = retaintimestmp + opts.ReplicationSourceTaggingTimestamp = taggingtimestmp return opts, nil } diff --git a/cmd/object-api-utils.go b/cmd/object-api-utils.go index 0f9ae1a84..d6944ed09 100644 --- a/cmd/object-api-utils.go +++ b/cmd/object-api-utils.go @@ -263,7 +263,7 @@ func cleanMetadata(metadata map[string]string) map[string]string { // Remove STANDARD StorageClass metadata = removeStandardStorageClass(metadata) // Clean meta etag keys 'md5Sum', 'etag', "expires", "x-amz-tagging". - return cleanMetadataKeys(metadata, "md5Sum", "etag", "expires", xhttp.AmzObjectTagging, "last-modified") + return cleanMetadataKeys(metadata, "md5Sum", "etag", "expires", xhttp.AmzObjectTagging, "last-modified", VersionPurgeStatusKey) } // Filter X-Amz-Storage-Class field only if it is set to STANDARD. diff --git a/cmd/object-handlers.go b/cmd/object-handlers.go index 628118632..e865d3f26 100644 --- a/cmd/object-handlers.go +++ b/cmd/object-handlers.go @@ -412,9 +412,10 @@ func (api objectAPIHandlers) getObjectHandler(ctx context.Context, objectAPI Obj reader *GetObjectReader proxy bool ) - if isProxyable(ctx, bucket) { + proxytgts := getproxyTargets(ctx, bucket, object, opts) + if !proxytgts.Empty() { // proxy to replication target if active-active replication is in place. - reader, proxy = proxyGetToReplicationTarget(ctx, bucket, object, rs, r.Header, opts) + reader, proxy = proxyGetToReplicationTarget(ctx, bucket, object, rs, r.Header, opts, proxytgts) if reader != nil && proxy { gr = reader } @@ -630,23 +631,22 @@ func (api objectAPIHandlers) headObjectHandler(ctx context.Context, objectAPI Ob if err != nil { var ( proxy bool - perr error oi ObjectInfo ) // proxy HEAD to replication target if active-active replication configured on bucket - if isProxyable(ctx, bucket) { - oi, proxy, perr = proxyHeadToReplicationTarget(ctx, bucket, object, opts) - if proxy && perr == nil { + proxytgts := getproxyTargets(ctx, bucket, object, opts) + if !proxytgts.Empty() { + oi, proxy = proxyHeadToReplicationTarget(ctx, bucket, object, opts, proxytgts) + if proxy { objInfo = oi } } - if !proxy || perr != nil { + if !proxy { if globalBucketVersioningSys.Enabled(bucket) { - if !objInfo.VersionPurgeStatus.Empty() { - // Shows the replication status of a permanent delete of a version + switch { + case !objInfo.VersionPurgeStatus.Empty(): w.Header()[xhttp.MinIODeleteReplicationStatus] = []string{string(objInfo.VersionPurgeStatus)} - } - if !objInfo.ReplicationStatus.Empty() && objInfo.DeleteMarker { + case !objInfo.ReplicationStatus.Empty() && objInfo.DeleteMarker: w.Header()[xhttp.MinIODeleteMarkerReplicationStatus] = []string{string(objInfo.ReplicationStatus)} } // Versioning enabled quite possibly object is deleted might be delete-marker @@ -1295,10 +1295,25 @@ func (api objectAPIHandlers) CopyObjectHandler(w http.ResponseWriter, r *http.Re } if objTags != "" { - srcInfo.UserDefined[xhttp.AmzObjectTagging] = objTags - } - srcInfo.UserDefined = filterReplicationStatusMetadata(srcInfo.UserDefined) + lastTaggingTimestamp := srcInfo.UserDefined[ReservedMetadataPrefixLower+TaggingTimestamp] + if dstOpts.ReplicationRequest { + srcTimestamp := dstOpts.ReplicationSourceTaggingTimestamp + if !srcTimestamp.IsZero() { + ondiskTimestamp, err := time.Parse(lastTaggingTimestamp, time.RFC3339Nano) + // update tagging metadata only if replica timestamp is newer than what's on disk + if err != nil || (err == nil && ondiskTimestamp.Before(srcTimestamp)) { + srcInfo.UserDefined[ReservedMetadataPrefixLower+TaggingTimestamp] = srcTimestamp.Format(time.RFC3339Nano) + srcInfo.UserDefined[xhttp.AmzObjectTagging] = objTags + } + } + } else { + srcInfo.UserDefined[xhttp.AmzObjectTagging] = objTags + srcInfo.UserDefined[ReservedMetadataPrefixLower+TaggingTimestamp] = UTCNow().Format(time.RFC3339Nano) + } + } + + srcInfo.UserDefined = filterReplicationStatusMetadata(srcInfo.UserDefined) srcInfo.UserDefined = objectlock.FilterObjectLockMetadata(srcInfo.UserDefined, true, true) retPerms := isPutActionAllowed(ctx, getRequestAuthType(r), dstBucket, dstObject, r, iampolicy.PutObjectRetentionAction) holdPerms := isPutActionAllowed(ctx, getRequestAuthType(r), dstBucket, dstObject, r, iampolicy.PutObjectLegalHoldAction) @@ -1310,21 +1325,53 @@ func (api objectAPIHandlers) CopyObjectHandler(w http.ResponseWriter, r *http.Re // apply default bucket configuration/governance headers for dest side. retentionMode, retentionDate, legalHold, s3Err := checkPutObjectLockAllowed(ctx, r, dstBucket, dstObject, getObjectInfo, retPerms, holdPerms) if s3Err == ErrNone && retentionMode.Valid() { - srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockMode)] = string(retentionMode) - srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockRetainUntilDate)] = retentionDate.UTC().Format(iso8601TimeFormat) + lastretentionTimestamp := srcInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockRetentionTimestamp] + if dstOpts.ReplicationRequest { + srcTimestamp := dstOpts.ReplicationSourceRetentionTimestamp + if !srcTimestamp.IsZero() { + ondiskTimestamp, err := time.Parse(lastretentionTimestamp, time.RFC3339Nano) + // update retention metadata only if replica timestamp is newer than what's on disk + if err != nil || (err == nil && ondiskTimestamp.Before(srcTimestamp)) { + srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockMode)] = string(retentionMode) + srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockRetainUntilDate)] = retentionDate.UTC().Format(iso8601TimeFormat) + srcInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockRetentionTimestamp] = srcTimestamp.Format(time.RFC3339Nano) + } + } + } else { + srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockMode)] = string(retentionMode) + srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockRetainUntilDate)] = retentionDate.UTC().Format(iso8601TimeFormat) + srcInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockRetentionTimestamp] = UTCNow().Format(time.RFC3339Nano) + } } + if s3Err == ErrNone && legalHold.Status.Valid() { - srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockLegalHold)] = string(legalHold.Status) + lastLegalHoldTimestamp := srcInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockLegalHoldTimestamp] + if dstOpts.ReplicationRequest { + srcTimestamp := dstOpts.ReplicationSourceLegalholdTimestamp + if !srcTimestamp.IsZero() { + ondiskTimestamp, err := time.Parse(lastLegalHoldTimestamp, time.RFC3339Nano) + // update legalhold metadata only if replica timestamp is newer than what's on disk + if err != nil || (err == nil && ondiskTimestamp.Before(srcTimestamp)) { + srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockLegalHold)] = string(legalHold.Status) + srcInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockRetentionTimestamp] = srcTimestamp.Format(time.RFC3339Nano) + } + } + } else { + srcInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockLegalHold)] = string(legalHold.Status) + } } if s3Err != ErrNone { writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL) return } if rs := r.Header.Get(xhttp.AmzBucketReplicationStatus); rs != "" { + srcInfo.UserDefined[ReservedMetadataPrefixLower+ReplicaStatus] = replication.Replica.String() + srcInfo.UserDefined[ReservedMetadataPrefixLower+ReplicaTimestamp] = UTCNow().Format(time.RFC3339Nano) srcInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = rs } - if ok, _ := mustReplicate(ctx, dstBucket, dstObject, getMustReplicateOptions(srcInfo, replication.UnsetReplicationType)); ok { - srcInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String() + if dsc := mustReplicate(ctx, dstBucket, dstObject, getMustReplicateOptions(srcInfo, replication.UnsetReplicationType, dstOpts)); dsc.ReplicateAny() { + srcInfo.UserDefined[ReservedMetadataPrefixLower+ReplicationStatus] = dsc.PendingStatus() + srcInfo.UserDefined[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) } // Store the preserved compression metadata. for k, v := range compressMetadata { @@ -1431,8 +1478,8 @@ func (api objectAPIHandlers) CopyObjectHandler(w http.ResponseWriter, r *http.Re response := generateCopyObjectResponse(objInfo.ETag, objInfo.ModTime) encodedSuccessResponse := encodeResponse(response) - if replicate, sync := mustReplicate(ctx, dstBucket, dstObject, getMustReplicateOptions(objInfo, replication.UnsetReplicationType)); replicate { - scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType) + if dsc := mustReplicate(ctx, dstBucket, dstObject, getMustReplicateOptions(objInfo, replication.UnsetReplicationType, dstOpts)); dsc.ReplicateAny() { + scheduleReplication(ctx, objInfo.Clone(), objectAPI, dsc, replication.ObjectReplicationType) } setPutObjHeaders(w, objInfo, false) @@ -1608,6 +1655,15 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) return } + if r.Header.Get(xhttp.AmzBucketReplicationStatus) == replication.Replica.String() { + if s3Err = isPutActionAllowed(ctx, getRequestAuthType(r), bucket, object, r, iampolicy.ReplicateObjectAction); s3Err != ErrNone { + writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL) + return + } + metadata[ReservedMetadataPrefixLower+ReplicaStatus] = replication.Replica.String() + metadata[ReservedMetadataPrefixLower+ReplicaTimestamp] = UTCNow().Format(time.RFC3339Nano) + defer globalReplicationStats.UpdateReplicaStat(bucket, size) + } // Check if bucket encryption is enabled sseConfig, _ := globalBucketSSEConfigSys.Get(bucket) @@ -1675,16 +1731,11 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL) return } - if ok, _ := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ + if dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ UserDefined: metadata, - }, replication.ObjectReplicationType)); ok { - metadata[xhttp.AmzBucketReplicationStatus] = replication.Pending.String() - } - if r.Header.Get(xhttp.AmzBucketReplicationStatus) == replication.Replica.String() { - if s3Err = isPutActionAllowed(ctx, getRequestAuthType(r), bucket, object, r, iampolicy.ReplicateObjectAction); s3Err != ErrNone { - writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL) - return - } + }, replication.ObjectReplicationType, opts)); dsc.ReplicateAny() { + metadata[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) + metadata[ReservedMetadataPrefixLower+ReplicationStatus] = dsc.PendingStatus() } var objectEncryptionKey crypto.ObjectKey if objectAPI.IsEncryptionSupported() { @@ -1770,10 +1821,10 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req } } } - if replicate, sync := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ + if dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ UserDefined: metadata, - }, replication.ObjectReplicationType)); replicate { - scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType) + }, replication.ObjectReplicationType, opts)); dsc.ReplicateAny() { + scheduleReplication(ctx, objInfo.Clone(), objectAPI, dsc, replication.ObjectReplicationType) } setPutObjHeaders(w, objInfo, false) @@ -1987,6 +2038,15 @@ func (api objectAPIHandlers) PutObjectExtractHandler(w http.ResponseWriter, r *h rawReader := hashReader pReader := NewPutObjReader(rawReader) + if r.Header.Get(xhttp.AmzBucketReplicationStatus) == replication.Replica.String() { + if s3Err = isPutActionAllowed(ctx, getRequestAuthType(r), bucket, object, r, iampolicy.ReplicateObjectAction); s3Err != ErrNone { + writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL) + return + } + metadata[ReservedMetadataPrefixLower+ReplicaStatus] = replication.Replica.String() + metadata[ReservedMetadataPrefixLower+ReplicaTimestamp] = UTCNow().Format(time.RFC3339Nano) + } + // get encryption options opts, err := putOpts(ctx, r, bucket, object, metadata) if err != nil { @@ -2010,17 +2070,12 @@ func (api objectAPIHandlers) PutObjectExtractHandler(w http.ResponseWriter, r *h return } - if ok, _ := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ + if dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ UserDefined: metadata, - }, replication.ObjectReplicationType)); ok { - metadata[xhttp.AmzBucketReplicationStatus] = replication.Pending.String() - } + }, replication.ObjectReplicationType, opts)); dsc.ReplicateAny() { + metadata[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) + metadata[ReservedMetadataPrefixLower+ReplicationStatus] = dsc.PendingStatus() - if r.Header.Get(xhttp.AmzBucketReplicationStatus) == replication.Replica.String() { - if s3Err = isPutActionAllowed(ctx, getRequestAuthType(r), bucket, object, r, iampolicy.ReplicateObjectAction); s3Err != ErrNone { - writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL) - return - } } var objectEncryptionKey crypto.ObjectKey @@ -2068,10 +2123,10 @@ func (api objectAPIHandlers) PutObjectExtractHandler(w http.ResponseWriter, r *h return } - if replicate, sync := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ + if dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ UserDefined: metadata, - }, replication.ObjectReplicationType)); replicate { - scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType) + }, replication.ObjectReplicationType, opts)); dsc.ReplicateAny() { + scheduleReplication(ctx, objInfo.Clone(), objectAPI, dsc, replication.ObjectReplicationType) } @@ -2182,10 +2237,11 @@ func (api objectAPIHandlers) NewMultipartUploadHandler(w http.ResponseWriter, r writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL) return } - if ok, _ := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ + if dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(ObjectInfo{ UserDefined: metadata, - }, replication.ObjectReplicationType)); ok { - metadata[xhttp.AmzBucketReplicationStatus] = replication.Pending.String() + }, replication.ObjectReplicationType, ObjectOptions{})); dsc.ReplicateAny() { + metadata[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) + metadata[ReservedMetadataPrefixLower+ReplicationStatus] = dsc.PendingStatus() } // We need to preserve the encryption headers set in EncryptRequest, // so we do not want to override them, copy them instead. @@ -3199,8 +3255,12 @@ func (api objectAPIHandlers) CompleteMultipartUploadHandler(w http.ResponseWrite } setPutObjHeaders(w, objInfo, false) - if replicate, sync := mustReplicate(ctx, bucket, object, getMustReplicateOptions(objInfo, replication.ObjectReplicationType)); replicate { - scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType) + if dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(objInfo, replication.ObjectReplicationType, opts)); dsc.ReplicateAny() { + scheduleReplication(ctx, objInfo.Clone(), objectAPI, dsc, replication.ObjectReplicationType) + } + if objInfo.ReplicationStatus == replication.Replica { + actualSize, _ := objInfo.GetActualSize() + globalReplicationStats.UpdateReplicaStat(bucket, actualSize) } // Write success response. @@ -3284,19 +3344,20 @@ func (api objectAPIHandlers) DeleteObjectHandler(w http.ResponseWriter, r *http. os.SetTransitionState(goi.TransitionedObject) } - replicateDel, replicateSync := checkReplicateDelete(ctx, bucket, ObjectToDelete{ObjectName: object, VersionID: opts.VersionID}, goi, gerr) - if replicateDel { - if opts.VersionID != "" { - opts.VersionPurgeStatus = Pending - } else { - opts.DeleteMarkerReplicationStatus = string(replication.Pending) - } + dsc := checkReplicateDelete(ctx, bucket, ObjectToDelete{ObjectName: object, VersionID: opts.VersionID}, goi, opts, gerr) + if dsc.ReplicateAny() { + opts.SetDeleteReplicationState(dsc, opts.VersionID) } vID := opts.VersionID if r.Header.Get(xhttp.AmzBucketReplicationStatus) == replication.Replica.String() { - opts.DeleteMarkerReplicationStatus = replication.Replica.String() - if opts.VersionPurgeStatus.Empty() { + // check if replica has permission to be deleted. + if apiErrCode := checkRequestAuthType(ctx, r, policy.ReplicateDeleteAction, bucket, object); apiErrCode != ErrNone { + writeErrorResponse(ctx, w, errorCodes.ToAPIErr(apiErrCode), r.URL) + return + } + opts.SetReplicaStatus(replication.Replica) + if opts.VersionPurgeStatus().Empty() { // opts.VersionID holds delete marker version ID to replicate and not yet present on disk vID = "" } @@ -3365,7 +3426,7 @@ func (api objectAPIHandlers) DeleteObjectHandler(w http.ResponseWriter, r *http. Host: handlers.GetSourceIP(r), }) - if replicateDel { + if dsc.ReplicateAny() { dmVersionID := "" versionID := "" if objInfo.DeleteMarker { @@ -3375,17 +3436,16 @@ func (api objectAPIHandlers) DeleteObjectHandler(w http.ResponseWriter, r *http. } dobj := DeletedObjectReplicationInfo{ DeletedObject: DeletedObject{ - ObjectName: object, - VersionID: versionID, - DeleteMarkerVersionID: dmVersionID, - DeleteMarkerReplicationStatus: string(objInfo.ReplicationStatus), - DeleteMarkerMTime: DeleteMarkerMTime{objInfo.ModTime}, - DeleteMarker: objInfo.DeleteMarker, - VersionPurgeStatus: objInfo.VersionPurgeStatus, + ObjectName: object, + VersionID: versionID, + DeleteMarkerVersionID: dmVersionID, + DeleteMarkerMTime: DeleteMarkerMTime{objInfo.ModTime}, + DeleteMarker: objInfo.DeleteMarker, + ReplicationState: objInfo.getReplicationState(dsc.String(), opts.VersionID, false), }, Bucket: bucket, } - scheduleReplicationDelete(ctx, dobj, objectAPI, replicateSync) + scheduleReplicationDelete(ctx, dobj, objectAPI) } // Remove the transitioned object whose object version is being overwritten. @@ -3461,9 +3521,12 @@ func (api objectAPIHandlers) PutObjectLegalHoldHandler(w http.ResponseWriter, r return } objInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockLegalHold)] = strings.ToUpper(string(legalHold.Status)) - replicate, sync := mustReplicate(ctx, bucket, object, getMustReplicateOptions(objInfo, replication.MetadataReplicationType)) - if replicate { - objInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String() + objInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockLegalHoldTimestamp] = UTCNow().Format(time.RFC3339Nano) + + dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(objInfo, replication.MetadataReplicationType, opts)) + if dsc.ReplicateAny() { + objInfo.UserDefined[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) + objInfo.UserDefined[ReservedMetadataPrefixLower+ReplicationStatus] = dsc.PendingStatus() } // if version-id is not specified retention is supposed to be set on the latest object. if opts.VersionID == "" { @@ -3481,8 +3544,8 @@ func (api objectAPIHandlers) PutObjectLegalHoldHandler(w http.ResponseWriter, r writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) return } - if replicate { - scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.MetadataReplicationType) + if dsc.ReplicateAny() { + scheduleReplication(ctx, objInfo.Clone(), objectAPI, dsc, replication.MetadataReplicationType) } writeSuccessResponseHeadersOnly(w) @@ -3640,9 +3703,12 @@ func (api objectAPIHandlers) PutObjectRetentionHandler(w http.ResponseWriter, r objInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockMode)] = "" objInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockRetainUntilDate)] = "" } - replicate, sync := mustReplicate(ctx, bucket, object, getMustReplicateOptions(objInfo, replication.MetadataReplicationType)) - if replicate { - objInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String() + objInfo.UserDefined[ReservedMetadataPrefixLower+ObjectLockRetentionTimestamp] = UTCNow().Format(time.RFC3339Nano) + + dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(objInfo, replication.MetadataReplicationType, opts)) + if dsc.ReplicateAny() { + objInfo.UserDefined[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) + objInfo.UserDefined[ReservedMetadataPrefixLower+ReplicationStatus] = dsc.PendingStatus() } // if version-id is not specified retention is supposed to be set on the latest object. if opts.VersionID == "" { @@ -3660,8 +3726,8 @@ func (api objectAPIHandlers) PutObjectRetentionHandler(w http.ResponseWriter, r writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) return } - if replicate { - scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.MetadataReplicationType) + if dsc.ReplicateAny() { + scheduleReplication(ctx, objInfo.Clone(), objectAPI, dsc, replication.MetadataReplicationType) } writeSuccessNoContent(w) @@ -3841,10 +3907,12 @@ func (api objectAPIHandlers) PutObjectTaggingHandler(w http.ResponseWriter, r *h oi := objInfo.Clone() oi.UserTags = tagsStr - replicate, sync := mustReplicate(ctx, bucket, object, getMustReplicateOptions(oi, replication.MetadataReplicationType)) - if replicate { + dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(oi, replication.MetadataReplicationType, opts)) + if dsc.ReplicateAny() { opts.UserDefined = make(map[string]string) - opts.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String() + opts.UserDefined[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) + opts.UserDefined[ReservedMetadataPrefixLower+ReplicationStatus] = dsc.PendingStatus() + opts.UserDefined[ReservedMetadataPrefixLower+TaggingTimestamp] = UTCNow().Format(time.RFC3339Nano) } // Put object tags @@ -3854,8 +3922,8 @@ func (api objectAPIHandlers) PutObjectTaggingHandler(w http.ResponseWriter, r *h return } - if replicate { - scheduleReplication(ctx, objInfo.Clone(), objAPI, sync, replication.MetadataReplicationType) + if dsc.ReplicateAny() { + scheduleReplication(ctx, objInfo.Clone(), objAPI, dsc, replication.MetadataReplicationType) } if objInfo.VersionID != "" { @@ -3916,10 +3984,11 @@ func (api objectAPIHandlers) DeleteObjectTaggingHandler(w http.ResponseWriter, r writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) return } - replicate, sync := mustReplicate(ctx, bucket, object, getMustReplicateOptions(oi, replication.MetadataReplicationType)) - if replicate { + dsc := mustReplicate(ctx, bucket, object, getMustReplicateOptions(oi, replication.MetadataReplicationType, opts)) + if dsc.ReplicateAny() { opts.UserDefined = make(map[string]string) - opts.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String() + opts.UserDefined[ReservedMetadataPrefixLower+ReplicationTimestamp] = UTCNow().Format(time.RFC3339Nano) + opts.UserDefined[ReservedMetadataPrefixLower+ReplicationStatus] = dsc.PendingStatus() } oi, err = objAPI.DeleteObjectTags(ctx, bucket, object, opts) @@ -3928,8 +3997,8 @@ func (api objectAPIHandlers) DeleteObjectTaggingHandler(w http.ResponseWriter, r return } - if replicate { - scheduleReplication(ctx, oi.Clone(), objAPI, sync, replication.MetadataReplicationType) + if dsc.ReplicateAny() { + scheduleReplication(ctx, oi.Clone(), objAPI, dsc, replication.MetadataReplicationType) } if oi.VersionID != "" { diff --git a/cmd/storage-datatypes.go b/cmd/storage-datatypes.go index 76e314a51..c39f857b5 100644 --- a/cmd/storage-datatypes.go +++ b/cmd/storage-datatypes.go @@ -169,11 +169,8 @@ type FileInfo struct { // Erasure info for all objects. Erasure ErasureInfo - // DeleteMarkerReplicationStatus is set when this FileInfo represents - // replication on a DeleteMarker - MarkDeleted bool // mark this version as deleted - DeleteMarkerReplicationStatus string - VersionPurgeStatus VersionPurgeStatusType + MarkDeleted bool // mark this version as deleted + ReplicationState ReplicationState // Internal replication state to be passed back in ObjectInfo Data []byte // optionally carries object data @@ -200,30 +197,6 @@ func (fi *FileInfo) SetInlineData() { // VersionPurgeStatusKey denotes purge status in metadata const VersionPurgeStatusKey = "purgestatus" -// VersionPurgeStatusType represents status of a versioned delete or permanent delete w.r.t bucket replication -type VersionPurgeStatusType string - -const ( - // Pending - versioned delete replication is pending. - Pending VersionPurgeStatusType = "PENDING" - - // Complete - versioned delete replication is now complete, erase version on disk. - Complete VersionPurgeStatusType = "COMPLETE" - - // Failed - versioned delete replication failed. - Failed VersionPurgeStatusType = "FAILED" -) - -// Empty returns true if purge status was not set. -func (v VersionPurgeStatusType) Empty() bool { - return string(v) == "" -} - -// Pending returns true if the version is pending purge. -func (v VersionPurgeStatusType) Pending() bool { - return v == Pending || v == Failed -} - // newFileInfo - initializes new FileInfo, allocates a fresh erasure info. func newFileInfo(object string, dataBlocks, parityBlocks int) (fi FileInfo) { fi.Erasure = ErasureInfo{ diff --git a/cmd/storage-datatypes_gen.go b/cmd/storage-datatypes_gen.go index 05cece815..5ef5d7663 100644 --- a/cmd/storage-datatypes_gen.go +++ b/cmd/storage-datatypes_gen.go @@ -550,8 +550,8 @@ func (z *FileInfo) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err) return } - if zb0001 != 25 { - err = msgp.ArrayError{Wanted: 25, Got: zb0001} + if zb0001 != 24 { + err = msgp.ArrayError{Wanted: 24, Got: zb0001} return } z.Volume, err = dc.ReadString() @@ -686,20 +686,11 @@ func (z *FileInfo) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "MarkDeleted") return } - z.DeleteMarkerReplicationStatus, err = dc.ReadString() + err = z.ReplicationState.DecodeMsg(dc) if err != nil { - err = msgp.WrapError(err, "DeleteMarkerReplicationStatus") + err = msgp.WrapError(err, "ReplicationState") return } - { - var zb0004 string - zb0004, err = dc.ReadString() - if err != nil { - err = msgp.WrapError(err, "VersionPurgeStatus") - return - } - z.VersionPurgeStatus = VersionPurgeStatusType(zb0004) - } z.Data, err = dc.ReadBytes(z.Data) if err != nil { err = msgp.WrapError(err, "Data") @@ -725,8 +716,8 @@ func (z *FileInfo) DecodeMsg(dc *msgp.Reader) (err error) { // EncodeMsg implements msgp.Encodable func (z *FileInfo) EncodeMsg(en *msgp.Writer) (err error) { - // array header, size 25 - err = en.Append(0xdc, 0x0, 0x19) + // array header, size 24 + err = en.Append(0xdc, 0x0, 0x18) if err != nil { return } @@ -844,14 +835,9 @@ func (z *FileInfo) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "MarkDeleted") return } - err = en.WriteString(z.DeleteMarkerReplicationStatus) + err = z.ReplicationState.EncodeMsg(en) if err != nil { - err = msgp.WrapError(err, "DeleteMarkerReplicationStatus") - return - } - err = en.WriteString(string(z.VersionPurgeStatus)) - if err != nil { - err = msgp.WrapError(err, "VersionPurgeStatus") + err = msgp.WrapError(err, "ReplicationState") return } err = en.WriteBytes(z.Data) @@ -880,8 +866,8 @@ func (z *FileInfo) EncodeMsg(en *msgp.Writer) (err error) { // MarshalMsg implements msgp.Marshaler func (z *FileInfo) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // array header, size 25 - o = append(o, 0xdc, 0x0, 0x19) + // array header, size 24 + o = append(o, 0xdc, 0x0, 0x18) o = msgp.AppendString(o, z.Volume) o = msgp.AppendString(o, z.Name) o = msgp.AppendString(o, z.VersionID) @@ -916,8 +902,11 @@ func (z *FileInfo) MarshalMsg(b []byte) (o []byte, err error) { return } o = msgp.AppendBool(o, z.MarkDeleted) - o = msgp.AppendString(o, z.DeleteMarkerReplicationStatus) - o = msgp.AppendString(o, string(z.VersionPurgeStatus)) + o, err = z.ReplicationState.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "ReplicationState") + return + } o = msgp.AppendBytes(o, z.Data) o = msgp.AppendInt(o, z.NumVersions) o = msgp.AppendTime(o, z.SuccessorModTime) @@ -933,8 +922,8 @@ func (z *FileInfo) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err) return } - if zb0001 != 25 { - err = msgp.ArrayError{Wanted: 25, Got: zb0001} + if zb0001 != 24 { + err = msgp.ArrayError{Wanted: 24, Got: zb0001} return } z.Volume, bts, err = msgp.ReadStringBytes(bts) @@ -1069,20 +1058,11 @@ func (z *FileInfo) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err, "MarkDeleted") return } - z.DeleteMarkerReplicationStatus, bts, err = msgp.ReadStringBytes(bts) + bts, err = z.ReplicationState.UnmarshalMsg(bts) if err != nil { - err = msgp.WrapError(err, "DeleteMarkerReplicationStatus") + err = msgp.WrapError(err, "ReplicationState") return } - { - var zb0004 string - zb0004, bts, err = msgp.ReadStringBytes(bts) - if err != nil { - err = msgp.WrapError(err, "VersionPurgeStatus") - return - } - z.VersionPurgeStatus = VersionPurgeStatusType(zb0004) - } z.Data, bts, err = msgp.ReadBytesBytes(bts, z.Data) if err != nil { err = msgp.WrapError(err, "Data") @@ -1120,7 +1100,7 @@ func (z *FileInfo) Msgsize() (s int) { for za0003 := range z.Parts { s += z.Parts[za0003].Msgsize() } - s += z.Erasure.Msgsize() + msgp.BoolSize + msgp.StringPrefixSize + len(z.DeleteMarkerReplicationStatus) + msgp.StringPrefixSize + len(string(z.VersionPurgeStatus)) + msgp.BytesPrefixSize + len(z.Data) + msgp.IntSize + msgp.TimeSize + msgp.BoolSize + s += z.Erasure.Msgsize() + msgp.BoolSize + z.ReplicationState.Msgsize() + msgp.BytesPrefixSize + len(z.Data) + msgp.IntSize + msgp.TimeSize + msgp.BoolSize return } @@ -1714,58 +1694,6 @@ func (z *FilesInfoVersions) Msgsize() (s int) { return } -// DecodeMsg implements msgp.Decodable -func (z *VersionPurgeStatusType) DecodeMsg(dc *msgp.Reader) (err error) { - { - var zb0001 string - zb0001, err = dc.ReadString() - if err != nil { - err = msgp.WrapError(err) - return - } - (*z) = VersionPurgeStatusType(zb0001) - } - return -} - -// EncodeMsg implements msgp.Encodable -func (z VersionPurgeStatusType) EncodeMsg(en *msgp.Writer) (err error) { - err = en.WriteString(string(z)) - if err != nil { - err = msgp.WrapError(err) - return - } - return -} - -// MarshalMsg implements msgp.Marshaler -func (z VersionPurgeStatusType) MarshalMsg(b []byte) (o []byte, err error) { - o = msgp.Require(b, z.Msgsize()) - o = msgp.AppendString(o, string(z)) - return -} - -// UnmarshalMsg implements msgp.Unmarshaler -func (z *VersionPurgeStatusType) UnmarshalMsg(bts []byte) (o []byte, err error) { - { - var zb0001 string - zb0001, bts, err = msgp.ReadStringBytes(bts) - if err != nil { - err = msgp.WrapError(err) - return - } - (*z) = VersionPurgeStatusType(zb0001) - } - o = bts - return -} - -// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message -func (z VersionPurgeStatusType) Msgsize() (s int) { - s = msgp.StringPrefixSize + len(string(z)) - return -} - // DecodeMsg implements msgp.Decodable func (z *VolInfo) DecodeMsg(dc *msgp.Reader) (err error) { var zb0001 uint32 diff --git a/cmd/storage-rest-common.go b/cmd/storage-rest-common.go index bb7ae362f..c78dc5da5 100644 --- a/cmd/storage-rest-common.go +++ b/cmd/storage-rest-common.go @@ -18,7 +18,7 @@ package cmd const ( - storageRESTVersion = "v39" // Add FileInfo.Fresh field + storageRESTVersion = "v40" // Add ReplicationState field storageRESTVersionPrefix = SlashSeparator + storageRESTVersion storageRESTPrefix = minioReservedBucketPath + "/storage" ) diff --git a/cmd/xl-storage-format-v2.go b/cmd/xl-storage-format-v2.go index 930c8b53b..4defc9bcb 100644 --- a/cmd/xl-storage-format-v2.go +++ b/cmd/xl-storage-format-v2.go @@ -23,6 +23,7 @@ import ( "errors" "fmt" "io" + "net/http" "sort" "strings" "sync" @@ -31,6 +32,7 @@ import ( "github.com/cespare/xxhash/v2" "github.com/google/uuid" "github.com/minio/minio/internal/bucket/lifecycle" + "github.com/minio/minio/internal/bucket/replication" xhttp "github.com/minio/minio/internal/http" "github.com/minio/minio/internal/logger" "github.com/tinylib/msgp/msgp" @@ -964,14 +966,8 @@ func (j xlMetaV2DeleteMarker) ToFileInfo(volume, path string) (FileInfo, error) VersionID: versionID, Deleted: true, } - for k, v := range j.MetaSys { - switch { - case equals(k, xhttp.AmzBucketReplicationStatus): - fi.DeleteMarkerReplicationStatus = string(v) - case equals(k, VersionPurgeStatusKey): - fi.VersionPurgeStatus = VersionPurgeStatusType(string(v)) - } - } + fi.ReplicationState = GetInternalReplicationState(j.MetaSys) + if j.FreeVersion() { fi.SetTierFreeVersion() fi.TransitionTier = string(j.MetaSys[ReservedMetadataPrefixLower+TransitionTier]) @@ -1050,12 +1046,15 @@ func (j xlMetaV2Object) ToFileInfo(volume, path string) (FileInfo, error) { } for k, v := range j.MetaSys { switch { - case equals(k, VersionPurgeStatusKey): - fi.VersionPurgeStatus = VersionPurgeStatusType(string(v)) - case strings.HasPrefix(strings.ToLower(k), ReservedMetadataPrefixLower): + case strings.HasPrefix(strings.ToLower(k), ReservedMetadataPrefixLower), equals(k, VersionPurgeStatusKey): fi.Metadata[k] = string(v) } } + fi.ReplicationState = getInternalReplicationState(fi.Metadata) + replStatus := fi.ReplicationState.CompositeReplicationStatus() + if replStatus != "" { + fi.Metadata[xhttp.AmzBucketReplicationStatus] = string(replStatus) + } fi.Erasure.Algorithm = j.ErasureAlgorithm.String() fi.Erasure.Index = j.ErasureIndex fi.Erasure.BlockSize = j.ErasureBlockSize @@ -1163,26 +1162,37 @@ func (z *xlMetaV2) DeleteVersion(fi FileInfo) (string, bool, error) { } } updateVersion := false - if fi.VersionPurgeStatus.Empty() && (fi.DeleteMarkerReplicationStatus == "REPLICA" || fi.DeleteMarkerReplicationStatus == "") { + if fi.VersionPurgeStatus().Empty() && (fi.DeleteMarkerReplicationStatus() == "REPLICA" || fi.DeleteMarkerReplicationStatus().Empty()) { updateVersion = fi.MarkDeleted } else { // for replication scenario - if fi.Deleted && fi.VersionPurgeStatus != Complete { - if !fi.VersionPurgeStatus.Empty() || fi.DeleteMarkerReplicationStatus != "" { + if fi.Deleted && fi.VersionPurgeStatus() != Complete { + if !fi.VersionPurgeStatus().Empty() || fi.DeleteMarkerReplicationStatus().Empty() { updateVersion = true } } // object or delete-marker versioned delete is not complete - if !fi.VersionPurgeStatus.Empty() && fi.VersionPurgeStatus != Complete { + if !fi.VersionPurgeStatus().Empty() && fi.VersionPurgeStatus() != Complete { updateVersion = true } } + if fi.Deleted { - if fi.DeleteMarkerReplicationStatus != "" { - ventry.DeleteMarker.MetaSys[xhttp.AmzBucketReplicationStatus] = []byte(fi.DeleteMarkerReplicationStatus) + if !fi.DeleteMarkerReplicationStatus().Empty() { + switch fi.DeleteMarkerReplicationStatus() { + case replication.Replica: + ventry.DeleteMarker.MetaSys[ReservedMetadataPrefixLower+ReplicaStatus] = []byte(string(fi.ReplicationState.ReplicaStatus)) + ventry.DeleteMarker.MetaSys[ReservedMetadataPrefixLower+ReplicaTimestamp] = []byte(fi.ReplicationState.ReplicaTimeStamp.Format(http.TimeFormat)) + default: + ventry.DeleteMarker.MetaSys[ReservedMetadataPrefixLower+ReplicationStatus] = []byte(fi.ReplicationState.ReplicationStatusInternal) + ventry.DeleteMarker.MetaSys[ReservedMetadataPrefixLower+ReplicationTimestamp] = []byte(fi.ReplicationState.ReplicationTimeStamp.Format(http.TimeFormat)) + } } - if !fi.VersionPurgeStatus.Empty() { - ventry.DeleteMarker.MetaSys[VersionPurgeStatusKey] = []byte(fi.VersionPurgeStatus) + if !fi.VersionPurgeStatus().Empty() { + ventry.DeleteMarker.MetaSys[VersionPurgeStatusKey] = []byte(fi.ReplicationState.VersionPurgeStatusInternal) + } + for k, v := range fi.ReplicationState.ResetStatusesMap { + ventry.DeleteMarker.MetaSys[k] = []byte(v) } } @@ -1205,17 +1215,25 @@ func (z *xlMetaV2) DeleteVersion(fi FileInfo) (string, bool, error) { if len(z.Versions[i].DeleteMarker.MetaSys) == 0 { z.Versions[i].DeleteMarker.MetaSys = make(map[string][]byte) } - delete(z.Versions[i].DeleteMarker.MetaSys, xhttp.AmzBucketReplicationStatus) - delete(z.Versions[i].DeleteMarker.MetaSys, VersionPurgeStatusKey) - if fi.DeleteMarkerReplicationStatus != "" { - z.Versions[i].DeleteMarker.MetaSys[xhttp.AmzBucketReplicationStatus] = []byte(fi.DeleteMarkerReplicationStatus) + if !fi.DeleteMarkerReplicationStatus().Empty() { + switch fi.DeleteMarkerReplicationStatus() { + case replication.Replica: + z.Versions[i].DeleteMarker.MetaSys[ReservedMetadataPrefixLower+ReplicaStatus] = []byte(string(fi.ReplicationState.ReplicaStatus)) + z.Versions[i].DeleteMarker.MetaSys[ReservedMetadataPrefixLower+ReplicaTimestamp] = []byte(fi.ReplicationState.ReplicaTimeStamp.Format(http.TimeFormat)) + default: + z.Versions[i].DeleteMarker.MetaSys[ReservedMetadataPrefixLower+ReplicationStatus] = []byte(fi.ReplicationState.ReplicationStatusInternal) + z.Versions[i].DeleteMarker.MetaSys[ReservedMetadataPrefixLower+ReplicationTimestamp] = []byte(fi.ReplicationState.ReplicationTimeStamp.Format(http.TimeFormat)) + } } - if !fi.VersionPurgeStatus.Empty() { - z.Versions[i].DeleteMarker.MetaSys[VersionPurgeStatusKey] = []byte(fi.VersionPurgeStatus) + if !fi.VersionPurgeStatus().Empty() { + z.Versions[i].DeleteMarker.MetaSys[VersionPurgeStatusKey] = []byte(fi.ReplicationState.VersionPurgeStatusInternal) + } + for k, v := range fi.ReplicationState.ResetStatusesMap { + z.Versions[i].DeleteMarker.MetaSys[k] = []byte(v) } } else { z.Versions = append(z.Versions[:i], z.Versions[i+1:]...) - if fi.MarkDeleted && (fi.VersionPurgeStatus.Empty() || (fi.VersionPurgeStatus != Complete)) { + if fi.MarkDeleted && (fi.VersionPurgeStatus().Empty() || (fi.VersionPurgeStatus() != Complete)) { z.Versions = append(z.Versions, ventry) } } @@ -1223,7 +1241,10 @@ func (z *xlMetaV2) DeleteVersion(fi FileInfo) (string, bool, error) { } case ObjectType: if version.ObjectV2.VersionID == uv && updateVersion { - z.Versions[i].ObjectV2.MetaSys[VersionPurgeStatusKey] = []byte(fi.VersionPurgeStatus) + z.Versions[i].ObjectV2.MetaSys[VersionPurgeStatusKey] = []byte(fi.ReplicationState.VersionPurgeStatusInternal) + for k, v := range fi.ReplicationState.ResetStatusesMap { + z.Versions[i].ObjectV2.MetaSys[k] = []byte(v) + } return "", len(z.Versions) == 0, nil } } diff --git a/cmd/xl-storage-format-v2_test.go b/cmd/xl-storage-format-v2_test.go index bf8360011..4b55a4132 100644 --- a/cmd/xl-storage-format-v2_test.go +++ b/cmd/xl-storage-format-v2_test.go @@ -65,12 +65,10 @@ func TestXLV2FormatData(t *testing.T) { Hash: nil, }}, }, - MarkDeleted: false, - DeleteMarkerReplicationStatus: "", - VersionPurgeStatus: "", - Data: data, - NumVersions: 1, - SuccessorModTime: time.Time{}, + MarkDeleted: false, + Data: data, + NumVersions: 1, + SuccessorModTime: time.Time{}, } failOnErr(xl.AddVersion(fi)) @@ -264,12 +262,10 @@ func TestDeleteVersionWithSharedDataDir(t *testing.T) { Hash: nil, }}, }, - MarkDeleted: false, - DeleteMarkerReplicationStatus: "", - VersionPurgeStatus: "", - Data: data, - NumVersions: 1, - SuccessorModTime: time.Time{}, + MarkDeleted: false, + Data: data, + NumVersions: 1, + SuccessorModTime: time.Time{}, } d0, d1, d2 := mustGetUUID(), mustGetUUID(), mustGetUUID() diff --git a/cmd/xl-storage-free-version_test.go b/cmd/xl-storage-free-version_test.go index d049779dd..40e6d1e22 100644 --- a/cmd/xl-storage-free-version_test.go +++ b/cmd/xl-storage-free-version_test.go @@ -70,11 +70,11 @@ func TestFreeVersion(t *testing.T) { Hash: nil, }}, }, - MarkDeleted: false, - DeleteMarkerReplicationStatus: "", - VersionPurgeStatus: "", - NumVersions: 1, - SuccessorModTime: time.Time{}, + MarkDeleted: false, + //DeleteMarkerReplicationStatus: "", + //VersionPurgeStatus: "", + NumVersions: 1, + SuccessorModTime: time.Time{}, } // Add a version with local content xl.AddVersion(fi) diff --git a/cmd/xl-storage.go b/cmd/xl-storage.go index 70a0c055e..6dba949f6 100644 --- a/cmd/xl-storage.go +++ b/cmd/xl-storage.go @@ -445,13 +445,16 @@ func (s *xlStorage) NSScanner(ctx context.Context, cache dataUsageCache, updates // Check if the current bucket has replication configuration if rcfg, err := globalBucketMetadataSys.GetReplicationConfig(ctx, cache.Info.Name); err == nil { if rcfg.HasActiveRules("", true) { - tgt := globalBucketTargetSys.GetRemoteBucketTargetByArn(ctx, cache.Info.Name, rcfg.RoleArn) - cache.Info.replication = replicationConfig{ - Config: rcfg, - ResetID: tgt.ResetID, - ResetBeforeDate: tgt.ResetBeforeDate} - if intDataUpdateTracker.debug { - console.Debugln(color.Green("scannerDisk:") + " replication: Active rules found") + tgts, err := globalBucketTargetSys.ListBucketTargets(ctx, cache.Info.Name) + if err == nil { + cache.Info.replication = replicationConfig{ + Config: rcfg, + remotes: tgts, + } + if intDataUpdateTracker.debug { + console.Debugln(color.Green("scannerDisk:") + " replication: Active rules found") + } + } } } diff --git a/docs/bucket/replication/DESIGN.md b/docs/bucket/replication/DESIGN.md index eb434dead..1ffbf86d0 100644 --- a/docs/bucket/replication/DESIGN.md +++ b/docs/bucket/replication/DESIGN.md @@ -20,6 +20,8 @@ The description above details one way replication from source to target w.r.t in For active-active replication, automatic failover occurs on `GET/HEAD` operations if object or object version requested qualifies for replication and is missing on one site, but present on the other. This allows the applications to take full advantage of two-way replication even before the two sites get fully synced. +In the case of multi destination replication, the replication status shows `COMPLETED` only after the replication operation succeeds on each of the targets specified in the replication configuration. If multiple targets are configured to use active-active replication and multi destination replication, the administrator should ensure that the replication features enabled (such as replica metadata sync, delete marker replication etc) are identical to avoid asymmetric state. This is because all replication activity is inherently a one-way operation from source to target, irrespective of the number of targets. + ### Replication of DeleteMarker and versioned Delete MinIO allows DeleteMarker replication and versioned delete replication by setting `--replicate delete,delete-marker` while setting up replication configuration using `mc replicate add`. The MinIO implementation is based on V2 configuration, however it has been extended to allow both DeleteMarker replication and replication of versioned deletes with the `DeleteMarkerReplication` and `DeleteReplication` fields in the replication configuration. By default, this is set to `Disabled` unless the user specifies it while adding a replication rule. @@ -37,56 +39,87 @@ Note that synchronous replication, i.e. when remote target is configured with -- Existing object replication works similar to regular replication. Objects qualifying for existing object replication are detected when scanner runs, and will be replicated if existing object replication is enabled and applicable replication rules are satisfied. Because replication depends on the immutability of versions, only pre-existing objects created while versioning was enabled can be replicated. Even if replication rules are disabled and re-enabled later, the objects created during the interim will be synced as the scanner queues them. For saving iops, objects qualifying for existing object replication are not marked as `PENDING` prior to replication. +Note that objects with `null` versions, i.e. objects created prior to enabling versioning cannot be replicated as this would break the immutability guarantees provided by versioning. For replicating such objects, `mc cp alias/bucket/object alias/bucket/object` can be performed to create a server side copy of the object as a versioned object - this versioned object will replicate if replication is enabled and the previously present `null` version can then be deleted. + If the remote site is fully lost and objects previously replicated need to be re-synced, the `mc replicate resync` command with optional flag of `--older-than` needs to be used to trigger re-syncing of previously replicated objects. This command generates a ResetID which is a unique UUID saved to the remote target config along with the applicable date(defaults to time of initiating the reset). All objects created prior to this date are eligible for re-replication if existing object replication is enabled for the replication rule the object satisfies. At the time of completion of replication, `X-Minio-Replication-Reset-Status` is set in the metadata with the timestamp of replication and ResetID. For saving iops, the objects which are re-replicated are not first set to `PENDING` state. +### Multi destination replication +The replication design for multiple sites works in a similar manner as described above for two site scenario. However there are some +important exceptions. + +Replication status on the source cluster will be marked as `COMPLETED` only after replication is completed on all targets. If one or more targets failed replication, the replication status is reflected as `PENDING`. + +If 3 or more targets are participating in active-active replication, the replication configuration for replica metadata sync, delete marker replication and delete replication should match to avoid inconsistent picture between the clusters. It is not recommended to turn on asymmetric replication - for e.g. if three sites A,B,C are participating in replication, it would be better to avoid replication setups like A -> [B, C], B -> A. In this particular example, an object uploaded to A will be replicated to B,C. If replica metadata sync is turned on in site B, any metadata updates on a replica version made in B would reflect in A, but not in C. + ### Internal metadata for replication `xl.meta` that is in use for [versioning](https://github.com/minio/minio/blob/master/docs/bucket/versioning/DESIGN.md) has additional metadata for replication of objects,delete markers and versioned deletes. -### Metadata for object replication +### Metadata for object replication - on source ``` ... - "MetaSys": {}, - "MetaUsr": { - "X-Amz-Replication-Status": "COMPLETED", - "content-type": "application/octet-stream", - "etag": "8315e643ed6a5d7c9962fc0a8ef9c11f" - }, - "PartASizes": [ - 26 - ], + "MetaSys": { + "x-minio-internal-inline-data": "dHJ1ZQ==", + "x-minio-internal-replication-status": "YXJuOm1pbmlvOnJlcGxpY2F0aW9uOjo2YjdmYzFlMS0wNmU4LTQxMTUtYjYxNy00YTgzZGIyODhmNTM6YnVja2V0PUNPTVBMRVRFRDthcm46bWluaW86cmVwbGljYXRpb246OmI5MGYxZWEzLWMzYWQtNDEyMy1iYWE2LWZjMDZhYmEyMjA2MjpidWNrZXQ9Q09NUExFVEVEOw==", + "x-minio-internal-replication-timestamp": "MjAyMS0wOS0xN1QwMTo0MzozOC40MDQwMDA0ODNa", + "x-minio-internal-tier-free-versionID": "OWZlZjk5N2QtMjMzZi00N2U3LTlkZmMtNWYxNzc3NzdlZTM2" + }, + "MetaUsr": { + "X-Amz-Replication-Status": "COMPLETED", + "content-type": "application/octet-stream", + "etag": "8315e643ed6a5d7c9962fc0a8ef9c11f" + }, ... ``` +### Metadata for object replication - on target + +``` +... + "MetaSys": { + "x-minio-internal-inline-data": "dHJ1ZQ==", + "x-minio-internal-replica-status": "UkVQTElDQQ==", + "x-minio-internal-replica-timestamp": "MjAyMS0wOS0xN1QwMTo0MzozOC4zODg5ODU4ODRa" + }, + "MetaUsr": { + "X-Amz-Replication-Status": "REPLICA", + "content-type": "application/octet-stream", + "etag": "8315e643ed6a5d7c9962fc0a8ef9c11f", + "x-amz-storage-class": "STANDARD" + }, +... +``` ### Additional replication metadata for DeleteMarker -``` -{ + { "DelObj": { - "ID": "8+jguy20TOuzUCN2PTrESA==", - "MTime": 1613601949645331516, - "MetaSys": { - "X-Amz-Replication-Status": "Q09NUExFVEVE" - } - }, - "Type": 2 - } + "ID": "u8H5pYQFRMKgkIgkpSKIkQ==", + "MTime": 1631843124147668389, + "MetaSys": { + "x-minio-internal-replication-status": "YXJuOm1pbmlvOnJlcGxpY2F0aW9uOjpiOTBmMWVhMy1jM2FkLTQxMjMtYmFhNi1mYzA2YWJhMjIwNjI6YnVja2V0PUNPTVBMRVRFRDthcm46bWluaW86cmVwbGljYXRpb246OjZiN2ZjMWUxLTA2ZTgtNDExNS1iNjE3LTRhODNkYjI4OGY1MzpidWNrZXQ9Q09NUExFVEVEOw==", + "x-minio-internal-replication-timestamp": "U3VuLCAzMSBEZWMgMDAwMCAxOTowMzo1OCBHTVQ=" + } + }, + "Type": 2 +} ``` ### Additional replication metadata for versioned delete ``` -{ - "DelObj": { - "ID": "8+jguy20TOuzUCN2PTrESA==", - "MTime": 1613601949645331516, - "MetaSys": { - "purgestatus": "RkFJTEVE" - } - }, - "Type": 2 - } +{ + "DelObj": { + "ID": "u8H5pYQFRMKgkIgkpSKIkQ==", + "MTime": 1631843124147668389, + "MetaSys": { + "purgestatus": "YXJuOm1pbmlvOnJlcGxpY2F0aW9uOjpiOTBmMWVhMy1jM2FkLTQxMjMtYmFhNi1mYzA2YWJhMjIwNjI6YnVja2V0PUNPTVBMRVRFO2FybjptaW5pbzpyZXBsaWNhdGlvbjo6NmI3ZmMxZTEtMDZlOC00MTE1LWI2MTctNGE4M2RiMjg4ZjUzOmJ1Y2tldD1GQUlMRUQ7", + "x-minio-internal-replication-status": "YXJuOm1pbmlvOnJlcGxpY2F0aW9uOjpiOTBmMWVhMy1jM2FkLTQxMjMtYmFhNi1mYzA2YWJhMjIwNjI6YnVja2V0PTthcm46bWluaW86cmVwbGljYXRpb246OjZiN2ZjMWUxLTA2ZTgtNDExNS1iNjE3LTRhODNkYjI4OGY1MzpidWNrZXQ9Ow==", + "x-minio-internal-replication-timestamp": "U3VuLCAzMSBEZWMgMDAwMCAxOTowMzo1OCBHTVQ=" + } + }, + "Type": 2 +} ``` ## Explore Further diff --git a/docs/bucket/replication/README.md b/docs/bucket/replication/README.md index 626c5dbb6..d8a343d65 100644 --- a/docs/bucket/replication/README.md +++ b/docs/bucket/replication/README.md @@ -11,6 +11,7 @@ To replicate objects in a bucket to a destination bucket on a target site either - Supports object locking/retention across source and destination buckets natively out of the box, unlike AWS S3. - Simpler implementation than [AWS S3 Bucket Replication Config](https://docs.aws.amazon.com/AmazonS3/latest/dev/replication-add-config.html) with requirements such as IAM Role, AccessControlTranslation, Metrics and SourceSelectionCriteria are not needed with MinIO. - Active-Active replication +- Multi destination replication ## How to use? Ensure that versioning is enabled on the source and target buckets with `mc version` command. If object locking is required, the buckets should have been created with `mc mb --with-lock` @@ -19,7 +20,7 @@ Create a replication target on the source cluster as shown below: ``` mc admin bucket remote add myminio/srcbucket https://accessKey:secretKey@replica-endpoint:9000/destbucket --service replication --region us-east-1 -Role ARN = 'arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket' +Remote ARN = 'arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket' ``` > The user running the above command needs *s3:GetReplicationConfiguration* and *s3:GetBucketVersioning* permission on the source cluster. We do not recommend running root credentials/super admin with replication, instead create a dedicated user. The access credentials used at the destination requires *s3:ReplicateObject* permission. @@ -100,14 +101,14 @@ Please note that the permissions required by the admin user on the target cluste Once successfully created and authorized, the `mc admin bucket remote add` command generates a replication target ARN. This command lists all the currently authorized replication targets: ``` mc admin bucket remote ls myminio/srcbucket --service "replication" -Role ARN = 'arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket' +Remote ARN = 'arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket' ``` -The replication configuration can now be added to the source bucket by applying the json file with replication configuration. The Role ARN above is passed in as a json element in the configuration. +The replication configuration can now be added to the source bucket by applying the json file with replication configuration. The Remote ARN above is passed in as a json element in the configuration. ```json { - "Role" :"arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket", + "Role" :"", "Rules": [ { "Status": "Enabled", @@ -130,7 +131,7 @@ The replication configuration can now be added to the source bucket by applying } }, "Destination": { - "Bucket": "arn:aws:s3:::destbucket", + "Bucket": "arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket", "StorageClass": "STANDARD" }, "SourceSelectionCriteria": { @@ -181,7 +182,7 @@ To add a replication rule allowing both delete marker replication, versioned del Additional permission of "s3:ReplicateDelete" action would need to be specified on the access key configured for the target cluster if Delete Marker replication or versioned delete replication is enabled. ``` -mc replicate add myminio/srcbucket/Tax --priority 1 --arn "arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket" --tags "Year=2019&Company=AcmeCorp" --storage-class "STANDARD" --remote-bucket "destbucket" --replicate "delete,delete-marker" +mc replicate add myminio/srcbucket/Tax --priority 1 --remote-bucket "arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket" --tags "Year=2019&Company=AcmeCorp" --storage-class "STANDARD" --replicate "delete,delete-marker" Replication configuration applied successfully to myminio/srcbucket. ``` @@ -215,7 +216,11 @@ This is an expensive operation and should be initiated only once - progress of t Note that ExistingObjectReplication needs to be enabled in the config via `mc replicate [add|edit]` by passing `existing-objects` as one of the values to `--replicate` flag. Only those objects meeting replication rules and having existing object replication enabled will be re-synced. -Multi site replication is currently not supported. +### Multi destination replication + +Replication from a source bucket to multiple destination buckets is supported. For each of the targets,repeat the steps to configure a remote target ARN and add replication rules to the source bucket's replication config. + +Note that on the source side, the `X-Amz-Replication-Status` changes from `PENDING` to `COMPLETED` after replication succeeds to each of the targets. On the destination side, a `X-Amz-Replication-Status` status of `REPLICA` indicates that the object was replicated successfully. Any replication failures are automatically re-attempted during a periodic disk scanner cycle. ## Explore Further - [MinIO Bucket Replication Design](https://github.com/minio/minio/blob/master/docs/bucket/replication/DESIGN.md) diff --git a/go.mod b/go.mod index 8721d14d9..5c9e0a40e 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( go.uber.org/atomic v1.7.0 go.uber.org/zap v1.16.1-0.20210329175301-c23abee72d19 golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e - golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c + golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba golang.org/x/tools v0.1.1 // indirect google.golang.org/api v0.31.0 diff --git a/go.sum b/go.sum index 8eaec5cae..bfd81cf47 100644 --- a/go.sum +++ b/go.sum @@ -1753,8 +1753,9 @@ golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210420072515-93ed5bcd2bfe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c h1:F1jZWGFhYfh0Ci55sIpILtKKK8p3i2/krTr0H1rg74I= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c h1:Lyn7+CqXIiC+LOR9aHD6jDK+hPcmAuCfuXztd1v4w1Q= +golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= diff --git a/internal/bucket/replication/datatypes.go b/internal/bucket/replication/datatypes.go new file mode 100644 index 000000000..6f87606a5 --- /dev/null +++ b/internal/bucket/replication/datatypes.go @@ -0,0 +1,47 @@ +// Copyright (c) 2015-2021 MinIO, Inc. +// +// This file is part of MinIO Object Storage stack +// +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// This program is distributed in the hope that it will be useful +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +package replication + +//go:generate msgp -file=$GOFILE + +// StatusType of Replication for x-amz-replication-status header +type StatusType string + +const ( + // Pending - replication is pending. + Pending StatusType = "PENDING" + + // Completed - replication completed ok. + Completed StatusType = "COMPLETED" + + // Failed - replication failed. + Failed StatusType = "FAILED" + + // Replica - this is a replica. + Replica StatusType = "REPLICA" +) + +// String returns string representation of status +func (s StatusType) String() string { + return string(s) +} + +// Empty returns true if this status is not set +func (s StatusType) Empty() bool { + return string(s) == "" +} diff --git a/internal/bucket/replication/datatypes_gen.go b/internal/bucket/replication/datatypes_gen.go new file mode 100644 index 000000000..73b495cde --- /dev/null +++ b/internal/bucket/replication/datatypes_gen.go @@ -0,0 +1,59 @@ +package replication + +// Code generated by github.com/tinylib/msgp DO NOT EDIT. + +import ( + "github.com/tinylib/msgp/msgp" +) + +// DecodeMsg implements msgp.Decodable +func (z *StatusType) DecodeMsg(dc *msgp.Reader) (err error) { + { + var zb0001 string + zb0001, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err) + return + } + (*z) = StatusType(zb0001) + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z StatusType) EncodeMsg(en *msgp.Writer) (err error) { + err = en.WriteString(string(z)) + if err != nil { + err = msgp.WrapError(err) + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z StatusType) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + o = msgp.AppendString(o, string(z)) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *StatusType) UnmarshalMsg(bts []byte) (o []byte, err error) { + { + var zb0001 string + zb0001, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + (*z) = StatusType(zb0001) + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z StatusType) Msgsize() (s int) { + s = msgp.StringPrefixSize + len(string(z)) + return +} diff --git a/internal/bucket/replication/datatypes_gen_test.go b/internal/bucket/replication/datatypes_gen_test.go new file mode 100644 index 000000000..e3cbaadfa --- /dev/null +++ b/internal/bucket/replication/datatypes_gen_test.go @@ -0,0 +1,3 @@ +package replication + +// Code generated by github.com/tinylib/msgp DO NOT EDIT. diff --git a/internal/bucket/replication/destination.go b/internal/bucket/replication/destination.go index 395a24e51..a106c1b2b 100644 --- a/internal/bucket/replication/destination.go +++ b/internal/bucket/replication/destination.go @@ -28,11 +28,15 @@ import ( // DestinationARNPrefix - destination ARN prefix as per AWS S3 specification. const DestinationARNPrefix = "arn:aws:s3:::" +// DestinationARNMinIOPrefix - destination ARN prefix for MinIO. +const DestinationARNMinIOPrefix = "arn:minio:replication:" + // Destination - destination in ReplicationConfiguration. type Destination struct { XMLName xml.Name `xml:"Destination" json:"Destination"` Bucket string `xml:"Bucket" json:"Bucket"` StorageClass string `xml:"StorageClass" json:"StorageClass"` + ARN string //EncryptionConfiguration TODO: not needed for MinIO } @@ -49,7 +53,20 @@ func (d Destination) IsValid() bool { } func (d Destination) String() string { - return DestinationARNPrefix + d.Bucket + return d.ARN + +} + +//LegacyArn returns true if arn format has prefix "arn:aws:s3:::" which was used +// prior to multi-destination +func (d Destination) LegacyArn() bool { + return strings.HasPrefix(d.ARN, DestinationARNPrefix) +} + +//TargetArn returns true if arn format has prefix "arn:minio:replication:::" used +// for multi-destination targets +func (d Destination) TargetArn() bool { + return strings.HasPrefix(d.ARN, DestinationARNMinIOPrefix) } // MarshalXML - encodes to XML data. @@ -107,7 +124,7 @@ func (d Destination) Validate(bucketName string) error { // parseDestination - parses string to Destination. func parseDestination(s string) (Destination, error) { - if !strings.HasPrefix(s, DestinationARNPrefix) { + if !strings.HasPrefix(s, DestinationARNPrefix) && !strings.HasPrefix(s, DestinationARNMinIOPrefix) { return Destination{}, Errorf("invalid destination '%s'", s) } @@ -115,5 +132,6 @@ func parseDestination(s string) (Destination, error) { return Destination{ Bucket: bucketName, + ARN: s, }, nil } diff --git a/internal/bucket/replication/replication.go b/internal/bucket/replication/replication.go index e221e7de1..8d81405b2 100644 --- a/internal/bucket/replication/replication.go +++ b/internal/bucket/replication/replication.go @@ -25,40 +25,14 @@ import ( "strings" ) -// StatusType of Replication for x-amz-replication-status header -type StatusType string - -const ( - // Pending - replication is pending. - Pending StatusType = "PENDING" - - // Completed - replication completed ok. - Completed StatusType = "COMPLETED" - - // Failed - replication failed. - Failed StatusType = "FAILED" - - // Replica - this is a replica. - Replica StatusType = "REPLICA" -) - -// String returns string representation of status -func (s StatusType) String() string { - return string(s) -} - -// Empty returns true if this status is not set -func (s StatusType) Empty() bool { - return string(s) == "" -} - var ( - errReplicationTooManyRules = Errorf("Replication configuration allows a maximum of 1000 rules") - errReplicationNoRule = Errorf("Replication configuration should have at least one rule") - errReplicationUniquePriority = Errorf("Replication configuration has duplicate priority") - errReplicationDestinationMismatch = Errorf("The destination bucket must be same for all rules") - errRoleArnMissing = Errorf("Missing required parameter `Role` in ReplicationConfiguration") - errInvalidSourceSelectionCriteria = Errorf("Invalid ReplicaModification status") + errReplicationTooManyRules = Errorf("Replication configuration allows a maximum of 1000 rules") + errReplicationNoRule = Errorf("Replication configuration should have at least one rule") + errReplicationUniquePriority = Errorf("Replication configuration has duplicate priority") + errRoleArnMissingLegacy = Errorf("Missing required parameter `Role` in ReplicationConfiguration") + errDestinationArnMissing = Errorf("Missing required parameter `Destination` in Replication rule") + errInvalidSourceSelectionCriteria = Errorf("Invalid ReplicaModification status") + errRoleArnPresentForMultipleTargets = Errorf("`Role` should be empty in ReplicationConfiguration for multiple targets") ) // Config - replication configuration specified in @@ -102,18 +76,14 @@ func (c Config) Validate(bucket string, sameTarget bool) error { if len(c.Rules) == 0 { return errReplicationNoRule } - if c.RoleArn == "" { - return errRoleArnMissing - } + // Validate all the rules in the replication config targetMap := make(map[string]struct{}) priorityMap := make(map[string]struct{}) + var legacyArn bool for _, r := range c.Rules { - if len(targetMap) == 0 { - targetMap[r.Destination.Bucket] = struct{}{} - } if _, ok := targetMap[r.Destination.Bucket]; !ok { - return errReplicationDestinationMismatch + targetMap[r.Destination.Bucket] = struct{}{} } if err := r.Validate(bucket, sameTarget); err != nil { return err @@ -122,6 +92,22 @@ func (c Config) Validate(bucket string, sameTarget bool) error { return errReplicationUniquePriority } priorityMap[strconv.Itoa(r.Priority)] = struct{}{} + + if r.Destination.LegacyArn() { + legacyArn = true + } + if c.RoleArn == "" && !r.Destination.TargetArn() { + return errDestinationArnMissing + } + } + // disallow combining old replication configuration which used RoleArn as target ARN with multiple + // destination replication + if c.RoleArn != "" && len(targetMap) > 1 { + return errRoleArnPresentForMultipleTargets + } + // validate RoleArn if destination used legacy ARN format. + if c.RoleArn == "" && legacyArn { + return errRoleArnMissingLegacy } return nil } @@ -137,6 +123,7 @@ const ( MetadataReplicationType HealReplicationType ExistingObjectReplicationType + ResyncReplicationType ) // Valid returns true if replication type is set @@ -150,18 +137,18 @@ type ObjectOpts struct { Name string UserTags string VersionID string - IsLatest bool DeleteMarker bool SSEC bool OpType Type Replica bool ExistingObject bool + TargetArn string } // FilterActionableRules returns the rules actions that need to be executed // after evaluating prefix/tag filtering func (c Config) FilterActionableRules(obj ObjectOpts) []Rule { - if obj.Name == "" { + if obj.Name == "" && obj.OpType != ResyncReplicationType { return nil } var rules []Rule @@ -169,6 +156,18 @@ func (c Config) FilterActionableRules(obj ObjectOpts) []Rule { if rule.Status == Disabled { continue } + + if obj.TargetArn != "" && rule.Destination.ARN != obj.TargetArn && c.RoleArn != obj.TargetArn { + continue + } + // Ignore other object level and prefix filters for resyncing target + if obj.OpType == ResyncReplicationType { + rules = append(rules, rule) + continue + } + if obj.ExistingObject && rule.ExistingObjectReplication.Status == Disabled { + continue + } if !strings.HasPrefix(obj.Name, rule.Prefix()) { continue } @@ -177,8 +176,9 @@ func (c Config) FilterActionableRules(obj ObjectOpts) []Rule { } } sort.Slice(rules[:], func(i, j int) bool { - return rules[i].Priority > rules[j].Priority + return rules[i].Priority > rules[j].Priority && rules[i].Destination.String() == rules[j].Destination.String() }) + return rules } @@ -205,7 +205,7 @@ func (c Config) Replicate(obj ObjectOpts) bool { if obj.OpType == DeleteReplicationType { switch { case obj.VersionID != "": - // // check MinIO extension for versioned deletes + // check MinIO extension for versioned deletes return rule.DeleteReplication.Status == Enabled default: return rule.DeleteMarkerReplication.Status == Enabled @@ -243,3 +243,27 @@ func (c Config) HasActiveRules(prefix string, recursive bool) bool { } return false } + +// FilterTargetArns returns a slice of distinct target arns in the config +func (c Config) FilterTargetArns(obj ObjectOpts) []string { + var arns []string + + tgtsMap := make(map[string]struct{}) + rules := c.FilterActionableRules(obj) + for _, rule := range rules { + if rule.Status == Disabled { + continue + } + if c.RoleArn != "" { + arns = append(arns, c.RoleArn) // use legacy RoleArn if present + return arns + } + if _, ok := tgtsMap[rule.Destination.ARN]; !ok { + tgtsMap[rule.Destination.ARN] = struct{}{} + } + } + for k := range tgtsMap { + arns = append(arns, k) + } + return arns +} diff --git a/internal/bucket/replication/replication_test.go b/internal/bucket/replication/replication_test.go index 13a79e747..19ba309ef 100644 --- a/internal/bucket/replication/replication_test.go +++ b/internal/bucket/replication/replication_test.go @@ -52,20 +52,20 @@ func TestParseAndValidateReplicationConfig(t *testing.T) { expectedParsingErr: nil, expectedValidationErr: nil, }, - //4 missing role in config + //4 missing role in config and destination ARN is in legacy format {inputConfig: `EnabledDisabledDisabledkey-prefixarn:aws:s3:::destinationbucket`, // destination bucket in config different from bucket specified destBucket: "destinationbucket", sameTarget: false, expectedParsingErr: nil, - expectedValidationErr: errRoleArnMissing, + expectedValidationErr: errDestinationArnMissing, }, //5 replication destination in different rules not identical - {inputConfig: `arn:aws:iam::AcctID:role/role-nameEnabledDisabledDisabledkey-prefixarn:aws:s3:::destinationbucketEnabled3DisabledDisabledkey-prefixarn:aws:s3:::destinationbucket2`, + {inputConfig: `EnabledDisabledDisabledkey-prefixarn:minio:replication:::destinationbucketEnabled3DisabledDisabledkey-prefixarn:minio:replication:::destinationbucket2`, destBucket: "destinationbucket", sameTarget: false, expectedParsingErr: nil, - expectedValidationErr: errReplicationDestinationMismatch, + expectedValidationErr: nil, }, //6 missing rule status in replication config {inputConfig: `arn:aws:iam::AcctID:role/role-nameDisabledDisabledkey-prefixarn:aws:s3:::destinationbucket`, @@ -116,6 +116,22 @@ func TestParseAndValidateReplicationConfig(t *testing.T) { expectedParsingErr: fmt.Errorf("invalid destination '%v'", "destinationbucket2"), expectedValidationErr: nil, }, + //13 missing role in config and destination ARN has target ARN + {inputConfig: `EnabledDisabledDisabledkey-prefixarn:minio:replication::8320b6d18f9032b4700f1f03b50d8d1853de8f22cab86931ee794e12f190852c:destinationbucket`, + // destination bucket in config different from bucket specified + destBucket: "destinationbucket", + sameTarget: false, + expectedParsingErr: nil, + expectedValidationErr: nil, + }, + //14 role absent in config and destination ARN has target ARN in invalid format + {inputConfig: `EnabledDisabledDisabledkey-prefixarn:xx:replication::8320b6d18f9032b4700f1f03b50d8d1853de8f22cab86931ee794e12f190852c:destinationbucket`, + // destination bucket in config different from bucket specified + destBucket: "destinationbucket", + sameTarget: false, + expectedParsingErr: fmt.Errorf("invalid destination '%v'", "arn:xx:replication::8320b6d18f9032b4700f1f03b50d8d1853de8f22cab86931ee794e12f190852c:destinationbucket"), + expectedValidationErr: nil, + }, } for i, tc := range testCases { t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) { @@ -333,3 +349,52 @@ func TestHasActiveRules(t *testing.T) { } } + +func TestFilterActionableRules(t *testing.T) { + testCases := []struct { + inputConfig string + prefix string + ExpectedRules []Rule + }{ + // case 1 - only one rule + {inputConfig: `arn:aws:iam::AcctID:role/role-nameEnabledDisabledDisabledprefix1arn:minio:replication:xxx::destinationbucket`, + prefix: "prefix", + ExpectedRules: []Rule{{Status: Enabled, Priority: 1, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}}}, + }, + // case 2 - multiple rules for same target, overlapping rules with different priority + {inputConfig: `arn:aws:iam::AcctID:role/role-nameEnabledDisabledDisabledprefix3arn:minio:replication:xxx::destinationbucketEnabledDisabledDisabledprefix1arn:minio:replication:xxx::destinationbucket`, + prefix: "prefix", + ExpectedRules: []Rule{ + {Status: Enabled, Priority: 3, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}}, + {Status: Enabled, Priority: 1, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}}, + }, + }, + // case 3 - multiple rules for different target, overlapping rules on a target + {inputConfig: `arn:aws:iam::AcctID:role/role-nameEnabledDisabledDisabledprefix2arn:minio:replication:xxx::destinationbucket2EnabledDisabledDisabledprefix4arn:minio:replication:xxx::destinationbucket2EnabledDisabledDisabledprefix3arn:minio:replication:xxx::destinationbucketEnabledDisabledDisabledprefix1arn:minio:replication:xxx::destinationbucket`, + prefix: "prefix", + ExpectedRules: []Rule{ + {Status: Enabled, Priority: 4, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket2", ARN: "arn:minio:replication:xxx::destinationbucket2"}}, + {Status: Enabled, Priority: 2, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket2", ARN: "arn:minio:replication:xxx::destinationbucket2"}}, + {Status: Enabled, Priority: 3, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}}, + {Status: Enabled, Priority: 1, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}}, + }, + }, + } + for _, tc := range testCases { + tc := tc + cfg, err := ParseConfig(bytes.NewReader([]byte(tc.inputConfig))) + if err != nil { + t.Fatalf("Got unexpected error: %v", err) + } + got := cfg.FilterActionableRules(ObjectOpts{Name: tc.prefix}) + if len(got) != len(tc.ExpectedRules) { + t.Fatalf("Expected matching number of actionable rules: `%v`, got: `%v`", tc.ExpectedRules, got) + + } + for i := range got { + if got[i].Destination.ARN != tc.ExpectedRules[i].Destination.ARN || got[i].Priority != tc.ExpectedRules[i].Priority { + t.Fatalf("Expected order of filtered rules to be identical: `%v`, got: `%v`", tc.ExpectedRules, got) + } + } + } +} diff --git a/internal/http/headers.go b/internal/http/headers.go index 2d5761943..6bec1e7f6 100644 --- a/internal/http/headers.go +++ b/internal/http/headers.go @@ -173,6 +173,12 @@ const ( // Header indicates replication reset status. MinIOReplicationResetStatus = "X-Minio-Replication-Reset-Status" + // Header indiicates last tag update time on source + MinIOSourceTaggingTimestamp = "X-Minio-Source-Replication-Tagging-Timestamp" + // Header indiicates last rtention update time on source + MinIOSourceObjectRetentionTimestamp = "X-Minio-Source-Replication-Retention-Timestamp" + // Header indiicates last rtention update time on source + MinIOSourceObjectLegalHoldTimestamp = "X-Minio-Source-Replication-LegalHold-Timestamp" // predicted date/time of transition MinIOTransition = "X-Minio-Transition" )