update sync controller for backup locations

Signed-off-by: Steve Kriss <steve@heptio.com>
pull/799/head
Steve Kriss 2018-08-21 16:52:49 -07:00
parent 2750aa71b9
commit 0e94fa37f9
9 changed files with 382 additions and 535 deletions

View File

@ -36,4 +36,8 @@ const (
// a backup/restore-specific timeout value for pod volume operations (i.e. // a backup/restore-specific timeout value for pod volume operations (i.e.
// restic backups/restores). // restic backups/restores).
PodVolumeOperationTimeoutAnnotation = "ark.heptio.com/pod-volume-timeout" PodVolumeOperationTimeoutAnnotation = "ark.heptio.com/pod-volume-timeout"
// StorageLocationLabel is the label key used to identify the storage
// location of a backup.
StorageLocationLabel = "ark.heptio.com/storage-location"
) )

View File

@ -1,97 +0,0 @@
/*
Copyright 2017 the Heptio Ark contributors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package cloudprovider
import (
"context"
"sync"
"time"
"github.com/sirupsen/logrus"
"k8s.io/apimachinery/pkg/util/wait"
"github.com/heptio/ark/pkg/apis/ark/v1"
)
// backupCacheBucket holds the backups and error from a ListBackups call.
type backupCacheBucket struct {
backups []*v1.Backup
error error
}
// backupCache caches ListBackups calls, refreshing them periodically.
type backupCache struct {
delegate BackupLister
lock sync.RWMutex
// This doesn't really need to be a map right now, but if we ever move to supporting multiple
// buckets, this will be ready for it.
buckets map[string]*backupCacheBucket
logger logrus.FieldLogger
}
var _ BackupLister = &backupCache{}
// NewBackupCache returns a new backup cache that refreshes from delegate every resyncPeriod.
func NewBackupCache(ctx context.Context, delegate BackupLister, resyncPeriod time.Duration, logger logrus.FieldLogger) BackupLister {
c := &backupCache{
delegate: delegate,
buckets: make(map[string]*backupCacheBucket),
logger: logger,
}
// Start the goroutine to refresh all buckets every resyncPeriod. This stops when ctx.Done() is
// available.
go wait.Until(c.refresh, resyncPeriod, ctx.Done())
return c
}
// refresh refreshes all the buckets currently in the cache by doing a live lookup via c.delegate.
func (c *backupCache) refresh() {
c.lock.Lock()
defer c.lock.Unlock()
c.logger.Debug("refreshing all cached backup lists from object storage")
for bucketName, bucket := range c.buckets {
c.logger.WithField("bucket", bucketName).Debug("Refreshing bucket")
bucket.backups, bucket.error = c.delegate.ListBackups(bucketName)
}
}
func (c *backupCache) ListBackups(bucketName string) ([]*v1.Backup, error) {
c.lock.RLock()
bucket, found := c.buckets[bucketName]
c.lock.RUnlock()
logContext := c.logger.WithField("bucket", bucketName)
if found {
logContext.Debug("Returning cached backup list")
return bucket.backups, bucket.error
}
logContext.Debug("Bucket is not in cache - doing a live lookup")
backups, err := c.delegate.ListBackups(bucketName)
c.lock.Lock()
c.buckets[bucketName] = &backupCacheBucket{backups: backups, error: err}
c.lock.Unlock()
return backups, err
}

View File

@ -1,170 +0,0 @@
/*
Copyright 2017 the Heptio Ark contributors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package cloudprovider
import (
"context"
"errors"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/heptio/ark/pkg/apis/ark/v1"
cloudprovidermocks "github.com/heptio/ark/pkg/cloudprovider/mocks"
"github.com/heptio/ark/pkg/util/test"
)
func TestNewBackupCache(t *testing.T) {
var (
delegate = &cloudprovidermocks.BackupLister{}
ctx, cancel = context.WithTimeout(context.Background(), 5*time.Second)
logger = test.NewLogger()
)
defer cancel()
c := NewBackupCache(ctx, delegate, 100*time.Millisecond, logger)
// nothing in cache, live lookup
bucket1 := []*v1.Backup{
test.NewTestBackup().WithName("backup1").Backup,
test.NewTestBackup().WithName("backup2").Backup,
}
delegate.On("ListBackups", "bucket1").Return(bucket1, nil).Once()
// should be updated via refresh
updatedBucket1 := []*v1.Backup{
test.NewTestBackup().WithName("backup2").Backup,
}
delegate.On("ListBackups", "bucket1").Return(updatedBucket1, nil)
// nothing in cache, live lookup
bucket2 := []*v1.Backup{
test.NewTestBackup().WithName("backup5").Backup,
test.NewTestBackup().WithName("backup6").Backup,
}
delegate.On("ListBackups", "bucket2").Return(bucket2, nil).Once()
// should be updated via refresh
updatedBucket2 := []*v1.Backup{
test.NewTestBackup().WithName("backup7").Backup,
}
delegate.On("ListBackups", "bucket2").Return(updatedBucket2, nil)
backups, err := c.ListBackups("bucket1")
assert.Equal(t, bucket1, backups)
assert.NoError(t, err)
backups, err = c.ListBackups("bucket2")
assert.Equal(t, bucket2, backups)
assert.NoError(t, err)
var done1, done2 bool
for {
select {
case <-ctx.Done():
t.Fatal("timed out")
default:
if done1 && done2 {
return
}
}
backups, err = c.ListBackups("bucket1")
if len(backups) == 1 {
if assert.Equal(t, updatedBucket1[0], backups[0]) {
done1 = true
}
}
backups, err = c.ListBackups("bucket2")
if len(backups) == 1 {
if assert.Equal(t, updatedBucket2[0], backups[0]) {
done2 = true
}
}
time.Sleep(100 * time.Millisecond)
}
}
func TestBackupCacheRefresh(t *testing.T) {
var (
delegate = &cloudprovidermocks.BackupLister{}
logger = test.NewLogger()
)
c := &backupCache{
delegate: delegate,
buckets: map[string]*backupCacheBucket{
"bucket1": {},
"bucket2": {},
},
logger: logger,
}
bucket1 := []*v1.Backup{
test.NewTestBackup().WithName("backup1").Backup,
test.NewTestBackup().WithName("backup2").Backup,
}
delegate.On("ListBackups", "bucket1").Return(bucket1, nil)
delegate.On("ListBackups", "bucket2").Return(nil, errors.New("bad"))
c.refresh()
assert.Equal(t, bucket1, c.buckets["bucket1"].backups)
assert.NoError(t, c.buckets["bucket1"].error)
assert.Empty(t, c.buckets["bucket2"].backups)
assert.EqualError(t, c.buckets["bucket2"].error, "bad")
}
func TestBackupCacheGetAllBackupsUsesCacheIfPresent(t *testing.T) {
var (
delegate = &cloudprovidermocks.BackupLister{}
logger = test.NewLogger()
bucket1 = []*v1.Backup{
test.NewTestBackup().WithName("backup1").Backup,
test.NewTestBackup().WithName("backup2").Backup,
}
)
c := &backupCache{
delegate: delegate,
buckets: map[string]*backupCacheBucket{
"bucket1": {
backups: bucket1,
},
},
logger: logger,
}
bucket2 := []*v1.Backup{
test.NewTestBackup().WithName("backup3").Backup,
test.NewTestBackup().WithName("backup4").Backup,
}
delegate.On("ListBackups", "bucket2").Return(bucket2, nil)
backups, err := c.ListBackups("bucket1")
assert.Equal(t, bucket1, backups)
assert.NoError(t, err)
backups, err = c.ListBackups("bucket2")
assert.Equal(t, bucket2, backups)
assert.NoError(t, err)
}

View File

@ -147,22 +147,6 @@ func DownloadBackup(objectStore ObjectStore, bucket, backupName string) (io.Read
return objectStore.GetObject(bucket, getBackupContentsKey(backupName, backupName)) return objectStore.GetObject(bucket, getBackupContentsKey(backupName, backupName))
} }
type liveBackupLister struct {
logger logrus.FieldLogger
objectStore ObjectStore
}
func NewLiveBackupLister(logger logrus.FieldLogger, objectStore ObjectStore) BackupLister {
return &liveBackupLister{
logger: logger,
objectStore: objectStore,
}
}
func (l *liveBackupLister) ListBackups(bucket string) ([]*api.Backup, error) {
return ListBackups(l.logger, l.objectStore, bucket)
}
func ListBackups(logger logrus.FieldLogger, objectStore ObjectStore, bucket string) ([]*api.Backup, error) { func ListBackups(logger logrus.FieldLogger, objectStore ObjectStore, bucket string) ([]*api.Backup, error) {
prefixes, err := objectStore.ListCommonPrefixes(bucket, "/") prefixes, err := objectStore.ListCommonPrefixes(bucket, "/")
if err != nil { if err != nil {

View File

@ -589,12 +589,6 @@ func (s *server) runControllers(config *api.Config, defaultBackupLocation *api.B
ctx := s.ctx ctx := s.ctx
var wg sync.WaitGroup var wg sync.WaitGroup
cloudBackupCacheResyncPeriod := durationMin(controller.GCSyncPeriod, s.config.backupSyncPeriod)
s.logger.Infof("Caching cloud backups every %s", cloudBackupCacheResyncPeriod)
liveBackupLister := cloudprovider.NewLiveBackupLister(s.logger, s.objectStore)
cachedBackupLister := cloudprovider.NewBackupCache(ctx, liveBackupLister, cloudBackupCacheResyncPeriod, s.logger)
go func() { go func() {
metricsMux := http.NewServeMux() metricsMux := http.NewServeMux()
metricsMux.Handle("/metrics", promhttp.Handler()) metricsMux.Handle("/metrics", promhttp.Handler())
@ -608,12 +602,13 @@ func (s *server) runControllers(config *api.Config, defaultBackupLocation *api.B
backupSyncController := controller.NewBackupSyncController( backupSyncController := controller.NewBackupSyncController(
s.arkClient.ArkV1(), s.arkClient.ArkV1(),
cachedBackupLister, s.sharedInformerFactory.Ark().V1().Backups(),
config.BackupStorageProvider.Bucket, s.sharedInformerFactory.Ark().V1().BackupStorageLocations(),
s.config.backupSyncPeriod, s.config.backupSyncPeriod,
s.namespace, s.namespace,
s.sharedInformerFactory.Ark().V1().Backups(), s.pluginRegistry,
s.logger, s.logger,
s.logLevel,
) )
wg.Add(1) wg.Add(1)
go func() { go func() {
@ -775,7 +770,7 @@ func (s *server) runControllers(config *api.Config, defaultBackupLocation *api.B
// SHARED INFORMERS HAVE TO BE STARTED AFTER ALL CONTROLLERS // SHARED INFORMERS HAVE TO BE STARTED AFTER ALL CONTROLLERS
go s.sharedInformerFactory.Start(ctx.Done()) go s.sharedInformerFactory.Start(ctx.Done())
// Remove this sometime after v0.8.0 // TODO(1.0): remove
cache.WaitForCacheSync(ctx.Done(), s.sharedInformerFactory.Ark().V1().Backups().Informer().HasSynced) cache.WaitForCacheSync(ctx.Done(), s.sharedInformerFactory.Ark().V1().Backups().Informer().HasSynced)
s.removeDeprecatedGCFinalizer() s.removeDeprecatedGCFinalizer()
@ -789,9 +784,10 @@ func (s *server) runControllers(config *api.Config, defaultBackupLocation *api.B
return nil return nil
} }
const gcFinalizer = "gc.ark.heptio.com" // TODO(1.0): remove
func (s *server) removeDeprecatedGCFinalizer() { func (s *server) removeDeprecatedGCFinalizer() {
const gcFinalizer = "gc.ark.heptio.com"
backups, err := s.sharedInformerFactory.Ark().V1().Backups().Lister().List(labels.Everything()) backups, err := s.sharedInformerFactory.Ark().V1().Backups().Lister().List(labels.Everything())
if err != nil { if err != nil {
s.logger.WithError(errors.WithStack(err)).Error("error listing backups from cache - unable to remove old finalizers") s.logger.WithError(errors.WithStack(err)).Error("error listing backups from cache - unable to remove old finalizers")

View File

@ -349,6 +349,12 @@ func (controller *backupController) getLocationAndValidate(itm *api.Backup, defa
itm.Spec.StorageLocation = defaultBackupLocation itm.Spec.StorageLocation = defaultBackupLocation
} }
// add the storage location as a label for easy filtering later.
if itm.Labels == nil {
itm.Labels = make(map[string]string)
}
itm.Labels[api.StorageLocationLabel] = itm.Spec.StorageLocation
var backupLocation *api.BackupStorageLocation var backupLocation *api.BackupStorageLocation
backupLocation, err := controller.backupLocationLister.BackupStorageLocations(itm.Namespace).Get(itm.Spec.StorageLocation) backupLocation, err := controller.backupLocationLister.BackupStorageLocations(itm.Namespace).Get(itm.Spec.StorageLocation)
if err != nil { if err != nil {

View File

@ -25,13 +25,12 @@ import (
"testing" "testing"
"time" "time"
"github.com/sirupsen/logrus"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime" "k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/util/clock" "k8s.io/apimachinery/pkg/util/clock"
core "k8s.io/client-go/testing" core "k8s.io/client-go/testing"
"github.com/sirupsen/logrus"
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -364,10 +363,14 @@ func TestProcessBackup(t *testing.T) {
type SpecPatch struct { type SpecPatch struct {
StorageLocation string `json:"storageLocation"` StorageLocation string `json:"storageLocation"`
} }
type ObjectMetaPatch struct {
Labels map[string]string `json:"labels"`
}
type Patch struct { type Patch struct {
Status StatusPatch `json:"status"` Status StatusPatch `json:"status"`
Spec SpecPatch `json:"spec,omitempty"` Spec SpecPatch `json:"spec,omitempty"`
ObjectMeta ObjectMetaPatch `json:"metadata,omitempty"`
} }
decode := func(decoder *json.Decoder) (interface{}, error) { decode := func(decoder *json.Decoder) (interface{}, error) {
@ -389,6 +392,11 @@ func TestProcessBackup(t *testing.T) {
Spec: SpecPatch{ Spec: SpecPatch{
StorageLocation: "default", StorageLocation: "default",
}, },
ObjectMeta: ObjectMetaPatch{
Labels: map[string]string{
v1.StorageLocationLabel: "default",
},
},
} }
} else { } else {
expected = Patch{ expected = Patch{
@ -397,6 +405,11 @@ func TestProcessBackup(t *testing.T) {
Phase: v1.BackupPhaseInProgress, Phase: v1.BackupPhaseInProgress,
Expiration: expiration, Expiration: expiration,
}, },
ObjectMeta: ObjectMetaPatch{
Labels: map[string]string{
v1.StorageLocationLabel: test.backup.Spec.StorageLocation,
},
},
} }
} }

View File

@ -17,7 +17,6 @@ limitations under the License.
package controller package controller
import ( import (
"context"
"time" "time"
"github.com/pkg/errors" "github.com/pkg/errors"
@ -27,133 +26,172 @@ import (
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/labels" "k8s.io/apimachinery/pkg/labels"
"k8s.io/apimachinery/pkg/util/sets" "k8s.io/apimachinery/pkg/util/sets"
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/client-go/tools/cache" "k8s.io/client-go/tools/cache"
api "github.com/heptio/ark/pkg/apis/ark/v1" arkv1api "github.com/heptio/ark/pkg/apis/ark/v1"
"github.com/heptio/ark/pkg/cloudprovider" "github.com/heptio/ark/pkg/cloudprovider"
arkv1client "github.com/heptio/ark/pkg/generated/clientset/versioned/typed/ark/v1" arkv1client "github.com/heptio/ark/pkg/generated/clientset/versioned/typed/ark/v1"
informers "github.com/heptio/ark/pkg/generated/informers/externalversions/ark/v1" informers "github.com/heptio/ark/pkg/generated/informers/externalversions/ark/v1"
listers "github.com/heptio/ark/pkg/generated/listers/ark/v1" listers "github.com/heptio/ark/pkg/generated/listers/ark/v1"
"github.com/heptio/ark/pkg/plugin"
"github.com/heptio/ark/pkg/util/kube" "github.com/heptio/ark/pkg/util/kube"
"github.com/heptio/ark/pkg/util/stringslice" "github.com/heptio/ark/pkg/util/stringslice"
) )
type backupSyncController struct { type backupSyncController struct {
client arkv1client.BackupsGetter *genericController
cloudBackupLister cloudprovider.BackupLister
bucket string client arkv1client.BackupsGetter
syncPeriod time.Duration backupLister listers.BackupLister
namespace string backupStorageLocationLister listers.BackupStorageLocationLister
backupLister listers.BackupLister namespace string
backupInformerSynced cache.InformerSynced newPluginManager func(logrus.FieldLogger) plugin.Manager
logger logrus.FieldLogger listCloudBackups func(logrus.FieldLogger, cloudprovider.ObjectStore, string) ([]*arkv1api.Backup, error)
} }
func NewBackupSyncController( func NewBackupSyncController(
client arkv1client.BackupsGetter, client arkv1client.BackupsGetter,
cloudBackupLister cloudprovider.BackupLister, backupInformer informers.BackupInformer,
bucket string, backupStorageLocationInformer informers.BackupStorageLocationInformer,
syncPeriod time.Duration, syncPeriod time.Duration,
namespace string, namespace string,
backupInformer informers.BackupInformer, pluginRegistry plugin.Registry,
logger logrus.FieldLogger, logger logrus.FieldLogger,
logLevel logrus.Level,
) Interface { ) Interface {
if syncPeriod < time.Minute { if syncPeriod < time.Minute {
logger.Infof("Provided backup sync period %v is too short. Setting to 1 minute", syncPeriod) logger.Infof("Provided backup sync period %v is too short. Setting to 1 minute", syncPeriod)
syncPeriod = time.Minute syncPeriod = time.Minute
} }
return &backupSyncController{
client: client,
cloudBackupLister: cloudBackupLister,
bucket: bucket,
syncPeriod: syncPeriod,
namespace: namespace,
backupLister: backupInformer.Lister(),
backupInformerSynced: backupInformer.Informer().HasSynced,
logger: logger,
}
}
// Run is a blocking function that continually runs the object storage -> Ark API c := &backupSyncController{
// sync process according to the controller's syncPeriod. It will return when it genericController: newGenericController("backup-sync", logger),
// receives on the ctx.Done() channel. client: client,
func (c *backupSyncController) Run(ctx context.Context, workers int) error { namespace: namespace,
c.logger.Info("Running backup sync controller") backupLister: backupInformer.Lister(),
c.logger.Info("Waiting for caches to sync") backupStorageLocationLister: backupStorageLocationInformer.Lister(),
if !cache.WaitForCacheSync(ctx.Done(), c.backupInformerSynced) {
return errors.New("timed out waiting for caches to sync") newPluginManager: func(logger logrus.FieldLogger) plugin.Manager {
return plugin.NewManager(logger, logLevel, pluginRegistry)
},
listCloudBackups: cloudprovider.ListBackups,
} }
c.logger.Info("Caches are synced")
wait.Until(c.run, c.syncPeriod, ctx.Done()) c.resyncFunc = c.run
return nil c.resyncPeriod = syncPeriod
c.cacheSyncWaiters = []cache.InformerSynced{
backupInformer.Informer().HasSynced,
backupStorageLocationInformer.Informer().HasSynced,
}
return c
} }
const gcFinalizer = "gc.ark.heptio.com" const gcFinalizer = "gc.ark.heptio.com"
func (c *backupSyncController) run() { func (c *backupSyncController) run() {
c.logger.Info("Syncing backups from object storage") c.logger.Info("Syncing backups from backup storage into cluster")
backups, err := c.cloudBackupLister.ListBackups(c.bucket)
locations, err := c.backupStorageLocationLister.BackupStorageLocations(c.namespace).List(labels.Everything())
if err != nil { if err != nil {
c.logger.WithError(err).Error("error listing backups") c.logger.WithError(errors.WithStack(err)).Error("Error getting backup storage locations from lister")
return return
} }
c.logger.WithField("backupCount", len(backups)).Info("Got backups from object storage")
cloudBackupNames := sets.NewString() pluginManager := c.newPluginManager(c.logger)
for _, cloudBackup := range backups {
logContext := c.logger.WithField("backup", kube.NamespaceAndName(cloudBackup))
logContext.Info("Syncing backup")
cloudBackupNames.Insert(cloudBackup.Name) for _, location := range locations {
log := c.logger.WithField("backupLocation", location.Name)
log.Info("Syncing backups from backup location")
// If we're syncing backups made by pre-0.8.0 versions, the server removes all finalizers objectStore, err := getObjectStoreForLocation(location, pluginManager)
// faster than the sync finishes. Just process them as we find them.
cloudBackup.Finalizers = stringslice.Except(cloudBackup.Finalizers, gcFinalizer)
cloudBackup.Namespace = c.namespace
cloudBackup.ResourceVersion = ""
// Backup only if backup does not exist in Kubernetes or if we are not able to get the backup for any reason.
_, err := c.client.Backups(cloudBackup.Namespace).Get(cloudBackup.Name, metav1.GetOptions{})
if err != nil { if err != nil {
log.WithError(err).Error("Error getting object store for location")
continue
}
backupsInBackupStore, err := c.listCloudBackups(log, objectStore, location.Spec.ObjectStorage.Bucket)
if err != nil {
log.WithError(err).Error("Error listing backups in object store")
continue
}
log.WithField("backupCount", len(backupsInBackupStore)).Info("Got backups from object store")
cloudBackupNames := sets.NewString()
for _, cloudBackup := range backupsInBackupStore {
log = log.WithField("backup", kube.NamespaceAndName(cloudBackup))
log.Debug("Checking cloud backup to see if it needs to be synced into the cluster")
cloudBackupNames.Insert(cloudBackup.Name)
// use the controller's namespace when getting the backup because that's where we
// are syncing backups to, regardless of the namespace of the cloud backup.
_, err := c.client.Backups(c.namespace).Get(cloudBackup.Name, metav1.GetOptions{})
if err == nil {
log.Debug("Backup already exists in cluster")
continue
}
if !kuberrs.IsNotFound(err) { if !kuberrs.IsNotFound(err) {
logContext.WithError(errors.WithStack(err)).Error("Error getting backup from client, proceeding with backup sync") log.WithError(errors.WithStack(err)).Error("Error getting backup from client, proceeding with sync into cluster")
} }
if _, err := c.client.Backups(cloudBackup.Namespace).Create(cloudBackup); err != nil && !kuberrs.IsAlreadyExists(err) { // remove the pre-v0.8.0 gcFinalizer if it exists
logContext.WithError(errors.WithStack(err)).Error("Error syncing backup from object storage") // TODO(1.0): remove this
cloudBackup.Finalizers = stringslice.Except(cloudBackup.Finalizers, gcFinalizer)
cloudBackup.Namespace = c.namespace
cloudBackup.ResourceVersion = ""
// update the StorageLocation field and label since the name of the location
// may be different in this cluster than in the cluster that created the
// backup.
cloudBackup.Spec.StorageLocation = location.Name
if cloudBackup.Labels == nil {
cloudBackup.Labels = make(map[string]string)
}
cloudBackup.Labels[arkv1api.StorageLocationLabel] = cloudBackup.Spec.StorageLocation
_, err = c.client.Backups(cloudBackup.Namespace).Create(cloudBackup)
switch {
case err != nil && kuberrs.IsAlreadyExists(err):
log.Debug("Backup already exists in cluster")
case err != nil && !kuberrs.IsAlreadyExists(err):
log.WithError(errors.WithStack(err)).Error("Error syncing backup into cluster")
default:
log.Debug("Synced backup into cluster")
} }
} }
}
c.deleteUnused(cloudBackupNames) c.deleteOrphanedBackups(location.Name, cloudBackupNames, log)
return }
} }
// deleteUnused deletes backup objects from Kubernetes if they are complete // deleteOrphanedBackups deletes backup objects from Kubernetes that have the specified location
// and there is no corresponding backup in the object storage. // and a phase of Completed, but no corresponding backup in object storage.
func (c *backupSyncController) deleteUnused(cloudBackupNames sets.String) { func (c *backupSyncController) deleteOrphanedBackups(locationName string, cloudBackupNames sets.String, log logrus.FieldLogger) {
// Backups objects in Kubernetes locationSelector := labels.Set(map[string]string{
backups, err := c.backupLister.Backups(c.namespace).List(labels.Everything()) arkv1api.StorageLocationLabel: locationName,
}).AsSelector()
backups, err := c.backupLister.Backups(c.namespace).List(locationSelector)
if err != nil { if err != nil {
c.logger.WithError(errors.WithStack(err)).Error("Error listing backup from Kubernetes") log.WithError(errors.WithStack(err)).Error("Error listing backups from cluster")
return
} }
if len(backups) == 0 { if len(backups) == 0 {
return return
} }
// For each completed backup object in Kubernetes, delete it if it
// does not have a corresponding backup in object storage
for _, backup := range backups { for _, backup := range backups {
if backup.Status.Phase == api.BackupPhaseCompleted && !cloudBackupNames.Has(backup.Name) { log = log.WithField("backup", backup.Name)
if err := c.client.Backups(backup.Namespace).Delete(backup.Name, nil); err != nil { if backup.Status.Phase != arkv1api.BackupPhaseCompleted || cloudBackupNames.Has(backup.Name) {
c.logger.WithError(errors.WithStack(err)).Error("Error deleting unused backup from Kubernetes") continue
} else { }
c.logger.Debugf("Deleted backup: %s", backup.Name)
} if err := c.client.Backups(backup.Namespace).Delete(backup.Name, nil); err != nil {
log.WithError(errors.WithStack(err)).Error("Error deleting orphaned backup from cluster")
} else {
log.Debug("Deleted orphaned backup from cluster")
} }
} }
return
} }

View File

@ -20,281 +20,354 @@ import (
"testing" "testing"
"time" "time"
apierrors "k8s.io/apimachinery/pkg/api/errors" "github.com/pkg/errors"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
"github.com/sirupsen/logrus"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/util/sets" "k8s.io/apimachinery/pkg/util/sets"
core "k8s.io/client-go/testing" core "k8s.io/client-go/testing"
"github.com/heptio/ark/pkg/apis/ark/v1" arkv1api "github.com/heptio/ark/pkg/apis/ark/v1"
cloudprovidermocks "github.com/heptio/ark/pkg/cloudprovider/mocks" "github.com/heptio/ark/pkg/cloudprovider"
"github.com/heptio/ark/pkg/generated/clientset/versioned/fake" "github.com/heptio/ark/pkg/generated/clientset/versioned/fake"
informers "github.com/heptio/ark/pkg/generated/informers/externalversions" informers "github.com/heptio/ark/pkg/generated/informers/externalversions"
"github.com/heptio/ark/pkg/plugin"
pluginmocks "github.com/heptio/ark/pkg/plugin/mocks"
"github.com/heptio/ark/pkg/util/stringslice" "github.com/heptio/ark/pkg/util/stringslice"
arktest "github.com/heptio/ark/pkg/util/test" arktest "github.com/heptio/ark/pkg/util/test"
"github.com/pkg/errors"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
) )
func defaultLocationsList(namespace string) []*arkv1api.BackupStorageLocation {
return []*arkv1api.BackupStorageLocation{
{
ObjectMeta: metav1.ObjectMeta{
Namespace: namespace,
Name: "location-1",
},
Spec: arkv1api.BackupStorageLocationSpec{
Provider: "objStoreProvider",
StorageType: arkv1api.StorageType{
ObjectStorage: &arkv1api.ObjectStorageLocation{
Bucket: "bucket-1",
},
},
},
},
{
ObjectMeta: metav1.ObjectMeta{
Namespace: namespace,
Name: "location-2",
},
Spec: arkv1api.BackupStorageLocationSpec{
Provider: "objStoreProvider",
StorageType: arkv1api.StorageType{
ObjectStorage: &arkv1api.ObjectStorageLocation{
Bucket: "bucket-2",
},
},
},
},
}
}
func TestBackupSyncControllerRun(t *testing.T) { func TestBackupSyncControllerRun(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
listBackupsError error namespace string
cloudBackups []*v1.Backup locations []*arkv1api.BackupStorageLocation
namespace string cloudBackups map[string][]*arkv1api.Backup
existingBackups sets.String existingBackups []*arkv1api.Backup
}{ }{
{ {
name: "no cloud backups", name: "no cloud backups",
}, },
{ {
name: "backup lister returns error on ListBackups", name: "normal case",
listBackupsError: errors.New("listBackups"),
},
{
name: "normal case",
cloudBackups: []*v1.Backup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup,
},
namespace: "ns-1", namespace: "ns-1",
locations: defaultLocationsList("ns-1"),
cloudBackups: map[string][]*arkv1api.Backup{
"bucket-1": {
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
},
"bucket-2": {
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup,
},
},
}, },
{ {
name: "Finalizer gets removed on sync", name: "gcFinalizer (only) gets removed on sync",
cloudBackups: []*v1.Backup{
arktest.NewTestBackup().WithNamespace("ns-1").WithFinalizers(gcFinalizer).Backup,
},
namespace: "ns-1", namespace: "ns-1",
locations: defaultLocationsList("ns-1"),
cloudBackups: map[string][]*arkv1api.Backup{
"bucket-1": {
arktest.NewTestBackup().WithNamespace("ns-1").WithFinalizers("a-finalizer", gcFinalizer, "some-other-finalizer").Backup,
},
},
}, },
{ {
name: "Only target finalizer is removed", name: "all synced backups get created in Ark server's namespace",
cloudBackups: []*v1.Backup{
arktest.NewTestBackup().WithNamespace("ns-1").WithFinalizers(gcFinalizer, "blah").Backup,
},
namespace: "ns-1",
},
{
name: "backups get created in Ark server's namespace",
cloudBackups: []*v1.Backup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-2").WithName("backup-2").Backup,
},
namespace: "heptio-ark", namespace: "heptio-ark",
locations: defaultLocationsList("heptio-ark"),
cloudBackups: map[string][]*arkv1api.Backup{
"bucket-1": {
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
},
"bucket-2": {
arktest.NewTestBackup().WithNamespace("ns-2").WithName("backup-3").Backup,
arktest.NewTestBackup().WithNamespace("heptio-ark").WithName("backup-4").Backup,
},
},
}, },
{ {
name: "normal case with backups that already exist in Kubernetes", name: "new backups get synced when some cloud backups already exist in the cluster",
cloudBackups: []*v1.Backup{ namespace: "ns-1",
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup, locations: defaultLocationsList("ns-1"),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup, cloudBackups: map[string][]*arkv1api.Backup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup, "bucket-1": {
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
},
"bucket-2": {
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-4").Backup,
},
},
existingBackups: []*arkv1api.Backup{
// add a label to each existing backup so we can differentiate it from the cloud
// backup during verification
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithLabel("i-exist", "true").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").WithLabel("i-exist", "true").Backup,
},
},
{
name: "backup storage location names and labels get updated",
namespace: "ns-1",
locations: defaultLocationsList("ns-1"),
cloudBackups: map[string][]*arkv1api.Backup{
"bucket-1": {
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithStorageLocation("foo").WithLabel(arkv1api.StorageLocationLabel, "foo").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
},
"bucket-2": {
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").WithStorageLocation("bar").WithLabel(arkv1api.StorageLocationLabel, "bar").Backup,
},
}, },
existingBackups: sets.NewString("backup-2", "backup-3"),
namespace: "ns-1",
}, },
} }
for _, test := range tests { for _, test := range tests {
t.Run(test.name, func(t *testing.T) { t.Run(test.name, func(t *testing.T) {
var ( var (
backupLister = &cloudprovidermocks.BackupLister{}
client = fake.NewSimpleClientset() client = fake.NewSimpleClientset()
sharedInformers = informers.NewSharedInformerFactory(client, 0) sharedInformers = informers.NewSharedInformerFactory(client, 0)
logger = arktest.NewLogger() pluginManager = &pluginmocks.Manager{}
objectStore = &arktest.ObjectStore{}
) )
c := NewBackupSyncController( c := NewBackupSyncController(
client.ArkV1(), client.ArkV1(),
backupLister, sharedInformers.Ark().V1().Backups(),
"bucket", sharedInformers.Ark().V1().BackupStorageLocations(),
time.Duration(0), time.Duration(0),
test.namespace, test.namespace,
sharedInformers.Ark().V1().Backups(), nil, // pluginRegistry
logger, arktest.NewLogger(),
logrus.DebugLevel,
).(*backupSyncController) ).(*backupSyncController)
backupLister.On("ListBackups", "bucket").Return(test.cloudBackups, test.listBackupsError) c.newPluginManager = func(_ logrus.FieldLogger) plugin.Manager { return pluginManager }
pluginManager.On("GetObjectStore", "objStoreProvider").Return(objectStore, nil)
pluginManager.On("CleanupClients").Return(nil)
objectStore.On("Init", mock.Anything).Return(nil)
expectedActions := make([]core.Action, 0) for _, location := range test.locations {
require.NoError(t, sharedInformers.Ark().V1().BackupStorageLocations().Informer().GetStore().Add(location))
}
client.PrependReactor("get", "backups", func(action core.Action) (bool, runtime.Object, error) { c.listCloudBackups = func(_ logrus.FieldLogger, _ cloudprovider.ObjectStore, bucket string) ([]*arkv1api.Backup, error) {
getAction := action.(core.GetAction) backups, ok := test.cloudBackups[bucket]
if test.existingBackups.Has(getAction.GetName()) { if !ok {
return true, nil, nil return nil, errors.New("bucket not found")
} }
// We return nil in place of the found backup object because
// we exclusively check for the error and don't use the object return backups, nil
// returned by the Get / Backups call. }
return true, nil, apierrors.NewNotFound(v1.SchemeGroupVersion.WithResource("backups").GroupResource(), getAction.GetName())
}) for _, existingBackup := range test.existingBackups {
require.NoError(t, sharedInformers.Ark().V1().Backups().Informer().GetStore().Add(existingBackup))
_, err := client.ArkV1().Backups(test.namespace).Create(existingBackup)
require.NoError(t, err)
}
client.ClearActions()
c.run() c.run()
// we only expect creates for items within the target bucket for bucket, backups := range test.cloudBackups {
for _, cloudBackup := range test.cloudBackups { for _, cloudBackup := range backups {
// Verify that the run function stripped the GC finalizer obj, err := client.ArkV1().Backups(test.namespace).Get(cloudBackup.Name, metav1.GetOptions{})
assert.False(t, stringslice.Has(cloudBackup.Finalizers, gcFinalizer)) require.NoError(t, err)
assert.Equal(t, test.namespace, cloudBackup.Namespace)
actionGet := core.NewGetAction( // did this cloud backup already exist in the cluster?
v1.SchemeGroupVersion.WithResource("backups"), var existing *arkv1api.Backup
test.namespace, for _, obj := range test.existingBackups {
cloudBackup.Name, if obj.Name == cloudBackup.Name {
) existing = obj
expectedActions = append(expectedActions, actionGet) break
}
}
if test.existingBackups.Has(cloudBackup.Name) { if existing != nil {
continue // if this cloud backup already exists in the cluster, make sure that what we get from the
// client is the existing backup, not the cloud one.
assert.Equal(t, existing, obj)
} else {
// verify that the GC finalizer is removed
assert.Equal(t, stringslice.Except(cloudBackup.Finalizers, gcFinalizer), obj.Finalizers)
// verify that the storage location field and label are set properly
for _, location := range test.locations {
if location.Spec.ObjectStorage.Bucket == bucket {
assert.Equal(t, location.Name, obj.Spec.StorageLocation)
assert.Equal(t, location.Name, obj.Labels[arkv1api.StorageLocationLabel])
break
}
}
}
} }
actionCreate := core.NewCreateAction(
v1.SchemeGroupVersion.WithResource("backups"),
test.namespace,
cloudBackup,
)
expectedActions = append(expectedActions, actionCreate)
} }
assert.Equal(t, expectedActions, client.Actions())
}) })
} }
} }
func TestDeleteUnused(t *testing.T) { func TestDeleteOrphanedBackups(t *testing.T) {
tests := []struct { tests := []struct {
name string name string
cloudBackups []*v1.Backup cloudBackups sets.String
k8sBackups []*arktest.TestBackup k8sBackups []*arktest.TestBackup
namespace string namespace string
expectedDeletes sets.String expectedDeletes sets.String
}{ }{
{ {
name: "no overlapping backups", name: "no overlapping backups",
namespace: "ns-1", namespace: "ns-1",
cloudBackups: []*v1.Backup{ cloudBackups: sets.NewString("backup-1", "backup-2", "backup-3"),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup,
},
k8sBackups: []*arktest.TestBackup{ k8sBackups: []*arktest.TestBackup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupA").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupA").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupB").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupB").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupC").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupC").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
}, },
expectedDeletes: sets.NewString("backupA", "backupB", "backupC"), expectedDeletes: sets.NewString("backupA", "backupB", "backupC"),
}, },
{ {
name: "some overlapping backups", name: "some overlapping backups",
namespace: "ns-1", namespace: "ns-1",
cloudBackups: []*v1.Backup{ cloudBackups: sets.NewString("backup-1", "backup-2", "backup-3"),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup,
},
k8sBackups: []*arktest.TestBackup{ k8sBackups: []*arktest.TestBackup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupC").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-C").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
}, },
expectedDeletes: sets.NewString("backupC"), expectedDeletes: sets.NewString("backup-C"),
}, },
{ {
name: "all overlapping backups", name: "all overlapping backups",
namespace: "ns-1", namespace: "ns-1",
cloudBackups: []*v1.Backup{ cloudBackups: sets.NewString("backup-1", "backup-2", "backup-3"),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup,
},
k8sBackups: []*arktest.TestBackup{ k8sBackups: []*arktest.TestBackup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
}, },
expectedDeletes: sets.NewString(), expectedDeletes: sets.NewString(),
}, },
{ {
name: "no overlapping backups but including backups that are not complete", name: "no overlapping backups but including backups that are not complete",
namespace: "ns-1", namespace: "ns-1",
cloudBackups: []*v1.Backup{ cloudBackups: sets.NewString("backup-1", "backup-2", "backup-3"),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup,
},
k8sBackups: []*arktest.TestBackup{ k8sBackups: []*arktest.TestBackup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupA").WithPhase(v1.BackupPhaseCompleted), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backupA").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("Deleting").WithPhase(v1.BackupPhaseDeleting), arktest.NewTestBackup().WithNamespace("ns-1").WithName("Deleting").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseDeleting),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("Failed").WithPhase(v1.BackupPhaseFailed), arktest.NewTestBackup().WithNamespace("ns-1").WithName("Failed").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseFailed),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("FailedValidation").WithPhase(v1.BackupPhaseFailedValidation), arktest.NewTestBackup().WithNamespace("ns-1").WithName("FailedValidation").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseFailedValidation),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("InProgress").WithPhase(v1.BackupPhaseInProgress), arktest.NewTestBackup().WithNamespace("ns-1").WithName("InProgress").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseInProgress),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("New").WithPhase(v1.BackupPhaseNew), arktest.NewTestBackup().WithNamespace("ns-1").WithName("New").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseNew),
}, },
expectedDeletes: sets.NewString("backupA"), expectedDeletes: sets.NewString("backupA"),
}, },
{ {
name: "all overlapping backups and all backups that are not complete", name: "all overlapping backups and all backups that are not complete",
namespace: "ns-1", namespace: "ns-1",
cloudBackups: []*v1.Backup{ cloudBackups: sets.NewString("backup-1", "backup-2", "backup-3"),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").Backup,
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").Backup,
},
k8sBackups: []*arktest.TestBackup{ k8sBackups: []*arktest.TestBackup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithPhase(v1.BackupPhaseFailed), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseFailed),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").WithPhase(v1.BackupPhaseFailedValidation), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseFailedValidation),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").WithPhase(v1.BackupPhaseInProgress), arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-3").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseInProgress),
}, },
expectedDeletes: sets.NewString(), expectedDeletes: sets.NewString(),
}, },
{
name: "no completed backups in other locations are deleted",
namespace: "ns-1",
cloudBackups: sets.NewString("backup-1", "backup-2", "backup-3"),
k8sBackups: []*arktest.TestBackup{
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-1").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-2").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-C").WithLabel(arkv1api.StorageLocationLabel, "default").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-4").WithLabel(arkv1api.StorageLocationLabel, "alternate").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-5").WithLabel(arkv1api.StorageLocationLabel, "alternate").WithPhase(arkv1api.BackupPhaseCompleted),
arktest.NewTestBackup().WithNamespace("ns-1").WithName("backup-6").WithLabel(arkv1api.StorageLocationLabel, "alternate").WithPhase(arkv1api.BackupPhaseCompleted),
},
expectedDeletes: sets.NewString("backup-C"),
},
} }
for _, test := range tests { for _, test := range tests {
t.Run(test.name, func(t *testing.T) { t.Run(test.name, func(t *testing.T) {
var ( var (
backupLister = &cloudprovidermocks.BackupLister{}
client = fake.NewSimpleClientset() client = fake.NewSimpleClientset()
sharedInformers = informers.NewSharedInformerFactory(client, 0) sharedInformers = informers.NewSharedInformerFactory(client, 0)
logger = arktest.NewLogger()
) )
c := NewBackupSyncController( c := NewBackupSyncController(
client.ArkV1(), client.ArkV1(),
backupLister, sharedInformers.Ark().V1().Backups(),
"bucket", sharedInformers.Ark().V1().BackupStorageLocations(),
time.Duration(0), time.Duration(0),
test.namespace, test.namespace,
sharedInformers.Ark().V1().Backups(), nil, // pluginRegistry
logger, arktest.NewLogger(),
logrus.InfoLevel,
).(*backupSyncController) ).(*backupSyncController)
expectedDeleteActions := make([]core.Action, 0) expectedDeleteActions := make([]core.Action, 0)
// setup: insert backups into Kubernetes
for _, backup := range test.k8sBackups { for _, backup := range test.k8sBackups {
// add test backup to informer
require.NoError(t, sharedInformers.Ark().V1().Backups().Informer().GetStore().Add(backup.Backup), "Error adding backup to informer")
// add test backup to client
_, err := client.Ark().Backups(test.namespace).Create(backup.Backup)
require.NoError(t, err, "Error adding backup to clientset")
// if we expect this backup to be deleted, set up the expected DeleteAction
if test.expectedDeletes.Has(backup.Name) { if test.expectedDeletes.Has(backup.Name) {
actionDelete := core.NewDeleteAction( actionDelete := core.NewDeleteAction(
v1.SchemeGroupVersion.WithResource("backups"), arkv1api.SchemeGroupVersion.WithResource("backups"),
test.namespace, test.namespace,
backup.Name, backup.Name,
) )
expectedDeleteActions = append(expectedDeleteActions, actionDelete) expectedDeleteActions = append(expectedDeleteActions, actionDelete)
} }
// add test backup to informer:
err := sharedInformers.Ark().V1().Backups().Informer().GetStore().Add(backup.Backup)
assert.NoError(t, err, "Error adding backup to informer")
// add test backup to kubernetes:
_, err = client.Ark().Backups(test.namespace).Create(backup.Backup)
assert.NoError(t, err, "Error deleting from clientset")
} }
// get names of client backups c.deleteOrphanedBackups("default", test.cloudBackups, arktest.NewLogger())
testBackupNames := sets.NewString()
for _, cloudBackup := range test.cloudBackups {
testBackupNames.Insert(cloudBackup.Name)
}
c.deleteUnused(testBackupNames)
numBackups, err := numBackups(t, client, c.namespace) numBackups, err := numBackups(t, client, c.namespace)
assert.NoError(t, err) assert.NoError(t, err)