Add support for existing object replication. (#12109)

Also adding an API to allow resyncing replication when
existing object replication is enabled and the remote target
is entirely lost. With the `mc replicate reset` command, the
objects that are eligible for replication as per the replication
config will be resynced to target if existing object replication
is enabled on the rule.
This commit is contained in:
Poorna Krishnamoorthy
2021-06-01 19:59:11 -07:00
committed by GitHub
parent 1f262daf6f
commit dbea8d2ee0
19 changed files with 928 additions and 334 deletions

View File

@@ -125,6 +125,7 @@ const (
ErrReplicationSourceNotVersionedError
ErrReplicationNeedsVersioningError
ErrReplicationBucketNeedsVersioningError
ErrReplicationNoMatchingRuleError
ErrObjectRestoreAlreadyInProgress
ErrNoSuchKey
ErrNoSuchUpload
@@ -859,6 +860,11 @@ var errorCodes = errorCodeMap{
Description: "Remote service connection error - please check remote service credentials and target bucket",
HTTPStatusCode: http.StatusNotFound,
},
ErrReplicationNoMatchingRuleError: {
Code: "XMinioReplicationNoMatchingRule",
Description: "No matching replication rule found for this object prefix",
HTTPStatusCode: http.StatusBadRequest,
},
ErrBucketRemoteIdenticalToSource: {
Code: "XMinioAdminRemoteIdenticalToSource",
Description: "The remote target cannot be identical to source",

View File

@@ -394,6 +394,9 @@ func registerAPIRouter(router *mux.Router) {
// PutBucketNotification
router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("putbucketnotification", maxClients(httpTraceAll(api.PutBucketNotificationHandler)))).Queries("notification", "")
// ResetBucketReplicationState - MinIO extension API
router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("resetbucketreplicationstate", maxClients(httpTraceAll(api.ResetBucketReplicationStateHandler)))).Queries("replication-reset", "")
// PutBucket
router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("putbucket", maxClients(httpTraceAll(api.PutBucketHandler))))

File diff suppressed because one or more lines are too long

View File

@@ -33,6 +33,7 @@ import (
"strconv"
"strings"
"sync"
"time"
"github.com/google/uuid"
"github.com/gorilla/mux"
@@ -611,7 +612,7 @@ func (api objectAPIHandlers) DeleteMultipleObjectsHandler(w http.ResponseWriter,
if replicateDeletes {
if dobj.DeleteMarkerReplicationStatus == string(replication.Pending) || dobj.VersionPurgeStatus == Pending {
dv := DeletedObjectVersionInfo{
dv := DeletedObjectReplicationInfo{
DeletedObject: dobj,
Bucket: bucket,
}
@@ -1686,3 +1687,90 @@ func (api objectAPIHandlers) GetBucketReplicationMetricsHandler(w http.ResponseW
}
w.(http.Flusher).Flush()
}
// ResetBucketReplicationStateHandler - starts a replication reset for all objects in a bucket which
// qualify for replication and re-sync the object(s) to target, provided ExistingObjectReplication is
// enabled for the qualifying rule. This API is a MinIO only extension provided for situations where
// 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.URL.Query().Get("older-than")
var (
days time.Duration
err error
)
if durationStr != "" {
days, err = time.ParseDuration(durationStr)
if err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, InvalidArgument{
Bucket: bucket,
Err: fmt.Errorf("invalid query parameter older-than %s for %s : %w", durationStr, bucket, err),
}), r.URL, guessIsBrowserReq(r))
}
}
objectAPI := api.ObjectAPI()
if objectAPI == nil {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(ErrServerNotInitialized), r.URL, guessIsBrowserReq(r))
return
}
if s3Error := checkRequestAuthType(ctx, r, policy.ResetBucketReplicationStateAction, bucket, ""); s3Error != ErrNone {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL, guessIsBrowserReq(r))
return
}
// Check if bucket exists.
if _, err := objectAPI.GetBucketInfo(ctx, bucket); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return
}
config, err := globalBucketMetadataSys.GetReplicationConfig(ctx, bucket)
if err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return
}
if !config.HasActiveRules("", true) {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(ErrReplicationNoMatchingRuleError), r.URL, guessIsBrowserReq(r))
return
}
target := globalBucketTargetSys.GetRemoteBucketTargetByArn(ctx, bucket, config.RoleArn)
target.ResetBeforeDate = UTCNow().AddDate(0, 0, -1*int(days/24))
target.ResetID = mustGetUUID()
if err = globalBucketTargetSys.SetTarget(ctx, bucket, &target, true); err != nil {
switch err.(type) {
case BucketRemoteConnectionErr:
writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErrWithErr(ErrReplicationRemoteConnectionError, err), r.URL)
default:
writeErrorResponseJSON(ctx, w, toAPIError(ctx, err), r.URL)
}
return
}
targets, err := globalBucketTargetSys.ListBucketTargets(ctx, bucket)
if err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return
}
tgtBytes, err := json.Marshal(&targets)
if err != nil {
writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErrWithErr(ErrAdminConfigBadJSON, err), r.URL)
return
}
if err = globalBucketMetadataSys.Update(bucket, bucketTargetsFile, tgtBytes); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return
}
data, err := json.Marshal(target.ResetID)
if err != nil {
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
return
}
// Write success response.
writeSuccessResponseJSON(w, data)
}

View File

@@ -91,32 +91,70 @@ func validateReplicationDestination(ctx context.Context, bucket string, rCfg *re
return false, BucketRemoteTargetNotFound{Bucket: bucket}
}
func mustReplicateWeb(ctx context.Context, r *http.Request, bucket, object string, meta map[string]string, replStatus string, permErr APIErrorCode) (replicate bool, sync bool) {
type mustReplicateOptions struct {
meta map[string]string
status replication.StatusType
opType replication.Type
}
func (o mustReplicateOptions) ReplicationStatus() (s replication.StatusType) {
if rs, ok := o.meta[xhttp.AmzBucketReplicationStatus]; ok {
return replication.StatusType(rs)
}
return s
}
func (o mustReplicateOptions) isExistingObjectReplication() bool {
return o.opType == replication.ExistingObjectReplicationType
}
func (o mustReplicateOptions) isMetadataReplication() bool {
return o.opType == replication.MetadataReplicationType
}
func getMustReplicateOptions(o ObjectInfo, op replication.Type) mustReplicateOptions {
if !op.Valid() {
op = replication.ObjectReplicationType
if o.metadataOnly {
op = replication.MetadataReplicationType
}
}
meta := cloneMSS(o.UserDefined)
if o.UserTags != "" {
meta[xhttp.AmzObjectTagging] = o.UserTags
}
return mustReplicateOptions{
meta: meta,
status: o.ReplicationStatus,
opType: op,
}
}
func mustReplicateWeb(ctx context.Context, r *http.Request, bucket, object string, meta map[string]string, replStatus replication.StatusType, permErr APIErrorCode) (replicate bool, sync bool) {
if permErr != ErrNone {
return
}
return mustReplicater(ctx, bucket, object, meta, replStatus, false)
return mustReplicater(ctx, bucket, object, mustReplicateOptions{
meta: meta,
status: replStatus,
opType: replication.ObjectReplicationType,
})
}
// mustReplicate returns 2 booleans - true if object meets replication criteria and true if replication is to be done in
// a synchronous manner.
func mustReplicate(ctx context.Context, r *http.Request, bucket, object string, meta map[string]string, replStatus string, metadataOnly bool) (replicate bool, sync bool) {
func mustReplicate(ctx context.Context, r *http.Request, bucket, object string, opts mustReplicateOptions) (replicate bool, sync bool) {
if s3Err := isPutActionAllowed(ctx, getRequestAuthType(r), bucket, "", r, iampolicy.GetReplicationConfigurationAction); s3Err != ErrNone {
return
}
return mustReplicater(ctx, bucket, object, meta, replStatus, metadataOnly)
return mustReplicater(ctx, bucket, object, opts)
}
// mustReplicater returns 2 booleans - true if object meets replication criteria and true if replication is to be done in
// a synchronous manner.
func mustReplicater(ctx context.Context, bucket, object string, meta map[string]string, replStatus string, metadataOnly bool) (replicate bool, sync bool) {
func mustReplicater(ctx context.Context, bucket, object string, mopts mustReplicateOptions) (replicate bool, sync bool) {
if globalIsGateway {
return replicate, sync
}
if rs, ok := meta[xhttp.AmzBucketReplicationStatus]; ok {
replStatus = rs
}
if replication.StatusType(replStatus) == replication.Replica && !metadataOnly {
replStatus := mopts.ReplicationStatus()
if replStatus == replication.Replica && !mopts.isMetadataReplication() {
return replicate, sync
}
cfg, err := getReplicationConfig(ctx, bucket)
@@ -124,11 +162,12 @@ func mustReplicater(ctx context.Context, bucket, object string, meta map[string]
return replicate, sync
}
opts := replication.ObjectOpts{
Name: object,
SSEC: crypto.SSEC.IsEncrypted(meta),
Replica: replication.StatusType(replStatus) == replication.Replica,
Name: object,
SSEC: crypto.SSEC.IsEncrypted(mopts.meta),
Replica: replStatus == replication.Replica,
ExistingObject: mopts.isExistingObjectReplication(),
}
tagStr, ok := meta[xhttp.AmzObjectTagging]
tagStr, ok := mopts.meta[xhttp.AmzObjectTagging]
if ok {
opts.UserTags = tagStr
}
@@ -226,7 +265,7 @@ func checkReplicateDelete(ctx context.Context, bucket string, dobj ObjectToDelet
// target cluster, the object version is marked deleted on the source and hidden from listing. It is permanently
// deleted from the source when the VersionPurgeStatus changes to "Complete", i.e after replication succeeds
// on target.
func replicateDelete(ctx context.Context, dobj DeletedObjectVersionInfo, objectAPI ObjectLayer) {
func replicateDelete(ctx context.Context, dobj DeletedObjectReplicationInfo, objectAPI ObjectLayer) {
bucket := dobj.Bucket
versionID := dobj.DeleteMarkerVersionID
if versionID == "" {
@@ -760,7 +799,9 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje
if objInfo.UserTags != "" {
objInfo.UserDefined[xhttp.AmzObjectTagging] = objInfo.UserTags
}
if ri.OpType == replication.ExistingObjectReplicationType {
objInfo.UserDefined[xhttp.MinIOReplicationResetStatus] = fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), ri.ResetID)
}
// FIXME: add support for missing replication events
// - event.ObjectReplicationMissedThreshold
// - event.ObjectReplicationReplicatedAfterThreshold
@@ -774,7 +815,8 @@ func replicateObject(ctx context.Context, ri ReplicateObjectInfo, objectAPI Obje
return
}
// Leave metadata in `PENDING` state if inline replication fails to save iops
if ri.OpType == replication.HealReplicationType || replicationStatus == replication.Completed {
if ri.OpType == replication.HealReplicationType ||
replicationStatus == replication.Completed {
// This lower level implementation is necessary to avoid write locks from CopyObject.
poolIdx, err := z.getPoolIdx(ctx, bucket, object, objInfo.Size)
if err != nil {
@@ -834,10 +876,12 @@ func filterReplicationStatusMetadata(metadata map[string]string) map[string]stri
return dst
}
// DeletedObjectVersionInfo has info on deleted object
type DeletedObjectVersionInfo struct {
// DeletedObjectReplicationInfo has info on deleted object
type DeletedObjectReplicationInfo struct {
DeletedObject
Bucket string
Bucket string
OpType replication.Type
ResetID string
}
var (
@@ -847,35 +891,40 @@ var (
// ReplicationPool describes replication pool
type ReplicationPool struct {
objLayer ObjectLayer
ctx context.Context
mrfWorkerKillCh chan struct{}
workerKillCh chan struct{}
replicaCh chan ReplicateObjectInfo
replicaDeleteCh chan DeletedObjectVersionInfo
mrfReplicaCh chan ReplicateObjectInfo
workerSize int
mrfWorkerSize int
workerWg sync.WaitGroup
mrfWorkerWg sync.WaitGroup
once sync.Once
mu sync.Mutex
objLayer ObjectLayer
ctx context.Context
mrfWorkerKillCh chan struct{}
workerKillCh chan struct{}
replicaCh chan ReplicateObjectInfo
replicaDeleteCh chan DeletedObjectReplicationInfo
mrfReplicaCh chan ReplicateObjectInfo
existingReplicaCh chan ReplicateObjectInfo
existingReplicaDeleteCh chan DeletedObjectReplicationInfo
workerSize int
mrfWorkerSize int
workerWg sync.WaitGroup
mrfWorkerWg sync.WaitGroup
once sync.Once
mu sync.Mutex
}
// NewReplicationPool creates a pool of replication workers of specified size
func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPoolOpts) *ReplicationPool {
pool := &ReplicationPool{
replicaCh: make(chan ReplicateObjectInfo, 100000),
replicaDeleteCh: make(chan DeletedObjectVersionInfo, 100000),
mrfReplicaCh: make(chan ReplicateObjectInfo, 100000),
workerKillCh: make(chan struct{}, opts.Workers),
mrfWorkerKillCh: make(chan struct{}, opts.FailedWorkers),
ctx: ctx,
objLayer: o,
replicaCh: make(chan ReplicateObjectInfo, 100000),
replicaDeleteCh: make(chan DeletedObjectReplicationInfo, 100000),
mrfReplicaCh: make(chan ReplicateObjectInfo, 100000),
workerKillCh: make(chan struct{}, opts.Workers),
mrfWorkerKillCh: make(chan struct{}, opts.FailedWorkers),
existingReplicaCh: make(chan ReplicateObjectInfo, 100000),
existingReplicaDeleteCh: make(chan DeletedObjectReplicationInfo, 100000),
ctx: ctx,
objLayer: o,
}
pool.ResizeWorkers(opts.Workers)
pool.ResizeFailedWorkers(opts.FailedWorkers)
go pool.AddExistingObjectReplicateWorker()
return pool
}
@@ -921,6 +970,26 @@ func (p *ReplicationPool) AddWorker() {
}
// AddExistingObjectReplicateWorker adds a worker to queue existing objects that need to be sync'd
func (p *ReplicationPool) AddExistingObjectReplicateWorker() {
for {
select {
case <-p.ctx.Done():
return
case oi, ok := <-p.existingReplicaCh:
if !ok {
return
}
replicateObject(p.ctx, oi, p.objLayer)
case doi, ok := <-p.existingReplicaDeleteCh:
if !ok {
return
}
replicateDelete(p.ctx, doi, p.objLayer)
}
}
}
// ResizeWorkers sets replication workers pool to new size
func (p *ReplicationPool) ResizeWorkers(n int) {
p.mu.Lock()
@@ -962,6 +1031,7 @@ func (p *ReplicationPool) queueReplicaFailedTask(ri ReplicateObjectInfo) {
p.once.Do(func() {
close(p.replicaCh)
close(p.mrfReplicaCh)
close(p.existingReplicaCh)
})
case p.mrfReplicaCh <- ri:
default:
@@ -972,27 +1042,44 @@ func (p *ReplicationPool) queueReplicaTask(ri ReplicateObjectInfo) {
if p == nil {
return
}
var ch chan ReplicateObjectInfo
switch ri.OpType {
case replication.ExistingObjectReplicationType:
ch = p.existingReplicaCh
default:
ch = p.replicaCh
}
select {
case <-GlobalContext.Done():
p.once.Do(func() {
close(p.replicaCh)
close(p.mrfReplicaCh)
close(p.existingReplicaCh)
})
case p.replicaCh <- ri:
case ch <- ri:
default:
}
}
func (p *ReplicationPool) queueReplicaDeleteTask(doi DeletedObjectVersionInfo) {
func (p *ReplicationPool) queueReplicaDeleteTask(doi DeletedObjectReplicationInfo) {
if p == nil {
return
}
var ch chan DeletedObjectReplicationInfo
switch doi.OpType {
case replication.ExistingObjectReplicationType:
ch = p.existingReplicaDeleteCh
default:
ch = p.replicaDeleteCh
}
select {
case <-GlobalContext.Done():
p.once.Do(func() {
close(p.replicaDeleteCh)
close(p.existingReplicaDeleteCh)
})
case p.replicaDeleteCh <- doi:
case ch <- doi:
default:
}
}
@@ -1157,7 +1244,78 @@ func scheduleReplication(ctx context.Context, objInfo ObjectInfo, o ObjectLayer,
}
}
func scheduleReplicationDelete(ctx context.Context, dv DeletedObjectVersionInfo, o ObjectLayer, sync bool) {
func scheduleReplicationDelete(ctx context.Context, dv DeletedObjectReplicationInfo, o ObjectLayer, sync bool) {
globalReplicationPool.queueReplicaDeleteTask(dv)
globalReplicationStats.Update(dv.Bucket, 0, replication.Pending, replication.StatusType(""), replication.DeleteReplicationType)
}
type replicationConfig struct {
Config *replication.Config
ResetID string
ResetBeforeDate time.Time
}
func (c replicationConfig) Empty() bool {
return c.Config == nil
}
func (c replicationConfig) Replicate(opts replication.ObjectOpts) bool {
return c.Config.Replicate(opts)
}
// Resync returns true if replication reset is requested
func (c replicationConfig) Resync(ctx context.Context, oi ObjectInfo) bool {
if c.Empty() {
return false
}
// existing object replication does not apply to un-versioned objects
if oi.VersionID == "" || oi.VersionID == nullVersionID {
return false
}
var replicate bool
if oi.DeleteMarker {
if c.Replicate(replication.ObjectOpts{
Name: oi.Name,
SSEC: crypto.SSEC.IsEncrypted(oi.UserDefined),
UserTags: oi.UserTags,
DeleteMarker: oi.DeleteMarker,
VersionID: oi.VersionID,
OpType: replication.DeleteReplicationType,
ExistingObject: true}) {
replicate = true
}
} else {
// Ignore previous replication status when deciding if object can be re-replicated
objInfo := oi.Clone()
objInfo.ReplicationStatus = replication.StatusType("")
replicate, _ = mustReplicater(ctx, oi.Bucket, oi.Name, getMustReplicateOptions(objInfo, replication.ExistingObjectReplicationType))
}
return c.resync(oi, replicate)
}
// wrapper function for testability. Returns true if a new reset is requested on
// already replicated objects OR object qualifies for existing object replication
// and no reset requested.
func (c replicationConfig) resync(oi ObjectInfo, replicate bool) bool {
if !replicate {
return false
}
rs, ok := oi.UserDefined[xhttp.MinIOReplicationResetStatus]
if !ok { // existing object replication is enabled and object version is unreplicated so far.
if c.ResetID != "" && oi.ModTime.Before(c.ResetBeforeDate) { // trigger replication if `mc replicate reset` requested
return true
}
return oi.ReplicationStatus != replication.Completed
}
if c.ResetID == "" || c.ResetBeforeDate.Equal(timeSentinel) { // no reset in progress
return false
}
// if already replicated, return true if a new reset was requested.
splits := strings.SplitN(rs, ";", 2)
newReset := splits[1] != c.ResetID
if !newReset && oi.ReplicationStatus == replication.Completed {
// already replicated and no reset requested
return false
}
return newReset && oi.ModTime.Before(c.ResetBeforeDate)
}

View File

@@ -0,0 +1,209 @@
// 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 (
"context"
"fmt"
"net/http"
"testing"
"time"
"github.com/minio/minio/internal/bucket/replication"
xhttp "github.com/minio/minio/internal/http"
)
var configs = []replication.Config{
{ // Config0 - Replication config has no filters, existing object replication enabled
Rules: []replication.Rule{
{
Status: replication.Enabled,
Priority: 1,
DeleteMarkerReplication: replication.DeleteMarkerReplication{Status: replication.Enabled},
DeleteReplication: replication.DeleteReplication{Status: replication.Enabled},
Filter: replication.Filter{},
ExistingObjectReplication: replication.ExistingObjectReplication{Status: replication.Enabled},
SourceSelectionCriteria: replication.SourceSelectionCriteria{
ReplicaModifications: replication.ReplicaModifications{Status: replication.Enabled},
},
},
},
},
}
var replicationConfigTests = []struct {
info ObjectInfo
name string
rcfg replicationConfig
expectedSync bool
}{
{ //1. no replication config
name: "no replication config",
info: ObjectInfo{Size: 100},
rcfg: replicationConfig{Config: nil},
expectedSync: false,
},
{ //2. existing object replication config enabled, no versioning
name: "existing object replication config enabled, no versioning",
info: ObjectInfo{Size: 100},
rcfg: replicationConfig{Config: &configs[0]},
expectedSync: false,
},
{ //3. existing object replication config enabled, versioning suspended
name: "existing object replication config enabled, versioning suspended",
info: ObjectInfo{Size: 100, VersionID: nullVersionID},
rcfg: replicationConfig{Config: &configs[0]},
expectedSync: false,
},
{ //4. existing object replication enabled, versioning enabled; no reset in progress
name: "existing object replication enabled, versioning enabled; no reset in progress",
info: ObjectInfo{Size: 100,
ReplicationStatus: replication.Completed,
VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9",
},
rcfg: replicationConfig{Config: &configs[0]},
expectedSync: false,
},
}
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)
}
}
}
var start = UTCNow().AddDate(0, 0, -1)
var replicationConfigTests2 = []struct {
info ObjectInfo
name string
replicate bool
rcfg replicationConfig
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",
},
replicate: true,
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",
},
replicate: true,
expectedSync: true,
},
{ //3. replication status unset
name: "existing object replication on pre-existing unreplicated object",
info: ObjectInfo{Size: 100,
ReplicationStatus: replication.StatusType(""),
VersionID: "a3348c34-c352-4498-82f0-1098e8b34df9",
},
replicate: true,
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",
},
replicate: true,
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",
},
replicate: true,
expectedSync: true,
rcfg: replicationConfig{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",
},
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",
},
replicate: true,
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",
},
replicate: true,
expectedSync: true,
rcfg: replicationConfig{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,
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),
},
replicate: true,
expectedSync: true,
rcfg: replicationConfig{ResetID: "abc", 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")},
},
replicate: true,
expectedSync: false,
rcfg: replicationConfig{ResetID: "xyz", ResetBeforeDate: UTCNow()},
},
}
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)
}
}
}

View File

@@ -375,7 +375,12 @@ func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, int
activeLifeCycle = f.oldCache.Info.lifeCycle
filter = nil
}
// If there are replication rules for the prefix, remove the filter.
var replicationCfg replicationConfig
if !f.oldCache.Info.replication.Empty() && f.oldCache.Info.replication.Config.HasActiveRules(prefix, true) {
replicationCfg = f.oldCache.Info.replication
filter = nil
}
// Check if we can skip it due to bloom filter...
if filter != nil && ok && existing.Compacted {
// If folder isn't in filter and we have data, skip it completely.
@@ -449,16 +454,16 @@ func (f *folderScanner) scanFolder(ctx context.Context, folder cachedFolder, int
// Get file size, ignore errors.
item := scannerItem{
Path: path.Join(f.root, entName),
Typ: typ,
bucket: bucket,
prefix: path.Dir(prefix),
objectName: path.Base(entName),
debug: f.dataUsageScannerDebug,
lifeCycle: activeLifeCycle,
heal: thisHash.mod(f.oldCache.Info.NextCycle, f.healObjectSelect/folder.objectHealProbDiv) && globalIsErasure,
Path: path.Join(f.root, entName),
Typ: typ,
bucket: bucket,
prefix: path.Dir(prefix),
objectName: path.Base(entName),
debug: f.dataUsageScannerDebug,
lifeCycle: activeLifeCycle,
replication: replicationCfg,
heal: thisHash.mod(f.oldCache.Info.NextCycle, f.healObjectSelect/folder.objectHealProbDiv) && globalIsErasure,
}
// if the drive belongs to an erasure set
// that is already being healed, skip the
// healing attempt on this drive.
@@ -808,12 +813,13 @@ type scannerItem struct {
Path string
Typ os.FileMode
bucket string // Bucket.
prefix string // Only the prefix if any, does not have final object name.
objectName string // Only the object name without prefixes.
lifeCycle *lifecycle.Lifecycle
heal bool // Has the object been selected for heal check?
debug bool
bucket string // Bucket.
prefix string // Only the prefix if any, does not have final object name.
objectName string // Only the object name without prefixes.
lifeCycle *lifecycle.Lifecycle
replication replicationConfig
heal bool // Has the object been selected for heal check?
debug bool
}
type sizeSummary struct {
@@ -1140,33 +1146,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)
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)
i.healReplicationDeletes(ctx, o, oi, existingObjResync)
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)
return
}
return
}
roi := ReplicateObjectInfo{ObjectInfo: oi, OpType: replication.HealReplicationType}
if existingObjResync {
roi.OpType = replication.ExistingObjectReplicationType
roi.ResetID = i.replication.ResetID
}
switch oi.ReplicationStatus {
case replication.Pending:
sizeS.pendingCount++
sizeS.pendingSize += oi.Size
globalReplicationPool.queueReplicaTask(ReplicateObjectInfo{ObjectInfo: oi, OpType: replication.HealReplicationType})
globalReplicationPool.queueReplicaTask(roi)
return
case replication.Failed:
sizeS.failedSize += oi.Size
sizeS.failedCount++
globalReplicationPool.queueReplicaTask(ReplicateObjectInfo{ObjectInfo: oi, OpType: replication.HealReplicationType})
globalReplicationPool.queueReplicaTask(roi)
return
case replication.Completed, "COMPLETE":
sizeS.replicatedSize += oi.Size
case replication.Replica:
sizeS.replicaSize += oi.Size
}
if existingObjResync {
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) {
func (i *scannerItem) healReplicationDeletes(ctx context.Context, o ObjectLayer, oi ObjectInfo, existingObject bool) {
// handle soft delete and permanent delete failures here.
if oi.DeleteMarker || !oi.VersionPurgeStatus.Empty() {
versionID := ""
@@ -1176,7 +1199,7 @@ func (i *scannerItem) healReplicationDeletes(ctx context.Context, o ObjectLayer,
} else {
versionID = oi.VersionID
}
globalReplicationPool.queueReplicaDeleteTask(DeletedObjectVersionInfo{
doi := DeletedObjectReplicationInfo{
DeletedObject: DeletedObject{
ObjectName: oi.Name,
DeleteMarkerVersionID: dmVersionID,
@@ -1187,7 +1210,12 @@ func (i *scannerItem) healReplicationDeletes(ctx context.Context, o ObjectLayer,
VersionPurgeStatus: oi.VersionPurgeStatus,
},
Bucket: oi.Bucket,
})
}
if existingObject {
doi.OpType = replication.ExistingObjectReplicationType
doi.ResetID = i.replication.ResetID
}
globalReplicationPool.queueReplicaDeleteTask(doi)
}
}

View File

@@ -161,7 +161,8 @@ type dataUsageCacheInfo struct {
// optional updates channel.
// If set updates will be sent regularly to this channel.
// Will not be closed when returned.
updates chan<- dataUsageEntry `msg:"-"`
updates chan<- dataUsageEntry `msg:"-"`
replication replicationConfig `msg:"-"`
}
func (e *dataUsageEntry) addSizes(summary sizeSummary) {

View File

@@ -230,6 +230,7 @@ type ReplicateObjectInfo struct {
ObjectInfo
OpType replication.Type
RetryCount uint32
ResetID string
}
// MultipartInfo captures metadata information about the uploadId

View File

@@ -1291,7 +1291,7 @@ func (api objectAPIHandlers) CopyObjectHandler(w http.ResponseWriter, r *http.Re
if rs := r.Header.Get(xhttp.AmzBucketReplicationStatus); rs != "" {
srcInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = rs
}
if ok, _ := mustReplicate(ctx, r, dstBucket, dstObject, srcInfo.UserDefined, srcInfo.ReplicationStatus.String(), srcInfo.metadataOnly); ok {
if ok, _ := mustReplicate(ctx, r, dstBucket, dstObject, getMustReplicateOptions(srcInfo, replication.UnsetReplicationType)); ok {
srcInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String()
}
// Store the preserved compression metadata.
@@ -1393,7 +1393,8 @@ func (api objectAPIHandlers) CopyObjectHandler(w http.ResponseWriter, r *http.Re
objInfo.ETag = getDecryptedETag(r.Header, objInfo, false)
response := generateCopyObjectResponse(objInfo.ETag, objInfo.ModTime)
encodedSuccessResponse := encodeResponse(response)
if replicate, sync := mustReplicate(ctx, r, dstBucket, dstObject, objInfo.UserDefined, objInfo.ReplicationStatus.String(), objInfo.metadataOnly); replicate {
if replicate, sync := mustReplicate(ctx, r, dstBucket, dstObject, getMustReplicateOptions(objInfo, replication.UnsetReplicationType)); replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
}
@@ -1637,7 +1638,9 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL, guessIsBrowserReq(r))
return
}
if ok, _ := mustReplicate(ctx, r, bucket, object, metadata, "", false); ok {
if ok, _ := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(ObjectInfo{
UserDefined: metadata,
}, replication.ObjectReplicationType)); ok {
metadata[xhttp.AmzBucketReplicationStatus] = replication.Pending.String()
}
if r.Header.Get(xhttp.AmzBucketReplicationStatus) == replication.Replica.String() {
@@ -1720,7 +1723,9 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req
}
}
}
if replicate, sync := mustReplicate(ctx, r, bucket, object, metadata, "", false); replicate {
if replicate, sync := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(ObjectInfo{
UserDefined: metadata,
}, replication.ObjectReplicationType)); replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
}
@@ -1956,7 +1961,9 @@ func (api objectAPIHandlers) PutObjectExtractHandler(w http.ResponseWriter, r *h
return
}
if ok, _ := mustReplicate(ctx, r, bucket, object, metadata, "", false); ok {
if ok, _ := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(ObjectInfo{
UserDefined: metadata,
}, replication.ObjectReplicationType)); ok {
metadata[xhttp.AmzBucketReplicationStatus] = replication.Pending.String()
}
@@ -2012,7 +2019,9 @@ func (api objectAPIHandlers) PutObjectExtractHandler(w http.ResponseWriter, r *h
return
}
if replicate, sync := mustReplicate(ctx, r, bucket, object, metadata, "", false); replicate {
if replicate, sync := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(ObjectInfo{
UserDefined: metadata,
}, replication.ObjectReplicationType)); replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
}
@@ -2124,7 +2133,9 @@ func (api objectAPIHandlers) NewMultipartUploadHandler(w http.ResponseWriter, r
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Err), r.URL, guessIsBrowserReq(r))
return
}
if ok, _ := mustReplicate(ctx, r, bucket, object, metadata, "", false); ok {
if ok, _ := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(ObjectInfo{
UserDefined: metadata,
}, replication.ObjectReplicationType)); ok {
metadata[xhttp.AmzBucketReplicationStatus] = replication.Pending.String()
}
// We need to preserve the encryption headers set in EncryptRequest,
@@ -3114,7 +3125,7 @@ func (api objectAPIHandlers) CompleteMultipartUploadHandler(w http.ResponseWrite
}
setPutObjHeaders(w, objInfo, false)
if replicate, sync := mustReplicate(ctx, r, bucket, object, objInfo.UserDefined, objInfo.ReplicationStatus.String(), false); replicate {
if replicate, sync := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(objInfo, replication.ObjectReplicationType)); replicate {
scheduleReplication(ctx, objInfo.Clone(), objectAPI, sync, replication.ObjectReplicationType)
}
@@ -3288,7 +3299,7 @@ func (api objectAPIHandlers) DeleteObjectHandler(w http.ResponseWriter, r *http.
} else {
versionID = objInfo.VersionID
}
dobj := DeletedObjectVersionInfo{
dobj := DeletedObjectReplicationInfo{
DeletedObject: DeletedObject{
ObjectName: object,
VersionID: versionID,
@@ -3375,7 +3386,7 @@ func (api objectAPIHandlers) PutObjectLegalHoldHandler(w http.ResponseWriter, r
return
}
objInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockLegalHold)] = strings.ToUpper(string(legalHold.Status))
replicate, sync := mustReplicate(ctx, r, bucket, object, objInfo.UserDefined, objInfo.ReplicationStatus.String(), true)
replicate, sync := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(objInfo, replication.MetadataReplicationType))
if replicate {
objInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String()
}
@@ -3554,7 +3565,7 @@ func (api objectAPIHandlers) PutObjectRetentionHandler(w http.ResponseWriter, r
objInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockMode)] = ""
objInfo.UserDefined[strings.ToLower(xhttp.AmzObjectLockRetainUntilDate)] = ""
}
replicate, sync := mustReplicate(ctx, r, bucket, object, objInfo.UserDefined, objInfo.ReplicationStatus.String(), true)
replicate, sync := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(objInfo, replication.MetadataReplicationType))
if replicate {
objInfo.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String()
}
@@ -3751,14 +3762,16 @@ func (api objectAPIHandlers) PutObjectTaggingHandler(w http.ResponseWriter, r *h
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return
}
replicate, sync := mustReplicate(ctx, r, bucket, object, map[string]string{xhttp.AmzObjectTagging: tags.String()}, objInfo.ReplicationStatus.String(), true)
tagsStr := tags.String()
oi := objInfo.Clone()
oi.UserTags = tagsStr
replicate, sync := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(oi, replication.MetadataReplicationType))
if replicate {
opts.UserDefined = make(map[string]string)
opts.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String()
}
tagsStr := tags.String()
// Put object tags
objInfo, err = objAPI.PutObjectTags(ctx, bucket, object, tagsStr, opts)
if err != nil {
@@ -3828,7 +3841,7 @@ func (api objectAPIHandlers) DeleteObjectTaggingHandler(w http.ResponseWriter, r
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL, guessIsBrowserReq(r))
return
}
replicate, sync := mustReplicate(ctx, r, bucket, object, map[string]string{xhttp.AmzObjectTagging: oi.UserTags}, oi.ReplicationStatus.String(), true)
replicate, sync := mustReplicate(ctx, r, bucket, object, getMustReplicateOptions(oi, replication.MetadataReplicationType))
if replicate {
opts.UserDefined = make(map[string]string)
opts.UserDefined[xhttp.AmzBucketReplicationStatus] = replication.Pending.String()

View File

@@ -814,7 +814,7 @@ next:
})
if replicateDel {
dobj := DeletedObjectVersionInfo{
dobj := DeletedObjectReplicationInfo{
DeletedObject: DeletedObject{
ObjectName: objectName,
DeleteMarkerVersionID: oi.VersionID,
@@ -948,7 +948,7 @@ next:
Host: sourceIP,
})
if dobj.DeleteMarkerReplicationStatus == string(replication.Pending) || dobj.VersionPurgeStatus == Pending {
dv := DeletedObjectVersionInfo{
dv := DeletedObjectReplicationInfo{
DeletedObject: dobj,
Bucket: args.BucketName,
}
@@ -1256,7 +1256,7 @@ func (web *webAPIHandlers) Upload(w http.ResponseWriter, r *http.Request) {
}
}
mustReplicate, sync := mustReplicateWeb(ctx, r, bucket, object, metadata, "", replPerms)
mustReplicate, sync := mustReplicateWeb(ctx, r, bucket, object, metadata, replication.StatusType(""), replPerms)
if mustReplicate {
metadata[xhttp.AmzBucketReplicationStatus] = string(replication.Pending)
}

View File

@@ -414,6 +414,19 @@ 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")
}
}
}
// return initialized object layer
objAPI := newObjectLayerFn()
// object layer not initialized, return.
@@ -452,7 +465,6 @@ func (s *xlStorage) NSScanner(ctx context.Context, cache dataUsageCache, updates
}
return sizeSummary{}, errSkipFile
}
sizeS := sizeSummary{}
for _, version := range fivs.Versions {
oi := version.ToObjectInfo(item.bucket, item.objectPath())