Replicate Expiry ILM configs while site replication (#18130)

Signed-off-by: Shubhendu Ram Tripathi <shubhendu@minio.io>
This commit is contained in:
Shubhendu 2023-11-21 23:18:06 +05:30 committed by GitHub
parent 41091d9472
commit 58306a9d34
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 1123 additions and 63 deletions

View File

@ -52,7 +52,8 @@ func (a adminAPIHandlers) SiteReplicationAdd(w http.ResponseWriter, r *http.Requ
return return
} }
status, err := globalSiteReplicationSys.AddPeerClusters(ctx, sites) opts := getSRAddOptions(r)
status, err := globalSiteReplicationSys.AddPeerClusters(ctx, sites, opts)
if err != nil { if err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL) writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
@ -68,6 +69,12 @@ func (a adminAPIHandlers) SiteReplicationAdd(w http.ResponseWriter, r *http.Requ
writeSuccessResponseJSON(w, body) writeSuccessResponseJSON(w, body)
} }
func getSRAddOptions(r *http.Request) (opts madmin.SRAddOptions) {
q := r.Form
opts.ReplicateILMExpiry = q.Get("replicateILMExpiry") == "true"
return
}
// SRPeerJoin - PUT /minio/admin/v3/site-replication/join // SRPeerJoin - PUT /minio/admin/v3/site-replication/join
// //
// used internally to tell current cluster to enable SR with // used internally to tell current cluster to enable SR with
@ -192,7 +199,7 @@ func (a adminAPIHandlers) SRPeerReplicateIAMItem(w http.ResponseWriter, r *http.
} }
} }
// SRPeerReplicateBucketItem - PUT /minio/admin/v3/site-replication/bucket-meta // SRPeerReplicateBucketItem - PUT /minio/admin/v3/site-replication/peer/bucket-meta
func (a adminAPIHandlers) SRPeerReplicateBucketItem(w http.ResponseWriter, r *http.Request) { func (a adminAPIHandlers) SRPeerReplicateBucketItem(w http.ResponseWriter, r *http.Request) {
ctx := r.Context() ctx := r.Context()
@ -253,6 +260,8 @@ func (a adminAPIHandlers) SRPeerReplicateBucketItem(w http.ResponseWriter, r *ht
err = globalSiteReplicationSys.PeerBucketObjectLockConfigHandler(ctx, item.Bucket, item.ObjectLockConfig, item.UpdatedAt) err = globalSiteReplicationSys.PeerBucketObjectLockConfigHandler(ctx, item.Bucket, item.ObjectLockConfig, item.UpdatedAt)
case madmin.SRBucketMetaTypeSSEConfig: case madmin.SRBucketMetaTypeSSEConfig:
err = globalSiteReplicationSys.PeerBucketSSEConfigHandler(ctx, item.Bucket, item.SSEConfig, item.UpdatedAt) err = globalSiteReplicationSys.PeerBucketSSEConfigHandler(ctx, item.Bucket, item.SSEConfig, item.UpdatedAt)
case madmin.SRBucketMetaLCConfig:
err = globalSiteReplicationSys.PeerBucketLCConfigHandler(ctx, item.Bucket, item.ExpiryLCConfig, item.UpdatedAt)
} }
if err != nil { if err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
@ -334,6 +343,7 @@ func (a adminAPIHandlers) SiteReplicationStatus(w http.ResponseWriter, r *http.R
opts.Users = true opts.Users = true
opts.Policies = true opts.Policies = true
opts.Groups = true opts.Groups = true
opts.ILMExpiryRules = true
} }
info, err := globalSiteReplicationSys.SiteReplicationStatus(ctx, objectAPI, opts) info, err := globalSiteReplicationSys.SiteReplicationStatus(ctx, objectAPI, opts)
if err != nil { if err != nil {
@ -383,7 +393,9 @@ func (a adminAPIHandlers) SiteReplicationEdit(w http.ResponseWriter, r *http.Req
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL) writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
return return
} }
status, err := globalSiteReplicationSys.EditPeerCluster(ctx, site)
opts := getSREditOptions(r)
status, err := globalSiteReplicationSys.EditPeerCluster(ctx, site, opts)
if err != nil { if err != nil {
logger.LogIf(ctx, err) logger.LogIf(ctx, err)
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL) writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
@ -398,6 +410,13 @@ func (a adminAPIHandlers) SiteReplicationEdit(w http.ResponseWriter, r *http.Req
writeSuccessResponseJSON(w, body) writeSuccessResponseJSON(w, body)
} }
func getSREditOptions(r *http.Request) (opts madmin.SREditOptions) {
q := r.Form
opts.DisableILMExpiryReplication = q.Get("disableILMExpiryReplication") == "true"
opts.EnableILMExpiryReplication = q.Get("enableILMExpiryReplication") == "true"
return
}
// SRPeerEdit - PUT /minio/admin/v3/site-replication/peer/edit // SRPeerEdit - PUT /minio/admin/v3/site-replication/peer/edit
// //
// used internally to tell current cluster to update endpoint for peer // used internally to tell current cluster to update endpoint for peer
@ -422,12 +441,37 @@ func (a adminAPIHandlers) SRPeerEdit(w http.ResponseWriter, r *http.Request) {
} }
} }
// SRStateEdit - PUT /minio/admin/v3/site-replication/state/edit
//
// used internally to tell current cluster to update site replication state
func (a adminAPIHandlers) SRStateEdit(w http.ResponseWriter, r *http.Request) {
ctx := r.Context()
objectAPI, _ := validateAdminReq(ctx, w, r, policy.SiteReplicationOperationAction)
if objectAPI == nil {
return
}
var state madmin.SRStateEditReq
if err := parseJSONBody(ctx, r.Body, &state, ""); err != nil {
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
return
}
if err := globalSiteReplicationSys.PeerStateEditReq(ctx, state); err != nil {
logger.LogIf(ctx, err)
writeErrorResponseJSON(ctx, w, toAdminAPIErr(ctx, err), r.URL)
return
}
}
func getSRStatusOptions(r *http.Request) (opts madmin.SRStatusOptions) { func getSRStatusOptions(r *http.Request) (opts madmin.SRStatusOptions) {
q := r.Form q := r.Form
opts.Buckets = q.Get("buckets") == "true" opts.Buckets = q.Get("buckets") == "true"
opts.Policies = q.Get("policies") == "true" opts.Policies = q.Get("policies") == "true"
opts.Groups = q.Get("groups") == "true" opts.Groups = q.Get("groups") == "true"
opts.Users = q.Get("users") == "true" opts.Users = q.Get("users") == "true"
opts.ILMExpiryRules = q.Get("ilm-expiry-rules") == "true"
opts.PeerState = q.Get("peer-state") == "true"
opts.Entity = madmin.GetSREntityType(q.Get("entity")) opts.Entity = madmin.GetSREntityType(q.Get("entity"))
opts.EntityValue = q.Get("entityvalue") opts.EntityValue = q.Get("entityvalue")
opts.ShowDeleted = q.Get("showDeleted") == "true" opts.ShowDeleted = q.Get("showDeleted") == "true"

View File

@ -376,6 +376,7 @@ func registerAdminRouter(router *mux.Router, enableConfigOps bool) {
adminRouter.Methods(http.MethodPut).Path(adminVersion + "/site-replication/peer/edit").HandlerFunc(adminMiddleware(adminAPI.SRPeerEdit)) adminRouter.Methods(http.MethodPut).Path(adminVersion + "/site-replication/peer/edit").HandlerFunc(adminMiddleware(adminAPI.SRPeerEdit))
adminRouter.Methods(http.MethodPut).Path(adminVersion + "/site-replication/peer/remove").HandlerFunc(adminMiddleware(adminAPI.SRPeerRemove)) adminRouter.Methods(http.MethodPut).Path(adminVersion + "/site-replication/peer/remove").HandlerFunc(adminMiddleware(adminAPI.SRPeerRemove))
adminRouter.Methods(http.MethodPut).Path(adminVersion+"/site-replication/resync/op").HandlerFunc(adminMiddleware(adminAPI.SiteReplicationResyncOp)).Queries("operation", "{operation:.*}") adminRouter.Methods(http.MethodPut).Path(adminVersion+"/site-replication/resync/op").HandlerFunc(adminMiddleware(adminAPI.SiteReplicationResyncOp)).Queries("operation", "{operation:.*}")
adminRouter.Methods(http.MethodPut).Path(adminVersion + "/site-replication/state/edit").HandlerFunc(adminMiddleware(adminAPI.SRStateEdit))
if globalIsDistErasure { if globalIsDistErasure {
// Top locks // Top locks

View File

@ -22,6 +22,7 @@ import (
"io" "io"
"net/http" "net/http"
"strconv" "strconv"
"time"
"github.com/minio/minio/internal/bucket/lifecycle" "github.com/minio/minio/internal/bucket/lifecycle"
xhttp "github.com/minio/minio/internal/http" xhttp "github.com/minio/minio/internal/http"
@ -86,6 +87,41 @@ func (api objectAPIHandlers) PutBucketLifecycleHandler(w http.ResponseWriter, r
return return
} }
// Create a map of updated set of rules in request
updatedRules := make(map[string]lifecycle.Rule, len(bucketLifecycle.Rules))
for _, rule := range bucketLifecycle.Rules {
updatedRules[rule.ID] = rule
}
// Get list of rules for the bucket from disk
meta, err := globalBucketMetadataSys.GetConfigFromDisk(ctx, bucket)
if err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return
}
expiryRuleRemoved := false
if len(meta.LifecycleConfigXML) > 0 {
var lcCfg lifecycle.Lifecycle
if err := xml.Unmarshal(meta.LifecycleConfigXML, &lcCfg); err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return
}
for _, rl := range lcCfg.Rules {
updRule, ok := updatedRules[rl.ID]
// original rule had expiry that is no longer in the new config,
// or rule is present but missing expiration flags
if (!rl.Expiration.IsNull() || !rl.NoncurrentVersionExpiration.IsNull()) &&
(!ok || (updRule.Expiration.IsNull() && updRule.NoncurrentVersionExpiration.IsNull())) {
expiryRuleRemoved = true
}
}
}
if bucketLifecycle.HasExpiry() || expiryRuleRemoved {
currtime := time.Now()
bucketLifecycle.ExpiryUpdatedAt = &currtime
}
configData, err := xml.Marshal(bucketLifecycle) configData, err := xml.Marshal(bucketLifecycle)
if err != nil { if err != nil {
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
@ -142,6 +178,8 @@ func (api objectAPIHandlers) GetBucketLifecycleHandler(w http.ResponseWriter, r
writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL) writeErrorResponse(ctx, w, toAPIError(ctx, err), r.URL)
return return
} }
// explicitly set ExpiryUpdatedAt nil as its meant for internal consumption only
config.ExpiryUpdatedAt = nil
configData, err := xml.Marshal(config) configData, err := xml.Marshal(config)
if err != nil { if err != nil {

View File

@ -19,6 +19,7 @@ package cmd
import ( import (
"context" "context"
"encoding/xml"
"errors" "errors"
"fmt" "fmt"
"sync" "sync"
@ -177,6 +178,40 @@ func (sys *BucketMetadataSys) save(ctx context.Context, meta BucketMetadata) err
// Delete delete the bucket metadata for the specified bucket. // Delete delete the bucket metadata for the specified bucket.
// must be used by all callers instead of using Update() with nil configData. // must be used by all callers instead of using Update() with nil configData.
func (sys *BucketMetadataSys) Delete(ctx context.Context, bucket string, configFile string) (updatedAt time.Time, err error) { func (sys *BucketMetadataSys) Delete(ctx context.Context, bucket string, configFile string) (updatedAt time.Time, err error) {
if configFile == bucketLifecycleConfig {
// Get bucket config from current site
meta, e := globalBucketMetadataSys.GetConfigFromDisk(ctx, bucket)
if e != nil && !errors.Is(e, errConfigNotFound) {
return updatedAt, e
}
var expiryRuleRemoved bool
if len(meta.LifecycleConfigXML) > 0 {
var lcCfg lifecycle.Lifecycle
if err := xml.Unmarshal(meta.LifecycleConfigXML, &lcCfg); err != nil {
return updatedAt, err
}
// find a single expiry rule set the flag
for _, rl := range lcCfg.Rules {
if !rl.Expiration.IsNull() || !rl.NoncurrentVersionExpiration.IsNull() {
expiryRuleRemoved = true
break
}
}
}
// Form empty ILM details with `ExpiryUpdatedAt` field and save
var cfgData []byte
if expiryRuleRemoved {
var lcCfg lifecycle.Lifecycle
currtime := time.Now()
lcCfg.ExpiryUpdatedAt = &currtime
cfgData, err = xml.Marshal(lcCfg)
if err != nil {
return updatedAt, err
}
}
return sys.updateAndParse(ctx, bucket, configFile, cfgData, false)
}
return sys.updateAndParse(ctx, bucket, configFile, nil, false) return sys.updateAndParse(ctx, bucket, configFile, nil, false)
} }
@ -267,7 +302,10 @@ func (sys *BucketMetadataSys) GetLifecycleConfig(bucket string) (*lifecycle.Life
} }
return nil, time.Time{}, err return nil, time.Time{}, err
} }
if meta.lifecycleConfig == nil { // there could be just `ExpiryUpdatedAt` field populated as part
// of last delete all. Treat this situation as not lifecycle configuration
// available
if meta.lifecycleConfig == nil || len(meta.lifecycleConfig.Rules) == 0 {
return nil, time.Time{}, BucketLifecycleNotFound{Bucket: bucket} return nil, time.Time{}, BucketLifecycleNotFound{Bucket: bucket}
} }
return meta.lifecycleConfig, meta.LifecycleConfigUpdatedAt, nil return meta.lifecycleConfig, meta.LifecycleConfigUpdatedAt, nil

View File

@ -41,6 +41,7 @@ import (
"github.com/minio/minio-go/v7/pkg/replication" "github.com/minio/minio-go/v7/pkg/replication"
"github.com/minio/minio-go/v7/pkg/set" "github.com/minio/minio-go/v7/pkg/set"
"github.com/minio/minio/internal/auth" "github.com/minio/minio/internal/auth"
"github.com/minio/minio/internal/bucket/lifecycle"
sreplication "github.com/minio/minio/internal/bucket/replication" sreplication "github.com/minio/minio/internal/bucket/replication"
"github.com/minio/minio/internal/logger" "github.com/minio/minio/internal/logger"
"github.com/minio/pkg/v2/policy" "github.com/minio/pkg/v2/policy"
@ -213,6 +214,7 @@ type srStateV1 struct {
// Peers maps peers by their deploymentID // Peers maps peers by their deploymentID
Peers map[string]madmin.PeerInfo `json:"peers"` Peers map[string]madmin.PeerInfo `json:"peers"`
ServiceAccountAccessKey string `json:"serviceAccountAccessKey"` ServiceAccountAccessKey string `json:"serviceAccountAccessKey"`
UpdatedAt time.Time `json:"updatedAt"`
} }
// srStateData represents the format of the current `srStateFile`. // srStateData represents the format of the current `srStateFile`.
@ -379,7 +381,7 @@ func (c *SiteReplicationSys) getSiteStatuses(ctx context.Context, sites ...madmi
} }
// AddPeerClusters - add cluster sites for replication configuration. // AddPeerClusters - add cluster sites for replication configuration.
func (c *SiteReplicationSys) AddPeerClusters(ctx context.Context, psites []madmin.PeerSite) (madmin.ReplicateAddStatus, error) { func (c *SiteReplicationSys) AddPeerClusters(ctx context.Context, psites []madmin.PeerSite, opts madmin.SRAddOptions) (madmin.ReplicateAddStatus, error) {
sites, serr := c.getSiteStatuses(ctx, psites...) sites, serr := c.getSiteStatuses(ctx, psites...)
if serr != nil { if serr != nil {
return madmin.ReplicateAddStatus{}, serr return madmin.ReplicateAddStatus{}, serr
@ -483,17 +485,39 @@ func (c *SiteReplicationSys) AddPeerClusters(ctx context.Context, psites []madmi
return madmin.ReplicateAddStatus{}, errSRBackendIssue(err) return madmin.ReplicateAddStatus{}, errSRBackendIssue(err)
} }
currTime := time.Now()
joinReq := madmin.SRPeerJoinReq{ joinReq := madmin.SRPeerJoinReq{
SvcAcctAccessKey: svcCred.AccessKey, SvcAcctAccessKey: svcCred.AccessKey,
SvcAcctSecretKey: secretKey, SvcAcctSecretKey: secretKey,
Peers: make(map[string]madmin.PeerInfo), Peers: make(map[string]madmin.PeerInfo),
UpdatedAt: currTime,
}
// check if few peers exist already and ILM expiry replcation is set to true
replicateILMExpirySet := false
if c.state.Peers != nil {
for _, pi := range c.state.Peers {
if pi.ReplicateILMExpiry {
replicateILMExpirySet = true
break
}
}
} }
for _, v := range sites { for _, v := range sites {
var peerReplicateILMExpiry bool
// if peers already exist and for one of them ReplicateILMExpiry
// set true, that means earlier replication of ILM expiry was set
// for the site replication. All new sites added to the setup should
// get this enabled as well
if replicateILMExpirySet {
peerReplicateILMExpiry = replicateILMExpirySet
} else {
peerReplicateILMExpiry = opts.ReplicateILMExpiry
}
joinReq.Peers[v.DeploymentID] = madmin.PeerInfo{ joinReq.Peers[v.DeploymentID] = madmin.PeerInfo{
Endpoint: v.Endpoint, Endpoint: v.Endpoint,
Name: v.Name, Name: v.Name,
DeploymentID: v.DeploymentID, DeploymentID: v.DeploymentID,
ReplicateILMExpiry: peerReplicateILMExpiry,
} }
} }
@ -548,6 +572,7 @@ func (c *SiteReplicationSys) AddPeerClusters(ctx context.Context, psites []madmi
Name: sites[selfIdx].Name, Name: sites[selfIdx].Name,
Peers: joinReq.Peers, Peers: joinReq.Peers,
ServiceAccountAccessKey: svcCred.AccessKey, ServiceAccountAccessKey: svcCred.AccessKey,
UpdatedAt: currTime,
} }
if err = c.saveToDisk(ctx, state); err != nil { if err = c.saveToDisk(ctx, state); err != nil {
@ -562,7 +587,7 @@ func (c *SiteReplicationSys) AddPeerClusters(ctx context.Context, psites []madmi
Status: madmin.ReplicateAddStatusSuccess, Status: madmin.ReplicateAddStatusSuccess,
} }
if err := c.syncToAllPeers(ctx); err != nil { if err := c.syncToAllPeers(ctx, opts); err != nil {
result.InitialSyncErrorMessage = err.Error() result.InitialSyncErrorMessage = err.Error()
} }
@ -594,10 +619,24 @@ func (c *SiteReplicationSys) PeerJoinReq(ctx context.Context, arg madmin.SRPeerJ
return errSRServiceAccount(fmt.Errorf("unable to create service account on %s: %v", ourName, err)) return errSRServiceAccount(fmt.Errorf("unable to create service account on %s: %v", ourName, err))
} }
peers := make(map[string]madmin.PeerInfo, len(arg.Peers))
for dID, pi := range arg.Peers {
if c.state.Peers != nil {
if existingPeer, ok := c.state.Peers[dID]; ok {
// retain existing ReplicateILMExpiry of peer if its already set
// and incoming arg has it false. it could be default false
if !pi.ReplicateILMExpiry && existingPeer.ReplicateILMExpiry {
pi.ReplicateILMExpiry = existingPeer.ReplicateILMExpiry
}
}
}
peers[dID] = pi
}
state := srState{ state := srState{
Name: ourName, Name: ourName,
Peers: arg.Peers, Peers: peers,
ServiceAccountAccessKey: arg.SvcAcctAccessKey, ServiceAccountAccessKey: arg.SvcAcctAccessKey,
UpdatedAt: arg.UpdatedAt,
} }
if err = c.saveToDisk(ctx, state); err != nil { if err = c.saveToDisk(ctx, state); err != nil {
return errSRBackendIssue(fmt.Errorf("unable to save cluster-replication state to drive on %s: %v", ourName, err)) return errSRBackendIssue(fmt.Errorf("unable to save cluster-replication state to drive on %s: %v", ourName, err))
@ -731,6 +770,7 @@ const (
deleteBucket = "DeleteBucket" deleteBucket = "DeleteBucket"
replicateIAMItem = "SRPeerReplicateIAMItem" replicateIAMItem = "SRPeerReplicateIAMItem"
replicateBucketMetadata = "SRPeerReplicateBucketMeta" replicateBucketMetadata = "SRPeerReplicateBucketMeta"
siteReplicationEdit = "SiteReplicationEdit"
) )
// MakeBucketHook - called during a regular make bucket call when cluster // MakeBucketHook - called during a regular make bucket call when cluster
@ -1681,6 +1721,35 @@ func (c *SiteReplicationSys) PeerBucketQuotaConfigHandler(ctx context.Context, b
return nil return nil
} }
// PeerBucketLCConfigHandler - copies/deletes lifecycle config to local cluster
func (c *SiteReplicationSys) PeerBucketLCConfigHandler(ctx context.Context, bucket string, expLCConfig *string, updatedAt time.Time) error {
// skip overwrite if local update is newer than peer update.
if !updatedAt.IsZero() {
if cfg, _, err := globalBucketMetadataSys.GetLifecycleConfig(bucket); err == nil && (cfg.ExpiryUpdatedAt != nil && cfg.ExpiryUpdatedAt.After(updatedAt)) {
return nil
}
}
if expLCConfig != nil {
configData, err := mergeWithCurrentLCConfig(ctx, bucket, expLCConfig, updatedAt)
if err != nil {
return wrapSRErr(err)
}
_, err = globalBucketMetadataSys.Update(ctx, bucket, bucketLifecycleConfig, configData)
if err != nil {
return wrapSRErr(err)
}
return nil
}
// Delete ILM config
_, err := globalBucketMetadataSys.Delete(ctx, bucket, bucketLifecycleConfig)
if err != nil {
return wrapSRErr(err)
}
return nil
}
// getAdminClient - NOTE: ensure to take at least a read lock on SiteReplicationSys // getAdminClient - NOTE: ensure to take at least a read lock on SiteReplicationSys
// before calling this. // before calling this.
func (c *SiteReplicationSys) getAdminClient(ctx context.Context, deploymentID string) (*madmin.AdminClient, error) { func (c *SiteReplicationSys) getAdminClient(ctx context.Context, deploymentID string) (*madmin.AdminClient, error) {
@ -1733,7 +1802,7 @@ func (c *SiteReplicationSys) listBuckets(ctx context.Context) ([]BucketInfo, err
// syncToAllPeers is used for syncing local data to all remote peers, it is // syncToAllPeers is used for syncing local data to all remote peers, it is
// called once during initial "AddPeerClusters" request. // called once during initial "AddPeerClusters" request.
func (c *SiteReplicationSys) syncToAllPeers(ctx context.Context) error { func (c *SiteReplicationSys) syncToAllPeers(ctx context.Context, addOpts madmin.SRAddOptions) error {
objAPI := newObjectLayerFn() objAPI := newObjectLayerFn()
if objAPI == nil { if objAPI == nil {
return errSRObjectLayerNotReady return errSRObjectLayerNotReady
@ -1822,6 +1891,7 @@ func (c *SiteReplicationSys) syncToAllPeers(ctx context.Context) error {
} }
} }
// Replicate existing bucket quotas settings
quotaConfigJSON, tm := meta.QuotaConfigJSON, meta.QuotaConfigUpdatedAt quotaConfigJSON, tm := meta.QuotaConfigJSON, meta.QuotaConfigUpdatedAt
if len(quotaConfigJSON) > 0 { if len(quotaConfigJSON) > 0 {
err = c.BucketMetaHook(ctx, madmin.SRBucketMeta{ err = c.BucketMetaHook(ctx, madmin.SRBucketMeta{
@ -1834,6 +1904,36 @@ func (c *SiteReplicationSys) syncToAllPeers(ctx context.Context) error {
return errSRBucketMetaError(err) return errSRBucketMetaError(err)
} }
} }
// Replicate ILM expiry rules if needed
if addOpts.ReplicateILMExpiry && (meta.lifecycleConfig != nil && meta.lifecycleConfig.HasExpiry()) {
var expLclCfg lifecycle.Lifecycle
expLclCfg.XMLName = meta.lifecycleConfig.XMLName
for _, rule := range meta.lifecycleConfig.Rules {
if !rule.Expiration.IsNull() || !rule.NoncurrentVersionExpiration.IsNull() {
// copy the non transition details of the rule
expLclCfg.Rules = append(expLclCfg.Rules, rule.CloneNonTransition())
}
}
currtime := time.Now()
expLclCfg.ExpiryUpdatedAt = &currtime
ilmConfigData, err := xml.Marshal(expLclCfg)
if err != nil {
return errSRBucketMetaError(err)
}
if len(ilmConfigData) > 0 {
configStr := base64.StdEncoding.EncodeToString(ilmConfigData)
err = c.BucketMetaHook(ctx, madmin.SRBucketMeta{
Type: madmin.SRBucketMetaLCConfig,
Bucket: bucket,
ExpiryLCConfig: &configStr,
UpdatedAt: time.Now(),
})
if err != nil {
return errSRBucketMetaError(err)
}
}
}
} }
// Order matters from now on how the information is // Order matters from now on how the information is
@ -2515,6 +2615,11 @@ type srGroupDesc struct {
DeploymentID string DeploymentID string
} }
type srILMExpiryRule struct {
madmin.ILMExpiryRule
DeploymentID string
}
// SiteReplicationStatus returns the site replication status across clusters participating in site replication. // SiteReplicationStatus returns the site replication status across clusters participating in site replication.
func (c *SiteReplicationSys) SiteReplicationStatus(ctx context.Context, objAPI ObjectLayer, opts madmin.SRStatusOptions) (info madmin.SRStatusInfo, err error) { func (c *SiteReplicationSys) SiteReplicationStatus(ctx context.Context, objAPI ObjectLayer, opts madmin.SRStatusOptions) (info madmin.SRStatusInfo, err error) {
sinfo, err := c.siteReplicationStatus(ctx, objAPI, opts) sinfo, err := c.siteReplicationStatus(ctx, objAPI, opts)
@ -2527,6 +2632,7 @@ func (c *SiteReplicationSys) SiteReplicationStatus(ctx context.Context, objAPI O
MaxUsers: sinfo.MaxUsers, MaxUsers: sinfo.MaxUsers,
MaxGroups: sinfo.MaxGroups, MaxGroups: sinfo.MaxGroups,
MaxPolicies: sinfo.MaxPolicies, MaxPolicies: sinfo.MaxPolicies,
MaxILMExpiryRules: sinfo.MaxILMExpiryRules,
Sites: sinfo.Sites, Sites: sinfo.Sites,
StatsSummary: sinfo.StatsSummary, StatsSummary: sinfo.StatsSummary,
Metrics: sinfo.Metrics, Metrics: sinfo.Metrics,
@ -2535,6 +2641,7 @@ func (c *SiteReplicationSys) SiteReplicationStatus(ctx context.Context, objAPI O
info.PolicyStats = make(map[string]map[string]madmin.SRPolicyStatsSummary) info.PolicyStats = make(map[string]map[string]madmin.SRPolicyStatsSummary)
info.UserStats = make(map[string]map[string]madmin.SRUserStatsSummary) info.UserStats = make(map[string]map[string]madmin.SRUserStatsSummary)
info.GroupStats = make(map[string]map[string]madmin.SRGroupStatsSummary) info.GroupStats = make(map[string]map[string]madmin.SRGroupStatsSummary)
info.ILMExpiryStats = make(map[string]map[string]madmin.SRILMExpiryStatsSummary)
numSites := len(info.Sites) numSites := len(info.Sites)
for b, stat := range sinfo.BucketStats { for b, stat := range sinfo.BucketStats {
for dID, st := range stat { for dID, st := range stat {
@ -2583,6 +2690,16 @@ func (c *SiteReplicationSys) SiteReplicationStatus(ctx context.Context, objAPI O
} }
} }
} }
for p, stat := range sinfo.ILMExpiryRulesStats {
for dID, st := range stat {
if st.ILMExpiryRuleMismatch || opts.Entity == madmin.SRILMExpiryRuleEntity {
if _, ok := info.ILMExpiryStats[p]; !ok {
info.ILMExpiryStats[p] = make(map[string]madmin.SRILMExpiryStatsSummary, numSites)
}
info.ILMExpiryStats[p][dID] = st.SRILMExpiryStatsSummary
}
}
}
return return
} }
@ -2645,6 +2762,7 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
for d, peer := range c.state.Peers { for d, peer := range c.state.Peers {
info.Sites[d] = peer info.Sites[d] = peer
} }
info.UpdatedAt = c.state.UpdatedAt
var maxBuckets int var maxBuckets int
for _, sri := range sris { for _, sri := range sris {
@ -2659,6 +2777,7 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
groupPolicyStats := make(map[string][]srPolicyMapping) groupPolicyStats := make(map[string][]srPolicyMapping)
userInfoStats := make(map[string][]srUserInfo) userInfoStats := make(map[string][]srUserInfo)
groupDescStats := make(map[string][]srGroupDesc) groupDescStats := make(map[string][]srGroupDesc)
ilmExpiryRuleStats := make(map[string][]srILMExpiryRule)
numSites := len(sris) numSites := len(sris)
allBuckets := set.NewStringSet() // across sites allBuckets := set.NewStringSet() // across sites
@ -2666,6 +2785,7 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
allUserWPolicies := set.NewStringSet() allUserWPolicies := set.NewStringSet()
allGroups := set.NewStringSet() allGroups := set.NewStringSet()
allGroupWPolicies := set.NewStringSet() allGroupWPolicies := set.NewStringSet()
allILMExpiryRules := set.NewStringSet()
allPolicies := set.NewStringSet() allPolicies := set.NewStringSet()
for _, sri := range sris { for _, sri := range sris {
@ -2687,6 +2807,9 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
for g := range sri.GroupPolicies { for g := range sri.GroupPolicies {
allGroupWPolicies.Add(g) allGroupWPolicies.Add(g)
} }
for r := range sri.ILMExpiryRules {
allILMExpiryRules.Add(r)
}
} }
for i, sri := range sris { for i, sri := range sris {
@ -2739,6 +2862,13 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
gd := sri.GroupDescMap[g] gd := sri.GroupDescMap[g]
groupDescStats[g][i] = srGroupDesc{GroupDesc: gd, DeploymentID: sri.DeploymentID} groupDescStats[g][i] = srGroupDesc{GroupDesc: gd, DeploymentID: sri.DeploymentID}
} }
for r := range allILMExpiryRules {
if _, ok := ilmExpiryRuleStats[r]; !ok {
ilmExpiryRuleStats[r] = make([]srILMExpiryRule, numSites)
}
rl := sri.ILMExpiryRules[r]
ilmExpiryRuleStats[r][i] = srILMExpiryRule{ILMExpiryRule: rl, DeploymentID: sri.DeploymentID}
}
} }
info.StatsSummary = make(map[string]madmin.SRSiteSummary, len(c.state.Peers)) info.StatsSummary = make(map[string]madmin.SRSiteSummary, len(c.state.Peers))
@ -2746,6 +2876,7 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
info.PolicyStats = make(map[string]map[string]srPolicyStatsSummary) info.PolicyStats = make(map[string]map[string]srPolicyStatsSummary)
info.UserStats = make(map[string]map[string]srUserStatsSummary) info.UserStats = make(map[string]map[string]srUserStatsSummary)
info.GroupStats = make(map[string]map[string]srGroupStatsSummary) info.GroupStats = make(map[string]map[string]srGroupStatsSummary)
info.ILMExpiryRulesStats = make(map[string]map[string]srILMExpiryRuleStatsSummary)
// collect user policy mapping replication status across sites // collect user policy mapping replication status across sites
if opts.Users || opts.Entity == madmin.SRUserEntity { if opts.Users || opts.Entity == madmin.SRUserEntity {
for u, pslc := range userPolicyStats { for u, pslc := range userPolicyStats {
@ -3110,6 +3241,54 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
} }
} }
} }
if opts.ILMExpiryRules || opts.Entity == madmin.SRILMExpiryRuleEntity {
// collect ILM expiry rules replication status across sites
for id, ilmExpRules := range ilmExpiryRuleStats {
var rules []*lifecycle.Rule
uRuleCount := 0
for _, rl := range ilmExpRules {
var rule lifecycle.Rule
if err := xml.Unmarshal([]byte(rl.ILMExpiryRule.ILMRule), &rule); err != nil {
continue
}
rules = append(rules, &rule)
uRuleCount++
sum := info.StatsSummary[rl.DeploymentID]
sum.TotalILMExpiryRulesCount++
info.StatsSummary[rl.DeploymentID] = sum
}
if len(info.ILMExpiryRulesStats[id]) == 0 {
info.ILMExpiryRulesStats[id] = make(map[string]srILMExpiryRuleStatsSummary)
}
ilmExpRuleMismatch := !isILMExpRuleReplicated(uRuleCount, numSites, rules)
for _, rl := range ilmExpRules {
dID := depIdx[rl.DeploymentID]
_, hasILMExpRule := sris[dID].ILMExpiryRules[id]
info.ILMExpiryRulesStats[id][rl.DeploymentID] = srILMExpiryRuleStatsSummary{
SRILMExpiryStatsSummary: madmin.SRILMExpiryStatsSummary{
ILMExpiryRuleMismatch: ilmExpRuleMismatch,
HasILMExpiryRules: hasILMExpRule,
},
ilmExpiryRule: rl,
}
switch {
case ilmExpRuleMismatch, opts.Entity == madmin.SRILMExpiryRuleEntity:
default:
sum := info.StatsSummary[rl.DeploymentID]
if !ilmExpRuleMismatch {
sum.ReplicatedILMExpiryRules++
}
info.StatsSummary[rl.DeploymentID] = sum
}
}
}
}
if opts.PeerState {
info.PeerStates = make(map[string]madmin.SRStateInfo, numSites)
for _, sri := range sris {
info.PeerStates[sri.DeploymentID] = sri.State
}
}
if opts.Metrics { if opts.Metrics {
m, err := globalSiteReplicationSys.getSiteMetrics(ctx) m, err := globalSiteReplicationSys.getSiteMetrics(ctx)
@ -3124,6 +3303,7 @@ func (c *SiteReplicationSys) siteReplicationStatus(ctx context.Context, objAPI O
info.MaxUsers = len(userInfoStats) info.MaxUsers = len(userInfoStats)
info.MaxGroups = len(groupDescStats) info.MaxGroups = len(groupDescStats)
info.MaxPolicies = len(policyStats) info.MaxPolicies = len(policyStats)
info.MaxILMExpiryRules = len(ilmExpiryRuleStats)
return return
} }
@ -3323,6 +3503,35 @@ func isBktReplCfgReplicated(total int, cfgs []*sreplication.Config) bool {
return true return true
} }
// isILMExpRuleReplicated returns true if count of replicated ILM Expiry rules matches total
// number of sites and ILM expiry rules are identical.
func isILMExpRuleReplicated(cntReplicated, total int, rules []*lifecycle.Rule) bool {
if cntReplicated > 0 && cntReplicated != total {
return false
}
// check if policies match between sites
var prev *lifecycle.Rule
for i, r := range rules {
if i == 0 {
prev = r
continue
}
// Check equality of rules
prevRData, err := xml.Marshal(prev)
if err != nil {
return false
}
rData, err := xml.Marshal(*r)
if err != nil {
return false
}
if !(string(prevRData) == string(rData)) {
return false
}
}
return true
}
// cache of IAM info fetched in last SiteReplicationMetaInfo call // cache of IAM info fetched in last SiteReplicationMetaInfo call
type srIAMCache struct { type srIAMCache struct {
sync.RWMutex sync.RWMutex
@ -3438,6 +3647,29 @@ func (c *SiteReplicationSys) SiteReplicationMetaInfo(ctx context.Context, objAPI
bms.ReplicationConfigUpdatedAt = meta.ReplicationConfigUpdatedAt bms.ReplicationConfigUpdatedAt = meta.ReplicationConfigUpdatedAt
} }
if meta.lifecycleConfig != nil {
var expLclCfg lifecycle.Lifecycle
expLclCfg.XMLName = meta.lifecycleConfig.XMLName
for _, rule := range meta.lifecycleConfig.Rules {
if !rule.Expiration.IsNull() || !rule.NoncurrentVersionExpiration.IsNull() {
// copy the non transition details of the rule
expLclCfg.Rules = append(expLclCfg.Rules, rule.CloneNonTransition())
}
}
expLclCfg.ExpiryUpdatedAt = meta.lifecycleConfig.ExpiryUpdatedAt
ilmConfigData, err := xml.Marshal(expLclCfg)
if err != nil {
return info, errSRBackendIssue(err)
}
expLclCfgStr := base64.StdEncoding.EncodeToString(ilmConfigData)
bms.ExpiryLCConfig = &expLclCfgStr
// if all non expiry rules only, ExpiryUpdatedAt would be nil
if meta.lifecycleConfig.ExpiryUpdatedAt != nil {
bms.ExpiryLCConfigUpdatedAt = *(meta.lifecycleConfig.ExpiryUpdatedAt)
}
}
info.Buckets[bucket] = bms info.Buckets[bucket] = bms
} }
} }
@ -3471,6 +3703,56 @@ func (c *SiteReplicationSys) SiteReplicationMetaInfo(ctx context.Context, objAPI
info.Policies[pname] = madmin.SRIAMPolicy{Policy: json.RawMessage(policyJSON), UpdatedAt: policyDoc.UpdateDate} info.Policies[pname] = madmin.SRIAMPolicy{Policy: json.RawMessage(policyJSON), UpdatedAt: policyDoc.UpdateDate}
} }
} }
if opts.ILMExpiryRules || opts.Entity == madmin.SRILMExpiryRuleEntity {
info.ILMExpiryRules = make(map[string]madmin.ILMExpiryRule)
buckets, err := objAPI.ListBuckets(ctx, BucketOptions{Deleted: opts.ShowDeleted})
if err != nil {
return info, errSRBackendIssue(err)
}
allRules := make(map[string]madmin.ILMExpiryRule)
for _, bucketInfo := range buckets {
bucket := bucketInfo.Name
bucketExists := bucketInfo.Deleted.IsZero() || (!bucketInfo.Created.IsZero() && bucketInfo.Created.After(bucketInfo.Deleted))
if !bucketExists {
continue
}
meta, err := globalBucketMetadataSys.GetConfigFromDisk(ctx, bucket)
if err != nil && !errors.Is(err, errConfigNotFound) {
return info, errSRBackendIssue(err)
}
if meta.lifecycleConfig != nil && meta.lifecycleConfig.HasExpiry() {
for _, rule := range meta.lifecycleConfig.Rules {
if !rule.Expiration.IsNull() || !rule.NoncurrentVersionExpiration.IsNull() {
// copy the non transition details of the rule
ruleData, err := xml.Marshal(rule.CloneNonTransition())
if err != nil {
return info, errSRBackendIssue(err)
}
allRules[rule.ID] = madmin.ILMExpiryRule{ILMRule: string(ruleData), Bucket: bucket, UpdatedAt: *(meta.lifecycleConfig.ExpiryUpdatedAt)}
}
}
}
}
if opts.Entity == madmin.SRILMExpiryRuleEntity {
if rule, ok := allRules[opts.EntityValue]; ok {
info.ILMExpiryRules[opts.EntityValue] = rule
}
} else {
for id, rule := range allRules {
info.ILMExpiryRules[id] = rule
}
}
}
if opts.PeerState {
info.State = madmin.SRStateInfo{
Name: c.state.Name,
Peers: c.state.Peers,
UpdatedAt: c.state.UpdatedAt,
}
}
if opts.Users || opts.Entity == madmin.SRUserEntity { if opts.Users || opts.Entity == madmin.SRUserEntity {
// Replicate policy mappings on local to all peers. // Replicate policy mappings on local to all peers.
@ -3602,7 +3884,7 @@ func (c *SiteReplicationSys) SiteReplicationMetaInfo(ctx context.Context, objAPI
} }
// EditPeerCluster - edits replication configuration and updates peer endpoint. // EditPeerCluster - edits replication configuration and updates peer endpoint.
func (c *SiteReplicationSys) EditPeerCluster(ctx context.Context, peer madmin.PeerInfo) (madmin.ReplicateEditStatus, error) { func (c *SiteReplicationSys) EditPeerCluster(ctx context.Context, peer madmin.PeerInfo, opts madmin.SREditOptions) (madmin.ReplicateEditStatus, error) {
sites, err := c.GetClusterInfo(ctx) sites, err := c.GetClusterInfo(ctx)
if err != nil { if err != nil {
return madmin.ReplicateEditStatus{}, errSRBackendIssue(err) return madmin.ReplicateEditStatus{}, errSRBackendIssue(err)
@ -3644,27 +3926,70 @@ func (c *SiteReplicationSys) EditPeerCluster(ctx context.Context, peer madmin.Pe
} }
} }
if !found { // if disable/enable ILM expiry replication, deployment id not needed.
// check for below error only if other options being updated (e.g. endpoint, sync, bandwidth)
if !opts.DisableILMExpiryReplication && !opts.EnableILMExpiryReplication && !found {
return madmin.ReplicateEditStatus{}, errSRInvalidRequest(fmt.Errorf("%s not found in existing replicated sites", peer.DeploymentID)) return madmin.ReplicateEditStatus{}, errSRInvalidRequest(fmt.Errorf("%s not found in existing replicated sites", peer.DeploymentID))
} }
successMsg := "Cluster replication configuration updated successfully with:"
var state srState var state srState
c.RLock() c.RLock()
pi := c.state.Peers[peer.DeploymentID]
state = c.state state = c.state
c.RUnlock() c.RUnlock()
prevPeerInfo := pi
// in case of --disable-ilm-expiry-replication and --enable-ilm-expiry-replication
// --deployment-id is not passed
var (
prevPeerInfo, pi madmin.PeerInfo
)
if peer.DeploymentID != "" {
pi = c.state.Peers[peer.DeploymentID]
prevPeerInfo = pi
if !peer.SyncState.Empty() { // update replication to peer to be sync/async if !peer.SyncState.Empty() { // update replication to peer to be sync/async
pi.SyncState = peer.SyncState pi.SyncState = peer.SyncState
successMsg = fmt.Sprintf("%s\n- sync state %s for peer %s", successMsg, peer.SyncState, peer.Name)
} }
if peer.Endpoint != "" { // `admin replicate update` requested an endpoint change if peer.Endpoint != "" { // `admin replicate update` requested an endpoint change
pi.Endpoint = peer.Endpoint pi.Endpoint = peer.Endpoint
successMsg = fmt.Sprintf("%s\n- endpoint %s for peer %s", successMsg, peer.Endpoint, peer.Name)
} }
if peer.DefaultBandwidth.IsSet { if peer.DefaultBandwidth.IsSet {
pi.DefaultBandwidth = peer.DefaultBandwidth pi.DefaultBandwidth = peer.DefaultBandwidth
pi.DefaultBandwidth.UpdatedAt = UTCNow() pi.DefaultBandwidth.UpdatedAt = UTCNow()
successMsg = fmt.Sprintf("%s\n- default bandwidth %v for peer %s", successMsg, peer.DefaultBandwidth.Limit, peer.Name)
} }
state.Peers[peer.DeploymentID] = pi state.Peers[peer.DeploymentID] = pi
}
// If ILM expiry replications enabled/disabled, set accordingly
if opts.DisableILMExpiryReplication {
for dID, pi := range state.Peers {
if !pi.ReplicateILMExpiry {
return madmin.ReplicateEditStatus{
Status: madmin.ReplicateAddStatusPartial,
ErrDetail: "ILM expiry already set to false",
}, nil
}
pi.ReplicateILMExpiry = false
state.Peers[dID] = pi
}
successMsg = fmt.Sprintf("%s\n- replicate-ilm-expiry: false", successMsg)
}
if opts.EnableILMExpiryReplication {
for dID, pi := range state.Peers {
if pi.ReplicateILMExpiry {
return madmin.ReplicateEditStatus{
Status: madmin.ReplicateAddStatusPartial,
ErrDetail: "ILM expiry already set to true",
}, nil
}
pi.ReplicateILMExpiry = true
state.Peers[dID] = pi
}
successMsg = fmt.Sprintf("%s\n- replicate-ilm-expiry: true", successMsg)
}
state.UpdatedAt = time.Now()
errs := make(map[string]error, len(state.Peers)) errs := make(map[string]error, len(state.Peers))
var wg sync.WaitGroup var wg sync.WaitGroup
@ -3673,6 +3998,15 @@ func (c *SiteReplicationSys) EditPeerCluster(ctx context.Context, peer madmin.Pe
if v.DeploymentID == globalDeploymentID() { if v.DeploymentID == globalDeploymentID() {
continue continue
} }
// if individual deployment change like mode, endpoint, default bandwidth
// send it to all sites. Else send the current node details to all sites
// for ILM expiry flag update
var p madmin.PeerInfo
if peer.DeploymentID != "" {
p = pi
} else {
p = v
}
wg.Add(1) wg.Add(1)
go func(pi madmin.PeerInfo, dID string) { go func(pi madmin.PeerInfo, dID string) {
defer wg.Done() defer wg.Done()
@ -3688,14 +4022,12 @@ func (c *SiteReplicationSys) EditPeerCluster(ctx context.Context, peer madmin.Pe
errs[dID] = errSRPeerResp(fmt.Errorf("unable to update peer %s: %w", pi.Name, err)) errs[dID] = errSRPeerResp(fmt.Errorf("unable to update peer %s: %w", pi.Name, err))
return return
} }
}(pi, dID) }(p, dID)
} }
wg.Wait() wg.Wait()
for dID, err := range errs { for dID, err := range errs {
if err != nil { logger.LogOnceIf(ctx, fmt.Errorf("unable to update peer %s: %w", state.Peers[dID].Name, err), "site-relication-edit")
return madmin.ReplicateEditStatus{}, errSRPeerResp(fmt.Errorf("unable to update peer %s: %w", state.Peers[dID].Name, err))
}
} }
// we can now save the cluster replication configuration state. // we can now save the cluster replication configuration state.
@ -3706,16 +4038,22 @@ func (c *SiteReplicationSys) EditPeerCluster(ctx context.Context, peer madmin.Pe
}, nil }, nil
} }
if peer.DeploymentID != "" {
if err = c.updateTargetEndpoints(ctx, prevPeerInfo, pi); err != nil { if err = c.updateTargetEndpoints(ctx, prevPeerInfo, pi); err != nil {
return madmin.ReplicateEditStatus{ return madmin.ReplicateEditStatus{
Status: madmin.ReplicateAddStatusPartial, Status: madmin.ReplicateAddStatusPartial,
ErrDetail: fmt.Sprintf("unable to update peer targets on local: %v", err), ErrDetail: fmt.Sprintf("unable to update peer targets on local: %v", err),
}, nil }, nil
} }
}
// set partial error message if remote site updates failed for few cases
if len(errs) > 0 {
successMsg = fmt.Sprintf("%s\n- partially failed for few remote sites as they could be down/unreachable at the moment", successMsg)
}
result := madmin.ReplicateEditStatus{ result := madmin.ReplicateEditStatus{
Success: true, Success: true,
Status: fmt.Sprintf("Cluster replication configuration updated with endpoint %s for peer %s successfully", peer.Endpoint, peer.Name), Status: successMsg,
} }
return result, nil return result, nil
} }
@ -3783,6 +4121,20 @@ func (c *SiteReplicationSys) updateTargetEndpoints(ctx context.Context, prevInfo
// to edit endpoint. // to edit endpoint.
func (c *SiteReplicationSys) PeerEditReq(ctx context.Context, arg madmin.PeerInfo) error { func (c *SiteReplicationSys) PeerEditReq(ctx context.Context, arg madmin.PeerInfo) error {
ourName := "" ourName := ""
// Set ReplicateILMExpiry for all peers
currTime := time.Now()
for i := range c.state.Peers {
p := c.state.Peers[i]
if p.ReplicateILMExpiry == arg.ReplicateILMExpiry {
// its already set due to previous edit req
break
}
p.ReplicateILMExpiry = arg.ReplicateILMExpiry
c.state.UpdatedAt = currTime
c.state.Peers[i] = p
}
for i := range c.state.Peers { for i := range c.state.Peers {
p := c.state.Peers[i] p := c.state.Peers[i]
if p.DeploymentID == arg.DeploymentID { if p.DeploymentID == arg.DeploymentID {
@ -3799,6 +4151,25 @@ func (c *SiteReplicationSys) PeerEditReq(ctx context.Context, arg madmin.PeerInf
return nil return nil
} }
// PeerStateEditReq - internal API handler to respond to a peer cluster's request
// to edit state.
func (c *SiteReplicationSys) PeerStateEditReq(ctx context.Context, arg madmin.SRStateEditReq) error {
if arg.UpdatedAt.After(c.state.UpdatedAt) {
state := c.state
// update only the ReplicateILMExpiry flag for the peers from incoming request
for _, peer := range arg.Peers {
currPeer := c.state.Peers[peer.DeploymentID]
currPeer.ReplicateILMExpiry = peer.ReplicateILMExpiry
state.Peers[peer.DeploymentID] = currPeer
}
state.UpdatedAt = arg.UpdatedAt
if err := c.saveToDisk(ctx, state); err != nil {
return errSRBackendIssue(fmt.Errorf("unable to save cluster-replication state to drive on %s: %v", state.Name, err))
}
}
return nil
}
const siteHealTimeInterval = 30 * time.Second const siteHealTimeInterval = 30 * time.Second
func (c *SiteReplicationSys) startHealRoutine(ctx context.Context, objAPI ObjectLayer) { func (c *SiteReplicationSys) startHealRoutine(ctx context.Context, objAPI ObjectLayer) {
@ -3860,6 +4231,11 @@ type srGroupStatsSummary struct {
groupPolicy srPolicyMapping groupPolicy srPolicyMapping
} }
type srILMExpiryRuleStatsSummary struct {
madmin.SRILMExpiryStatsSummary
ilmExpiryRule srILMExpiryRule
}
type srStatusInfo struct { type srStatusInfo struct {
// SRStatusInfo returns detailed status on site replication status // SRStatusInfo returns detailed status on site replication status
Enabled bool Enabled bool
@ -3867,6 +4243,7 @@ type srStatusInfo struct {
MaxUsers int // maximum users seen across sites MaxUsers int // maximum users seen across sites
MaxGroups int // maximum groups seen across sites MaxGroups int // maximum groups seen across sites
MaxPolicies int // maximum policies across sites MaxPolicies int // maximum policies across sites
MaxILMExpiryRules int // maximum ILM expiry rules across sites
Sites map[string]madmin.PeerInfo // deployment->sitename Sites map[string]madmin.PeerInfo // deployment->sitename
StatsSummary map[string]madmin.SRSiteSummary // map of deployment id -> site stat StatsSummary map[string]madmin.SRSiteSummary // map of deployment id -> site stat
// BucketStats map of bucket to slice of deployment IDs with stats. This is populated only if there are // BucketStats map of bucket to slice of deployment IDs with stats. This is populated only if there are
@ -3881,7 +4258,13 @@ type srStatusInfo struct {
// GroupStats map of group to slice of deployment IDs with stats. This is populated only if there are // GroupStats map of group to slice of deployment IDs with stats. This is populated only if there are
// mismatches or if a specific bucket's stats are requested // mismatches or if a specific bucket's stats are requested
GroupStats map[string]map[string]srGroupStatsSummary GroupStats map[string]map[string]srGroupStatsSummary
// ILMExpiryRulesStats map of ILM expiry rules to slice of deployment IDs with stats. This is populated only if there are
// mismatches or if a specific ILM expiry rule's stats are requested
ILMExpiryRulesStats map[string]map[string]srILMExpiryRuleStatsSummary
// PeerStates map of site replication sites to their site replication states
PeerStates map[string]madmin.SRStateInfo
Metrics madmin.SRMetricsSummary Metrics madmin.SRMetricsSummary
UpdatedAt time.Time
} }
// SRBucketDeleteOp - type of delete op // SRBucketDeleteOp - type of delete op
@ -3910,21 +4293,81 @@ func getSRBucketDeleteOp(isSiteReplicated bool) SRBucketDeleteOp {
return MarkDelete return MarkDelete
} }
func (c *SiteReplicationSys) healILMExpiryConfig(ctx context.Context, objAPI ObjectLayer, info srStatusInfo) error {
c.RLock()
defer c.RUnlock()
if !c.enabled {
return nil
}
var (
latestID, latestPeerName string
lastUpdate time.Time
latestPeers map[string]madmin.PeerInfo
)
for dID, ps := range info.PeerStates {
if lastUpdate.IsZero() {
lastUpdate = ps.UpdatedAt
latestID = dID
latestPeers = ps.Peers
}
if ps.UpdatedAt.After(lastUpdate) {
lastUpdate = ps.UpdatedAt
latestID = dID
latestPeers = ps.Peers
}
}
latestPeerName = info.Sites[latestID].Name
for dID, ps := range info.PeerStates {
// If latest peers ILM expiry flags are equal to current peer, no need to heal
flagEqual := true
for id, peer := range latestPeers {
if !(ps.Peers[id].ReplicateILMExpiry == peer.ReplicateILMExpiry) {
flagEqual = false
break
}
}
if flagEqual {
continue
}
// Dont apply the self state to self
if dID == globalDeploymentID() {
continue
}
// Send details to other sites for healing
admClient, err := c.getAdminClient(ctx, dID)
if err != nil {
return wrapSRErr(err)
}
if err = admClient.SRStateEdit(ctx, madmin.SRStateEditReq{Peers: latestPeers, UpdatedAt: lastUpdate}); err != nil {
logger.LogIf(ctx, c.annotatePeerErr(ps.Name, siteReplicationEdit,
fmt.Errorf("Unable to heal site replication state for peer %s from peer %s : %w",
ps.Name, latestPeerName, err)))
}
}
return nil
}
func (c *SiteReplicationSys) healBuckets(ctx context.Context, objAPI ObjectLayer) error { func (c *SiteReplicationSys) healBuckets(ctx context.Context, objAPI ObjectLayer) error {
buckets, err := c.listBuckets(ctx) buckets, err := c.listBuckets(ctx)
if err != nil { if err != nil {
return err return err
} }
ilmExpiryCfgHealed := false
for _, bi := range buckets { for _, bi := range buckets {
bucket := bi.Name bucket := bi.Name
info, err := c.siteReplicationStatus(ctx, objAPI, madmin.SRStatusOptions{ info, err := c.siteReplicationStatus(ctx, objAPI, madmin.SRStatusOptions{
Entity: madmin.SRBucketEntity, Entity: madmin.SRBucketEntity,
EntityValue: bucket, EntityValue: bucket,
ShowDeleted: true, ShowDeleted: true,
ILMExpiryRules: true,
PeerState: true,
}) })
if err != nil { if err != nil {
logger.LogIf(ctx, err) return err
continue
} }
c.healBucket(ctx, objAPI, bucket, info) c.healBucket(ctx, objAPI, bucket, info)
@ -3937,12 +4380,96 @@ func (c *SiteReplicationSys) healBuckets(ctx context.Context, objAPI ObjectLayer
c.healBucketPolicies(ctx, objAPI, bucket, info) c.healBucketPolicies(ctx, objAPI, bucket, info)
c.healTagMetadata(ctx, objAPI, bucket, info) c.healTagMetadata(ctx, objAPI, bucket, info)
c.healBucketQuotaConfig(ctx, objAPI, bucket, info) c.healBucketQuotaConfig(ctx, objAPI, bucket, info)
if !ilmExpiryCfgHealed {
c.healILMExpiryConfig(ctx, objAPI, info)
ilmExpiryCfgHealed = true
}
if ilmExpiryReplicationEnabled(c.state.Peers) {
c.healBucketILMExpiry(ctx, objAPI, bucket, info)
}
} }
// Notification and ILM are site specific settings. // Notification and ILM are site specific settings.
} }
return nil return nil
} }
func (c *SiteReplicationSys) healBucketILMExpiry(ctx context.Context, objAPI ObjectLayer, bucket string, info srStatusInfo) error {
bs := info.BucketStats[bucket]
c.RLock()
defer c.RUnlock()
if !c.enabled {
return nil
}
var (
latestID, latestPeerName string
lastUpdate time.Time
latestExpLCConfig *string
)
for dID, ss := range bs {
if lastUpdate.IsZero() {
lastUpdate = ss.meta.ExpiryLCConfigUpdatedAt
latestID = dID
latestExpLCConfig = ss.meta.ExpiryLCConfig
}
// avoid considering just created buckets as latest. Perhaps this site
// just joined cluster replication and yet to be sync'd
if ss.meta.CreatedAt.Equal(ss.meta.ExpiryLCConfigUpdatedAt) {
continue
}
if ss.meta.ExpiryLCConfigUpdatedAt.After(lastUpdate) {
lastUpdate = ss.meta.ExpiryLCConfigUpdatedAt
latestID = dID
latestExpLCConfig = ss.meta.ExpiryLCConfig
}
}
latestPeerName = info.Sites[latestID].Name
var err error
if latestExpLCConfig != nil {
_, err = base64.StdEncoding.DecodeString(*latestExpLCConfig)
if err != nil {
return err
}
}
for dID, bStatus := range bs {
if latestExpLCConfig != nil && bStatus.meta.ExpiryLCConfig != nil && strings.EqualFold(*latestExpLCConfig, *bStatus.meta.ExpiryLCConfig) {
continue
}
finalConfigData, err := mergeWithCurrentLCConfig(ctx, bucket, latestExpLCConfig, lastUpdate)
if err != nil {
return wrapSRErr(err)
}
if dID == globalDeploymentID() {
if _, err := globalBucketMetadataSys.Update(ctx, bucket, bucketLifecycleConfig, finalConfigData); err != nil {
logger.LogIf(ctx, fmt.Errorf("Unable to heal bucket ILM expiry data from peer site %s : %w", latestPeerName, err))
}
continue
}
admClient, err := c.getAdminClient(ctx, dID)
if err != nil {
return wrapSRErr(err)
}
peerName := info.Sites[dID].Name
if err = admClient.SRPeerReplicateBucketMeta(ctx, madmin.SRBucketMeta{
Type: madmin.SRBucketMetaLCConfig,
Bucket: bucket,
ExpiryLCConfig: latestExpLCConfig,
UpdatedAt: lastUpdate,
}); err != nil {
logger.LogIf(ctx, c.annotatePeerErr(peerName, replicateBucketMetadata,
fmt.Errorf("Unable to heal bucket ILM expiry data for peer %s from peer %s : %w",
peerName, latestPeerName, err)))
}
}
return nil
}
func (c *SiteReplicationSys) healTagMetadata(ctx context.Context, objAPI ObjectLayer, bucket string, info srStatusInfo) error { func (c *SiteReplicationSys) healTagMetadata(ctx context.Context, objAPI ObjectLayer, bucket string, info srStatusInfo) error {
bs := info.BucketStats[bucket] bs := info.BucketStats[bucket]
@ -5505,3 +6032,107 @@ func (c *SiteReplicationSys) getSiteMetrics(ctx context.Context) (madmin.SRMetri
sm.Uptime = UTCNow().Unix() - globalBootTime.Unix() sm.Uptime = UTCNow().Unix() - globalBootTime.Unix()
return sm, nil return sm, nil
} }
// mergeWithCurrentLCConfig - merges the given ilm expiry configuration with existing for the current site and returns
func mergeWithCurrentLCConfig(ctx context.Context, bucket string, expLCCfg *string, updatedAt time.Time) ([]byte, error) {
// Get bucket config from current site
meta, e := globalBucketMetadataSys.GetConfigFromDisk(ctx, bucket)
if e != nil && !errors.Is(e, errConfigNotFound) {
return []byte{}, e
}
rMap := make(map[string]lifecycle.Rule)
var xmlName xml.Name
if len(meta.LifecycleConfigXML) > 0 {
var lcCfg lifecycle.Lifecycle
if err := xml.Unmarshal(meta.LifecycleConfigXML, &lcCfg); err != nil {
return []byte{}, err
}
for _, rl := range lcCfg.Rules {
rMap[rl.ID] = rl
}
xmlName = meta.lifecycleConfig.XMLName
}
// get latest expiry rules
newRMap := make(map[string]lifecycle.Rule)
if expLCCfg != nil {
var cfg lifecycle.Lifecycle
expLcCfgData, err := base64.StdEncoding.DecodeString(*expLCCfg)
if err != nil {
return []byte{}, err
}
if err := xml.Unmarshal(expLcCfgData, &cfg); err != nil {
return []byte{}, err
}
for _, rl := range cfg.Rules {
newRMap[rl.ID] = rl
}
xmlName = cfg.XMLName
}
// check if current expiry rules are there in new one. if not remove the expiration
// part of rule as they may have been removed from latest updated one
for id, rl := range rMap {
if !rl.Expiration.IsNull() || !rl.NoncurrentVersionExpiration.IsNull() {
if _, ok := newRMap[id]; !ok {
// if rule getting removed was pure expiry rule (may be got to this site
// as part of replication of expiry rules), remove it. Otherwise remove
// only the expiry part of it
if rl.Transition.IsNull() && rl.NoncurrentVersionTransition.IsNull() {
delete(rMap, id)
} else {
rl.Expiration = lifecycle.Expiration{}
rl.NoncurrentVersionExpiration = lifecycle.NoncurrentVersionExpiration{}
rMap[id] = rl
}
}
}
}
// append now
for id, rl := range newRMap {
// if rule is already in original list update non tranisition details with latest
// else simply add to the map. This may happen if ILM expiry replication
// was disabled for sometime and rules were updated independently in different
// sites. Latest changes would get applied but merge only the non transition details
if _, ok := rMap[id]; ok {
rMap[id] = rl.CloneNonTransition()
} else {
rMap[id] = rl
}
}
var rules []lifecycle.Rule
for _, rule := range rMap {
rules = append(rules, rule)
}
// no rules, return
if len(rules) == 0 {
return []byte{}, nil
}
// get final list for write
finalLcCfg := lifecycle.Lifecycle{
XMLName: xmlName,
Rules: rules,
ExpiryUpdatedAt: &updatedAt,
}
if err := finalLcCfg.Validate(); err != nil {
return []byte{}, err
}
finalConfigData, err := xml.Marshal(finalLcCfg)
if err != nil {
return []byte{}, err
}
return finalConfigData, nil
}
func ilmExpiryReplicationEnabled(sites map[string]madmin.PeerInfo) bool {
flag := true
for _, pi := range sites {
flag = flag && pi.ReplicateILMExpiry
}
return flag
}

View File

@ -0,0 +1,272 @@
#!/usr/bin/env bash
set -x
trap 'catch $LINENO' ERR
# shellcheck disable=SC2120
catch() {
if [ $# -ne 0 ]; then
echo "error on line $1"
for site in sitea siteb sitec sited; do
echo "$site server logs ========="
cat "/tmp/${site}_1.log"
echo "==========================="
cat "/tmp/${site}_2.log"
done
fi
echo "Cleaning up instances of MinIO"
pkill minio
pkill -9 minio
rm -rf /tmp/multisitea
rm -rf /tmp/multisiteb
rm -rf /tmp/multisitec
rm -rf /tmp/multisited
rm -rf /tmp/data
}
catch
set -e
export MINIO_CI_CD=1
export MINIO_BROWSER=off
export MINIO_ROOT_USER="minio"
export MINIO_ROOT_PASSWORD="minio123"
export MINIO_KMS_AUTO_ENCRYPTION=off
export MINIO_PROMETHEUS_AUTH_TYPE=public
export MINIO_KMS_SECRET_KEY=my-minio-key:OSMM+vkKUTCvQs9YL/CVMIMt43HFhkUpqJxTmGl6rYw=
unset MINIO_KMS_KES_CERT_FILE
unset MINIO_KMS_KES_KEY_FILE
unset MINIO_KMS_KES_ENDPOINT
unset MINIO_KMS_KES_KEY_NAME
if [ ! -f ./mc ]; then
wget --quiet -O mc https://dl.minio.io/client/mc/release/linux-amd64/mc &&
chmod +x mc
fi
minio server --address 127.0.0.1:9001 "http://127.0.0.1:9001/tmp/multisitea/data/disterasure/xl{1...4}" \
"http://127.0.0.1:9002/tmp/multisitea/data/disterasure/xl{5...8}" >/tmp/sitea_1.log 2>&1 &
minio server --address 127.0.0.1:9002 "http://127.0.0.1:9001/tmp/multisitea/data/disterasure/xl{1...4}" \
"http://127.0.0.1:9002/tmp/multisitea/data/disterasure/xl{5...8}" >/tmp/sitea_2.log 2>&1 &
minio server --address 127.0.0.1:9003 "http://127.0.0.1:9003/tmp/multisiteb/data/disterasure/xl{1...4}" \
"http://127.0.0.1:9004/tmp/multisiteb/data/disterasure/xl{5...8}" >/tmp/siteb_1.log 2>&1 &
minio server --address 127.0.0.1:9004 "http://127.0.0.1:9003/tmp/multisiteb/data/disterasure/xl{1...4}" \
"http://127.0.0.1:9004/tmp/multisiteb/data/disterasure/xl{5...8}" >/tmp/siteb_2.log 2>&1 &
minio server --address 127.0.0.1:9005 "http://127.0.0.1:9005/tmp/multisitec/data/disterasure/xl{1...4}" \
"http://127.0.0.1:9006/tmp/multisitec/data/disterasure/xl{5...8}" >/tmp/sitec_1.log 2>&1 &
minio server --address 127.0.0.1:9006 "http://127.0.0.1:9005/tmp/multisitec/data/disterasure/xl{1...4}" \
"http://127.0.0.1:9006/tmp/multisitec/data/disterasure/xl{5...8}" >/tmp/sitec_2.log 2>&1 &
minio server --address 127.0.0.1:9007 "http://127.0.0.1:9007/tmp/multisited/data/disterasure/xl{1...4}" \
"http://127.0.0.1:9008/tmp/multisited/data/disterasure/xl{5...8}" >/tmp/sited_1.log 2>&1 &
minio server --address 127.0.0.1:9008 "http://127.0.0.1:9007/tmp/multisited/data/disterasure/xl{1...4}" \
"http://127.0.0.1:9008/tmp/multisited/data/disterasure/xl{5...8}" >/tmp/sited_2.log 2>&1 &
# Wait to make sure all MinIO instances are up
sleep 20s
export MC_HOST_sitea=http://minio:minio123@127.0.0.1:9001
export MC_HOST_siteb=http://minio:minio123@127.0.0.1:9004
export MC_HOST_sitec=http://minio:minio123@127.0.0.1:9006
export MC_HOST_sited=http://minio:minio123@127.0.0.1:9008
./mc mb sitea/bucket
./mc mb sitec/bucket
## Setup site replication
./mc admin replicate add sitea siteb --replicate-ilm-expiry
## Add warm tier
./mc ilm tier add minio sitea WARM-TIER --endpoint http://localhost:9006 --access-key minio --secret-key minio123 --bucket bucket
## Add ILM rules
./mc ilm add sitea/bucket --transition-days 0 --transition-tier WARM-TIER --transition-days 0 --noncurrent-expire-days 2 --expire-days 3 --prefix "myprefix" --tags "tag1=val1&tag2=val2"
./mc ilm rule list sitea/bucket
## Check ilm expiry flag
./mc admin replicate info sitea --json
flag1=$(./mc admin replicate info sitea --json | jq '.sites[0]."replicate-ilm-expiry"')
flag2=$(./mc admin replicate info sitea --json | jq '.sites[1]."replicate-ilm-expiry"')
if [ "$flag1" != "true" ]; then
echo "BUG: Expected ILM expiry replication not set for 'sitea'"
exit 1
fi
if [ "$flag2" != "true" ]; then
echo "BUG: Expected ILM expiry replication not set for 'siteb'"
exit 1
fi
## Check if ILM expiry rules replicated
sleep 20
./mc ilm rule list siteb/bucket
count=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules | length')
if [ $count -ne 1 ]; then
echo "BUG: ILM expiry rules not replicated to 'siteb'"
exit 1
fi
## Check replication of rules prefix and tags
prefix=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Prefix' | sed 's/"//g')
tagName1=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Tags[0].Key' | sed 's/"//g')
tagVal1=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Tags[0].Value' | sed 's/"//g')
tagName2=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Tags[1].Key' | sed 's/"//g')
tagVal2=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Tags[1].Value' | sed 's/"//g')
if [ "${prefix}" != "myprefix" ]; then
echo "BUG: ILM expiry rules prefix not replicated to 'siteb'"
exit 1
fi
if [ "${tagName1}" != "tag1" ] || [ "${tagVal1}" != "val1" ] || [ "${tagName2}" != "tag2" ] || [ "${tagVal2}" != "val2" ]; then
echo "BUG: ILM expiry rules tags not replicated to 'siteb'"
exit 1
fi
## Check edit of ILM expiry rule and its replication
id=$(./mc ilm rule list sitea/bucket --json | jq '.config.Rules[] | select(.Expiration.Days==3) | .ID' | sed 's/"//g')
./mc ilm edit --id "${id}" --expire-days "100" sitea/bucket
sleep 30
count1=$(./mc ilm rule list sitea/bucket --json | jq '.config.Rules[0].Expiration.Days')
count2=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Expiration.Days')
if [ $count1 -ne 100 ]; then
echo "BUG: Expiration days not changed on 'sitea'"
exit 1
fi
if [ $count2 -ne 100 ]; then
echo "BUG: Modified ILM expiry rule not replicated to 'siteb'"
exit 1
fi
## Check disabling of ILM expiry rules replication
./mc admin replicate update sitea --disable-ilm-expiry-replication
flag=$(./mc admin replicate info sitea --json | jq '.sites[] | select (.name=="sitea") | ."replicate-ilm-expiry"')
if [ "$flag" != "false" ]; then
echo "BUG: ILM expiry replication not disabled for 'sitea'"
exit 1
fi
flag=$(./mc admin replicate info siteb --json | jq '.sites[] | select (.name=="sitea") | ."replicate-ilm-expiry"')
if [ "$flag" != "false" ]; then
echo "BUG: ILM expiry replication not disabled for 'siteb'"
exit 1
fi
## Perform individual updates of rules to sites
./mc ilm edit --id "${id}" --expire-days "999" sitea/bucket
sleep 1
./mc ilm edit --id "${id}" --expire-days "888" siteb/bucket # when ilm expiry re-enabled, this should win
## Check re-enabling of ILM expiry rules replication
./mc admin replicate update sitea --enable-ilm-expiry-replication
flag=$(./mc admin replicate info sitea --json | jq '.sites[] | select (.name=="sitea") | ."replicate-ilm-expiry"')
if [ "$flag" != "true" ]; then
echo "BUG: ILM expiry replication not enabled for 'sitea'"
exit 1
fi
flag=$(./mc admin replicate info siteb --json | jq '.sites[] | select (.name=="sitea") | ."replicate-ilm-expiry"')
if [ "$flag" != "true" ]; then
echo "BUG: ILM expiry replication not enabled for 'siteb'"
exit 1
fi
## Check if latest updated rules get replicated to all sites post re-enable of ILM expiry rules replication
sleep 30
count1=$(./mc ilm rule list sitea/bucket --json | jq '.config.Rules[0].Expiration.Days')
count2=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Expiration.Days')
if [ $count1 -ne 888 ]; then
echo "BUG: Latest expiration days not updated on 'sitea'"
exit 1
fi
if [ $count2 -ne 888 ]; then
echo "BUG: Latest expiration days not updated on 'siteb'"
exit 1
fi
## Check replication of edit of prefix, tags and status of ILM Expiry Rules
./mc ilm rule edit --id "${id}" --prefix "newprefix" --tags "ntag1=nval1&ntag2=nval2" --disable sitea/bucket
sleep 30
nprefix=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Prefix' | sed 's/"//g')
ntagName1=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Tags[0].Key' | sed 's/"//g')
ntagVal1=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Tags[0].Value' | sed 's/"//g')
ntagName2=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Tags[1].Key' | sed 's/"//g')
ntagVal2=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Filter.And.Tags[1].Value' | sed 's/"//g')
st=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[0].Status' | sed 's/"//g')
if [ "${nprefix}" != "newprefix" ]; then
echo "BUG: ILM expiry rules prefix not replicated to 'siteb'"
exit 1
fi
if [ "${ntagName1}" != "ntag1" ] || [ "${ntagVal1}" != "nval1" ] || [ "${ntagName2}" != "ntag2" ] || [ "${ntagVal2}" != "nval2" ]; then
echo "BUG: ILM expiry rules tags not replicated to 'siteb'"
exit 1
fi
if [ "${st}" != "Disabled" ]; then
echo "BUG: ILM expiry rules status not replicated to 'siteb'"
exit 1
fi
## Check replication of deleted ILM expiry rules
./mc ilm rule remove --id "${id}" sitea/bucket
sleep 30
# should error as rule doesnt exist
error=$(./mc ilm rule list siteb/bucket --json | jq '.error.cause.message' | sed 's/"//g')
if [ "$error" != "The lifecycle configuration does not exist" ]; then
echo "BUG: Removed ILM expiry rule not replicated to 'siteb'"
exit 1
fi
## Check addition of new replication site to existing site replication setup
# Add rules again as previous tests removed all
./mc ilm add sitea/bucket --transition-days 0 --transition-tier WARM-TIER --transition-days 0 --noncurrent-expire-days 2 --expire-days 3 --prefix "myprefix" --tags "tag1=val1&tag2=val2"
./mc admin replicate add sitea siteb sited
sleep 30
# Check site replication info and status for new site
sitesCount=$(mc admin replicate info sited --json | jq '.sites | length')
if [ ${sitesCount} -ne 3 ]; then
echo "BUG: New site 'sited' not appearing in site replication info"
exit 1
fi
flag3=$(./mc admin replicate info sited --json | jq '.sites[2]."replicate-ilm-expiry"')
if [ "${flag3}" != "true" ]; then
echo "BUG: ILM expiry replication not enabled for 'sited'"
exit 1
fi
rulesCount=$(./mc ilm rule list sited/bucket --json | jq '.config.Rules | length')
if [ ${rulesCount} -ne 1 ]; then
echo "BUG: ILM expiry rules not replicated to 'sited'"
exit 1
fi
prefix=$(./mc ilm rule list sited/bucket --json | jq '.config.Rules[0].Filter.And.Prefix' | sed 's/"//g')
tagName1=$(./mc ilm rule list sited/bucket --json | jq '.config.Rules[0].Filter.And.Tags[0].Key' | sed 's/"//g')
tagVal1=$(./mc ilm rule list sited/bucket --json | jq '.config.Rules[0].Filter.And.Tags[0].Value' | sed 's/"//g')
tagName2=$(./mc ilm rule list sited/bucket --json | jq '.config.Rules[0].Filter.And.Tags[1].Key' | sed 's/"//g')
tagVal2=$(./mc ilm rule list sited/bucket --json | jq '.config.Rules[0].Filter.And.Tags[1].Value' | sed 's/"//g')
if [ "${prefix}" != "myprefix" ]; then
echo "BUG: ILM expiry rules prefix not replicated to 'sited'"
exit 1
fi
if [ "${tagName1}" != "tag1" ] || [ "${tagVal1}" != "val1" ] || [ "${tagName2}" != "tag2" ] || [ "${tagVal2}" != "val2" ]; then
echo "BUG: ILM expiry rules tags not replicated to 'sited'"
exit 1
fi
## Check replication of deleted ILM expiry rules when target has transition part as well
## Only the expiry part of rules should get removed as part if replication of removal from
## other site
id=$(./mc ilm rule list siteb/bucket --json | jq '.config.Rules[] | select(.Expiration.Days==3) | .ID' | sed 's/"//g')
# Remove rule from siteb
./mc ilm rule remove --id "${id}" siteb/bucket
sleep 30 # allow to replicate
# sitea should still contain the transition portion of rule
transitionRuleDays=$(./mc ilm rule list sitea/bucket --json | jq '.config.Rules[0].Transition.Days')
expirationRuleDet=$(./mc ilm rule list sitea/bucket --json | jq '.config.Rules[0].Expiration')
if [ ${transitionRuleDays} -ne 0 ]; then
echo "BUG: Transition rules not retained as part of replication of deleted ILM expiry rules on 'sitea'"
exit 1
fi
if [ ${expirationRuleDet} != null ]; then
echo "BUG: removed ILM expiry rule not replicated to 'sitea'"
exit 1
fi
catch

6
go.mod
View File

@ -247,3 +247,9 @@ require (
gopkg.in/square/go-jose.v2 v2.6.0 // indirect gopkg.in/square/go-jose.v2 v2.6.0 // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect
) )
replace github.com/minio/madmin-go/v3 v3.0.29 => github.com/shtripat/madmin-go/v3 v3.0.0-20231106151808-5082883cc33c
replace github.com/minio/mc v0.0.0-20231030184332-9f2fb2b6a9f8 => github.com/shtripat/mc v0.0.0-20231109083216-6c94adcab7f8
replace github.com/minio/console v0.41.0 => github.com/shtripat/minio-console v0.0.0-20231107130354-bf0c7604ae99

12
go.sum
View File

@ -470,8 +470,6 @@ github.com/minio/cli v1.24.2 h1:J+fCUh9mhPLjN3Lj/YhklXvxj8mnyE/D6FpFduXJ2jg=
github.com/minio/cli v1.24.2/go.mod h1:bYxnK0uS629N3Bq+AOZZ+6lwF77Sodk4+UL9vNuXhOY= github.com/minio/cli v1.24.2/go.mod h1:bYxnK0uS629N3Bq+AOZZ+6lwF77Sodk4+UL9vNuXhOY=
github.com/minio/colorjson v1.0.6 h1:m7TUvpvt0u7FBmVIEQNIa0T4NBQlxrcMBp4wJKsg2Ik= github.com/minio/colorjson v1.0.6 h1:m7TUvpvt0u7FBmVIEQNIa0T4NBQlxrcMBp4wJKsg2Ik=
github.com/minio/colorjson v1.0.6/go.mod h1:LUXwS5ZGNb6Eh9f+t+3uJiowD3XsIWtsvTriUBeqgYs= github.com/minio/colorjson v1.0.6/go.mod h1:LUXwS5ZGNb6Eh9f+t+3uJiowD3XsIWtsvTriUBeqgYs=
github.com/minio/console v0.41.0 h1:NjvBij5Hg4GLkO/iAUfZ4imATA/rKNtgVhnn3sEuKDo=
github.com/minio/console v0.41.0/go.mod h1:LTDngEa3Z/s9+2oUb3eBtaVsS/vQFuWTH9d8Z2Pe1mo=
github.com/minio/csvparser v1.0.0 h1:xJEHcYK8ZAjeW4hNV9Zu30u+/2o4UyPnYgyjWp8b7ZU= github.com/minio/csvparser v1.0.0 h1:xJEHcYK8ZAjeW4hNV9Zu30u+/2o4UyPnYgyjWp8b7ZU=
github.com/minio/csvparser v1.0.0/go.mod h1:lKXskSLzPgC5WQyzP7maKH7Sl1cqvANXo9YCto8zbtM= github.com/minio/csvparser v1.0.0/go.mod h1:lKXskSLzPgC5WQyzP7maKH7Sl1cqvANXo9YCto8zbtM=
github.com/minio/dnscache v0.1.1 h1:AMYLqomzskpORiUA1ciN9k7bZT1oB3YZN4cEIi88W5o= github.com/minio/dnscache v0.1.1 h1:AMYLqomzskpORiUA1ciN9k7bZT1oB3YZN4cEIi88W5o=
@ -484,10 +482,6 @@ github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA
github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY= github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY=
github.com/minio/kes-go v0.2.0 h1:HA33arq9s3MErbsj3PAXFVfFo4U4yw7lTKQ5kWFrpCA= github.com/minio/kes-go v0.2.0 h1:HA33arq9s3MErbsj3PAXFVfFo4U4yw7lTKQ5kWFrpCA=
github.com/minio/kes-go v0.2.0/go.mod h1:VorHLaIYis9/MxAHAtXN4d8PUMNKhIxTIlvFt0hBOEo= github.com/minio/kes-go v0.2.0/go.mod h1:VorHLaIYis9/MxAHAtXN4d8PUMNKhIxTIlvFt0hBOEo=
github.com/minio/madmin-go/v3 v3.0.29 h1:3bNLArtxIFud5wyb5/DnF5DGLBvcSJyzCA44EclX1Ow=
github.com/minio/madmin-go/v3 v3.0.29/go.mod h1:4QN2NftLSV7MdlT50dkrenOMmNVHluxTvlqJou3hte8=
github.com/minio/mc v0.0.0-20231030184332-9f2fb2b6a9f8 h1:3WUMQABG8FytpYHRtLHjrnztcUB09hlIrh7rQI9H+tY=
github.com/minio/mc v0.0.0-20231030184332-9f2fb2b6a9f8/go.mod h1:SoPU55ntH5d6IEq6jRBn6e/7SpwI/eSNdBDWmH7nwHk=
github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34=
github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM=
github.com/minio/minio-go/v6 v6.0.46/go.mod h1:qD0lajrGW49lKZLtXKtCB4X/qkMf0a5tBvN2PaZg7Gg= github.com/minio/minio-go/v6 v6.0.46/go.mod h1:qD0lajrGW49lKZLtXKtCB4X/qkMf0a5tBvN2PaZg7Gg=
@ -661,6 +655,12 @@ github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFt
github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ=
github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU=
github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k=
github.com/shtripat/madmin-go/v3 v3.0.0-20231106151808-5082883cc33c h1:QvvwLkuqdj5muc3jgHvZzeSEYG+xZjWc5OuZgxLW53I=
github.com/shtripat/madmin-go/v3 v3.0.0-20231106151808-5082883cc33c/go.mod h1:4QN2NftLSV7MdlT50dkrenOMmNVHluxTvlqJou3hte8=
github.com/shtripat/mc v0.0.0-20231109083216-6c94adcab7f8 h1:K9T44eOsoeruwfBAACXi25YcLu3mN+2MXGdA753IOTE=
github.com/shtripat/mc v0.0.0-20231109083216-6c94adcab7f8/go.mod h1:F6gQ1/r7HLWnB8zy2kuck8voYNcBtFu6QfN4SS9uZ6w=
github.com/shtripat/minio-console v0.0.0-20231107130354-bf0c7604ae99 h1:4iAjs0cyV9XpgEGzsQu9y70h4KsCkvHSif2YeCe35z4=
github.com/shtripat/minio-console v0.0.0-20231107130354-bf0c7604ae99/go.mod h1:Dw108EQHoZeERWn/LoZYZCds8/GKoVzOucqCit0fvyY=
github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
github.com/sirupsen/logrus v1.4.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q=

View File

@ -99,6 +99,7 @@ func (a Action) Delete() bool {
type Lifecycle struct { type Lifecycle struct {
XMLName xml.Name `xml:"LifecycleConfiguration"` XMLName xml.Name `xml:"LifecycleConfiguration"`
Rules []Rule `xml:"Rule"` Rules []Rule `xml:"Rule"`
ExpiryUpdatedAt *time.Time `xml:"ExpiryUpdatedAt,omitempty"`
} }
// HasTransition returns 'true' if lifecycle document has Transition enabled. // HasTransition returns 'true' if lifecycle document has Transition enabled.
@ -111,6 +112,16 @@ func (lc Lifecycle) HasTransition() bool {
return false return false
} }
// HasExpiry returns 'true' if lifecycle document has Expiry enabled.
func (lc Lifecycle) HasExpiry() bool {
for _, rule := range lc.Rules {
if !rule.Expiration.IsNull() || !rule.NoncurrentVersionExpiration.IsNull() {
return true
}
}
return false
}
// UnmarshalXML - decodes XML data. // UnmarshalXML - decodes XML data.
func (lc *Lifecycle) UnmarshalXML(d *xml.Decoder, start xml.StartElement) (err error) { func (lc *Lifecycle) UnmarshalXML(d *xml.Decoder, start xml.StartElement) (err error) {
switch start.Name.Local { switch start.Name.Local {
@ -137,6 +148,12 @@ func (lc *Lifecycle) UnmarshalXML(d *xml.Decoder, start xml.StartElement) (err e
return err return err
} }
lc.Rules = append(lc.Rules, r) lc.Rules = append(lc.Rules, r)
case "ExpiryUpdatedAt":
var t time.Time
if err = d.DecodeElement(&t, &start); err != nil {
return err
}
lc.ExpiryUpdatedAt = &t
default: default:
return xml.UnmarshalError(fmt.Sprintf("expected element type <Rule> but have <%s>", se.Name.Local)) return xml.UnmarshalError(fmt.Sprintf("expected element type <Rule> but have <%s>", se.Name.Local))
} }

View File

@ -163,3 +163,16 @@ func (r Rule) Validate() error {
} }
return nil return nil
} }
// CloneNonTransition - returns a clone of the object containing non transition rules
func (r Rule) CloneNonTransition() Rule {
return Rule{
XMLName: r.XMLName,
ID: r.ID,
Status: r.Status,
Filter: r.Filter,
Prefix: r.Prefix,
Expiration: r.Expiration,
NoncurrentVersionExpiration: r.NoncurrentVersionExpiration,
}
}