mirror of
https://github.com/minio/minio.git
synced 2025-11-07 21:02:58 -05:00
rename all remaining packages to internal/ (#12418)
This is to ensure that there are no projects that try to import `minio/minio/pkg` into their own repo. Any such common packages should go to `https://github.com/minio/pkg`
This commit is contained in:
88
internal/bucket/bandwidth/measurement.go
Normal file
88
internal/bucket/bandwidth/measurement.go
Normal file
@@ -0,0 +1,88 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package bandwidth
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
// betaBucket is the weight used to calculate exponential moving average
|
||||
betaBucket = 0.1 // Number of averages considered = 1/(1-betaObject)
|
||||
)
|
||||
|
||||
// bucketMeasurement captures the bandwidth details for one bucket
|
||||
type bucketMeasurement struct {
|
||||
lock sync.Mutex
|
||||
bytesSinceLastWindow uint64 // Total bytes since last window was processed
|
||||
startTime time.Time // Start time for window
|
||||
expMovingAvg float64 // Previously calculate sliding window
|
||||
}
|
||||
|
||||
// newBucketMeasurement creates a new instance of the measurement with the initial start time.
|
||||
func newBucketMeasurement(initTime time.Time) *bucketMeasurement {
|
||||
return &bucketMeasurement{
|
||||
startTime: initTime,
|
||||
}
|
||||
}
|
||||
|
||||
// incrementBytes add bytes reported for a bucket.
|
||||
func (m *bucketMeasurement) incrementBytes(bytes uint64) {
|
||||
atomic.AddUint64(&m.bytesSinceLastWindow, bytes)
|
||||
}
|
||||
|
||||
// updateExponentialMovingAverage processes the measurements captured so far.
|
||||
func (m *bucketMeasurement) updateExponentialMovingAverage(endTime time.Time) {
|
||||
// Calculate aggregate avg bandwidth and exp window avg
|
||||
m.lock.Lock()
|
||||
defer func() {
|
||||
m.startTime = endTime
|
||||
m.lock.Unlock()
|
||||
}()
|
||||
|
||||
if endTime.Before(m.startTime) {
|
||||
return
|
||||
}
|
||||
|
||||
duration := endTime.Sub(m.startTime)
|
||||
|
||||
bytesSinceLastWindow := atomic.SwapUint64(&m.bytesSinceLastWindow, 0)
|
||||
|
||||
if m.expMovingAvg == 0 {
|
||||
// Should address initial calculation and should be fine for resuming from 0
|
||||
m.expMovingAvg = float64(bytesSinceLastWindow) / duration.Seconds()
|
||||
return
|
||||
}
|
||||
|
||||
increment := float64(bytesSinceLastWindow) / duration.Seconds()
|
||||
m.expMovingAvg = exponentialMovingAverage(betaBucket, m.expMovingAvg, increment)
|
||||
}
|
||||
|
||||
// exponentialMovingAverage calculates the exponential moving average
|
||||
func exponentialMovingAverage(beta, previousAvg, incrementAvg float64) float64 {
|
||||
return (1-beta)*incrementAvg + beta*previousAvg
|
||||
}
|
||||
|
||||
// getExpMovingAvgBytesPerSecond returns the exponential moving average for the bucket in bytes
|
||||
func (m *bucketMeasurement) getExpMovingAvgBytesPerSecond() float64 {
|
||||
m.lock.Lock()
|
||||
defer m.lock.Unlock()
|
||||
return m.expMovingAvg
|
||||
}
|
||||
155
internal/bucket/bandwidth/monitor.go
Normal file
155
internal/bucket/bandwidth/monitor.go
Normal file
@@ -0,0 +1,155 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package bandwidth
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/minio/madmin-go"
|
||||
)
|
||||
|
||||
// throttleBandwidth gets the throttle for bucket with the configured value
|
||||
func (m *Monitor) throttleBandwidth(ctx context.Context, bucket string, bandwidthBytesPerSecond int64, clusterBandwidth int64) *throttle {
|
||||
m.lock.Lock()
|
||||
defer m.lock.Unlock()
|
||||
throttle, ok := m.bucketThrottle[bucket]
|
||||
if !ok {
|
||||
throttle = newThrottle(ctx, bandwidthBytesPerSecond, clusterBandwidth)
|
||||
m.bucketThrottle[bucket] = throttle
|
||||
return throttle
|
||||
}
|
||||
throttle.SetBandwidth(bandwidthBytesPerSecond, clusterBandwidth)
|
||||
return throttle
|
||||
}
|
||||
|
||||
// Monitor implements the monitoring for bandwidth measurements.
|
||||
type Monitor struct {
|
||||
lock sync.Mutex // lock for all updates
|
||||
|
||||
activeBuckets map[string]*bucketMeasurement // Buckets with objects in flight
|
||||
|
||||
bucketMovingAvgTicker *time.Ticker // Ticker for calculating moving averages
|
||||
|
||||
bucketThrottle map[string]*throttle
|
||||
|
||||
doneCh <-chan struct{}
|
||||
}
|
||||
|
||||
// NewMonitor returns a monitor with defaults.
|
||||
func NewMonitor(doneCh <-chan struct{}) *Monitor {
|
||||
m := &Monitor{
|
||||
activeBuckets: make(map[string]*bucketMeasurement),
|
||||
bucketMovingAvgTicker: time.NewTicker(2 * time.Second),
|
||||
bucketThrottle: make(map[string]*throttle),
|
||||
doneCh: doneCh,
|
||||
}
|
||||
go m.trackEWMA()
|
||||
return m
|
||||
}
|
||||
|
||||
// SelectionFunction for buckets
|
||||
type SelectionFunction func(bucket string) bool
|
||||
|
||||
// SelectBuckets will select all the buckets passed in.
|
||||
func SelectBuckets(buckets ...string) SelectionFunction {
|
||||
if len(buckets) == 0 {
|
||||
return func(bucket string) bool {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return func(bucket string) bool {
|
||||
for _, b := range buckets {
|
||||
if b == "" || b == bucket {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// GetReport gets the report for all bucket bandwidth details.
|
||||
func (m *Monitor) GetReport(selectBucket SelectionFunction) *madmin.BucketBandwidthReport {
|
||||
m.lock.Lock()
|
||||
defer m.lock.Unlock()
|
||||
return m.getReport(selectBucket)
|
||||
}
|
||||
|
||||
func (m *Monitor) getReport(selectBucket SelectionFunction) *madmin.BucketBandwidthReport {
|
||||
report := &madmin.BucketBandwidthReport{
|
||||
BucketStats: make(map[string]madmin.BandwidthDetails),
|
||||
}
|
||||
for bucket, bucketMeasurement := range m.activeBuckets {
|
||||
if !selectBucket(bucket) {
|
||||
continue
|
||||
}
|
||||
bucketThrottle, ok := m.bucketThrottle[bucket]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
report.BucketStats[bucket] = madmin.BandwidthDetails{
|
||||
LimitInBytesPerSecond: bucketThrottle.clusterBandwidth,
|
||||
CurrentBandwidthInBytesPerSecond: bucketMeasurement.getExpMovingAvgBytesPerSecond(),
|
||||
}
|
||||
}
|
||||
return report
|
||||
}
|
||||
|
||||
func (m *Monitor) trackEWMA() {
|
||||
for {
|
||||
select {
|
||||
case <-m.bucketMovingAvgTicker.C:
|
||||
m.updateMovingAvg()
|
||||
case <-m.doneCh:
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (m *Monitor) getBucketMeasurement(bucket string, initTime time.Time) *bucketMeasurement {
|
||||
bucketTracker, ok := m.activeBuckets[bucket]
|
||||
if !ok {
|
||||
bucketTracker = newBucketMeasurement(initTime)
|
||||
m.activeBuckets[bucket] = bucketTracker
|
||||
}
|
||||
return bucketTracker
|
||||
}
|
||||
|
||||
func (m *Monitor) updateMovingAvg() {
|
||||
m.lock.Lock()
|
||||
defer m.lock.Unlock()
|
||||
for _, bucketMeasurement := range m.activeBuckets {
|
||||
bucketMeasurement.updateExponentialMovingAverage(time.Now())
|
||||
}
|
||||
}
|
||||
|
||||
// track returns the measurement object for bucket and object
|
||||
func (m *Monitor) track(bucket string, object string) *bucketMeasurement {
|
||||
m.lock.Lock()
|
||||
defer m.lock.Unlock()
|
||||
return m.getBucketMeasurement(bucket, time.Now())
|
||||
}
|
||||
|
||||
// DeleteBucket deletes monitoring the 'bucket'
|
||||
func (m *Monitor) DeleteBucket(bucket string) {
|
||||
m.lock.Lock()
|
||||
defer m.lock.Unlock()
|
||||
delete(m.activeBuckets, bucket)
|
||||
delete(m.bucketThrottle, bucket)
|
||||
}
|
||||
159
internal/bucket/bandwidth/monitor_test.go
Normal file
159
internal/bucket/bandwidth/monitor_test.go
Normal file
@@ -0,0 +1,159 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package bandwidth
|
||||
|
||||
import (
|
||||
"context"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/minio/madmin-go"
|
||||
)
|
||||
|
||||
const (
|
||||
oneMiB uint64 = 1024 * 1024
|
||||
)
|
||||
|
||||
func TestMonitor_GetThrottle(t *testing.T) {
|
||||
type fields struct {
|
||||
bucketThrottles map[string]*throttle
|
||||
bucket string
|
||||
bpi int64
|
||||
}
|
||||
t1 := newThrottle(context.Background(), 100, 1024*1024)
|
||||
t2 := newThrottle(context.Background(), 200, 1024*1024)
|
||||
tests := []struct {
|
||||
name string
|
||||
fields fields
|
||||
want *throttle
|
||||
}{
|
||||
{
|
||||
name: "Existing",
|
||||
fields: fields{
|
||||
bucketThrottles: map[string]*throttle{"bucket": t1},
|
||||
bucket: "bucket",
|
||||
bpi: 100,
|
||||
},
|
||||
want: t1,
|
||||
},
|
||||
{
|
||||
name: "new",
|
||||
fields: fields{
|
||||
bucketThrottles: map[string]*throttle{"bucket": t1},
|
||||
bucket: "bucket2",
|
||||
bpi: 200,
|
||||
},
|
||||
want: t2,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
tt := tt
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
m := &Monitor{
|
||||
bucketThrottle: tt.fields.bucketThrottles,
|
||||
}
|
||||
if got := m.throttleBandwidth(context.Background(), tt.fields.bucket, tt.fields.bpi, 1024*1024); got.bytesPerInterval != tt.want.bytesPerInterval {
|
||||
t.Errorf("throttleBandwidth() = %v, want %v", got, tt.want)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestMonitor_GetReport(t *testing.T) {
|
||||
type fields struct {
|
||||
activeBuckets map[string]*bucketMeasurement
|
||||
endTime time.Time
|
||||
update2 uint64
|
||||
endTime2 time.Time
|
||||
}
|
||||
start := time.Now()
|
||||
m0 := newBucketMeasurement(start)
|
||||
m0.incrementBytes(0)
|
||||
m1MiBPS := newBucketMeasurement(start)
|
||||
m1MiBPS.incrementBytes(oneMiB)
|
||||
tests := []struct {
|
||||
name string
|
||||
fields fields
|
||||
want *madmin.BucketBandwidthReport
|
||||
want2 *madmin.BucketBandwidthReport
|
||||
}{
|
||||
{
|
||||
name: "ZeroToOne",
|
||||
fields: fields{
|
||||
activeBuckets: map[string]*bucketMeasurement{
|
||||
"bucket": m0,
|
||||
},
|
||||
endTime: start.Add(1 * time.Second),
|
||||
update2: oneMiB,
|
||||
endTime2: start.Add(2 * time.Second),
|
||||
},
|
||||
want: &madmin.BucketBandwidthReport{
|
||||
BucketStats: map[string]madmin.BandwidthDetails{"bucket": {LimitInBytesPerSecond: 1024 * 1024, CurrentBandwidthInBytesPerSecond: 0}},
|
||||
},
|
||||
want2: &madmin.BucketBandwidthReport{
|
||||
BucketStats: map[string]madmin.BandwidthDetails{"bucket": {LimitInBytesPerSecond: 1024 * 1024, CurrentBandwidthInBytesPerSecond: (1024 * 1024) / start.Add(2*time.Second).Sub(start.Add(1*time.Second)).Seconds()}},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "OneToTwo",
|
||||
fields: fields{
|
||||
activeBuckets: map[string]*bucketMeasurement{
|
||||
"bucket": m1MiBPS,
|
||||
},
|
||||
endTime: start.Add(1 * time.Second),
|
||||
update2: 2 * oneMiB,
|
||||
endTime2: start.Add(2 * time.Second),
|
||||
},
|
||||
want: &madmin.BucketBandwidthReport{
|
||||
BucketStats: map[string]madmin.BandwidthDetails{"bucket": {LimitInBytesPerSecond: 1024 * 1024, CurrentBandwidthInBytesPerSecond: float64(oneMiB)}},
|
||||
},
|
||||
want2: &madmin.BucketBandwidthReport{
|
||||
BucketStats: map[string]madmin.BandwidthDetails{"bucket": {
|
||||
LimitInBytesPerSecond: 1024 * 1024,
|
||||
CurrentBandwidthInBytesPerSecond: exponentialMovingAverage(betaBucket, float64(oneMiB), 2*float64(oneMiB))}},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
tt := tt
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
thr := throttle{
|
||||
bytesPerSecond: 1024 * 1024,
|
||||
clusterBandwidth: 1024 * 1024,
|
||||
}
|
||||
m := &Monitor{
|
||||
activeBuckets: tt.fields.activeBuckets,
|
||||
bucketThrottle: map[string]*throttle{"bucket": &thr},
|
||||
}
|
||||
m.activeBuckets["bucket"].updateExponentialMovingAverage(tt.fields.endTime)
|
||||
got := m.GetReport(SelectBuckets())
|
||||
if !reflect.DeepEqual(got, tt.want) {
|
||||
t.Errorf("GetReport() = %v, want %v", got, tt.want)
|
||||
}
|
||||
m.activeBuckets["bucket"].incrementBytes(tt.fields.update2)
|
||||
m.activeBuckets["bucket"].updateExponentialMovingAverage(tt.fields.endTime2)
|
||||
got = m.GetReport(SelectBuckets())
|
||||
if !reflect.DeepEqual(got, tt.want2) {
|
||||
t.Errorf("GetReport() = %v, want %v", got, tt.want2)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
80
internal/bucket/bandwidth/reader.go
Normal file
80
internal/bucket/bandwidth/reader.go
Normal file
@@ -0,0 +1,80 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package bandwidth
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
)
|
||||
|
||||
// MonitoredReader monitors the bandwidth
|
||||
type MonitoredReader struct {
|
||||
opts *MonitorReaderOptions
|
||||
bucketMeasurement *bucketMeasurement // bucket measurement object
|
||||
reader io.Reader // Reader to wrap
|
||||
throttle *throttle // throttle the rate at which replication occur
|
||||
monitor *Monitor // Monitor reference
|
||||
lastErr error // last error reported, if this non-nil all reads will fail.
|
||||
}
|
||||
|
||||
// MonitorReaderOptions provides configurable options for monitor reader implementation.
|
||||
type MonitorReaderOptions struct {
|
||||
Bucket string
|
||||
Object string
|
||||
HeaderSize int
|
||||
BandwidthBytesPerSec int64
|
||||
ClusterBandwidth int64
|
||||
}
|
||||
|
||||
// NewMonitoredReader returns a io.Reader that reports bandwidth details.
|
||||
func NewMonitoredReader(ctx context.Context, monitor *Monitor, reader io.Reader, opts *MonitorReaderOptions) *MonitoredReader {
|
||||
return &MonitoredReader{
|
||||
opts: opts,
|
||||
bucketMeasurement: monitor.track(opts.Bucket, opts.Object),
|
||||
reader: reader,
|
||||
throttle: monitor.throttleBandwidth(ctx, opts.Bucket, opts.BandwidthBytesPerSec, opts.ClusterBandwidth),
|
||||
monitor: monitor,
|
||||
}
|
||||
}
|
||||
|
||||
// Read wraps the read reader
|
||||
func (m *MonitoredReader) Read(p []byte) (n int, err error) {
|
||||
if m.lastErr != nil {
|
||||
err = m.lastErr
|
||||
return
|
||||
}
|
||||
|
||||
p = p[:m.throttle.GetLimitForBytes(int64(len(p)))]
|
||||
|
||||
n, err = m.reader.Read(p)
|
||||
if err != nil {
|
||||
m.lastErr = err
|
||||
}
|
||||
|
||||
update := n + m.opts.HeaderSize
|
||||
unused := len(p) - update
|
||||
|
||||
m.bucketMeasurement.incrementBytes(uint64(update))
|
||||
m.opts.HeaderSize = 0 // Set to 0 post first read
|
||||
|
||||
if unused > 0 {
|
||||
m.throttle.ReleaseUnusedBandwidth(int64(unused))
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
121
internal/bucket/bandwidth/throttle.go
Normal file
121
internal/bucket/bandwidth/throttle.go
Normal file
@@ -0,0 +1,121 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package bandwidth
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
throttleInternal = 250 * time.Millisecond
|
||||
)
|
||||
|
||||
// throttle implements the throttling for bandwidth
|
||||
type throttle struct {
|
||||
generateTicker *time.Ticker // Ticker to generate available bandwidth
|
||||
freeBytes int64 // unused bytes in the interval
|
||||
bytesPerSecond int64 // max limit for bandwidth
|
||||
bytesPerInterval int64 // bytes allocated for the interval
|
||||
clusterBandwidth int64 // Cluster wide bandwidth needed for reporting
|
||||
cond *sync.Cond // Used to notify waiting threads for bandwidth availability
|
||||
goGenerate int64 // Flag to track if generate routine should be running. 0 == stopped
|
||||
ctx context.Context // Context for generate
|
||||
}
|
||||
|
||||
// newThrottle returns a new bandwidth throttle. Set bytesPerSecond to 0 for no limit
|
||||
func newThrottle(ctx context.Context, bytesPerSecond int64, clusterBandwidth int64) *throttle {
|
||||
if bytesPerSecond == 0 {
|
||||
return &throttle{}
|
||||
}
|
||||
t := &throttle{
|
||||
bytesPerSecond: bytesPerSecond,
|
||||
generateTicker: time.NewTicker(throttleInternal),
|
||||
clusterBandwidth: clusterBandwidth,
|
||||
ctx: ctx,
|
||||
}
|
||||
|
||||
t.cond = sync.NewCond(&sync.Mutex{})
|
||||
t.SetBandwidth(bytesPerSecond, clusterBandwidth)
|
||||
t.freeBytes = t.bytesPerInterval
|
||||
return t
|
||||
}
|
||||
|
||||
// GetLimitForBytes gets the bytes that are possible to send within the limit
|
||||
// if want is <= 0 or no bandwidth limit set, returns want.
|
||||
// Otherwise a value > 0 will always be returned.
|
||||
func (t *throttle) GetLimitForBytes(want int64) int64 {
|
||||
if want <= 0 || atomic.LoadInt64(&t.bytesPerInterval) == 0 {
|
||||
return want
|
||||
}
|
||||
t.cond.L.Lock()
|
||||
defer t.cond.L.Unlock()
|
||||
for {
|
||||
var send int64
|
||||
freeBytes := atomic.LoadInt64(&t.freeBytes)
|
||||
send = want
|
||||
if freeBytes < want {
|
||||
send = freeBytes
|
||||
if send <= 0 {
|
||||
t.cond.Wait()
|
||||
continue
|
||||
}
|
||||
}
|
||||
atomic.AddInt64(&t.freeBytes, -send)
|
||||
|
||||
// Bandwidth was consumed, start generate routine to allocate bandwidth
|
||||
if atomic.CompareAndSwapInt64(&t.goGenerate, 0, 1) {
|
||||
go t.generateBandwidth(t.ctx)
|
||||
}
|
||||
return send
|
||||
}
|
||||
}
|
||||
|
||||
// SetBandwidth sets a new bandwidth limit in bytes per second.
|
||||
func (t *throttle) SetBandwidth(bandwidthBiPS int64, clusterBandwidth int64) {
|
||||
bpi := int64(throttleInternal) * bandwidthBiPS / int64(time.Second)
|
||||
atomic.StoreInt64(&t.bytesPerInterval, bpi)
|
||||
}
|
||||
|
||||
// ReleaseUnusedBandwidth releases bandwidth that was allocated for a user
|
||||
func (t *throttle) ReleaseUnusedBandwidth(bytes int64) {
|
||||
atomic.AddInt64(&t.freeBytes, bytes)
|
||||
}
|
||||
|
||||
// generateBandwidth periodically allocates new bandwidth to use
|
||||
func (t *throttle) generateBandwidth(ctx context.Context) {
|
||||
for {
|
||||
select {
|
||||
case <-t.generateTicker.C:
|
||||
if atomic.LoadInt64(&t.freeBytes) == atomic.LoadInt64(&t.bytesPerInterval) {
|
||||
// No bandwidth consumption stop the routine.
|
||||
atomic.StoreInt64(&t.goGenerate, 0)
|
||||
return
|
||||
}
|
||||
// A new window is available
|
||||
t.cond.L.Lock()
|
||||
atomic.StoreInt64(&t.freeBytes, atomic.LoadInt64(&t.bytesPerInterval))
|
||||
t.cond.Broadcast()
|
||||
t.cond.L.Unlock()
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
159
internal/bucket/encryption/bucket-sse-config.go
Normal file
159
internal/bucket/encryption/bucket-sse-config.go
Normal file
@@ -0,0 +1,159 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"errors"
|
||||
"io"
|
||||
"net/http"
|
||||
|
||||
"github.com/minio/minio/internal/crypto"
|
||||
xhttp "github.com/minio/minio/internal/http"
|
||||
)
|
||||
|
||||
const (
|
||||
// AES256 is used with SSE-S3
|
||||
AES256 SSEAlgorithm = "AES256"
|
||||
// AWSKms is used with SSE-KMS
|
||||
AWSKms SSEAlgorithm = "aws:kms"
|
||||
)
|
||||
|
||||
// SSEAlgorithm - represents valid SSE algorithms supported; currently only AES256 is supported
|
||||
type SSEAlgorithm string
|
||||
|
||||
// UnmarshalXML - Unmarshals XML tag to valid SSE algorithm
|
||||
func (alg *SSEAlgorithm) UnmarshalXML(d *xml.Decoder, start xml.StartElement) error {
|
||||
var s string
|
||||
if err := d.DecodeElement(&s, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
switch s {
|
||||
case string(AES256):
|
||||
*alg = AES256
|
||||
case string(AWSKms):
|
||||
*alg = AWSKms
|
||||
default:
|
||||
return errors.New("Unknown SSE algorithm")
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML - Marshals given SSE algorithm to valid XML
|
||||
func (alg *SSEAlgorithm) MarshalXML(e *xml.Encoder, start xml.StartElement) error {
|
||||
return e.EncodeElement(string(*alg), start)
|
||||
}
|
||||
|
||||
// EncryptionAction - for ApplyServerSideEncryptionByDefault XML tag
|
||||
type EncryptionAction struct {
|
||||
Algorithm SSEAlgorithm `xml:"SSEAlgorithm,omitempty"`
|
||||
MasterKeyID string `xml:"KMSMasterKeyID,omitempty"`
|
||||
}
|
||||
|
||||
// SSERule - for ServerSideEncryptionConfiguration XML tag
|
||||
type SSERule struct {
|
||||
DefaultEncryptionAction EncryptionAction `xml:"ApplyServerSideEncryptionByDefault"`
|
||||
}
|
||||
|
||||
const xmlNS = "http://s3.amazonaws.com/doc/2006-03-01/"
|
||||
|
||||
// BucketSSEConfig - represents default bucket encryption configuration
|
||||
type BucketSSEConfig struct {
|
||||
XMLNS string `xml:"xmlns,attr,omitempty"`
|
||||
XMLName xml.Name `xml:"ServerSideEncryptionConfiguration"`
|
||||
Rules []SSERule `xml:"Rule"`
|
||||
}
|
||||
|
||||
// ParseBucketSSEConfig - Decodes given XML to a valid default bucket encryption config
|
||||
func ParseBucketSSEConfig(r io.Reader) (*BucketSSEConfig, error) {
|
||||
var config BucketSSEConfig
|
||||
err := xml.NewDecoder(r).Decode(&config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Validates server-side encryption config rules
|
||||
// Only one rule is allowed on AWS S3
|
||||
if len(config.Rules) != 1 {
|
||||
return nil, errors.New("only one server-side encryption rule is allowed at a time")
|
||||
}
|
||||
|
||||
for _, rule := range config.Rules {
|
||||
switch rule.DefaultEncryptionAction.Algorithm {
|
||||
case AES256:
|
||||
if rule.DefaultEncryptionAction.MasterKeyID != "" {
|
||||
return nil, errors.New("MasterKeyID is allowed with aws:kms only")
|
||||
}
|
||||
case AWSKms:
|
||||
if rule.DefaultEncryptionAction.MasterKeyID == "" {
|
||||
return nil, errors.New("MasterKeyID is missing with aws:kms")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if config.XMLNS == "" {
|
||||
config.XMLNS = xmlNS
|
||||
}
|
||||
return &config, nil
|
||||
}
|
||||
|
||||
// Apply applies the SSE bucket configuration on the given HTTP headers and
|
||||
// sets the specified SSE headers.
|
||||
//
|
||||
// Apply does not overwrite any existing SSE headers. Further, it will
|
||||
// set minimal SSE-KMS headers if autoEncrypt is true and the BucketSSEConfig
|
||||
// is nil.
|
||||
func (b *BucketSSEConfig) Apply(headers http.Header, autoEncrypt bool) {
|
||||
if _, ok := crypto.IsRequested(headers); ok {
|
||||
return
|
||||
}
|
||||
if b == nil {
|
||||
if autoEncrypt {
|
||||
headers.Set(xhttp.AmzServerSideEncryption, xhttp.AmzEncryptionKMS)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
switch b.Algorithm() {
|
||||
case xhttp.AmzEncryptionAES:
|
||||
headers.Set(xhttp.AmzServerSideEncryption, xhttp.AmzEncryptionAES)
|
||||
case xhttp.AmzEncryptionKMS:
|
||||
headers.Set(xhttp.AmzServerSideEncryption, xhttp.AmzEncryptionKMS)
|
||||
headers.Set(xhttp.AmzServerSideEncryptionKmsID, b.KeyID())
|
||||
}
|
||||
}
|
||||
|
||||
// Algorithm returns the SSE algorithm specified by the SSE configuration.
|
||||
func (b *BucketSSEConfig) Algorithm() SSEAlgorithm {
|
||||
for _, rule := range b.Rules {
|
||||
return rule.DefaultEncryptionAction.Algorithm
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// KeyID returns the KMS key ID specified by the SSE configuration.
|
||||
// If the SSE configuration does not specify SSE-KMS it returns an
|
||||
// empty key ID.
|
||||
func (b *BucketSSEConfig) KeyID() string {
|
||||
for _, rule := range b.Rules {
|
||||
return rule.DefaultEncryptionAction.MasterKeyID
|
||||
}
|
||||
return ""
|
||||
}
|
||||
141
internal/bucket/encryption/bucket-sse-config_test.go
Normal file
141
internal/bucket/encryption/bucket-sse-config_test.go
Normal file
@@ -0,0 +1,141 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/xml"
|
||||
"errors"
|
||||
"testing"
|
||||
)
|
||||
|
||||
// TestParseBucketSSEConfig performs basic sanity tests on ParseBucketSSEConfig
|
||||
func TestParseBucketSSEConfig(t *testing.T) {
|
||||
actualAES256NoNSConfig := &BucketSSEConfig{
|
||||
XMLName: xml.Name{
|
||||
Local: "ServerSideEncryptionConfiguration",
|
||||
},
|
||||
Rules: []SSERule{
|
||||
{
|
||||
DefaultEncryptionAction: EncryptionAction{
|
||||
Algorithm: AES256,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
actualAES256Config := &BucketSSEConfig{
|
||||
XMLNS: xmlNS,
|
||||
XMLName: xml.Name{
|
||||
Local: "ServerSideEncryptionConfiguration",
|
||||
},
|
||||
Rules: []SSERule{
|
||||
{
|
||||
DefaultEncryptionAction: EncryptionAction{
|
||||
Algorithm: AES256,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
actualKMSConfig := &BucketSSEConfig{
|
||||
XMLNS: xmlNS,
|
||||
XMLName: xml.Name{
|
||||
Local: "ServerSideEncryptionConfiguration",
|
||||
},
|
||||
Rules: []SSERule{
|
||||
{
|
||||
DefaultEncryptionAction: EncryptionAction{
|
||||
Algorithm: AWSKms,
|
||||
MasterKeyID: "arn:aws:kms:us-east-1:1234/5678example",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
inputXML string
|
||||
expectedErr error
|
||||
shouldPass bool
|
||||
expectedConfig *BucketSSEConfig
|
||||
}{
|
||||
// 1. Valid XML SSE-S3
|
||||
{
|
||||
inputXML: `<ServerSideEncryptionConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ApplyServerSideEncryptionByDefault><SSEAlgorithm>AES256</SSEAlgorithm></ApplyServerSideEncryptionByDefault></Rule></ServerSideEncryptionConfiguration>`,
|
||||
expectedErr: nil,
|
||||
shouldPass: true,
|
||||
expectedConfig: actualAES256Config,
|
||||
},
|
||||
// 2. Valid XML SSE-KMS
|
||||
{
|
||||
inputXML: `<ServerSideEncryptionConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ApplyServerSideEncryptionByDefault><SSEAlgorithm>aws:kms</SSEAlgorithm><KMSMasterKeyID>arn:aws:kms:us-east-1:1234/5678example</KMSMasterKeyID></ApplyServerSideEncryptionByDefault></Rule></ServerSideEncryptionConfiguration>`,
|
||||
expectedErr: nil,
|
||||
shouldPass: true,
|
||||
expectedConfig: actualKMSConfig,
|
||||
},
|
||||
// 3. Invalid - more than one rule
|
||||
{
|
||||
inputXML: `<ServerSideEncryptionConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ApplyServerSideEncryptionByDefault><SSEAlgorithm>AES256</SSEAlgorithm></ApplyServerSideEncryptionByDefault></Rule><Rule><ApplyServerSideEncryptionByDefault><SSEAlgorithm>AES256</SSEAlgorithm></ApplyServerSideEncryptionByDefault></Rule></ServerSideEncryptionConfiguration>`,
|
||||
expectedErr: errors.New("only one server-side encryption rule is allowed at a time"),
|
||||
shouldPass: false,
|
||||
},
|
||||
// 4. Invalid XML - master key ID present along with AES256
|
||||
{
|
||||
inputXML: `<ServerSideEncryptionConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ApplyServerSideEncryptionByDefault><SSEAlgorithm>AES256</SSEAlgorithm><KMSMasterKeyID>arn:aws:kms:us-east-1:1234/5678example</KMSMasterKeyID></ApplyServerSideEncryptionByDefault></Rule></ServerSideEncryptionConfiguration>`,
|
||||
expectedErr: errors.New("MasterKeyID is allowed with aws:kms only"),
|
||||
shouldPass: false,
|
||||
},
|
||||
// 5. Invalid XML - master key ID not provided when algorithm is set to aws:kms algorithm
|
||||
{
|
||||
inputXML: `<ServerSideEncryptionConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ApplyServerSideEncryptionByDefault><SSEAlgorithm>aws:kms</SSEAlgorithm></ApplyServerSideEncryptionByDefault></Rule></ServerSideEncryptionConfiguration>`,
|
||||
expectedErr: errors.New("MasterKeyID is missing with aws:kms"),
|
||||
shouldPass: false,
|
||||
},
|
||||
// 6. Invalid Algorithm
|
||||
{
|
||||
inputXML: `<ServerSideEncryptionConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ApplyServerSideEncryptionByDefault><SSEAlgorithm>InvalidAlgorithm</SSEAlgorithm></ApplyServerSideEncryptionByDefault></Rule></ServerSideEncryptionConfiguration>`,
|
||||
expectedErr: errors.New("Unknown SSE algorithm"),
|
||||
shouldPass: false,
|
||||
},
|
||||
// 7. Valid XML without the namespace set
|
||||
{
|
||||
inputXML: `<ServerSideEncryptionConfiguration><Rule><ApplyServerSideEncryptionByDefault><SSEAlgorithm>AES256</SSEAlgorithm></ApplyServerSideEncryptionByDefault></Rule></ServerSideEncryptionConfiguration>`,
|
||||
expectedErr: nil,
|
||||
shouldPass: true,
|
||||
expectedConfig: actualAES256NoNSConfig,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tc := range testCases {
|
||||
_, err := ParseBucketSSEConfig(bytes.NewReader([]byte(tc.inputXML)))
|
||||
if tc.shouldPass && err != nil {
|
||||
t.Fatalf("Test case %d: Expected to succeed but got %s", i+1, err)
|
||||
}
|
||||
|
||||
if !tc.shouldPass {
|
||||
if err == nil || err != nil && err.Error() != tc.expectedErr.Error() {
|
||||
t.Fatalf("Test case %d: Expected %s but got %s", i+1, tc.expectedErr, err)
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
if expectedXML, err := xml.Marshal(tc.expectedConfig); err != nil || !bytes.Equal(expectedXML, []byte(tc.inputXML)) {
|
||||
t.Fatalf("Test case %d: Expected bucket encryption XML %s but got %s", i+1, string(expectedXML), tc.inputXML)
|
||||
}
|
||||
}
|
||||
}
|
||||
29
internal/bucket/lifecycle/action_string.go
Normal file
29
internal/bucket/lifecycle/action_string.go
Normal file
@@ -0,0 +1,29 @@
|
||||
// Code generated by "stringer -type Action lifecycle.go"; DO NOT EDIT.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import "strconv"
|
||||
|
||||
func _() {
|
||||
// An "invalid array index" compiler error signifies that the constant values have changed.
|
||||
// Re-run the stringer command to generate them again.
|
||||
var x [1]struct{}
|
||||
_ = x[NoneAction-0]
|
||||
_ = x[DeleteAction-1]
|
||||
_ = x[DeleteVersionAction-2]
|
||||
_ = x[TransitionAction-3]
|
||||
_ = x[TransitionVersionAction-4]
|
||||
_ = x[DeleteRestoredAction-5]
|
||||
_ = x[DeleteRestoredVersionAction-6]
|
||||
}
|
||||
|
||||
const _Action_name = "NoneActionDeleteActionDeleteVersionActionTransitionActionTransitionVersionActionDeleteRestoredActionDeleteRestoredVersionAction"
|
||||
|
||||
var _Action_index = [...]uint8{0, 10, 22, 41, 57, 80, 100, 127}
|
||||
|
||||
func (i Action) String() string {
|
||||
if i < 0 || i >= Action(len(_Action_index)-1) {
|
||||
return "Action(" + strconv.FormatInt(int64(i), 10) + ")"
|
||||
}
|
||||
return _Action_name[_Action_index[i]:_Action_index[i+1]]
|
||||
}
|
||||
74
internal/bucket/lifecycle/and.go
Normal file
74
internal/bucket/lifecycle/and.go
Normal file
@@ -0,0 +1,74 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
)
|
||||
|
||||
var errDuplicateTagKey = Errorf("Duplicate Tag Keys are not allowed")
|
||||
|
||||
// And - a tag to combine a prefix and multiple tags for lifecycle configuration rule.
|
||||
type And struct {
|
||||
XMLName xml.Name `xml:"And"`
|
||||
Prefix Prefix `xml:"Prefix,omitempty"`
|
||||
Tags []Tag `xml:"Tag,omitempty"`
|
||||
}
|
||||
|
||||
// isEmpty returns true if Tags field is null
|
||||
func (a And) isEmpty() bool {
|
||||
return len(a.Tags) == 0 && !a.Prefix.set
|
||||
}
|
||||
|
||||
// Validate - validates the And field
|
||||
func (a And) Validate() error {
|
||||
emptyPrefix := !a.Prefix.set
|
||||
emptyTags := len(a.Tags) == 0
|
||||
|
||||
if emptyPrefix && emptyTags {
|
||||
return nil
|
||||
}
|
||||
|
||||
if emptyPrefix && !emptyTags || !emptyPrefix && emptyTags {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
|
||||
if a.ContainsDuplicateTag() {
|
||||
return errDuplicateTagKey
|
||||
}
|
||||
for _, t := range a.Tags {
|
||||
if err := t.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ContainsDuplicateTag - returns true if duplicate keys are present in And
|
||||
func (a And) ContainsDuplicateTag() bool {
|
||||
x := make(map[string]struct{}, len(a.Tags))
|
||||
|
||||
for _, t := range a.Tags {
|
||||
if _, has := x[t.Key]; has {
|
||||
return true
|
||||
}
|
||||
x[t.Key] = struct{}{}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
45
internal/bucket/lifecycle/error.go
Normal file
45
internal/bucket/lifecycle/error.go
Normal file
@@ -0,0 +1,45 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
// Error is the generic type for any error happening during tag
|
||||
// parsing.
|
||||
type Error struct {
|
||||
err error
|
||||
}
|
||||
|
||||
// Errorf - formats according to a format specifier and returns
|
||||
// the string as a value that satisfies error of type tagging.Error
|
||||
func Errorf(format string, a ...interface{}) error {
|
||||
return Error{err: fmt.Errorf(format, a...)}
|
||||
}
|
||||
|
||||
// Unwrap the internal error.
|
||||
func (e Error) Unwrap() error { return e.err }
|
||||
|
||||
// Error 'error' compatible method.
|
||||
func (e Error) Error() string {
|
||||
if e.err == nil {
|
||||
return "lifecycle: cause <nil>"
|
||||
}
|
||||
return e.err.Error()
|
||||
}
|
||||
195
internal/bucket/lifecycle/expiration.go
Normal file
195
internal/bucket/lifecycle/expiration.go
Normal file
@@ -0,0 +1,195 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
errLifecycleInvalidDate = Errorf("Date must be provided in ISO 8601 format")
|
||||
errLifecycleInvalidDays = Errorf("Days must be positive integer when used with Expiration")
|
||||
errLifecycleInvalidExpiration = Errorf("Exactly one of Days (positive integer) or Date (positive ISO 8601 format) should be present inside Expiration.")
|
||||
errLifecycleInvalidDeleteMarker = Errorf("Delete marker cannot be specified with Days or Date in a Lifecycle Expiration Policy")
|
||||
errLifecycleDateNotMidnight = Errorf("'Date' must be at midnight GMT")
|
||||
)
|
||||
|
||||
// ExpirationDays is a type alias to unmarshal Days in Expiration
|
||||
type ExpirationDays int
|
||||
|
||||
// UnmarshalXML parses number of days from Expiration and validates if
|
||||
// greater than zero
|
||||
func (eDays *ExpirationDays) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
var numDays int
|
||||
err := d.DecodeElement(&numDays, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if numDays <= 0 {
|
||||
return errLifecycleInvalidDays
|
||||
}
|
||||
*eDays = ExpirationDays(numDays)
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML encodes number of days to expire if it is non-zero and
|
||||
// encodes empty string otherwise
|
||||
func (eDays ExpirationDays) MarshalXML(e *xml.Encoder, startElement xml.StartElement) error {
|
||||
if eDays == 0 {
|
||||
return nil
|
||||
}
|
||||
return e.EncodeElement(int(eDays), startElement)
|
||||
}
|
||||
|
||||
// ExpirationDate is a embedded type containing time.Time to unmarshal
|
||||
// Date in Expiration
|
||||
type ExpirationDate struct {
|
||||
time.Time
|
||||
}
|
||||
|
||||
// UnmarshalXML parses date from Expiration and validates date format
|
||||
func (eDate *ExpirationDate) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
var dateStr string
|
||||
err := d.DecodeElement(&dateStr, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// While AWS documentation mentions that the date specified
|
||||
// must be present in ISO 8601 format, in reality they allow
|
||||
// users to provide RFC 3339 compliant dates.
|
||||
expDate, err := time.Parse(time.RFC3339, dateStr)
|
||||
if err != nil {
|
||||
return errLifecycleInvalidDate
|
||||
}
|
||||
// Allow only date timestamp specifying midnight GMT
|
||||
hr, min, sec := expDate.Clock()
|
||||
nsec := expDate.Nanosecond()
|
||||
loc := expDate.Location()
|
||||
if !(hr == 0 && min == 0 && sec == 0 && nsec == 0 && loc.String() == time.UTC.String()) {
|
||||
return errLifecycleDateNotMidnight
|
||||
}
|
||||
|
||||
*eDate = ExpirationDate{expDate}
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML encodes expiration date if it is non-zero and encodes
|
||||
// empty string otherwise
|
||||
func (eDate ExpirationDate) MarshalXML(e *xml.Encoder, startElement xml.StartElement) error {
|
||||
if eDate.Time.IsZero() {
|
||||
return nil
|
||||
}
|
||||
return e.EncodeElement(eDate.Format(time.RFC3339), startElement)
|
||||
}
|
||||
|
||||
// ExpireDeleteMarker represents value of ExpiredObjectDeleteMarker field in Expiration XML element.
|
||||
type ExpireDeleteMarker struct {
|
||||
val bool
|
||||
set bool
|
||||
}
|
||||
|
||||
// Expiration - expiration actions for a rule in lifecycle configuration.
|
||||
type Expiration struct {
|
||||
XMLName xml.Name `xml:"Expiration"`
|
||||
Days ExpirationDays `xml:"Days,omitempty"`
|
||||
Date ExpirationDate `xml:"Date,omitempty"`
|
||||
DeleteMarker ExpireDeleteMarker `xml:"ExpiredObjectDeleteMarker"`
|
||||
|
||||
set bool
|
||||
}
|
||||
|
||||
// MarshalXML encodes delete marker boolean into an XML form.
|
||||
func (b ExpireDeleteMarker) MarshalXML(e *xml.Encoder, startElement xml.StartElement) error {
|
||||
if !b.set {
|
||||
return nil
|
||||
}
|
||||
return e.EncodeElement(b.val, startElement)
|
||||
}
|
||||
|
||||
// UnmarshalXML decodes delete marker boolean from the XML form.
|
||||
func (b *ExpireDeleteMarker) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
var exp bool
|
||||
err := d.DecodeElement(&exp, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
b.val = exp
|
||||
b.set = true
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML encodes expiration field into an XML form.
|
||||
func (e Expiration) MarshalXML(enc *xml.Encoder, startElement xml.StartElement) error {
|
||||
if !e.set {
|
||||
return nil
|
||||
}
|
||||
type expirationWrapper Expiration
|
||||
return enc.EncodeElement(expirationWrapper(e), startElement)
|
||||
}
|
||||
|
||||
// UnmarshalXML decodes expiration field from the XML form.
|
||||
func (e *Expiration) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
type expirationWrapper Expiration
|
||||
var exp expirationWrapper
|
||||
err := d.DecodeElement(&exp, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*e = Expiration(exp)
|
||||
e.set = true
|
||||
return nil
|
||||
}
|
||||
|
||||
// Validate - validates the "Expiration" element
|
||||
func (e Expiration) Validate() error {
|
||||
if !e.set {
|
||||
return nil
|
||||
}
|
||||
|
||||
// DeleteMarker cannot be specified if date or dates are specified.
|
||||
if (!e.IsDaysNull() || !e.IsDateNull()) && e.DeleteMarker.set {
|
||||
return errLifecycleInvalidDeleteMarker
|
||||
}
|
||||
|
||||
if !e.DeleteMarker.set && e.IsDaysNull() && e.IsDateNull() {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
|
||||
// Both expiration days and date are specified
|
||||
if !e.IsDaysNull() && !e.IsDateNull() {
|
||||
return errLifecycleInvalidExpiration
|
||||
}
|
||||
|
||||
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.Time.IsZero()
|
||||
}
|
||||
|
||||
// IsNull returns true if both date and days fields are null
|
||||
func (e Expiration) IsNull() bool {
|
||||
return e.IsDaysNull() && e.IsDateNull()
|
||||
}
|
||||
113
internal/bucket/lifecycle/expiration_test.go
Normal file
113
internal/bucket/lifecycle/expiration_test.go
Normal file
@@ -0,0 +1,113 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"fmt"
|
||||
"testing"
|
||||
)
|
||||
|
||||
// appropriate errors on validation
|
||||
func TestInvalidExpiration(t *testing.T) {
|
||||
testCases := []struct {
|
||||
inputXML string
|
||||
expectedErr error
|
||||
}{
|
||||
{ // Expiration with zero days
|
||||
inputXML: ` <Expiration>
|
||||
<Days>0</Days>
|
||||
</Expiration>`,
|
||||
expectedErr: errLifecycleInvalidDays,
|
||||
},
|
||||
{ // Expiration with invalid date
|
||||
inputXML: ` <Expiration>
|
||||
<Date>invalid date</Date>
|
||||
</Expiration>`,
|
||||
expectedErr: errLifecycleInvalidDate,
|
||||
},
|
||||
{ // Expiration with both number of days nor a date
|
||||
inputXML: `<Expiration>
|
||||
<Date>2019-04-20T00:01:00Z</Date>
|
||||
</Expiration>`,
|
||||
expectedErr: errLifecycleDateNotMidnight,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tc := range testCases {
|
||||
t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) {
|
||||
var expiration Expiration
|
||||
err := xml.Unmarshal([]byte(tc.inputXML), &expiration)
|
||||
if err != tc.expectedErr {
|
||||
t.Fatalf("%d: Expected %v but got %v", i+1, tc.expectedErr, err)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
validationTestCases := []struct {
|
||||
inputXML string
|
||||
expectedErr error
|
||||
}{
|
||||
{ // Expiration with a valid ISO 8601 date
|
||||
inputXML: `<Expiration>
|
||||
<Date>2019-04-20T00:00:00Z</Date>
|
||||
</Expiration>`,
|
||||
expectedErr: nil,
|
||||
},
|
||||
{ // Expiration with a valid number of days
|
||||
inputXML: `<Expiration>
|
||||
<Days>3</Days>
|
||||
</Expiration>`,
|
||||
expectedErr: nil,
|
||||
},
|
||||
{ // Expiration with neither number of days nor a date
|
||||
inputXML: `<Expiration>
|
||||
</Expiration>`,
|
||||
expectedErr: errXMLNotWellFormed,
|
||||
},
|
||||
{ // Expiration with both number of days and a date
|
||||
inputXML: `<Expiration>
|
||||
<Days>3</Days>
|
||||
<Date>2019-04-20T00:00:00Z</Date>
|
||||
</Expiration>`,
|
||||
expectedErr: errLifecycleInvalidExpiration,
|
||||
},
|
||||
{ // Expiration with both ExpiredObjectDeleteMarker and days
|
||||
inputXML: `<Expiration>
|
||||
<Days>3</Days>
|
||||
<ExpiredObjectDeleteMarker>false</ExpiredObjectDeleteMarker>
|
||||
</Expiration>`,
|
||||
expectedErr: errLifecycleInvalidDeleteMarker,
|
||||
},
|
||||
}
|
||||
for i, tc := range validationTestCases {
|
||||
t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) {
|
||||
var expiration Expiration
|
||||
err := xml.Unmarshal([]byte(tc.inputXML), &expiration)
|
||||
if err != nil {
|
||||
t.Fatalf("%d: %v", i+1, err)
|
||||
}
|
||||
|
||||
err = expiration.Validate()
|
||||
if err != tc.expectedErr {
|
||||
t.Fatalf("%d: got: %v, expected: %v", i+1, err, tc.expectedErr)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
178
internal/bucket/lifecycle/filter.go
Normal file
178
internal/bucket/lifecycle/filter.go
Normal file
@@ -0,0 +1,178 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"io"
|
||||
)
|
||||
|
||||
var (
|
||||
errInvalidFilter = Errorf("Filter must have exactly one of Prefix, Tag, or And specified")
|
||||
)
|
||||
|
||||
// Filter - a filter for a lifecycle configuration Rule.
|
||||
type Filter struct {
|
||||
XMLName xml.Name `xml:"Filter"`
|
||||
set bool
|
||||
|
||||
Prefix Prefix
|
||||
|
||||
And And
|
||||
andSet bool
|
||||
|
||||
Tag Tag
|
||||
tagSet bool
|
||||
// Caching tags, only once
|
||||
cachedTags []string
|
||||
}
|
||||
|
||||
// MarshalXML - produces the xml representation of the Filter struct
|
||||
// only one of Prefix, And and Tag should be present in the output.
|
||||
func (f Filter) MarshalXML(e *xml.Encoder, start xml.StartElement) error {
|
||||
if err := e.EncodeToken(start); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
switch {
|
||||
case !f.And.isEmpty():
|
||||
if err := e.EncodeElement(f.And, xml.StartElement{Name: xml.Name{Local: "And"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
case !f.Tag.IsEmpty():
|
||||
if err := e.EncodeElement(f.Tag, xml.StartElement{Name: xml.Name{Local: "Tag"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
default:
|
||||
// Always print Prefix field when both And & Tag are empty
|
||||
if err := e.EncodeElement(f.Prefix, xml.StartElement{Name: xml.Name{Local: "Prefix"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return e.EncodeToken(xml.EndElement{Name: start.Name})
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (f *Filter) UnmarshalXML(d *xml.Decoder, start xml.StartElement) (err error) {
|
||||
f.set = true
|
||||
for {
|
||||
// Read tokens from the XML document in a stream.
|
||||
t, err := d.Token()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
switch se := t.(type) {
|
||||
case xml.StartElement:
|
||||
switch se.Name.Local {
|
||||
case "Prefix":
|
||||
var p Prefix
|
||||
if err = d.DecodeElement(&p, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
f.Prefix = p
|
||||
case "And":
|
||||
var and And
|
||||
if err = d.DecodeElement(&and, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
f.And = and
|
||||
f.andSet = true
|
||||
case "Tag":
|
||||
var tag Tag
|
||||
if err = d.DecodeElement(&tag, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
f.Tag = tag
|
||||
f.tagSet = true
|
||||
default:
|
||||
return errUnknownXMLTag
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// IsEmpty returns true if Filter is not specified in the XML
|
||||
func (f Filter) IsEmpty() bool {
|
||||
return !f.set
|
||||
}
|
||||
|
||||
// Validate - validates the filter element
|
||||
func (f Filter) Validate() error {
|
||||
if f.IsEmpty() {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
// A Filter must have exactly one of Prefix, Tag, or And specified.
|
||||
if !f.And.isEmpty() {
|
||||
if f.Prefix.set {
|
||||
return errInvalidFilter
|
||||
}
|
||||
if !f.Tag.IsEmpty() {
|
||||
return errInvalidFilter
|
||||
}
|
||||
if err := f.And.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if f.Prefix.set {
|
||||
if !f.Tag.IsEmpty() {
|
||||
return errInvalidFilter
|
||||
}
|
||||
}
|
||||
if !f.Tag.IsEmpty() {
|
||||
if f.Prefix.set {
|
||||
return errInvalidFilter
|
||||
}
|
||||
if err := f.Tag.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// TestTags tests if the object tags satisfy the Filter tags requirement,
|
||||
// it returns true if there is no tags in the underlying Filter.
|
||||
func (f Filter) TestTags(tags []string) bool {
|
||||
if f.cachedTags == nil {
|
||||
tags := make([]string, 0)
|
||||
for _, t := range append(f.And.Tags, f.Tag) {
|
||||
if !t.IsEmpty() {
|
||||
tags = append(tags, t.String())
|
||||
}
|
||||
}
|
||||
f.cachedTags = tags
|
||||
}
|
||||
for _, ct := range f.cachedTags {
|
||||
foundTag := false
|
||||
for _, t := range tags {
|
||||
if ct == t {
|
||||
foundTag = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !foundTag {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
126
internal/bucket/lifecycle/filter_test.go
Normal file
126
internal/bucket/lifecycle/filter_test.go
Normal file
@@ -0,0 +1,126 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"fmt"
|
||||
"testing"
|
||||
)
|
||||
|
||||
// TestUnsupportedFilters checks if parsing Filter xml with
|
||||
// unsupported elements returns appropriate errors
|
||||
func TestUnsupportedFilters(t *testing.T) {
|
||||
testCases := []struct {
|
||||
inputXML string
|
||||
expectedErr error
|
||||
}{
|
||||
{ // Filter with And tags
|
||||
inputXML: ` <Filter>
|
||||
<And>
|
||||
<Prefix>key-prefix</Prefix>
|
||||
</And>
|
||||
</Filter>`,
|
||||
expectedErr: errXMLNotWellFormed,
|
||||
},
|
||||
{ // Filter with Tag tags
|
||||
inputXML: ` <Filter>
|
||||
<Tag>
|
||||
<Key>key1</Key>
|
||||
<Value>value1</Value>
|
||||
</Tag>
|
||||
</Filter>`,
|
||||
expectedErr: nil,
|
||||
},
|
||||
{ // Filter with Prefix tag
|
||||
inputXML: ` <Filter>
|
||||
<Prefix>key-prefix</Prefix>
|
||||
</Filter>`,
|
||||
expectedErr: nil,
|
||||
},
|
||||
{ // Filter without And and multiple Tag tags
|
||||
inputXML: ` <Filter>
|
||||
<Prefix>key-prefix</Prefix>
|
||||
<Tag>
|
||||
<Key>key1</Key>
|
||||
<Value>value1</Value>
|
||||
</Tag>
|
||||
<Tag>
|
||||
<Key>key2</Key>
|
||||
<Value>value2</Value>
|
||||
</Tag>
|
||||
</Filter>`,
|
||||
expectedErr: errInvalidFilter,
|
||||
},
|
||||
{ // Filter with And, Prefix & multiple Tag tags
|
||||
inputXML: ` <Filter>
|
||||
<And>
|
||||
<Prefix>key-prefix</Prefix>
|
||||
<Tag>
|
||||
<Key>key1</Key>
|
||||
<Value>value1</Value>
|
||||
</Tag>
|
||||
<Tag>
|
||||
<Key>key2</Key>
|
||||
<Value>value2</Value>
|
||||
</Tag>
|
||||
</And>
|
||||
</Filter>`,
|
||||
expectedErr: nil,
|
||||
},
|
||||
{ // Filter with And and multiple Tag tags
|
||||
inputXML: ` <Filter>
|
||||
<And>
|
||||
<Prefix></Prefix>
|
||||
<Tag>
|
||||
<Key>key1</Key>
|
||||
<Value>value1</Value>
|
||||
</Tag>
|
||||
<Tag>
|
||||
<Key>key2</Key>
|
||||
<Value>value2</Value>
|
||||
</Tag>
|
||||
</And>
|
||||
</Filter>`,
|
||||
expectedErr: nil,
|
||||
},
|
||||
{ // Filter without And and single Tag tag
|
||||
inputXML: ` <Filter>
|
||||
<Prefix>key-prefix</Prefix>
|
||||
<Tag>
|
||||
<Key>key1</Key>
|
||||
<Value>value1</Value>
|
||||
</Tag>
|
||||
</Filter>`,
|
||||
expectedErr: errInvalidFilter,
|
||||
},
|
||||
}
|
||||
for i, tc := range testCases {
|
||||
t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) {
|
||||
var filter Filter
|
||||
err := xml.Unmarshal([]byte(tc.inputXML), &filter)
|
||||
if err != nil {
|
||||
t.Fatalf("%d: Expected no error but got %v", i+1, err)
|
||||
}
|
||||
err = filter.Validate()
|
||||
if err != tc.expectedErr {
|
||||
t.Fatalf("%d: Expected %v but got %v", i+1, tc.expectedErr, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
462
internal/bucket/lifecycle/lifecycle.go
Normal file
462
internal/bucket/lifecycle/lifecycle.go
Normal file
@@ -0,0 +1,462 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"fmt"
|
||||
"io"
|
||||
"strings"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
errLifecycleTooManyRules = Errorf("Lifecycle configuration allows a maximum of 1000 rules")
|
||||
errLifecycleNoRule = Errorf("Lifecycle configuration should have at least one rule")
|
||||
errLifecycleDuplicateID = Errorf("Lifecycle configuration has rule with the same ID. Rule ID must be unique.")
|
||||
errXMLNotWellFormed = Errorf("The XML you provided was not well-formed or did not validate against our published schema")
|
||||
)
|
||||
|
||||
const (
|
||||
// TransitionComplete marks completed transition
|
||||
TransitionComplete = "complete"
|
||||
// TransitionPending - transition is yet to be attempted
|
||||
TransitionPending = "pending"
|
||||
)
|
||||
|
||||
// Action represents a delete action or other transition
|
||||
// actions that will be implemented later.
|
||||
type Action int
|
||||
|
||||
//go:generate stringer -type Action $GOFILE
|
||||
|
||||
const (
|
||||
// NoneAction means no action required after evaluating lifecycle rules
|
||||
NoneAction Action = iota
|
||||
// DeleteAction means the object needs to be removed after evaluating lifecycle rules
|
||||
DeleteAction
|
||||
// DeleteVersionAction deletes a particular version
|
||||
DeleteVersionAction
|
||||
// TransitionAction transitions a particular object after evaluating lifecycle transition rules
|
||||
TransitionAction
|
||||
//TransitionVersionAction transitions a particular object version after evaluating lifecycle transition rules
|
||||
TransitionVersionAction
|
||||
// DeleteRestoredAction means the temporarily restored object needs to be removed after evaluating lifecycle rules
|
||||
DeleteRestoredAction
|
||||
// DeleteRestoredVersionAction deletes a particular version that was temporarily restored
|
||||
DeleteRestoredVersionAction
|
||||
)
|
||||
|
||||
// Lifecycle - Configuration for bucket lifecycle.
|
||||
type Lifecycle struct {
|
||||
XMLName xml.Name `xml:"LifecycleConfiguration"`
|
||||
Rules []Rule `xml:"Rule"`
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (lc *Lifecycle) UnmarshalXML(d *xml.Decoder, start xml.StartElement) (err error) {
|
||||
switch start.Name.Local {
|
||||
case "LifecycleConfiguration", "BucketLifecycleConfiguration":
|
||||
default:
|
||||
return xml.UnmarshalError(fmt.Sprintf("expected element type <LifecycleConfiguration>/<BucketLifecycleConfiguration> but have <%s>",
|
||||
start.Name.Local))
|
||||
}
|
||||
for {
|
||||
// Read tokens from the XML document in a stream.
|
||||
t, err := d.Token()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
switch se := t.(type) {
|
||||
case xml.StartElement:
|
||||
switch se.Name.Local {
|
||||
case "Rule":
|
||||
var r Rule
|
||||
if err = d.DecodeElement(&r, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
lc.Rules = append(lc.Rules, r)
|
||||
default:
|
||||
return xml.UnmarshalError(fmt.Sprintf("expected element type <Rule> but have <%s>", se.Name.Local))
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// HasActiveRules - returns whether policy has active rules for.
|
||||
// Optionally a prefix can be supplied.
|
||||
// If recursive is specified the function will also return true if any level below the
|
||||
// prefix has active rules. If no prefix is specified recursive is effectively true.
|
||||
func (lc Lifecycle) HasActiveRules(prefix string, recursive bool) bool {
|
||||
if len(lc.Rules) == 0 {
|
||||
return false
|
||||
}
|
||||
for _, rule := range lc.Rules {
|
||||
if rule.Status == Disabled {
|
||||
continue
|
||||
}
|
||||
|
||||
if len(prefix) > 0 && len(rule.GetPrefix()) > 0 {
|
||||
if !recursive {
|
||||
// If not recursive, incoming prefix must be in rule prefix
|
||||
if !strings.HasPrefix(prefix, rule.GetPrefix()) {
|
||||
continue
|
||||
}
|
||||
}
|
||||
if recursive {
|
||||
// If recursive, we can skip this rule if it doesn't match the tested prefix.
|
||||
if !strings.HasPrefix(prefix, rule.GetPrefix()) && !strings.HasPrefix(rule.GetPrefix(), prefix) {
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if rule.NoncurrentVersionExpiration.NoncurrentDays > 0 {
|
||||
return true
|
||||
}
|
||||
if rule.NoncurrentVersionTransition.NoncurrentDays > 0 {
|
||||
return true
|
||||
}
|
||||
if rule.Expiration.IsNull() && rule.Transition.IsNull() {
|
||||
continue
|
||||
}
|
||||
if !rule.Expiration.IsDateNull() && rule.Expiration.Date.Before(time.Now()) {
|
||||
return true
|
||||
}
|
||||
if !rule.Transition.IsDateNull() && rule.Transition.Date.Before(time.Now()) {
|
||||
return true
|
||||
}
|
||||
if !rule.Expiration.IsDaysNull() || !rule.Transition.IsDaysNull() {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// ParseLifecycleConfig - parses data in given reader to Lifecycle.
|
||||
func ParseLifecycleConfig(reader io.Reader) (*Lifecycle, error) {
|
||||
var lc Lifecycle
|
||||
if err := xml.NewDecoder(reader).Decode(&lc); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &lc, nil
|
||||
}
|
||||
|
||||
// Validate - validates the lifecycle configuration
|
||||
func (lc Lifecycle) Validate() error {
|
||||
// Lifecycle config can't have more than 1000 rules
|
||||
if len(lc.Rules) > 1000 {
|
||||
return errLifecycleTooManyRules
|
||||
}
|
||||
// Lifecycle config should have at least one rule
|
||||
if len(lc.Rules) == 0 {
|
||||
return errLifecycleNoRule
|
||||
}
|
||||
// Validate all the rules in the lifecycle config
|
||||
for _, r := range lc.Rules {
|
||||
if err := r.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// Make sure Rule ID is unique
|
||||
for i := range lc.Rules {
|
||||
if i == len(lc.Rules)-1 {
|
||||
break
|
||||
}
|
||||
otherRules := lc.Rules[i+1:]
|
||||
for _, otherRule := range otherRules {
|
||||
if lc.Rules[i].ID == otherRule.ID {
|
||||
return errLifecycleDuplicateID
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// FilterActionableRules returns the rules actions that need to be executed
|
||||
// after evaluating prefix/tag filtering
|
||||
func (lc Lifecycle) FilterActionableRules(obj ObjectOpts) []Rule {
|
||||
if obj.Name == "" {
|
||||
return nil
|
||||
}
|
||||
var rules []Rule
|
||||
for _, rule := range lc.Rules {
|
||||
if rule.Status == Disabled {
|
||||
continue
|
||||
}
|
||||
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)
|
||||
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
|
||||
}
|
||||
// The NoncurrentVersionTransition action requests MinIO to transition
|
||||
// noncurrent versions of objects x days after the objects become
|
||||
// noncurrent.
|
||||
if !rule.NoncurrentVersionTransition.IsDaysNull() {
|
||||
rules = append(rules, rule)
|
||||
continue
|
||||
}
|
||||
|
||||
if rule.Filter.TestTags(strings.Split(obj.UserTags, "&")) {
|
||||
rules = append(rules, rule)
|
||||
}
|
||||
if !rule.Transition.IsNull() {
|
||||
rules = append(rules, rule)
|
||||
}
|
||||
}
|
||||
return rules
|
||||
}
|
||||
|
||||
// ObjectOpts provides information to deduce the lifecycle actions
|
||||
// which can be triggered on the resultant object.
|
||||
type ObjectOpts struct {
|
||||
Name string
|
||||
UserTags string
|
||||
ModTime time.Time
|
||||
VersionID string
|
||||
IsLatest bool
|
||||
DeleteMarker bool
|
||||
NumVersions int
|
||||
SuccessorModTime time.Time
|
||||
TransitionStatus string
|
||||
RestoreOngoing bool
|
||||
RestoreExpires time.Time
|
||||
RemoteTiersImmediately []string // strictly for debug only
|
||||
}
|
||||
|
||||
// doesMatchDebugTiers returns true if tier matches one of the debugTiers, false
|
||||
// otherwise.
|
||||
func doesMatchDebugTiers(tier string, debugTiers []string) bool {
|
||||
for _, t := range debugTiers {
|
||||
if strings.ToUpper(tier) == strings.ToUpper(t) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// ExpiredObjectDeleteMarker returns true if an object version referred to by o
|
||||
// is the only version remaining and is a delete marker. It returns false
|
||||
// otherwise.
|
||||
func (o ObjectOpts) ExpiredObjectDeleteMarker() bool {
|
||||
return o.DeleteMarker && o.NumVersions == 1
|
||||
}
|
||||
|
||||
// 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 {
|
||||
var action = NoneAction
|
||||
if obj.ModTime.IsZero() {
|
||||
return action
|
||||
}
|
||||
for _, rule := range lc.FilterActionableRules(obj) {
|
||||
if obj.ExpiredObjectDeleteMarker() && 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.
|
||||
return DeleteVersionAction
|
||||
}
|
||||
|
||||
if !rule.NoncurrentVersionExpiration.IsDaysNull() {
|
||||
if obj.VersionID != "" && !obj.IsLatest && !obj.SuccessorModTime.IsZero() {
|
||||
// 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 time.Now().After(ExpectedExpiryTime(obj.SuccessorModTime, int(rule.NoncurrentVersionExpiration.NoncurrentDays))) {
|
||||
return DeleteVersionAction
|
||||
}
|
||||
}
|
||||
|
||||
if obj.VersionID != "" && obj.ExpiredObjectDeleteMarker() {
|
||||
// From https: //docs.aws.amazon.com/AmazonS3/latest/dev/lifecycle-configuration-examples.html :
|
||||
// The NoncurrentVersionExpiration action in the same Lifecycle configuration removes noncurrent objects X days
|
||||
// after they become noncurrent. Thus, in this example, all object versions are permanently removed X days after
|
||||
// object creation. You will have expired object delete markers, but Amazon S3 detects and removes the expired
|
||||
// object delete markers for you.
|
||||
if time.Now().After(ExpectedExpiryTime(obj.ModTime, int(rule.NoncurrentVersionExpiration.NoncurrentDays))) {
|
||||
return DeleteVersionAction
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if !rule.NoncurrentVersionTransition.IsDaysNull() {
|
||||
if obj.VersionID != "" && !obj.IsLatest && !obj.SuccessorModTime.IsZero() && !obj.DeleteMarker && obj.TransitionStatus != TransitionComplete {
|
||||
// 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 time.Now().After(ExpectedExpiryTime(obj.SuccessorModTime, int(rule.NoncurrentVersionTransition.NoncurrentDays))) {
|
||||
return TransitionVersionAction
|
||||
}
|
||||
|
||||
// this if condition is strictly for debug purposes to force immediate
|
||||
// transition to remote tier if _MINIO_DEBUG_REMOTE_TIERS_IMMEDIATELY is set
|
||||
if doesMatchDebugTiers(rule.NoncurrentVersionTransition.StorageClass, obj.RemoteTiersImmediately) {
|
||||
return TransitionVersionAction
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Remove the object or simply add a delete marker (once) in a versioned bucket
|
||||
if obj.VersionID == "" || obj.IsLatest && !obj.DeleteMarker {
|
||||
switch {
|
||||
case !rule.Expiration.IsDateNull():
|
||||
if time.Now().UTC().After(rule.Expiration.Date.Time) {
|
||||
return DeleteAction
|
||||
}
|
||||
case !rule.Expiration.IsDaysNull():
|
||||
if time.Now().UTC().After(ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days))) {
|
||||
return DeleteAction
|
||||
}
|
||||
}
|
||||
|
||||
if obj.TransitionStatus != TransitionComplete {
|
||||
switch {
|
||||
case !rule.Transition.IsDateNull():
|
||||
if time.Now().UTC().After(rule.Transition.Date.Time) {
|
||||
action = TransitionAction
|
||||
}
|
||||
case !rule.Transition.IsDaysNull():
|
||||
if time.Now().UTC().After(ExpectedExpiryTime(obj.ModTime, int(rule.Transition.Days))) {
|
||||
action = TransitionAction
|
||||
}
|
||||
|
||||
}
|
||||
// this if condition is strictly for debug purposes to force immediate
|
||||
// transition to remote tier if _MINIO_DEBUG_REMOTE_TIERS_IMMEDIATELY is set
|
||||
if !rule.Transition.IsNull() && doesMatchDebugTiers(rule.Transition.StorageClass, obj.RemoteTiersImmediately) {
|
||||
action = TransitionAction
|
||||
}
|
||||
|
||||
if !obj.RestoreExpires.IsZero() && time.Now().After(obj.RestoreExpires) {
|
||||
if obj.VersionID != "" {
|
||||
action = DeleteRestoredVersionAction
|
||||
} else {
|
||||
action = DeleteRestoredAction
|
||||
}
|
||||
}
|
||||
}
|
||||
if !obj.RestoreExpires.IsZero() && time.Now().After(obj.RestoreExpires) {
|
||||
if obj.VersionID != "" {
|
||||
action = DeleteRestoredVersionAction
|
||||
} else {
|
||||
action = DeleteRestoredAction
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
return action
|
||||
}
|
||||
|
||||
// ExpectedExpiryTime calculates the expiry, transition or restore date/time based on a object modtime.
|
||||
// The expected transition or restore time is always a midnight time following the the object
|
||||
// modification time plus the number of transition/restore days.
|
||||
// e.g. If the object modtime is `Thu May 21 13:42:50 GMT 2020` and the object should
|
||||
// transition in 1 day, then the expected transition time is `Fri, 23 May 2020 00:00:00 GMT`
|
||||
func ExpectedExpiryTime(modTime time.Time, days int) time.Time {
|
||||
t := modTime.UTC().Add(time.Duration(days+1) * 24 * time.Hour)
|
||||
return t.Truncate(24 * time.Hour)
|
||||
}
|
||||
|
||||
// PredictExpiryTime returns the expiry date/time of a given object
|
||||
// after evaluating the current lifecycle document.
|
||||
func (lc Lifecycle) PredictExpiryTime(obj ObjectOpts) (string, time.Time) {
|
||||
if obj.DeleteMarker {
|
||||
// We don't need to send any x-amz-expiration for delete marker.
|
||||
return "", time.Time{}
|
||||
}
|
||||
|
||||
var finalExpiryDate time.Time
|
||||
var finalExpiryRuleID string
|
||||
|
||||
// Iterate over all actionable rules and find the earliest
|
||||
// expiration date and its associated rule ID.
|
||||
for _, rule := range lc.FilterActionableRules(obj) {
|
||||
if !rule.NoncurrentVersionExpiration.IsDaysNull() && !obj.IsLatest && obj.VersionID != "" {
|
||||
return rule.ID, ExpectedExpiryTime(obj.SuccessorModTime, int(rule.NoncurrentVersionExpiration.NoncurrentDays))
|
||||
}
|
||||
|
||||
if !rule.Expiration.IsDateNull() {
|
||||
if finalExpiryDate.IsZero() || finalExpiryDate.After(rule.Expiration.Date.Time) {
|
||||
finalExpiryRuleID = rule.ID
|
||||
finalExpiryDate = rule.Expiration.Date.Time
|
||||
}
|
||||
}
|
||||
if !rule.Expiration.IsDaysNull() {
|
||||
expectedExpiry := ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days))
|
||||
if finalExpiryDate.IsZero() || finalExpiryDate.After(expectedExpiry) {
|
||||
finalExpiryRuleID = rule.ID
|
||||
finalExpiryDate = expectedExpiry
|
||||
}
|
||||
}
|
||||
}
|
||||
return finalExpiryRuleID, finalExpiryDate
|
||||
}
|
||||
|
||||
// PredictTransitionTime returns the transition date/time of a given object
|
||||
// after evaluating the current lifecycle document.
|
||||
func (lc Lifecycle) PredictTransitionTime(obj ObjectOpts) (string, time.Time) {
|
||||
if obj.DeleteMarker {
|
||||
// We don't need to send any x-minio-transition for delete marker.
|
||||
return "", time.Time{}
|
||||
}
|
||||
|
||||
if obj.TransitionStatus == TransitionComplete {
|
||||
return "", time.Time{}
|
||||
}
|
||||
|
||||
var finalTransitionDate time.Time
|
||||
var finalTransitionRuleID string
|
||||
|
||||
// Iterate over all actionable rules and find the earliest
|
||||
// transition date and its associated rule ID.
|
||||
for _, rule := range lc.FilterActionableRules(obj) {
|
||||
switch {
|
||||
case !rule.Transition.IsDateNull():
|
||||
if finalTransitionDate.IsZero() || finalTransitionDate.After(rule.Transition.Date.Time) {
|
||||
finalTransitionRuleID = rule.ID
|
||||
finalTransitionDate = rule.Transition.Date.Time
|
||||
}
|
||||
case !rule.Transition.IsDaysNull():
|
||||
expectedTransition := ExpectedExpiryTime(obj.ModTime, int(rule.Expiration.Days))
|
||||
if finalTransitionDate.IsZero() || finalTransitionDate.After(expectedTransition) {
|
||||
finalTransitionRuleID = rule.ID
|
||||
finalTransitionDate = expectedTransition
|
||||
}
|
||||
}
|
||||
}
|
||||
return finalTransitionRuleID, finalTransitionDate
|
||||
}
|
||||
431
internal/bucket/lifecycle/lifecycle_test.go
Normal file
431
internal/bucket/lifecycle/lifecycle_test.go
Normal file
@@ -0,0 +1,431 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/xml"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestParseAndValidateLifecycleConfig(t *testing.T) {
|
||||
testCases := []struct {
|
||||
inputConfig string
|
||||
expectedParsingErr error
|
||||
expectedValidationErr error
|
||||
}{
|
||||
{ // Valid lifecycle config
|
||||
inputConfig: `<LifecycleConfiguration>
|
||||
<Rule>
|
||||
<ID>testRule1</ID>
|
||||
<Filter>
|
||||
<Prefix>prefix</Prefix>
|
||||
</Filter>
|
||||
<Status>Enabled</Status>
|
||||
<Expiration><Days>3</Days></Expiration>
|
||||
</Rule>
|
||||
<Rule>
|
||||
<ID>testRule2</ID>
|
||||
<Filter>
|
||||
<Prefix>another-prefix</Prefix>
|
||||
</Filter>
|
||||
<Status>Enabled</Status>
|
||||
<Expiration><Days>3</Days></Expiration>
|
||||
</Rule>
|
||||
</LifecycleConfiguration>`,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: nil,
|
||||
},
|
||||
{ // Valid lifecycle config
|
||||
inputConfig: `<LifecycleConfiguration>
|
||||
<Rule>
|
||||
<Filter>
|
||||
<And><Tag><Key>key1</Key><Value>val1</Value><Key>key2</Key><Value>val2</Value></Tag></And>
|
||||
</Filter>
|
||||
<Expiration><Days>3</Days></Expiration>
|
||||
</Rule>
|
||||
</LifecycleConfiguration>`,
|
||||
expectedParsingErr: errDuplicatedXMLTag,
|
||||
expectedValidationErr: nil,
|
||||
},
|
||||
{ // lifecycle config with no rules
|
||||
inputConfig: `<LifecycleConfiguration>
|
||||
</LifecycleConfiguration>`,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errLifecycleNoRule,
|
||||
},
|
||||
{ // lifecycle config with rules having overlapping prefix
|
||||
inputConfig: `<LifecycleConfiguration><Rule><ID>rule1</ID><Status>Enabled</Status><Filter><Prefix>/a/b</Prefix></Filter><Expiration><Days>3</Days></Expiration></Rule><Rule><ID>rule2</ID><Status>Enabled</Status><Filter><And><Prefix>/a/b/c</Prefix><Tag><Key>key1</Key><Value>val1</Value></Tag></And></Filter><Expiration><Days>3</Days></Expiration></Rule></LifecycleConfiguration> `,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: nil,
|
||||
},
|
||||
{ // lifecycle config with rules having duplicate ID
|
||||
inputConfig: `<LifecycleConfiguration><Rule><ID>duplicateID</ID><Status>Enabled</Status><Filter><Prefix>/a/b</Prefix></Filter><Expiration><Days>3</Days></Expiration></Rule><Rule><ID>duplicateID</ID><Status>Enabled</Status><Filter><And><Prefix>/x/z</Prefix><Tag><Key>key1</Key><Value>val1</Value></Tag></And></Filter><Expiration><Days>4</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errLifecycleDuplicateID,
|
||||
},
|
||||
// Missing <Tag> in <And>
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ID>sample-rule-2</ID><Filter><And><Prefix>/a/b/c</Prefix></And></Filter><Status>Enabled</Status><Expiration><Days>1</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errXMLNotWellFormed,
|
||||
},
|
||||
// Lifecycle with the deprecated Prefix tag
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ID>rule</ID><Prefix /><Status>Enabled</Status><Expiration><Days>1</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: nil,
|
||||
},
|
||||
// Lifecycle with empty Filter tag
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><ID>rule</ID><Filter></Filter><Status>Enabled</Status><Expiration><Days>1</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: nil,
|
||||
},
|
||||
}
|
||||
|
||||
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 != tc.expectedParsingErr {
|
||||
t.Fatalf("%d: Expected %v during parsing but got %v", i+1, tc.expectedParsingErr, err)
|
||||
}
|
||||
if tc.expectedParsingErr != nil {
|
||||
// We already expect a parsing error,
|
||||
// no need to continue this test.
|
||||
return
|
||||
}
|
||||
err = lc.Validate()
|
||||
if err != tc.expectedValidationErr {
|
||||
t.Fatalf("%d: Expected %v during parsing but got %v", i+1, tc.expectedValidationErr, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestMarshalLifecycleConfig checks if lifecycleconfig xml
|
||||
// marshaling/unmarshaling can handle output from each other
|
||||
func TestMarshalLifecycleConfig(t *testing.T) {
|
||||
// Time at midnight UTC
|
||||
midnightTS := ExpirationDate{time.Date(2019, time.April, 20, 0, 0, 0, 0, time.UTC)}
|
||||
lc := Lifecycle{
|
||||
Rules: []Rule{
|
||||
{
|
||||
Status: "Enabled",
|
||||
Filter: Filter{Prefix: Prefix{string: "prefix-1", set: true}},
|
||||
Expiration: Expiration{Days: ExpirationDays(3)},
|
||||
},
|
||||
{
|
||||
Status: "Enabled",
|
||||
Filter: Filter{Prefix: Prefix{string: "prefix-1", set: true}},
|
||||
Expiration: Expiration{Date: midnightTS},
|
||||
},
|
||||
{
|
||||
Status: "Enabled",
|
||||
Filter: Filter{Prefix: Prefix{string: "prefix-1", set: true}},
|
||||
Expiration: Expiration{Date: midnightTS},
|
||||
NoncurrentVersionTransition: NoncurrentVersionTransition{NoncurrentDays: 2, StorageClass: "TEST"},
|
||||
},
|
||||
},
|
||||
}
|
||||
b, err := xml.MarshalIndent(&lc, "", "\t")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
var lc1 Lifecycle
|
||||
err = xml.Unmarshal(b, &lc1)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ruleSet := make(map[string]struct{})
|
||||
for _, rule := range lc.Rules {
|
||||
ruleBytes, err := xml.Marshal(rule)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
ruleSet[string(ruleBytes)] = struct{}{}
|
||||
}
|
||||
for _, rule := range lc1.Rules {
|
||||
ruleBytes, err := xml.Marshal(rule)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if _, ok := ruleSet[string(ruleBytes)]; !ok {
|
||||
t.Fatalf("Expected %v to be equal to %v, %v missing", lc, lc1, rule)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestExpectedExpiryTime(t *testing.T) {
|
||||
testCases := []struct {
|
||||
modTime time.Time
|
||||
days ExpirationDays
|
||||
expected time.Time
|
||||
}{
|
||||
{
|
||||
time.Date(2020, time.March, 15, 10, 10, 10, 0, time.UTC),
|
||||
4,
|
||||
time.Date(2020, time.March, 20, 0, 0, 0, 0, time.UTC),
|
||||
},
|
||||
{
|
||||
time.Date(2020, time.March, 15, 0, 0, 0, 0, time.UTC),
|
||||
1,
|
||||
time.Date(2020, time.March, 17, 0, 0, 0, 0, time.UTC),
|
||||
},
|
||||
}
|
||||
|
||||
for i, tc := range testCases {
|
||||
t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) {
|
||||
got := ExpectedExpiryTime(tc.modTime, int(tc.days))
|
||||
if !got.Equal(tc.expected) {
|
||||
t.Fatalf("Expected %v to be equal to %v", got, tc.expected)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func TestComputeActions(t *testing.T) {
|
||||
testCases := []struct {
|
||||
inputConfig string
|
||||
objectName string
|
||||
objectTags 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,
|
||||
},
|
||||
// No modTime, should be none-action
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
objectName: "foodir/fooobject",
|
||||
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,
|
||||
},
|
||||
// Test rule with empty prefix e.g. for whole bucket
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix></Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
objectName: "foxdir/fooobject/foo.txt",
|
||||
objectModTime: time.Now().UTC().Add(-10 * 24 * time.Hour), // Created 10 days ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
// 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().UTC().Truncate(24*time.Hour).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().UTC().Truncate(24*time.Hour).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,
|
||||
},
|
||||
// Should remove (Tags match)
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><And><Prefix>foodir/</Prefix><Tag><Key>tag1</Key><Value>value1</Value></Tag></And></Filter><Status>Enabled</Status><Expiration><Date>` + time.Now().UTC().Truncate(24*time.Hour).Add(-24*time.Hour).Format(time.RFC3339) + `</Date></Expiration></Rule></LifecycleConfiguration>`,
|
||||
objectName: "foodir/fooobject",
|
||||
objectTags: "tag1=value1&tag2=value2",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
// Should remove (Multiple Rules, Tags match)
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><And><Prefix>foodir/</Prefix><Tag><Key>tag1</Key><Value>value1</Value></Tag><Tag><Key>tag2</Key><Value>value2</Value></Tag></And></Filter><Status>Enabled</Status><Expiration><Date>` + time.Now().Truncate(24*time.Hour).UTC().Add(-24*time.Hour).Format(time.RFC3339) + `</Date></Expiration></Rule><Rule><Filter><And><Prefix>abc/</Prefix><Tag><Key>tag2</Key><Value>value</Value></Tag></And></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",
|
||||
objectTags: "tag1=value1&tag2=value2",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
// Should remove (Tags match)
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><And><Prefix>foodir/</Prefix><Tag><Key>tag1</Key><Value>value1</Value></Tag><Tag><Key>tag2</Key><Value>value2</Value></Tag></And></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",
|
||||
objectTags: "tag1=value1&tag2=value2",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
// Should remove (Tags match with inverted order)
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><And><Tag><Key>factory</Key><Value>true</Value></Tag><Tag><Key>storeforever</Key><Value>false</Value></Tag></And></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: "fooobject",
|
||||
objectTags: "storeforever=false&factory=true",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
|
||||
// Should not remove (Tags don't match)
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><And><Prefix>foodir/</Prefix><Tag><Key>tag</Key><Value>value1</Value></Tag></And></Filter><Status>Enabled</Status><Expiration><Date>` + time.Now().UTC().Truncate(24*time.Hour).Add(-24*time.Hour).Format(time.RFC3339) + `</Date></Expiration></Rule></LifecycleConfiguration>`,
|
||||
objectName: "foodir/fooobject",
|
||||
objectTags: "tag1=value1",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: NoneAction,
|
||||
},
|
||||
// Should not remove (Tags match, but prefix doesn't match)
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><And><Prefix>foodir/</Prefix><Tag><Key>tag1</Key><Value>value1</Value></Tag></And></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: "foxdir/fooobject",
|
||||
objectTags: "tag1=value1",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: NoneAction,
|
||||
},
|
||||
// Should remove - empty prefix, tags match, date expiration kicked in
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><And><Tag><Key>tag1</Key><Value>value1</Value></Tag></And></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: "foxdir/fooobject",
|
||||
objectTags: "tag1=value1",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
// Should remove - empty prefix, tags match, object is expired based on specified Days
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><And><Prefix></Prefix><Tag><Key>tag1</Key><Value>value1</Value></Tag></And></Filter><Status>Enabled</Status><Expiration><Days>1</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
objectName: "foxdir/fooobject",
|
||||
objectTags: "tag1=value1",
|
||||
objectModTime: time.Now().UTC().Add(-48 * time.Hour), // Created 2 day ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
// Should remove, the second rule has expiration kicked in
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Status>Enabled</Status><Expiration><Date>` + time.Now().Truncate(24*time.Hour).UTC().Add(24*time.Hour).Format(time.RFC3339) + `</Date></Expiration></Rule><Rule><Filter><Prefix>foxdir/</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: "foxdir/fooobject",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
// Should accept BucketLifecycleConfiguration root tag
|
||||
{
|
||||
inputConfig: `<BucketLifecycleConfiguration><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></BucketLifecycleConfiguration>`,
|
||||
objectName: "foodir/fooobject",
|
||||
objectModTime: time.Now().UTC().Add(-24 * time.Hour), // Created 1 day ago
|
||||
expectedAction: DeleteAction,
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
tc := tc
|
||||
t.Run("", func(t *testing.T) {
|
||||
lc, err := ParseLifecycleConfig(bytes.NewReader([]byte(tc.inputConfig)))
|
||||
if err != nil {
|
||||
t.Fatalf("Got unexpected error: %v", err)
|
||||
}
|
||||
if resultAction := lc.ComputeAction(ObjectOpts{
|
||||
Name: tc.objectName,
|
||||
UserTags: tc.objectTags,
|
||||
ModTime: tc.objectModTime,
|
||||
IsLatest: true,
|
||||
}); resultAction != tc.expectedAction {
|
||||
t.Fatalf("Expected action: `%v`, got: `%v`", tc.expectedAction, resultAction)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
func TestHasActiveRules(t *testing.T) {
|
||||
testCases := []struct {
|
||||
inputConfig string
|
||||
prefix string
|
||||
expectedNonRec bool
|
||||
expectedRec bool
|
||||
}{
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
prefix: "foodir/foobject",
|
||||
expectedNonRec: true, expectedRec: true,
|
||||
},
|
||||
{ // empty prefix
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
prefix: "foodir/foobject/foo.txt",
|
||||
expectedNonRec: true, expectedRec: true,
|
||||
},
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
prefix: "zdir/foobject",
|
||||
expectedNonRec: false, expectedRec: false,
|
||||
},
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/zdir/</Prefix></Filter><Status>Enabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
prefix: "foodir/",
|
||||
expectedNonRec: false, expectedRec: true,
|
||||
},
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix></Prefix></Filter><Status>Disabled</Status><Expiration><Days>5</Days></Expiration></Rule></LifecycleConfiguration>`,
|
||||
prefix: "foodir/",
|
||||
expectedNonRec: false, expectedRec: false,
|
||||
},
|
||||
{
|
||||
inputConfig: `<LifecycleConfiguration><Rule><Filter><Prefix>foodir/</Prefix></Filter><Status>Enabled</Status><Expiration><Date>2999-01-01T00:00:00.000Z</Date></Expiration></Rule></LifecycleConfiguration>`,
|
||||
prefix: "foodir/foobject",
|
||||
expectedNonRec: false, expectedRec: false,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tc := range testCases {
|
||||
tc := tc
|
||||
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("Got unexpected error: %v", err)
|
||||
}
|
||||
if got := lc.HasActiveRules(tc.prefix, false); got != tc.expectedNonRec {
|
||||
t.Fatalf("Expected result with recursive set to false: `%v`, got: `%v`", tc.expectedNonRec, got)
|
||||
}
|
||||
if got := lc.HasActiveRules(tc.prefix, true); got != tc.expectedRec {
|
||||
t.Fatalf("Expected result with recursive set to true: `%v`, got: `%v`", tc.expectedRec, got)
|
||||
}
|
||||
|
||||
})
|
||||
|
||||
}
|
||||
}
|
||||
114
internal/bucket/lifecycle/noncurrentversion.go
Normal file
114
internal/bucket/lifecycle/noncurrentversion.go
Normal file
@@ -0,0 +1,114 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
)
|
||||
|
||||
// NoncurrentVersionExpiration - an action for lifecycle configuration rule.
|
||||
type NoncurrentVersionExpiration struct {
|
||||
XMLName xml.Name `xml:"NoncurrentVersionExpiration"`
|
||||
NoncurrentDays ExpirationDays `xml:"NoncurrentDays,omitempty"`
|
||||
set bool
|
||||
}
|
||||
|
||||
// MarshalXML if non-current days not set to non zero value
|
||||
func (n NoncurrentVersionExpiration) MarshalXML(e *xml.Encoder, start xml.StartElement) error {
|
||||
if n.IsDaysNull() {
|
||||
return nil
|
||||
}
|
||||
type noncurrentVersionExpirationWrapper NoncurrentVersionExpiration
|
||||
return e.EncodeElement(noncurrentVersionExpirationWrapper(n), start)
|
||||
}
|
||||
|
||||
// UnmarshalXML decodes NoncurrentVersionExpiration
|
||||
func (n *NoncurrentVersionExpiration) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
type noncurrentVersionExpirationWrapper NoncurrentVersionExpiration
|
||||
var val noncurrentVersionExpirationWrapper
|
||||
err := d.DecodeElement(&val, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*n = NoncurrentVersionExpiration(val)
|
||||
n.set = true
|
||||
return nil
|
||||
}
|
||||
|
||||
// IsDaysNull returns true if days field is null
|
||||
func (n NoncurrentVersionExpiration) IsDaysNull() bool {
|
||||
return n.NoncurrentDays == ExpirationDays(0)
|
||||
}
|
||||
|
||||
// Validate returns an error with wrong value
|
||||
func (n NoncurrentVersionExpiration) Validate() error {
|
||||
if !n.set {
|
||||
return nil
|
||||
}
|
||||
val := int(n.NoncurrentDays)
|
||||
if val <= 0 {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// NoncurrentVersionTransition - an action for lifecycle configuration rule.
|
||||
type NoncurrentVersionTransition struct {
|
||||
NoncurrentDays ExpirationDays `xml:"NoncurrentDays"`
|
||||
StorageClass string `xml:"StorageClass"`
|
||||
set bool
|
||||
}
|
||||
|
||||
// MarshalXML is extended to leave out
|
||||
// <NoncurrentVersionTransition></NoncurrentVersionTransition> tags
|
||||
func (n NoncurrentVersionTransition) MarshalXML(e *xml.Encoder, start xml.StartElement) error {
|
||||
if n.NoncurrentDays == ExpirationDays(0) {
|
||||
return nil
|
||||
}
|
||||
type noncurrentVersionTransitionWrapper NoncurrentVersionTransition
|
||||
return e.EncodeElement(noncurrentVersionTransitionWrapper(n), start)
|
||||
}
|
||||
|
||||
// IsDaysNull returns true if days field is null
|
||||
func (n NoncurrentVersionTransition) IsDaysNull() bool {
|
||||
return n.NoncurrentDays == ExpirationDays(0)
|
||||
}
|
||||
|
||||
// UnmarshalXML decodes NoncurrentVersionExpiration
|
||||
func (n *NoncurrentVersionTransition) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
type noncurrentVersionTransitionWrapper NoncurrentVersionTransition
|
||||
var val noncurrentVersionTransitionWrapper
|
||||
err := d.DecodeElement(&val, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*n = NoncurrentVersionTransition(val)
|
||||
n.set = true
|
||||
return nil
|
||||
}
|
||||
|
||||
// Validate returns an error with wrong value
|
||||
func (n NoncurrentVersionTransition) Validate() error {
|
||||
if !n.set {
|
||||
return nil
|
||||
}
|
||||
if int(n.NoncurrentDays) <= 0 || n.StorageClass == "" {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
return nil
|
||||
}
|
||||
51
internal/bucket/lifecycle/prefix.go
Normal file
51
internal/bucket/lifecycle/prefix.go
Normal file
@@ -0,0 +1,51 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
)
|
||||
|
||||
// Prefix holds the prefix xml tag in <Rule> and <Filter>
|
||||
type Prefix struct {
|
||||
string
|
||||
set bool
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (p *Prefix) UnmarshalXML(d *xml.Decoder, start xml.StartElement) (err error) {
|
||||
var s string
|
||||
if err = d.DecodeElement(&s, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
*p = Prefix{string: s, set: true}
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML - decodes XML data.
|
||||
func (p Prefix) MarshalXML(e *xml.Encoder, startElement xml.StartElement) error {
|
||||
if !p.set {
|
||||
return nil
|
||||
}
|
||||
return e.EncodeElement(p.string, startElement)
|
||||
}
|
||||
|
||||
// String returns the prefix string
|
||||
func (p Prefix) String() string {
|
||||
return p.string
|
||||
}
|
||||
186
internal/bucket/lifecycle/rule.go
Normal file
186
internal/bucket/lifecycle/rule.go
Normal file
@@ -0,0 +1,186 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/xml"
|
||||
|
||||
"github.com/google/uuid"
|
||||
)
|
||||
|
||||
// Status represents lifecycle configuration status
|
||||
type Status string
|
||||
|
||||
// Supported status types
|
||||
const (
|
||||
Enabled Status = "Enabled"
|
||||
Disabled Status = "Disabled"
|
||||
)
|
||||
|
||||
// Rule - a rule for lifecycle configuration.
|
||||
type Rule struct {
|
||||
XMLName xml.Name `xml:"Rule"`
|
||||
ID string `xml:"ID,omitempty"`
|
||||
Status Status `xml:"Status"`
|
||||
Filter Filter `xml:"Filter,omitempty"`
|
||||
Prefix Prefix `xml:"Prefix,omitempty"`
|
||||
Expiration Expiration `xml:"Expiration,omitempty"`
|
||||
Transition Transition `xml:"Transition,omitempty"`
|
||||
// FIXME: add a type to catch unsupported AbortIncompleteMultipartUpload AbortIncompleteMultipartUpload `xml:"AbortIncompleteMultipartUpload,omitempty"`
|
||||
NoncurrentVersionExpiration NoncurrentVersionExpiration `xml:"NoncurrentVersionExpiration,omitempty"`
|
||||
NoncurrentVersionTransition NoncurrentVersionTransition `xml:"NoncurrentVersionTransition,omitempty"`
|
||||
}
|
||||
|
||||
var (
|
||||
errInvalidRuleID = Errorf("ID length is limited to 255 characters")
|
||||
errEmptyRuleStatus = Errorf("Status should not be empty")
|
||||
errInvalidRuleStatus = Errorf("Status must be set to either Enabled or Disabled")
|
||||
)
|
||||
|
||||
// generates random UUID
|
||||
func getNewUUID() (string, error) {
|
||||
u, err := uuid.NewRandom()
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
return u.String(), nil
|
||||
}
|
||||
|
||||
// validateID - checks if ID is valid or not.
|
||||
func (r Rule) validateID() error {
|
||||
IDLen := len(r.ID)
|
||||
// generate new ID when not provided
|
||||
// cannot be longer than 255 characters
|
||||
if IDLen == 0 {
|
||||
if newID, err := getNewUUID(); err == nil {
|
||||
r.ID = newID
|
||||
} else {
|
||||
return err
|
||||
}
|
||||
} else if IDLen > 255 {
|
||||
return errInvalidRuleID
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// validateStatus - checks if status is valid or not.
|
||||
func (r Rule) validateStatus() error {
|
||||
// Status can't be empty
|
||||
if len(r.Status) == 0 {
|
||||
return errEmptyRuleStatus
|
||||
}
|
||||
|
||||
// Status must be one of Enabled or Disabled
|
||||
if r.Status != Enabled && r.Status != Disabled {
|
||||
return errInvalidRuleStatus
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r Rule) validateExpiration() error {
|
||||
return r.Expiration.Validate()
|
||||
}
|
||||
|
||||
func (r Rule) validateNoncurrentExpiration() error {
|
||||
return r.NoncurrentVersionExpiration.Validate()
|
||||
}
|
||||
|
||||
func (r Rule) validatePrefixAndFilter() error {
|
||||
if !r.Prefix.set && r.Filter.IsEmpty() || r.Prefix.set && !r.Filter.IsEmpty() {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
if !r.Prefix.set {
|
||||
return r.Filter.Validate()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r Rule) validateTransition() error {
|
||||
return r.Transition.Validate()
|
||||
}
|
||||
|
||||
func (r Rule) validateNoncurrentTransition() error {
|
||||
return r.NoncurrentVersionTransition.Validate()
|
||||
}
|
||||
|
||||
// GetPrefix - a rule can either have prefix under <rule></rule>, <filter></filter>
|
||||
// or under <filter><and></and></filter>. This method returns the prefix from the
|
||||
// location where it is available.
|
||||
func (r Rule) GetPrefix() string {
|
||||
if p := r.Prefix.String(); p != "" {
|
||||
return p
|
||||
}
|
||||
if p := r.Filter.Prefix.String(); p != "" {
|
||||
return p
|
||||
}
|
||||
if p := r.Filter.And.Prefix.String(); p != "" {
|
||||
return p
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// Tags - a rule can either have tag under <filter></filter> or under
|
||||
// <filter><and></and></filter>. This method returns all the tags from the
|
||||
// rule in the format tag1=value1&tag2=value2
|
||||
func (r Rule) Tags() string {
|
||||
if !r.Filter.Tag.IsEmpty() {
|
||||
return r.Filter.Tag.String()
|
||||
}
|
||||
if len(r.Filter.And.Tags) != 0 {
|
||||
var buf bytes.Buffer
|
||||
for _, t := range r.Filter.And.Tags {
|
||||
if buf.Len() > 0 {
|
||||
buf.WriteString("&")
|
||||
}
|
||||
buf.WriteString(t.String())
|
||||
}
|
||||
return buf.String()
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// Validate - validates the rule element
|
||||
func (r Rule) Validate() error {
|
||||
if err := r.validateID(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.validateStatus(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.validateExpiration(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.validateNoncurrentExpiration(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.validatePrefixAndFilter(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.validateTransition(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.validateNoncurrentTransition(); err != nil {
|
||||
return err
|
||||
}
|
||||
if !r.Expiration.set && !r.Transition.set && !r.NoncurrentVersionExpiration.set && !r.NoncurrentVersionTransition.set {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
return nil
|
||||
}
|
||||
79
internal/bucket/lifecycle/rule_test.go
Normal file
79
internal/bucket/lifecycle/rule_test.go
Normal file
@@ -0,0 +1,79 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"fmt"
|
||||
"testing"
|
||||
)
|
||||
|
||||
// TestInvalidRules checks if Rule xml with invalid elements returns
|
||||
// appropriate errors on validation
|
||||
func TestInvalidRules(t *testing.T) {
|
||||
invalidTestCases := []struct {
|
||||
inputXML string
|
||||
expectedErr error
|
||||
}{
|
||||
{ // Rule with ID longer than 255 characters
|
||||
inputXML: ` <Rule>
|
||||
<ID> babababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababababab </ID>
|
||||
</Rule>`,
|
||||
expectedErr: errInvalidRuleID,
|
||||
},
|
||||
{ // Rule with empty ID
|
||||
inputXML: `<Rule>
|
||||
<ID></ID>
|
||||
<Filter><Prefix></Prefix></Filter>
|
||||
<Expiration>
|
||||
<Days>365</Days>
|
||||
</Expiration>
|
||||
<Status>Enabled</Status>
|
||||
</Rule>`,
|
||||
expectedErr: nil,
|
||||
},
|
||||
{ // Rule with empty status
|
||||
inputXML: ` <Rule>
|
||||
<ID>rule with empty status</ID>
|
||||
<Status></Status>
|
||||
</Rule>`,
|
||||
expectedErr: errEmptyRuleStatus,
|
||||
},
|
||||
{ // Rule with invalid status
|
||||
inputXML: ` <Rule>
|
||||
<ID>rule with invalid status</ID>
|
||||
<Status>OK</Status>
|
||||
</Rule>`,
|
||||
expectedErr: errInvalidRuleStatus,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tc := range invalidTestCases {
|
||||
t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) {
|
||||
var rule Rule
|
||||
err := xml.Unmarshal([]byte(tc.inputXML), &rule)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
if err := rule.Validate(); err != tc.expectedErr {
|
||||
t.Fatalf("%d: Expected %v but got %v", i+1, tc.expectedErr, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
102
internal/bucket/lifecycle/tag.go
Normal file
102
internal/bucket/lifecycle/tag.go
Normal file
@@ -0,0 +1,102 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"io"
|
||||
"unicode/utf8"
|
||||
)
|
||||
|
||||
// Tag - a tag for a lifecycle configuration Rule filter.
|
||||
type Tag struct {
|
||||
XMLName xml.Name `xml:"Tag"`
|
||||
Key string `xml:"Key,omitempty"`
|
||||
Value string `xml:"Value,omitempty"`
|
||||
}
|
||||
|
||||
var (
|
||||
errInvalidTagKey = Errorf("The TagKey you have provided is invalid")
|
||||
errInvalidTagValue = Errorf("The TagValue you have provided is invalid")
|
||||
|
||||
errDuplicatedXMLTag = Errorf("duplicated XML Tag")
|
||||
errUnknownXMLTag = Errorf("unknown XML Tag")
|
||||
)
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (tag *Tag) UnmarshalXML(d *xml.Decoder, start xml.StartElement) (err error) {
|
||||
var keyAlreadyParsed, valueAlreadyParsed bool
|
||||
for {
|
||||
// Read tokens from the XML document in a stream.
|
||||
t, err := d.Token()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
switch se := t.(type) {
|
||||
case xml.StartElement:
|
||||
var s string
|
||||
if err = d.DecodeElement(&s, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
switch se.Name.Local {
|
||||
case "Key":
|
||||
if keyAlreadyParsed {
|
||||
return errDuplicatedXMLTag
|
||||
}
|
||||
tag.Key = s
|
||||
keyAlreadyParsed = true
|
||||
case "Value":
|
||||
if valueAlreadyParsed {
|
||||
return errDuplicatedXMLTag
|
||||
}
|
||||
tag.Value = s
|
||||
valueAlreadyParsed = true
|
||||
default:
|
||||
return errUnknownXMLTag
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (tag Tag) String() string {
|
||||
return tag.Key + "=" + tag.Value
|
||||
}
|
||||
|
||||
// IsEmpty returns whether this tag is empty or not.
|
||||
func (tag Tag) IsEmpty() bool {
|
||||
return tag.Key == ""
|
||||
}
|
||||
|
||||
// Validate checks this tag.
|
||||
func (tag Tag) Validate() error {
|
||||
if len(tag.Key) == 0 || utf8.RuneCountInString(tag.Key) > 128 {
|
||||
return errInvalidTagKey
|
||||
}
|
||||
|
||||
if utf8.RuneCountInString(tag.Value) > 256 {
|
||||
return errInvalidTagValue
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
165
internal/bucket/lifecycle/transition.go
Normal file
165
internal/bucket/lifecycle/transition.go
Normal file
@@ -0,0 +1,165 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lifecycle
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"time"
|
||||
)
|
||||
|
||||
var (
|
||||
errTransitionInvalidDays = Errorf("Days must be 0 or greater when used with Transition")
|
||||
errTransitionInvalidDate = Errorf("Date must be provided in ISO 8601 format")
|
||||
errTransitionInvalid = Errorf("Exactly one of Days (0 or greater) or Date (positive ISO 8601 format) should be present inside Expiration.")
|
||||
errTransitionDateNotMidnight = Errorf("'Date' must be at midnight GMT")
|
||||
)
|
||||
|
||||
// TransitionDate is a embedded type containing time.Time to unmarshal
|
||||
// Date in Transition
|
||||
type TransitionDate struct {
|
||||
time.Time
|
||||
}
|
||||
|
||||
// UnmarshalXML parses date from Transition and validates date format
|
||||
func (tDate *TransitionDate) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
var dateStr string
|
||||
err := d.DecodeElement(&dateStr, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// While AWS documentation mentions that the date specified
|
||||
// must be present in ISO 8601 format, in reality they allow
|
||||
// users to provide RFC 3339 compliant dates.
|
||||
trnDate, err := time.Parse(time.RFC3339, dateStr)
|
||||
if err != nil {
|
||||
return errTransitionInvalidDate
|
||||
}
|
||||
// Allow only date timestamp specifying midnight GMT
|
||||
hr, min, sec := trnDate.Clock()
|
||||
nsec := trnDate.Nanosecond()
|
||||
loc := trnDate.Location()
|
||||
if !(hr == 0 && min == 0 && sec == 0 && nsec == 0 && loc.String() == time.UTC.String()) {
|
||||
return errTransitionDateNotMidnight
|
||||
}
|
||||
|
||||
*tDate = TransitionDate{trnDate}
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML encodes expiration date if it is non-zero and encodes
|
||||
// empty string otherwise
|
||||
func (tDate TransitionDate) MarshalXML(e *xml.Encoder, startElement xml.StartElement) error {
|
||||
if tDate.Time.IsZero() {
|
||||
return nil
|
||||
}
|
||||
return e.EncodeElement(tDate.Format(time.RFC3339), startElement)
|
||||
}
|
||||
|
||||
// TransitionDays is a type alias to unmarshal Days in Transition
|
||||
type TransitionDays int
|
||||
|
||||
// UnmarshalXML parses number of days from Transition and validates if
|
||||
// >= 0
|
||||
func (tDays *TransitionDays) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
var numDays int
|
||||
err := d.DecodeElement(&numDays, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if numDays < 0 {
|
||||
return errTransitionInvalidDays
|
||||
}
|
||||
*tDays = TransitionDays(numDays)
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML encodes number of days to expire if it is non-zero and
|
||||
// encodes empty string otherwise
|
||||
func (tDays TransitionDays) MarshalXML(e *xml.Encoder, startElement xml.StartElement) error {
|
||||
if tDays == 0 {
|
||||
return nil
|
||||
}
|
||||
return e.EncodeElement(int(tDays), startElement)
|
||||
}
|
||||
|
||||
// Transition - transition actions for a rule in lifecycle configuration.
|
||||
type Transition struct {
|
||||
XMLName xml.Name `xml:"Transition"`
|
||||
Days TransitionDays `xml:"Days,omitempty"`
|
||||
Date TransitionDate `xml:"Date,omitempty"`
|
||||
StorageClass string `xml:"StorageClass,omitempty"`
|
||||
|
||||
set bool
|
||||
}
|
||||
|
||||
// MarshalXML encodes transition field into an XML form.
|
||||
func (t Transition) MarshalXML(enc *xml.Encoder, start xml.StartElement) error {
|
||||
if !t.set {
|
||||
return nil
|
||||
}
|
||||
type transitionWrapper Transition
|
||||
return enc.EncodeElement(transitionWrapper(t), start)
|
||||
}
|
||||
|
||||
// UnmarshalXML decodes transition field from the XML form.
|
||||
func (t *Transition) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
type transitionWrapper Transition
|
||||
var trw transitionWrapper
|
||||
err := d.DecodeElement(&trw, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*t = Transition(trw)
|
||||
t.set = true
|
||||
return nil
|
||||
}
|
||||
|
||||
// Validate - validates the "Transition" element
|
||||
func (t Transition) Validate() error {
|
||||
if !t.set {
|
||||
return nil
|
||||
}
|
||||
|
||||
if t.IsDaysNull() && t.IsDateNull() {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
|
||||
// Both transition days and date are specified
|
||||
if !t.IsDaysNull() && !t.IsDateNull() {
|
||||
return errTransitionInvalid
|
||||
}
|
||||
if t.StorageClass == "" {
|
||||
return errXMLNotWellFormed
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// IsDaysNull returns true if days field is null
|
||||
func (t Transition) IsDaysNull() bool {
|
||||
return t.Days == TransitionDays(0)
|
||||
}
|
||||
|
||||
// IsDateNull returns true if date field is null
|
||||
func (t Transition) IsDateNull() bool {
|
||||
return t.Date.Time.IsZero()
|
||||
}
|
||||
|
||||
// IsNull returns true if both date and days fields are null
|
||||
func (t Transition) IsNull() bool {
|
||||
return t.IsDaysNull() && t.IsDateNull()
|
||||
}
|
||||
576
internal/bucket/object/lock/lock.go
Normal file
576
internal/bucket/object/lock/lock.go
Normal file
@@ -0,0 +1,576 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lock
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/xml"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"net/http"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/beevik/ntp"
|
||||
"github.com/minio/minio/internal/logger"
|
||||
"github.com/minio/pkg/env"
|
||||
)
|
||||
|
||||
// RetMode - object retention mode.
|
||||
type RetMode string
|
||||
|
||||
const (
|
||||
// RetGovernance - governance mode.
|
||||
RetGovernance RetMode = "GOVERNANCE"
|
||||
|
||||
// RetCompliance - compliance mode.
|
||||
RetCompliance RetMode = "COMPLIANCE"
|
||||
)
|
||||
|
||||
// Valid - returns if retention mode is valid
|
||||
func (r RetMode) Valid() bool {
|
||||
switch r {
|
||||
case RetGovernance, RetCompliance:
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func parseRetMode(modeStr string) (mode RetMode) {
|
||||
switch strings.ToUpper(modeStr) {
|
||||
case "GOVERNANCE":
|
||||
mode = RetGovernance
|
||||
case "COMPLIANCE":
|
||||
mode = RetCompliance
|
||||
}
|
||||
return mode
|
||||
}
|
||||
|
||||
// LegalHoldStatus - object legal hold status.
|
||||
type LegalHoldStatus string
|
||||
|
||||
const (
|
||||
// LegalHoldOn - legal hold is on.
|
||||
LegalHoldOn LegalHoldStatus = "ON"
|
||||
|
||||
// LegalHoldOff - legal hold is off.
|
||||
LegalHoldOff LegalHoldStatus = "OFF"
|
||||
)
|
||||
|
||||
// Valid - returns true if legal hold status has valid values
|
||||
func (l LegalHoldStatus) Valid() bool {
|
||||
switch l {
|
||||
case LegalHoldOn, LegalHoldOff:
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func parseLegalHoldStatus(holdStr string) (st LegalHoldStatus) {
|
||||
switch strings.ToUpper(holdStr) {
|
||||
case "ON":
|
||||
st = LegalHoldOn
|
||||
case "OFF":
|
||||
st = LegalHoldOff
|
||||
}
|
||||
return st
|
||||
}
|
||||
|
||||
// Bypass retention governance header.
|
||||
const (
|
||||
AmzObjectLockBypassRetGovernance = "X-Amz-Bypass-Governance-Retention"
|
||||
AmzObjectLockRetainUntilDate = "X-Amz-Object-Lock-Retain-Until-Date"
|
||||
AmzObjectLockMode = "X-Amz-Object-Lock-Mode"
|
||||
AmzObjectLockLegalHold = "X-Amz-Object-Lock-Legal-Hold"
|
||||
)
|
||||
|
||||
var (
|
||||
// ErrMalformedBucketObjectConfig -indicates that the bucket object lock config is malformed
|
||||
ErrMalformedBucketObjectConfig = errors.New("invalid bucket object lock config")
|
||||
// ErrInvalidRetentionDate - indicates that retention date needs to be in ISO 8601 format
|
||||
ErrInvalidRetentionDate = errors.New("date must be provided in ISO 8601 format")
|
||||
// ErrPastObjectLockRetainDate - indicates that retention date must be in the future
|
||||
ErrPastObjectLockRetainDate = errors.New("the retain until date must be in the future")
|
||||
// ErrUnknownWORMModeDirective - indicates that the retention mode is invalid
|
||||
ErrUnknownWORMModeDirective = errors.New("unknown WORM mode directive")
|
||||
// ErrObjectLockMissingContentMD5 - indicates missing Content-MD5 header for put object requests with locking
|
||||
ErrObjectLockMissingContentMD5 = errors.New("content-MD5 HTTP header is required for Put Object requests with Object Lock parameters")
|
||||
// ErrObjectLockInvalidHeaders indicates that object lock headers are missing
|
||||
ErrObjectLockInvalidHeaders = errors.New("x-amz-object-lock-retain-until-date and x-amz-object-lock-mode must both be supplied")
|
||||
// ErrMalformedXML - generic error indicating malformed XML
|
||||
ErrMalformedXML = errors.New("the XML you provided was not well-formed or did not validate against our published schema")
|
||||
)
|
||||
|
||||
const (
|
||||
ntpServerEnv = "MINIO_NTP_SERVER"
|
||||
)
|
||||
|
||||
var (
|
||||
ntpServer = env.Get(ntpServerEnv, "")
|
||||
)
|
||||
|
||||
// UTCNowNTP - is similar in functionality to UTCNow()
|
||||
// but only used when we do not wish to rely on system
|
||||
// time.
|
||||
func UTCNowNTP() (time.Time, error) {
|
||||
// ntp server is disabled
|
||||
if ntpServer == "" {
|
||||
return time.Now().UTC(), nil
|
||||
}
|
||||
return ntp.Time(ntpServer)
|
||||
}
|
||||
|
||||
// Retention - bucket level retention configuration.
|
||||
type Retention struct {
|
||||
Mode RetMode
|
||||
Validity time.Duration
|
||||
LockEnabled bool
|
||||
}
|
||||
|
||||
// Retain - check whether given date is retainable by validity time.
|
||||
func (r Retention) Retain(created time.Time) bool {
|
||||
t, err := UTCNowNTP()
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
// Retain
|
||||
return true
|
||||
}
|
||||
return created.Add(r.Validity).After(t)
|
||||
}
|
||||
|
||||
// DefaultRetention - default retention configuration.
|
||||
type DefaultRetention struct {
|
||||
XMLName xml.Name `xml:"DefaultRetention"`
|
||||
Mode RetMode `xml:"Mode"`
|
||||
Days *uint64 `xml:"Days"`
|
||||
Years *uint64 `xml:"Years"`
|
||||
}
|
||||
|
||||
// Maximum support retention days and years supported by AWS S3.
|
||||
const (
|
||||
// This tested by using `mc lock` command
|
||||
maximumRetentionDays = 36500
|
||||
maximumRetentionYears = 100
|
||||
)
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (dr *DefaultRetention) UnmarshalXML(d *xml.Decoder, start xml.StartElement) error {
|
||||
// Make subtype to avoid recursive UnmarshalXML().
|
||||
type defaultRetention DefaultRetention
|
||||
retention := defaultRetention{}
|
||||
|
||||
if err := d.DecodeElement(&retention, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
switch retention.Mode {
|
||||
case RetGovernance, RetCompliance:
|
||||
default:
|
||||
return fmt.Errorf("unknown retention mode %v", retention.Mode)
|
||||
}
|
||||
|
||||
if retention.Days == nil && retention.Years == nil {
|
||||
return fmt.Errorf("either Days or Years must be specified")
|
||||
}
|
||||
|
||||
if retention.Days != nil && retention.Years != nil {
|
||||
return fmt.Errorf("either Days or Years must be specified, not both")
|
||||
}
|
||||
|
||||
if retention.Days != nil {
|
||||
if *retention.Days == 0 {
|
||||
return fmt.Errorf("Default retention period must be a positive integer value for 'Days'")
|
||||
}
|
||||
if *retention.Days > maximumRetentionDays {
|
||||
return fmt.Errorf("Default retention period too large for 'Days' %d", *retention.Days)
|
||||
}
|
||||
} else if *retention.Years == 0 {
|
||||
return fmt.Errorf("Default retention period must be a positive integer value for 'Years'")
|
||||
} else if *retention.Years > maximumRetentionYears {
|
||||
return fmt.Errorf("Default retention period too large for 'Years' %d", *retention.Years)
|
||||
}
|
||||
|
||||
*dr = DefaultRetention(retention)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Config - object lock configuration specified in
|
||||
// https://docs.aws.amazon.com/AmazonS3/latest/API/Type_API_ObjectLockConfiguration.html
|
||||
type Config struct {
|
||||
XMLNS string `xml:"xmlns,attr,omitempty"`
|
||||
XMLName xml.Name `xml:"ObjectLockConfiguration"`
|
||||
ObjectLockEnabled string `xml:"ObjectLockEnabled"`
|
||||
Rule *struct {
|
||||
DefaultRetention DefaultRetention `xml:"DefaultRetention"`
|
||||
} `xml:"Rule,omitempty"`
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (config *Config) UnmarshalXML(d *xml.Decoder, start xml.StartElement) error {
|
||||
// Make subtype to avoid recursive UnmarshalXML().
|
||||
type objectLockConfig Config
|
||||
parsedConfig := objectLockConfig{}
|
||||
|
||||
if err := d.DecodeElement(&parsedConfig, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if parsedConfig.ObjectLockEnabled != "Enabled" {
|
||||
return fmt.Errorf("only 'Enabled' value is allowed to ObjectLockEnabled element")
|
||||
}
|
||||
|
||||
*config = Config(parsedConfig)
|
||||
return nil
|
||||
}
|
||||
|
||||
// ToRetention - convert to Retention type.
|
||||
func (config *Config) ToRetention() Retention {
|
||||
r := Retention{
|
||||
LockEnabled: config.ObjectLockEnabled == "Enabled",
|
||||
}
|
||||
if config.Rule != nil {
|
||||
r.Mode = config.Rule.DefaultRetention.Mode
|
||||
|
||||
t, err := UTCNowNTP()
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
// Do not change any configuration
|
||||
// upon NTP failure.
|
||||
return r
|
||||
}
|
||||
|
||||
if config.Rule.DefaultRetention.Days != nil {
|
||||
r.Validity = t.AddDate(0, 0, int(*config.Rule.DefaultRetention.Days)).Sub(t)
|
||||
} else {
|
||||
r.Validity = t.AddDate(int(*config.Rule.DefaultRetention.Years), 0, 0).Sub(t)
|
||||
}
|
||||
}
|
||||
|
||||
return r
|
||||
}
|
||||
|
||||
// Maximum 4KiB size per object lock config.
|
||||
const maxObjectLockConfigSize = 1 << 12
|
||||
|
||||
// ParseObjectLockConfig parses ObjectLockConfig from xml
|
||||
func ParseObjectLockConfig(reader io.Reader) (*Config, error) {
|
||||
config := Config{}
|
||||
if err := xml.NewDecoder(io.LimitReader(reader, maxObjectLockConfigSize)).Decode(&config); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &config, nil
|
||||
}
|
||||
|
||||
// NewObjectLockConfig returns a initialized lock.Config struct
|
||||
func NewObjectLockConfig() *Config {
|
||||
return &Config{
|
||||
ObjectLockEnabled: "Enabled",
|
||||
}
|
||||
}
|
||||
|
||||
// RetentionDate is a embedded type containing time.Time to unmarshal
|
||||
// Date in Retention
|
||||
type RetentionDate struct {
|
||||
time.Time
|
||||
}
|
||||
|
||||
// UnmarshalXML parses date from Retention and validates date format
|
||||
func (rDate *RetentionDate) UnmarshalXML(d *xml.Decoder, startElement xml.StartElement) error {
|
||||
var dateStr string
|
||||
err := d.DecodeElement(&dateStr, &startElement)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
// While AWS documentation mentions that the date specified
|
||||
// must be present in ISO 8601 format, in reality they allow
|
||||
// users to provide RFC 3339 compliant dates.
|
||||
retDate, err := time.Parse(time.RFC3339, dateStr)
|
||||
if err != nil {
|
||||
return ErrInvalidRetentionDate
|
||||
}
|
||||
|
||||
*rDate = RetentionDate{retDate}
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML encodes expiration date if it is non-zero and encodes
|
||||
// empty string otherwise
|
||||
func (rDate *RetentionDate) MarshalXML(e *xml.Encoder, startElement xml.StartElement) error {
|
||||
if *rDate == (RetentionDate{time.Time{}}) {
|
||||
return nil
|
||||
}
|
||||
return e.EncodeElement(rDate.Format(time.RFC3339), startElement)
|
||||
}
|
||||
|
||||
// ObjectRetention specified in
|
||||
// https://docs.aws.amazon.com/AmazonS3/latest/API/API_PutObjectRetention.html
|
||||
type ObjectRetention struct {
|
||||
XMLNS string `xml:"xmlns,attr,omitempty"`
|
||||
XMLName xml.Name `xml:"Retention"`
|
||||
Mode RetMode `xml:"Mode,omitempty"`
|
||||
RetainUntilDate RetentionDate `xml:"RetainUntilDate,omitempty"`
|
||||
}
|
||||
|
||||
// Maximum 4KiB size per object retention config.
|
||||
const maxObjectRetentionSize = 1 << 12
|
||||
|
||||
// ParseObjectRetention constructs ObjectRetention struct from xml input
|
||||
func ParseObjectRetention(reader io.Reader) (*ObjectRetention, error) {
|
||||
ret := ObjectRetention{}
|
||||
if err := xml.NewDecoder(io.LimitReader(reader, maxObjectRetentionSize)).Decode(&ret); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if ret.Mode != "" && !ret.Mode.Valid() {
|
||||
return &ret, ErrUnknownWORMModeDirective
|
||||
}
|
||||
|
||||
if ret.Mode.Valid() && ret.RetainUntilDate.IsZero() {
|
||||
return &ret, ErrMalformedXML
|
||||
}
|
||||
|
||||
if !ret.Mode.Valid() && !ret.RetainUntilDate.IsZero() {
|
||||
return &ret, ErrMalformedXML
|
||||
}
|
||||
|
||||
t, err := UTCNowNTP()
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return &ret, ErrPastObjectLockRetainDate
|
||||
}
|
||||
|
||||
if !ret.RetainUntilDate.IsZero() && ret.RetainUntilDate.Before(t) {
|
||||
return &ret, ErrPastObjectLockRetainDate
|
||||
}
|
||||
|
||||
return &ret, nil
|
||||
}
|
||||
|
||||
// IsObjectLockRetentionRequested returns true if object lock retention headers are set.
|
||||
func IsObjectLockRetentionRequested(h http.Header) bool {
|
||||
if _, ok := h[AmzObjectLockMode]; ok {
|
||||
return true
|
||||
}
|
||||
if _, ok := h[AmzObjectLockRetainUntilDate]; ok {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// IsObjectLockLegalHoldRequested returns true if object lock legal hold header is set.
|
||||
func IsObjectLockLegalHoldRequested(h http.Header) bool {
|
||||
_, ok := h[AmzObjectLockLegalHold]
|
||||
return ok
|
||||
}
|
||||
|
||||
// IsObjectLockGovernanceBypassSet returns true if object lock governance bypass header is set.
|
||||
func IsObjectLockGovernanceBypassSet(h http.Header) bool {
|
||||
return strings.ToLower(h.Get(AmzObjectLockBypassRetGovernance)) == "true"
|
||||
}
|
||||
|
||||
// IsObjectLockRequested returns true if legal hold or object lock retention headers are requested.
|
||||
func IsObjectLockRequested(h http.Header) bool {
|
||||
return IsObjectLockLegalHoldRequested(h) || IsObjectLockRetentionRequested(h)
|
||||
}
|
||||
|
||||
// ParseObjectLockRetentionHeaders parses http headers to extract retention mode and retention date
|
||||
func ParseObjectLockRetentionHeaders(h http.Header) (rmode RetMode, r RetentionDate, err error) {
|
||||
retMode := h.Get(AmzObjectLockMode)
|
||||
dateStr := h.Get(AmzObjectLockRetainUntilDate)
|
||||
if len(retMode) == 0 || len(dateStr) == 0 {
|
||||
return rmode, r, ErrObjectLockInvalidHeaders
|
||||
}
|
||||
|
||||
rmode = parseRetMode(retMode)
|
||||
if !rmode.Valid() {
|
||||
return rmode, r, ErrUnknownWORMModeDirective
|
||||
}
|
||||
|
||||
var retDate time.Time
|
||||
// While AWS documentation mentions that the date specified
|
||||
// must be present in ISO 8601 format, in reality they allow
|
||||
// users to provide RFC 3339 compliant dates.
|
||||
retDate, err = time.Parse(time.RFC3339, dateStr)
|
||||
if err != nil {
|
||||
return rmode, r, ErrInvalidRetentionDate
|
||||
}
|
||||
|
||||
t, err := UTCNowNTP()
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return rmode, r, ErrPastObjectLockRetainDate
|
||||
}
|
||||
|
||||
if retDate.Before(t) {
|
||||
return rmode, r, ErrPastObjectLockRetainDate
|
||||
}
|
||||
|
||||
return rmode, RetentionDate{retDate}, nil
|
||||
|
||||
}
|
||||
|
||||
// GetObjectRetentionMeta constructs ObjectRetention from metadata
|
||||
func GetObjectRetentionMeta(meta map[string]string) ObjectRetention {
|
||||
var mode RetMode
|
||||
var retainTill RetentionDate
|
||||
|
||||
var modeStr, tillStr string
|
||||
ok := false
|
||||
|
||||
modeStr, ok = meta[strings.ToLower(AmzObjectLockMode)]
|
||||
if !ok {
|
||||
modeStr, ok = meta[AmzObjectLockMode]
|
||||
}
|
||||
if ok {
|
||||
mode = parseRetMode(modeStr)
|
||||
} else {
|
||||
return ObjectRetention{}
|
||||
}
|
||||
|
||||
tillStr, ok = meta[strings.ToLower(AmzObjectLockRetainUntilDate)]
|
||||
if !ok {
|
||||
tillStr, ok = meta[AmzObjectLockRetainUntilDate]
|
||||
}
|
||||
if ok {
|
||||
if t, e := time.Parse(time.RFC3339, tillStr); e == nil {
|
||||
retainTill = RetentionDate{t.UTC()}
|
||||
}
|
||||
}
|
||||
return ObjectRetention{XMLNS: "http://s3.amazonaws.com/doc/2006-03-01/", Mode: mode, RetainUntilDate: retainTill}
|
||||
}
|
||||
|
||||
// GetObjectLegalHoldMeta constructs ObjectLegalHold from metadata
|
||||
func GetObjectLegalHoldMeta(meta map[string]string) ObjectLegalHold {
|
||||
holdStr, ok := meta[strings.ToLower(AmzObjectLockLegalHold)]
|
||||
if !ok {
|
||||
holdStr, ok = meta[AmzObjectLockLegalHold]
|
||||
}
|
||||
if ok {
|
||||
return ObjectLegalHold{XMLNS: "http://s3.amazonaws.com/doc/2006-03-01/", Status: parseLegalHoldStatus(holdStr)}
|
||||
}
|
||||
return ObjectLegalHold{}
|
||||
}
|
||||
|
||||
// ParseObjectLockLegalHoldHeaders parses request headers to construct ObjectLegalHold
|
||||
func ParseObjectLockLegalHoldHeaders(h http.Header) (lhold ObjectLegalHold, err error) {
|
||||
holdStatus, ok := h[AmzObjectLockLegalHold]
|
||||
if ok {
|
||||
lh := parseLegalHoldStatus(holdStatus[0])
|
||||
if !lh.Valid() {
|
||||
return lhold, ErrUnknownWORMModeDirective
|
||||
}
|
||||
lhold = ObjectLegalHold{XMLNS: "http://s3.amazonaws.com/doc/2006-03-01/", Status: lh}
|
||||
}
|
||||
return lhold, nil
|
||||
|
||||
}
|
||||
|
||||
// ObjectLegalHold specified in
|
||||
// https://docs.aws.amazon.com/AmazonS3/latest/API/API_PutObjectLegalHold.html
|
||||
type ObjectLegalHold struct {
|
||||
XMLNS string `xml:"xmlns,attr,omitempty"`
|
||||
XMLName xml.Name `xml:"LegalHold"`
|
||||
Status LegalHoldStatus `xml:"Status,omitempty"`
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (l *ObjectLegalHold) UnmarshalXML(d *xml.Decoder, start xml.StartElement) (err error) {
|
||||
switch start.Name.Local {
|
||||
case "LegalHold", "ObjectLockLegalHold":
|
||||
default:
|
||||
return xml.UnmarshalError(fmt.Sprintf("expected element type <LegalHold>/<ObjectLockLegalHold> but have <%s>",
|
||||
start.Name.Local))
|
||||
}
|
||||
for {
|
||||
// Read tokens from the XML document in a stream.
|
||||
t, err := d.Token()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
switch se := t.(type) {
|
||||
case xml.StartElement:
|
||||
switch se.Name.Local {
|
||||
case "Status":
|
||||
var st LegalHoldStatus
|
||||
if err = d.DecodeElement(&st, &se); err != nil {
|
||||
return err
|
||||
}
|
||||
l.Status = st
|
||||
default:
|
||||
return xml.UnmarshalError(fmt.Sprintf("expected element type <Status> but have <%s>", se.Name.Local))
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// IsEmpty returns true if struct is empty
|
||||
func (l *ObjectLegalHold) IsEmpty() bool {
|
||||
return !l.Status.Valid()
|
||||
}
|
||||
|
||||
// ParseObjectLegalHold decodes the XML into ObjectLegalHold
|
||||
func ParseObjectLegalHold(reader io.Reader) (hold *ObjectLegalHold, err error) {
|
||||
hold = &ObjectLegalHold{}
|
||||
if err = xml.NewDecoder(reader).Decode(hold); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if !hold.Status.Valid() {
|
||||
return nil, ErrMalformedXML
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// FilterObjectLockMetadata filters object lock metadata if s3:GetObjectRetention permission is denied or if isCopy flag set.
|
||||
func FilterObjectLockMetadata(metadata map[string]string, filterRetention, filterLegalHold bool) map[string]string {
|
||||
// Copy on write
|
||||
dst := metadata
|
||||
var copied bool
|
||||
delKey := func(key string) {
|
||||
if _, ok := metadata[key]; !ok {
|
||||
return
|
||||
}
|
||||
if !copied {
|
||||
dst = make(map[string]string, len(metadata))
|
||||
for k, v := range metadata {
|
||||
dst[k] = v
|
||||
}
|
||||
copied = true
|
||||
}
|
||||
delete(dst, key)
|
||||
}
|
||||
legalHold := GetObjectLegalHoldMeta(metadata)
|
||||
if !legalHold.Status.Valid() || filterLegalHold {
|
||||
delKey(AmzObjectLockLegalHold)
|
||||
}
|
||||
|
||||
ret := GetObjectRetentionMeta(metadata)
|
||||
if !ret.Mode.Valid() || filterRetention {
|
||||
delKey(AmzObjectLockMode)
|
||||
delKey(AmzObjectLockRetainUntilDate)
|
||||
return dst
|
||||
}
|
||||
return dst
|
||||
}
|
||||
586
internal/bucket/object/lock/lock_test.go
Normal file
586
internal/bucket/object/lock/lock_test.go
Normal file
@@ -0,0 +1,586 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package lock
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net/http"
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
xhttp "github.com/minio/minio/internal/http"
|
||||
)
|
||||
|
||||
func TestParseMode(t *testing.T) {
|
||||
testCases := []struct {
|
||||
value string
|
||||
expectedMode RetMode
|
||||
}{
|
||||
{
|
||||
value: "governance",
|
||||
expectedMode: RetGovernance,
|
||||
},
|
||||
{
|
||||
value: "complIAnce",
|
||||
expectedMode: RetCompliance,
|
||||
},
|
||||
{
|
||||
value: "gce",
|
||||
expectedMode: "",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
if parseRetMode(tc.value) != tc.expectedMode {
|
||||
t.Errorf("Expected Mode %s, got %s", tc.expectedMode, parseRetMode(tc.value))
|
||||
}
|
||||
}
|
||||
}
|
||||
func TestParseLegalHoldStatus(t *testing.T) {
|
||||
tests := []struct {
|
||||
value string
|
||||
expectedStatus LegalHoldStatus
|
||||
}{
|
||||
{
|
||||
value: "ON",
|
||||
expectedStatus: LegalHoldOn,
|
||||
},
|
||||
{
|
||||
value: "Off",
|
||||
expectedStatus: LegalHoldOff,
|
||||
},
|
||||
{
|
||||
value: "x",
|
||||
expectedStatus: "",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
actualStatus := parseLegalHoldStatus(tt.value)
|
||||
if actualStatus != tt.expectedStatus {
|
||||
t.Errorf("Expected legal hold status %s, got %s", tt.expectedStatus, actualStatus)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestUnmarshalDefaultRetention checks if default retention
|
||||
// marshaling and unmarshaling work as expected
|
||||
func TestUnmarshalDefaultRetention(t *testing.T) {
|
||||
days := uint64(4)
|
||||
years := uint64(1)
|
||||
zerodays := uint64(0)
|
||||
invalidDays := uint64(maximumRetentionDays + 1)
|
||||
tests := []struct {
|
||||
value DefaultRetention
|
||||
expectedErr error
|
||||
expectErr bool
|
||||
}{
|
||||
{
|
||||
value: DefaultRetention{Mode: "retain"},
|
||||
expectedErr: fmt.Errorf("unknown retention mode retain"),
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: DefaultRetention{Mode: RetGovernance},
|
||||
expectedErr: fmt.Errorf("either Days or Years must be specified"),
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: DefaultRetention{Mode: RetGovernance, Days: &days},
|
||||
expectedErr: nil,
|
||||
expectErr: false,
|
||||
},
|
||||
{
|
||||
value: DefaultRetention{Mode: RetGovernance, Years: &years},
|
||||
expectedErr: nil,
|
||||
expectErr: false,
|
||||
},
|
||||
{
|
||||
value: DefaultRetention{Mode: RetGovernance, Days: &days, Years: &years},
|
||||
expectedErr: fmt.Errorf("either Days or Years must be specified, not both"),
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: DefaultRetention{Mode: RetGovernance, Days: &zerodays},
|
||||
expectedErr: fmt.Errorf("Default retention period must be a positive integer value for 'Days'"),
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: DefaultRetention{Mode: RetGovernance, Days: &invalidDays},
|
||||
expectedErr: fmt.Errorf("Default retention period too large for 'Days' %d", invalidDays),
|
||||
expectErr: true,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
d, err := xml.MarshalIndent(&tt.value, "", "\t")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
var dr DefaultRetention
|
||||
err = xml.Unmarshal(d, &dr)
|
||||
if tt.expectedErr == nil {
|
||||
if err != nil {
|
||||
t.Fatalf("error: expected = <nil>, got = %v", err)
|
||||
}
|
||||
} else if err == nil {
|
||||
t.Fatalf("error: expected = %v, got = <nil>", tt.expectedErr)
|
||||
} else if tt.expectedErr.Error() != err.Error() {
|
||||
t.Fatalf("error: expected = %v, got = %v", tt.expectedErr, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestParseObjectLockConfig(t *testing.T) {
|
||||
tests := []struct {
|
||||
value string
|
||||
expectedErr error
|
||||
expectErr bool
|
||||
}{
|
||||
{
|
||||
value: `<ObjectLockConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><ObjectLockEnabled>yes</ObjectLockEnabled></ObjectLockConfiguration>`,
|
||||
expectedErr: fmt.Errorf("only 'Enabled' value is allowed to ObjectLockEnabled element"),
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: `<ObjectLockConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><ObjectLockEnabled>Enabled</ObjectLockEnabled><Rule><DefaultRetention><Mode>COMPLIANCE</Mode><Days>0</Days></DefaultRetention></Rule></ObjectLockConfiguration>`,
|
||||
expectedErr: fmt.Errorf("Default retention period must be a positive integer value for 'Days'"),
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: `<ObjectLockConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><ObjectLockEnabled>Enabled</ObjectLockEnabled><Rule><DefaultRetention><Mode>COMPLIANCE</Mode><Days>30</Days></DefaultRetention></Rule></ObjectLockConfiguration>`,
|
||||
expectedErr: nil,
|
||||
expectErr: false,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
_, err := ParseObjectLockConfig(strings.NewReader(tt.value))
|
||||
if tt.expectedErr == nil {
|
||||
if err != nil {
|
||||
t.Fatalf("error: expected = <nil>, got = %v", err)
|
||||
}
|
||||
} else if err == nil {
|
||||
t.Fatalf("error: expected = %v, got = <nil>", tt.expectedErr)
|
||||
} else if tt.expectedErr.Error() != err.Error() {
|
||||
t.Fatalf("error: expected = %v, got = %v", tt.expectedErr, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestParseObjectRetention(t *testing.T) {
|
||||
tests := []struct {
|
||||
value string
|
||||
expectedErr error
|
||||
expectErr bool
|
||||
}{
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><Retention xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Mode>string</Mode><RetainUntilDate>2020-01-02T15:04:05Z</RetainUntilDate></Retention>`,
|
||||
expectedErr: ErrUnknownWORMModeDirective,
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><Retention xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Mode>COMPLIANCE</Mode><RetainUntilDate>2017-01-02T15:04:05Z</RetainUntilDate></Retention>`,
|
||||
expectedErr: ErrPastObjectLockRetainDate,
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><Retention xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Mode>GOVERNANCE</Mode><RetainUntilDate>2057-01-02T15:04:05Z</RetainUntilDate></Retention>`,
|
||||
expectedErr: nil,
|
||||
expectErr: false,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
_, err := ParseObjectRetention(strings.NewReader(tt.value))
|
||||
if tt.expectedErr == nil {
|
||||
if err != nil {
|
||||
t.Fatalf("error: expected = <nil>, got = %v", err)
|
||||
}
|
||||
} else if err == nil {
|
||||
t.Fatalf("error: expected = %v, got = <nil>", tt.expectedErr)
|
||||
} else if tt.expectedErr.Error() != err.Error() {
|
||||
t.Fatalf("error: expected = %v, got = %v", tt.expectedErr, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestIsObjectLockRequested(t *testing.T) {
|
||||
tests := []struct {
|
||||
header http.Header
|
||||
expectedVal bool
|
||||
}{
|
||||
{
|
||||
header: http.Header{
|
||||
"Authorization": []string{"AWS4-HMAC-SHA256 <cred_string>"},
|
||||
"X-Amz-Content-Sha256": []string{""},
|
||||
"Content-Encoding": []string{""},
|
||||
},
|
||||
expectedVal: false,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
AmzObjectLockLegalHold: []string{""},
|
||||
},
|
||||
expectedVal: true,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
AmzObjectLockRetainUntilDate: []string{""},
|
||||
AmzObjectLockMode: []string{""},
|
||||
},
|
||||
expectedVal: true,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
AmzObjectLockBypassRetGovernance: []string{""},
|
||||
},
|
||||
expectedVal: false,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
actualVal := IsObjectLockRequested(tt.header)
|
||||
if actualVal != tt.expectedVal {
|
||||
t.Fatalf("error: expected %v, actual %v", tt.expectedVal, actualVal)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestIsObjectLockGovernanceBypassSet(t *testing.T) {
|
||||
tests := []struct {
|
||||
header http.Header
|
||||
expectedVal bool
|
||||
}{
|
||||
{
|
||||
header: http.Header{
|
||||
"Authorization": []string{"AWS4-HMAC-SHA256 <cred_string>"},
|
||||
"X-Amz-Content-Sha256": []string{""},
|
||||
"Content-Encoding": []string{""},
|
||||
},
|
||||
expectedVal: false,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
AmzObjectLockLegalHold: []string{""},
|
||||
},
|
||||
expectedVal: false,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
AmzObjectLockRetainUntilDate: []string{""},
|
||||
AmzObjectLockMode: []string{""},
|
||||
},
|
||||
expectedVal: false,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
AmzObjectLockBypassRetGovernance: []string{""},
|
||||
},
|
||||
expectedVal: false,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
AmzObjectLockBypassRetGovernance: []string{"true"},
|
||||
},
|
||||
expectedVal: true,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
actualVal := IsObjectLockGovernanceBypassSet(tt.header)
|
||||
if actualVal != tt.expectedVal {
|
||||
t.Fatalf("error: expected %v, actual %v", tt.expectedVal, actualVal)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestParseObjectLockRetentionHeaders(t *testing.T) {
|
||||
tests := []struct {
|
||||
header http.Header
|
||||
expectedErr error
|
||||
}{
|
||||
{
|
||||
header: http.Header{
|
||||
"Authorization": []string{"AWS4-HMAC-SHA256 <cred_string>"},
|
||||
"X-Amz-Content-Sha256": []string{""},
|
||||
"Content-Encoding": []string{""},
|
||||
},
|
||||
expectedErr: ErrObjectLockInvalidHeaders,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
xhttp.AmzObjectLockMode: []string{"lock"},
|
||||
xhttp.AmzObjectLockRetainUntilDate: []string{"2017-01-02"},
|
||||
},
|
||||
expectedErr: ErrUnknownWORMModeDirective,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
xhttp.AmzObjectLockMode: []string{"governance"},
|
||||
},
|
||||
expectedErr: ErrObjectLockInvalidHeaders,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
xhttp.AmzObjectLockRetainUntilDate: []string{"2017-01-02"},
|
||||
xhttp.AmzObjectLockMode: []string{"governance"},
|
||||
},
|
||||
expectedErr: ErrInvalidRetentionDate,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
xhttp.AmzObjectLockRetainUntilDate: []string{"2017-01-02T15:04:05Z"},
|
||||
xhttp.AmzObjectLockMode: []string{"governance"},
|
||||
},
|
||||
expectedErr: ErrPastObjectLockRetainDate,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
xhttp.AmzObjectLockMode: []string{"governance"},
|
||||
xhttp.AmzObjectLockRetainUntilDate: []string{"2017-01-02T15:04:05Z"},
|
||||
},
|
||||
expectedErr: ErrPastObjectLockRetainDate,
|
||||
},
|
||||
{
|
||||
header: http.Header{
|
||||
xhttp.AmzObjectLockMode: []string{"governance"},
|
||||
xhttp.AmzObjectLockRetainUntilDate: []string{"2087-01-02T15:04:05Z"},
|
||||
},
|
||||
expectedErr: nil,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
_, _, err := ParseObjectLockRetentionHeaders(tt.header)
|
||||
if tt.expectedErr == nil {
|
||||
if err != nil {
|
||||
t.Fatalf("Case %d error: expected = <nil>, got = %v", i, err)
|
||||
}
|
||||
} else if err == nil {
|
||||
t.Fatalf("Case %d error: expected = %v, got = <nil>", i, tt.expectedErr)
|
||||
} else if tt.expectedErr.Error() != err.Error() {
|
||||
t.Fatalf("Case %d error: expected = %v, got = %v", i, tt.expectedErr, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestGetObjectRetentionMeta(t *testing.T) {
|
||||
tests := []struct {
|
||||
metadata map[string]string
|
||||
expected ObjectRetention
|
||||
}{
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"Authorization": "AWS4-HMAC-SHA256 <cred_string>",
|
||||
"X-Amz-Content-Sha256": "",
|
||||
"Content-Encoding": "",
|
||||
},
|
||||
expected: ObjectRetention{},
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-mode": "governance",
|
||||
},
|
||||
expected: ObjectRetention{Mode: RetGovernance},
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-retain-until-date": "2020-02-01",
|
||||
},
|
||||
expected: ObjectRetention{RetainUntilDate: RetentionDate{time.Date(2020, 2, 1, 12, 0, 0, 0, time.UTC)}},
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
o := GetObjectRetentionMeta(tt.metadata)
|
||||
if o.Mode != tt.expected.Mode {
|
||||
t.Fatalf("Case %d expected %v, got %v", i, tt.expected.Mode, o.Mode)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestGetObjectLegalHoldMeta(t *testing.T) {
|
||||
tests := []struct {
|
||||
metadata map[string]string
|
||||
expected ObjectLegalHold
|
||||
}{
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-mode": "governance",
|
||||
},
|
||||
expected: ObjectLegalHold{},
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-legal-hold": "on",
|
||||
},
|
||||
expected: ObjectLegalHold{Status: LegalHoldOn},
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-legal-hold": "off",
|
||||
},
|
||||
expected: ObjectLegalHold{Status: LegalHoldOff},
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-legal-hold": "X",
|
||||
},
|
||||
expected: ObjectLegalHold{Status: ""},
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
o := GetObjectLegalHoldMeta(tt.metadata)
|
||||
if o.Status != tt.expected.Status {
|
||||
t.Fatalf("Case %d expected %v, got %v", i, tt.expected.Status, o.Status)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestParseObjectLegalHold(t *testing.T) {
|
||||
tests := []struct {
|
||||
value string
|
||||
expectedErr error
|
||||
expectErr bool
|
||||
}{
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><LegalHold xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Status>string</Status></LegalHold>`,
|
||||
expectedErr: ErrMalformedXML,
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><LegalHold xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Status>ON</Status></LegalHold>`,
|
||||
expectedErr: nil,
|
||||
expectErr: false,
|
||||
},
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><ObjectLockLegalHold xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Status>ON</Status></ObjectLockLegalHold>`,
|
||||
expectedErr: nil,
|
||||
expectErr: false,
|
||||
},
|
||||
// invalid Status key
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><ObjectLockLegalHold xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><MyStatus>ON</MyStatus></ObjectLockLegalHold>`,
|
||||
expectedErr: errors.New("expected element type <Status> but have <MyStatus>"),
|
||||
expectErr: true,
|
||||
},
|
||||
// invalid XML attr
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><UnknownLegalHold xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Status>ON</Status></UnknownLegalHold>`,
|
||||
expectedErr: errors.New("expected element type <LegalHold>/<ObjectLockLegalHold> but have <UnknownLegalHold>"),
|
||||
expectErr: true,
|
||||
},
|
||||
{
|
||||
value: `<?xml version="1.0" encoding="UTF-8"?><LegalHold xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Status>On</Status></LegalHold>`,
|
||||
expectedErr: ErrMalformedXML,
|
||||
expectErr: true,
|
||||
},
|
||||
}
|
||||
for i, tt := range tests {
|
||||
_, err := ParseObjectLegalHold(strings.NewReader(tt.value))
|
||||
if tt.expectedErr == nil {
|
||||
if err != nil {
|
||||
t.Fatalf("Case %d error: expected = <nil>, got = %v", i, err)
|
||||
}
|
||||
} else if err == nil {
|
||||
t.Fatalf("Case %d error: expected = %v, got = <nil>", i, tt.expectedErr)
|
||||
} else if tt.expectedErr.Error() != err.Error() {
|
||||
t.Fatalf("Case %d error: expected = %v, got = %v", i, tt.expectedErr, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
func TestFilterObjectLockMetadata(t *testing.T) {
|
||||
tests := []struct {
|
||||
metadata map[string]string
|
||||
filterRetention bool
|
||||
filterLegalHold bool
|
||||
expected map[string]string
|
||||
}{
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"Authorization": "AWS4-HMAC-SHA256 <cred_string>",
|
||||
"X-Amz-Content-Sha256": "",
|
||||
"Content-Encoding": "",
|
||||
},
|
||||
expected: map[string]string{
|
||||
"Authorization": "AWS4-HMAC-SHA256 <cred_string>",
|
||||
"X-Amz-Content-Sha256": "",
|
||||
"Content-Encoding": "",
|
||||
},
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-mode": "governance",
|
||||
},
|
||||
expected: map[string]string{
|
||||
"x-amz-object-lock-mode": "governance",
|
||||
},
|
||||
filterRetention: false,
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-mode": "governance",
|
||||
"x-amz-object-lock-retain-until-date": "2020-02-01",
|
||||
},
|
||||
expected: map[string]string{},
|
||||
filterRetention: true,
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-legal-hold": "off",
|
||||
},
|
||||
expected: map[string]string{},
|
||||
filterLegalHold: true,
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-legal-hold": "on",
|
||||
},
|
||||
expected: map[string]string{"x-amz-object-lock-legal-hold": "on"},
|
||||
filterLegalHold: false,
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-legal-hold": "on",
|
||||
"x-amz-object-lock-mode": "governance",
|
||||
"x-amz-object-lock-retain-until-date": "2020-02-01",
|
||||
},
|
||||
expected: map[string]string{},
|
||||
filterRetention: true,
|
||||
filterLegalHold: true,
|
||||
},
|
||||
{
|
||||
metadata: map[string]string{
|
||||
"x-amz-object-lock-legal-hold": "on",
|
||||
"x-amz-object-lock-mode": "governance",
|
||||
"x-amz-object-lock-retain-until-date": "2020-02-01",
|
||||
},
|
||||
expected: map[string]string{"x-amz-object-lock-legal-hold": "on",
|
||||
"x-amz-object-lock-mode": "governance",
|
||||
"x-amz-object-lock-retain-until-date": "2020-02-01"},
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
o := FilterObjectLockMetadata(tt.metadata, tt.filterRetention, tt.filterLegalHold)
|
||||
if !reflect.DeepEqual(o, tt.metadata) {
|
||||
t.Fatalf("Case %d expected %v, got %v", i, tt.metadata, o)
|
||||
}
|
||||
}
|
||||
}
|
||||
63
internal/bucket/replication/and.go
Normal file
63
internal/bucket/replication/and.go
Normal file
@@ -0,0 +1,63 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
)
|
||||
|
||||
// And - a tag to combine a prefix and multiple tags for replication configuration rule.
|
||||
type And struct {
|
||||
XMLName xml.Name `xml:"And" json:"And"`
|
||||
Prefix string `xml:"Prefix,omitempty" json:"Prefix,omitempty"`
|
||||
Tags []Tag `xml:"Tag,omitempty" json:"Tag,omitempty"`
|
||||
}
|
||||
|
||||
var errDuplicateTagKey = Errorf("Duplicate Tag Keys are not allowed")
|
||||
|
||||
// isEmpty returns true if Tags field is null
|
||||
func (a And) isEmpty() bool {
|
||||
return len(a.Tags) == 0 && a.Prefix == ""
|
||||
}
|
||||
|
||||
// Validate - validates the And field
|
||||
func (a And) Validate() error {
|
||||
if a.ContainsDuplicateTag() {
|
||||
return errDuplicateTagKey
|
||||
}
|
||||
for _, t := range a.Tags {
|
||||
if err := t.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ContainsDuplicateTag - returns true if duplicate keys are present in And
|
||||
func (a And) ContainsDuplicateTag() bool {
|
||||
x := make(map[string]struct{}, len(a.Tags))
|
||||
|
||||
for _, t := range a.Tags {
|
||||
if _, has := x[t.Key]; has {
|
||||
return true
|
||||
}
|
||||
x[t.Key] = struct{}{}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
119
internal/bucket/replication/destination.go
Normal file
119
internal/bucket/replication/destination.go
Normal file
@@ -0,0 +1,119 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/minio/pkg/wildcard"
|
||||
)
|
||||
|
||||
// DestinationARNPrefix - destination ARN prefix as per AWS S3 specification.
|
||||
const DestinationARNPrefix = "arn:aws:s3:::"
|
||||
|
||||
// Destination - destination in ReplicationConfiguration.
|
||||
type Destination struct {
|
||||
XMLName xml.Name `xml:"Destination" json:"Destination"`
|
||||
Bucket string `xml:"Bucket" json:"Bucket"`
|
||||
StorageClass string `xml:"StorageClass" json:"StorageClass"`
|
||||
//EncryptionConfiguration TODO: not needed for MinIO
|
||||
}
|
||||
|
||||
func (d Destination) isValidStorageClass() bool {
|
||||
if d.StorageClass == "" {
|
||||
return true
|
||||
}
|
||||
return d.StorageClass == "STANDARD" || d.StorageClass == "REDUCED_REDUNDANCY"
|
||||
}
|
||||
|
||||
// IsValid - checks whether Destination is valid or not.
|
||||
func (d Destination) IsValid() bool {
|
||||
return d.Bucket != "" || !d.isValidStorageClass()
|
||||
}
|
||||
|
||||
func (d Destination) String() string {
|
||||
return DestinationARNPrefix + d.Bucket
|
||||
}
|
||||
|
||||
// MarshalXML - encodes to XML data.
|
||||
func (d Destination) MarshalXML(e *xml.Encoder, start xml.StartElement) error {
|
||||
if err := e.EncodeToken(start); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := e.EncodeElement(d.String(), xml.StartElement{Name: xml.Name{Local: "Bucket"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
if d.StorageClass != "" {
|
||||
if err := e.EncodeElement(d.StorageClass, xml.StartElement{Name: xml.Name{Local: "StorageClass"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return e.EncodeToken(xml.EndElement{Name: start.Name})
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (d *Destination) UnmarshalXML(dec *xml.Decoder, start xml.StartElement) (err error) {
|
||||
// Make subtype to avoid recursive UnmarshalXML().
|
||||
type destination Destination
|
||||
dest := destination{}
|
||||
|
||||
if err := dec.DecodeElement(&dest, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
parsedDest, err := parseDestination(dest.Bucket)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if dest.StorageClass != "" {
|
||||
switch dest.StorageClass {
|
||||
case "STANDARD", "REDUCED_REDUNDANCY":
|
||||
default:
|
||||
return fmt.Errorf("unknown storage class %s", dest.StorageClass)
|
||||
}
|
||||
}
|
||||
parsedDest.StorageClass = dest.StorageClass
|
||||
*d = parsedDest
|
||||
return nil
|
||||
}
|
||||
|
||||
// Validate - validates Resource is for given bucket or not.
|
||||
func (d Destination) Validate(bucketName string) error {
|
||||
if !d.IsValid() {
|
||||
return Errorf("invalid destination")
|
||||
}
|
||||
|
||||
if !wildcard.Match(d.Bucket, bucketName) {
|
||||
return Errorf("bucket name does not match")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// parseDestination - parses string to Destination.
|
||||
func parseDestination(s string) (Destination, error) {
|
||||
if !strings.HasPrefix(s, DestinationARNPrefix) {
|
||||
return Destination{}, Errorf("invalid destination '%s'", s)
|
||||
}
|
||||
|
||||
bucketName := strings.TrimPrefix(s, DestinationARNPrefix)
|
||||
|
||||
return Destination{
|
||||
Bucket: bucketName,
|
||||
}, nil
|
||||
}
|
||||
45
internal/bucket/replication/error.go
Normal file
45
internal/bucket/replication/error.go
Normal file
@@ -0,0 +1,45 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
// Error is the generic type for any error happening during tag
|
||||
// parsing.
|
||||
type Error struct {
|
||||
err error
|
||||
}
|
||||
|
||||
// Errorf - formats according to a format specifier and returns
|
||||
// the string as a value that satisfies error of type tagging.Error
|
||||
func Errorf(format string, a ...interface{}) error {
|
||||
return Error{err: fmt.Errorf(format, a...)}
|
||||
}
|
||||
|
||||
// Unwrap the internal error.
|
||||
func (e Error) Unwrap() error { return e.err }
|
||||
|
||||
// Error 'error' compatible method.
|
||||
func (e Error) Error() string {
|
||||
if e.err == nil {
|
||||
return "replication: cause <nil>"
|
||||
}
|
||||
return e.err.Error()
|
||||
}
|
||||
121
internal/bucket/replication/filter.go
Normal file
121
internal/bucket/replication/filter.go
Normal file
@@ -0,0 +1,121 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
)
|
||||
|
||||
var (
|
||||
errInvalidFilter = Errorf("Filter must have exactly one of Prefix, Tag, or And specified")
|
||||
)
|
||||
|
||||
// Filter - a filter for a replication configuration Rule.
|
||||
type Filter struct {
|
||||
XMLName xml.Name `xml:"Filter" json:"Filter"`
|
||||
Prefix string
|
||||
And And
|
||||
Tag Tag
|
||||
// Caching tags, only once
|
||||
cachedTags map[string]struct{}
|
||||
}
|
||||
|
||||
// IsEmpty returns true if filter is not set
|
||||
func (f Filter) IsEmpty() bool {
|
||||
return f.And.isEmpty() && f.Tag.IsEmpty() && f.Prefix == ""
|
||||
}
|
||||
|
||||
// MarshalXML - produces the xml representation of the Filter struct
|
||||
// only one of Prefix, And and Tag should be present in the output.
|
||||
func (f Filter) MarshalXML(e *xml.Encoder, start xml.StartElement) error {
|
||||
if err := e.EncodeToken(start); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
switch {
|
||||
case !f.And.isEmpty():
|
||||
if err := e.EncodeElement(f.And, xml.StartElement{Name: xml.Name{Local: "And"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
case !f.Tag.IsEmpty():
|
||||
if err := e.EncodeElement(f.Tag, xml.StartElement{Name: xml.Name{Local: "Tag"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
default:
|
||||
// Always print Prefix field when both And & Tag are empty
|
||||
if err := e.EncodeElement(f.Prefix, xml.StartElement{Name: xml.Name{Local: "Prefix"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return e.EncodeToken(xml.EndElement{Name: start.Name})
|
||||
}
|
||||
|
||||
// Validate - validates the filter element
|
||||
func (f Filter) Validate() error {
|
||||
// A Filter must have exactly one of Prefix, Tag, or And specified.
|
||||
if !f.And.isEmpty() {
|
||||
if f.Prefix != "" {
|
||||
return errInvalidFilter
|
||||
}
|
||||
if !f.Tag.IsEmpty() {
|
||||
return errInvalidFilter
|
||||
}
|
||||
if err := f.And.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if f.Prefix != "" {
|
||||
if !f.Tag.IsEmpty() {
|
||||
return errInvalidFilter
|
||||
}
|
||||
}
|
||||
if !f.Tag.IsEmpty() {
|
||||
if err := f.Tag.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// TestTags tests if the object tags satisfy the Filter tags requirement,
|
||||
// it returns true if there is no tags in the underlying Filter.
|
||||
func (f *Filter) TestTags(ttags []string) bool {
|
||||
if f.cachedTags == nil {
|
||||
tags := make(map[string]struct{})
|
||||
for _, t := range append(f.And.Tags, f.Tag) {
|
||||
if !t.IsEmpty() {
|
||||
tags[t.String()] = struct{}{}
|
||||
}
|
||||
}
|
||||
f.cachedTags = tags
|
||||
}
|
||||
for ct := range f.cachedTags {
|
||||
foundTag := false
|
||||
for _, t := range ttags {
|
||||
if ct == t {
|
||||
foundTag = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !foundTag {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
234
internal/bucket/replication/replication.go
Normal file
234
internal/bucket/replication/replication.go
Normal file
@@ -0,0 +1,234 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"io"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
)
|
||||
|
||||
// StatusType of Replication for x-amz-replication-status header
|
||||
type StatusType string
|
||||
|
||||
const (
|
||||
// Pending - replication is pending.
|
||||
Pending StatusType = "PENDING"
|
||||
|
||||
// Completed - replication completed ok.
|
||||
Completed StatusType = "COMPLETED"
|
||||
|
||||
// Failed - replication failed.
|
||||
Failed StatusType = "FAILED"
|
||||
|
||||
// Replica - this is a replica.
|
||||
Replica StatusType = "REPLICA"
|
||||
)
|
||||
|
||||
// String returns string representation of status
|
||||
func (s StatusType) String() string {
|
||||
return string(s)
|
||||
}
|
||||
|
||||
// Empty returns true if this status is not set
|
||||
func (s StatusType) Empty() bool {
|
||||
return string(s) == ""
|
||||
}
|
||||
|
||||
var (
|
||||
errReplicationTooManyRules = Errorf("Replication configuration allows a maximum of 1000 rules")
|
||||
errReplicationNoRule = Errorf("Replication configuration should have at least one rule")
|
||||
errReplicationUniquePriority = Errorf("Replication configuration has duplicate priority")
|
||||
errReplicationDestinationMismatch = Errorf("The destination bucket must be same for all rules")
|
||||
errRoleArnMissing = Errorf("Missing required parameter `Role` in ReplicationConfiguration")
|
||||
errInvalidSourceSelectionCriteria = Errorf("Invalid ReplicaModification status")
|
||||
)
|
||||
|
||||
// Config - replication configuration specified in
|
||||
// https://docs.aws.amazon.com/AmazonS3/latest/dev/replication-add-config.html
|
||||
type Config struct {
|
||||
XMLName xml.Name `xml:"ReplicationConfiguration" json:"-"`
|
||||
Rules []Rule `xml:"Rule" json:"Rules"`
|
||||
// RoleArn is being reused for MinIO replication ARN
|
||||
RoleArn string `xml:"Role" json:"Role"`
|
||||
}
|
||||
|
||||
// Maximum 2MiB size per replication config.
|
||||
const maxReplicationConfigSize = 2 << 20
|
||||
|
||||
// ParseConfig parses ReplicationConfiguration from xml
|
||||
func ParseConfig(reader io.Reader) (*Config, error) {
|
||||
config := Config{}
|
||||
if err := xml.NewDecoder(io.LimitReader(reader, maxReplicationConfigSize)).Decode(&config); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// By default, set replica modification to enabled if unset.
|
||||
for i := range config.Rules {
|
||||
if len(config.Rules[i].SourceSelectionCriteria.ReplicaModifications.Status) == 0 {
|
||||
config.Rules[i].SourceSelectionCriteria = SourceSelectionCriteria{
|
||||
ReplicaModifications: ReplicaModifications{
|
||||
Status: Enabled,
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
return &config, nil
|
||||
}
|
||||
|
||||
// Validate - validates the replication configuration
|
||||
func (c Config) Validate(bucket string, sameTarget bool) error {
|
||||
// replication config can't have more than 1000 rules
|
||||
if len(c.Rules) > 1000 {
|
||||
return errReplicationTooManyRules
|
||||
}
|
||||
// replication config should have at least one rule
|
||||
if len(c.Rules) == 0 {
|
||||
return errReplicationNoRule
|
||||
}
|
||||
if c.RoleArn == "" {
|
||||
return errRoleArnMissing
|
||||
}
|
||||
// Validate all the rules in the replication config
|
||||
targetMap := make(map[string]struct{})
|
||||
priorityMap := make(map[string]struct{})
|
||||
for _, r := range c.Rules {
|
||||
if len(targetMap) == 0 {
|
||||
targetMap[r.Destination.Bucket] = struct{}{}
|
||||
}
|
||||
if _, ok := targetMap[r.Destination.Bucket]; !ok {
|
||||
return errReplicationDestinationMismatch
|
||||
}
|
||||
if err := r.Validate(bucket, sameTarget); err != nil {
|
||||
return err
|
||||
}
|
||||
if _, ok := priorityMap[strconv.Itoa(r.Priority)]; ok {
|
||||
return errReplicationUniquePriority
|
||||
}
|
||||
priorityMap[strconv.Itoa(r.Priority)] = struct{}{}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Type - replication type enum
|
||||
type Type int
|
||||
|
||||
// Types of replication
|
||||
const (
|
||||
ObjectReplicationType Type = 1 + iota
|
||||
DeleteReplicationType
|
||||
MetadataReplicationType
|
||||
HealReplicationType
|
||||
)
|
||||
|
||||
// ObjectOpts provides information to deduce whether replication
|
||||
// can be triggered on the resultant object.
|
||||
type ObjectOpts struct {
|
||||
Name string
|
||||
UserTags string
|
||||
VersionID string
|
||||
IsLatest bool
|
||||
DeleteMarker bool
|
||||
SSEC bool
|
||||
OpType Type
|
||||
Replica bool
|
||||
}
|
||||
|
||||
// FilterActionableRules returns the rules actions that need to be executed
|
||||
// after evaluating prefix/tag filtering
|
||||
func (c Config) FilterActionableRules(obj ObjectOpts) []Rule {
|
||||
if obj.Name == "" {
|
||||
return nil
|
||||
}
|
||||
var rules []Rule
|
||||
for _, rule := range c.Rules {
|
||||
if rule.Status == Disabled {
|
||||
continue
|
||||
}
|
||||
if !strings.HasPrefix(obj.Name, rule.Prefix()) {
|
||||
continue
|
||||
}
|
||||
if rule.Filter.TestTags(strings.Split(obj.UserTags, "&")) {
|
||||
rules = append(rules, rule)
|
||||
}
|
||||
}
|
||||
sort.Slice(rules[:], func(i, j int) bool {
|
||||
return rules[i].Priority > rules[j].Priority
|
||||
})
|
||||
return rules
|
||||
}
|
||||
|
||||
// GetDestination returns destination bucket and storage class.
|
||||
func (c Config) GetDestination() Destination {
|
||||
if len(c.Rules) > 0 {
|
||||
return c.Rules[0].Destination
|
||||
}
|
||||
return Destination{}
|
||||
}
|
||||
|
||||
// Replicate returns true if the object should be replicated.
|
||||
func (c Config) Replicate(obj ObjectOpts) bool {
|
||||
if obj.SSEC {
|
||||
return false
|
||||
}
|
||||
for _, rule := range c.FilterActionableRules(obj) {
|
||||
if rule.Status == Disabled {
|
||||
continue
|
||||
}
|
||||
if obj.OpType == DeleteReplicationType {
|
||||
switch {
|
||||
case obj.VersionID != "":
|
||||
// // check MinIO extension for versioned deletes
|
||||
return rule.DeleteReplication.Status == Enabled
|
||||
default:
|
||||
return rule.DeleteMarkerReplication.Status == Enabled
|
||||
}
|
||||
} // regular object/metadata replication
|
||||
return rule.MetadataReplicate(obj)
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// HasActiveRules - returns whether replication policy has active rules
|
||||
// Optionally a prefix can be supplied.
|
||||
// If recursive is specified the function will also return true if any level below the
|
||||
// prefix has active rules. If no prefix is specified recursive is effectively true.
|
||||
func (c Config) HasActiveRules(prefix string, recursive bool) bool {
|
||||
if len(c.Rules) == 0 {
|
||||
return false
|
||||
}
|
||||
for _, rule := range c.Rules {
|
||||
if rule.Status == Disabled {
|
||||
continue
|
||||
}
|
||||
if len(prefix) > 0 && len(rule.Filter.Prefix) > 0 {
|
||||
// incoming prefix must be in rule prefix
|
||||
if !recursive && !strings.HasPrefix(prefix, rule.Filter.Prefix) {
|
||||
continue
|
||||
}
|
||||
// If recursive, we can skip this rule if it doesn't match the tested prefix or level below prefix
|
||||
// does not match
|
||||
if recursive && !strings.HasPrefix(rule.Prefix(), prefix) && !strings.HasPrefix(prefix, rule.Prefix()) {
|
||||
continue
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
335
internal/bucket/replication/replication_test.go
Normal file
335
internal/bucket/replication/replication_test.go
Normal file
@@ -0,0 +1,335 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestParseAndValidateReplicationConfig(t *testing.T) {
|
||||
testCases := []struct {
|
||||
inputConfig string
|
||||
expectedParsingErr error
|
||||
expectedValidationErr error
|
||||
destBucket string
|
||||
sameTarget bool
|
||||
}{
|
||||
{ //1 Invalid delete marker status in replication config
|
||||
inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>string</Status></DeleteMarkerReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errInvalidDeleteMarkerReplicationStatus,
|
||||
},
|
||||
//2 Invalid delete replication status in replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errDeleteReplicationMissing,
|
||||
},
|
||||
//3 valid replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: nil,
|
||||
},
|
||||
//4 missing role in config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
// destination bucket in config different from bucket specified
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errRoleArnMissing,
|
||||
},
|
||||
//5 replication destination in different rules not identical
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule><Rule><Status>Enabled</Status><Priority>3</Priority><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket2</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errReplicationDestinationMismatch,
|
||||
},
|
||||
//6 missing rule status in replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errEmptyRuleStatus,
|
||||
},
|
||||
//7 invalid rule status in replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enssabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errInvalidRuleStatus,
|
||||
},
|
||||
//8 invalid rule id exceeds length allowed in replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><ID>vsUVERgOc8zZYagLSzSa5lE8qeI6nh1lyLNS4R9W052yfecrhhepGboswSWMMNO8CPcXM4GM3nKyQ72EadlMzzZBFoYWKn7ju5GoE5w9c57a0piHR1vexpdd9FrMquiruvAJ0MTGVupm0EegMVxoIOdjx7VgZhGrmi2XDvpVEFT7WmYMA9fSK297XkTHWyECaNHBySJ1Qp4vwX8tPNauKpfHx4kzUpnKe1PZbptGMWbY5qTcwlNuMhVSmgFffShq</ID><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errInvalidRuleID,
|
||||
},
|
||||
//9 invalid priority status in replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errReplicationUniquePriority,
|
||||
},
|
||||
//10 no rule in replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: nil,
|
||||
expectedValidationErr: errReplicationNoRule,
|
||||
},
|
||||
//11 no destination in replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: Errorf("invalid destination '%v'", ""),
|
||||
expectedValidationErr: nil,
|
||||
},
|
||||
//12 destination not matching ARN in replication config
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>destinationbucket2</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
destBucket: "destinationbucket",
|
||||
sameTarget: false,
|
||||
expectedParsingErr: fmt.Errorf("invalid destination '%v'", "destinationbucket2"),
|
||||
expectedValidationErr: nil,
|
||||
},
|
||||
}
|
||||
for i, tc := range testCases {
|
||||
t.Run(fmt.Sprintf("Test %d", i+1), func(t *testing.T) {
|
||||
cfg, err := ParseConfig(bytes.NewReader([]byte(tc.inputConfig)))
|
||||
if err != nil && tc.expectedParsingErr != nil && err.Error() != tc.expectedParsingErr.Error() {
|
||||
t.Fatalf("%d: Expected '%v' during parsing but got '%v'", i+1, tc.expectedParsingErr, err)
|
||||
}
|
||||
if err == nil && tc.expectedParsingErr != nil {
|
||||
t.Fatalf("%d: Expected '%v' during parsing but got '%v'", i+1, tc.expectedParsingErr, err)
|
||||
}
|
||||
if tc.expectedParsingErr != nil {
|
||||
// We already expect a parsing error,
|
||||
// no need to continue this test.
|
||||
return
|
||||
}
|
||||
err = cfg.Validate(tc.destBucket, tc.sameTarget)
|
||||
if err != tc.expectedValidationErr {
|
||||
t.Fatalf("%d: Expected %v during parsing but got %v", i+1, tc.expectedValidationErr, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
func TestReplicate(t *testing.T) {
|
||||
cfgs := []Config{
|
||||
{ // Config0 - Replication config has no filters, all replication enabled
|
||||
Rules: []Rule{
|
||||
{
|
||||
Status: Enabled,
|
||||
Priority: 3,
|
||||
DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled},
|
||||
DeleteReplication: DeleteReplication{Status: Enabled},
|
||||
Filter: Filter{},
|
||||
},
|
||||
},
|
||||
},
|
||||
{ // Config1 - Replication config has no filters, delete,delete-marker replication disabled
|
||||
Rules: []Rule{
|
||||
{
|
||||
Status: Enabled,
|
||||
Priority: 3,
|
||||
DeleteMarkerReplication: DeleteMarkerReplication{Status: Disabled},
|
||||
DeleteReplication: DeleteReplication{Status: Disabled},
|
||||
Filter: Filter{},
|
||||
},
|
||||
},
|
||||
},
|
||||
{ // Config2 - Replication config has filters and more than 1 matching rule, delete,delete-marker replication disabled
|
||||
Rules: []Rule{
|
||||
{
|
||||
Status: Enabled,
|
||||
Priority: 2,
|
||||
DeleteMarkerReplication: DeleteMarkerReplication{Status: Disabled},
|
||||
DeleteReplication: DeleteReplication{Status: Enabled},
|
||||
Filter: Filter{Prefix: "xy", And: And{}, Tag: Tag{Key: "k1", Value: "v1"}},
|
||||
},
|
||||
{
|
||||
Status: Enabled,
|
||||
Priority: 1,
|
||||
DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled},
|
||||
DeleteReplication: DeleteReplication{Status: Disabled},
|
||||
Filter: Filter{Prefix: "xyz"},
|
||||
},
|
||||
},
|
||||
},
|
||||
{ // Config3 - Replication config has filters and no overlapping rules
|
||||
Rules: []Rule{
|
||||
{
|
||||
Status: Enabled,
|
||||
Priority: 2,
|
||||
DeleteMarkerReplication: DeleteMarkerReplication{Status: Disabled},
|
||||
DeleteReplication: DeleteReplication{Status: Enabled},
|
||||
Filter: Filter{Prefix: "xy", And: And{}, Tag: Tag{Key: "k1", Value: "v1"}},
|
||||
},
|
||||
{
|
||||
Status: Enabled,
|
||||
Priority: 1,
|
||||
DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled},
|
||||
DeleteReplication: DeleteReplication{Status: Disabled},
|
||||
Filter: Filter{Prefix: "abc"},
|
||||
},
|
||||
},
|
||||
},
|
||||
{ // Config4 - Replication config has filters and SourceSelectionCriteria Disabled
|
||||
Rules: []Rule{
|
||||
{
|
||||
Status: Enabled,
|
||||
Priority: 2,
|
||||
DeleteMarkerReplication: DeleteMarkerReplication{Status: Enabled},
|
||||
DeleteReplication: DeleteReplication{Status: Enabled},
|
||||
SourceSelectionCriteria: SourceSelectionCriteria{ReplicaModifications: ReplicaModifications{Status: Disabled}},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
testCases := []struct {
|
||||
opts ObjectOpts
|
||||
c Config
|
||||
expectedResult bool
|
||||
}{
|
||||
// using config 1 - no filters, all replication enabled
|
||||
{ObjectOpts{}, cfgs[0], false}, //1. invalid ObjectOpts missing object name
|
||||
{ObjectOpts{Name: "c1test"}, cfgs[0], true}, //2. valid ObjectOpts passing empty Filter
|
||||
{ObjectOpts{Name: "c1test", VersionID: "vid"}, cfgs[0], true}, //3. valid ObjectOpts passing empty Filter
|
||||
|
||||
{ObjectOpts{Name: "c1test", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[0], true}, //4. DeleteMarker version replication valid case - matches DeleteMarkerReplication status
|
||||
{ObjectOpts{Name: "c1test", VersionID: "vid", OpType: DeleteReplicationType}, cfgs[0], true}, //5. permanent delete of version, matches DeleteReplication status - valid case
|
||||
{ObjectOpts{Name: "c1test", VersionID: "vid", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[0], true}, //6. permanent delete of version, matches DeleteReplication status
|
||||
{ObjectOpts{Name: "c1test", VersionID: "vid", DeleteMarker: true, SSEC: true, OpType: DeleteReplicationType}, cfgs[0], false}, //7. permanent delete of version, disqualified by SSE-C
|
||||
{ObjectOpts{Name: "c1test", DeleteMarker: true, SSEC: true, OpType: DeleteReplicationType}, cfgs[0], false}, //8. setting DeleteMarker on SSE-C encrypted object, disqualified by SSE-C
|
||||
{ObjectOpts{Name: "c1test", SSEC: true}, cfgs[0], false}, //9. replication of SSE-C encrypted object, disqualified
|
||||
|
||||
// using config 2 - no filters, only replication of object, metadata enabled
|
||||
{ObjectOpts{Name: "c2test"}, cfgs[1], true}, //10. valid ObjectOpts passing empty Filter
|
||||
{ObjectOpts{Name: "c2test", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[1], false}, //11. DeleteMarker version replication not allowed due to DeleteMarkerReplication status
|
||||
{ObjectOpts{Name: "c2test", VersionID: "vid", OpType: DeleteReplicationType}, cfgs[1], false}, //12. permanent delete of version, disallowed by DeleteReplication status
|
||||
{ObjectOpts{Name: "c2test", VersionID: "vid", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[1], false}, //13. permanent delete of DeleteMarker version, disallowed by DeleteReplication status
|
||||
{ObjectOpts{Name: "c2test", VersionID: "vid", DeleteMarker: true, SSEC: true, OpType: DeleteReplicationType}, cfgs[1], false}, //14. permanent delete of version, disqualified by SSE-C & DeleteReplication status
|
||||
{ObjectOpts{Name: "c2test", DeleteMarker: true, SSEC: true, OpType: DeleteReplicationType}, cfgs[1], false}, //15. setting DeleteMarker on SSE-C encrypted object, disqualified by SSE-C & DeleteMarkerReplication status
|
||||
{ObjectOpts{Name: "c2test", SSEC: true}, cfgs[1], false}, //16. replication of SSE-C encrypted object, disqualified by default
|
||||
// using config 2 - has more than one rule with overlapping prefixes
|
||||
{ObjectOpts{Name: "xy/c3test", UserTags: "k1=v1"}, cfgs[2], true}, //17. matches rule 1 for replication of content/metadata
|
||||
{ObjectOpts{Name: "xyz/c3test", UserTags: "k1=v1"}, cfgs[2], true}, //18. matches rule 1 for replication of content/metadata
|
||||
{ObjectOpts{Name: "xyz/c3test", UserTags: "k1=v1", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[2], false}, //19. matches rule 1 - DeleteMarker replication disallowed by rule
|
||||
{ObjectOpts{Name: "xyz/c3test", UserTags: "k1=v1", DeleteMarker: true, VersionID: "vid", OpType: DeleteReplicationType}, cfgs[2], true}, //20. matches rule 1 - DeleteReplication allowed by rule for permanent delete of DeleteMarker
|
||||
{ObjectOpts{Name: "xyz/c3test", UserTags: "k1=v1", VersionID: "vid", OpType: DeleteReplicationType}, cfgs[2], true}, //21. matches rule 1 - DeleteReplication allowed by rule for permanent delete of version
|
||||
{ObjectOpts{Name: "xyz/c3test"}, cfgs[2], true}, //22. matches rule 2 for replication of content/metadata
|
||||
{ObjectOpts{Name: "xy/c3test", UserTags: "k1=v2"}, cfgs[2], false}, //23. does not match rule1 because tag value does not pass filter
|
||||
{ObjectOpts{Name: "xyz/c3test", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[2], true}, //24. matches rule 2 - DeleteMarker replication allowed by rule
|
||||
{ObjectOpts{Name: "xyz/c3test", DeleteMarker: true, VersionID: "vid", OpType: DeleteReplicationType}, cfgs[2], false}, //25. matches rule 2 - DeleteReplication disallowed by rule for permanent delete of DeleteMarker
|
||||
{ObjectOpts{Name: "xyz/c3test", VersionID: "vid", OpType: DeleteReplicationType}, cfgs[2], false}, //26. matches rule 1 - DeleteReplication disallowed by rule for permanent delete of version
|
||||
{ObjectOpts{Name: "abc/c3test"}, cfgs[2], false}, //27. matches no rule because object prefix does not match
|
||||
|
||||
// using config 3 - has no overlapping rules
|
||||
{ObjectOpts{Name: "xy/c4test", UserTags: "k1=v1"}, cfgs[3], true}, //28. matches rule 1 for replication of content/metadata
|
||||
{ObjectOpts{Name: "xa/c4test", UserTags: "k1=v1"}, cfgs[3], false}, //29. no rule match object prefix not in rules
|
||||
{ObjectOpts{Name: "xyz/c4test", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[3], false}, //30. rule 1 not matched because of tags filter
|
||||
{ObjectOpts{Name: "xyz/c4test", UserTags: "k1=v1", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[3], false}, //31. matches rule 1 - DeleteMarker replication disallowed by rule
|
||||
{ObjectOpts{Name: "xyz/c4test", UserTags: "k1=v1", DeleteMarker: true, VersionID: "vid", OpType: DeleteReplicationType}, cfgs[3], true}, //32. matches rule 1 - DeleteReplication allowed by rule for permanent delete of DeleteMarker
|
||||
{ObjectOpts{Name: "xyz/c4test", UserTags: "k1=v1", VersionID: "vid", OpType: DeleteReplicationType}, cfgs[3], true}, //33. matches rule 1 - DeleteReplication allowed by rule for permanent delete of version
|
||||
{ObjectOpts{Name: "abc/c4test"}, cfgs[3], true}, //34. matches rule 2 for replication of content/metadata
|
||||
{ObjectOpts{Name: "abc/c4test", UserTags: "k1=v2"}, cfgs[3], true}, //35. matches rule 2 for replication of content/metadata
|
||||
{ObjectOpts{Name: "abc/c4test", DeleteMarker: true, OpType: DeleteReplicationType}, cfgs[3], true}, //36. matches rule 2 - DeleteMarker replication allowed by rule
|
||||
{ObjectOpts{Name: "abc/c4test", DeleteMarker: true, VersionID: "vid", OpType: DeleteReplicationType}, cfgs[3], false}, //37. matches rule 2 - DeleteReplication disallowed by rule for permanent delete of DeleteMarker
|
||||
{ObjectOpts{Name: "abc/c4test", VersionID: "vid", OpType: DeleteReplicationType}, cfgs[3], false}, //38. matches rule 2 - DeleteReplication disallowed by rule for permanent delete of version
|
||||
// using config 4 - with replica modification sync disabled.
|
||||
{ObjectOpts{Name: "xy/c5test", UserTags: "k1=v1", Replica: true}, cfgs[4], false}, //39. replica syncing disabled, this object is a replica
|
||||
{ObjectOpts{Name: "xa/c5test", UserTags: "k1=v1", Replica: false}, cfgs[4], true}, //40. replica syncing disabled, this object is NOT a replica
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result := testCase.c.Replicate(testCase.opts)
|
||||
|
||||
if result != testCase.expectedResult {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestHasActiveRules(t *testing.T) {
|
||||
testCases := []struct {
|
||||
inputConfig string
|
||||
prefix string
|
||||
expectedNonRec bool
|
||||
expectedRec bool
|
||||
}{
|
||||
|
||||
// case 1 - only one rule which is in Disabled status
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Disabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
prefix: "miss/prefix",
|
||||
expectedNonRec: false,
|
||||
expectedRec: false,
|
||||
},
|
||||
// case 2 - only one rule which matches prefix filter
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Filter><Prefix>key/prefix</Prefix></Filter><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
prefix: "key/prefix1",
|
||||
expectedNonRec: true,
|
||||
expectedRec: true,
|
||||
},
|
||||
// case 3 - empty prefix
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
prefix: "key-prefix",
|
||||
expectedNonRec: true,
|
||||
expectedRec: true,
|
||||
},
|
||||
// case 4 - has Filter based on prefix
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Filter><Prefix>testdir/dir1/</Prefix></Filter><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
prefix: "testdir/",
|
||||
expectedNonRec: false,
|
||||
expectedRec: true,
|
||||
},
|
||||
//case 5 - has filter with prefix and tags, here we are not matching on tags
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Filter>
|
||||
<And><Prefix>key-prefix</Prefix><Tag><Key>key1</Key><Value>value1</Value></Tag><Tag><Key>key2</Key><Value>value2</Value></Tag></And></Filter><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination></Rule></ReplicationConfiguration>`,
|
||||
prefix: "testdir/",
|
||||
expectedNonRec: true,
|
||||
expectedRec: true,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tc := range testCases {
|
||||
tc := tc
|
||||
t.Run(fmt.Sprintf("Test_%d", i+1), func(t *testing.T) {
|
||||
cfg, err := ParseConfig(bytes.NewReader([]byte(tc.inputConfig)))
|
||||
if err != nil {
|
||||
t.Fatalf("Got unexpected error: %v", err)
|
||||
}
|
||||
if got := cfg.HasActiveRules(tc.prefix, false); got != tc.expectedNonRec {
|
||||
t.Fatalf("Expected result with recursive set to false: `%v`, got: `%v`", tc.expectedNonRec, got)
|
||||
}
|
||||
if got := cfg.HasActiveRules(tc.prefix, true); got != tc.expectedRec {
|
||||
t.Fatalf("Expected result with recursive set to true: `%v`, got: `%v`", tc.expectedRec, got)
|
||||
}
|
||||
|
||||
})
|
||||
|
||||
}
|
||||
}
|
||||
213
internal/bucket/replication/rule.go
Normal file
213
internal/bucket/replication/rule.go
Normal file
@@ -0,0 +1,213 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/xml"
|
||||
)
|
||||
|
||||
// Status represents Enabled/Disabled status
|
||||
type Status string
|
||||
|
||||
// Supported status types
|
||||
const (
|
||||
Enabled Status = "Enabled"
|
||||
Disabled Status = "Disabled"
|
||||
)
|
||||
|
||||
// DeleteMarkerReplication - whether delete markers are replicated - https://docs.aws.amazon.com/AmazonS3/latest/dev/replication-add-config.html
|
||||
type DeleteMarkerReplication struct {
|
||||
Status Status `xml:"Status"` // should be set to "Disabled" by default
|
||||
}
|
||||
|
||||
// IsEmpty returns true if DeleteMarkerReplication is not set
|
||||
func (d DeleteMarkerReplication) IsEmpty() bool {
|
||||
return len(d.Status) == 0
|
||||
}
|
||||
|
||||
// Validate validates whether the status is disabled.
|
||||
func (d DeleteMarkerReplication) Validate() error {
|
||||
if d.IsEmpty() {
|
||||
return errDeleteMarkerReplicationMissing
|
||||
}
|
||||
if d.Status != Disabled && d.Status != Enabled {
|
||||
return errInvalidDeleteMarkerReplicationStatus
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// DeleteReplication - whether versioned deletes are replicated - this is a MinIO only
|
||||
// extension.
|
||||
type DeleteReplication struct {
|
||||
Status Status `xml:"Status"` // should be set to "Disabled" by default
|
||||
}
|
||||
|
||||
// IsEmpty returns true if DeleteReplication is not set
|
||||
func (d DeleteReplication) IsEmpty() bool {
|
||||
return len(d.Status) == 0
|
||||
}
|
||||
|
||||
// Validate validates whether the status is disabled.
|
||||
func (d DeleteReplication) Validate() error {
|
||||
if d.IsEmpty() {
|
||||
return errDeleteReplicationMissing
|
||||
}
|
||||
if d.Status != Disabled && d.Status != Enabled {
|
||||
return errInvalidDeleteReplicationStatus
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (d *DeleteReplication) UnmarshalXML(dec *xml.Decoder, start xml.StartElement) (err error) {
|
||||
// Make subtype to avoid recursive UnmarshalXML().
|
||||
type deleteReplication DeleteReplication
|
||||
drep := deleteReplication{}
|
||||
|
||||
if err := dec.DecodeElement(&drep, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
if len(drep.Status) == 0 {
|
||||
drep.Status = Disabled
|
||||
}
|
||||
d.Status = drep.Status
|
||||
return nil
|
||||
}
|
||||
|
||||
// Rule - a rule for replication configuration.
|
||||
type Rule struct {
|
||||
XMLName xml.Name `xml:"Rule" json:"Rule"`
|
||||
ID string `xml:"ID,omitempty" json:"ID,omitempty"`
|
||||
Status Status `xml:"Status" json:"Status"`
|
||||
Priority int `xml:"Priority" json:"Priority"`
|
||||
DeleteMarkerReplication DeleteMarkerReplication `xml:"DeleteMarkerReplication" json:"DeleteMarkerReplication"`
|
||||
// MinIO extension to replicate versioned deletes
|
||||
DeleteReplication DeleteReplication `xml:"DeleteReplication" json:"DeleteReplication"`
|
||||
Destination Destination `xml:"Destination" json:"Destination"`
|
||||
SourceSelectionCriteria SourceSelectionCriteria `xml:"SourceSelectionCriteria" json:"SourceSelectionCriteria"`
|
||||
Filter Filter `xml:"Filter" json:"Filter"`
|
||||
}
|
||||
|
||||
var (
|
||||
errInvalidRuleID = Errorf("ID must be less than 255 characters")
|
||||
errEmptyRuleStatus = Errorf("Status should not be empty")
|
||||
errInvalidRuleStatus = Errorf("Status must be set to either Enabled or Disabled")
|
||||
errDeleteMarkerReplicationMissing = Errorf("DeleteMarkerReplication must be specified")
|
||||
errPriorityMissing = Errorf("Priority must be specified")
|
||||
errInvalidDeleteMarkerReplicationStatus = Errorf("Delete marker replication status is invalid")
|
||||
errDestinationSourceIdentical = Errorf("Destination bucket cannot be the same as the source bucket.")
|
||||
errDeleteReplicationMissing = Errorf("Delete replication must be specified")
|
||||
errInvalidDeleteReplicationStatus = Errorf("Delete replication is either enable|disable")
|
||||
)
|
||||
|
||||
// validateID - checks if ID is valid or not.
|
||||
func (r Rule) validateID() error {
|
||||
// cannot be longer than 255 characters
|
||||
if len(r.ID) > 255 {
|
||||
return errInvalidRuleID
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// validateStatus - checks if status is valid or not.
|
||||
func (r Rule) validateStatus() error {
|
||||
// Status can't be empty
|
||||
if len(r.Status) == 0 {
|
||||
return errEmptyRuleStatus
|
||||
}
|
||||
|
||||
// Status must be one of Enabled or Disabled
|
||||
if r.Status != Enabled && r.Status != Disabled {
|
||||
return errInvalidRuleStatus
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (r Rule) validateFilter() error {
|
||||
return r.Filter.Validate()
|
||||
}
|
||||
|
||||
// Prefix - a rule can either have prefix under <filter></filter> or under
|
||||
// <filter><and></and></filter>. This method returns the prefix from the
|
||||
// location where it is available
|
||||
func (r Rule) Prefix() string {
|
||||
if r.Filter.Prefix != "" {
|
||||
return r.Filter.Prefix
|
||||
}
|
||||
return r.Filter.And.Prefix
|
||||
}
|
||||
|
||||
// Tags - a rule can either have tag under <filter></filter> or under
|
||||
// <filter><and></and></filter>. This method returns all the tags from the
|
||||
// rule in the format tag1=value1&tag2=value2
|
||||
func (r Rule) Tags() string {
|
||||
if !r.Filter.Tag.IsEmpty() {
|
||||
return r.Filter.Tag.String()
|
||||
}
|
||||
if len(r.Filter.And.Tags) != 0 {
|
||||
var buf bytes.Buffer
|
||||
for _, t := range r.Filter.And.Tags {
|
||||
if buf.Len() > 0 {
|
||||
buf.WriteString("&")
|
||||
}
|
||||
buf.WriteString(t.String())
|
||||
}
|
||||
return buf.String()
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// Validate - validates the rule element
|
||||
func (r Rule) Validate(bucket string, sameTarget bool) error {
|
||||
if err := r.validateID(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.validateStatus(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.validateFilter(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.DeleteMarkerReplication.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.DeleteReplication.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := r.SourceSelectionCriteria.Validate(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if r.Priority < 0 {
|
||||
return errPriorityMissing
|
||||
}
|
||||
if r.Destination.Bucket == bucket && sameTarget {
|
||||
return errDestinationSourceIdentical
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// MetadataReplicate returns true if object is not a replica or in the case of replicas,
|
||||
// replica modification sync is enabled.
|
||||
func (r Rule) MetadataReplicate(obj ObjectOpts) bool {
|
||||
if !obj.Replica {
|
||||
return true
|
||||
}
|
||||
return obj.Replica && r.SourceSelectionCriteria.ReplicaModifications.Status == Enabled
|
||||
}
|
||||
68
internal/bucket/replication/rule_test.go
Normal file
68
internal/bucket/replication/rule_test.go
Normal file
@@ -0,0 +1,68 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMetadataReplicate(t *testing.T) {
|
||||
testCases := []struct {
|
||||
inputConfig string
|
||||
opts ObjectOpts
|
||||
expectedResult bool
|
||||
}{
|
||||
// case 1 - rule with replica modification enabled; not a replica
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination><SourceSelectionCriteria><ReplicaModifications><Status>Enabled</Status></ReplicaModifications></SourceSelectionCriteria></Rule></ReplicationConfiguration>`,
|
||||
opts: ObjectOpts{Name: "c1test", DeleteMarker: false, OpType: ObjectReplicationType, Replica: false}, //1. Replica mod sync enabled; not a replica
|
||||
expectedResult: true,
|
||||
},
|
||||
// case 2 - rule with replica modification disabled; a replica
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination><SourceSelectionCriteria><ReplicaModifications><Status>Disabled</Status></ReplicaModifications></SourceSelectionCriteria></Rule></ReplicationConfiguration>`,
|
||||
opts: ObjectOpts{Name: "c2test", DeleteMarker: false, OpType: ObjectReplicationType, Replica: true}, //1. Replica mod sync enabled; a replica
|
||||
expectedResult: false,
|
||||
},
|
||||
// case 3 - rule with replica modification disabled; not a replica
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination><SourceSelectionCriteria><ReplicaModifications><Status>Disabled</Status></ReplicaModifications></SourceSelectionCriteria></Rule></ReplicationConfiguration>`,
|
||||
opts: ObjectOpts{Name: "c2test", DeleteMarker: false, OpType: ObjectReplicationType, Replica: false}, //1. Replica mod sync disabled; not a replica
|
||||
expectedResult: true,
|
||||
},
|
||||
|
||||
// case 4 - rule with replica modification enabled; a replica
|
||||
{inputConfig: `<ReplicationConfiguration xmlns="http://s3.amazonaws.com/doc/2006-03-01/"><Role>arn:aws:iam::AcctID:role/role-name</Role><Rule><Status>Enabled</Status><DeleteMarkerReplication><Status>Disabled</Status></DeleteMarkerReplication><DeleteReplication><Status>Disabled</Status></DeleteReplication><Prefix>key-prefix</Prefix><Destination><Bucket>arn:aws:s3:::destinationbucket</Bucket></Destination><SourceSelectionCriteria><ReplicaModifications><Status>Enabled</Status></ReplicaModifications></SourceSelectionCriteria></Rule></ReplicationConfiguration>`,
|
||||
opts: ObjectOpts{Name: "c2test", DeleteMarker: false, OpType: MetadataReplicationType, Replica: true}, //1. Replica mod sync enabled; a replica
|
||||
expectedResult: true,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tc := range testCases {
|
||||
tc := tc
|
||||
t.Run(fmt.Sprintf("Test_%d", i+1), func(t *testing.T) {
|
||||
cfg, err := ParseConfig(bytes.NewReader([]byte(tc.inputConfig)))
|
||||
if err != nil {
|
||||
t.Fatalf("Got unexpected error: %v", err)
|
||||
}
|
||||
if got := cfg.Rules[0].MetadataReplicate(tc.opts); got != tc.expectedResult {
|
||||
t.Fatalf("Expected result with recursive set to false: `%v`, got: `%v`", tc.expectedResult, got)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
}
|
||||
76
internal/bucket/replication/sourceselectioncriteria.go
Normal file
76
internal/bucket/replication/sourceselectioncriteria.go
Normal file
@@ -0,0 +1,76 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
)
|
||||
|
||||
// ReplicaModifications specifies if replica modification sync is enabled
|
||||
type ReplicaModifications struct {
|
||||
Status Status `xml:"Status" json:"Status"`
|
||||
}
|
||||
|
||||
// SourceSelectionCriteria - specifies additional source selection criteria in ReplicationConfiguration.
|
||||
type SourceSelectionCriteria struct {
|
||||
ReplicaModifications ReplicaModifications `xml:"ReplicaModifications" json:"ReplicaModifications"`
|
||||
}
|
||||
|
||||
// IsValid - checks whether SourceSelectionCriteria is valid or not.
|
||||
func (s SourceSelectionCriteria) IsValid() bool {
|
||||
return s.ReplicaModifications.Status == Enabled || s.ReplicaModifications.Status == Disabled
|
||||
}
|
||||
|
||||
// Validate source selection criteria
|
||||
func (s SourceSelectionCriteria) Validate() error {
|
||||
if (s == SourceSelectionCriteria{}) {
|
||||
return nil
|
||||
}
|
||||
if !s.IsValid() {
|
||||
return errInvalidSourceSelectionCriteria
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// UnmarshalXML - decodes XML data.
|
||||
func (s *SourceSelectionCriteria) UnmarshalXML(dec *xml.Decoder, start xml.StartElement) (err error) {
|
||||
// Make subtype to avoid recursive UnmarshalXML().
|
||||
type sourceSelectionCriteria SourceSelectionCriteria
|
||||
ssc := sourceSelectionCriteria{}
|
||||
if err := dec.DecodeElement(&ssc, &start); err != nil {
|
||||
return err
|
||||
}
|
||||
if len(ssc.ReplicaModifications.Status) == 0 {
|
||||
ssc.ReplicaModifications.Status = Enabled
|
||||
}
|
||||
*s = SourceSelectionCriteria(ssc)
|
||||
return nil
|
||||
}
|
||||
|
||||
// MarshalXML - encodes to XML data.
|
||||
func (s SourceSelectionCriteria) MarshalXML(e *xml.Encoder, start xml.StartElement) error {
|
||||
if err := e.EncodeToken(start); err != nil {
|
||||
return err
|
||||
}
|
||||
if s.IsValid() {
|
||||
if err := e.EncodeElement(s.ReplicaModifications, xml.StartElement{Name: xml.Name{Local: "ReplicaModifications"}}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return e.EncodeToken(xml.EndElement{Name: start.Name})
|
||||
}
|
||||
57
internal/bucket/replication/tag.go
Normal file
57
internal/bucket/replication/tag.go
Normal file
@@ -0,0 +1,57 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package replication
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"unicode/utf8"
|
||||
)
|
||||
|
||||
// Tag - a tag for a replication configuration Rule filter.
|
||||
type Tag struct {
|
||||
XMLName xml.Name `xml:"Tag" json:"Tag"`
|
||||
Key string `xml:"Key,omitempty" json:"Key,omitempty"`
|
||||
Value string `xml:"Value,omitempty" json:"Value,omitempty"`
|
||||
}
|
||||
|
||||
var (
|
||||
errInvalidTagKey = Errorf("The TagKey you have provided is invalid")
|
||||
errInvalidTagValue = Errorf("The TagValue you have provided is invalid")
|
||||
)
|
||||
|
||||
func (tag Tag) String() string {
|
||||
return tag.Key + "=" + tag.Value
|
||||
}
|
||||
|
||||
// IsEmpty returns whether this tag is empty or not.
|
||||
func (tag Tag) IsEmpty() bool {
|
||||
return tag.Key == ""
|
||||
}
|
||||
|
||||
// Validate checks this tag.
|
||||
func (tag Tag) Validate() error {
|
||||
if len(tag.Key) == 0 || utf8.RuneCountInString(tag.Key) > 128 {
|
||||
return errInvalidTagKey
|
||||
}
|
||||
|
||||
if utf8.RuneCountInString(tag.Value) > 256 {
|
||||
return errInvalidTagValue
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
45
internal/bucket/versioning/error.go
Normal file
45
internal/bucket/versioning/error.go
Normal file
@@ -0,0 +1,45 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package versioning
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
// Error is the generic type for any error happening during tag
|
||||
// parsing.
|
||||
type Error struct {
|
||||
err error
|
||||
}
|
||||
|
||||
// Errorf - formats according to a format specifier and returns
|
||||
// the string as a value that satisfies error of type tagging.Error
|
||||
func Errorf(format string, a ...interface{}) error {
|
||||
return Error{err: fmt.Errorf(format, a...)}
|
||||
}
|
||||
|
||||
// Unwrap the internal error.
|
||||
func (e Error) Unwrap() error { return e.err }
|
||||
|
||||
// Error 'error' compatible method.
|
||||
func (e Error) Error() string {
|
||||
if e.err == nil {
|
||||
return "versioning: cause <nil>"
|
||||
}
|
||||
return e.err.Error()
|
||||
}
|
||||
80
internal/bucket/versioning/versioning.go
Normal file
80
internal/bucket/versioning/versioning.go
Normal file
@@ -0,0 +1,80 @@
|
||||
// Copyright (c) 2015-2021 MinIO, Inc.
|
||||
//
|
||||
// This file is part of MinIO Object Storage stack
|
||||
//
|
||||
// This program is free software: you can redistribute it and/or modify
|
||||
// it under the terms of the GNU Affero General Public License as published by
|
||||
// the Free Software Foundation, either version 3 of the License, or
|
||||
// (at your option) any later version.
|
||||
//
|
||||
// This program is distributed in the hope that it will be useful
|
||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
// GNU Affero General Public License for more details.
|
||||
//
|
||||
// You should have received a copy of the GNU Affero General Public License
|
||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
||||
|
||||
package versioning
|
||||
|
||||
import (
|
||||
"encoding/xml"
|
||||
"io"
|
||||
)
|
||||
|
||||
// State - enabled/disabled/suspended states
|
||||
// for multifactor and status of versioning.
|
||||
type State string
|
||||
|
||||
// Various supported states
|
||||
const (
|
||||
Enabled State = "Enabled"
|
||||
// Disabled State = "Disabled" only used by MFA Delete not supported yet.
|
||||
Suspended State = "Suspended"
|
||||
)
|
||||
|
||||
// Versioning - Configuration for bucket versioning.
|
||||
type Versioning struct {
|
||||
XMLNS string `xml:"xmlns,attr,omitempty"`
|
||||
XMLName xml.Name `xml:"VersioningConfiguration"`
|
||||
// MFADelete State `xml:"MFADelete,omitempty"` // not supported yet.
|
||||
Status State `xml:"Status,omitempty"`
|
||||
}
|
||||
|
||||
// Validate - validates the versioning configuration
|
||||
func (v Versioning) Validate() error {
|
||||
// Not supported yet
|
||||
// switch v.MFADelete {
|
||||
// case Enabled, Disabled:
|
||||
// default:
|
||||
// return Errorf("unsupported MFADelete state %s", v.MFADelete)
|
||||
// }
|
||||
switch v.Status {
|
||||
case Enabled, Suspended:
|
||||
default:
|
||||
return Errorf("unsupported Versioning status %s", v.Status)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Enabled - returns true if versioning is enabled
|
||||
func (v Versioning) Enabled() bool {
|
||||
return v.Status == Enabled
|
||||
}
|
||||
|
||||
// Suspended - returns true if versioning is suspended
|
||||
func (v Versioning) Suspended() bool {
|
||||
return v.Status == Suspended
|
||||
}
|
||||
|
||||
// ParseConfig - parses data in given reader to VersioningConfiguration.
|
||||
func ParseConfig(reader io.Reader) (*Versioning, error) {
|
||||
var v Versioning
|
||||
if err := xml.NewDecoder(reader).Decode(&v); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := v.Validate(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &v, nil
|
||||
}
|
||||
Reference in New Issue
Block a user