update restore process using snapshot locations
Signed-off-by: Wayne Witzel III <wayne@riotousliving.com>pull/948/head
parent
268080ad09
commit
406b50a71b
|
@ -459,7 +459,7 @@ func (ib *defaultItemBackupper) takePVSnapshot(obj runtime.Unstructured, log log
|
|||
}
|
||||
|
||||
log.Info("Snapshotting PersistentVolume")
|
||||
snapshot := volumeSnapshot(ib.backupRequest.Backup, volumeID, volumeType, pvFailureDomainZone, location, iops)
|
||||
snapshot := volumeSnapshot(ib.backupRequest.Backup, metadata.GetName(), volumeID, volumeType, pvFailureDomainZone, location, iops)
|
||||
|
||||
var errs []error
|
||||
snapshotID, err := blockStore.CreateSnapshot(snapshot.Spec.ProviderVolumeID, snapshot.Spec.VolumeAZ, tags)
|
||||
|
@ -477,16 +477,17 @@ func (ib *defaultItemBackupper) takePVSnapshot(obj runtime.Unstructured, log log
|
|||
return kubeerrs.NewAggregate(errs)
|
||||
}
|
||||
|
||||
func volumeSnapshot(backup *api.Backup, volumeID, volumeType, az, location string, iops *int64) *volume.Snapshot {
|
||||
func volumeSnapshot(backup *api.Backup, volumeName, volumeID, volumeType, az, location string, iops *int64) *volume.Snapshot {
|
||||
return &volume.Snapshot{
|
||||
Spec: volume.SnapshotSpec{
|
||||
BackupName: backup.Name,
|
||||
BackupUID: string(backup.UID),
|
||||
Location: location,
|
||||
ProviderVolumeID: volumeID,
|
||||
VolumeType: volumeType,
|
||||
VolumeAZ: az,
|
||||
VolumeIOPS: iops,
|
||||
BackupName: backup.Name,
|
||||
BackupUID: string(backup.UID),
|
||||
Location: location,
|
||||
PersistentVolumeName: volumeName,
|
||||
ProviderVolumeID: volumeID,
|
||||
VolumeType: volumeType,
|
||||
VolumeAZ: az,
|
||||
VolumeIOPS: iops,
|
||||
},
|
||||
Status: volume.SnapshotStatus{
|
||||
Phase: volume.SnapshotPhaseNew,
|
||||
|
|
|
@ -717,9 +717,7 @@ func (s *server) runControllers(config *api.Config, defaultVolumeSnapshotLocatio
|
|||
restorer, err := restore.NewKubernetesRestorer(
|
||||
s.discoveryHelper,
|
||||
client.NewDynamicFactory(s.dynamicClient),
|
||||
nil,
|
||||
s.config.restoreResourcePriorities,
|
||||
s.arkClient.ArkV1(),
|
||||
s.kubeClient.CoreV1().Namespaces(),
|
||||
s.resticManager,
|
||||
s.config.podVolumeOperationTimeout,
|
||||
|
@ -735,6 +733,7 @@ func (s *server) runControllers(config *api.Config, defaultVolumeSnapshotLocatio
|
|||
restorer,
|
||||
s.sharedInformerFactory.Ark().V1().Backups(),
|
||||
s.sharedInformerFactory.Ark().V1().BackupStorageLocations(),
|
||||
s.sharedInformerFactory.Ark().V1().VolumeSnapshotLocations(),
|
||||
false,
|
||||
s.logger,
|
||||
s.logLevel,
|
||||
|
|
|
@ -68,17 +68,18 @@ var nonRestorableResources = []string{
|
|||
type restoreController struct {
|
||||
*genericController
|
||||
|
||||
namespace string
|
||||
restoreClient arkv1client.RestoresGetter
|
||||
backupClient arkv1client.BackupsGetter
|
||||
restorer restore.Restorer
|
||||
pvProviderExists bool
|
||||
backupLister listers.BackupLister
|
||||
restoreLister listers.RestoreLister
|
||||
backupLocationLister listers.BackupStorageLocationLister
|
||||
restoreLogLevel logrus.Level
|
||||
defaultBackupLocation string
|
||||
metrics *metrics.ServerMetrics
|
||||
namespace string
|
||||
restoreClient arkv1client.RestoresGetter
|
||||
backupClient arkv1client.BackupsGetter
|
||||
restorer restore.Restorer
|
||||
pvProviderExists bool
|
||||
backupLister listers.BackupLister
|
||||
restoreLister listers.RestoreLister
|
||||
backupLocationLister listers.BackupStorageLocationLister
|
||||
snapshotLocationLister listers.VolumeSnapshotLocationLister
|
||||
restoreLogLevel logrus.Level
|
||||
defaultBackupLocation string
|
||||
metrics *metrics.ServerMetrics
|
||||
|
||||
newPluginManager func(logger logrus.FieldLogger) plugin.Manager
|
||||
newBackupStore func(*api.BackupStorageLocation, persistence.ObjectStoreGetter, logrus.FieldLogger) (persistence.BackupStore, error)
|
||||
|
@ -92,6 +93,7 @@ func NewRestoreController(
|
|||
restorer restore.Restorer,
|
||||
backupInformer informers.BackupInformer,
|
||||
backupLocationInformer informers.BackupStorageLocationInformer,
|
||||
snapshotLocationInformer informers.VolumeSnapshotLocationInformer,
|
||||
pvProviderExists bool,
|
||||
logger logrus.FieldLogger,
|
||||
restoreLogLevel logrus.Level,
|
||||
|
@ -100,18 +102,19 @@ func NewRestoreController(
|
|||
metrics *metrics.ServerMetrics,
|
||||
) Interface {
|
||||
c := &restoreController{
|
||||
genericController: newGenericController("restore", logger),
|
||||
namespace: namespace,
|
||||
restoreClient: restoreClient,
|
||||
backupClient: backupClient,
|
||||
restorer: restorer,
|
||||
pvProviderExists: pvProviderExists,
|
||||
backupLister: backupInformer.Lister(),
|
||||
restoreLister: restoreInformer.Lister(),
|
||||
backupLocationLister: backupLocationInformer.Lister(),
|
||||
restoreLogLevel: restoreLogLevel,
|
||||
defaultBackupLocation: defaultBackupLocation,
|
||||
metrics: metrics,
|
||||
genericController: newGenericController("restore", logger),
|
||||
namespace: namespace,
|
||||
restoreClient: restoreClient,
|
||||
backupClient: backupClient,
|
||||
restorer: restorer,
|
||||
pvProviderExists: pvProviderExists,
|
||||
backupLister: backupInformer.Lister(),
|
||||
restoreLister: restoreInformer.Lister(),
|
||||
backupLocationLister: backupLocationInformer.Lister(),
|
||||
snapshotLocationLister: snapshotLocationInformer.Lister(),
|
||||
restoreLogLevel: restoreLogLevel,
|
||||
defaultBackupLocation: defaultBackupLocation,
|
||||
metrics: metrics,
|
||||
|
||||
// use variables to refer to these functions so they can be
|
||||
// replaced with fakes for testing.
|
||||
|
@ -124,6 +127,7 @@ func NewRestoreController(
|
|||
backupInformer.Informer().HasSynced,
|
||||
restoreInformer.Informer().HasSynced,
|
||||
backupLocationInformer.Informer().HasSynced,
|
||||
snapshotLocationInformer.Informer().HasSynced,
|
||||
)
|
||||
|
||||
restoreInformer.Informer().AddEventHandler(
|
||||
|
@ -233,6 +237,7 @@ func (c *restoreController) processRestore(key string) error {
|
|||
restore,
|
||||
actions,
|
||||
info,
|
||||
pluginManager,
|
||||
)
|
||||
|
||||
restore.Status.Warnings = len(restoreWarnings.Ark) + len(restoreWarnings.Cluster)
|
||||
|
@ -482,6 +487,7 @@ func (c *restoreController) runRestore(
|
|||
restore *api.Restore,
|
||||
actions []restore.ItemAction,
|
||||
info backupInfo,
|
||||
pluginManager plugin.Manager,
|
||||
) (restoreWarnings, restoreErrors api.RestoreResult, restoreFailure error) {
|
||||
logFile, err := ioutil.TempFile("", "")
|
||||
if err != nil {
|
||||
|
@ -531,10 +537,18 @@ func (c *restoreController) runRestore(
|
|||
}
|
||||
defer closeAndRemoveFile(resultsFile, c.logger)
|
||||
|
||||
volumeSnapshots, err := info.backupStore.GetBackupVolumeSnapshots(restore.Spec.BackupName)
|
||||
if err != nil {
|
||||
log.WithError(errors.WithStack(err)).Error("Error fetching volume snapshots")
|
||||
restoreErrors.Ark = append(restoreErrors.Ark, err.Error())
|
||||
restoreFailure = err
|
||||
return
|
||||
}
|
||||
|
||||
// Any return statement above this line means a total restore failure
|
||||
// Some failures after this line *may* be a total restore failure
|
||||
log.Info("starting restore")
|
||||
restoreWarnings, restoreErrors = c.restorer.Restore(log, restore, info.backup, backupFile, actions)
|
||||
restoreWarnings, restoreErrors = c.restorer.Restore(log, restore, info.backup, volumeSnapshots, backupFile, actions, c.snapshotLocationLister, pluginManager)
|
||||
log.Info("restore completed")
|
||||
|
||||
// Try to upload the log file. This is best-effort. If we fail, we'll add to the ark errors.
|
||||
|
|
|
@ -24,20 +24,10 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/runtime"
|
||||
core "k8s.io/client-go/testing"
|
||||
"k8s.io/client-go/tools/cache"
|
||||
|
||||
api "github.com/heptio/ark/pkg/apis/ark/v1"
|
||||
"github.com/heptio/ark/pkg/generated/clientset/versioned/fake"
|
||||
informers "github.com/heptio/ark/pkg/generated/informers/externalversions"
|
||||
listers "github.com/heptio/ark/pkg/generated/listers/ark/v1"
|
||||
"github.com/heptio/ark/pkg/metrics"
|
||||
"github.com/heptio/ark/pkg/persistence"
|
||||
persistencemocks "github.com/heptio/ark/pkg/persistence/mocks"
|
||||
|
@ -46,6 +36,16 @@ import (
|
|||
"github.com/heptio/ark/pkg/restore"
|
||||
"github.com/heptio/ark/pkg/util/collections"
|
||||
arktest "github.com/heptio/ark/pkg/util/test"
|
||||
"github.com/heptio/ark/pkg/volume"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/runtime"
|
||||
core "k8s.io/client-go/testing"
|
||||
"k8s.io/client-go/tools/cache"
|
||||
)
|
||||
|
||||
func TestFetchBackupInfo(t *testing.T) {
|
||||
|
@ -104,6 +104,7 @@ func TestFetchBackupInfo(t *testing.T) {
|
|||
restorer,
|
||||
sharedInformers.Ark().V1().Backups(),
|
||||
sharedInformers.Ark().V1().BackupStorageLocations(),
|
||||
sharedInformers.Ark().V1().VolumeSnapshotLocations(),
|
||||
false,
|
||||
logger,
|
||||
logrus.InfoLevel,
|
||||
|
@ -197,6 +198,7 @@ func TestProcessRestoreSkips(t *testing.T) {
|
|||
restorer,
|
||||
sharedInformers.Ark().V1().Backups(),
|
||||
sharedInformers.Ark().V1().BackupStorageLocations(),
|
||||
sharedInformers.Ark().V1().VolumeSnapshotLocations(),
|
||||
false, // pvProviderExists
|
||||
logger,
|
||||
logrus.InfoLevel,
|
||||
|
@ -422,6 +424,7 @@ func TestProcessRestore(t *testing.T) {
|
|||
restorer,
|
||||
sharedInformers.Ark().V1().Backups(),
|
||||
sharedInformers.Ark().V1().BackupStorageLocations(),
|
||||
sharedInformers.Ark().V1().VolumeSnapshotLocations(),
|
||||
test.allowRestoreSnapshots,
|
||||
logger,
|
||||
logrus.InfoLevel,
|
||||
|
@ -498,6 +501,16 @@ func TestProcessRestore(t *testing.T) {
|
|||
backupStore.On("PutRestoreLog", test.backup.Name, test.restore.Name, mock.Anything).Return(test.putRestoreLogErr)
|
||||
|
||||
backupStore.On("PutRestoreResults", test.backup.Name, test.restore.Name, mock.Anything).Return(nil)
|
||||
|
||||
volumeSnapshots := []*volume.Snapshot{
|
||||
{
|
||||
Spec: volume.SnapshotSpec{
|
||||
PersistentVolumeName: "test-pv",
|
||||
BackupName: test.backup.Name,
|
||||
},
|
||||
},
|
||||
}
|
||||
backupStore.On("GetBackupVolumeSnapshots", test.backup.Name).Return(volumeSnapshots, nil)
|
||||
}
|
||||
|
||||
var (
|
||||
|
@ -629,6 +642,7 @@ func TestvalidateAndCompleteWhenScheduleNameSpecified(t *testing.T) {
|
|||
nil,
|
||||
sharedInformers.Ark().V1().Backups(),
|
||||
sharedInformers.Ark().V1().BackupStorageLocations(),
|
||||
sharedInformers.Ark().V1().VolumeSnapshotLocations(),
|
||||
false,
|
||||
logger,
|
||||
logrus.DebugLevel,
|
||||
|
@ -815,8 +829,11 @@ func (r *fakeRestorer) Restore(
|
|||
log logrus.FieldLogger,
|
||||
restore *api.Restore,
|
||||
backup *api.Backup,
|
||||
volumeSnapshots []*volume.Snapshot,
|
||||
backupReader io.Reader,
|
||||
actions []restore.ItemAction,
|
||||
snapshotLocationLister listers.VolumeSnapshotLocationLister,
|
||||
blockStoreGetter restore.BlockStoreGetter,
|
||||
) (api.RestoreResult, api.RestoreResult) {
|
||||
res := r.Called(log, restore, backup, backupReader, actions)
|
||||
|
||||
|
|
|
@ -50,7 +50,7 @@ import (
|
|||
"github.com/heptio/ark/pkg/client"
|
||||
"github.com/heptio/ark/pkg/cloudprovider"
|
||||
"github.com/heptio/ark/pkg/discovery"
|
||||
arkv1client "github.com/heptio/ark/pkg/generated/clientset/versioned/typed/ark/v1"
|
||||
listers "github.com/heptio/ark/pkg/generated/listers/ark/v1"
|
||||
"github.com/heptio/ark/pkg/kuberesource"
|
||||
"github.com/heptio/ark/pkg/restic"
|
||||
"github.com/heptio/ark/pkg/util/boolptr"
|
||||
|
@ -58,12 +58,25 @@ import (
|
|||
"github.com/heptio/ark/pkg/util/filesystem"
|
||||
"github.com/heptio/ark/pkg/util/kube"
|
||||
arksync "github.com/heptio/ark/pkg/util/sync"
|
||||
"github.com/heptio/ark/pkg/volume"
|
||||
)
|
||||
|
||||
type BlockStoreGetter interface {
|
||||
GetBlockStore(name string) (cloudprovider.BlockStore, error)
|
||||
}
|
||||
|
||||
// Restorer knows how to restore a backup.
|
||||
type Restorer interface {
|
||||
// Restore restores the backup data from backupReader, returning warnings and errors.
|
||||
Restore(log logrus.FieldLogger, restore *api.Restore, backup *api.Backup, backupReader io.Reader, actions []ItemAction) (api.RestoreResult, api.RestoreResult)
|
||||
Restore(log logrus.FieldLogger,
|
||||
restore *api.Restore,
|
||||
backup *api.Backup,
|
||||
volumeSnapshots []*volume.Snapshot,
|
||||
backupReader io.Reader,
|
||||
actions []ItemAction,
|
||||
snapshotLocationLister listers.VolumeSnapshotLocationLister,
|
||||
blockStoreGetter BlockStoreGetter,
|
||||
) (api.RestoreResult, api.RestoreResult)
|
||||
}
|
||||
|
||||
type gvString string
|
||||
|
@ -73,8 +86,6 @@ type kindString string
|
|||
type kubernetesRestorer struct {
|
||||
discoveryHelper discovery.Helper
|
||||
dynamicFactory client.DynamicFactory
|
||||
blockStore cloudprovider.BlockStore
|
||||
backupClient arkv1client.BackupsGetter
|
||||
namespaceClient corev1.NamespaceInterface
|
||||
resticRestorerFactory restic.RestorerFactory
|
||||
resticTimeout time.Duration
|
||||
|
@ -145,9 +156,7 @@ func prioritizeResources(helper discovery.Helper, priorities []string, includedR
|
|||
func NewKubernetesRestorer(
|
||||
discoveryHelper discovery.Helper,
|
||||
dynamicFactory client.DynamicFactory,
|
||||
blockStore cloudprovider.BlockStore,
|
||||
resourcePriorities []string,
|
||||
backupClient arkv1client.BackupsGetter,
|
||||
namespaceClient corev1.NamespaceInterface,
|
||||
resticRestorerFactory restic.RestorerFactory,
|
||||
resticTimeout time.Duration,
|
||||
|
@ -156,22 +165,29 @@ func NewKubernetesRestorer(
|
|||
return &kubernetesRestorer{
|
||||
discoveryHelper: discoveryHelper,
|
||||
dynamicFactory: dynamicFactory,
|
||||
blockStore: blockStore,
|
||||
backupClient: backupClient,
|
||||
namespaceClient: namespaceClient,
|
||||
resticRestorerFactory: resticRestorerFactory,
|
||||
resticTimeout: resticTimeout,
|
||||
resourcePriorities: resourcePriorities,
|
||||
logger: logger,
|
||||
|
||||
fileSystem: filesystem.NewFileSystem(),
|
||||
fileSystem: filesystem.NewFileSystem(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Restore executes a restore into the target Kubernetes cluster according to the restore spec
|
||||
// and using data from the provided backup/backup reader. Returns a warnings and errors RestoreResult,
|
||||
// respectively, summarizing info about the restore.
|
||||
func (kr *kubernetesRestorer) Restore(log logrus.FieldLogger, restore *api.Restore, backup *api.Backup, backupReader io.Reader, actions []ItemAction) (api.RestoreResult, api.RestoreResult) {
|
||||
func (kr *kubernetesRestorer) Restore(
|
||||
log logrus.FieldLogger,
|
||||
restore *api.Restore,
|
||||
backup *api.Backup,
|
||||
volumeSnapshots []*volume.Snapshot,
|
||||
backupReader io.Reader,
|
||||
actions []ItemAction,
|
||||
snapshotLocationLister listers.VolumeSnapshotLocationLister,
|
||||
blockStoreGetter BlockStoreGetter,
|
||||
) (api.RestoreResult, api.RestoreResult) {
|
||||
|
||||
// metav1.LabelSelectorAsSelector converts a nil LabelSelector to a
|
||||
// Nothing Selector, i.e. a selector that matches nothing. We want
|
||||
// a selector that matches everything. This can be accomplished by
|
||||
|
@ -220,11 +236,14 @@ func (kr *kubernetesRestorer) Restore(log logrus.FieldLogger, restore *api.Resto
|
|||
}
|
||||
|
||||
pvRestorer := &pvRestorer{
|
||||
logger: log,
|
||||
snapshotVolumes: backup.Spec.SnapshotVolumes,
|
||||
restorePVs: restore.Spec.RestorePVs,
|
||||
volumeBackups: backup.Status.VolumeBackups,
|
||||
blockStore: kr.blockStore,
|
||||
logger: log,
|
||||
backupName: restore.Spec.BackupName,
|
||||
backupNamespace: backup.Namespace,
|
||||
snapshotVolumes: backup.Spec.SnapshotVolumes,
|
||||
restorePVs: restore.Spec.RestorePVs,
|
||||
volumeSnapshots: volumeSnapshots,
|
||||
blockStoreGetter: blockStoreGetter,
|
||||
snapshotLocationLister: snapshotLocationLister,
|
||||
}
|
||||
|
||||
restoreCtx := &context{
|
||||
|
@ -238,7 +257,7 @@ func (kr *kubernetesRestorer) Restore(log logrus.FieldLogger, restore *api.Resto
|
|||
fileSystem: kr.fileSystem,
|
||||
namespaceClient: kr.namespaceClient,
|
||||
actions: resolvedActions,
|
||||
blockStore: kr.blockStore,
|
||||
blockStoreGetter: blockStoreGetter,
|
||||
resticRestorer: resticRestorer,
|
||||
pvsToProvision: sets.NewString(),
|
||||
pvRestorer: pvRestorer,
|
||||
|
@ -319,7 +338,7 @@ type context struct {
|
|||
fileSystem filesystem.Interface
|
||||
namespaceClient corev1.NamespaceInterface
|
||||
actions []resolvedAction
|
||||
blockStore cloudprovider.BlockStore
|
||||
blockStoreGetter BlockStoreGetter
|
||||
resticRestorer restic.Restorer
|
||||
globalWaitGroup arksync.ErrorGroup
|
||||
resourceWaitGroup sync.WaitGroup
|
||||
|
@ -887,11 +906,14 @@ type PVRestorer interface {
|
|||
}
|
||||
|
||||
type pvRestorer struct {
|
||||
logger logrus.FieldLogger
|
||||
snapshotVolumes *bool
|
||||
restorePVs *bool
|
||||
volumeBackups map[string]*api.VolumeBackupInfo
|
||||
blockStore cloudprovider.BlockStore
|
||||
logger logrus.FieldLogger
|
||||
backupName string
|
||||
backupNamespace string
|
||||
snapshotVolumes *bool
|
||||
restorePVs *bool
|
||||
volumeSnapshots []*volume.Snapshot
|
||||
blockStoreGetter BlockStoreGetter
|
||||
snapshotLocationLister listers.VolumeSnapshotLocationLister
|
||||
}
|
||||
|
||||
func (r *pvRestorer) executePVAction(obj *unstructured.Unstructured) (*unstructured.Unstructured, error) {
|
||||
|
@ -902,7 +924,7 @@ func (r *pvRestorer) executePVAction(obj *unstructured.Unstructured) (*unstructu
|
|||
|
||||
spec, err := collections.GetMap(obj.UnstructuredContent(), "spec")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
|
||||
delete(spec, "claimRef")
|
||||
|
@ -918,43 +940,50 @@ func (r *pvRestorer) executePVAction(obj *unstructured.Unstructured) (*unstructu
|
|||
return obj, nil
|
||||
}
|
||||
|
||||
// If we can't find a snapshot record for this particular PV, it most likely wasn't a PV that Ark
|
||||
// could snapshot, so return early instead of trying to restore from a snapshot.
|
||||
backupInfo, found := r.volumeBackups[pvName]
|
||||
if !found {
|
||||
log := r.logger.WithFields(logrus.Fields{"persistentVolume": pvName})
|
||||
|
||||
var foundSnapshot *volume.Snapshot
|
||||
for _, snapshot := range r.volumeSnapshots {
|
||||
if snapshot.Spec.PersistentVolumeName == pvName {
|
||||
foundSnapshot = snapshot
|
||||
break
|
||||
}
|
||||
}
|
||||
if foundSnapshot == nil {
|
||||
log.Info("skipping no snapshot found")
|
||||
return obj, nil
|
||||
}
|
||||
|
||||
// Past this point, we expect to be doing a restore
|
||||
|
||||
if r.blockStore == nil {
|
||||
return nil, errors.New("you must configure a persistentVolumeProvider to restore PersistentVolumes from snapshots")
|
||||
}
|
||||
|
||||
log := r.logger.WithFields(
|
||||
logrus.Fields{
|
||||
"persistentVolume": pvName,
|
||||
"snapshot": backupInfo.SnapshotID,
|
||||
},
|
||||
)
|
||||
|
||||
log.Info("restoring persistent volume from snapshot")
|
||||
volumeID, err := r.blockStore.CreateVolumeFromSnapshot(backupInfo.SnapshotID, backupInfo.Type, backupInfo.AvailabilityZone, backupInfo.Iops)
|
||||
location, err := r.snapshotLocationLister.VolumeSnapshotLocations(r.backupNamespace).Get(foundSnapshot.Spec.Location)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
|
||||
blockStore, err := r.blockStoreGetter.GetBlockStore(location.Spec.Provider)
|
||||
if err != nil {
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
|
||||
if err := blockStore.Init(location.Spec.Config); err != nil {
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
|
||||
volumeID, err := blockStore.CreateVolumeFromSnapshot(foundSnapshot.Status.ProviderSnapshotID, foundSnapshot.Spec.VolumeType, foundSnapshot.Spec.VolumeAZ, foundSnapshot.Spec.VolumeIOPS)
|
||||
if err != nil {
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
|
||||
log = log.WithFields(logrus.Fields{"snapshot": foundSnapshot.Status.ProviderSnapshotID})
|
||||
log.Info("successfully restored persistent volume from snapshot")
|
||||
|
||||
updated1, err := r.blockStore.SetVolumeID(obj, volumeID)
|
||||
// used to be update1 which is then cast to Unstructured and returned
|
||||
updated1, err := blockStore.SetVolumeID(obj, volumeID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
return nil, errors.WithStack(err)
|
||||
}
|
||||
|
||||
updated2, ok := updated1.(*unstructured.Unstructured)
|
||||
if !ok {
|
||||
return nil, errors.Errorf("unexpected type %T", updated1)
|
||||
}
|
||||
|
||||
return updated2, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -21,13 +21,20 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"k8s.io/client-go/kubernetes/scheme"
|
||||
|
||||
api "github.com/heptio/ark/pkg/apis/ark/v1"
|
||||
"github.com/heptio/ark/pkg/cloudprovider"
|
||||
"github.com/heptio/ark/pkg/generated/clientset/versioned/fake"
|
||||
informers "github.com/heptio/ark/pkg/generated/informers/externalversions"
|
||||
"github.com/heptio/ark/pkg/kuberesource"
|
||||
"github.com/heptio/ark/pkg/util/collections"
|
||||
"github.com/heptio/ark/pkg/util/logging"
|
||||
arktest "github.com/heptio/ark/pkg/util/test"
|
||||
"github.com/heptio/ark/pkg/volume"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"k8s.io/api/core/v1"
|
||||
k8serrors "k8s.io/apimachinery/pkg/api/errors"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
|
@ -37,14 +44,8 @@ import (
|
|||
"k8s.io/apimachinery/pkg/runtime/schema"
|
||||
"k8s.io/apimachinery/pkg/util/sets"
|
||||
"k8s.io/apimachinery/pkg/watch"
|
||||
"k8s.io/client-go/kubernetes/scheme"
|
||||
corev1 "k8s.io/client-go/kubernetes/typed/core/v1"
|
||||
|
||||
api "github.com/heptio/ark/pkg/apis/ark/v1"
|
||||
"github.com/heptio/ark/pkg/cloudprovider"
|
||||
"github.com/heptio/ark/pkg/kuberesource"
|
||||
"github.com/heptio/ark/pkg/util/boolptr"
|
||||
"github.com/heptio/ark/pkg/util/collections"
|
||||
arktest "github.com/heptio/ark/pkg/util/test"
|
||||
)
|
||||
|
||||
func TestPrioritizeResources(t *testing.T) {
|
||||
|
@ -580,6 +581,12 @@ func TestRestoreResourceForNamespace(t *testing.T) {
|
|||
},
|
||||
}
|
||||
|
||||
var (
|
||||
client = fake.NewSimpleClientset()
|
||||
sharedInformers = informers.NewSharedInformerFactory(client, 0)
|
||||
snapshotLocationLister = sharedInformers.Ark().V1().VolumeSnapshotLocations().Lister()
|
||||
)
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
resourceClient := &arktest.FakeDynamicClient{}
|
||||
|
@ -619,9 +626,16 @@ func TestRestoreResourceForNamespace(t *testing.T) {
|
|||
BackupName: "my-backup",
|
||||
},
|
||||
},
|
||||
backup: &api.Backup{},
|
||||
log: arktest.NewLogger(),
|
||||
pvRestorer: &pvRestorer{},
|
||||
backup: &api.Backup{},
|
||||
log: arktest.NewLogger(),
|
||||
pvRestorer: &pvRestorer{
|
||||
logger: logging.DefaultLogger(logrus.DebugLevel),
|
||||
blockStoreGetter: &fakeBlockStoreGetter{
|
||||
volumeMap: map[api.VolumeBackupInfo]string{{SnapshotID: "snap-1"}: "volume-1"},
|
||||
volumeID: "volume-1",
|
||||
},
|
||||
snapshotLocationLister: snapshotLocationLister,
|
||||
},
|
||||
}
|
||||
|
||||
warnings, errors := ctx.restoreResource(test.resourcePath, test.namespace, test.resourcePath)
|
||||
|
@ -1179,11 +1193,22 @@ func TestIsCompleted(t *testing.T) {
|
|||
func TestExecutePVAction(t *testing.T) {
|
||||
iops := int64(1000)
|
||||
|
||||
locationsFake := map[string]*api.VolumeSnapshotLocation{
|
||||
"default": arktest.NewTestVolumeSnapshotLocation().WithName("default-name").VolumeSnapshotLocation,
|
||||
}
|
||||
|
||||
var locations []string
|
||||
for key := range locationsFake {
|
||||
locations = append(locations, key)
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
obj *unstructured.Unstructured
|
||||
restore *api.Restore
|
||||
backup *api.Backup
|
||||
backup *arktest.TestBackup
|
||||
volumeSnapshots []*volume.Snapshot
|
||||
locations map[string]*api.VolumeSnapshotLocation
|
||||
volumeMap map[api.VolumeBackupInfo]string
|
||||
noBlockStore bool
|
||||
expectedErr bool
|
||||
|
@ -1207,21 +1232,28 @@ func TestExecutePVAction(t *testing.T) {
|
|||
name: "ensure spec.claimRef, spec.storageClassName are deleted",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithAnnotations("a", "b").WithSpec("claimRef", "storageClassName", "someOtherField").Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(false).Restore,
|
||||
backup: &api.Backup{},
|
||||
backup: arktest.NewTestBackup().WithName("backup1").WithPhase(api.BackupPhaseInProgress).WithVolumeSnapshotLocations(locations),
|
||||
expectedRes: NewTestUnstructured().WithAnnotations("a", "b").WithName("pv-1").WithSpec("someOtherField").Unstructured,
|
||||
},
|
||||
{
|
||||
name: "if backup.spec.snapshotVolumes is false, ignore restore.spec.restorePVs and return early",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithAnnotations("a", "b").WithSpec("claimRef", "storageClassName", "someOtherField").Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(true).Restore,
|
||||
backup: &api.Backup{Spec: api.BackupSpec{SnapshotVolumes: boolptr.False()}},
|
||||
backup: arktest.NewTestBackup().WithName("backup1").WithPhase(api.BackupPhaseInProgress).WithSnapshotVolumes(false),
|
||||
expectedRes: NewTestUnstructured().WithName("pv-1").WithAnnotations("a", "b").WithSpec("someOtherField").Unstructured,
|
||||
},
|
||||
{
|
||||
name: "not restoring, return early",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpec().Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(false).Restore,
|
||||
backup: &api.Backup{Status: api.BackupStatus{VolumeBackups: map[string]*api.VolumeBackupInfo{"pv-1": {SnapshotID: "snap-1"}}}},
|
||||
name: "not restoring, return early",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpec().Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(false).Restore,
|
||||
backup: arktest.NewTestBackup().WithName("backup1").WithPhase(api.BackupPhaseInProgress).WithVolumeSnapshotLocations(locations),
|
||||
volumeSnapshots: []*volume.Snapshot{
|
||||
{
|
||||
Spec: volume.SnapshotSpec{BackupName: "backup1", Location: "default-name", ProviderVolumeID: "volume-1", PersistentVolumeName: "pv-1", VolumeType: "gp", VolumeIOPS: &iops},
|
||||
Status: volume.SnapshotStatus{ProviderSnapshotID: "snap-1"},
|
||||
},
|
||||
},
|
||||
locations: locationsFake,
|
||||
expectedErr: false,
|
||||
expectedRes: NewTestUnstructured().WithName("pv-1").WithSpec().Unstructured,
|
||||
},
|
||||
|
@ -1229,23 +1261,38 @@ func TestExecutePVAction(t *testing.T) {
|
|||
name: "restoring, return without error if there is no PV->BackupInfo map",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpec("xyz").Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(true).Restore,
|
||||
backup: &api.Backup{Status: api.BackupStatus{}},
|
||||
backup: arktest.NewTestBackup().WithName("backup1").WithPhase(api.BackupPhaseInProgress).WithVolumeSnapshotLocations(locations),
|
||||
locations: locationsFake,
|
||||
expectedErr: false,
|
||||
expectedRes: NewTestUnstructured().WithName("pv-1").WithSpec("xyz").Unstructured,
|
||||
},
|
||||
{
|
||||
name: "restoring, return early if there is PV->BackupInfo map but no entry for this PV",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpec("xyz").Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(true).Restore,
|
||||
backup: &api.Backup{Status: api.BackupStatus{VolumeBackups: map[string]*api.VolumeBackupInfo{"another-pv": {}}}},
|
||||
name: "restoring, return early if there is PV->BackupInfo map but no entry for this PV",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpec("xyz").Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(true).Restore,
|
||||
backup: arktest.NewTestBackup().WithName("backup1").WithPhase(api.BackupPhaseInProgress).WithVolumeSnapshotLocations(locations),
|
||||
volumeSnapshots: []*volume.Snapshot{
|
||||
{
|
||||
Spec: volume.SnapshotSpec{BackupName: "backup1", Location: "default-name", ProviderVolumeID: "volume-1", PersistentVolumeName: "another-pv", VolumeType: "gp", VolumeIOPS: &iops},
|
||||
Status: volume.SnapshotStatus{ProviderSnapshotID: "another-snap-1"},
|
||||
},
|
||||
},
|
||||
locations: locationsFake,
|
||||
expectedErr: false,
|
||||
expectedRes: NewTestUnstructured().WithName("pv-1").WithSpec("xyz").Unstructured,
|
||||
},
|
||||
{
|
||||
name: "volume type and IOPS are correctly passed to CreateVolume",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpec("xyz").Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(true).Restore,
|
||||
backup: &api.Backup{Status: api.BackupStatus{VolumeBackups: map[string]*api.VolumeBackupInfo{"pv-1": {SnapshotID: "snap-1", Type: "gp", Iops: &iops}}}},
|
||||
name: "volume type and IOPS are correctly passed to CreateVolume",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpec("xyz").Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().WithRestorePVs(true).Restore,
|
||||
backup: arktest.NewTestBackup().WithName("backup1").WithPhase(api.BackupPhaseInProgress).WithVolumeSnapshotLocations(locations),
|
||||
locations: locationsFake,
|
||||
volumeSnapshots: []*volume.Snapshot{
|
||||
{
|
||||
Spec: volume.SnapshotSpec{BackupName: "backup1", Location: "default-name", ProviderVolumeID: "volume-1", PersistentVolumeName: "pv-1", VolumeType: "gp", VolumeIOPS: &iops},
|
||||
Status: volume.SnapshotStatus{ProviderSnapshotID: "snap-1"},
|
||||
},
|
||||
},
|
||||
volumeMap: map[api.VolumeBackupInfo]string{{SnapshotID: "snap-1", Type: "gp", Iops: &iops}: "volume-1"},
|
||||
volumeID: "volume-1",
|
||||
expectedErr: false,
|
||||
|
@ -1253,12 +1300,16 @@ func TestExecutePVAction(t *testing.T) {
|
|||
expectedRes: NewTestUnstructured().WithName("pv-1").WithSpec("xyz").Unstructured,
|
||||
},
|
||||
{
|
||||
name: "restoring, blockStore=nil, backup has at least 1 snapshot -> error",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpecField("awsElasticBlockStore", make(map[string]interface{})).Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().Restore,
|
||||
backup: &api.Backup{Status: api.BackupStatus{VolumeBackups: map[string]*api.VolumeBackupInfo{"pv-1": {SnapshotID: "snap-1"}}}},
|
||||
volumeMap: map[api.VolumeBackupInfo]string{{SnapshotID: "snap-1"}: "volume-1"},
|
||||
volumeID: "volume-1",
|
||||
name: "restoring, blockStore=nil, backup has at least 1 snapshot -> error",
|
||||
obj: NewTestUnstructured().WithName("pv-1").WithSpecField("awsElasticBlockStore", make(map[string]interface{})).Unstructured,
|
||||
restore: arktest.NewDefaultTestRestore().Restore,
|
||||
backup: arktest.NewTestBackup().WithName("backup1").WithPhase(api.BackupPhaseInProgress).WithVolumeSnapshotLocations(locations),
|
||||
volumeSnapshots: []*volume.Snapshot{
|
||||
{
|
||||
Spec: volume.SnapshotSpec{BackupName: "backup1", Location: "default-name", ProviderVolumeID: "volume-1", PersistentVolumeName: "pv-1", VolumeType: "gp", VolumeIOPS: &iops},
|
||||
Status: volume.SnapshotStatus{ProviderSnapshotID: "snap-1"},
|
||||
},
|
||||
},
|
||||
noBlockStore: true,
|
||||
expectedErr: true,
|
||||
expectedRes: NewTestUnstructured().WithName("pv-1").WithSpecField("awsElasticBlockStore", make(map[string]interface{})).Unstructured,
|
||||
|
@ -1268,25 +1319,42 @@ func TestExecutePVAction(t *testing.T) {
|
|||
for _, test := range tests {
|
||||
t.Run(test.name, func(t *testing.T) {
|
||||
var (
|
||||
blockStore cloudprovider.BlockStore
|
||||
fakeBlockStore *arktest.FakeBlockStore
|
||||
blockStoreGetter BlockStoreGetter
|
||||
testBlockStoreGetter *fakeBlockStoreGetter
|
||||
|
||||
client = fake.NewSimpleClientset()
|
||||
sharedInformers = informers.NewSharedInformerFactory(client, 0)
|
||||
snapshotLocationLister = sharedInformers.Ark().V1().VolumeSnapshotLocations().Lister()
|
||||
)
|
||||
|
||||
if !test.noBlockStore {
|
||||
fakeBlockStore = &arktest.FakeBlockStore{
|
||||
RestorableVolumes: test.volumeMap,
|
||||
VolumeID: test.volumeID,
|
||||
testBlockStoreGetter = &fakeBlockStoreGetter{
|
||||
volumeMap: test.volumeMap,
|
||||
volumeID: test.volumeID,
|
||||
}
|
||||
blockStore = fakeBlockStore
|
||||
testBlockStoreGetter.GetBlockStore("default")
|
||||
blockStoreGetter = testBlockStoreGetter
|
||||
for _, location := range test.locations {
|
||||
require.NoError(t, sharedInformers.Ark().V1().VolumeSnapshotLocations().Informer().GetStore().Add(location))
|
||||
}
|
||||
} else {
|
||||
assert.Equal(t, nil, blockStoreGetter)
|
||||
}
|
||||
|
||||
r := &pvRestorer{
|
||||
logger: arktest.NewLogger(),
|
||||
restorePVs: test.restore.Spec.RestorePVs,
|
||||
blockStore: blockStore,
|
||||
logger: logging.DefaultLogger(logrus.DebugLevel),
|
||||
backupName: "backup1",
|
||||
backupNamespace: api.DefaultNamespace,
|
||||
restorePVs: test.restore.Spec.RestorePVs,
|
||||
snapshotLocationLister: snapshotLocationLister,
|
||||
blockStoreGetter: blockStoreGetter,
|
||||
}
|
||||
if test.backup != nil {
|
||||
backup := test.backup.DeepCopy()
|
||||
backup.Spec.VolumeSnapshotLocations = test.backup.Spec.VolumeSnapshotLocations
|
||||
|
||||
r.snapshotVolumes = test.backup.Spec.SnapshotVolumes
|
||||
r.volumeBackups = test.backup.Status.VolumeBackups
|
||||
r.volumeSnapshots = test.volumeSnapshots
|
||||
}
|
||||
|
||||
res, err := r.executePVAction(test.obj)
|
||||
|
@ -1298,9 +1366,9 @@ func TestExecutePVAction(t *testing.T) {
|
|||
require.NoError(t, err)
|
||||
|
||||
if test.expectSetVolumeID {
|
||||
assert.Equal(t, test.volumeID, fakeBlockStore.VolumeIDSet)
|
||||
assert.Equal(t, test.volumeID, testBlockStoreGetter.fakeBlockStore.VolumeIDSet)
|
||||
} else {
|
||||
assert.Equal(t, "", fakeBlockStore.VolumeIDSet)
|
||||
assert.Equal(t, "", testBlockStoreGetter.fakeBlockStore.VolumeIDSet)
|
||||
}
|
||||
assert.Equal(t, test.expectedRes, res)
|
||||
})
|
||||
|
@ -1622,6 +1690,22 @@ type fakeAction struct {
|
|||
resource string
|
||||
}
|
||||
|
||||
type fakeBlockStoreGetter struct {
|
||||
fakeBlockStore *arktest.FakeBlockStore
|
||||
volumeMap map[api.VolumeBackupInfo]string
|
||||
volumeID string
|
||||
}
|
||||
|
||||
func (r *fakeBlockStoreGetter) GetBlockStore(provider string) (cloudprovider.BlockStore, error) {
|
||||
if r.fakeBlockStore == nil {
|
||||
r.fakeBlockStore = &arktest.FakeBlockStore{
|
||||
RestorableVolumes: r.volumeMap,
|
||||
VolumeID: r.volumeID,
|
||||
}
|
||||
}
|
||||
return r.fakeBlockStore, nil
|
||||
}
|
||||
|
||||
func newFakeAction(resource string) *fakeAction {
|
||||
return &fakeAction{resource}
|
||||
}
|
||||
|
|
|
@ -36,6 +36,9 @@ type SnapshotSpec struct {
|
|||
// Location is the name of the VolumeSnapshotLocation where this snapshot is stored.
|
||||
Location string `json:"location"`
|
||||
|
||||
// PersistentVolumeName is the Kubernetes name for the volume.
|
||||
PersistentVolumeName string `json:persistentVolumeName`
|
||||
|
||||
// ProviderVolumeID is the provider's ID for the volume.
|
||||
ProviderVolumeID string `json:"providerVolumeID"`
|
||||
|
||||
|
|
Loading…
Reference in New Issue