mirror of https://github.com/minio/minio.git
Add bucket lifecycle expiry feature (#7834)
This commit is contained in:
parent
a8296445ad
commit
1ce8d2c476
|
@ -24,6 +24,7 @@ import (
|
||||||
"github.com/gorilla/mux"
|
"github.com/gorilla/mux"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
"github.com/minio/minio/pkg/lifecycle"
|
"github.com/minio/minio/pkg/lifecycle"
|
||||||
|
"github.com/minio/minio/pkg/policy"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
@ -47,7 +48,7 @@ func (api objectAPIHandlers) PutBucketLifecycleHandler(w http.ResponseWriter, r
|
||||||
vars := mux.Vars(r)
|
vars := mux.Vars(r)
|
||||||
bucket := vars["bucket"]
|
bucket := vars["bucket"]
|
||||||
|
|
||||||
if s3Error := checkRequestAuthType(ctx, r, lifecycle.PutBucketLifecycleAction, bucket, ""); s3Error != ErrNone {
|
if s3Error := checkRequestAuthType(ctx, r, policy.PutBucketLifecycleAction, bucket, ""); s3Error != ErrNone {
|
||||||
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL, guessIsBrowserReq(r))
|
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL, guessIsBrowserReq(r))
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
@ -97,7 +98,7 @@ func (api objectAPIHandlers) GetBucketLifecycleHandler(w http.ResponseWriter, r
|
||||||
vars := mux.Vars(r)
|
vars := mux.Vars(r)
|
||||||
bucket := vars["bucket"]
|
bucket := vars["bucket"]
|
||||||
|
|
||||||
if s3Error := checkRequestAuthType(ctx, r, lifecycle.GetBucketLifecycleAction, bucket, ""); s3Error != ErrNone {
|
if s3Error := checkRequestAuthType(ctx, r, policy.GetBucketLifecycleAction, bucket, ""); s3Error != ErrNone {
|
||||||
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL, guessIsBrowserReq(r))
|
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL, guessIsBrowserReq(r))
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
@ -140,7 +141,7 @@ func (api objectAPIHandlers) DeleteBucketLifecycleHandler(w http.ResponseWriter,
|
||||||
vars := mux.Vars(r)
|
vars := mux.Vars(r)
|
||||||
bucket := vars["bucket"]
|
bucket := vars["bucket"]
|
||||||
|
|
||||||
if s3Error := checkRequestAuthType(ctx, r, lifecycle.DeleteBucketLifecycleAction, bucket, ""); s3Error != ErrNone {
|
if s3Error := checkRequestAuthType(ctx, r, policy.PutBucketLifecycleAction, bucket, ""); s3Error != ErrNone {
|
||||||
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL, guessIsBrowserReq(r))
|
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(s3Error), r.URL, guessIsBrowserReq(r))
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,164 @@
|
||||||
|
/*
|
||||||
|
* MinIO Cloud Storage, (C) 2019 MinIO, Inc.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
* you may not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package cmd
|
||||||
|
|
||||||
|
import (
|
||||||
|
"context"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/minio/minio/cmd/logger"
|
||||||
|
"github.com/minio/minio/pkg/lifecycle"
|
||||||
|
)
|
||||||
|
|
||||||
|
const (
|
||||||
|
bgLifecycleInterval = 24 * time.Hour
|
||||||
|
bgLifecycleTick = time.Hour
|
||||||
|
)
|
||||||
|
|
||||||
|
type lifecycleOps struct {
|
||||||
|
LastActivity time.Time
|
||||||
|
}
|
||||||
|
|
||||||
|
// Register to the daily objects listing
|
||||||
|
var globalLifecycleOps = &lifecycleOps{}
|
||||||
|
|
||||||
|
func getLocalBgLifecycleOpsStatus() BgLifecycleOpsStatus {
|
||||||
|
return BgLifecycleOpsStatus{
|
||||||
|
LastActivity: globalLifecycleOps.LastActivity,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// initDailyLifecycle starts the routine that receives the daily
|
||||||
|
// listing of all objects and applies any matching bucket lifecycle
|
||||||
|
// rules.
|
||||||
|
func initDailyLifecycle() {
|
||||||
|
go startDailyLifecycle()
|
||||||
|
}
|
||||||
|
|
||||||
|
func startDailyLifecycle() {
|
||||||
|
var objAPI ObjectLayer
|
||||||
|
var ctx = context.Background()
|
||||||
|
|
||||||
|
// Wait until the object API is ready
|
||||||
|
for {
|
||||||
|
objAPI = newObjectLayerFn()
|
||||||
|
if objAPI == nil {
|
||||||
|
time.Sleep(time.Second)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
break
|
||||||
|
}
|
||||||
|
|
||||||
|
// Calculate the time of the last lifecycle operation in all peers node of the cluster
|
||||||
|
computeLastLifecycleActivity := func(status []BgOpsStatus) time.Time {
|
||||||
|
var lastAct time.Time
|
||||||
|
for _, st := range status {
|
||||||
|
if st.LifecycleOps.LastActivity.After(lastAct) {
|
||||||
|
lastAct = st.LifecycleOps.LastActivity
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return lastAct
|
||||||
|
}
|
||||||
|
|
||||||
|
for {
|
||||||
|
// Check if we should perform lifecycle ops based on the last lifecycle activity, sleep one hour otherwise
|
||||||
|
allLifecycleStatus := []BgOpsStatus{
|
||||||
|
{LifecycleOps: getLocalBgLifecycleOpsStatus()},
|
||||||
|
}
|
||||||
|
if globalIsDistXL {
|
||||||
|
allLifecycleStatus = append(allLifecycleStatus, globalNotificationSys.BackgroundOpsStatus()...)
|
||||||
|
}
|
||||||
|
lastAct := computeLastLifecycleActivity(allLifecycleStatus)
|
||||||
|
if !lastAct.IsZero() && time.Since(lastAct) < bgLifecycleInterval {
|
||||||
|
time.Sleep(bgLifecycleTick)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Perform one lifecycle operation
|
||||||
|
err := lifecycleRound(ctx, objAPI)
|
||||||
|
switch err.(type) {
|
||||||
|
// Unable to hold a lock means there is another
|
||||||
|
// instance doing the lifecycle round round
|
||||||
|
case OperationTimedOut:
|
||||||
|
time.Sleep(bgLifecycleTick)
|
||||||
|
default:
|
||||||
|
logger.LogIf(ctx, err)
|
||||||
|
time.Sleep(time.Minute)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func lifecycleRound(ctx context.Context, objAPI ObjectLayer) error {
|
||||||
|
|
||||||
|
zeroDuration := time.Millisecond
|
||||||
|
zeroDynamicTimeout := newDynamicTimeout(zeroDuration, zeroDuration)
|
||||||
|
|
||||||
|
// Lock to avoid concurrent lifecycle ops from other nodes
|
||||||
|
sweepLock := globalNSMutex.NewNSLock(ctx, "system", "daily-lifecycle-ops")
|
||||||
|
if err := sweepLock.GetLock(zeroDynamicTimeout); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
defer sweepLock.Unlock()
|
||||||
|
|
||||||
|
buckets, err := objAPI.ListBuckets(ctx)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, bucket := range buckets {
|
||||||
|
// Check if the current bucket has a configured lifecycle policy, skip otherwise
|
||||||
|
l, ok := globalLifecycleSys.Get(bucket.Name)
|
||||||
|
if !ok {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
// Calculate the common prefix of all lifecycle rules
|
||||||
|
var prefixes []string
|
||||||
|
for _, rule := range l.Rules {
|
||||||
|
prefixes = append(prefixes, rule.Filter.Prefix)
|
||||||
|
}
|
||||||
|
commonPrefix := lcp(prefixes)
|
||||||
|
|
||||||
|
// List all objects and calculate lifecycle action based on object name & object modtime
|
||||||
|
marker := ""
|
||||||
|
for {
|
||||||
|
res, err := objAPI.ListObjects(ctx, bucket.Name, commonPrefix, marker, "", 1000)
|
||||||
|
if err != nil {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
for _, obj := range res.Objects {
|
||||||
|
// Find the action that need to be executed
|
||||||
|
action := l.ComputeAction(obj.Name, obj.ModTime)
|
||||||
|
switch action {
|
||||||
|
case lifecycle.DeleteAction:
|
||||||
|
objAPI.DeleteObject(ctx, bucket.Name, obj.Name)
|
||||||
|
default:
|
||||||
|
// Nothing
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if !res.IsTruncated {
|
||||||
|
break
|
||||||
|
} else {
|
||||||
|
marker = res.NextMarker
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
|
@ -50,6 +50,15 @@ func (sys *LifecycleSys) Set(bucketName string, lifecycle lifecycle.Lifecycle) {
|
||||||
sys.bucketLifecycleMap[bucketName] = lifecycle
|
sys.bucketLifecycleMap[bucketName] = lifecycle
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Get - gets lifecycle config associated to a given bucket name.
|
||||||
|
func (sys *LifecycleSys) Get(bucketName string) (lifecycle lifecycle.Lifecycle, ok bool) {
|
||||||
|
sys.Lock()
|
||||||
|
defer sys.Unlock()
|
||||||
|
|
||||||
|
l, ok := sys.bucketLifecycleMap[bucketName]
|
||||||
|
return l, ok
|
||||||
|
}
|
||||||
|
|
||||||
func saveLifecycleConfig(ctx context.Context, objAPI ObjectLayer, bucketName string, bucketLifecycle *lifecycle.Lifecycle) error {
|
func saveLifecycleConfig(ctx context.Context, objAPI ObjectLayer, bucketName string, bucketLifecycle *lifecycle.Lifecycle) error {
|
||||||
data, err := xml.Marshal(bucketLifecycle)
|
data, err := xml.Marshal(bucketLifecycle)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
|
|
@ -263,6 +263,24 @@ func (sys *NotificationSys) BackgroundHealStatus() []madmin.BgHealState {
|
||||||
return states
|
return states
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// BackgroundOpsStatus - returns the status of all background operations of all peers
|
||||||
|
func (sys *NotificationSys) BackgroundOpsStatus() []BgOpsStatus {
|
||||||
|
states := make([]BgOpsStatus, len(sys.peerClients))
|
||||||
|
for idx, client := range sys.peerClients {
|
||||||
|
if client == nil {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
st, err := client.BackgroundOpsStatus()
|
||||||
|
if err != nil {
|
||||||
|
logger.LogIf(context.Background(), err)
|
||||||
|
} else {
|
||||||
|
states[idx] = st
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return states
|
||||||
|
}
|
||||||
|
|
||||||
// StartProfiling - start profiling on remote peers, by initiating a remote RPC.
|
// StartProfiling - start profiling on remote peers, by initiating a remote RPC.
|
||||||
func (sys *NotificationSys) StartProfiling(profiler string) []NotificationPeerErr {
|
func (sys *NotificationSys) StartProfiling(profiler string) []NotificationPeerErr {
|
||||||
ng := WithNPeers(len(sys.peerClients))
|
ng := WithNPeers(len(sys.peerClients))
|
||||||
|
|
|
@ -479,6 +479,32 @@ func (client *peerRESTClient) BackgroundHealStatus() (madmin.BgHealState, error)
|
||||||
return state, err
|
return state, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// BgLifecycleOpsStatus describes the status
|
||||||
|
// of the background lifecycle operations
|
||||||
|
type BgLifecycleOpsStatus struct {
|
||||||
|
LastActivity time.Time
|
||||||
|
}
|
||||||
|
|
||||||
|
// BgOpsStatus describes the status of all operations performed
|
||||||
|
// in background such as auto-healing and lifecycle.
|
||||||
|
// Notice: We need to increase peer REST API version when adding
|
||||||
|
// new fields to this struct.
|
||||||
|
type BgOpsStatus struct {
|
||||||
|
LifecycleOps BgLifecycleOpsStatus
|
||||||
|
}
|
||||||
|
|
||||||
|
func (client *peerRESTClient) BackgroundOpsStatus() (BgOpsStatus, error) {
|
||||||
|
respBody, err := client.call(peerRESTMethodBackgroundOpsStatus, nil, nil, -1)
|
||||||
|
if err != nil {
|
||||||
|
return BgOpsStatus{}, err
|
||||||
|
}
|
||||||
|
defer http.DrainBody(respBody)
|
||||||
|
|
||||||
|
state := BgOpsStatus{}
|
||||||
|
err = gob.NewDecoder(respBody).Decode(&state)
|
||||||
|
return state, err
|
||||||
|
}
|
||||||
|
|
||||||
func (client *peerRESTClient) doTrace(traceCh chan interface{}, doneCh chan struct{}, trcAll, trcErr bool) {
|
func (client *peerRESTClient) doTrace(traceCh chan interface{}, doneCh chan struct{}, trcAll, trcErr bool) {
|
||||||
values := make(url.Values)
|
values := make(url.Values)
|
||||||
values.Set(peerRESTTraceAll, strconv.FormatBool(trcAll))
|
values.Set(peerRESTTraceAll, strconv.FormatBool(trcAll))
|
||||||
|
|
|
@ -16,7 +16,7 @@
|
||||||
|
|
||||||
package cmd
|
package cmd
|
||||||
|
|
||||||
const peerRESTVersion = "v2"
|
const peerRESTVersion = "v3"
|
||||||
const peerRESTPath = minioReservedBucketPath + "/peer/" + peerRESTVersion
|
const peerRESTPath = minioReservedBucketPath + "/peer/" + peerRESTVersion
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
@ -27,6 +27,7 @@ const (
|
||||||
peerRESTMethodDeleteBucket = "deletebucket"
|
peerRESTMethodDeleteBucket = "deletebucket"
|
||||||
peerRESTMethodSignalService = "signalservice"
|
peerRESTMethodSignalService = "signalservice"
|
||||||
peerRESTMethodBackgroundHealStatus = "backgroundhealstatus"
|
peerRESTMethodBackgroundHealStatus = "backgroundhealstatus"
|
||||||
|
peerRESTMethodBackgroundOpsStatus = "backgroundopsstatus"
|
||||||
peerRESTMethodGetLocks = "getlocks"
|
peerRESTMethodGetLocks = "getlocks"
|
||||||
peerRESTMethodBucketPolicyRemove = "removebucketpolicy"
|
peerRESTMethodBucketPolicyRemove = "removebucketpolicy"
|
||||||
peerRESTMethodLoadUser = "loaduser"
|
peerRESTMethodLoadUser = "loaduser"
|
||||||
|
|
|
@ -784,6 +784,22 @@ func (s *peerRESTServer) BackgroundHealStatusHandler(w http.ResponseWriter, r *h
|
||||||
logger.LogIf(ctx, gob.NewEncoder(w).Encode(state))
|
logger.LogIf(ctx, gob.NewEncoder(w).Encode(state))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (s *peerRESTServer) BackgroundOpsStatusHandler(w http.ResponseWriter, r *http.Request) {
|
||||||
|
if !s.IsValid(w, r) {
|
||||||
|
s.writeErrorResponse(w, errors.New("invalid request"))
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
ctx := newContext(r, w, "BackgroundOpsStatus")
|
||||||
|
|
||||||
|
state := BgOpsStatus{
|
||||||
|
LifecycleOps: getLocalBgLifecycleOpsStatus(),
|
||||||
|
}
|
||||||
|
|
||||||
|
defer w.(http.Flusher).Flush()
|
||||||
|
logger.LogIf(ctx, gob.NewEncoder(w).Encode(state))
|
||||||
|
}
|
||||||
|
|
||||||
func (s *peerRESTServer) writeErrorResponse(w http.ResponseWriter, err error) {
|
func (s *peerRESTServer) writeErrorResponse(w http.ResponseWriter, err error) {
|
||||||
w.WriteHeader(http.StatusForbidden)
|
w.WriteHeader(http.StatusForbidden)
|
||||||
w.Write([]byte(err.Error()))
|
w.Write([]byte(err.Error()))
|
||||||
|
@ -831,6 +847,7 @@ func registerPeerRESTHandlers(router *mux.Router) {
|
||||||
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodReloadFormat).HandlerFunc(httpTraceHdrs(server.ReloadFormatHandler)).Queries(restQueries(peerRESTDryRun)...)
|
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodReloadFormat).HandlerFunc(httpTraceHdrs(server.ReloadFormatHandler)).Queries(restQueries(peerRESTDryRun)...)
|
||||||
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodBucketLifecycleSet).HandlerFunc(httpTraceHdrs(server.SetBucketLifecycleHandler)).Queries(restQueries(peerRESTBucket)...)
|
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodBucketLifecycleSet).HandlerFunc(httpTraceHdrs(server.SetBucketLifecycleHandler)).Queries(restQueries(peerRESTBucket)...)
|
||||||
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodBucketLifecycleRemove).HandlerFunc(httpTraceHdrs(server.RemoveBucketLifecycleHandler)).Queries(restQueries(peerRESTBucket)...)
|
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodBucketLifecycleRemove).HandlerFunc(httpTraceHdrs(server.RemoveBucketLifecycleHandler)).Queries(restQueries(peerRESTBucket)...)
|
||||||
|
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodBackgroundOpsStatus).HandlerFunc(server.BackgroundOpsStatusHandler)
|
||||||
|
|
||||||
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodTrace).HandlerFunc(server.TraceHandler)
|
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodTrace).HandlerFunc(server.TraceHandler)
|
||||||
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodBackgroundHealStatus).HandlerFunc(server.BackgroundHealStatusHandler)
|
subrouter.Methods(http.MethodPost).Path(SlashSeparator + peerRESTMethodBackgroundHealStatus).HandlerFunc(server.BackgroundHealStatusHandler)
|
||||||
|
|
|
@ -385,6 +385,8 @@ func serverMain(ctx *cli.Context) {
|
||||||
// - compression
|
// - compression
|
||||||
verifyObjectLayerFeatures("server", newObject)
|
verifyObjectLayerFeatures("server", newObject)
|
||||||
|
|
||||||
|
initDailyLifecycle()
|
||||||
|
|
||||||
if globalIsXL {
|
if globalIsXL {
|
||||||
initBackgroundHealing()
|
initBackgroundHealing()
|
||||||
initDailyHeal()
|
initDailyHeal()
|
||||||
|
|
33
cmd/utils.go
33
cmd/utils.go
|
@ -475,3 +475,36 @@ func reverseStringSlice(input []string) {
|
||||||
input[left], input[right] = input[right], input[left]
|
input[left], input[right] = input[right], input[left]
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// lcp finds the longest common prefix of the input strings.
|
||||||
|
// It compares by bytes instead of runes (Unicode code points).
|
||||||
|
// It's up to the caller to do Unicode normalization if desired
|
||||||
|
// (e.g. see golang.org/x/text/unicode/norm).
|
||||||
|
func lcp(l []string) string {
|
||||||
|
// Special cases first
|
||||||
|
switch len(l) {
|
||||||
|
case 0:
|
||||||
|
return ""
|
||||||
|
case 1:
|
||||||
|
return l[0]
|
||||||
|
}
|
||||||
|
// LCP of min and max (lexigraphically)
|
||||||
|
// is the LCP of the whole set.
|
||||||
|
min, max := l[0], l[0]
|
||||||
|
for _, s := range l[1:] {
|
||||||
|
switch {
|
||||||
|
case s < min:
|
||||||
|
min = s
|
||||||
|
case s > max:
|
||||||
|
max = s
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for i := 0; i < len(min) && i < len(max); i++ {
|
||||||
|
if min[i] != max[i] {
|
||||||
|
return min[:i]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// In the case where lengths are not equal but all bytes
|
||||||
|
// are equal, min is the answer ("foo" < "foobar").
|
||||||
|
return min
|
||||||
|
}
|
||||||
|
|
|
@ -479,3 +479,26 @@ func TestQueries(t *testing.T) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestLCP(t *testing.T) {
|
||||||
|
var testCases = []struct {
|
||||||
|
prefixes []string
|
||||||
|
commonPrefix string
|
||||||
|
}{
|
||||||
|
{[]string{"", ""}, ""},
|
||||||
|
{[]string{"a", "b"}, ""},
|
||||||
|
{[]string{"a", "a"}, "a"},
|
||||||
|
{[]string{"a/", "a/"}, "a/"},
|
||||||
|
{[]string{"abcd/", ""}, ""},
|
||||||
|
{[]string{"abcd/foo/", "abcd/bar/"}, "abcd/"},
|
||||||
|
{[]string{"abcd/foo/bar/", "abcd/foo/bar/zoo"}, "abcd/foo/bar/"},
|
||||||
|
}
|
||||||
|
|
||||||
|
for i, test := range testCases {
|
||||||
|
foundPrefix := lcp(test.prefixes)
|
||||||
|
if foundPrefix != test.commonPrefix {
|
||||||
|
t.Fatalf("Test %d: Common prefix found: `%v`, expected: `%v`", i+1, foundPrefix, test.commonPrefix)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
|
@ -0,0 +1,53 @@
|
||||||
|
# Object Lifecycle Configuration Quickstart Guide [![Slack](https://slack.min.io/slack?type=svg)](https://slack.min.io) [![Go Report Card](https://goreportcard.com/badge/minio/minio)](https://goreportcard.com/report/minio/minio) [![Docker Pulls](https://img.shields.io/docker/pulls/minio/minio.svg?maxAge=604800)](https://hub.docker.com/r/minio/minio/)
|
||||||
|
|
||||||
|
Enable object lifecycle configuration on buckets to setup automatic deletion of objects after a specified number of days or a specified date.
|
||||||
|
|
||||||
|
## 1. Prerequisites
|
||||||
|
- Install MinIO - [MinIO Quickstart Guide](https://docs.min.io/docs/minio-quickstart-guide).
|
||||||
|
- Install AWS Cli - [Installing AWS Command Line Interface](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-install.html)
|
||||||
|
|
||||||
|
|
||||||
|
## 2. Enable bucket lifecycle configuration
|
||||||
|
|
||||||
|
1. Create a bucket lifecycle configuration which expires the objects under the prefix `uploads/2015` on `2020-01-01T00:00:00.000Z` date and the objects under `temporary-uploads/` after 7 days. Generate it as shown below:
|
||||||
|
|
||||||
|
```sh
|
||||||
|
$ cat >bucket-lifecycle.json << EOF
|
||||||
|
{
|
||||||
|
"Rules": [
|
||||||
|
{
|
||||||
|
"Expiration": {
|
||||||
|
"Date": "2020-01-01T00:00:00.000Z"
|
||||||
|
},
|
||||||
|
"ID": "Delete very old messenger pictures",
|
||||||
|
"Filter": {
|
||||||
|
"Prefix": "uploads/2015/"
|
||||||
|
},
|
||||||
|
"Status": "Enabled"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"Expiration": {
|
||||||
|
"Days": 7
|
||||||
|
},
|
||||||
|
"ID": "Delete temporary uploads",
|
||||||
|
"Filter": {
|
||||||
|
"Prefix": "temporary-uploads/"
|
||||||
|
},
|
||||||
|
"Status": "Enabled"
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
EOF
|
||||||
|
```
|
||||||
|
|
||||||
|
2. Enable bucket lifecycle configuration using `aws-cli`:
|
||||||
|
|
||||||
|
```sh
|
||||||
|
$ export AWS_ACCESS_KEY_ID="your-access-key"
|
||||||
|
$ export AWS_SECRET_ACCESS_KEY="your-secret-key"
|
||||||
|
$ aws s3api put-bucket-lifecycle-configuration --bucket your-bucket --endpoint-url http://minio-server-address:port --lifecycle-configuration file://bucket-lifecycle.json
|
||||||
|
```
|
||||||
|
|
||||||
|
## Explore Further
|
||||||
|
- [MinIO | Golang Client API Reference](https://docs.min.io/docs/golang-client-api-reference.html#SetBucketLifecycle)
|
||||||
|
- [Object Lifecycle Management](https://docs.aws.amazon.com/AmazonS3/latest/dev/object-lifecycle-mgmt.html)
|
|
@ -76,6 +76,12 @@ const (
|
||||||
// ListMultipartUploadPartsAction - ListParts Rest API action.
|
// ListMultipartUploadPartsAction - ListParts Rest API action.
|
||||||
ListMultipartUploadPartsAction = "s3:ListMultipartUploadParts"
|
ListMultipartUploadPartsAction = "s3:ListMultipartUploadParts"
|
||||||
|
|
||||||
|
// PutBucketLifecycleAction - PutBucketLifecycle Rest API action.
|
||||||
|
PutBucketLifecycleAction = "s3:PutBucketLifecycle"
|
||||||
|
|
||||||
|
// GetBucketLifecycleAction - GetBucketLifecycle Rest API action.
|
||||||
|
GetBucketLifecycleAction = "s3:GetBucketLifecycle"
|
||||||
|
|
||||||
// PutBucketNotificationAction - PutObjectNotification Rest API action.
|
// PutBucketNotificationAction - PutObjectNotification Rest API action.
|
||||||
PutBucketNotificationAction = "s3:PutBucketNotification"
|
PutBucketNotificationAction = "s3:PutBucketNotification"
|
||||||
|
|
||||||
|
@ -110,6 +116,8 @@ var supportedActions = map[Action]struct{}{
|
||||||
PutBucketNotificationAction: {},
|
PutBucketNotificationAction: {},
|
||||||
PutBucketPolicyAction: {},
|
PutBucketPolicyAction: {},
|
||||||
PutObjectAction: {},
|
PutObjectAction: {},
|
||||||
|
GetBucketLifecycleAction: {},
|
||||||
|
PutBucketLifecycleAction: {},
|
||||||
}
|
}
|
||||||
|
|
||||||
// isObjectAction - returns whether action is object type or not.
|
// isObjectAction - returns whether action is object type or not.
|
||||||
|
|
|
@ -1,33 +0,0 @@
|
||||||
/*
|
|
||||||
* MinIO Cloud Storage, (C) 2019 MinIO, Inc.
|
|
||||||
*
|
|
||||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
|
||||||
* you may not use this file except in compliance with the License.
|
|
||||||
* You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package lifecycle
|
|
||||||
|
|
||||||
// Action - policy action.
|
|
||||||
// Refer https://docs.aws.amazon.com/IAM/latest/UserGuide/list_amazons3.html
|
|
||||||
// for more information about available actions.
|
|
||||||
type Action string
|
|
||||||
|
|
||||||
const (
|
|
||||||
// PutBucketLifecycleAction - PutBucketLifecycle Rest API action.
|
|
||||||
PutBucketLifecycleAction = "s3:PutBucketLifecycle"
|
|
||||||
|
|
||||||
// GetBucketLifecycleAction - GetBucketLifecycle Rest API action.
|
|
||||||
GetBucketLifecycleAction = "s3:GetBucketLifecycle"
|
|
||||||
|
|
||||||
// DeleteBucketLifecycleAction - DeleteBucketLifecycleAction Rest API action.
|
|
||||||
DeleteBucketLifecycleAction = "s3:DeleteBucketLifecycle"
|
|
||||||
)
|
|
|
@ -107,13 +107,29 @@ type Expiration struct {
|
||||||
// Validate - validates the "Expiration" element
|
// Validate - validates the "Expiration" element
|
||||||
func (e Expiration) Validate() error {
|
func (e Expiration) Validate() error {
|
||||||
// Neither expiration days or date is specified
|
// Neither expiration days or date is specified
|
||||||
if e.Days == ExpirationDays(0) && e.Date == (ExpirationDate{time.Time{}}) {
|
if e.IsDaysNull() && e.IsDateNull() {
|
||||||
return errLifecycleInvalidExpiration
|
return errLifecycleInvalidExpiration
|
||||||
}
|
}
|
||||||
|
|
||||||
// Both expiration days and date are specified
|
// Both expiration days and date are specified
|
||||||
if e.Days != ExpirationDays(0) && e.Date != (ExpirationDate{time.Time{}}) {
|
if !e.IsDaysNull() && !e.IsDateNull() {
|
||||||
return errLifecycleInvalidExpiration
|
return errLifecycleInvalidExpiration
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// IsDaysNull returns true if days field is null
|
||||||
|
func (e Expiration) IsDaysNull() bool {
|
||||||
|
return e.Days == ExpirationDays(0)
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
// IsDateNull returns true if date field is null
|
||||||
|
func (e Expiration) IsDateNull() bool {
|
||||||
|
return e.Date == ExpirationDate{time.Time{}}
|
||||||
|
}
|
||||||
|
|
||||||
|
// IsNull returns true if both date and days fields are null
|
||||||
|
func (e Expiration) IsNull() bool {
|
||||||
|
return e.IsDaysNull() && e.IsDateNull()
|
||||||
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ import (
|
||||||
"errors"
|
"errors"
|
||||||
"io"
|
"io"
|
||||||
"strings"
|
"strings"
|
||||||
|
"time"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
|
@ -29,6 +30,17 @@ var (
|
||||||
errLifecycleOverlappingPrefix = errors.New("Lifecycle configuration has rules with overlapping prefix")
|
errLifecycleOverlappingPrefix = errors.New("Lifecycle configuration has rules with overlapping prefix")
|
||||||
)
|
)
|
||||||
|
|
||||||
|
// Action represents a delete action or other transition
|
||||||
|
// actions that will be implemented later.
|
||||||
|
type Action int
|
||||||
|
|
||||||
|
const (
|
||||||
|
// NoneAction means no action required after evaluting lifecycle rules
|
||||||
|
NoneAction Action = iota
|
||||||
|
// DeleteAction means the object needs to be removed after evaluting lifecycle rules
|
||||||
|
DeleteAction
|
||||||
|
)
|
||||||
|
|
||||||
// Lifecycle - Configuration for bucket lifecycle.
|
// Lifecycle - Configuration for bucket lifecycle.
|
||||||
type Lifecycle struct {
|
type Lifecycle struct {
|
||||||
XMLName xml.Name `xml:"LifecycleConfiguration"`
|
XMLName xml.Name `xml:"LifecycleConfiguration"`
|
||||||
|
@ -84,3 +96,35 @@ func (lc Lifecycle) Validate() error {
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// FilterRuleActions returns the expiration and transition from the object name
|
||||||
|
// after evaluating all rules.
|
||||||
|
func (lc Lifecycle) FilterRuleActions(objName string) (Expiration, Transition) {
|
||||||
|
for _, rule := range lc.Rules {
|
||||||
|
if strings.ToLower(rule.Status) != "enabled" {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
if strings.HasPrefix(objName, rule.Filter.Prefix) {
|
||||||
|
return rule.Expiration, Transition{}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return Expiration{}, Transition{}
|
||||||
|
}
|
||||||
|
|
||||||
|
// ComputeAction returns the action to perform by evaluating all lifecycle rules
|
||||||
|
// against the object name and its modification time.
|
||||||
|
func (lc Lifecycle) ComputeAction(objName string, modTime time.Time) Action {
|
||||||
|
var action = NoneAction
|
||||||
|
exp, _ := lc.FilterRuleActions(objName)
|
||||||
|
if !exp.IsDateNull() {
|
||||||
|
if time.Now().After(exp.Date.Time) {
|
||||||
|
action = DeleteAction
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if !exp.IsDaysNull() {
|
||||||
|
if time.Now().After(modTime.Add(time.Duration(exp.Days) * 24 * time.Hour)) {
|
||||||
|
action = DeleteAction
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return action
|
||||||
|
}
|
||||||
|
|
|
@ -158,3 +158,73 @@ func TestMarshalLifecycleConfig(t *testing.T) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestComputeActions(t *testing.T) {
|
||||||
|
testCases := []struct {
|
||||||
|
inputConfig string
|
||||||
|
objectName string
|
||||||
|
objectModTime time.Time
|
||||||
|
expectedAction Action
|
||||||
|
}{
|
||||||
|
// Empty object name (unexpected case) should always return NoneAction
|
||||||
|
{
|
||||||
|
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>prefix</Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||||
|
expectedAction: NoneAction,
|
||||||
|
},
|
||||||
|
// Disabled should always return NoneAction
|
||||||
|
{
|
||||||
|
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Disabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||||
|
objectName: "foodir/fooobject",
|
||||||
|
objectModTime: time.Now().UTC().Add(-10 * 24 * time.Hour), // Created 10 days ago
|
||||||
|
expectedAction: NoneAction,
|
||||||
|
},
|
||||||
|
// Prefix not matched
|
||||||
|
{
|
||||||
|
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||||
|
objectName: "foxdir/fooobject",
|
||||||
|
objectModTime: time.Now().UTC().Add(-10 * 24 * time.Hour), // Created 10 days ago
|
||||||
|
expectedAction: NoneAction,
|
||||||
|
},
|
||||||
|
// Too early to remove (test Days)
|
||||||
|
{
|
||||||
|
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||||
|
objectName: "foxdir/fooobject",
|
||||||
|
objectModTime: time.Now().UTC().Add(-10 * 24 * time.Hour), // Created 10 days ago
|
||||||
|
expectedAction: NoneAction,
|
||||||
|
},
|
||||||
|
// Should remove (test Days)
|
||||||
|
{
|
||||||
|
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||||
|
objectName: "foodir/fooobject",
|
||||||
|
objectModTime: time.Now().UTC().Add(-6 * 24 * time.Hour), // Created 6 days ago
|
||||||
|
expectedAction: DeleteAction,
|
||||||
|
},
|
||||||
|
// Too early to remove (test Date)
|
||||||
|
{
|
||||||
|
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Date>` + time.Now().Truncate(24*time.Hour).UTC().Add(24*time.Hour).Format(time.RFC3339) + `</Date></Expiration></Rule></LifecycleConfiguration>`,
|
||||||
|
objectName: "foodir/fooobject",
|
||||||
|
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||||
|
expectedAction: NoneAction,
|
||||||
|
},
|
||||||
|
// Should remove (test Days)
|
||||||
|
{
|
||||||
|
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Date>` + time.Now().Truncate(24*time.Hour).UTC().Add(-24*time.Hour).Format(time.RFC3339) + `</Date></Expiration></Rule></LifecycleConfiguration>`,
|
||||||
|
objectName: "foodir/fooobject",
|
||||||
|
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||||
|
expectedAction: DeleteAction,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
for i, tc := range testCases {
|
||||||
|
t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) {
|
||||||
|
lc, err := ParseLifecycleConfig(bytes.NewReader([]byte(tc.inputConfig)))
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("%d: Got unexpected error: %v", i+1, err)
|
||||||
|
}
|
||||||
|
if resultAction := lc.ComputeAction(tc.objectName, tc.objectModTime); resultAction != tc.expectedAction {
|
||||||
|
t.Fatalf("%d: Expected action: `%v`, got: `%v`", i+1, tc.expectedAction, resultAction)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -83,6 +83,12 @@ const (
|
||||||
|
|
||||||
// PutObjectAction - PutObject Rest API action.
|
// PutObjectAction - PutObject Rest API action.
|
||||||
PutObjectAction = "s3:PutObject"
|
PutObjectAction = "s3:PutObject"
|
||||||
|
|
||||||
|
// PutBucketLifecycleAction - PutBucketLifecycle Rest API action.
|
||||||
|
PutBucketLifecycleAction = "s3:PutBucketLifecycle"
|
||||||
|
|
||||||
|
// GetBucketLifecycleAction - GetBucketLifecycle Rest API action.
|
||||||
|
GetBucketLifecycleAction = "s3:GetBucketLifecycle"
|
||||||
)
|
)
|
||||||
|
|
||||||
// isObjectAction - returns whether action is object type or not.
|
// isObjectAction - returns whether action is object type or not.
|
||||||
|
@ -113,6 +119,8 @@ func (action Action) IsValid() bool {
|
||||||
case ListMultipartUploadPartsAction, PutBucketNotificationAction:
|
case ListMultipartUploadPartsAction, PutBucketNotificationAction:
|
||||||
fallthrough
|
fallthrough
|
||||||
case PutBucketPolicyAction, PutObjectAction:
|
case PutBucketPolicyAction, PutObjectAction:
|
||||||
|
fallthrough
|
||||||
|
case PutBucketLifecycleAction, GetBucketLifecycleAction:
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue