mirror of
https://github.com/minio/minio.git
synced 2025-11-09 21:49:46 -05:00
Add support for multi site replication (#12880)
This commit is contained in:
committed by
GitHub
parent
0b8c5a6872
commit
c4373ef290
@@ -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
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
602
cmd/bucket-replication-utils.go
Normal file
602
cmd/bucket-replication-utils.go
Normal file
@@ -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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
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
|
||||
}
|
||||
1285
cmd/bucket-replication-utils_gen.go
Normal file
1285
cmd/bucket-replication-utils_gen.go
Normal file
File diff suppressed because it is too large
Load Diff
688
cmd/bucket-replication-utils_gen_test.go
Normal file
688
cmd/bucket-replication-utils_gen_test.go
Normal file
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
247
cmd/bucket-replication-utils_test.go
Normal file
247
cmd/bucket-replication-utils_test.go
Normal file
@@ -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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
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)
|
||||
}
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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)
|
||||
|
||||
87
cmd/data-usage-utils.go
Normal file
87
cmd/data-usage-utils.go
Normal file
@@ -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 <http://www.gnu.org/licenses/>.
|
||||
|
||||
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"`
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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{}
|
||||
}
|
||||
|
||||
@@ -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{
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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 != "" {
|
||||
|
||||
@@ -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{
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -18,7 +18,7 @@
|
||||
package cmd
|
||||
|
||||
const (
|
||||
storageRESTVersion = "v39" // Add FileInfo.Fresh field
|
||||
storageRESTVersion = "v40" // Add ReplicationState field
|
||||
storageRESTVersionPrefix = SlashSeparator + storageRESTVersion
|
||||
storageRESTPrefix = minioReservedBucketPath + "/storage"
|
||||
)
|
||||
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user