Refactor replication resync to be an active process (#14266)

When resync is triggered, walk the bucket namespace and
resync objects that are unreplicated. This PR also adds
an API to report resync progress.
This commit is contained in:
Poorna
2022-02-10 10:16:52 -08:00
committed by GitHub
parent a2230868e0
commit ed3418c046
12 changed files with 1717 additions and 139 deletions

View File

@@ -343,6 +343,9 @@ func registerAPIRouter(router *mux.Router) {
// ListenNotification
router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("listennotification", maxClients(gz(httpTraceAll(api.ListenNotificationHandler))))).Queries("events", "{events:.*}")
// ResetBucketReplicationStatus - MinIO extension API
router.Methods(http.MethodGet).HandlerFunc(
collectAPIStats("resetbucketreplicationstatus", maxClients(gz(httpTraceAll(api.ResetBucketReplicationStatusHandler))))).Queries("replication-reset-status", "")
// Dummy Bucket Calls
// GetBucketACL -- this is a dummy call.
@@ -417,9 +420,10 @@ func registerAPIRouter(router *mux.Router) {
// PutBucketNotification
router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("putbucketnotification", maxClients(gz(httpTraceAll(api.PutBucketNotificationHandler))))).Queries("notification", "")
// ResetBucketReplicationState - MinIO extension API
// ResetBucketReplicationStart - MinIO extension API
router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("resetbucketreplicationstate", maxClients(gz(httpTraceAll(api.ResetBucketReplicationStateHandler))))).Queries("replication-reset", "")
collectAPIStats("resetbucketreplicationstart", maxClients(gz(httpTraceAll(api.ResetBucketReplicationStartHandler))))).Queries("replication-reset", "")
// PutBucket
router.Methods(http.MethodPut).HandlerFunc(
collectAPIStats("putbucket", maxClients(gz(httpTraceAll(api.PutBucketHandler)))))

View File

@@ -1301,7 +1301,7 @@ func (api objectAPIHandlers) DeleteBucketHandler(w http.ResponseWriter, r *http.
}
globalNotificationSys.DeleteBucketMetadata(ctx, bucket)
globalReplicationPool.deleteResyncMetadata(ctx, bucket)
// Call site replication hook.
if err := globalSiteReplicationSys.DeleteBucketHook(ctx, bucket, forceDelete); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
@@ -1760,12 +1760,13 @@ func (api objectAPIHandlers) GetBucketReplicationMetricsHandler(w http.ResponseW
}
}
// ResetBucketReplicationStateHandler - starts a replication reset for all objects in a bucket which
// ResetBucketReplicationStartHandler - 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")
// remote target is entirely lost,and previously replicated objects need to be re-synced. If resync is
// already in progress it returns an error
func (api objectAPIHandlers) ResetBucketReplicationStartHandler(w http.ResponseWriter, r *http.Request) {
ctx := newContext(r, w, "ResetBucketReplicationStart")
defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r))
vars := mux.Vars(r)
@@ -1789,6 +1790,7 @@ func (api objectAPIHandlers) ResetBucketReplicationStateHandler(w http.ResponseW
}), r.URL)
}
}
resetBeforeDate := UTCNow().AddDate(0, 0, -1*int(days/24))
objectAPI := api.ObjectAPI()
if objectAPI == nil {
@@ -1825,7 +1827,7 @@ func (api objectAPIHandlers) ResetBucketReplicationStateHandler(w http.ResponseW
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),
Err: fmt.Errorf("Remote target ARN %s missing or ineligible for replication resync", arn),
}), r.URL)
return
}
@@ -1849,22 +1851,90 @@ func (api objectAPIHandlers) ResetBucketReplicationStateHandler(w http.ResponseW
default:
writeErrorResponseJSON(ctx, w, toAPIError(ctx, err), r.URL)
}
}
if err := startReplicationResync(ctx, bucket, arn, resetID, resetBeforeDate, objectAPI); err != nil {
writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErrWithErr(ErrBadRequest, InvalidArgument{
Bucket: bucket,
Err: err,
}), r.URL)
return
}
targets, err := globalBucketTargetSys.ListBucketTargets(ctx, bucket)
if err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return
}
tgtBytes, err := json.Marshal(&targets)
if err != nil {
writeErrorResponseJSON(ctx, w, errorCodes.ToAPIErrWithErr(ErrAdminConfigBadJSON, err), r.URL)
return
}
if err = globalBucketMetadataSys.Update(ctx, bucket, bucketTargetsFile, tgtBytes); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return
}
data, err := json.Marshal(rinfo)
if err != nil {
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
return
}
// Write success response.
writeSuccessResponseJSON(w, data)
}
// ResetBucketReplicationStatusHandler - returns the status of replication reset.
// This API is a MinIO only extension
func (api objectAPIHandlers) ResetBucketReplicationStatusHandler(w http.ResponseWriter, r *http.Request) {
ctx := newContext(r, w, "ResetBucketReplicationStatus")
defer logger.AuditLog(ctx, w, r, mustGetClaimsFromToken(r))
vars := mux.Vars(r)
bucket := vars["bucket"]
arn := r.URL.Query().Get("arn")
var err error
objectAPI := api.ObjectAPI()
if objectAPI == nil {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(ErrServerNotInitialized), r.URL)
return
}
if s3Error := checkRequestAuthType(ctx, r, policy.ResetBucketReplicationStateAction, bucket, ""); s3Error != ErrNone {
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL)
return
}
// Check if bucket exists.
if _, err := objectAPI.GetBucketInfo(ctx, bucket); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return
}
if _, err := globalBucketMetadataSys.GetReplicationConfig(ctx, bucket); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return
}
globalReplicationPool.resyncState.RLock()
brs, ok := globalReplicationPool.resyncState.statusMap[bucket]
if !ok {
brs, err = loadBucketResyncMetadata(ctx, bucket, objectAPI)
if err != nil {
writeErrorResponse(ctx, w, errorCodes.ToAPIErrWithErr(ErrBadRequest, InvalidArgument{
Bucket: bucket,
Err: fmt.Errorf("No replication resync status available for %s", arn),
}), r.URL)
}
return
}
var rinfo ResyncTargetsInfo
for tarn, st := range brs.TargetsMap {
if arn != "" && tarn != arn {
continue
}
rinfo.Targets = append(rinfo.Targets, ResyncTarget{
Arn: tarn,
ResetID: st.ResyncID,
StartTime: st.StartTime,
EndTime: st.EndTime,
ResyncStatus: st.ResyncStatus.String(),
ReplicatedSize: st.ReplicatedSize,
ReplicatedCount: st.ReplicatedCount,
FailedSize: st.FailedSize,
FailedCount: st.FailedCount,
Bucket: st.Bucket,
Object: st.Object,
})
}
globalReplicationPool.resyncState.RUnlock()
data, err := json.Marshal(rinfo)
if err != nil {
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)

View File

@@ -375,6 +375,7 @@ func deleteBucketMetadata(ctx context.Context, obj objectDeleter, bucket string)
metadataFiles := []string{
dataUsageCacheName,
bucketMetadataFile,
path.Join(replicationDir, resyncFileName),
}
for _, metaFile := range metadataFiles {
configFile := path.Join(bucketMetaPrefix, bucket, metaFile)

View File

@@ -23,6 +23,7 @@ import (
"regexp"
"strconv"
"strings"
"sync"
"time"
"github.com/minio/minio/internal/bucket/replication"
@@ -576,8 +577,23 @@ type ResyncTargetsInfo struct {
// 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"`
Arn string `json:"arn"`
ResetID string `json:"resetid"`
StartTime time.Time `json:"startTime"`
EndTime time.Time `json:"endTime"`
// Status of resync operation
ResyncStatus string `json:"resyncStatus,omitempty"`
// Completed size in bytes
ReplicatedSize int64 `json:"completedReplicationSize"`
// Failed size in bytes
FailedSize int64 `json:"failedReplicationSize"`
// Total number of failed operations
FailedCount int64 `json:"failedReplicationCount"`
// Total number of failed operations
ReplicatedCount int64 `json:"replicationCount"`
// Last bucket/object replicated.
Bucket string `json:"bucket,omitempty"`
Object string `json:"object,omitempty"`
}
// VersionPurgeStatusType represents status of a versioned delete or permanent delete w.r.t bucket replication
@@ -603,3 +619,83 @@ func (v VersionPurgeStatusType) Empty() bool {
func (v VersionPurgeStatusType) Pending() bool {
return v == Pending || v == Failed
}
type replicationResyncState struct {
// map of bucket to their resync status
statusMap map[string]BucketReplicationResyncStatus
sync.RWMutex
}
const (
replicationDir = "replication"
resyncFileName = "resync.bin"
resyncMetaFormat = 1
resyncMetaVersionV1 = 1
resyncMetaVersion = resyncMetaVersionV1
)
// ResyncStatusType status of resync operation
type ResyncStatusType int
const (
// NoResync - no resync in progress
NoResync ResyncStatusType = iota
// ResyncStarted - resync in progress
ResyncStarted
// ResyncCompleted - resync finished
ResyncCompleted
// ResyncFailed - resync failed
ResyncFailed
)
func (rt ResyncStatusType) String() string {
switch rt {
case ResyncStarted:
return "Ongoing"
case ResyncCompleted:
return "Completed"
case ResyncFailed:
return "Failed"
default:
return ""
}
}
// TargetReplicationResyncStatus status of resync of bucket for a specific target
type TargetReplicationResyncStatus struct {
StartTime time.Time `json:"startTime" msg:"st"`
EndTime time.Time `json:"endTime" msg:"et"`
// Resync ID assigned to this reset
ResyncID string `json:"resyncID" msg:"id"`
// ResyncBeforeDate - resync all objects created prior to this date
ResyncBeforeDate time.Time `json:"resyncBeforeDate" msg:"rdt"`
// Status of resync operation
ResyncStatus ResyncStatusType `json:"resyncStatus" msg:"rst"`
// Failed size in bytes
FailedSize int64 `json:"failedReplicationSize" msg:"fs"`
// Total number of failed operations
FailedCount int64 `json:"failedReplicationCount" msg:"frc"`
// Completed size in bytes
ReplicatedSize int64 `json:"completedReplicationSize" msg:"rs"`
// Total number of failed operations
ReplicatedCount int64 `json:"replicationCount" msg:"rrc"`
// Last bucket/object replicated.
Bucket string `json:"-" msg:"bkt"`
Object string `json:"-" msg:"obj"`
}
// BucketReplicationResyncStatus captures current replication resync status
type BucketReplicationResyncStatus struct {
Version int `json:"version" msg:"v"`
// map of remote arn to their resync status for a bucket
TargetsMap map[string]TargetReplicationResyncStatus `json:"resyncMap,omitempty" msg:"brs"`
ID int `json:"id" msg:"id"`
LastUpdate time.Time `json:"lastUpdate" msg:"lu"`
}
func newBucketResyncStatus(bucket string) BucketReplicationResyncStatus {
return BucketReplicationResyncStatus{
TargetsMap: make(map[string]TargetReplicationResyncStatus),
Version: resyncMetaVersion,
}
}

File diff suppressed because it is too large Load Diff

View File

@@ -9,6 +9,119 @@ import (
"github.com/tinylib/msgp/msgp"
)
func TestMarshalUnmarshalBucketReplicationResyncStatus(t *testing.T) {
v := BucketReplicationResyncStatus{}
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 BenchmarkMarshalMsgBucketReplicationResyncStatus(b *testing.B) {
v := BucketReplicationResyncStatus{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgBucketReplicationResyncStatus(b *testing.B) {
v := BucketReplicationResyncStatus{}
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 BenchmarkUnmarshalBucketReplicationResyncStatus(b *testing.B) {
v := BucketReplicationResyncStatus{}
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 TestEncodeDecodeBucketReplicationResyncStatus(t *testing.T) {
v := BucketReplicationResyncStatus{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeBucketReplicationResyncStatus Msgsize() is inaccurate")
}
vn := BucketReplicationResyncStatus{}
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 BenchmarkEncodeBucketReplicationResyncStatus(b *testing.B) {
v := BucketReplicationResyncStatus{}
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 BenchmarkDecodeBucketReplicationResyncStatus(b *testing.B) {
v := BucketReplicationResyncStatus{}
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 TestMarshalUnmarshalReplicateDecision(t *testing.T) {
v := ReplicateDecision{}
bts, err := v.MarshalMsg(nil)
@@ -686,3 +799,116 @@ func BenchmarkDecodeResyncTargetsInfo(b *testing.B) {
}
}
}
func TestMarshalUnmarshalTargetReplicationResyncStatus(t *testing.T) {
v := TargetReplicationResyncStatus{}
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 BenchmarkMarshalMsgTargetReplicationResyncStatus(b *testing.B) {
v := TargetReplicationResyncStatus{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgTargetReplicationResyncStatus(b *testing.B) {
v := TargetReplicationResyncStatus{}
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 BenchmarkUnmarshalTargetReplicationResyncStatus(b *testing.B) {
v := TargetReplicationResyncStatus{}
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 TestEncodeDecodeTargetReplicationResyncStatus(t *testing.T) {
v := TargetReplicationResyncStatus{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeTargetReplicationResyncStatus Msgsize() is inaccurate")
}
vn := TargetReplicationResyncStatus{}
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 BenchmarkEncodeTargetReplicationResyncStatus(b *testing.B) {
v := TargetReplicationResyncStatus{}
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 BenchmarkDecodeTargetReplicationResyncStatus(b *testing.B) {
v := TargetReplicationResyncStatus{}
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

@@ -19,10 +19,13 @@ package cmd
import (
"context"
"encoding/binary"
"errors"
"fmt"
"io"
"math"
"net/http"
"path"
"reflect"
"strings"
"sync"
@@ -998,6 +1001,7 @@ func replicateObjectToTarget(ctx context.Context, ri ReplicateObjectInfo, object
OpType: ri.OpType,
ReplicationAction: rAction,
}
if ri.ObjectInfo.TargetReplicationStatus(tgt.ARN) == replication.Completed && !ri.ExistingObjResync.Empty() && !ri.ExistingObjResync.mustResyncTarget(tgt.ARN) {
rinfo.ReplicationStatus = replication.Completed
rinfo.ReplicationResynced = true
@@ -1057,6 +1061,13 @@ func replicateObjectToTarget(ctx context.Context, ri ReplicateObjectInfo, object
})
return rinfo
}
defer func() {
if rinfo.ReplicationStatus == replication.Completed && ri.OpType == replication.ExistingObjectReplicationType && tgt.ResetID != "" {
rinfo.ResyncTimestamp = fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), tgt.ResetID)
rinfo.ReplicationResynced = true
}
rinfo.Duration = time.Since(startTime)
}()
rAction = replicateAll
oi, cerr := tgt.StatObject(ctx, tgt.Bucket, object, miniogo.StatObjectOptions{
@@ -1089,13 +1100,8 @@ func replicateObjectToTarget(ctx context.Context, ri ReplicateObjectInfo, object
// Note: Replication Stats would have been updated despite metadata update failure.
gr.Close()
closeOnDefer = false
return replicatedTargetInfo{
ReplicationStatus: replication.Completed,
Size: sz,
Arn: tgt.ARN,
ReplicationAction: rAction,
PrevReplicationStatus: objInfo.TargetReplicationStatus(tgt.ARN),
}
rinfo.ReplicationAction = rAction
rinfo.ReplicationStatus = replication.Completed
}
return
}
@@ -1103,13 +1109,6 @@ func replicateObjectToTarget(ctx context.Context, ri ReplicateObjectInfo, object
rinfo.ReplicationStatus = replication.Completed
rinfo.Size = size
rinfo.ReplicationAction = rAction
defer func() {
if rinfo.ReplicationStatus == replication.Completed && ri.OpType == replication.ExistingObjectReplicationType && tgt.ResetID != "" {
rinfo.ResyncTimestamp = fmt.Sprintf("%s;%s", UTCNow().Format(http.TimeFormat), tgt.ResetID)
rinfo.ReplicationResynced = true
}
rinfo.Duration = time.Since(startTime)
}()
// use core client to avoid doing multipart on PUT
c := &miniogo.Core{Client: tgt.Client}
if rAction != replicateAll {
@@ -1308,6 +1307,7 @@ type ReplicationPool struct {
existingReplicaDeleteCh chan DeletedObjectReplicationInfo
workerSize int
mrfWorkerSize int
resyncState replicationResyncState
workerWg sync.WaitGroup
mrfWorkerWg sync.WaitGroup
once sync.Once
@@ -1324,6 +1324,7 @@ func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPool
mrfWorkerKillCh: make(chan struct{}, opts.FailedWorkers),
existingReplicaCh: make(chan ReplicateObjectInfo, 100000),
existingReplicaDeleteCh: make(chan DeletedObjectReplicationInfo, 100000),
resyncState: replicationResyncState{statusMap: make(map[string]BucketReplicationResyncStatus)},
ctx: ctx,
objLayer: o,
}
@@ -1331,6 +1332,7 @@ func NewReplicationPool(ctx context.Context, o ObjectLayer, opts replicationPool
pool.ResizeWorkers(opts.Workers)
pool.ResizeFailedWorkers(opts.FailedWorkers)
go pool.AddExistingObjectReplicateWorker()
go pool.periodicResyncMetaSave(ctx, o)
return pool
}
@@ -1870,3 +1872,333 @@ func getLatestReplicationStats(bucket string, u BucketUsageInfo) (s BucketReplic
s.ReplicatedSize = int64(math.Max(float64(s.ReplicatedSize), float64(latestTotReplicatedSize)))
return s
}
const resyncTimeInterval = time.Minute * 10
// periodicResyncMetaSave saves in-memory resync meta stats to disk in periodic intervals
func (p *ReplicationPool) periodicResyncMetaSave(ctx context.Context, objectAPI ObjectLayer) {
resyncTimer := time.NewTimer(resyncTimeInterval)
defer resyncTimer.Stop()
for {
select {
case <-resyncTimer.C:
resyncTimer.Reset(resyncTimeInterval)
now := UTCNow()
p.resyncState.RLock()
for bucket, brs := range p.resyncState.statusMap {
var updt bool
for _, st := range brs.TargetsMap {
// if resync in progress or just ended, needs to save to disk
if st.EndTime.Equal(timeSentinel) || now.Sub(st.EndTime) <= resyncTimeInterval {
updt = true
break
}
}
if updt {
brs.LastUpdate = now
if err := saveResyncStatus(ctx, bucket, brs, objectAPI); err != nil {
logger.LogIf(ctx, fmt.Errorf("Could not save resync metadata to disk for %s - %w", bucket, err))
continue
}
}
}
p.resyncState.RUnlock()
case <-ctx.Done():
// server could be restarting - need
// to exit immediately
return
}
}
}
// resyncBucket resyncs all qualifying objects as per replication rules for the target
// ARN
func resyncBucket(ctx context.Context, bucket, arn string, heal bool, objectAPI ObjectLayer) {
resyncStatus := ResyncFailed
defer func() {
globalReplicationPool.resyncState.Lock()
m := globalReplicationPool.resyncState.statusMap[bucket]
st := m.TargetsMap[arn]
st.EndTime = UTCNow()
st.ResyncStatus = resyncStatus
m.TargetsMap[arn] = st
globalReplicationPool.resyncState.Unlock()
}()
// Allocate new results channel to receive ObjectInfo.
objInfoCh := make(chan ObjectInfo)
cfg, err := getReplicationConfig(ctx, bucket)
if err != nil {
logger.LogIf(ctx, fmt.Errorf("Replication resync of %s for arn %s failed with %w", bucket, arn, err))
return
}
tgts, err := globalBucketTargetSys.ListBucketTargets(ctx, bucket)
if err != nil {
logger.LogIf(ctx, fmt.Errorf("Replication resync of %s for arn %s failed %w", bucket, arn, err))
return
}
rcfg := replicationConfig{
Config: cfg,
remotes: tgts,
}
tgtArns := cfg.FilterTargetArns(
replication.ObjectOpts{
OpType: replication.ResyncReplicationType,
TargetArn: arn,
})
if len(tgtArns) != 1 {
logger.LogIf(ctx, fmt.Errorf("Replication resync failed for %s - arn specified %s is missing in the replication config", bucket, arn))
return
}
tgt := globalBucketTargetSys.GetRemoteTargetClient(ctx, arn)
if tgt == nil {
logger.LogIf(ctx, fmt.Errorf("Replication resync failed for %s - target could not be created for arn %s", bucket, arn))
return
}
// Walk through all object versions - note ascending order of walk needed to ensure delete marker replicated to
// target after object version is first created.
if err := objectAPI.Walk(ctx, bucket, "", objInfoCh, ObjectOptions{WalkAscending: true}); err != nil {
logger.LogIf(ctx, err)
return
}
globalReplicationPool.resyncState.RLock()
m := globalReplicationPool.resyncState.statusMap[bucket]
st := m.TargetsMap[arn]
globalReplicationPool.resyncState.RUnlock()
var lastCheckpoint string
if st.ResyncStatus == ResyncStarted || st.ResyncStatus == ResyncFailed {
lastCheckpoint = st.Object
}
for obj := range objInfoCh {
if heal && lastCheckpoint != "" && lastCheckpoint != obj.Name {
continue
}
lastCheckpoint = ""
roi := getHealReplicateObjectInfo(obj, rcfg)
if !roi.ExistingObjResync.mustResync() {
continue
}
if roi.DeleteMarker || !roi.VersionPurgeStatus.Empty() {
versionID := ""
dmVersionID := ""
if roi.VersionPurgeStatus.Empty() {
dmVersionID = roi.VersionID
} else {
versionID = roi.VersionID
}
doi := DeletedObjectReplicationInfo{
DeletedObject: DeletedObject{
ObjectName: roi.Name,
DeleteMarkerVersionID: dmVersionID,
VersionID: versionID,
ReplicationState: roi.getReplicationState(roi.Dsc.String(), versionID, true),
DeleteMarkerMTime: DeleteMarkerMTime{roi.ModTime},
DeleteMarker: roi.DeleteMarker,
},
Bucket: roi.Bucket,
OpType: replication.ExistingObjectReplicationType,
}
replicateDelete(ctx, doi, objectAPI, ReplicateDelete)
} else {
roi.OpType = replication.ExistingObjectReplicationType
replicateObject(ctx, roi, objectAPI, ReplicateExisting)
}
_, err = tgt.StatObject(ctx, tgt.Bucket, roi.Name, miniogo.StatObjectOptions{
VersionID: roi.VersionID,
Internal: miniogo.AdvancedGetOptions{
ReplicationProxyRequest: "false",
},
})
globalReplicationPool.resyncState.Lock()
m = globalReplicationPool.resyncState.statusMap[bucket]
st = m.TargetsMap[arn]
st.Object = roi.Name
if err != nil {
if roi.DeleteMarker && isErrMethodNotAllowed(ErrorRespToObjectError(err, bucket, roi.Name)) {
st.ReplicatedCount++
} else {
st.FailedCount++
}
} else {
st.ReplicatedCount++
st.ReplicatedSize += roi.Size
}
m.TargetsMap[arn] = st
globalReplicationPool.resyncState.Unlock()
}
resyncStatus = ResyncCompleted
}
// start replication resync for the remote target ARN specified
func startReplicationResync(ctx context.Context, bucket, arn, resyncID string, resyncBeforeDate time.Time, objAPI ObjectLayer) error {
if bucket == "" {
return fmt.Errorf("bucket name is empty")
}
if arn == "" {
return fmt.Errorf("target ARN specified for resync is empty")
}
// Check if the current bucket has quota restrictions, if not skip it
cfg, err := getReplicationConfig(ctx, bucket)
if err != nil {
return err
}
tgtArns := cfg.FilterTargetArns(
replication.ObjectOpts{
OpType: replication.ResyncReplicationType,
TargetArn: arn,
})
if len(tgtArns) == 0 {
return fmt.Errorf("arn %s specified for resync not found in replication config", arn)
}
data, err := loadBucketResyncMetadata(ctx, bucket, objAPI)
if err != nil {
return err
}
// validate if resync is in progress for this arn
for tArn, st := range data.TargetsMap {
if arn == tArn && st.ResyncStatus == ResyncStarted {
return fmt.Errorf("Resync of bucket %s is already in progress for remote bucket %s", bucket, arn)
}
}
status := TargetReplicationResyncStatus{
ResyncID: resyncID,
ResyncBeforeDate: resyncBeforeDate,
StartTime: UTCNow(),
ResyncStatus: ResyncStarted,
Bucket: bucket,
}
data.TargetsMap[arn] = status
if err = saveResyncStatus(ctx, bucket, data, objAPI); err != nil {
return err
}
globalReplicationPool.resyncState.Lock()
defer globalReplicationPool.resyncState.Unlock()
brs, ok := globalReplicationPool.resyncState.statusMap[bucket]
if !ok {
brs = BucketReplicationResyncStatus{
Version: resyncMetaVersion,
TargetsMap: make(map[string]TargetReplicationResyncStatus),
}
}
brs.TargetsMap[arn] = status
globalReplicationPool.resyncState.statusMap[bucket] = brs
go resyncBucket(GlobalContext, bucket, arn, false, objAPI)
return nil
}
// delete resync metadata from replication resync state in memory
func (p *ReplicationPool) deleteResyncMetadata(ctx context.Context, bucket string) {
if p == nil {
return
}
p.resyncState.Lock()
delete(p.resyncState.statusMap, bucket)
defer p.resyncState.Unlock()
}
// initResync - initializes bucket replication resync for all buckets.
func (p *ReplicationPool) initResync(ctx context.Context, buckets []BucketInfo, objAPI ObjectLayer) error {
if objAPI == nil {
return errServerNotInitialized
}
// replication applies only to erasure coded setups
if !globalIsErasure {
return nil
}
// Load bucket metadata sys in background
go p.loadResync(ctx, buckets, objAPI)
return nil
}
// Loads bucket replication resync statuses into memory.
func (p *ReplicationPool) loadResync(ctx context.Context, buckets []BucketInfo, objAPI ObjectLayer) {
for index := range buckets {
meta, err := loadBucketResyncMetadata(ctx, buckets[index].Name, objAPI)
if err != nil {
if errors.Is(err, errVolumeNotFound) {
meta = newBucketResyncStatus(buckets[index].Name)
} else {
logger.LogIf(ctx, err)
continue
}
}
p.resyncState.statusMap[buckets[index].Name] = meta
}
for index := range buckets {
bucket := buckets[index].Name
m, ok := p.resyncState.statusMap[bucket]
if ok {
for arn, st := range m.TargetsMap {
if st.ResyncStatus == ResyncFailed || st.ResyncStatus == ResyncStarted {
go resyncBucket(ctx, bucket, arn, true, objAPI)
}
}
}
}
}
// load bucket resync metadata from disk
func loadBucketResyncMetadata(ctx context.Context, bucket string, objAPI ObjectLayer) (brs BucketReplicationResyncStatus, e error) {
brs = newBucketResyncStatus(bucket)
resyncDirPath := path.Join(bucketMetaPrefix, bucket, replicationDir)
data, err := readConfig(GlobalContext, objAPI, pathJoin(resyncDirPath, resyncFileName))
if err != nil && err != errConfigNotFound {
return brs, err
}
if len(data) == 0 {
// Seems to be empty.
return brs, nil
}
if len(data) <= 4 {
return brs, fmt.Errorf("replication resync: no data")
}
// Read resync meta header
switch binary.LittleEndian.Uint16(data[0:2]) {
case resyncMetaFormat:
default:
return brs, fmt.Errorf("resyncMeta: unknown format: %d", binary.LittleEndian.Uint16(data[0:2]))
}
switch binary.LittleEndian.Uint16(data[2:4]) {
case resyncMetaVersion:
default:
return brs, fmt.Errorf("resyncMeta: unknown version: %d", binary.LittleEndian.Uint16(data[2:4]))
}
// OK, parse data.
if _, err = brs.UnmarshalMsg(data[4:]); err != nil {
return brs, err
}
switch brs.Version {
case resyncMetaVersionV1:
default:
return brs, fmt.Errorf("unexpected resync meta version: %d", brs.Version)
}
return brs, nil
}
// save resync status to resync.bin
func saveResyncStatus(ctx context.Context, bucket string, brs BucketReplicationResyncStatus, objectAPI ObjectLayer) error {
data := make([]byte, 4, brs.Msgsize()+4)
// Initialize the resync meta header.
binary.LittleEndian.PutUint16(data[0:2], resyncMetaFormat)
binary.LittleEndian.PutUint16(data[2:4], resyncMetaVersion)
buf, err := brs.MarshalMsg(data)
if err != nil {
return err
}
configFile := path.Join(bucketMetaPrefix, bucket, replicationDir, resyncFileName)
return saveConfig(ctx, objectAPI, configFile, buf)
}

View File

@@ -1668,7 +1668,13 @@ func (z *erasureServerPools) Walk(ctx context.Context, bucket, prefix string, re
cancel()
return
}
if opts.WalkAscending {
for i := len(fivs.Versions) - 1; i >= 0; i-- {
version := fivs.Versions[i]
results <- version.ToObjectInfo(bucket, version.Name)
}
return
}
for _, version := range fivs.Versions {
results <- version.ToObjectInfo(bucket, version.Name)
}

View File

@@ -72,7 +72,8 @@ type ObjectOptions struct {
MaxParity bool
// Mutate set to 'true' if the call is namespace mutation call
Mutate bool
Mutate bool
WalkAscending bool // return Walk results in ascending order of versions
}
// ExpirationOptions represents object options for object expiration at objectLayer.

View File

@@ -577,6 +577,8 @@ func serverMain(ctx *cli.Context) {
if err != nil {
logger.LogIf(GlobalContext, fmt.Errorf("Unable to list buckets to heal: %w", err))
}
// initialize replication resync state.
go globalReplicationPool.initResync(GlobalContext, buckets, newObject)
// Populate existing buckets to the etcd backend
if globalDNSConfig != nil {