Add support for multi site replication (#12880)

This commit is contained in:
Poorna Krishnamoorthy 2021-09-18 16:31:35 -04:00 committed by GitHub
parent 0b8c5a6872
commit c4373ef290
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
52 changed files with 6492 additions and 1230 deletions

View File

@ -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

View File

@ -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.

View File

@ -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

View File

@ -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 {

View File

@ -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
}

View 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
}

File diff suppressed because it is too large Load Diff

View 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)
}
}
}

View 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

View File

@ -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)
}
}
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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)

View File

@ -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
}

View File

@ -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)
}

View File

@ -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

View File

@ -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
}

View File

@ -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
View 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"`
}

View File

@ -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
}

View File

@ -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

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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

View File

@ -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{}
}

View File

@ -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{

View File

@ -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
}

View File

@ -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

View File

@ -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)

View File

@ -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
}

View File

@ -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.

View File

@ -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 != "" {

View File

@ -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{

View File

@ -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

View File

@ -18,7 +18,7 @@
package cmd
const (
storageRESTVersion = "v39" // Add FileInfo.Fresh field
storageRESTVersion = "v40" // Add ReplicationState field
storageRESTVersionPrefix = SlashSeparator + storageRESTVersion
storageRESTPrefix = minioReservedBucketPath + "/storage"
)

View File

@ -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
}
}

View File

@ -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()

View File

@ -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)

View File

@ -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")
}
}
}
}

View File

@ -20,6 +20,8 @@ The description above details one way replication from source to target w.r.t in
For active-active replication, automatic failover occurs on `GET/HEAD` operations if object or object version requested qualifies for replication and is missing on one site, but present on the other. This allows the applications to take full advantage of two-way replication even before the two sites get fully synced.
In the case of multi destination replication, the replication status shows `COMPLETED` only after the replication operation succeeds on each of the targets specified in the replication configuration. If multiple targets are configured to use active-active replication and multi destination replication, the administrator should ensure that the replication features enabled (such as replica metadata sync, delete marker replication etc) are identical to avoid asymmetric state. This is because all replication activity is inherently a one-way operation from source to target, irrespective of the number of targets.
### Replication of DeleteMarker and versioned Delete
MinIO allows DeleteMarker replication and versioned delete replication by setting `--replicate delete,delete-marker` while setting up replication configuration using `mc replicate add`. The MinIO implementation is based on V2 configuration, however it has been extended to allow both DeleteMarker replication and replication of versioned deletes with the `DeleteMarkerReplication` and `DeleteReplication` fields in the replication configuration. By default, this is set to `Disabled` unless the user specifies it while adding a replication rule.
@ -37,56 +39,87 @@ Note that synchronous replication, i.e. when remote target is configured with --
Existing object replication works similar to regular replication. Objects qualifying for existing object replication are detected when scanner runs, and will be replicated if existing object replication is enabled and applicable replication rules are satisfied. Because replication depends on the immutability of versions, only pre-existing objects created while versioning was enabled can be replicated. Even if replication rules are disabled and re-enabled later, the objects created during the interim will be synced as the scanner queues them. For saving iops, objects qualifying for
existing object replication are not marked as `PENDING` prior to replication.
Note that objects with `null` versions, i.e. objects created prior to enabling versioning cannot be replicated as this would break the immutability guarantees provided by versioning. For replicating such objects, `mc cp alias/bucket/object alias/bucket/object` can be performed to create a server side copy of the object as a versioned object - this versioned object will replicate if replication is enabled and the previously present `null` version can then be deleted.
If the remote site is fully lost and objects previously replicated need to be re-synced, the `mc replicate resync` command with optional flag of `--older-than` needs to be used to trigger re-syncing of previously replicated objects. This command generates a ResetID which is a unique UUID saved to the remote target config along with the applicable date(defaults to time of initiating the reset). All objects created prior to this date are eligible for re-replication if existing object replication is enabled for the replication rule the object satisfies. At the time of completion of replication, `X-Minio-Replication-Reset-Status` is set in the metadata with the timestamp of replication and ResetID. For saving iops, the objects which are re-replicated are not first set to `PENDING` state.
### Multi destination replication
The replication design for multiple sites works in a similar manner as described above for two site scenario. However there are some
important exceptions.
Replication status on the source cluster will be marked as `COMPLETED` only after replication is completed on all targets. If one or more targets failed replication, the replication status is reflected as `PENDING`.
If 3 or more targets are participating in active-active replication, the replication configuration for replica metadata sync, delete marker replication and delete replication should match to avoid inconsistent picture between the clusters. It is not recommended to turn on asymmetric replication - for e.g. if three sites A,B,C are participating in replication, it would be better to avoid replication setups like A -> [B, C], B -> A. In this particular example, an object uploaded to A will be replicated to B,C. If replica metadata sync is turned on in site B, any metadata updates on a replica version made in B would reflect in A, but not in C.
### Internal metadata for replication
`xl.meta` that is in use for [versioning](https://github.com/minio/minio/blob/master/docs/bucket/versioning/DESIGN.md) has additional metadata for replication of objects,delete markers and versioned deletes.
### Metadata for object replication
### Metadata for object replication - on source
```
...
"MetaSys": {},
"MetaUsr": {
"X-Amz-Replication-Status": "COMPLETED",
"content-type": "application/octet-stream",
"etag": "8315e643ed6a5d7c9962fc0a8ef9c11f"
},
"PartASizes": [
26
],
"MetaSys": {
"x-minio-internal-inline-data": "dHJ1ZQ==",
"x-minio-internal-replication-status": "YXJuOm1pbmlvOnJlcGxpY2F0aW9uOjo2YjdmYzFlMS0wNmU4LTQxMTUtYjYxNy00YTgzZGIyODhmNTM6YnVja2V0PUNPTVBMRVRFRDthcm46bWluaW86cmVwbGljYXRpb246OmI5MGYxZWEzLWMzYWQtNDEyMy1iYWE2LWZjMDZhYmEyMjA2MjpidWNrZXQ9Q09NUExFVEVEOw==",
"x-minio-internal-replication-timestamp": "MjAyMS0wOS0xN1QwMTo0MzozOC40MDQwMDA0ODNa",
"x-minio-internal-tier-free-versionID": "OWZlZjk5N2QtMjMzZi00N2U3LTlkZmMtNWYxNzc3NzdlZTM2"
},
"MetaUsr": {
"X-Amz-Replication-Status": "COMPLETED",
"content-type": "application/octet-stream",
"etag": "8315e643ed6a5d7c9962fc0a8ef9c11f"
},
...
```
### Metadata for object replication - on target
```
...
"MetaSys": {
"x-minio-internal-inline-data": "dHJ1ZQ==",
"x-minio-internal-replica-status": "UkVQTElDQQ==",
"x-minio-internal-replica-timestamp": "MjAyMS0wOS0xN1QwMTo0MzozOC4zODg5ODU4ODRa"
},
"MetaUsr": {
"X-Amz-Replication-Status": "REPLICA",
"content-type": "application/octet-stream",
"etag": "8315e643ed6a5d7c9962fc0a8ef9c11f",
"x-amz-storage-class": "STANDARD"
},
...
```
### Additional replication metadata for DeleteMarker
```
{
{
"DelObj": {
"ID": "8+jguy20TOuzUCN2PTrESA==",
"MTime": 1613601949645331516,
"MetaSys": {
"X-Amz-Replication-Status": "Q09NUExFVEVE"
}
},
"Type": 2
}
"ID": "u8H5pYQFRMKgkIgkpSKIkQ==",
"MTime": 1631843124147668389,
"MetaSys": {
"x-minio-internal-replication-status": "YXJuOm1pbmlvOnJlcGxpY2F0aW9uOjpiOTBmMWVhMy1jM2FkLTQxMjMtYmFhNi1mYzA2YWJhMjIwNjI6YnVja2V0PUNPTVBMRVRFRDthcm46bWluaW86cmVwbGljYXRpb246OjZiN2ZjMWUxLTA2ZTgtNDExNS1iNjE3LTRhODNkYjI4OGY1MzpidWNrZXQ9Q09NUExFVEVEOw==",
"x-minio-internal-replication-timestamp": "U3VuLCAzMSBEZWMgMDAwMCAxOTowMzo1OCBHTVQ="
}
},
"Type": 2
}
```
### Additional replication metadata for versioned delete
```
{
"DelObj": {
"ID": "8+jguy20TOuzUCN2PTrESA==",
"MTime": 1613601949645331516,
"MetaSys": {
"purgestatus": "RkFJTEVE"
}
},
"Type": 2
}
{
"DelObj": {
"ID": "u8H5pYQFRMKgkIgkpSKIkQ==",
"MTime": 1631843124147668389,
"MetaSys": {
"purgestatus": "YXJuOm1pbmlvOnJlcGxpY2F0aW9uOjpiOTBmMWVhMy1jM2FkLTQxMjMtYmFhNi1mYzA2YWJhMjIwNjI6YnVja2V0PUNPTVBMRVRFO2FybjptaW5pbzpyZXBsaWNhdGlvbjo6NmI3ZmMxZTEtMDZlOC00MTE1LWI2MTctNGE4M2RiMjg4ZjUzOmJ1Y2tldD1GQUlMRUQ7",
"x-minio-internal-replication-status": "YXJuOm1pbmlvOnJlcGxpY2F0aW9uOjpiOTBmMWVhMy1jM2FkLTQxMjMtYmFhNi1mYzA2YWJhMjIwNjI6YnVja2V0PTthcm46bWluaW86cmVwbGljYXRpb246OjZiN2ZjMWUxLTA2ZTgtNDExNS1iNjE3LTRhODNkYjI4OGY1MzpidWNrZXQ9Ow==",
"x-minio-internal-replication-timestamp": "U3VuLCAzMSBEZWMgMDAwMCAxOTowMzo1OCBHTVQ="
}
},
"Type": 2
}
```
## Explore Further

View File

@ -11,6 +11,7 @@ To replicate objects in a bucket to a destination bucket on a target site either
- Supports object locking/retention across source and destination buckets natively out of the box, unlike AWS S3.
- Simpler implementation than [AWS S3 Bucket Replication Config](https://docs.aws.amazon.com/AmazonS3/latest/dev/replication-add-config.html) with requirements such as IAM Role, AccessControlTranslation, Metrics and SourceSelectionCriteria are not needed with MinIO.
- Active-Active replication
- Multi destination replication
## How to use?
Ensure that versioning is enabled on the source and target buckets with `mc version` command. If object locking is required, the buckets should have been created with `mc mb --with-lock`
@ -19,7 +20,7 @@ Create a replication target on the source cluster as shown below:
```
mc admin bucket remote add myminio/srcbucket https://accessKey:secretKey@replica-endpoint:9000/destbucket --service replication --region us-east-1
Role ARN = 'arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket'
Remote ARN = 'arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket'
```
> The user running the above command needs *s3:GetReplicationConfiguration* and *s3:GetBucketVersioning* permission on the source cluster. We do not recommend running root credentials/super admin with replication, instead create a dedicated user. The access credentials used at the destination requires *s3:ReplicateObject* permission.
@ -100,14 +101,14 @@ Please note that the permissions required by the admin user on the target cluste
Once successfully created and authorized, the `mc admin bucket remote add` command generates a replication target ARN. This command lists all the currently authorized replication targets:
```
mc admin bucket remote ls myminio/srcbucket --service "replication"
Role ARN = 'arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket'
Remote ARN = 'arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket'
```
The replication configuration can now be added to the source bucket by applying the json file with replication configuration. The Role ARN above is passed in as a json element in the configuration.
The replication configuration can now be added to the source bucket by applying the json file with replication configuration. The Remote ARN above is passed in as a json element in the configuration.
```json
{
"Role" :"arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket",
"Role" :"",
"Rules": [
{
"Status": "Enabled",
@ -130,7 +131,7 @@ The replication configuration can now be added to the source bucket by applying
}
},
"Destination": {
"Bucket": "arn:aws:s3:::destbucket",
"Bucket": "arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket",
"StorageClass": "STANDARD"
},
"SourceSelectionCriteria": {
@ -181,7 +182,7 @@ To add a replication rule allowing both delete marker replication, versioned del
Additional permission of "s3:ReplicateDelete" action would need to be specified on the access key configured for the target cluster if Delete Marker replication or versioned delete replication is enabled.
```
mc replicate add myminio/srcbucket/Tax --priority 1 --arn "arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket" --tags "Year=2019&Company=AcmeCorp" --storage-class "STANDARD" --remote-bucket "destbucket" --replicate "delete,delete-marker"
mc replicate add myminio/srcbucket/Tax --priority 1 --remote-bucket "arn:minio:replication:us-east-1:c5be6b16-769d-432a-9ef1-4567081f3566:destbucket" --tags "Year=2019&Company=AcmeCorp" --storage-class "STANDARD" --replicate "delete,delete-marker"
Replication configuration applied successfully to myminio/srcbucket.
```
@ -215,7 +216,11 @@ This is an expensive operation and should be initiated only once - progress of t
Note that ExistingObjectReplication needs to be enabled in the config via `mc replicate [add|edit]` by passing `existing-objects` as one of the values to `--replicate` flag. Only those objects meeting replication rules and having existing object replication enabled will be re-synced.
Multi site replication is currently not supported.
### Multi destination replication
Replication from a source bucket to multiple destination buckets is supported. For each of the targets,repeat the steps to configure a remote target ARN and add replication rules to the source bucket's replication config.
Note that on the source side, the `X-Amz-Replication-Status` changes from `PENDING` to `COMPLETED` after replication succeeds to each of the targets. On the destination side, a `X-Amz-Replication-Status` status of `REPLICA` indicates that the object was replicated successfully. Any replication failures are automatically re-attempted during a periodic disk scanner cycle.
## Explore Further
- [MinIO Bucket Replication Design](https://github.com/minio/minio/blob/master/docs/bucket/replication/DESIGN.md)

2
go.mod
View File

@ -82,7 +82,7 @@ require (
go.uber.org/atomic v1.7.0
go.uber.org/zap v1.16.1-0.20210329175301-c23abee72d19
golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e
golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c
golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c
golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba
golang.org/x/tools v0.1.1 // indirect
google.golang.org/api v0.31.0

3
go.sum
View File

@ -1753,8 +1753,9 @@ golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20210420072515-93ed5bcd2bfe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c h1:F1jZWGFhYfh0Ci55sIpILtKKK8p3i2/krTr0H1rg74I=
golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c h1:Lyn7+CqXIiC+LOR9aHD6jDK+hPcmAuCfuXztd1v4w1Q=
golang.org/x/sys v0.0.0-20210819135213-f52c844e1c1c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw=
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=

View File

@ -0,0 +1,47 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package replication
//go:generate msgp -file=$GOFILE
// StatusType of Replication for x-amz-replication-status header
type StatusType string
const (
// Pending - replication is pending.
Pending StatusType = "PENDING"
// Completed - replication completed ok.
Completed StatusType = "COMPLETED"
// Failed - replication failed.
Failed StatusType = "FAILED"
// Replica - this is a replica.
Replica StatusType = "REPLICA"
)
// String returns string representation of status
func (s StatusType) String() string {
return string(s)
}
// Empty returns true if this status is not set
func (s StatusType) Empty() bool {
return string(s) == ""
}

View File

@ -0,0 +1,59 @@
package replication
// Code generated by github.com/tinylib/msgp DO NOT EDIT.
import (
"github.com/tinylib/msgp/msgp"
)
// DecodeMsg implements msgp.Decodable
func (z *StatusType) DecodeMsg(dc *msgp.Reader) (err error) {
{
var zb0001 string
zb0001, err = dc.ReadString()
if err != nil {
err = msgp.WrapError(err)
return
}
(*z) = StatusType(zb0001)
}
return
}
// EncodeMsg implements msgp.Encodable
func (z StatusType) EncodeMsg(en *msgp.Writer) (err error) {
err = en.WriteString(string(z))
if err != nil {
err = msgp.WrapError(err)
return
}
return
}
// MarshalMsg implements msgp.Marshaler
func (z StatusType) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.Require(b, z.Msgsize())
o = msgp.AppendString(o, string(z))
return
}
// UnmarshalMsg implements msgp.Unmarshaler
func (z *StatusType) UnmarshalMsg(bts []byte) (o []byte, err error) {
{
var zb0001 string
zb0001, bts, err = msgp.ReadStringBytes(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
(*z) = StatusType(zb0001)
}
o = bts
return
}
// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message
func (z StatusType) Msgsize() (s int) {
s = msgp.StringPrefixSize + len(string(z))
return
}

View File

@ -0,0 +1,3 @@
package replication
// Code generated by github.com/tinylib/msgp DO NOT EDIT.

View File

@ -28,11 +28,15 @@ import (
// DestinationARNPrefix - destination ARN prefix as per AWS S3 specification.
const DestinationARNPrefix = "arn:aws:s3:::"
// DestinationARNMinIOPrefix - destination ARN prefix for MinIO.
const DestinationARNMinIOPrefix = "arn:minio:replication:"
// Destination - destination in ReplicationConfiguration.
type Destination struct {
XMLName xml.Name `xml:"Destination" json:"Destination"`
Bucket string `xml:"Bucket" json:"Bucket"`
StorageClass string `xml:"StorageClass" json:"StorageClass"`
ARN string
//EncryptionConfiguration TODO: not needed for MinIO
}
@ -49,7 +53,20 @@ func (d Destination) IsValid() bool {
}
func (d Destination) String() string {
return DestinationARNPrefix + d.Bucket
return d.ARN
}
//LegacyArn returns true if arn format has prefix "arn:aws:s3:::" which was used
// prior to multi-destination
func (d Destination) LegacyArn() bool {
return strings.HasPrefix(d.ARN, DestinationARNPrefix)
}
//TargetArn returns true if arn format has prefix "arn:minio:replication:::" used
// for multi-destination targets
func (d Destination) TargetArn() bool {
return strings.HasPrefix(d.ARN, DestinationARNMinIOPrefix)
}
// MarshalXML - encodes to XML data.
@ -107,7 +124,7 @@ func (d Destination) Validate(bucketName string) error {
// parseDestination - parses string to Destination.
func parseDestination(s string) (Destination, error) {
if !strings.HasPrefix(s, DestinationARNPrefix) {
if !strings.HasPrefix(s, DestinationARNPrefix) && !strings.HasPrefix(s, DestinationARNMinIOPrefix) {
return Destination{}, Errorf("invalid destination '%s'", s)
}
@ -115,5 +132,6 @@ func parseDestination(s string) (Destination, error) {
return Destination{
Bucket: bucketName,
ARN: s,
}, nil
}

View File

@ -25,40 +25,14 @@ import (
"strings"
)
// StatusType of Replication for x-amz-replication-status header
type StatusType string
const (
// Pending - replication is pending.
Pending StatusType = "PENDING"
// Completed - replication completed ok.
Completed StatusType = "COMPLETED"
// Failed - replication failed.
Failed StatusType = "FAILED"
// Replica - this is a replica.
Replica StatusType = "REPLICA"
)
// String returns string representation of status
func (s StatusType) String() string {
return string(s)
}
// Empty returns true if this status is not set
func (s StatusType) Empty() bool {
return string(s) == ""
}
var (
errReplicationTooManyRules = Errorf("Replication configuration allows a maximum of 1000 rules")
errReplicationNoRule = Errorf("Replication configuration should have at least one rule")
errReplicationUniquePriority = Errorf("Replication configuration has duplicate priority")
errReplicationDestinationMismatch = Errorf("The destination bucket must be same for all rules")
errRoleArnMissing = Errorf("Missing required parameter `Role` in ReplicationConfiguration")
errInvalidSourceSelectionCriteria = Errorf("Invalid ReplicaModification status")
errReplicationTooManyRules = Errorf("Replication configuration allows a maximum of 1000 rules")
errReplicationNoRule = Errorf("Replication configuration should have at least one rule")
errReplicationUniquePriority = Errorf("Replication configuration has duplicate priority")
errRoleArnMissingLegacy = Errorf("Missing required parameter `Role` in ReplicationConfiguration")
errDestinationArnMissing = Errorf("Missing required parameter `Destination` in Replication rule")
errInvalidSourceSelectionCriteria = Errorf("Invalid ReplicaModification status")
errRoleArnPresentForMultipleTargets = Errorf("`Role` should be empty in ReplicationConfiguration for multiple targets")
)
// Config - replication configuration specified in
@ -102,18 +76,14 @@ func (c Config) Validate(bucket string, sameTarget bool) error {
if len(c.Rules) == 0 {
return errReplicationNoRule
}
if c.RoleArn == "" {
return errRoleArnMissing
}
// Validate all the rules in the replication config
targetMap := make(map[string]struct{})
priorityMap := make(map[string]struct{})
var legacyArn bool
for _, r := range c.Rules {
if len(targetMap) == 0 {
targetMap[r.Destination.Bucket] = struct{}{}
}
if _, ok := targetMap[r.Destination.Bucket]; !ok {
return errReplicationDestinationMismatch
targetMap[r.Destination.Bucket] = struct{}{}
}
if err := r.Validate(bucket, sameTarget); err != nil {
return err
@ -122,6 +92,22 @@ func (c Config) Validate(bucket string, sameTarget bool) error {
return errReplicationUniquePriority
}
priorityMap[strconv.Itoa(r.Priority)] = struct{}{}
if r.Destination.LegacyArn() {
legacyArn = true
}
if c.RoleArn == "" && !r.Destination.TargetArn() {
return errDestinationArnMissing
}
}
// disallow combining old replication configuration which used RoleArn as target ARN with multiple
// destination replication
if c.RoleArn != "" && len(targetMap) > 1 {
return errRoleArnPresentForMultipleTargets
}
// validate RoleArn if destination used legacy ARN format.
if c.RoleArn == "" && legacyArn {
return errRoleArnMissingLegacy
}
return nil
}
@ -137,6 +123,7 @@ const (
MetadataReplicationType
HealReplicationType
ExistingObjectReplicationType
ResyncReplicationType
)
// Valid returns true if replication type is set
@ -150,18 +137,18 @@ type ObjectOpts struct {
Name string
UserTags string
VersionID string
IsLatest bool
DeleteMarker bool
SSEC bool
OpType Type
Replica bool
ExistingObject bool
TargetArn string
}
// FilterActionableRules returns the rules actions that need to be executed
// after evaluating prefix/tag filtering
func (c Config) FilterActionableRules(obj ObjectOpts) []Rule {
if obj.Name == "" {
if obj.Name == "" && obj.OpType != ResyncReplicationType {
return nil
}
var rules []Rule
@ -169,6 +156,18 @@ func (c Config) FilterActionableRules(obj ObjectOpts) []Rule {
if rule.Status == Disabled {
continue
}
if obj.TargetArn != "" && rule.Destination.ARN != obj.TargetArn && c.RoleArn != obj.TargetArn {
continue
}
// Ignore other object level and prefix filters for resyncing target
if obj.OpType == ResyncReplicationType {
rules = append(rules, rule)
continue
}
if obj.ExistingObject && rule.ExistingObjectReplication.Status == Disabled {
continue
}
if !strings.HasPrefix(obj.Name, rule.Prefix()) {
continue
}
@ -177,8 +176,9 @@ func (c Config) FilterActionableRules(obj ObjectOpts) []Rule {
}
}
sort.Slice(rules[:], func(i, j int) bool {
return rules[i].Priority > rules[j].Priority
return rules[i].Priority > rules[j].Priority && rules[i].Destination.String() == rules[j].Destination.String()
})
return rules
}
@ -205,7 +205,7 @@ func (c Config) Replicate(obj ObjectOpts) bool {
if obj.OpType == DeleteReplicationType {
switch {
case obj.VersionID != "":
// // check MinIO extension for versioned deletes
// check MinIO extension for versioned deletes
return rule.DeleteReplication.Status == Enabled
default:
return rule.DeleteMarkerReplication.Status == Enabled
@ -243,3 +243,27 @@ func (c Config) HasActiveRules(prefix string, recursive bool) bool {
}
return false
}
// FilterTargetArns returns a slice of distinct target arns in the config
func (c Config) FilterTargetArns(obj ObjectOpts) []string {
var arns []string
tgtsMap := make(map[string]struct{})
rules := c.FilterActionableRules(obj)
for _, rule := range rules {
if rule.Status == Disabled {
continue
}
if c.RoleArn != "" {
arns = append(arns, c.RoleArn) // use legacy RoleArn if present
return arns
}
if _, ok := tgtsMap[rule.Destination.ARN]; !ok {
tgtsMap[rule.Destination.ARN] = struct{}{}
}
}
for k := range tgtsMap {
arns = append(arns, k)
}
return arns
}

View File

@ -52,20 +52,20 @@ func TestParseAndValidateReplicationConfig(t *testing.T) {
expectedParsingErr: nil,
expectedValidationErr: nil,
},
//4 missing role in config
//4 missing role in config and destination ARN is in legacy format
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
// destination bucket in config different from bucket specified
destBucket: "destinationbucket",
sameTarget: false,
expectedParsingErr: nil,
expectedValidationErr: errRoleArnMissing,
expectedValidationErr: errDestinationArnMissing,
},
//5 replication destination in different rules not identical
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule><Rule><Status>Enabled</Status><Priority>3</Priority><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket2</Bucket></Destination></Rule></ReplicationConfiguration>`,
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role></Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:minio:replication:::destinationbucket</Bucket></Destination></Rule><Rule><Status>Enabled</Status><Priority>3</Priority><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:minio:replication:::destinationbucket2</Bucket></Destination></Rule></ReplicationConfiguration>`,
destBucket: "destinationbucket",
sameTarget: false,
expectedParsingErr: nil,
expectedValidationErr: errReplicationDestinationMismatch,
expectedValidationErr: nil,
},
//6 missing rule status in replication config
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
@ -116,6 +116,22 @@ func TestParseAndValidateReplicationConfig(t *testing.T) {
expectedParsingErr: fmt.Errorf("invalid destination '%v'", "destinationbucket2"),
expectedValidationErr: nil,
},
//13 missing role in config and destination ARN has target ARN
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:minio:replication::8320b6d18f9032b4700f1f03b50d8d1853de8f22cab86931ee794e12f190852c:destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
// destination bucket in config different from bucket specified
destBucket: "destinationbucket",
sameTarget: false,
expectedParsingErr: nil,
expectedValidationErr: nil,
},
//14 role absent in config and destination ARN has target ARN in invalid format
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:xx:replication::8320b6d18f9032b4700f1f03b50d8d1853de8f22cab86931ee794e12f190852c:destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
// destination bucket in config different from bucket specified
destBucket: "destinationbucket",
sameTarget: false,
expectedParsingErr: fmt.Errorf("invalid destination '%v'", "arn:xx:replication::8320b6d18f9032b4700f1f03b50d8d1853de8f22cab86931ee794e12f190852c:destinationbucket"),
expectedValidationErr: nil,
},
}
for i, tc := range testCases {
t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) {
@ -333,3 +349,52 @@ func TestHasActiveRules(t *testing.T) {
}
}
func TestFilterActionableRules(t *testing.T) {
testCases := []struct {
inputConfig string
prefix string
ExpectedRules []Rule
}{
// case 1 - only one rule
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>prefix</Prefix><Priority>1</Priority><Destination><Bucket>arn:minio:replication:xxx::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
prefix: "prefix",
ExpectedRules: []Rule{{Status: Enabled, Priority: 1, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}}},
},
// case 2 - multiple rules for same target, overlapping rules with different priority
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>prefix</Prefix><Priority>3</Priority><Destination><Bucket>arn:minio:replication:xxx::destinationbucket</Bucket></Destination></Rule><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>prefix</Prefix><Priority>1</Priority><Destination><Bucket>arn:minio:replication:xxx::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
prefix: "prefix",
ExpectedRules: []Rule{
{Status: Enabled, Priority: 3, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}},
{Status: Enabled, Priority: 1, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}},
},
},
// case 3 - multiple rules for different target, overlapping rules on a target
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>prefix</Prefix><Priority>2</Priority><Destination><Bucket>arn:minio:replication:xxx::destinationbucket2</Bucket></Destination></Rule><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>prefix</Prefix><Priority>4</Priority><Destination><Bucket>arn:minio:replication:xxx::destinationbucket2</Bucket></Destination></Rule><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>prefix</Prefix><Priority>3</Priority><Destination><Bucket>arn:minio:replication:xxx::destinationbucket</Bucket></Destination></Rule><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>prefix</Prefix><Priority>1</Priority><Destination><Bucket>arn:minio:replication:xxx::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
prefix: "prefix",
ExpectedRules: []Rule{
{Status: Enabled, Priority: 4, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket2", ARN: "arn:minio:replication:xxx::destinationbucket2"}},
{Status: Enabled, Priority: 2, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket2", ARN: "arn:minio:replication:xxx::destinationbucket2"}},
{Status: Enabled, Priority: 3, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}},
{Status: Enabled, Priority: 1, DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled}, DeleteReplication: DeleteReplication{Status: Disabled}, Destination: Destination{Bucket: "destinationbucket", ARN: "arn:minio:replication:xxx::destinationbucket"}},
},
},
}
for _, tc := range testCases {
tc := tc
cfg, err := ParseConfig(bytes.NewReader([]byte(tc.inputConfig)))
if err != nil {
t.Fatalf("Got unexpected error: %v", err)
}
got := cfg.FilterActionableRules(ObjectOpts{Name: tc.prefix})
if len(got) != len(tc.ExpectedRules) {
t.Fatalf("Expected matching number of actionable rules: `%v`, got: `%v`", tc.ExpectedRules, got)
}
for i := range got {
if got[i].Destination.ARN != tc.ExpectedRules[i].Destination.ARN || got[i].Priority != tc.ExpectedRules[i].Priority {
t.Fatalf("Expected order of filtered rules to be identical: `%v`, got: `%v`", tc.ExpectedRules, got)
}
}
}
}

View File

@ -173,6 +173,12 @@ const (
// Header indicates replication reset status.
MinIOReplicationResetStatus = "X-Minio-Replication-Reset-Status"
// Header indiicates last tag update time on source
MinIOSourceTaggingTimestamp = "X-Minio-Source-Replication-Tagging-Timestamp"
// Header indiicates last rtention update time on source
MinIOSourceObjectRetentionTimestamp = "X-Minio-Source-Replication-Retention-Timestamp"
// Header indiicates last rtention update time on source
MinIOSourceObjectLegalHoldTimestamp = "X-Minio-Source-Replication-LegalHold-Timestamp"
// predicted date/time of transition
MinIOTransition = "X-Minio-Transition"
)