lifecycle: refactor rules filtering and tagging support (#15914)

This commit is contained in:
Anis Elleuch 2022-10-21 18:46:53 +01:00 committed by GitHub
parent 5aba2aedb3
commit ac85c2af76
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 229 additions and 168 deletions

View File

@ -155,9 +155,14 @@ type newerNoncurrentTask struct {
versions []ObjectToDelete versions []ObjectToDelete
} }
type transitionTask struct {
tier string
objInfo ObjectInfo
}
type transitionState struct { type transitionState struct {
once sync.Once once sync.Once
transitionCh chan ObjectInfo transitionCh chan transitionTask
ctx context.Context ctx context.Context
objAPI ObjectLayer objAPI ObjectLayer
@ -171,13 +176,13 @@ type transitionState struct {
lastDayStats map[string]*lastDayTierStats lastDayStats map[string]*lastDayTierStats
} }
func (t *transitionState) queueTransitionTask(oi ObjectInfo) { func (t *transitionState) queueTransitionTask(oi ObjectInfo, sc string) {
select { select {
case <-GlobalContext.Done(): case <-GlobalContext.Done():
t.once.Do(func() { t.once.Do(func() {
close(t.transitionCh) close(t.transitionCh)
}) })
case t.transitionCh <- oi: case t.transitionCh <- transitionTask{objInfo: oi, tier: sc}:
default: default:
} }
} }
@ -186,7 +191,7 @@ var globalTransitionState *transitionState
func newTransitionState(ctx context.Context, objAPI ObjectLayer) *transitionState { func newTransitionState(ctx context.Context, objAPI ObjectLayer) *transitionState {
return &transitionState{ return &transitionState{
transitionCh: make(chan ObjectInfo, 10000), transitionCh: make(chan transitionTask, 10000),
ctx: ctx, ctx: ctx,
objAPI: objAPI, objAPI: objAPI,
killCh: make(chan struct{}), killCh: make(chan struct{}),
@ -213,27 +218,25 @@ func (t *transitionState) worker(ctx context.Context, objectAPI ObjectLayer) {
return return
case <-ctx.Done(): case <-ctx.Done():
return return
case oi, ok := <-t.transitionCh: case task, ok := <-t.transitionCh:
if !ok { if !ok {
return return
} }
atomic.AddInt32(&t.activeTasks, 1) atomic.AddInt32(&t.activeTasks, 1)
var tier string if err := transitionObject(ctx, objectAPI, task.objInfo, task.tier); err != nil {
var err error logger.LogIf(ctx, fmt.Errorf("Transition failed for %s/%s version:%s with %w",
if tier, err = transitionObject(ctx, objectAPI, oi); err != nil { task.objInfo.Bucket, task.objInfo.Name, task.objInfo.VersionID, err))
logger.LogIf(ctx, fmt.Errorf("Transition failed for %s/%s version:%s with %w", oi.Bucket, oi.Name, oi.VersionID, err))
} else { } else {
ts := tierStats{ ts := tierStats{
TotalSize: uint64(oi.Size), TotalSize: uint64(task.objInfo.Size),
NumVersions: 1, NumVersions: 1,
} }
if oi.IsLatest { if task.objInfo.IsLatest {
ts.NumObjects = 1 ts.NumObjects = 1
} }
t.addLastDayStats(tier, ts) t.addLastDayStats(task.tier, ts)
} }
atomic.AddInt32(&t.activeTasks, -1) 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). // This is to be called after a successful upload of an object (version).
func enqueueTransitionImmediate(obj ObjectInfo) { func enqueueTransitionImmediate(obj ObjectInfo) {
if lc, err := globalLifecycleSys.Get(obj.Bucket); err == nil { 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: 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 // 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 // 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. // to the transition tier without decrypting or re-encrypting.
func transitionObject(ctx context.Context, objectAPI ObjectLayer, oi ObjectInfo) (string, error) { func transitionObject(ctx context.Context, objectAPI ObjectLayer, oi ObjectInfo, tier string) error {
lc, err := globalLifecycleSys.Get(oi.Bucket)
if err != nil {
return "", err
}
tier := lc.TransitionTier(oi.ToLifecycleOpts())
opts := ObjectOptions{ opts := ObjectOptions{
Transition: TransitionOptions{ Transition: TransitionOptions{
Status: lifecycle.TransitionPending, Status: lifecycle.TransitionPending,
@ -420,7 +419,7 @@ func transitionObject(ctx context.Context, objectAPI ObjectLayer, oi ObjectInfo)
VersionSuspended: globalBucketVersioningSys.PrefixSuspended(oi.Bucket, oi.Name), VersionSuspended: globalBucketVersioningSys.PrefixSuspended(oi.Bucket, oi.Name),
MTime: oi.ModTime, 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 { type auditTierOp struct {

View File

@ -963,21 +963,21 @@ func (i *scannerItem) applyLifecycle(ctx context.Context, o ObjectLayer, oi Obje
versionID := oi.VersionID versionID := oi.VersionID
rCfg, _ := globalBucketObjectLockSys.Get(i.bucket) rCfg, _ := globalBucketObjectLockSys.Get(i.bucket)
action := evalActionFromLifecycle(ctx, *i.lifeCycle, rCfg, oi) lcEvt := evalActionFromLifecycle(ctx, *i.lifeCycle, rCfg, oi)
if i.debug { if i.debug {
if versionID != "" { 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 { } 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: 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: case lifecycle.TransitionAction, lifecycle.TransitionVersionAction:
return applyLifecycleAction(action, oi), size return applyLifecycleAction(lcEvt.Action, oi, lcEvt.StorageClass), size
default: default:
// No action. // No action.
return false, size return false, size
@ -1109,21 +1109,21 @@ func (i *scannerItem) applyActions(ctx context.Context, o ObjectLayer, oi Object
return size return size
} }
func evalActionFromLifecycle(ctx context.Context, lc lifecycle.Lifecycle, lr lock.Retention, obj ObjectInfo) (action lifecycle.Action) { func evalActionFromLifecycle(ctx context.Context, lc lifecycle.Lifecycle, lr lock.Retention, obj ObjectInfo) lifecycle.Event {
action = lc.ComputeAction(obj.ToLifecycleOpts()) event := lc.Eval(obj.ToLifecycleOpts(), time.Now().UTC())
if serverDebugLog { 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 { if event.Action == lifecycle.NoneAction {
return action return event
} }
switch action { switch event.Action {
case lifecycle.DeleteVersionAction, lifecycle.DeleteRestoredVersionAction: case lifecycle.DeleteVersionAction, lifecycle.DeleteRestoredVersionAction:
// Defensive code, should never happen // Defensive code, should never happen
if obj.VersionID == "" { if obj.VersionID == "" {
return lifecycle.NoneAction return lifecycle.Event{Action: lifecycle.NoneAction}
} }
if lr.LockEnabled && enforceRetentionForDeletion(ctx, obj) { if lr.LockEnabled && enforceRetentionForDeletion(ctx, obj) {
if serverDebugLog { 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) 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 { if obj.DeleteMarker {
return false return false
} }
globalTransitionState.queueTransitionTask(obj) globalTransitionState.queueTransitionTask(obj, storageClass)
return true 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 // 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 { switch action {
case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: case lifecycle.DeleteVersionAction, lifecycle.DeleteAction:
success = applyExpiryRule(obj, false, action == lifecycle.DeleteVersionAction) success = applyExpiryRule(obj, false, action == lifecycle.DeleteVersionAction)
case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:
success = applyExpiryRule(obj, true, action == lifecycle.DeleteRestoredVersionAction) success = applyExpiryRule(obj, true, action == lifecycle.DeleteRestoredVersionAction)
case lifecycle.TransitionAction, lifecycle.TransitionVersionAction: case lifecycle.TransitionAction, lifecycle.TransitionVersionAction:
success = applyTransitionRule(obj) success = applyTransitionRule(obj, storageClass)
} }
return return
} }

View File

@ -1530,9 +1530,9 @@ func (er erasureObjects) DeleteObject(ctx context.Context, bucket, object string
if opts.Expiration.Expire { if opts.Expiration.Expire {
goi, _, err := er.getObjectInfoAndQuorum(ctx, bucket, object, opts) goi, _, err := er.getObjectInfoAndQuorum(ctx, bucket, object, opts)
if err == nil { if err == nil {
action := evalActionFromLifecycle(ctx, *lc, rcfg, goi) evt := evalActionFromLifecycle(ctx, *lc, rcfg, goi)
var isErr bool var isErr bool
switch action { switch evt.Action {
case lifecycle.NoneAction: case lifecycle.NoneAction:
isErr = true isErr = true
case lifecycle.TransitionAction, lifecycle.TransitionVersionAction: case lifecycle.TransitionAction, lifecycle.TransitionVersionAction:

View File

@ -701,14 +701,14 @@ func (z *erasureServerPools) decommissionPool(ctx context.Context, idx int, pool
} }
versioned := vc != nil && vc.Versioned(object) versioned := vc != nil && vc.Versioned(object)
objInfo := fi.ToObjectInfo(bucket, object, versioned) objInfo := fi.ToObjectInfo(bucket, object, versioned)
action := evalActionFromLifecycle(ctx, *lc, lr, objInfo) evt := evalActionFromLifecycle(ctx, *lc, lr, objInfo)
switch action { switch evt.Action {
case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: case lifecycle.DeleteVersionAction, lifecycle.DeleteAction:
globalExpiryState.enqueueByDays(objInfo, false, action == lifecycle.DeleteVersionAction) globalExpiryState.enqueueByDays(objInfo, false, evt.Action == lifecycle.DeleteVersionAction)
// Skip this entry. // Skip this entry.
return true return true
case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:
globalExpiryState.enqueueByDays(objInfo, true, action == lifecycle.DeleteRestoredVersionAction) globalExpiryState.enqueueByDays(objInfo, true, evt.Action == lifecycle.DeleteRestoredVersionAction)
// Skip this entry. // Skip this entry.
return true return true
} }

View File

@ -1302,8 +1302,8 @@ func (z *erasureServerPools) ListObjects(ctx context.Context, bucket, prefix, ma
objInfo, err := z.GetObjectInfo(ctx, bucket, prefix, ObjectOptions{NoLock: true}) objInfo, err := z.GetObjectInfo(ctx, bucket, prefix, ObjectOptions{NoLock: true})
if err == nil { if err == nil {
if opts.Lifecycle != nil { if opts.Lifecycle != nil {
action := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, objInfo) evt := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, objInfo)
switch action { switch evt.Action {
case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: case lifecycle.DeleteVersionAction, lifecycle.DeleteAction:
fallthrough fallthrough
case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:

View File

@ -1477,9 +1477,9 @@ func (es *erasureSingle) DeleteObject(ctx context.Context, bucket, object string
} }
if opts.Expiration.Expire { if opts.Expiration.Expire {
action := evalActionFromLifecycle(ctx, *lc, rcfg, goi) evt := evalActionFromLifecycle(ctx, *lc, rcfg, goi)
var isErr bool var isErr bool
switch action { switch evt.Action {
case lifecycle.NoneAction: case lifecycle.NoneAction:
isErr = true isErr = true
case lifecycle.TransitionAction, lifecycle.TransitionVersionAction: 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}) objInfo, err := es.GetObjectInfo(ctx, bucket, prefix, ObjectOptions{NoLock: true})
if err == nil { if err == nil {
if opts.Lifecycle != nil { if opts.Lifecycle != nil {
action := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, objInfo) evt := evalActionFromLifecycle(ctx, *opts.Lifecycle, opts.Retention, objInfo)
switch action { switch evt.Action {
case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: case lifecycle.DeleteVersionAction, lifecycle.DeleteAction:
fallthrough fallthrough
case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:

View File

@ -627,14 +627,14 @@ func applyBucketActions(ctx context.Context, o listPathOptions, in <-chan metaCa
objInfo := fi.ToObjectInfo(o.Bucket, obj.name, versioned) objInfo := fi.ToObjectInfo(o.Bucket, obj.name, versioned)
if o.Lifecycle != nil { if o.Lifecycle != nil {
action := evalActionFromLifecycle(ctx, *o.Lifecycle, o.Retention, objInfo) evt := evalActionFromLifecycle(ctx, *o.Lifecycle, o.Retention, objInfo)
switch action { switch evt.Action {
case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: case lifecycle.DeleteVersionAction, lifecycle.DeleteAction:
globalExpiryState.enqueueByDays(objInfo, false, action == lifecycle.DeleteVersionAction) globalExpiryState.enqueueByDays(objInfo, false, evt.Action == lifecycle.DeleteVersionAction)
// Skip this entry. // Skip this entry.
continue continue
case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction: case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:
globalExpiryState.enqueueByDays(objInfo, true, action == lifecycle.DeleteRestoredVersionAction) globalExpiryState.enqueueByDays(objInfo, true, evt.Action == lifecycle.DeleteRestoredVersionAction)
// Skip this entry. // Skip this entry.
continue continue
} }

View File

@ -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 // Automatically remove the object/version is an expiry lifecycle rule can be applied
if lc, err := globalLifecycleSys.Get(bucket); err == nil { if lc, err := globalLifecycleSys.Get(bucket); err == nil {
rcfg, _ := globalBucketObjectLockSys.Get(bucket) rcfg, _ := globalBucketObjectLockSys.Get(bucket)
action := evalActionFromLifecycle(ctx, *lc, rcfg, objInfo) evt := evalActionFromLifecycle(ctx, *lc, rcfg, objInfo)
var success bool var success bool
switch action { switch evt.Action {
case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: 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: case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:
// Restored object delete would be still allowed to proceed as success // Restored object delete would be still allowed to proceed as success
// since transition behavior is slightly different. // since transition behavior is slightly different.
applyExpiryRule(objInfo, true, action == lifecycle.DeleteRestoredVersionAction) applyExpiryRule(objInfo, true, evt.Action == lifecycle.DeleteRestoredVersionAction)
} }
if success { if success {
writeErrorResponseHeadersOnly(w, errorCodes.ToAPIErr(ErrNoSuchKey)) 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 // Automatically remove the object/version is an expiry lifecycle rule can be applied
if lc, err := globalLifecycleSys.Get(bucket); err == nil { if lc, err := globalLifecycleSys.Get(bucket); err == nil {
rcfg, _ := globalBucketObjectLockSys.Get(bucket) rcfg, _ := globalBucketObjectLockSys.Get(bucket)
action := evalActionFromLifecycle(ctx, *lc, rcfg, objInfo) evt := evalActionFromLifecycle(ctx, *lc, rcfg, objInfo)
var success bool var success bool
switch action { switch evt.Action {
case lifecycle.DeleteVersionAction, lifecycle.DeleteAction: 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: case lifecycle.DeleteRestoredAction, lifecycle.DeleteRestoredVersionAction:
// Restored object delete would be still allowed to proceed as success // Restored object delete would be still allowed to proceed as success
// since transition behavior is slightly different. // since transition behavior is slightly different.
applyExpiryRule(objInfo, true, action == lifecycle.DeleteRestoredVersionAction) applyExpiryRule(objInfo, true, evt.Action == lifecycle.DeleteRestoredVersionAction)
} }
if success { if success {
writeErrorResponseHeadersOnly(w, errorCodes.ToAPIErr(ErrNoSuchKey)) writeErrorResponseHeadersOnly(w, errorCodes.ToAPIErr(ErrNoSuchKey))

View File

@ -235,9 +235,8 @@ func (lc Lifecycle) Validate() error {
return nil return nil
} }
// FilterActionableRules returns the rules actions that need to be executed // FilterRules returns the rules filtered by the status, prefix and tags
// after evaluating prefix/tag filtering func (lc Lifecycle) FilterRules(obj ObjectOpts) []Rule {
func (lc Lifecycle) FilterActionableRules(obj ObjectOpts) []Rule {
if obj.Name == "" { if obj.Name == "" {
return nil return nil
} }
@ -249,40 +248,10 @@ func (lc Lifecycle) FilterActionableRules(obj ObjectOpts) []Rule {
if !strings.HasPrefix(obj.Name, rule.GetPrefix()) { if !strings.HasPrefix(obj.Name, rule.GetPrefix()) {
continue continue
} }
// Indicates whether MinIO will remove a delete marker with no if !rule.Filter.TestTags(obj.UserTags) {
// 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)
continue 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) 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)
}
} }
return rules return rules
} }
@ -310,13 +279,15 @@ func (o ObjectOpts) ExpiredObjectDeleteMarker() bool {
return o.DeleteMarker && o.NumVersions == 1 return o.DeleteMarker && o.NumVersions == 1
} }
type lifecycleEvent struct { // Event contains a lifecycle action with associated info
EventAction Action type Event struct {
Action Action
RuleID string RuleID string
Due time.Time Due time.Time
StorageClass string
} }
type lifecycleEvents []lifecycleEvent type lifecycleEvents []Event
func (es lifecycleEvents) Len() int { func (es lifecycleEvents) Len() int {
return len(es) return len(es)
@ -330,11 +301,11 @@ func (es lifecycleEvents) Less(i, j int) bool {
if es[i].Due.Equal(es[j].Due) { if es[i].Due.Equal(es[j].Due) {
// Prefer Expiration over Transition for both current and noncurrent // Prefer Expiration over Transition for both current and noncurrent
// versions // versions
switch es[i].EventAction { switch es[i].Action {
case DeleteAction, DeleteVersionAction: case DeleteAction, DeleteVersionAction:
return true return true
} }
switch es[j].EventAction { switch es[j].Action {
case DeleteAction, DeleteVersionAction: case DeleteAction, DeleteVersionAction:
return false return false
} }
@ -345,11 +316,11 @@ func (es lifecycleEvents) Less(i, j int) bool {
return es[i].Due.Before(es[j].Due) 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. // 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 { func (lc Lifecycle) Eval(obj ObjectOpts, now time.Time) Event {
var events []lifecycleEvent var events []Event
if obj.ModTime.IsZero() { if obj.ModTime.IsZero() {
return lifecycleEvent{} return Event{}
} }
// Handle expiry of restored object; NB Restored Objects have expiry set on // Handle expiry of restored object; NB Restored Objects have expiry set on
@ -361,21 +332,21 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent {
action = DeleteRestoredVersionAction action = DeleteRestoredVersionAction
} }
events = append(events, lifecycleEvent{ events = append(events, Event{
EventAction: action, Action: action,
Due: now, Due: now,
}) })
} }
for _, rule := range lc.FilterActionableRules(obj) { for _, rule := range lc.FilterRules(obj) {
if obj.ExpiredObjectDeleteMarker() { if obj.ExpiredObjectDeleteMarker() {
if rule.Expiration.DeleteMarker.val { if rule.Expiration.DeleteMarker.val {
// Indicates whether MinIO will remove a delete marker with no noncurrent versions. // 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; // 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 // if set to false the policy takes no action. This cannot be specified with Days or
// Date in a Lifecycle Expiration Policy. // Date in a Lifecycle Expiration Policy.
events = append(events, lifecycleEvent{ events = append(events, Event{
EventAction: DeleteVersionAction, Action: DeleteVersionAction,
RuleID: rule.ID, RuleID: rule.ID,
Due: now, Due: now,
}) })
@ -388,8 +359,8 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent {
// once delete markers are old enough to satisfy the age criteria. // once delete markers are old enough to satisfy the age criteria.
// https://docs.aws.amazon.com/AmazonS3/latest/userguide/lifecycle-configuration-examples.html // https://docs.aws.amazon.com/AmazonS3/latest/userguide/lifecycle-configuration-examples.html
if expectedExpiry := ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days)); now.After(expectedExpiry) { if expectedExpiry := ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days)); now.After(expectedExpiry) {
events = append(events, lifecycleEvent{ events = append(events, Event{
EventAction: DeleteVersionAction, Action: DeleteVersionAction,
RuleID: rule.ID, RuleID: rule.ID,
Due: expectedExpiry, Due: expectedExpiry,
}) })
@ -410,8 +381,8 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent {
// Non current versions should be deleted if their age exceeds non current days configuration // 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 // 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) { if expectedExpiry := ExpectedExpiryTime(obj.SuccessorModTime, int(rule.NoncurrentVersionExpiration.NoncurrentDays)); now.After(expectedExpiry) {
events = append(events, lifecycleEvent{ events = append(events, Event{
EventAction: DeleteVersionAction, Action: DeleteVersionAction,
RuleID: rule.ID, RuleID: rule.ID,
Due: expectedExpiry, 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 // 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 // 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) { if due, ok := rule.NoncurrentVersionTransition.NextDue(obj); ok && now.After(due) {
events = append(events, lifecycleEvent{ events = append(events, Event{
EventAction: TransitionVersionAction, Action: TransitionVersionAction,
RuleID: rule.ID, RuleID: rule.ID,
Due: due, Due: due,
StorageClass: rule.NoncurrentVersionTransition.StorageClass,
}) })
} }
} }
@ -437,16 +409,16 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent {
switch { switch {
case !rule.Expiration.IsDateNull(): case !rule.Expiration.IsDateNull():
if time.Now().UTC().After(rule.Expiration.Date.Time) { if time.Now().UTC().After(rule.Expiration.Date.Time) {
events = append(events, lifecycleEvent{ events = append(events, Event{
EventAction: DeleteAction, Action: DeleteAction,
RuleID: rule.ID, RuleID: rule.ID,
Due: rule.Expiration.Date.Time, Due: rule.Expiration.Date.Time,
}) })
} }
case !rule.Expiration.IsDaysNull(): case !rule.Expiration.IsDaysNull():
if expectedExpiry := ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days)); now.After(expectedExpiry) { if expectedExpiry := ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days)); now.After(expectedExpiry) {
events = append(events, lifecycleEvent{ events = append(events, Event{
EventAction: DeleteAction, Action: DeleteAction,
RuleID: rule.ID, RuleID: rule.ID,
Due: expectedExpiry, Due: expectedExpiry,
}) })
@ -455,10 +427,11 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent {
if obj.TransitionStatus != TransitionComplete { if obj.TransitionStatus != TransitionComplete {
if due, ok := rule.Transition.NextDue(obj); ok && now.After(due) { if due, ok := rule.Transition.NextDue(obj); ok && now.After(due) {
events = append(events, lifecycleEvent{ events = append(events, Event{
EventAction: TransitionAction, Action: TransitionAction,
RuleID: rule.ID, RuleID: rule.ID,
Due: due, Due: due,
StorageClass: rule.Transition.StorageClass,
}) })
} }
} }
@ -470,15 +443,15 @@ func (lc Lifecycle) eval(obj ObjectOpts, now time.Time) lifecycleEvent {
return events[0] return events[0]
} }
return lifecycleEvent{ return Event{
EventAction: NoneAction, Action: NoneAction,
} }
} }
// ComputeAction returns the action to perform by evaluating all lifecycle rules // ComputeAction returns the action to perform by evaluating all lifecycle rules
// against the object name and its modification time. // against the object name and its modification time.
func (lc Lifecycle) ComputeAction(obj ObjectOpts) Action { 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. // 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 // SetPredictionHeaders sets time to expiry and transition headers on w for a
// given obj. // given obj.
func (lc Lifecycle) SetPredictionHeaders(w http.ResponseWriter, obj ObjectOpts) { func (lc Lifecycle) SetPredictionHeaders(w http.ResponseWriter, obj ObjectOpts) {
event := lc.eval(obj, time.Time{}) event := lc.Eval(obj, time.Time{})
switch event.EventAction { switch event.Action {
case DeleteAction, DeleteVersionAction: case DeleteAction, DeleteVersionAction:
w.Header()[xhttp.AmzExpiration] = []string{ w.Header()[xhttp.AmzExpiration] = []string{
fmt.Sprintf(`expiry-date="%s", rule-id="%s"`, event.Due.Format(http.TimeFormat), event.RuleID), 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 // NoncurrentVersionsExpirationLimit returns the number of noncurrent versions
// to be retained from the first applicable rule per S3 behavior. // to be retained from the first applicable rule per S3 behavior.
func (lc Lifecycle) NoncurrentVersionsExpirationLimit(obj ObjectOpts) (string, int, int) { func (lc Lifecycle) NoncurrentVersionsExpirationLimit(obj ObjectOpts) (string, int, int) {
var lim int var lim int
var days int var days int
var ruleID string var ruleID string
for _, rule := range lc.FilterActionableRules(obj) { for _, rule := range lc.FilterRules(obj) {
if rule.NoncurrentVersionExpiration.NewerNoncurrentVersions == 0 { if rule.NoncurrentVersionExpiration.NewerNoncurrentVersions == 0 {
continue continue
} }

View File

@ -725,18 +725,120 @@ func TestTransitionTier(t *testing.T) {
}, },
} }
now := time.Now().UTC()
obj1 := ObjectOpts{ obj1 := ObjectOpts{
Name: "obj1", Name: "obj1",
IsLatest: true, IsLatest: true,
ModTime: now,
} }
obj2 := ObjectOpts{ 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" { if evt.StorageClass != "TIER-1" {
t.Fatalf("Expected TIER-2 but got %s", got) 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)
} }
} }