diff --git a/cmd/bucket-lifecycle.go b/cmd/bucket-lifecycle.go index e6fcdd0f6..829b0113d 100644 --- a/cmd/bucket-lifecycle.go +++ b/cmd/bucket-lifecycle.go @@ -155,9 +155,14 @@ type newerNoncurrentTask struct { versions []ObjectToDelete } +type transitionTask struct { + tier string + objInfo ObjectInfo +} + type transitionState struct { once sync.Once - transitionCh chan ObjectInfo + transitionCh chan transitionTask ctx context.Context objAPI ObjectLayer @@ -171,13 +176,13 @@ type transitionState struct { lastDayStats map[string]*lastDayTierStats } -func (t *transitionState) queueTransitionTask(oi ObjectInfo) { +func (t *transitionState) queueTransitionTask(oi ObjectInfo, sc string) { select { case <-GlobalContext.Done(): t.once.Do(func() { close(t.transitionCh) }) - case t.transitionCh <- oi: + case t.transitionCh <- transitionTask{objInfo: oi, tier: sc}: default: } } @@ -186,7 +191,7 @@ var globalTransitionState *transitionState func newTransitionState(ctx context.Context, objAPI ObjectLayer) *transitionState { return &transitionState{ - transitionCh: make(chan ObjectInfo, 10000), + transitionCh: make(chan transitionTask, 10000), ctx: ctx, objAPI: objAPI, killCh: make(chan struct{}), @@ -213,27 +218,25 @@ func (t *transitionState) worker(ctx context.Context, objectAPI ObjectLayer) { return case <-ctx.Done(): return - case oi, ok := <-t.transitionCh: + case task, ok := <-t.transitionCh: if !ok { return } atomic.AddInt32(&t.activeTasks, 1) - var tier string - var err error - if tier, err = transitionObject(ctx, objectAPI, oi); err != nil { - logger.LogIf(ctx, fmt.Errorf("Transition failed for %s/%s version:%s with %w", oi.Bucket, oi.Name, oi.VersionID, err)) + if err := transitionObject(ctx, objectAPI, task.objInfo, task.tier); err != nil { + logger.LogIf(ctx, fmt.Errorf("Transition failed for %s/%s version:%s with %w", + task.objInfo.Bucket, task.objInfo.Name, task.objInfo.VersionID, err)) } else { ts := tierStats{ - TotalSize: uint64(oi.Size), + TotalSize: uint64(task.objInfo.Size), NumVersions: 1, } - if oi.IsLatest { + if task.objInfo.IsLatest { ts.NumObjects = 1 } - t.addLastDayStats(tier, ts) + t.addLastDayStats(task.tier, ts) } atomic.AddInt32(&t.activeTasks, -1) - } } } @@ -304,9 +307,10 @@ func validateTransitionTier(lc *lifecycle.Lifecycle) error { // This is to be called after a successful upload of an object (version). func enqueueTransitionImmediate(obj ObjectInfo) { if lc, err := globalLifecycleSys.Get(obj.Bucket); err == nil { - switch lc.ComputeAction(obj.ToLifecycleOpts()) { + event := lc.Eval(obj.ToLifecycleOpts(), time.Now()) + switch event.Action { case lifecycle.TransitionAction, lifecycle.TransitionVersionAction: - globalTransitionState.queueTransitionTask(obj) + globalTransitionState.queueTransitionTask(obj, event.StorageClass) } } } @@ -403,12 +407,7 @@ func genTransitionObjName(bucket string) (string, error) { // storage specified by the transition ARN, the metadata is left behind on source cluster and original content // is moved to the transition tier. Note that in the case of encrypted objects, entire encrypted stream is moved // to the transition tier without decrypting or re-encrypting. -func transitionObject(ctx context.Context, objectAPI ObjectLayer, oi ObjectInfo) (string, error) { - lc, err := globalLifecycleSys.Get(oi.Bucket) - if err != nil { - return "", err - } - tier := lc.TransitionTier(oi.ToLifecycleOpts()) +func transitionObject(ctx context.Context, objectAPI ObjectLayer, oi ObjectInfo, tier string) error { opts := ObjectOptions{ Transition: TransitionOptions{ Status: lifecycle.TransitionPending, @@ -420,7 +419,7 @@ func transitionObject(ctx context.Context, objectAPI ObjectLayer, oi ObjectInfo) VersionSuspended: globalBucketVersioningSys.PrefixSuspended(oi.Bucket, oi.Name), MTime: oi.ModTime, } - return tier, objectAPI.TransitionObject(ctx, oi.Bucket, oi.Name, opts) + return objectAPI.TransitionObject(ctx, oi.Bucket, oi.Name, opts) } type auditTierOp struct { diff --git a/cmd/data-scanner.go b/cmd/data-scanner.go index e8f949c52..1d238dd59 100644 --- a/cmd/data-scanner.go +++ b/cmd/data-scanner.go @@ -963,21 +963,21 @@ func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, oi Obje versionID := oi.VersionID rCfg, _ := globalBucketObjectLockSys.Get(i.bucket) - action := evalActionFromLifecycle(ctx, *i.lifeCycle, rCfg, oi) + lcEvt := evalActionFromLifecycle(ctx, *i.lifeCycle, rCfg, oi) if i.debug { if versionID != "" { - console.Debugf(applyActionsLogPrefix+" lifecycle: %q (version-id=%s), Initial scan: %v\n", i.objectPath(), versionID, action) + console.Debugf(applyActionsLogPrefix+" lifecycle: %q (version-id=%s), Initial scan: %v\n", i.objectPath(), versionID, lcEvt.Action) } else { - console.Debugf(applyActionsLogPrefix+" lifecycle: %q Initial scan: %v\n", i.objectPath(), action) + console.Debugf(applyActionsLogPrefix+" lifecycle: %q Initial scan: %v\n", i.objectPath(), lcEvt.Action) } } - defer globalScannerMetrics.timeILM(action) + defer globalScannerMetrics.timeILM(lcEvt.Action) - switch action { + switch lcEvt.Action { case lifecycle.DeleteAction, lifecycle.DeleteVersionAction, lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: - return applyLifecycleAction(action, oi), 0 + return applyLifecycleAction(lcEvt.Action, oi, ""), 0 case lifecycle.TransitionAction, lifecycle.TransitionVersionAction: - return applyLifecycleAction(action, oi), size + return applyLifecycleAction(lcEvt.Action, oi, lcEvt.StorageClass), size default: // No action. return false, size @@ -1109,21 +1109,21 @@ func (i *scannerItem) applyActions(ctx context.Context, o ObjectLayer, oi Object return size } -func evalActionFromLifecycle(ctx context.Context, lc lifecycle.Lifecycle, lr lock.Retention, obj ObjectInfo) (action lifecycle.Action) { - action = lc.ComputeAction(obj.ToLifecycleOpts()) +func evalActionFromLifecycle(ctx context.Context, lc lifecycle.Lifecycle, lr lock.Retention, obj ObjectInfo) lifecycle.Event { + event := lc.Eval(obj.ToLifecycleOpts(), time.Now().UTC()) if serverDebugLog { - console.Debugf(applyActionsLogPrefix+" lifecycle: Secondary scan: %v\n", action) + console.Debugf(applyActionsLogPrefix+" lifecycle: Secondary scan: %v\n", event.Action) } - if action == lifecycle.NoneAction { - return action + if event.Action == lifecycle.NoneAction { + return event } - switch action { + switch event.Action { case lifecycle.DeleteVersionAction, lifecycle.DeleteRestoredVersionAction: // Defensive code, should never happen if obj.VersionID == "" { - return lifecycle.NoneAction + return lifecycle.Event{Action: lifecycle.NoneAction} } if lr.LockEnabled && enforceRetentionForDeletion(ctx, obj) { if serverDebugLog { @@ -1133,18 +1133,18 @@ func evalActionFromLifecycle(ctx context.Context, lc lifecycle.Lifecycle, lr loc console.Debugf(applyActionsLogPrefix+" lifecycle: %s is locked, not deleting\n", obj.Name) } } - return lifecycle.NoneAction + return lifecycle.Event{Action: lifecycle.NoneAction} } } - return action + return event } -func applyTransitionRule(obj ObjectInfo) bool { +func applyTransitionRule(obj ObjectInfo, storageClass string) bool { if obj.DeleteMarker { return false } - globalTransitionState.queueTransitionTask(obj) + globalTransitionState.queueTransitionTask(obj, storageClass) return true } @@ -1213,14 +1213,14 @@ func applyExpiryRule(obj ObjectInfo, restoredObject, applyOnVersion bool) bool { } // Perform actions (removal or transitioning of objects), return true the action is successfully performed -func applyLifecycleAction(action lifecycle.Action, obj ObjectInfo) (success bool) { +func applyLifecycleAction(action lifecycle.Action, obj ObjectInfo, storageClass string) (success bool) { switch action { case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: success = applyExpiryRule(obj, false, action == lifecycle.DeleteVersionAction) case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: success = applyExpiryRule(obj, true, action == lifecycle.DeleteRestoredVersionAction) case lifecycle.TransitionAction, lifecycle.TransitionVersionAction: - success = applyTransitionRule(obj) + success = applyTransitionRule(obj, storageClass) } return } diff --git a/cmd/erasure-object.go b/cmd/erasure-object.go index 2ddeb745c..314b36efa 100644 --- a/cmd/erasure-object.go +++ b/cmd/erasure-object.go @@ -1530,9 +1530,9 @@ func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string if opts.Expiration.Expire { goi, _, err := er.getObjectInfoAndQuorum(ctx, bucket, object, opts) if err == nil { - action := evalActionFromLifecycle(ctx, *lc, rcfg, goi) + evt := evalActionFromLifecycle(ctx, *lc, rcfg, goi) var isErr bool - switch action { + switch evt.Action { case lifecycle.NoneAction: isErr = true case lifecycle.TransitionAction, lifecycle.TransitionVersionAction: diff --git a/cmd/erasure-server-pool-decom.go b/cmd/erasure-server-pool-decom.go index 6674f946d..bd5b7357a 100644 --- a/cmd/erasure-server-pool-decom.go +++ b/cmd/erasure-server-pool-decom.go @@ -701,14 +701,14 @@ func (z *erasureServerPools) decommissionPool(ctx context.Context, idx int, pool } versioned := vc != nil && vc.Versioned(object) objInfo := fi.ToObjectInfo(bucket, object, versioned) - action := evalActionFromLifecycle(ctx, *lc, lr, objInfo) - switch action { + evt := evalActionFromLifecycle(ctx, *lc, lr, objInfo) + switch evt.Action { case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: - globalExpiryState.enqueueByDays(objInfo, false, action == lifecycle.DeleteVersionAction) + globalExpiryState.enqueueByDays(objInfo, false, evt.Action == lifecycle.DeleteVersionAction) // Skip this entry. return true case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: - globalExpiryState.enqueueByDays(objInfo, true, action == lifecycle.DeleteRestoredVersionAction) + globalExpiryState.enqueueByDays(objInfo, true, evt.Action == lifecycle.DeleteRestoredVersionAction) // Skip this entry. return true } diff --git a/cmd/erasure-server-pool.go b/cmd/erasure-server-pool.go index 409493240..1e545269e 100644 --- a/cmd/erasure-server-pool.go +++ b/cmd/erasure-server-pool.go @@ -1302,8 +1302,8 @@ func (z *erasureServerPools) ListObjects(ctx context.Context, bucket, prefix, ma objInfo, err := z.GetObjectInfo(ctx, bucket, prefix, ObjectOptions{NoLock: true}) if err == nil { if opts.Lifecycle != nil { - action := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, objInfo) - switch action { + evt := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, objInfo) + switch evt.Action { case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: fallthrough case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: diff --git a/cmd/erasure-single-drive.go b/cmd/erasure-single-drive.go index 89b23f152..8d0076790 100644 --- a/cmd/erasure-single-drive.go +++ b/cmd/erasure-single-drive.go @@ -1477,9 +1477,9 @@ func (es *erasureSingle) DeleteObject(ctx context.Context, bucket, object string } if opts.Expiration.Expire { - action := evalActionFromLifecycle(ctx, *lc, rcfg, goi) + evt := evalActionFromLifecycle(ctx, *lc, rcfg, goi) var isErr bool - switch action { + switch evt.Action { case lifecycle.NoneAction: isErr = true case lifecycle.TransitionAction, lifecycle.TransitionVersionAction: @@ -2962,8 +2962,8 @@ func (es *erasureSingle) ListObjects(ctx context.Context, bucket, prefix, marker objInfo, err := es.GetObjectInfo(ctx, bucket, prefix, ObjectOptions{NoLock: true}) if err == nil { if opts.Lifecycle != nil { - action := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, objInfo) - switch action { + evt := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, objInfo) + switch evt.Action { case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: fallthrough case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: diff --git a/cmd/metacache-server-pool.go b/cmd/metacache-server-pool.go index 3cb1522fa..522b4a21b 100644 --- a/cmd/metacache-server-pool.go +++ b/cmd/metacache-server-pool.go @@ -627,14 +627,14 @@ func applyBucketActions(ctx context.Context, o listPathOptions, in <-chan metaCa objInfo := fi.ToObjectInfo(o.Bucket, obj.name, versioned) if o.Lifecycle != nil { - action := evalActionFromLifecycle(ctx, *o.Lifecycle, o.Retention, objInfo) - switch action { + evt := evalActionFromLifecycle(ctx, *o.Lifecycle, o.Retention, objInfo) + switch evt.Action { case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: - globalExpiryState.enqueueByDays(objInfo, false, action == lifecycle.DeleteVersionAction) + globalExpiryState.enqueueByDays(objInfo, false, evt.Action == lifecycle.DeleteVersionAction) // Skip this entry. continue case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: - globalExpiryState.enqueueByDays(objInfo, true, action == lifecycle.DeleteRestoredVersionAction) + globalExpiryState.enqueueByDays(objInfo, true, evt.Action == lifecycle.DeleteRestoredVersionAction) // Skip this entry. continue } diff --git a/cmd/object-handlers.go b/cmd/object-handlers.go index ae2cbd2a7..4ea15546b 100644 --- a/cmd/object-handlers.go +++ b/cmd/object-handlers.go @@ -495,15 +495,15 @@ func (api objectAPIHandlers) getObjectHandler(ctx context.Context, objectAPI Obj // Automatically remove the object/version is an expiry lifecycle rule can be applied if lc, err := globalLifecycleSys.Get(bucket); err == nil { rcfg, _ := globalBucketObjectLockSys.Get(bucket) - action := evalActionFromLifecycle(ctx, *lc, rcfg, objInfo) + evt := evalActionFromLifecycle(ctx, *lc, rcfg, objInfo) var success bool - switch action { + switch evt.Action { case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: - success = applyExpiryRule(objInfo, false, action == lifecycle.DeleteVersionAction) + success = applyExpiryRule(objInfo, false, evt.Action == lifecycle.DeleteVersionAction) case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: // Restored object delete would be still allowed to proceed as success // since transition behavior is slightly different. - applyExpiryRule(objInfo, true, action == lifecycle.DeleteRestoredVersionAction) + applyExpiryRule(objInfo, true, evt.Action == lifecycle.DeleteRestoredVersionAction) } if success { writeErrorResponseHeadersOnly(w, errorCodes.ToAPIErr(ErrNoSuchKey)) @@ -757,15 +757,15 @@ func (api objectAPIHandlers) headObjectHandler(ctx context.Context, objectAPI Ob // Automatically remove the object/version is an expiry lifecycle rule can be applied if lc, err := globalLifecycleSys.Get(bucket); err == nil { rcfg, _ := globalBucketObjectLockSys.Get(bucket) - action := evalActionFromLifecycle(ctx, *lc, rcfg, objInfo) + evt := evalActionFromLifecycle(ctx, *lc, rcfg, objInfo) var success bool - switch action { + switch evt.Action { case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: - success = applyExpiryRule(objInfo, false, action == lifecycle.DeleteVersionAction) + success = applyExpiryRule(objInfo, false, evt.Action == lifecycle.DeleteVersionAction) case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: // Restored object delete would be still allowed to proceed as success // since transition behavior is slightly different. - applyExpiryRule(objInfo, true, action == lifecycle.DeleteRestoredVersionAction) + applyExpiryRule(objInfo, true, evt.Action == lifecycle.DeleteRestoredVersionAction) } if success { writeErrorResponseHeadersOnly(w, errorCodes.ToAPIErr(ErrNoSuchKey)) diff --git a/internal/bucket/lifecycle/lifecycle.go b/internal/bucket/lifecycle/lifecycle.go index 59c857cc2..04204f345 100644 --- a/internal/bucket/lifecycle/lifecycle.go +++ b/internal/bucket/lifecycle/lifecycle.go @@ -235,9 +235,8 @@ func (lc Lifecycle) Validate() error { return nil } -// FilterActionableRules returns the rules actions that need to be executed -// after evaluating prefix/tag filtering -func (lc Lifecycle) FilterActionableRules(obj ObjectOpts) []Rule { +// FilterRules returns the rules filtered by the status, prefix and tags +func (lc Lifecycle) FilterRules(obj ObjectOpts) []Rule { if obj.Name == "" { return nil } @@ -249,40 +248,10 @@ func (lc Lifecycle) FilterActionableRules(obj ObjectOpts) []Rule { if !strings.HasPrefix(obj.Name, rule.GetPrefix()) { continue } - // Indicates whether MinIO will remove a delete marker with no - // noncurrent versions. If set to true, the delete marker will - // be expired; if set to false the policy takes no action. This - // cannot be specified with Days or Date in a Lifecycle - // Expiration Policy. - if rule.Expiration.DeleteMarker.val { - rules = append(rules, rule) + if !rule.Filter.TestTags(obj.UserTags) { continue } - // The NoncurrentVersionExpiration action requests MinIO to expire - // noncurrent versions of objects x days after the objects become - // noncurrent. - if !rule.NoncurrentVersionExpiration.IsDaysNull() { - rules = append(rules, rule) - continue - } - if rule.NoncurrentVersionExpiration.NewerNoncurrentVersions > 0 { - rules = append(rules, rule) - continue - } - // The NoncurrentVersionTransition action requests MinIO to transition - // noncurrent versions of objects x days after the objects become - // noncurrent. - if !rule.NoncurrentVersionTransition.IsNull() { - rules = append(rules, rule) - continue - } - - if rule.Filter.TestTags(obj.UserTags) { - rules = append(rules, rule) - } - if !rule.Transition.IsNull() { - rules = append(rules, rule) - } + rules = append(rules, rule) } return rules } @@ -310,13 +279,15 @@ func (o ObjectOpts) ExpiredObjectDeleteMarker() bool { return o.DeleteMarker && o.NumVersions == 1 } -type lifecycleEvent struct { - EventAction Action - RuleID string - Due time.Time +// Event contains a lifecycle action with associated info +type Event struct { + Action Action + RuleID string + Due time.Time + StorageClass string } -type lifecycleEvents []lifecycleEvent +type lifecycleEvents []Event func (es lifecycleEvents) Len() int { return len(es) @@ -330,11 +301,11 @@ func (es lifecycleEvents) Less(i, j int) bool { if es[i].Due.Equal(es[j].Due) { // Prefer Expiration over Transition for both current and noncurrent // versions - switch es[i].EventAction { + switch es[i].Action { case DeleteAction, DeleteVersionAction: return true } - switch es[j].EventAction { + switch es[j].Action { case DeleteAction, DeleteVersionAction: return false } @@ -345,11 +316,11 @@ func (es lifecycleEvents) Less(i, j int) bool { return es[i].Due.Before(es[j].Due) } -// eval returns the lifecycle event applicable at now. If now is the zero value of time.Time, it returns the upcoming lifecycle event. -func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent { - var events []lifecycleEvent +// Eval returns the lifecycle event applicable at now. If now is the zero value of time.Time, it returns the upcoming lifecycle event. +func (lc Lifecycle) Eval(obj ObjectOpts, now time.Time) Event { + var events []Event if obj.ModTime.IsZero() { - return lifecycleEvent{} + return Event{} } // Handle expiry of restored object; NB Restored Objects have expiry set on @@ -361,23 +332,23 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent { action = DeleteRestoredVersionAction } - events = append(events, lifecycleEvent{ - EventAction: action, - Due: now, + events = append(events, Event{ + Action: action, + Due: now, }) } - for _, rule := range lc.FilterActionableRules(obj) { + for _, rule := range lc.FilterRules(obj) { if obj.ExpiredObjectDeleteMarker() { if rule.Expiration.DeleteMarker.val { // Indicates whether MinIO will remove a delete marker with no noncurrent versions. // Only latest marker is removed. If set to true, the delete marker will be expired; // if set to false the policy takes no action. This cannot be specified with Days or // Date in a Lifecycle Expiration Policy. - events = append(events, lifecycleEvent{ - EventAction: DeleteVersionAction, - RuleID: rule.ID, - Due: now, + events = append(events, Event{ + Action: DeleteVersionAction, + RuleID: rule.ID, + Due: now, }) // No other conflicting actions apply to an expired object delete marker break @@ -388,10 +359,10 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent { // once delete markers are old enough to satisfy the age criteria. // https://docs.aws.amazon.com/AmazonS3/latest/userguide/lifecycle-configuration-examples.html if expectedExpiry := ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days)); now.After(expectedExpiry) { - events = append(events, lifecycleEvent{ - EventAction: DeleteVersionAction, - RuleID: rule.ID, - Due: expectedExpiry, + events = append(events, Event{ + Action: DeleteVersionAction, + RuleID: rule.ID, + Due: expectedExpiry, }) // No other conflicting actions apply to an expired object delete marker break @@ -410,10 +381,10 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent { // Non current versions should be deleted if their age exceeds non current days configuration // https://docs.aws.amazon.com/AmazonS3/latest/dev/intro-lifecycle-rules.html#intro-lifecycle-rules-actions if expectedExpiry := ExpectedExpiryTime(obj.SuccessorModTime, int(rule.NoncurrentVersionExpiration.NoncurrentDays)); now.After(expectedExpiry) { - events = append(events, lifecycleEvent{ - EventAction: DeleteVersionAction, - RuleID: rule.ID, - Due: expectedExpiry, + events = append(events, Event{ + Action: DeleteVersionAction, + RuleID: rule.ID, + Due: expectedExpiry, }) } } @@ -423,10 +394,11 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent { // Non current versions should be transitioned if their age exceeds non current days configuration // https://docs.aws.amazon.com/AmazonS3/latest/dev/intro-lifecycle-rules.html#intro-lifecycle-rules-actions if due, ok := rule.NoncurrentVersionTransition.NextDue(obj); ok && now.After(due) { - events = append(events, lifecycleEvent{ - EventAction: TransitionVersionAction, - RuleID: rule.ID, - Due: due, + events = append(events, Event{ + Action: TransitionVersionAction, + RuleID: rule.ID, + Due: due, + StorageClass: rule.NoncurrentVersionTransition.StorageClass, }) } } @@ -437,28 +409,29 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent { switch { case !rule.Expiration.IsDateNull(): if time.Now().UTC().After(rule.Expiration.Date.Time) { - events = append(events, lifecycleEvent{ - EventAction: DeleteAction, - RuleID: rule.ID, - Due: rule.Expiration.Date.Time, + events = append(events, Event{ + Action: DeleteAction, + RuleID: rule.ID, + Due: rule.Expiration.Date.Time, }) } case !rule.Expiration.IsDaysNull(): if expectedExpiry := ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days)); now.After(expectedExpiry) { - events = append(events, lifecycleEvent{ - EventAction: DeleteAction, - RuleID: rule.ID, - Due: expectedExpiry, + events = append(events, Event{ + Action: DeleteAction, + RuleID: rule.ID, + Due: expectedExpiry, }) } } if obj.TransitionStatus != TransitionComplete { if due, ok := rule.Transition.NextDue(obj); ok && now.After(due) { - events = append(events, lifecycleEvent{ - EventAction: TransitionAction, - RuleID: rule.ID, - Due: due, + events = append(events, Event{ + Action: TransitionAction, + RuleID: rule.ID, + Due: due, + StorageClass: rule.Transition.StorageClass, }) } } @@ -470,15 +443,15 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent { return events[0] } - return lifecycleEvent{ - EventAction: NoneAction, + return Event{ + Action: NoneAction, } } // ComputeAction returns the action to perform by evaluating all lifecycle rules // against the object name and its modification time. func (lc Lifecycle) ComputeAction(obj ObjectOpts) Action { - return lc.eval(obj, time.Now().UTC()).EventAction + return lc.Eval(obj, time.Now().UTC()).Action } // ExpectedExpiryTime calculates the expiry, transition or restore date/time based on a object modtime. @@ -498,8 +471,8 @@ func ExpectedExpiryTime(modTime time.Time, days int) time.Time { // SetPredictionHeaders sets time to expiry and transition headers on w for a // given obj. func (lc Lifecycle) SetPredictionHeaders(w http.ResponseWriter, obj ObjectOpts) { - event := lc.eval(obj, time.Time{}) - switch event.EventAction { + event := lc.Eval(obj, time.Time{}) + switch event.Action { case DeleteAction, DeleteVersionAction: w.Header()[xhttp.AmzExpiration] = []string{ fmt.Sprintf(`expiry-date="%s", rule-id="%s"`, event.Due.Format(http.TimeFormat), event.RuleID), @@ -511,26 +484,13 @@ func (lc Lifecycle) SetPredictionHeaders(w http.ResponseWriter, obj ObjectOpts) } } -// TransitionTier returns remote tier that applies to obj per ILM rules. -func (lc Lifecycle) TransitionTier(obj ObjectOpts) string { - for _, rule := range lc.FilterActionableRules(obj) { - if obj.IsLatest && rule.Transition.StorageClass != "" { - return rule.Transition.StorageClass - } - if !obj.IsLatest && rule.NoncurrentVersionTransition.StorageClass != "" { - return rule.NoncurrentVersionTransition.StorageClass - } - } - return "" -} - // NoncurrentVersionsExpirationLimit returns the number of noncurrent versions // to be retained from the first applicable rule per S3 behavior. func (lc Lifecycle) NoncurrentVersionsExpirationLimit(obj ObjectOpts) (string, int, int) { var lim int var days int var ruleID string - for _, rule := range lc.FilterActionableRules(obj) { + for _, rule := range lc.FilterRules(obj) { if rule.NoncurrentVersionExpiration.NewerNoncurrentVersions == 0 { continue } diff --git a/internal/bucket/lifecycle/lifecycle_test.go b/internal/bucket/lifecycle/lifecycle_test.go index 41748d9ff..7bf7c35db 100644 --- a/internal/bucket/lifecycle/lifecycle_test.go +++ b/internal/bucket/lifecycle/lifecycle_test.go @@ -725,18 +725,120 @@ func TestTransitionTier(t *testing.T) { }, } + now := time.Now().UTC() + obj1 := ObjectOpts{ Name: "obj1", IsLatest: true, + ModTime: now, } + obj2 := ObjectOpts{ - Name: "obj2", + Name: "obj2", + ModTime: now, } - if got := lc.TransitionTier(obj1); got != "TIER-1" { - t.Fatalf("Expected TIER-1 but got %s", got) + + // Go back seven days in the past + now = now.Add(7 * 24 * time.Hour) + + evt := lc.Eval(obj1, now) + if evt.Action != TransitionAction { + t.Fatalf("Expected action: %s but got %s", TransitionAction, evt.Action) } - if got := lc.TransitionTier(obj2); got != "TIER-2" { - t.Fatalf("Expected TIER-2 but got %s", got) + if evt.StorageClass != "TIER-1" { + t.Fatalf("Expected TIER-1 but got %s", evt.StorageClass) + } + + evt = lc.Eval(obj2, now) + if evt.Action != TransitionVersionAction { + t.Fatalf("Expected action: %s but got %s", TransitionVersionAction, evt.Action) + } + if evt.StorageClass != "TIER-2" { + t.Fatalf("Expected TIER-2 but got %s", evt.StorageClass) + } +} + +func TestTransitionTierWithPrefixAndTags(t *testing.T) { + lc := Lifecycle{ + Rules: []Rule{ + { + ID: "rule-1", + Status: "Enabled", + Filter: Filter{ + Prefix: Prefix{ + set: true, + string: "abcd/", + }, + }, + Transition: Transition{ + Days: TransitionDays(3), + StorageClass: "TIER-1", + }, + }, + { + ID: "rule-2", + Status: "Enabled", + Filter: Filter{ + tagSet: true, + Tag: Tag{ + Key: "priority", + Value: "low", + }, + }, + Transition: Transition{ + Days: TransitionDays(3), + StorageClass: "TIER-2", + }, + }, + }, + } + + now := time.Now().UTC() + + obj1 := ObjectOpts{ + Name: "obj1", + IsLatest: true, + ModTime: now, + } + + obj2 := ObjectOpts{ + Name: "abcd/obj2", + IsLatest: true, + ModTime: now, + } + + obj3 := ObjectOpts{ + Name: "obj3", + IsLatest: true, + ModTime: now, + UserTags: "priority=low", + } + + // Go back seven days in the past + now = now.Add(7 * 24 * time.Hour) + + // Eval object 1 + evt := lc.Eval(obj1, now) + if evt.Action != NoneAction { + t.Fatalf("Expected action: %s but got %s", NoneAction, evt.Action) + } + + // Eval object 2 + evt = lc.Eval(obj2, now) + if evt.Action != TransitionAction { + t.Fatalf("Expected action: %s but got %s", TransitionAction, evt.Action) + } + if evt.StorageClass != "TIER-1" { + t.Fatalf("Expected TIER-1 but got %s", evt.StorageClass) + } + + // Eval object 3 + evt = lc.Eval(obj3, now) + if evt.Action != TransitionAction { + t.Fatalf("Expected action: %s but got %s", TransitionAction, evt.Action) + } + if evt.StorageClass != "TIER-2" { + t.Fatalf("Expected TIER-2 but got %s", evt.StorageClass) } }