From 16cecba81fb579ba26e39672f713f0a14f41ca3a Mon Sep 17 00:00:00 2001 From: Swapnil Mhamane Date: Wed, 25 Sep 2019 14:49:27 +0530 Subject: [PATCH] Support config file for configration Signed-off-by: Swapnil Mhamane --- Makefile | 2 + cmd/initializer.go | 56 +-- cmd/miscellaneous.go | 10 - cmd/options.go | 228 +++++++++++ cmd/restore.go | 66 +--- cmd/server.go | 369 +---------------- cmd/snapshot.go | 74 +--- cmd/types.go | 64 +-- example/00-backup-restore-server-config.yaml | 45 +++ go.mod | 2 +- pkg/{etcdutil => defragmentor}/defrag.go | 38 +- pkg/{etcdutil => defragmentor}/defrag_test.go | 43 +- .../defragmentor_suite_test.go} | 8 +- pkg/etcdutil/etcdutil.go | 42 +- pkg/etcdutil/init.go | 54 +++ pkg/etcdutil/types.go | 29 +- pkg/initializer/initializer.go | 14 +- .../validator/datavalidator_test.go | 13 + .../validator/validator_suite_test.go | 56 +-- pkg/server/backuprestoreserver.go | 332 ++++++++++++++++ pkg/server/httpAPI.go | 4 +- pkg/server/init.go | 119 ++++++ pkg/server/types.go | 36 ++ pkg/snapshot/restorer/init.go | 80 ++++ pkg/snapshot/restorer/restorer.go | 24 +- pkg/snapshot/restorer/restorer_suite_test.go | 59 +-- pkg/snapshot/restorer/restorer_test.go | 348 ++++++---------- pkg/snapshot/restorer/types.go | 23 +- pkg/snapshot/snapshotter/garbagecollector.go | 22 +- pkg/snapshot/snapshotter/init.go | 70 ++++ pkg/snapshot/snapshotter/snapshotter.go | 90 ++--- pkg/snapshot/snapshotter/snapshotter_test.go | 374 ++++++------------ pkg/snapshot/snapshotter/types.go | 27 +- pkg/snapstore/abs_snapstore.go | 8 +- pkg/snapstore/gcs_snapstore.go | 8 +- pkg/snapstore/init.go | 56 +++ pkg/snapstore/oss_snapstore.go | 10 +- pkg/snapstore/s3_snapstore.go | 10 +- pkg/snapstore/swift_snapstore.go | 8 +- pkg/snapstore/types.go | 10 +- pkg/wrappers/duration.go | 46 +++ 41 files changed, 1624 insertions(+), 1353 deletions(-) create mode 100644 cmd/options.go create mode 100644 example/00-backup-restore-server-config.yaml rename pkg/{etcdutil => defragmentor}/defrag.go (77%) rename pkg/{etcdutil => defragmentor}/defrag_test.go (74%) rename pkg/{etcdutil/etcdutil_suite_test.go => defragmentor/defragmentor_suite_test.go} (91%) create mode 100644 pkg/etcdutil/init.go create mode 100644 pkg/server/backuprestoreserver.go create mode 100644 pkg/server/init.go create mode 100644 pkg/server/types.go create mode 100644 pkg/snapshot/restorer/init.go create mode 100644 pkg/snapshot/snapshotter/init.go create mode 100644 pkg/snapstore/init.go create mode 100644 pkg/wrappers/duration.go diff --git a/Makefile b/Makefile index d53ab73c2..a63b6b9a6 100644 --- a/Makefile +++ b/Makefile @@ -20,6 +20,8 @@ BUILD_DIR := build BIN_DIR := bin COVERPROFILE := test/output/coverprofile.out +.DEFAULT_GOAL := build-local + .PHONY: revendor revendor: @env GO111MODULE=on go mod vendor -v diff --git a/cmd/initializer.go b/cmd/initializer.go index b45b317e1..a3485c909 100644 --- a/cmd/initializer.go +++ b/cmd/initializer.go @@ -17,13 +17,11 @@ package cmd import ( "context" "fmt" - "path" "github.com/coreos/etcd/pkg/types" "github.com/gardener/etcd-backup-restore/pkg/initializer" "github.com/gardener/etcd-backup-restore/pkg/initializer/validator" "github.com/gardener/etcd-backup-restore/pkg/snapshot/restorer" - "github.com/gardener/etcd-backup-restore/pkg/snapstore" "github.com/sirupsen/logrus" "github.com/spf13/cobra" ) @@ -31,27 +29,33 @@ import ( // NewInitializeCommand returns the command to initialize etcd by validating the data // directory and restoring from cloud store if needed. func NewInitializeCommand(ctx context.Context) *cobra.Command { - + opts := newInitializerOptions() // restoreCmd represents the restore command initializeCmd := &cobra.Command{ Use: "initialize", Short: "initialize an etcd instance.", Long: fmt.Sprintf(`Initializes an etcd instance. Data directory is checked for corruption and restored in case of corruption.`), Run: func(cmd *cobra.Command, args []string) { - var mode validator.Mode logger := logrus.New() + if err := opts.validate(); err != nil { + logger.Fatalf("failed to validate the options: %v", err) + return + } - clusterUrlsMap, err := types.NewURLsMap(restoreCluster) + opts.complete() + + clusterUrlsMap, err := types.NewURLsMap(opts.restorerOptions.restorationConfig.InitialCluster) if err != nil { logger.Fatalf("failed creating url map for restore cluster: %v", err) } - peerUrls, err := types.NewURLs(restorePeerURLs) + peerUrls, err := types.NewURLs(opts.restorerOptions.restorationConfig.InitialAdvertisePeerURLs) if err != nil { logger.Fatalf("failed parsing peers urls for restore cluster: %v", err) } - switch validator.Mode(validationMode) { + var mode validator.Mode + switch validator.Mode(opts.validatorOptions.ValidationMode) { case validator.Full: mode = validator.Full case validator.Sanity: @@ -60,41 +64,19 @@ func NewInitializeCommand(ctx context.Context) *cobra.Command { logger.Fatal("validation-mode can only be one of these values [full/sanity]") } - options := &restorer.RestoreOptions{ - RestoreDataDir: path.Clean(restoreDataDir), - Name: restoreName, - ClusterURLs: clusterUrlsMap, - PeerURLs: peerUrls, - ClusterToken: restoreClusterToken, - SkipHashCheck: skipHashCheck, - MaxFetchers: restoreMaxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, + restoreOptions := &restorer.RestoreOptions{ + Config: opts.restorerOptions.restorationConfig, + ClusterURLs: clusterUrlsMap, + PeerURLs: peerUrls, } - var snapstoreConfig *snapstore.Config - if storageProvider != "" { - snapstoreConfig = &snapstore.Config{ - Provider: storageProvider, - Container: storageContainer, - Prefix: path.Join(storagePrefix, backupFormatVersion), - MaxParallelChunkUploads: maxParallelChunkUploads, - TempDir: snapstoreTempDir, - } - } - etcdInitializer := initializer.NewInitializer(options, snapstoreConfig, logger) - err = etcdInitializer.Initialize(mode, failBelowRevision) - if err != nil { + etcdInitializer := initializer.NewInitializer(restoreOptions, opts.restorerOptions.snapstoreConfig, logger) + if err := etcdInitializer.Initialize(mode, opts.validatorOptions.FailBelowRevision); err != nil { logger.Fatalf("initializer failed. %v", err) } }, } - initializeEtcdFlags(initializeCmd) - initializeSnapstoreFlags(initializeCmd) - initializeValidatorFlags(initializeCmd) - initializeCmd.Flags().Int64Var(&failBelowRevision, "experimental-fail-below-revision", 0, "minimum required etcd revision, below which validation fails") - return initializeCmd -} -func initializeValidatorFlags(cmd *cobra.Command) { - cmd.Flags().StringVar(&validationMode, "validation-mode", string(validator.Full), "mode to do data initialization[full/sanity]") + opts.addFlags(initializeCmd.Flags()) + return initializeCmd } diff --git a/cmd/miscellaneous.go b/cmd/miscellaneous.go index 471ce937c..0a0d95c6a 100644 --- a/cmd/miscellaneous.go +++ b/cmd/miscellaneous.go @@ -18,18 +18,8 @@ import ( "runtime" ver "github.com/gardener/etcd-backup-restore/pkg/version" - "github.com/spf13/cobra" ) -// initializeSnapstoreFlags adds the snapstore related flags to -func initializeSnapstoreFlags(cmd *cobra.Command) { - cmd.Flags().StringVar(&storageProvider, "storage-provider", "", "snapshot storage provider") - cmd.Flags().StringVar(&storageContainer, "store-container", "", "container which will be used as snapstore") - cmd.Flags().StringVar(&storagePrefix, "store-prefix", "", "prefix or directory inside container under which snapstore is created") - cmd.Flags().IntVar(&maxParallelChunkUploads, "max-parallel-chunk-uploads", 5, "maximum number of parallel chunk uploads allowed ") - cmd.Flags().StringVar(&snapstoreTempDir, "snapstore-temp-directory", "/tmp", "temporary directory for processing") -} - func printVersionInfo() { logger.Infof("etcd-backup-restore Version: %s", ver.Version) logger.Infof("Git SHA: %s", ver.GitSHA) diff --git a/cmd/options.go b/cmd/options.go new file mode 100644 index 000000000..8c0f0a34b --- /dev/null +++ b/cmd/options.go @@ -0,0 +1,228 @@ +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cmd + +import ( + "context" + "io/ioutil" + + "github.com/gardener/etcd-backup-restore/pkg/snapshot/snapshotter" + + "github.com/gardener/etcd-backup-restore/pkg/etcdutil" + + "github.com/gardener/etcd-backup-restore/pkg/snapstore" + + "github.com/gardener/etcd-backup-restore/pkg/initializer/validator" + "github.com/gardener/etcd-backup-restore/pkg/snapshot/restorer" + + "github.com/gardener/etcd-backup-restore/pkg/server" + "github.com/ghodss/yaml" + "github.com/sirupsen/logrus" + flag "github.com/spf13/pflag" +) + +type serverOptions struct { + ConfigFile string + Version bool + LogLevel uint32 + Logger *logrus.Logger + Config *server.BackupRestoreComponentConfig +} + +// newServerOptions returns a new Options object. +func newServerOptions() *serverOptions { + logger := logrus.New() + logger.SetLevel(logrus.InfoLevel) + + return &serverOptions{ + LogLevel: 4, + Version: false, + Config: server.NewBackupRestoreComponentConfig(), + Logger: logger, + } +} + +func (o *serverOptions) validate() error { + return o.Config.Validate() +} + +func (o *serverOptions) addFlags(fs *flag.FlagSet) { + fs.StringVar(&o.ConfigFile, "config-file", o.ConfigFile, "path to the configuration file") + fs.Uint32Var(&o.LogLevel, "log-level", o.LogLevel, "verbosity level of logs") + o.Config.AddFlags(fs) +} + +func (o *serverOptions) complete() { + o.Config.Complete() + o.Logger.SetLevel(logrus.Level(o.LogLevel)) +} + +func (o *serverOptions) loadConfigFromFile() error { + if len(o.ConfigFile) != 0 { + data, err := ioutil.ReadFile(o.ConfigFile) + if err != nil { + return err + } + config := server.NewBackupRestoreComponentConfig() + if err := yaml.Unmarshal(data, config); err != nil { + return err + } + o.Config = config + } + // TODO: Overwrite config with flags + return nil +} + +func (o *serverOptions) run(ctx context.Context) error { + brServer, err := server.NewBackupRestoreServer(o.Logger, o.Config) + if err != nil { + return err + } + return brServer.Run(ctx) +} + +type initializerOptions struct { + validatorOptions *validatorOptions + restorerOptions *restorerOptions +} + +// newInitializerOptions returns the validation config. +func newInitializerOptions() *initializerOptions { + return &initializerOptions{ + validatorOptions: newValidatorOptions(), + restorerOptions: newRestorerOptions(), + } +} + +// AddFlags adds the flags to flagset. +func (c *initializerOptions) addFlags(fs *flag.FlagSet) { + c.validatorOptions.addFlags(fs) + c.restorerOptions.addFlags(fs) +} + +// Validate validates the config. +func (c *initializerOptions) validate() error { + if err := c.validatorOptions.validate(); err != nil { + return err + } + + return c.restorerOptions.validate() +} + +// Complete completes the config. +func (c *initializerOptions) complete() { + c.restorerOptions.complete() +} + +type restorerOptions struct { + restorationConfig *restorer.RestorationConfig + snapstoreConfig *snapstore.Config +} + +// newRestorerOptions returns the validation config. +func newRestorerOptions() *restorerOptions { + return &restorerOptions{ + restorationConfig: restorer.NewRestorationConfig(), + snapstoreConfig: snapstore.NewSnapstoreConfig(), + } +} + +// AddFlags adds the flags to flagset. +func (c *restorerOptions) addFlags(fs *flag.FlagSet) { + c.restorationConfig.AddFlags(fs) + c.snapstoreConfig.AddFlags(fs) +} + +// Validate validates the config. +func (c *restorerOptions) validate() error { + if err := c.snapstoreConfig.Validate(); err != nil { + return err + } + + return c.restorationConfig.Validate() +} + +// complete completes the config. +func (c *restorerOptions) complete() { + c.snapstoreConfig.Complete() +} + +type validatorOptions struct { + ValidationMode string `json:"validationMode,omitempty"` + FailBelowRevision int64 `json:"experimentalFailBelowRevision,omitempty"` +} + +// newValidatorOptions returns the validation config. +func newValidatorOptions() *validatorOptions { + return &validatorOptions{ + ValidationMode: string(validator.Full), + } +} + +// AddFlags adds the flags to flagset. +func (c *validatorOptions) addFlags(fs *flag.FlagSet) { + fs.StringVar(&c.ValidationMode, "validation-mode", string(c.ValidationMode), "mode to do data initialization[full/sanity]") + fs.Int64Var(&c.FailBelowRevision, "experimental-fail-below-revision", c.FailBelowRevision, "minimum required etcd revision, below which validation fails") +} + +// Validate validates the config. +func (c *validatorOptions) validate() error { + return nil +} + +type snapshotterOptions struct { + etcdConnectionConfig *etcdutil.EtcdConnectionConfig + snapstoreConfig *snapstore.Config + snapshotterConfig *snapshotter.Config + defragmentationSchedule string +} + +// newSnapshotterOptions returns the snapshotter options. +func newSnapshotterOptions() *snapshotterOptions { + return &snapshotterOptions{ + etcdConnectionConfig: etcdutil.NewEtcdConnectionConfig(), + snapstoreConfig: snapstore.NewSnapstoreConfig(), + snapshotterConfig: snapshotter.NewSnapshotterConfig(), + defragmentationSchedule: "0 0 */3 * *", + } +} + +// AddFlags adds the flags to flagset. +func (c *snapshotterOptions) addFlags(fs *flag.FlagSet) { + c.etcdConnectionConfig.AddFlags(fs) + c.snapstoreConfig.AddFlags(fs) + c.snapshotterConfig.AddFlags(fs) + + // Miscellaneous + fs.StringVar(&c.defragmentationSchedule, "defragmentation-schedule", c.defragmentationSchedule, "schedule to defragment etcd data directory") +} + +// Validate validates the config. +func (c *snapshotterOptions) validate() error { + if err := c.snapstoreConfig.Validate(); err != nil { + return err + } + + if err := c.snapshotterConfig.Validate(); err != nil { + return err + } + + return c.etcdConnectionConfig.Validate() +} + +// complete completes the config. +func (c *snapshotterOptions) complete() { + c.snapstoreConfig.Complete() +} diff --git a/cmd/restore.go b/cmd/restore.go index 416117264..b2f81c809 100644 --- a/cmd/restore.go +++ b/cmd/restore.go @@ -17,7 +17,6 @@ package cmd import ( "context" "fmt" - "path" "github.com/coreos/etcd/pkg/types" "github.com/gardener/etcd-backup-restore/pkg/miscellaneous" @@ -29,7 +28,7 @@ import ( // NewRestoreCommand returns the command to restore func NewRestoreCommand(ctx context.Context) *cobra.Command { - + opts := newRestorerOptions() // restoreCmd represents the restore command restoreCmd := &cobra.Command{ Use: "restore", @@ -41,26 +40,28 @@ func NewRestoreCommand(ctx context.Context) *cobra.Command { - Restore etcd data diretory from full snapshot. */ logger := logrus.New() - clusterUrlsMap, err := types.NewURLsMap(restoreCluster) + if err := opts.validate(); err != nil { + logger.Fatalf("failed to validate the options: %v", err) + return + } + + opts.complete() + + clusterUrlsMap, err := types.NewURLsMap(opts.restorationConfig.InitialCluster) if err != nil { logger.Fatalf("failed creating url map for restore cluster: %v", err) } - peerUrls, err := types.NewURLs(restorePeerURLs) + peerUrls, err := types.NewURLs(opts.restorationConfig.InitialAdvertisePeerURLs) if err != nil { logger.Fatalf("failed parsing peers urls for restore cluster: %v", err) } - snapstoreConfig := &snapstore.Config{ - Provider: storageProvider, - Container: storageContainer, - Prefix: path.Join(storagePrefix, backupFormatVersion), - MaxParallelChunkUploads: maxParallelChunkUploads, - TempDir: snapstoreTempDir, - } - store, err := snapstore.GetSnapstore(snapstoreConfig) + + store, err := snapstore.GetSnapstore(opts.snapstoreConfig) if err != nil { logger.Fatalf("failed to create snapstore from configured storage provider: %v", err) } + logger.Info("Finding latest set of snapshot to recover from...") baseSnap, deltaSnapList, err := miscellaneous.GetLatestFullSnapshotAndDeltaSnapList(store) if err != nil { @@ -74,20 +75,14 @@ func NewRestoreCommand(ctx context.Context) *cobra.Command { rs := restorer.NewRestorer(store, logrus.NewEntry(logger)) options := &restorer.RestoreOptions{ - RestoreDataDir: path.Clean(restoreDataDir), - Name: restoreName, - BaseSnapshot: *baseSnap, - DeltaSnapList: deltaSnapList, - ClusterURLs: clusterUrlsMap, - PeerURLs: peerUrls, - ClusterToken: restoreClusterToken, - SkipHashCheck: skipHashCheck, - MaxFetchers: restoreMaxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, + Config: opts.restorationConfig, + BaseSnapshot: *baseSnap, + DeltaSnapList: deltaSnapList, + ClusterURLs: clusterUrlsMap, + PeerURLs: peerUrls, } - err = rs.Restore(*options) - if err != nil { + if err := rs.Restore(*options); err != nil { logger.Fatalf("Failed to restore snapshot: %v", err) return } @@ -95,27 +90,6 @@ func NewRestoreCommand(ctx context.Context) *cobra.Command { }, } - initializeSnapstoreFlags(restoreCmd) - initializeEtcdFlags(restoreCmd) + opts.addFlags(restoreCmd.Flags()) return restoreCmd } - -// initializeEtcdFlags adds the etcd related flags to -func initializeEtcdFlags(cmd *cobra.Command) { - cmd.Flags().StringVarP(&restoreDataDir, "data-dir", "d", fmt.Sprintf("%s.etcd", defaultName), "path to the data directory") - cmd.Flags().StringVar(&restoreCluster, "initial-cluster", initialClusterFromName(defaultName), "initial cluster configuration for restore bootstrap") - cmd.Flags().StringVar(&restoreClusterToken, "initial-cluster-token", "etcd-cluster", "initial cluster token for the etcd cluster during restore bootstrap") - cmd.Flags().StringArrayVar(&restorePeerURLs, "initial-advertise-peer-urls", []string{defaultInitialAdvertisePeerURLs}, "list of this member's peer URLs to advertise to the rest of the cluster") - cmd.Flags().StringVar(&restoreName, "name", defaultName, "human-readable name for this member") - cmd.Flags().BoolVar(&skipHashCheck, "skip-hash-check", false, "ignore snapshot integrity hash value (required if copied from data directory)") - cmd.Flags().IntVar(&restoreMaxFetchers, "max-fetchers", 6, "maximum number of threads that will fetch delta snapshots in parallel") - cmd.Flags().Int64Var(&embeddedEtcdQuotaBytes, "embedded-etcd-quota-bytes", int64(8*1024*1024*1024), "maximum backend quota for the embedded etcd used for applying delta snapshots") -} - -func initialClusterFromName(name string) string { - n := name - if name == "" { - n = defaultName - } - return fmt.Sprintf("%s=http://localhost:2380", n) -} diff --git a/cmd/server.go b/cmd/server.go index b72a12a02..e5ef8ab5d 100644 --- a/cmd/server.go +++ b/cmd/server.go @@ -16,380 +16,47 @@ package cmd import ( "context" - "fmt" - "net/http" - "os" - "path" - "sync/atomic" - "time" - "github.com/coreos/etcd/pkg/types" - "github.com/gardener/etcd-backup-restore/pkg/errors" - "github.com/gardener/etcd-backup-restore/pkg/etcdutil" - "github.com/gardener/etcd-backup-restore/pkg/initializer" - "github.com/gardener/etcd-backup-restore/pkg/metrics" - "github.com/gardener/etcd-backup-restore/pkg/server" - "github.com/gardener/etcd-backup-restore/pkg/snapshot/restorer" - "github.com/gardener/etcd-backup-restore/pkg/snapshot/snapshotter" - "github.com/gardener/etcd-backup-restore/pkg/snapstore" - "github.com/prometheus/client_golang/prometheus" - cron "github.com/robfig/cron/v3" - "github.com/sirupsen/logrus" + "github.com/ghodss/yaml" + "github.com/spf13/cobra" ) // NewServerCommand create cobra command for snapshot func NewServerCommand(ctx context.Context) *cobra.Command { + opts := newServerOptions() var serverCmd = &cobra.Command{ Use: "server", Short: "start the http server with backup scheduler.", Long: `Server will keep listening for http request to deliver its functionality through http endpoints.`, Run: func(cmd *cobra.Command, args []string) { - // TODO: Refactor the code to move it to coordination logic to server package. printVersionInfo() - // TODO: move this to validate config - clusterUrlsMap, err := types.NewURLsMap(restoreCluster) - if err != nil { - logger.Fatalf("failed creating url map for restore cluster: %v", err) - } - peerUrls, err := types.NewURLs(restorePeerURLs) - if err != nil { - logger.Fatalf("failed parsing peers urls for restore cluster: %v", err) - } - - defragSchedule, err := cron.ParseStandard(defragmentationSchedule) - if err != nil { - logger.Fatalf("failed to parse defragmentation schedule: %v", err) - } - - // TODO: move this to build/complete config - options := &restorer.RestoreOptions{ - RestoreDataDir: path.Clean(restoreDataDir), - Name: restoreName, - ClusterURLs: clusterUrlsMap, - PeerURLs: peerUrls, - ClusterToken: restoreClusterToken, - SkipHashCheck: skipHashCheck, - MaxFetchers: restoreMaxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, + if err := opts.loadConfigFromFile(); err != nil { + opts.Logger.Fatalf("failed to load the config from file: %v", err) + return } - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - etcdEndpoints, - etcdUsername, - etcdPassword) - - if storageProvider == "" { - logger.Warnf("No snapstore storage provider configured. Will not start backup schedule.") - runServerWithoutSnapshotter(ctx, tlsConfig, options, defragSchedule) + if err := opts.validate(); err != nil { + opts.Logger.Fatalf("failed to validate the options: %v", err) return } - runServerWithSnapshotter(ctx, tlsConfig, options, defragSchedule) - }, - } - - initializeServerFlags(serverCmd) - initializeSnapshotterFlags(serverCmd) - initializeSnapstoreFlags(serverCmd) - initializeEtcdFlags(serverCmd) - return serverCmd -} - -// startHTTPServer creates and starts the HTTP handler -// with status 503 (Service Unavailable) -func startHTTPServer(initializer initializer.Initializer, ssr *snapshotter.Snapshotter) (*server.HTTPHandler, error) { - enableTLS := serverTLSCertFile != "" && serverTLSKeyFile != "" - if enableTLS { - // Check for existence of server cert and key files before proceeding - if _, err := os.Stat(serverTLSCertFile); err != nil { - logger.Errorf("TLS enabled but server TLS cert file is invalid. Will not start HTTPS server: %v", err) - return nil, err - } - if _, err := os.Stat(serverTLSKeyFile); err != nil { - logger.Errorf("TLS enabled but server TLS key file is invalid. Will not start HTTPS server: %v", err) - return nil, err - } - } - - // Start http handler with Error state and wait till snapshotter is up - // and running before setting the status to OK. - handler := &server.HTTPHandler{ - Port: port, - Initializer: initializer, - Snapshotter: ssr, - Logger: logger, - StopCh: make(chan struct{}), - EnableProfiling: enableProfiling, - ReqCh: make(chan struct{}), - AckCh: make(chan struct{}), - EnableTLS: enableTLS, - ServerTLSCertFile: serverTLSCertFile, - ServerTLSKeyFile: serverTLSKeyFile, - } - handler.SetStatus(http.StatusServiceUnavailable) - logger.Info("Registering the http request handlers...") - handler.RegisterHandler() - logger.Info("Starting the http server...") - go handler.Start() - - return handler, nil -} -// runServerWithoutSnapshotter runs the etcd-backup-restore -// for the case where snapshotter is configured correctly -func runServerWithSnapshotter(ctx context.Context, tlsConfig *etcdutil.TLSConfig, restoreOpts *restorer.RestoreOptions, defragSchedule cron.Schedule) { - ackCh := make(chan struct{}) - snapstoreConfig := &snapstore.Config{ - Provider: storageProvider, - Container: storageContainer, - Prefix: path.Join(storagePrefix, backupFormatVersion), - MaxParallelChunkUploads: maxParallelChunkUploads, - TempDir: snapstoreTempDir, - } - - etcdInitializer := initializer.NewInitializer(restoreOpts, snapstoreConfig, logger) - - ss, err := snapstore.GetSnapstore(snapstoreConfig) - if err != nil { - logger.Fatalf("Failed to create snapstore from configured storage provider: %v", err) - } - logger.Infof("Created snapstore from provider: %s", storageProvider) + opts.complete() - snapshotterConfig, err := snapshotter.NewSnapshotterConfig( - fullSnapshotSchedule, - ss, - maxBackups, - deltaSnapshotMemoryLimit, - deltaSnapshotInterval, - etcdConnectionTimeout, - garbageCollectionPeriod, - garbageCollectionPolicy, - tlsConfig) - if err != nil { - logger.Fatalf("failed to create snapshotter config: %v", err) - } - - logger.Infof("Creating snapshotter...") - ssr := snapshotter.NewSnapshotter( - logrus.NewEntry(logger), - snapshotterConfig, - ) - - handler, err := startHTTPServer(etcdInitializer, ssr) - if err != nil { - logger.Fatalf("Failed to start HTTP server: %v", err) - } - defer handler.Stop() - - ssrStopCh := make(chan struct{}) - go handleSsrStopRequest(ctx, handler, ssr, ackCh, ssrStopCh) - go handleAckState(handler, ackCh) - - go etcdutil.DefragDataPeriodically(ctx, tlsConfig, defragSchedule, etcdConnectionTimeout, ssr.TriggerFullSnapshot, logrus.NewEntry(logger)) - - runEtcdProbeLoopWithSnapshotter(ctx, tlsConfig, handler, ssr, ssrStopCh, ackCh) -} - -// runEtcdProbeLoopWithoutSnapshotter runs the etcd probe loop -// for the case where snapshotter is configured correctly -func runEtcdProbeLoopWithSnapshotter(ctx context.Context, tlsConfig *etcdutil.TLSConfig, handler *server.HTTPHandler, ssr *snapshotter.Snapshotter, ssrStopCh chan struct{}, ackCh chan struct{}) { - var ( - err error - initialDeltaSnapshotTaken bool - ) - - for { - logger.Infof("Probing etcd...") - select { - case <-ctx.Done(): - logger.Info("Shutting down...") - return - default: - err = ProbeEtcd(tlsConfig) - } - if err != nil { - logger.Errorf("Failed to probe etcd: %v", err) - handler.SetStatus(http.StatusServiceUnavailable) - continue - } - - // The decision to either take an initial delta snapshot or - // or a full snapshot directly is based on whether there has - // been a previous full snapshot (if not, we assume the etcd - // to be a fresh etcd) or it has been more than 24 hours since - // the last full snapshot was taken. - // If this is not the case, we take a delta snapshot by first - // collecting all the delta events since the previous snapshot - // and take a delta snapshot of these (there may be multiple - // delta snapshots based on the amount of events collected and - // the delta snapshot memory limit), after which a full snapshot - // is taken and the regular snapshot schedule comes into effect. - - // TODO: write code to find out if prev full snapshot is older than it is - // supposed to be, according to the given cron schedule, instead of the - // hard-coded "24 hours" full snapshot interval - if ssr.PrevFullSnapshot != nil && time.Since(ssr.PrevFullSnapshot.CreatedOn).Hours() <= 24 { - ssrStopped, err := ssr.CollectEventsSincePrevSnapshot(ssrStopCh) - if ssrStopped { - logger.Info("Snapshotter stopped.") - ackCh <- emptyStruct - handler.SetStatus(http.StatusServiceUnavailable) - logger.Info("Shutting down...") + optsJSON, err := yaml.Marshal(opts.Config) + if err != nil { + opts.Logger.Fatalf("failed to print the options: %v", err) return } - if err == nil { - if err := ssr.TakeDeltaSnapshot(); err != nil { - logger.Warnf("Failed to take first delta snapshot: snapshotter failed with error: %v", err) - continue - } - initialDeltaSnapshotTaken = true - } else { - logger.Warnf("Failed to collect events for first delta snapshot(s): %v", err) - } - } - if !initialDeltaSnapshotTaken { - // need to take a full snapshot here - metrics.SnapshotRequired.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindDelta}).Set(0) - metrics.SnapshotRequired.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull}).Set(1) - if err := ssr.TakeFullSnapshotAndResetTimer(); err != nil { - logger.Errorf("Failed to take substitute first full snapshot: %v", err) - continue - } - } + opts.Logger.Infof("%s", optsJSON) - // set server's healthz endpoint status to OK so that - // etcd is marked as ready to serve traffic - handler.SetStatus(http.StatusOK) - - ssr.SsrStateMutex.Lock() - ssr.SsrState = snapshotter.SnapshotterActive - ssr.SsrStateMutex.Unlock() - gcStopCh := make(chan struct{}) - go ssr.RunGarbageCollector(gcStopCh) - logger.Infof("Starting snapshotter...") - if err := ssr.Run(ssrStopCh, initialDeltaSnapshotTaken); err != nil { - if etcdErr, ok := err.(*errors.EtcdError); ok == true { - logger.Errorf("Snapshotter failed with etcd error: %v", etcdErr) - } else { - logger.Fatalf("Snapshotter failed with error: %v", err) + if err := opts.run(ctx); err != nil { + opts.Logger.Fatalf("failed to run server: %v", err) } - } - logger.Infof("Snapshotter stopped.") - ackCh <- emptyStruct - handler.SetStatus(http.StatusServiceUnavailable) - close(gcStopCh) - } -} - -// runServerWithoutSnapshotter runs the etcd-backup-restore -// for the case where snapshotter is not configured -func runServerWithoutSnapshotter(ctx context.Context, tlsConfig *etcdutil.TLSConfig, restoreOpts *restorer.RestoreOptions, defragSchedule cron.Schedule) { - etcdInitializer := initializer.NewInitializer(restoreOpts, nil, logger) - - // If no storage provider is given, snapshotter will be nil, in which - // case the status is set to OK as soon as etcd probe is successful - handler, err := startHTTPServer(etcdInitializer, nil) - if err != nil { - logger.Fatalf("Failed to start HTTP server: %v", err) - } - defer handler.Stop() - - // start defragmentation without trigerring full snapshot - // after each successful data defragmentation - go etcdutil.DefragDataPeriodically(ctx, tlsConfig, defragSchedule, etcdConnectionTimeout, nil, logrus.NewEntry(logger)) - - runEtcdProbeLoopWithoutSnapshotter(ctx, tlsConfig, handler) -} - -// runEtcdProbeLoopWithoutSnapshotter runs the etcd probe loop -// for the case where snapshotter is not configured -func runEtcdProbeLoopWithoutSnapshotter(ctx context.Context, tlsConfig *etcdutil.TLSConfig, handler *server.HTTPHandler) { - var err error - for { - logger.Infof("Probing etcd...") - select { - case <-ctx.Done(): - logger.Info("Shutting down...") - return - default: - err = ProbeEtcd(tlsConfig) - } - if err != nil { - logger.Errorf("Failed to probe etcd: %v", err) - handler.SetStatus(http.StatusServiceUnavailable) - continue - } - - handler.SetStatus(http.StatusOK) - <-ctx.Done() - handler.SetStatus(http.StatusServiceUnavailable) - logger.Infof("Received stop signal. Terminating !!") - return - } -} - -// ProbeEtcd will make the snapshotter probe for etcd endpoint to be available -// before it starts taking regular snapshots. -func ProbeEtcd(tlsConfig *etcdutil.TLSConfig) error { - client, err := etcdutil.GetTLSClientForEtcd(tlsConfig) - if err != nil { - return &errors.EtcdError{ - Message: fmt.Sprintf("failed to create etcd client: %v", err), - } - } - - ctx, cancel := context.WithTimeout(context.TODO(), etcdConnectionTimeout) - defer cancel() - if _, err := client.Get(ctx, "foo"); err != nil { - logger.Errorf("Failed to connect to client: %v", err) - return err - } - return nil -} - -func handleAckState(handler *server.HTTPHandler, ackCh chan struct{}) { - for { - <-ackCh - if atomic.CompareAndSwapUint32(&handler.AckState, server.HandlerAckWaiting, server.HandlerAckDone) { - handler.AckCh <- emptyStruct - } - } -} - -// handleSsrStopRequest responds to handlers request and stop interrupt. -func handleSsrStopRequest(ctx context.Context, handler *server.HTTPHandler, ssr *snapshotter.Snapshotter, ackCh, ssrStopCh chan struct{}) { - for { - var ok bool - select { - case _, ok = <-handler.ReqCh: - case _, ok = <-ctx.Done(): - } - - ssr.SsrStateMutex.Lock() - if ssr.SsrState == snapshotter.SnapshotterActive { - ssr.SsrStateMutex.Unlock() - ssrStopCh <- emptyStruct - } else { - ssr.SsrState = snapshotter.SnapshotterInactive - ssr.SsrStateMutex.Unlock() - ackCh <- emptyStruct - } - if !ok { - return - } + }, } -} -// initializeServerFlags adds the flags to -func initializeServerFlags(serverCmd *cobra.Command) { - serverCmd.Flags().IntVarP(&port, "server-port", "p", defaultServerPort, "port on which server should listen") - serverCmd.Flags().BoolVar(&enableProfiling, "enable-profiling", false, "enable profiling") - serverCmd.Flags().StringVar(&serverTLSCertFile, "server-cert", "", "TLS certificate file for backup-restore server") - serverCmd.Flags().StringVar(&serverTLSKeyFile, "server-key", "", "TLS key file for backup-restore server") + opts.addFlags(serverCmd.Flags()) + return serverCmd } diff --git a/cmd/snapshot.go b/cmd/snapshot.go index 7a22176dd..3db493d6a 100644 --- a/cmd/snapshot.go +++ b/cmd/snapshot.go @@ -16,10 +16,9 @@ package cmd import ( "context" - "path" - "time" - "github.com/gardener/etcd-backup-restore/pkg/etcdutil" + "github.com/gardener/etcd-backup-restore/pkg/defragmentor" + "github.com/gardener/etcd-backup-restore/pkg/snapshot/snapshotter" "github.com/gardener/etcd-backup-restore/pkg/snapstore" cron "github.com/robfig/cron/v3" @@ -29,6 +28,7 @@ import ( // NewSnapshotCommand create cobra command for snapshot func NewSnapshotCommand(ctx context.Context) *cobra.Command { + opts := newSnapshotterOptions() var command = &cobra.Command{ Use: "snapshot", Short: "takes the snapshot of etcd periodically.", @@ -36,50 +36,31 @@ func NewSnapshotCommand(ctx context.Context) *cobra.Command { storing snapshots on various cloud storage providers as well as local disk location.`, Run: func(cmd *cobra.Command, args []string) { printVersionInfo() - snapstoreConfig := &snapstore.Config{ - Provider: storageProvider, - Container: storageContainer, - Prefix: path.Join(storagePrefix, backupFormatVersion), - MaxParallelChunkUploads: maxParallelChunkUploads, - TempDir: snapstoreTempDir, + logger := logrus.NewEntry(logrus.New()) + if err := opts.validate(); err != nil { + logger.Fatalf("failed to validate the options: %v", err) + return } - ss, err := snapstore.GetSnapstore(snapstoreConfig) + + opts.complete() + + ss, err := snapstore.GetSnapstore(opts.snapstoreConfig) if err != nil { logger.Fatalf("Failed to create snapstore from configured storage provider: %v", err) } - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - etcdEndpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := snapshotter.NewSnapshotterConfig( - fullSnapshotSchedule, - ss, - maxBackups, - deltaSnapshotMemoryLimit, - deltaSnapshotInterval, - etcdConnectionTimeout, - garbageCollectionPeriod, - garbageCollectionPolicy, - tlsConfig) + ssr, err := snapshotter.NewSnapshotter(logger, opts.snapshotterConfig, ss, opts.etcdConnectionConfig) if err != nil { - logger.Fatalf("failed to create snapstore config: %v", err) + logger.Fatalf("Failed to create snapshotter: %v", err) } - ssr := snapshotter.NewSnapshotter( - logrus.NewEntry(logger), - snapshotterConfig) - defragSchedule, err := cron.ParseStandard(defragmentationSchedule) + defragSchedule, err := cron.ParseStandard(opts.defragmentationSchedule) if err != nil { logger.Fatalf("failed to parse defragmentation schedule: %v", err) return } - go etcdutil.DefragDataPeriodically(ctx, tlsConfig, defragSchedule, etcdConnectionTimeout, ssr.TriggerFullSnapshot, logrus.NewEntry(logger)) + + go defragmentor.DefragDataPeriodically(ctx, opts.etcdConnectionConfig, defragSchedule, ssr.TriggerFullSnapshot, logger) go ssr.RunGarbageCollector(ctx.Done()) if err := ssr.Run(ctx.Done(), true); err != nil { @@ -89,27 +70,6 @@ storing snapshots on various cloud storage providers as well as local disk locat return }, } - initializeSnapstoreFlags(command) - initializeSnapshotterFlags(command) + opts.addFlags(command.Flags()) return command } - -// initializeSnapshotterFlags adds snapshotter related flags to -func initializeSnapshotterFlags(cmd *cobra.Command) { - cmd.Flags().StringSliceVarP(&etcdEndpoints, "endpoints", "e", []string{"127.0.0.1:2379"}, "comma separated list of etcd endpoints") - cmd.Flags().StringVarP(&fullSnapshotSchedule, "schedule", "s", "* */1 * * *", "schedule for snapshots") - cmd.Flags().DurationVarP(&deltaSnapshotInterval, "delta-snapshot-period", "i", snapshotter.DefaultDeltaSnapshotInterval, "Period after which delta snapshot will be persisted. If this value is set to be lesser than 1 seconds, delta snapshotting will be disabled.") - cmd.Flags().IntVar(&deltaSnapshotMemoryLimit, "delta-snapshot-memory-limit", snapshotter.DefaultDeltaSnapMemoryLimit, "memory limit after which delta snapshots will be taken") - cmd.Flags().IntVarP(&maxBackups, "max-backups", "m", snapshotter.DefaultMaxBackups, "maximum number of previous backups to keep") - cmd.Flags().DurationVar(&etcdConnectionTimeout, "etcd-connection-timeout", 30*time.Second, "etcd client connection timeout") - cmd.Flags().DurationVar(&garbageCollectionPeriod, "garbage-collection-period", 60*time.Second, "Period for garbage collecting old backups") - cmd.Flags().StringVar(&garbageCollectionPolicy, "garbage-collection-policy", snapshotter.GarbageCollectionPolicyExponential, "Policy for garbage collecting old backups") - cmd.Flags().BoolVar(&insecureTransport, "insecure-transport", true, "disable transport security for client connections") - cmd.Flags().BoolVar(&insecureSkipVerify, "insecure-skip-tls-verify", false, "skip server certificate verification") - cmd.Flags().StringVar(&certFile, "cert", "", "identify secure client using this TLS certificate file") - cmd.Flags().StringVar(&keyFile, "key", "", "identify secure client using this TLS key file") - cmd.Flags().StringVar(&caFile, "cacert", "", "verify certificates of TLS-enabled secure servers using this CA bundle") - cmd.Flags().StringVar(&etcdUsername, "etcd-username", "", "etcd server username, if one is required") - cmd.Flags().StringVar(&etcdPassword, "etcd-password", "", "etcd server password, if one is required") - cmd.Flags().StringVar(&defragmentationSchedule, "defragmentation-schedule", "0 0 */3 * *", "schedule to defragment etcd data directory") -} diff --git a/cmd/types.go b/cmd/types.go index b9150a53e..9d56181a7 100644 --- a/cmd/types.go +++ b/cmd/types.go @@ -14,66 +14,10 @@ package cmd -import ( - "time" - - "github.com/sirupsen/logrus" -) - -const ( - backupFormatVersion = "v1" - defaultServerPort = 8080 - defaultName = "default" - defaultInitialAdvertisePeerURLs = "http://localhost:2380" -) +import "github.com/sirupsen/logrus" var ( - logger = logrus.New() - version bool - //snapshotter flags - fullSnapshotSchedule string - etcdEndpoints []string - etcdUsername string - etcdPassword string - deltaSnapshotMemoryLimit int - deltaSnapshotInterval time.Duration - etcdConnectionTimeout time.Duration - garbageCollectionPeriod time.Duration - garbageCollectionPolicy string - maxBackups int - insecureTransport bool - insecureSkipVerify bool - certFile string - keyFile string - caFile string - defragmentationSchedule string - - //server flags - port int - enableProfiling bool - serverTLSCertFile string - serverTLSKeyFile string - - //restore flags - restoreCluster string - restoreClusterToken string - restoreDataDir string - restorePeerURLs []string - restoreName string - skipHashCheck bool - restoreMaxFetchers int - embeddedEtcdQuotaBytes int64 - - //snapstore flags - storageProvider string - storageContainer string - storagePrefix string - maxParallelChunkUploads int - snapstoreTempDir string - - //initializer flags - validationMode string - failBelowRevision int64 + version bool + logger = logrus.New() + emptyStruct struct{} ) - -var emptyStruct struct{} diff --git a/example/00-backup-restore-server-config.yaml b/example/00-backup-restore-server-config.yaml new file mode 100644 index 000000000..78a815434 --- /dev/null +++ b/example/00-backup-restore-server-config.yaml @@ -0,0 +1,45 @@ +etcdConnectionConfig: + endpoints: + - "http://localhost:2379" + # username: admin + # password: admin + connectionTimeout: 10s + # insecureTransport: true + # insecureSkipVerify: true + # certFile: "ssl/etcd/tls.crt" + # keyFile: "ssl/etcd/tls.key" + # caFile: "ssl/etcd/ca.crt" + +serverConfig: + port: 8080 + # enableProfiling: true + # server-cert: "ssl/etcdbr/tls.crt" + # server-key: "ssl/etcdbr/tls.crt" + +snapshotterConfig: + schedule: "0 */1 * * *" + deltaSnapshotPeriod: 20s + # deltaSnapshotMemoryLimit: 10000000 + # garbageCollectionPeriod: 1m + # garbageCollectionPolicy: "Exponential" + # maxBackups: 7 + +snapstoreConfig: + provider: "Local" + #container: "backup" + # prefix: "etcd-test" + maxParallelChunkUploads: 5 + tempDir: "/tmp" + +restorationConfig: + initialCluster: "default=http://localhost:2380" + initialClusterToken: "etcd-cluster" + restoreDataDir: "default.etcd" + initialAdvertisePeerURLs: + - "http://localhost:2380" + name: "default" + skipHashCheck: false + maxFetchers: 6 + embeddedEtcdQuotaBytes: 8589934592 + +defragmentationSchedule: "0 0 */3 * *" \ No newline at end of file diff --git a/go.mod b/go.mod index ee7df169c..a13162b14 100644 --- a/go.mod +++ b/go.mod @@ -41,7 +41,7 @@ require ( github.com/sirupsen/logrus v1.4.2 github.com/soheilhy/cmux v0.1.4 // indirect github.com/spf13/cobra v0.0.5 - github.com/spf13/pflag v1.0.5 // indirect + github.com/spf13/pflag v1.0.5 github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5 // indirect github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect go.etcd.io/bbolt v1.3.3 // indirect diff --git a/pkg/etcdutil/defrag.go b/pkg/defragmentor/defrag.go similarity index 77% rename from pkg/etcdutil/defrag.go rename to pkg/defragmentor/defrag.go index 8e7a374a9..b0f2fcb5e 100644 --- a/pkg/etcdutil/defrag.go +++ b/pkg/defragmentor/defrag.go @@ -12,38 +12,41 @@ // See the License for the specific language governing permissions and // limitations under the License. -package etcdutil +package defragmentor import ( "context" "time" + "github.com/gardener/etcd-backup-restore/pkg/etcdutil" "github.com/gardener/etcd-backup-restore/pkg/metrics" "github.com/prometheus/client_golang/prometheus" cron "github.com/robfig/cron/v3" "github.com/sirupsen/logrus" ) +const ( + etcdDialTimeout = time.Second * 30 +) + // CallbackFunc is type decalration for callback function for defragmentor type CallbackFunc func(ctx context.Context) error // defragmentorJob implement the cron.Job for etcd defragmentation. type defragmentorJob struct { - ctx context.Context - tlsConfig *TLSConfig - etcdConnectionTimeout time.Duration - logger *logrus.Entry - callback CallbackFunc + ctx context.Context + etcdConnectionConfig *etcdutil.EtcdConnectionConfig + logger *logrus.Entry + callback CallbackFunc } // NewDefragmentorJob returns the new defragmentor job. -func NewDefragmentorJob(ctx context.Context, tlsConfig *TLSConfig, etcdConnectionTimeout time.Duration, logger *logrus.Entry, callback CallbackFunc) cron.Job { +func NewDefragmentorJob(ctx context.Context, etcdConnectionConfig *etcdutil.EtcdConnectionConfig, logger *logrus.Entry, callback CallbackFunc) cron.Job { return &defragmentorJob{ - ctx: ctx, - tlsConfig: tlsConfig, - etcdConnectionTimeout: etcdConnectionTimeout, - logger: logger.WithField("job", "defragmentor"), - callback: callback, + ctx: ctx, + etcdConnectionConfig: etcdConnectionConfig, + logger: logger.WithField("job", "defragmentor"), + callback: callback, } } @@ -61,14 +64,14 @@ func (d *defragmentorJob) Run() { // defragData defragment the data directory of each etcd member. func (d *defragmentorJob) defragData() error { - client, err := GetTLSClientForEtcd(d.tlsConfig) + client, err := etcdutil.GetTLSClientForEtcd(d.etcdConnectionConfig) if err != nil { d.logger.Warnf("failed to create etcd client for defragmentation") return err } defer client.Close() - for _, ep := range d.tlsConfig.endpoints { + for _, ep := range d.etcdConnectionConfig.Endpoints { var dbSizeBeforeDefrag, dbSizeAfterDefrag int64 d.logger.Infof("Defragmenting etcd member[%s]", ep) statusReqCtx, cancel := context.WithTimeout(d.ctx, etcdDialTimeout) @@ -80,7 +83,7 @@ func (d *defragmentorJob) defragData() error { dbSizeBeforeDefrag = status.DbSize } start := time.Now() - defragCtx, cancel := context.WithTimeout(d.ctx, d.etcdConnectionTimeout) + defragCtx, cancel := context.WithTimeout(d.ctx, d.etcdConnectionConfig.ConnectionTimeout.Duration) _, err = client.Defragment(defragCtx, ep) cancel() if err != nil { @@ -106,8 +109,9 @@ func (d *defragmentorJob) defragData() error { } // DefragDataPeriodically defragments the data directory of each etcd member. -func DefragDataPeriodically(ctx context.Context, tlsConfig *TLSConfig, defragmentationSchedule cron.Schedule, etcdConnectionTimeout time.Duration, callback CallbackFunc, logger *logrus.Entry) { - defragmentorJob := NewDefragmentorJob(ctx, tlsConfig, etcdConnectionTimeout, logger, callback) +func DefragDataPeriodically(ctx context.Context, etcdConnectionConfig *etcdutil.EtcdConnectionConfig, defragmentationSchedule cron.Schedule, callback CallbackFunc, logger *logrus.Entry) { + defragmentorJob := NewDefragmentorJob(ctx, etcdConnectionConfig, logger, callback) + // TODO: Sync logrus logger to cron logger jobRunner := cron.New(cron.WithChain(cron.SkipIfStillRunning(cron.DefaultLogger))) jobRunner.Schedule(defragmentationSchedule, defragmentorJob) diff --git a/pkg/etcdutil/defrag_test.go b/pkg/defragmentor/defrag_test.go similarity index 74% rename from pkg/etcdutil/defrag_test.go rename to pkg/defragmentor/defrag_test.go index 8dffdfa01..83dfc59c6 100644 --- a/pkg/etcdutil/defrag_test.go +++ b/pkg/defragmentor/defrag_test.go @@ -12,14 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -package etcdutil_test +package defragmentor_test import ( "context" "fmt" "time" - . "github.com/gardener/etcd-backup-restore/pkg/etcdutil" + "github.com/gardener/etcd-backup-restore/pkg/etcdutil" + + . "github.com/gardener/etcd-backup-restore/pkg/defragmentor" . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" cron "github.com/robfig/cron/v3" @@ -27,37 +29,38 @@ import ( var _ = Describe("Defrag", func() { var ( - tlsConfig *TLSConfig - etcdConnectionTimeout = 30 * time.Second - keyPrefix = "/defrag/key-" - valuePrefix = "val" - etcdUsername string - etcdPassword string + etcdConnectionConfig *etcdutil.EtcdConnectionConfig + keyPrefix = "/defrag/key-" + valuePrefix = "val" ) + BeforeEach(func() { - tlsConfig = NewTLSConfig("", "", "", true, true, endpoints, etcdUsername, etcdPassword) + etcdConnectionConfig = etcdutil.NewEtcdConnectionConfig() + etcdConnectionConfig.Endpoints = endpoints + etcdConnectionConfig.ConnectionTimeout.Duration = 30 * time.Second }) + Context("Defragmentation", func() { BeforeEach(func() { now := time.Now().Unix() - client, err := GetTLSClientForEtcd(tlsConfig) + client, err := etcdutil.GetTLSClientForEtcd(etcdConnectionConfig) defer client.Close() - logger.Infof("TLSConfig %v, Endpoint %v", tlsConfig, endpoints) + logger.Infof("etcdConnectionConfig %v, Endpoint %v", etcdConnectionConfig, endpoints) Expect(err).ShouldNot(HaveOccurred()) for index := 0; index <= 1000; index++ { - ctx, cancel := context.WithTimeout(testCtx, etcdConnectionTimeout) + ctx, cancel := context.WithTimeout(testCtx, etcdConnectionConfig.ConnectionTimeout.Duration) client.Put(ctx, fmt.Sprintf("%s%d%d", keyPrefix, now, index), valuePrefix) cancel() } for index := 0; index <= 500; index++ { - ctx, cancel := context.WithTimeout(testCtx, etcdConnectionTimeout) + ctx, cancel := context.WithTimeout(testCtx, etcdConnectionConfig.ConnectionTimeout.Duration) client.Delete(ctx, fmt.Sprintf("%s%d%d", keyPrefix, now, index)) cancel() } }) It("should defragment and reduce size of DB within time", func() { - client, err := GetTLSClientForEtcd(tlsConfig) + client, err := etcdutil.GetTLSClientForEtcd(etcdConnectionConfig) Expect(err).ShouldNot(HaveOccurred()) defer client.Close() ctx, cancel := context.WithTimeout(testCtx, etcdDialTimeout) @@ -67,7 +70,7 @@ var _ = Describe("Defrag", func() { oldDBSize := oldStatus.DbSize oldRevision := oldStatus.Header.GetRevision() - defragmentorJob := NewDefragmentorJob(testCtx, tlsConfig, etcdConnectionTimeout, logger, nil) + defragmentorJob := NewDefragmentorJob(testCtx, etcdConnectionConfig, logger, nil) defragmentorJob.Run() ctx, cancel = context.WithTimeout(testCtx, etcdDialTimeout) @@ -80,8 +83,8 @@ var _ = Describe("Defrag", func() { }) It("should keep size of DB same in case of timeout", func() { - etcdConnectionTimeout = time.Second - client, err := GetTLSClientForEtcd(tlsConfig) + etcdConnectionConfig.ConnectionTimeout.Duration = time.Microsecond + client, err := etcdutil.GetTLSClientForEtcd(etcdConnectionConfig) Expect(err).ShouldNot(HaveOccurred()) defer client.Close() ctx, cancel := context.WithTimeout(testCtx, etcdDialTimeout) @@ -91,7 +94,7 @@ var _ = Describe("Defrag", func() { oldDBSize := oldStatus.DbSize oldRevision := oldStatus.Header.GetRevision() - defragmentorJob := NewDefragmentorJob(testCtx, tlsConfig, time.Microsecond, logger, nil) + defragmentorJob := NewDefragmentorJob(testCtx, etcdConnectionConfig, logger, nil) defragmentorJob.Run() ctx, cancel = context.WithTimeout(testCtx, etcdDialTimeout) @@ -108,7 +111,7 @@ var _ = Describe("Defrag", func() { minimumExpectedDefragCount := 2 defragSchedule, _ := cron.ParseStandard("*/1 * * * *") - client, err := GetTLSClientForEtcd(tlsConfig) + client, err := etcdutil.GetTLSClientForEtcd(etcdConnectionConfig) Expect(err).ShouldNot(HaveOccurred()) defer client.Close() statusReqCtx, cancelStatusReq := context.WithTimeout(testCtx, etcdDialTimeout) @@ -120,7 +123,7 @@ var _ = Describe("Defrag", func() { defragThreadCtx, cancelDefragThread := context.WithTimeout(testCtx, time.Second*time.Duration(135)) defer cancelDefragThread() - DefragDataPeriodically(defragThreadCtx, tlsConfig, defragSchedule, etcdConnectionTimeout, func(_ context.Context) error { + DefragDataPeriodically(defragThreadCtx, etcdConnectionConfig, defragSchedule, func(_ context.Context) error { defragCount++ return nil }, logger) diff --git a/pkg/etcdutil/etcdutil_suite_test.go b/pkg/defragmentor/defragmentor_suite_test.go similarity index 91% rename from pkg/etcdutil/etcdutil_suite_test.go rename to pkg/defragmentor/defragmentor_suite_test.go index 28fe6d8b2..d7c69bd14 100644 --- a/pkg/etcdutil/etcdutil_suite_test.go +++ b/pkg/defragmentor/defragmentor_suite_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package etcdutil_test +package defragmentor_test import ( "context" @@ -36,15 +36,15 @@ const ( var ( testCtx = context.Background() - logger = logrus.New().WithField("suite", "etcdutil") + logger = logrus.New().WithField("suite", "defragmentor") etcd *embed.Etcd endpoints []string err error ) -func TestEtcdutil(t *testing.T) { +func TestDefragmentor(t *testing.T) { RegisterFailHandler(Fail) - RunSpecs(t, "Etcdutil Suite") + RunSpecs(t, "Defragmentor Suite") } var _ = SynchronizedBeforeSuite(func() []byte { diff --git a/pkg/etcdutil/etcdutil.go b/pkg/etcdutil/etcdutil.go index e9bdf34cd..cb7cb877f 100644 --- a/pkg/etcdutil/etcdutil.go +++ b/pkg/etcdutil/etcdutil.go @@ -15,48 +15,36 @@ package etcdutil import ( + "context" "crypto/tls" "github.com/coreos/etcd/clientv3" "github.com/coreos/etcd/pkg/transport" ) -// NewTLSConfig returns the TLSConfig object. -func NewTLSConfig(cert, key, caCert string, insecureTr, skipVerify bool, endpoints []string, username, password string) *TLSConfig { - return &TLSConfig{ - cert: cert, - key: key, - caCert: caCert, - insecureTr: insecureTr, - skipVerify: skipVerify, - endpoints: endpoints, - username: username, - password: password, - } -} - // GetTLSClientForEtcd creates an etcd client using the TLS config params. -func GetTLSClientForEtcd(tlsConfig *TLSConfig) (*clientv3.Client, error) { +func GetTLSClientForEtcd(tlsConfig *EtcdConnectionConfig) (*clientv3.Client, error) { // set tls if any one tls option set var cfgtls *transport.TLSInfo tlsinfo := transport.TLSInfo{} - if tlsConfig.cert != "" { - tlsinfo.CertFile = tlsConfig.cert + if tlsConfig.CertFile != "" { + tlsinfo.CertFile = tlsConfig.CertFile cfgtls = &tlsinfo } - if tlsConfig.key != "" { - tlsinfo.KeyFile = tlsConfig.key + if tlsConfig.KeyFile != "" { + tlsinfo.KeyFile = tlsConfig.KeyFile cfgtls = &tlsinfo } - if tlsConfig.caCert != "" { - tlsinfo.CAFile = tlsConfig.caCert + if tlsConfig.CaFile != "" { + tlsinfo.CAFile = tlsConfig.CaFile cfgtls = &tlsinfo } cfg := &clientv3.Config{ - Endpoints: tlsConfig.endpoints, + Endpoints: tlsConfig.Endpoints, + Context: context.TODO(), // TODO: Use the context comming as parameter. } if cfgtls != nil { @@ -70,19 +58,19 @@ func GetTLSClientForEtcd(tlsConfig *TLSConfig) (*clientv3.Client, error) { // if key/cert is not given but user wants secure connection, we // should still setup an empty tls configuration for gRPC to setup // secure connection. - if cfg.TLS == nil && !tlsConfig.insecureTr { + if cfg.TLS == nil && !tlsConfig.InsecureTransport { cfg.TLS = &tls.Config{} } // If the user wants to skip TLS verification then we should set // the InsecureSkipVerify flag in tls configuration. - if tlsConfig.skipVerify && cfg.TLS != nil { + if tlsConfig.InsecureSkipVerify && cfg.TLS != nil { cfg.TLS.InsecureSkipVerify = true } - if tlsConfig.username != "" && tlsConfig.password != "" { - cfg.Username = tlsConfig.username - cfg.Password = tlsConfig.password + if tlsConfig.Username != "" && tlsConfig.Password != "" { + cfg.Username = tlsConfig.Username + cfg.Password = tlsConfig.Password } return clientv3.New(*cfg) diff --git a/pkg/etcdutil/init.go b/pkg/etcdutil/init.go new file mode 100644 index 000000000..abd1fa649 --- /dev/null +++ b/pkg/etcdutil/init.go @@ -0,0 +1,54 @@ +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 etcdutil + +import ( + "fmt" + "time" + + "github.com/gardener/etcd-backup-restore/pkg/wrappers" + flag "github.com/spf13/pflag" +) + +// NewEtcdConnectionConfig returns etcd connection config. +func NewEtcdConnectionConfig() *EtcdConnectionConfig { + return &EtcdConnectionConfig{ + Endpoints: []string{"127.0.0.1:2379"}, + ConnectionTimeout: wrappers.Duration{Duration: 30 * time.Second}, + InsecureTransport: true, + InsecureSkipVerify: false, + } +} + +// AddFlags adds the flags to flagset. +func (c *EtcdConnectionConfig) AddFlags(fs *flag.FlagSet) { + fs.StringSliceVarP(&c.Endpoints, "endpoints", "e", c.Endpoints, "comma separated list of etcd endpoints") + fs.StringVar(&c.Username, "etcd-username", c.Username, "etcd server username, if one is required") + fs.StringVar(&c.Password, "etcd-password", c.Password, "etcd server password, if one is required") + fs.DurationVar(&c.ConnectionTimeout.Duration, "etcd-connection-timeout", c.ConnectionTimeout.Duration, "etcd client connection timeout") + fs.BoolVar(&c.InsecureTransport, "insecure-transport", c.InsecureTransport, "disable transport security for client connections") + fs.BoolVar(&c.InsecureSkipVerify, "insecure-skip-tls-verify", c.InsecureTransport, "skip server certificate verification") + fs.StringVar(&c.CertFile, "cert", c.CertFile, "identify secure client using this TLS certificate file") + fs.StringVar(&c.KeyFile, "key", c.KeyFile, "identify secure client using this TLS key file") + fs.StringVar(&c.CaFile, "cacert", c.CaFile, "verify certificates of TLS-enabled secure servers using this CA bundle") +} + +// Validate validates the config. +func (c *EtcdConnectionConfig) Validate() error { + if c.ConnectionTimeout.Duration <= 0 { + return fmt.Errorf("connection timeout should be greater than zero") + } + return nil +} diff --git a/pkg/etcdutil/types.go b/pkg/etcdutil/types.go index c663aeef4..246782c6a 100644 --- a/pkg/etcdutil/types.go +++ b/pkg/etcdutil/types.go @@ -1,4 +1,4 @@ -// Copyright (c) 2018 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,19 +15,20 @@ package etcdutil import ( - "time" + "github.com/gardener/etcd-backup-restore/pkg/wrappers" ) -const etcdDialTimeout = time.Second * 30 - -// TLSConfig holds cert information and settings for TLS. -type TLSConfig struct { - cert string - key string - caCert string - insecureTr bool - skipVerify bool - endpoints []string - username string - password string +// EtcdConnectionConfig holds the etcd connection config. +type EtcdConnectionConfig struct { + // Endpoints are the endpoints from which the backup will be take or defragmentation will be called. + // This need not be necessary match the entire etcd cluster. + Endpoints []string `json:"endpoints"` + Username string `json:"username,omitempty"` + Password string `json:"password,omitempty"` + ConnectionTimeout wrappers.Duration `json:"connectionTimeout,omitempty"` + InsecureTransport bool `json:"insecureTransport,omitempty"` + InsecureSkipVerify bool `json:"insecureSkipVerify,omitempty"` + CertFile string `json:"certFile,omitempty"` + KeyFile string `json:"keyFile,omitempty"` + CaFile string `json:"caFile,omitempty"` } diff --git a/pkg/initializer/initializer.go b/pkg/initializer/initializer.go index dd923fa4f..2b31033b4 100644 --- a/pkg/initializer/initializer.go +++ b/pkg/initializer/initializer.go @@ -29,10 +29,6 @@ import ( "github.com/sirupsen/logrus" ) -const ( - backupFormatVersion = "v1" -) - // Initialize has the following steps: // * Check if data directory exists. // - If data directory exists @@ -80,7 +76,7 @@ func NewInitializer(options *restorer.RestoreOptions, snapstoreConfig *snapstore }, Validator: &validator.DataValidator{ Config: &validator.Config{ - DataDir: options.RestoreDataDir, + DataDir: options.Config.RestoreDataDir, SnapstoreConfig: snapstoreConfig, }, Logger: logger, @@ -96,7 +92,7 @@ func NewInitializer(options *restorer.RestoreOptions, snapstoreConfig *snapstore // bootstrapping a new data directory or if restoration failed func (e *EtcdInitializer) restoreCorruptData() (bool, error) { logger := e.Logger - dataDir := e.Config.RestoreOptions.RestoreDataDir + dataDir := e.Config.RestoreOptions.Config.RestoreDataDir if e.Config.SnapstoreConfig == nil { logger.Warnf("No snapstore storage provider configured.") @@ -123,9 +119,9 @@ func (e *EtcdInitializer) restoreCorruptData() (bool, error) { e.Config.RestoreOptions.BaseSnapshot = *baseSnap e.Config.RestoreOptions.DeltaSnapList = deltaSnapList tempRestoreOptions := *e.Config.RestoreOptions - tempRestoreOptions.RestoreDataDir = fmt.Sprintf("%s.%s", tempRestoreOptions.RestoreDataDir, "part") + tempRestoreOptions.Config.RestoreDataDir = fmt.Sprintf("%s.%s", tempRestoreOptions.Config.RestoreDataDir, "part") - if err := e.removeDir(tempRestoreOptions.RestoreDataDir); err != nil { + if err := e.removeDir(tempRestoreOptions.Config.RestoreDataDir); err != nil { return false, fmt.Errorf("failed to delete previous temporary data directory: %v", err) } @@ -148,7 +144,7 @@ func (e *EtcdInitializer) restoreCorruptData() (bool, error) { // and false if directory removal failed or if directory // never existed (bootstrap case) func (e *EtcdInitializer) restoreWithEmptySnapstore() (bool, error) { - dataDir := e.Config.RestoreOptions.RestoreDataDir + dataDir := e.Config.RestoreOptions.Config.RestoreDataDir e.Logger.Infof("Removing directory(%s) since snapstore is empty.", dataDir) // If data directory doesn't exist, it means we are bootstrapping diff --git a/pkg/initializer/validator/datavalidator_test.go b/pkg/initializer/validator/datavalidator_test.go index 1f3fcea98..c011d5e69 100644 --- a/pkg/initializer/validator/datavalidator_test.go +++ b/pkg/initializer/validator/datavalidator_test.go @@ -1,3 +1,16 @@ +// Copyright (c) 2018 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 validator_test import ( diff --git a/pkg/initializer/validator/validator_suite_test.go b/pkg/initializer/validator/validator_suite_test.go index 521440bc0..19ed6d585 100644 --- a/pkg/initializer/validator/validator_suite_test.go +++ b/pkg/initializer/validator/validator_suite_test.go @@ -87,60 +87,26 @@ var _ = SynchronizedBeforeSuite(func() []byte { // runSnapshotter creates a snapshotter object and runs it for a duration specified by 'snapshotterDurationSeconds' func runSnapshotter(logger *logrus.Entry, deltaSnapshotPeriod time.Duration, endpoints []string, stopCh <-chan struct{}) error { - var ( - store snapstore.SnapStore - certFile string - keyFile string - caFile string - insecureTransport = true - insecureSkipVerify = true - maxBackups = 1 - deltaSnapshotMemoryLimit = 10 * 1024 * 1024 //10Mib - etcdConnectionTimeout = 10 * time.Second - garbageCollectionPeriod = 60 * time.Second - schedule = "0 0 1 1 *" - garbageCollectionPolicy = snapshotter.GarbageCollectionPolicyLimitBased - etcdUsername string - etcdPassword string - ) - - store, err = snapstore.GetSnapstore(&snapstore.Config{Container: snapstoreDir, Provider: "Local"}) + store, err := snapstore.GetSnapstore(&snapstore.Config{Container: snapstoreDir, Provider: "Local"}) if err != nil { return err } - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword, - ) + etcdConnectionConfig := etcdutil.NewEtcdConnectionConfig() + etcdConnectionConfig.Endpoints = endpoints + etcdConnectionConfig.ConnectionTimeout.Duration = 10 * time.Second + logger.Infof("etcdConnectionConfig %v", etcdConnectionConfig) - logger.Infof("tlsconfig %v", tlsConfig) - snapshotterConfig, err := snapshotter.NewSnapshotterConfig( - schedule, - store, - maxBackups, - deltaSnapshotMemoryLimit, - deltaSnapshotPeriod, - etcdConnectionTimeout, - garbageCollectionPeriod, - garbageCollectionPolicy, - tlsConfig, - ) + snapshotterConfig := snapshotter.NewSnapshotterConfig() + snapshotterConfig.GarbageCollectionPolicy = snapshotter.GarbageCollectionPolicyLimitBased + snapshotterConfig.FullSnapshotSchedule = "0 0 1 1 *" + snapshotterConfig.MaxBackups = 1 + + ssr, err := snapshotter.NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) if err != nil { return err } - ssr := snapshotter.NewSnapshotter( - logger, - snapshotterConfig, - ) - return ssr.Run(stopCh, true) } diff --git a/pkg/server/backuprestoreserver.go b/pkg/server/backuprestoreserver.go new file mode 100644 index 000000000..91b8cbbd0 --- /dev/null +++ b/pkg/server/backuprestoreserver.go @@ -0,0 +1,332 @@ +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 server + +import ( + "context" + "fmt" + "net/http" + "sync/atomic" + "time" + + "github.com/gardener/etcd-backup-restore/pkg/errors" + "github.com/gardener/etcd-backup-restore/pkg/metrics" + "github.com/prometheus/client_golang/prometheus" + + "github.com/coreos/etcd/pkg/types" + "github.com/gardener/etcd-backup-restore/pkg/defragmentor" + "github.com/gardener/etcd-backup-restore/pkg/etcdutil" + "github.com/gardener/etcd-backup-restore/pkg/initializer" + "github.com/gardener/etcd-backup-restore/pkg/snapshot/restorer" + "github.com/gardener/etcd-backup-restore/pkg/snapshot/snapshotter" + "github.com/gardener/etcd-backup-restore/pkg/snapstore" + cron "github.com/robfig/cron/v3" + "github.com/sirupsen/logrus" +) + +// BackupRestoreServer holds the details for backup-restore server. +type BackupRestoreServer struct { + config *BackupRestoreComponentConfig + logger *logrus.Entry + defragmentationSchedule cron.Schedule +} + +// NewBackupRestoreServer return new backup restore server. +func NewBackupRestoreServer(logger *logrus.Logger, config *BackupRestoreComponentConfig) (*BackupRestoreServer, error) { + parsedDefragSchedule, err := cron.ParseStandard(config.DefragmentationSchedule) + if err != nil { + // Ideally this case should not occur, since this check is done at the config validaitions. + return nil, err + } + return &BackupRestoreServer{ + logger: logger.WithField("actor", "backup-restore-server"), + config: config, + defragmentationSchedule: parsedDefragSchedule, + }, nil +} + +// Run starts the backup restore server. +func (b *BackupRestoreServer) Run(ctx context.Context) error { + clusterURLsMap, err := types.NewURLsMap(b.config.RestorationConfig.InitialCluster) + if err != nil { + // Ideally this case should not occur, since this check is done at the config validaitions. + b.logger.Fatalf("failed creating url map for restore cluster: %v", err) + } + + peerURLs, err := types.NewURLs(b.config.RestorationConfig.InitialAdvertisePeerURLs) + if err != nil { + // Ideally this case should not occur, since this check is done at the config validaitions. + b.logger.Fatalf("failed creating url map for restore cluster: %v", err) + } + + options := &restorer.RestoreOptions{ + Config: b.config.RestorationConfig, + ClusterURLs: clusterURLsMap, + PeerURLs: peerURLs, + } + + if b.config.SnapstoreConfig == nil { + b.logger.Warnf("No snapstore storage provider configured. Will not start backup schedule.") + b.runServerWithoutSnapshotter(ctx, options) + return nil + } + return b.runServerWithSnapshotter(ctx, options) +} + +// startHTTPServer creates and starts the HTTP handler +// with status 503 (Service Unavailable) +func (b *BackupRestoreServer) startHTTPServer(initializer initializer.Initializer, ssr *snapshotter.Snapshotter) *HTTPHandler { + // Start http handler with Error state and wait till snapshotter is up + // and running before setting the status to OK. + handler := &HTTPHandler{ + Port: b.config.ServerConfig.Port, + Initializer: initializer, + Snapshotter: ssr, + Logger: b.logger, + StopCh: make(chan struct{}), + EnableProfiling: b.config.ServerConfig.EnableProfiling, + ReqCh: make(chan struct{}), + AckCh: make(chan struct{}), + EnableTLS: (b.config.ServerConfig.TLSCertFile != "" && b.config.ServerConfig.TLSKeyFile != ""), + ServerTLSCertFile: b.config.ServerConfig.TLSCertFile, + ServerTLSKeyFile: b.config.ServerConfig.TLSKeyFile, + } + handler.SetStatus(http.StatusServiceUnavailable) + b.logger.Info("Registering the http request handlers...") + handler.RegisterHandler() + b.logger.Info("Starting the http server...") + go handler.Start() + + return handler +} + +// runServerWithoutSnapshotter runs the etcd-backup-restore +// for the case where snapshotter is not configured +func (b *BackupRestoreServer) runServerWithoutSnapshotter(ctx context.Context, restoreOpts *restorer.RestoreOptions) { + etcdInitializer := initializer.NewInitializer(restoreOpts, nil, b.logger.Logger) + + // If no storage provider is given, snapshotter will be nil, in which + // case the status is set to OK as soon as etcd probe is successful + handler := b.startHTTPServer(etcdInitializer, nil) + defer handler.Stop() + + // start defragmentation without trigerring full snapshot + // after each successful data defragmentation + go defragmentor.DefragDataPeriodically(ctx, b.config.EtcdConnectionConfig, b.defragmentationSchedule, nil, b.logger) + + b.runEtcdProbeLoopWithoutSnapshotter(ctx, handler) +} + +// runServerWithoutSnapshotter runs the etcd-backup-restore +// for the case where snapshotter is configured correctly +func (b *BackupRestoreServer) runServerWithSnapshotter(ctx context.Context, restoreOpts *restorer.RestoreOptions) error { + ackCh := make(chan struct{}) + + etcdInitializer := initializer.NewInitializer(restoreOpts, b.config.SnapstoreConfig, b.logger.Logger) + + b.logger.Infof("Creating snapstore from provider: %s", b.config.SnapstoreConfig.Provider) + ss, err := snapstore.GetSnapstore(b.config.SnapstoreConfig) + if err != nil { + return fmt.Errorf("failed to create snapstore from configured storage provider: %v", err) + } + + b.logger.Infof("Creating snapshotter...") + ssr, err := snapshotter.NewSnapshotter(b.logger, b.config.SnapshotterConfig, ss, b.config.EtcdConnectionConfig) + if err != nil { + return err + } + + handler := b.startHTTPServer(etcdInitializer, ssr) + defer handler.Stop() + + ssrStopCh := make(chan struct{}) + go handleSsrStopRequest(ctx, handler, ssr, ackCh, ssrStopCh) + go handleAckState(handler, ackCh) + + go defragmentor.DefragDataPeriodically(ctx, b.config.EtcdConnectionConfig, b.defragmentationSchedule, ssr.TriggerFullSnapshot, b.logger) + + b.runEtcdProbeLoopWithSnapshotter(ctx, handler, ssr, ssrStopCh, ackCh) + return nil +} + +// runEtcdProbeLoopWithoutSnapshotter runs the etcd probe loop +// for the case where snapshotter is configured correctly +func (b *BackupRestoreServer) runEtcdProbeLoopWithSnapshotter(ctx context.Context, handler *HTTPHandler, ssr *snapshotter.Snapshotter, ssrStopCh chan struct{}, ackCh chan struct{}) { + var ( + err error + initialDeltaSnapshotTaken bool + ) + + for { + b.logger.Infof("Probing etcd...") + select { + case <-ctx.Done(): + b.logger.Info("Shutting down...") + return + default: + err = b.probeEtcd(ctx) + } + if err != nil { + b.logger.Errorf("Failed to probe etcd: %v", err) + handler.SetStatus(http.StatusServiceUnavailable) + continue + } + + // The decision to either take an initial delta snapshot or + // or a full snapshot directly is based on whether there has + // been a previous full snapshot (if not, we assume the etcd + // to be a fresh etcd) or it has been more than 24 hours since + // the last full snapshot was taken. + // If this is not the case, we take a delta snapshot by first + // collecting all the delta events since the previous snapshot + // and take a delta snapshot of these (there may be multiple + // delta snapshots based on the amount of events collected and + // the delta snapshot memory limit), after which a full snapshot + // is taken and the regular snapshot schedule comes into effect. + + // TODO: write code to find out if prev full snapshot is older than it is + // supposed to be, according to the given cron schedule, instead of the + // hard-coded "24 hours" full snapshot interval + if ssr.PrevFullSnapshot != nil && time.Since(ssr.PrevFullSnapshot.CreatedOn).Hours() <= 24 { + ssrStopped, err := ssr.CollectEventsSincePrevSnapshot(ssrStopCh) + if ssrStopped { + b.logger.Info("Snapshotter stopped.") + ackCh <- emptyStruct + handler.SetStatus(http.StatusServiceUnavailable) + b.logger.Info("Shutting down...") + return + } + if err == nil { + if err := ssr.TakeDeltaSnapshot(); err != nil { + b.logger.Warnf("Failed to take first delta snapshot: snapshotter failed with error: %v", err) + continue + } + initialDeltaSnapshotTaken = true + } else { + b.logger.Warnf("Failed to collect events for first delta snapshot(s): %v", err) + } + } + if !initialDeltaSnapshotTaken { + // need to take a full snapshot here + metrics.SnapshotRequired.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindDelta}).Set(0) + metrics.SnapshotRequired.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull}).Set(1) + if err := ssr.TakeFullSnapshotAndResetTimer(); err != nil { + b.logger.Errorf("Failed to take substitute first full snapshot: %v", err) + continue + } + } + + // set server's healthz endpoint status to OK so that + // etcd is marked as ready to serve traffic + handler.SetStatus(http.StatusOK) + + ssr.SsrStateMutex.Lock() + ssr.SsrState = snapshotter.SnapshotterActive + ssr.SsrStateMutex.Unlock() + gcStopCh := make(chan struct{}) + go ssr.RunGarbageCollector(gcStopCh) + b.logger.Infof("Starting snapshotter...") + if err := ssr.Run(ssrStopCh, initialDeltaSnapshotTaken); err != nil { + if etcdErr, ok := err.(*errors.EtcdError); ok == true { + b.logger.Errorf("Snapshotter failed with etcd error: %v", etcdErr) + } else { + b.logger.Fatalf("Snapshotter failed with error: %v", err) + } + } + b.logger.Infof("Snapshotter stopped.") + ackCh <- emptyStruct + handler.SetStatus(http.StatusServiceUnavailable) + close(gcStopCh) + } +} + +// runEtcdProbeLoopWithoutSnapshotter runs the etcd probe loop +// for the case where snapshotter is not configured +func (b *BackupRestoreServer) runEtcdProbeLoopWithoutSnapshotter(ctx context.Context, handler *HTTPHandler) { + var err error + for { + b.logger.Infof("Probing etcd...") + select { + case <-ctx.Done(): + b.logger.Info("Shutting down...") + return + default: + err = b.probeEtcd(ctx) + } + if err != nil { + b.logger.Errorf("Failed to probe etcd: %v", err) + handler.SetStatus(http.StatusServiceUnavailable) + continue + } + + handler.SetStatus(http.StatusOK) + <-ctx.Done() + handler.SetStatus(http.StatusServiceUnavailable) + b.logger.Infof("Received stop signal. Terminating !!") + return + } +} + +// probeEtcd will make the snapshotter probe for etcd endpoint to be available +// before it starts taking regular snapshots. +func (b *BackupRestoreServer) probeEtcd(ctx context.Context) error { + client, err := etcdutil.GetTLSClientForEtcd(b.config.EtcdConnectionConfig) + if err != nil { + return &errors.EtcdError{ + Message: fmt.Sprintf("failed to create etcd client: %v", err), + } + } + + ctx, cancel := context.WithTimeout(ctx, b.config.EtcdConnectionConfig.ConnectionTimeout.Duration) + defer cancel() + if _, err := client.Get(ctx, "foo"); err != nil { + b.logger.Errorf("Failed to connect to client: %v", err) + return err + } + return nil +} + +func handleAckState(handler *HTTPHandler, ackCh chan struct{}) { + for { + <-ackCh + if atomic.CompareAndSwapUint32(&handler.AckState, HandlerAckWaiting, HandlerAckDone) { + handler.AckCh <- emptyStruct + } + } +} + +// handleSsrStopRequest responds to handlers request and stop interrupt. +func handleSsrStopRequest(ctx context.Context, handler *HTTPHandler, ssr *snapshotter.Snapshotter, ackCh, ssrStopCh chan struct{}) { + for { + var ok bool + select { + case _, ok = <-handler.ReqCh: + case _, ok = <-ctx.Done(): + } + + ssr.SsrStateMutex.Lock() + if ssr.SsrState == snapshotter.SnapshotterActive { + ssr.SsrStateMutex.Unlock() + ssrStopCh <- emptyStruct + } else { + ssr.SsrState = snapshotter.SnapshotterInactive + ssr.SsrStateMutex.Unlock() + ackCh <- emptyStruct + } + if !ok { + return + } + } +} diff --git a/pkg/server/httpAPI.go b/pkg/server/httpAPI.go index 85ea0433f..a0b2c93ca 100644 --- a/pkg/server/httpAPI.go +++ b/pkg/server/httpAPI.go @@ -60,9 +60,9 @@ const ( type HTTPHandler struct { Initializer initializer.Initializer Snapshotter *snapshotter.Snapshotter - Port int + Port uint server *http.Server - Logger *logrus.Logger + Logger *logrus.Entry initializationStatusMutex sync.Mutex AckState uint32 initializationStatus string diff --git a/pkg/server/init.go b/pkg/server/init.go new file mode 100644 index 000000000..6f210785c --- /dev/null +++ b/pkg/server/init.go @@ -0,0 +1,119 @@ +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 server + +import ( + "fmt" + "os" + + "github.com/gardener/etcd-backup-restore/pkg/etcdutil" + + "github.com/gardener/etcd-backup-restore/pkg/snapshot/restorer" + "github.com/gardener/etcd-backup-restore/pkg/snapshot/snapshotter" + "github.com/gardener/etcd-backup-restore/pkg/snapstore" + "github.com/robfig/cron/v3" + flag "github.com/spf13/pflag" +) + +// NewBackupRestoreComponentConfig returns the backup-restore componenet config. +func NewBackupRestoreComponentConfig() *BackupRestoreComponentConfig { + return &BackupRestoreComponentConfig{ + EtcdConnectionConfig: etcdutil.NewEtcdConnectionConfig(), + ServerConfig: NewHTTPServerConfig(), + SnapshotterConfig: snapshotter.NewSnapshotterConfig(), + SnapstoreConfig: snapstore.NewSnapstoreConfig(), + RestorationConfig: restorer.NewRestorationConfig(), + DefragmentationSchedule: "0 0 */3 * *", + } +} + +// AddFlags adds the flags to flagset. +func (c *BackupRestoreComponentConfig) AddFlags(fs *flag.FlagSet) { + c.EtcdConnectionConfig.AddFlags(fs) + c.ServerConfig.AddFlags(fs) + c.SnapshotterConfig.AddFlags(fs) + c.SnapstoreConfig.AddFlags(fs) + c.RestorationConfig.AddFlags(fs) + + // Miscellaneous + fs.StringVar(&c.DefragmentationSchedule, "defragmentation-schedule", c.DefragmentationSchedule, "schedule to defragment etcd data directory") +} + +// Validate validates the config. +func (c *BackupRestoreComponentConfig) Validate() error { + if err := c.EtcdConnectionConfig.Validate(); err != nil { + return err + } + if err := c.ServerConfig.Validate(); err != nil { + return err + } + if err := c.SnapshotterConfig.Validate(); err != nil { + return err + } + if err := c.SnapstoreConfig.Validate(); err != nil { + return err + } + if err := c.RestorationConfig.Validate(); err != nil { + return err + } + if _, err := cron.ParseStandard(c.DefragmentationSchedule); err != nil { + return err + } + return nil +} + +// Complete completes the config. +func (c *BackupRestoreComponentConfig) Complete() { + c.SnapstoreConfig.Complete() +} + +// HTTPServerConfig holds the server config. +type HTTPServerConfig struct { + Port uint `json:"port,omitempty"` + EnableProfiling bool `json:"enableProfiling,omitempty"` + TLSCertFile string `json:"server-cert,omitempty"` + TLSKeyFile string `json:"server-key,omitempty"` +} + +// NewHTTPServerConfig returns the config for http server +func NewHTTPServerConfig() *HTTPServerConfig { + return &HTTPServerConfig{ + Port: defaultServerPort, + EnableProfiling: false, + } +} + +// AddFlags adds the flags to flagset. +func (c *HTTPServerConfig) AddFlags(fs *flag.FlagSet) { + fs.UintVarP(&c.Port, "server-port", "p", c.Port, "port on which server should listen") + fs.BoolVar(&c.EnableProfiling, "enable-profiling", c.EnableProfiling, "enable profiling") + fs.StringVar(&c.TLSCertFile, "server-cert", "", "TLS certificate file for backup-restore server") + fs.StringVar(&c.TLSKeyFile, "server-key", "", "TLS key file for backup-restore server") +} + +// Validate validates the config.E +func (c *HTTPServerConfig) Validate() error { + enableTLS := c.TLSCertFile != "" && c.TLSKeyFile != "" + if enableTLS { + // Check for existence of server cert and key files before proceeding + if _, err := os.Stat(c.TLSCertFile); err != nil { + return fmt.Errorf("TLS enabled but server TLS cert file is invalid. Will not start HTTPS server: %v", err) + } + if _, err := os.Stat(c.TLSKeyFile); err != nil { + return fmt.Errorf("TLS enabled but server TLS key file is invalid. Will not start HTTPS server: %v", err) + } + } + return nil +} diff --git a/pkg/server/types.go b/pkg/server/types.go new file mode 100644 index 000000000..77dd39925 --- /dev/null +++ b/pkg/server/types.go @@ -0,0 +1,36 @@ +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 server + +import ( + "github.com/gardener/etcd-backup-restore/pkg/etcdutil" + "github.com/gardener/etcd-backup-restore/pkg/snapshot/restorer" + "github.com/gardener/etcd-backup-restore/pkg/snapshot/snapshotter" + "github.com/gardener/etcd-backup-restore/pkg/snapstore" +) + +const ( + defaultServerPort = 8080 +) + +// BackupRestoreComponentConfig holds the component configuration. +type BackupRestoreComponentConfig struct { + EtcdConnectionConfig *etcdutil.EtcdConnectionConfig `json:"etcdConnectionConfig,omitempty"` + ServerConfig *HTTPServerConfig `json:"serverConfig,omitempty"` + SnapshotterConfig *snapshotter.Config `json:"snapshotterConfig,omitempty"` + SnapstoreConfig *snapstore.Config `json:"snapstoreConfig,omitempty"` + RestorationConfig *restorer.RestorationConfig `json:"restorationConfig,omitempty"` + DefragmentationSchedule string `json:"defragmentationSchedule"` +} diff --git a/pkg/snapshot/restorer/init.go b/pkg/snapshot/restorer/init.go new file mode 100644 index 000000000..62db38802 --- /dev/null +++ b/pkg/snapshot/restorer/init.go @@ -0,0 +1,80 @@ +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 restorer + +import ( + "fmt" + "path" + + "github.com/coreos/etcd/pkg/types" + flag "github.com/spf13/pflag" +) + +const ( + defaultName = "default" + defaultInitialAdvertisePeerURLs = "http://localhost:2380" +) + +// NewRestorationConfig returns the restoration config. +func NewRestorationConfig() *RestorationConfig { + return &RestorationConfig{ + InitialCluster: initialClusterFromName(defaultName), + InitialClusterToken: "etcd-cluster", + RestoreDataDir: fmt.Sprintf("%s.etcd", defaultName), + InitialAdvertisePeerURLs: []string{defaultInitialAdvertisePeerURLs}, + Name: defaultName, + SkipHashCheck: false, + MaxFetchers: 6, + EmbeddedEtcdQuotaBytes: int64(8 * 1024 * 1024 * 1024), + } +} + +// AddFlags adds the flags to flagset. +func (c *RestorationConfig) AddFlags(fs *flag.FlagSet) { + fs.StringVar(&c.InitialCluster, "initial-cluster", c.InitialCluster, "initial cluster configuration for restore bootstrap") + fs.StringVar(&c.InitialClusterToken, "initial-cluster-token", c.InitialClusterToken, "initial cluster token for the etcd cluster during restore bootstrap") + fs.StringVarP(&c.RestoreDataDir, "data-dir", "d", c.RestoreDataDir, "path to the data directory") + fs.StringArrayVar(&c.InitialAdvertisePeerURLs, "initial-advertise-peer-urls", c.InitialAdvertisePeerURLs, "list of this member's peer URLs to advertise to the rest of the cluster") + fs.StringVar(&c.Name, "name", c.Name, "human-readable name for this member") + fs.BoolVar(&c.SkipHashCheck, "skip-hash-check", c.SkipHashCheck, "ignore snapshot integrity hash value (required if copied from data directory)") + fs.UintVar(&c.MaxFetchers, "max-fetchers", c.MaxFetchers, "maximum number of threads that will fetch delta snapshots in parallel") + fs.Int64Var(&c.EmbeddedEtcdQuotaBytes, "embedded-etcd-quota-bytes", c.EmbeddedEtcdQuotaBytes, "maximum backend quota for the embedded etcd used for applying delta snapshots") +} + +// Validate validates the config. +func (c *RestorationConfig) Validate() error { + if _, err := types.NewURLsMap(c.InitialCluster); err != nil { + return fmt.Errorf("failed creating url map for restore cluster: %v", err) + } + if _, err := types.NewURLs(c.InitialAdvertisePeerURLs); err != nil { + return fmt.Errorf("failed parsing peers urls for restore cluster: %v", err) + } + if c.MaxFetchers <= 0 { + return fmt.Errorf("max fetchers should be greater than zero") + } + if c.EmbeddedEtcdQuotaBytes <= 0 { + return fmt.Errorf("Etcd Quota size for etcd must be greater than 0") + } + c.RestoreDataDir = path.Clean(c.RestoreDataDir) + return nil +} + +func initialClusterFromName(name string) string { + n := name + if name == "" { + n = defaultName + } + return fmt.Sprintf("%s=http://localhost:2380", n) +} diff --git a/pkg/snapshot/restorer/restorer.go b/pkg/snapshot/restorer/restorer.go index 61e1caa3d..ee7fc9932 100644 --- a/pkg/snapshot/restorer/restorer.go +++ b/pkg/snapshot/restorer/restorer.go @@ -61,14 +61,6 @@ func NewRestorer(store snapstore.SnapStore, logger *logrus.Entry) *Restorer { // Restore restore the etcd data directory as per specified restore options. func (r *Restorer) Restore(ro RestoreOptions) error { - if ro.MaxFetchers < 1 { - return fmt.Errorf("Maximum number of fetchers should be greater than zero. Input MaxFetchers: %d", ro.MaxFetchers) - } - if ro.EmbeddedEtcdQuotaBytes <= 0 { - r.logger.Infof("Quota size for etcd must be greater than 0. Input EmbeddedEtcdQuotaBytes: %d. Defaulting to 8GB.", ro.EmbeddedEtcdQuotaBytes) - ro.EmbeddedEtcdQuotaBytes = int64(8 * 1024 * 1024 * 1024) - } - ro.RestoreDataDir = path.Clean(ro.RestoreDataDir) if err := r.restoreFromBaseSnapshot(ro); err != nil { return fmt.Errorf("failed to restore from the base snapshot :%v", err) } @@ -105,21 +97,21 @@ func (r *Restorer) restoreFromBaseSnapshot(ro RestoreOptions) error { } r.logger.Infof("Restoring from base snapshot: %s", path.Join(ro.BaseSnapshot.SnapDir, ro.BaseSnapshot.SnapName)) cfg := etcdserver.ServerConfig{ - InitialClusterToken: ro.ClusterToken, + InitialClusterToken: ro.Config.InitialClusterToken, InitialPeerURLsMap: ro.ClusterURLs, PeerURLs: ro.PeerURLs, - Name: ro.Name, + Name: ro.Config.Name, } if err := cfg.VerifyBootstrap(); err != nil { return err } - cl, err := membership.NewClusterFromURLsMap(ro.ClusterToken, ro.ClusterURLs) + cl, err := membership.NewClusterFromURLsMap(ro.Config.InitialClusterToken, ro.ClusterURLs) if err != nil { return err } - memberDir := filepath.Join(ro.RestoreDataDir, "member") + memberDir := filepath.Join(ro.Config.RestoreDataDir, "member") if _, err := os.Stat(memberDir); err == nil { return fmt.Errorf("member directory in data directory(%q) exists", memberDir) } @@ -129,7 +121,7 @@ func (r *Restorer) restoreFromBaseSnapshot(ro RestoreOptions) error { if err = makeDB(snapdir, ro.BaseSnapshot, len(cl.Members()), r.store, false); err != nil { return err } - return makeWALAndSnap(walDir, snapdir, cl, ro.Name) + return makeWALAndSnap(walDir, snapdir, cl, ro.Config.Name) } // makeDB copies the database snapshot to the snapshot directory. @@ -313,7 +305,7 @@ func makeWALAndSnap(waldir, snapdir string, cl *membership.RaftCluster, restoreN // startEmbeddedEtcd starts the embedded etcd server. func startEmbeddedEtcd(logger *logrus.Entry, ro RestoreOptions) (*embed.Etcd, error) { cfg := embed.NewConfig() - cfg.Dir = filepath.Join(ro.RestoreDataDir) + cfg.Dir = filepath.Join(ro.Config.RestoreDataDir) DefaultListenPeerURLs := "http://localhost:0" DefaultListenClientURLs := "http://localhost:0" DefaultInitialAdvertisePeerURLs := "http://localhost:0" @@ -327,7 +319,7 @@ func startEmbeddedEtcd(logger *logrus.Entry, ro RestoreOptions) (*embed.Etcd, er cfg.APUrls = []url.URL{*apurl} cfg.ACUrls = []url.URL{*acurl} cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name) - cfg.QuotaBackendBytes = ro.EmbeddedEtcdQuotaBytes + cfg.QuotaBackendBytes = ro.Config.EmbeddedEtcdQuotaBytes e, err := embed.StartEtcd(cfg) if err != nil { return nil, err @@ -346,7 +338,7 @@ func startEmbeddedEtcd(logger *logrus.Entry, ro RestoreOptions) (*embed.Etcd, er // applyDeltaSnapshots fetches the events from delta snapshots in parallel and applies them to the embedded etcd sequentially. func (r *Restorer) applyDeltaSnapshots(client *clientv3.Client, ro RestoreOptions) error { snapList := ro.DeltaSnapList - numMaxFetchers := ro.MaxFetchers + numMaxFetchers := ro.Config.MaxFetchers firstDeltaSnap := snapList[0] diff --git a/pkg/snapshot/restorer/restorer_suite_test.go b/pkg/snapshot/restorer/restorer_suite_test.go index bc43a5665..0f481ea2b 100644 --- a/pkg/snapshot/restorer/restorer_suite_test.go +++ b/pkg/snapshot/restorer/restorer_suite_test.go @@ -22,6 +22,8 @@ import ( "testing" "time" + "github.com/gardener/etcd-backup-restore/pkg/wrappers" + "github.com/coreos/etcd/embed" "github.com/gardener/etcd-backup-restore/pkg/etcdutil" "github.com/gardener/etcd-backup-restore/pkg/snapshot/snapshotter" @@ -103,57 +105,28 @@ func cleanUp() { // runSnapshotter creates a snapshotter object and runs it for a duration specified by 'snapshotterDurationSeconds' func runSnapshotter(logger *logrus.Entry, deltaSnapshotPeriod time.Duration, endpoints []string, stopCh <-chan struct{}, startWithFullSnapshot bool) error { - var ( - store snapstore.SnapStore - certFile string - keyFile string - caFile string - insecureTransport = true - insecureSkipVerify = true - maxBackups = 1 - etcdConnectionTimeout = 10 * time.Second - garbageCollectionPeriod = 60 * time.Second - schedule = "0 0 1 1 *" - garbageCollectionPolicy = snapshotter.GarbageCollectionPolicyLimitBased - etcdUsername string - etcdPassword string - ) - - store, err = snapstore.GetSnapstore(&snapstore.Config{Container: snapstoreDir, Provider: "Local"}) + store, err := snapstore.GetSnapstore(&snapstore.Config{Container: snapstoreDir, Provider: "Local"}) if err != nil { return err } - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword, - ) + etcdConnectionConfig := etcdutil.NewEtcdConnectionConfig() + etcdConnectionConfig.ConnectionTimeout.Duration = 10 * time.Second + etcdConnectionConfig.Endpoints = endpoints + + snapshotterConfig := &snapshotter.Config{ + FullSnapshotSchedule: "0 0 1 1 *", + DeltaSnapshotPeriod: wrappers.Duration{Duration: deltaSnapshotPeriod}, + DeltaSnapshotMemoryLimit: snapshotter.DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: time.Minute}, + GarbageCollectionPolicy: snapshotter.GarbageCollectionPolicyLimitBased, + MaxBackups: 1, + } - snapshotterConfig, err := snapshotter.NewSnapshotterConfig( - schedule, - store, - maxBackups, - snapshotter.DefaultDeltaSnapMemoryLimit, - deltaSnapshotPeriod, - etcdConnectionTimeout, - garbageCollectionPeriod, - garbageCollectionPolicy, - tlsConfig, - ) + ssr, err := snapshotter.NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) if err != nil { return err } - ssr := snapshotter.NewSnapshotter( - logger, - snapshotterConfig, - ) - return ssr.Run(stopCh, startWithFullSnapshot) } diff --git a/pkg/snapshot/restorer/restorer_test.go b/pkg/snapshot/restorer/restorer_test.go index 6c0ec2b82..63678340e 100644 --- a/pkg/snapshot/restorer/restorer_test.go +++ b/pkg/snapshot/restorer/restorer_test.go @@ -38,43 +38,36 @@ import ( var _ = Describe("Running Restorer", func() { var ( - store snapstore.SnapStore - rstr *Restorer - - restoreCluster string - restoreClusterToken string - restoreDataDir string - restorePeerURLs []string - restoreName string - skipHashCheck bool - maxFetchers int - embeddedEtcdQuotaBytes int64 - - clusterUrlsMap types.URLsMap - peerUrls types.URLs - baseSnapshot *snapstore.Snapshot - deltaSnapList snapstore.SnapList - wg *sync.WaitGroup + store snapstore.SnapStore + rstr *Restorer + restorePeerURLs []string + clusterUrlsMap types.URLsMap + peerUrls types.URLs + baseSnapshot *snapstore.Snapshot + deltaSnapList snapstore.SnapList + wg *sync.WaitGroup + ) + const ( + restoreName string = "default" + restoreClusterToken string = "etcd-cluster" + restoreCluster string = "default=http://localhost:2380" + skipHashCheck bool = false + maxFetchers uint = 6 + embeddedEtcdQuotaBytes int64 = 8 * 1024 * 1024 * 1024 ) BeforeEach(func() { wg = &sync.WaitGroup{} - restoreDataDir = etcdDir - restoreClusterToken = "etcd-cluster" - restoreName = "default" - restoreCluster = restoreName + "=http://localhost:2380" restorePeerURLs = []string{"http://localhost:2380"} clusterUrlsMap, err = types.NewURLsMap(restoreCluster) Expect(err).ShouldNot(HaveOccurred()) peerUrls, err = types.NewURLs(restorePeerURLs) Expect(err).ShouldNot(HaveOccurred()) - skipHashCheck = false - maxFetchers = 6 - embeddedEtcdQuotaBytes = 8 * 1024 * 1024 * 1024 - }) Describe("For pre-loaded Snapstore", func() { + var restoreOpts RestoreOptions + BeforeEach(func() { err = corruptEtcdDir() Expect(err).ShouldNot(HaveOccurred()) @@ -86,190 +79,103 @@ var _ = Describe("Running Restorer", func() { Expect(err).ShouldNot(HaveOccurred()) rstr = NewRestorer(store, logger) - }) - - Context("with zero fetchers", func() { - It("should return error", func() { - maxFetchers = 0 - - restoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, - } - err = rstr.Restore(restoreOptions) - Expect(err).Should(HaveOccurred()) - }) + restoreOpts = RestoreOptions{ + Config: &RestorationConfig{ + RestoreDataDir: etcdDir, + InitialClusterToken: restoreClusterToken, + InitialCluster: restoreCluster, + Name: restoreName, + InitialAdvertisePeerURLs: restorePeerURLs, + SkipHashCheck: skipHashCheck, + MaxFetchers: maxFetchers, + EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, + }, + BaseSnapshot: *baseSnapshot, + DeltaSnapList: deltaSnapList, + ClusterURLs: clusterUrlsMap, + PeerURLs: peerUrls, + } }) Context("with embedded etcd quota not set", func() { It("should be set to default value of 8 GB and restore", func() { - embeddedEtcdQuotaBytes = 0 - - RestoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, - } - err = rstr.Restore(RestoreOptions) - Expect(err).ShouldNot(HaveOccurred()) + restoreOpts.Config.EmbeddedEtcdQuotaBytes = 0 + err = restoreOpts.Config.Validate() + Expect(err).Should(HaveOccurred()) }) }) Context("with invalid cluster URLS", func() { It("should fail with an error ", func() { - restoreCluster = restoreName + "=http://localhost:2390" - restorePeerURLs = []string{"http://localhost:2390"} - clusterUrlsMap, err = types.NewURLsMap(restoreCluster) - - RestoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, - } - err = rstr.Restore(RestoreOptions) - Expect(err).Should(HaveOccurred()) + restoreOpts.Config.InitialCluster = restoreName + "=http://localhost:2390" + restoreOpts.Config.InitialAdvertisePeerURLs = []string{"http://localhost:2390"} + restoreOpts.ClusterURLs, err = types.NewURLsMap(restoreOpts.Config.InitialCluster) + err = rstr.Restore(restoreOpts) + Expect(err).Should(HaveOccurred()) }) }) Context("with invalid restore directory", func() { It("should fail to restore", func() { + restoreOpts.Config.RestoreDataDir = "" - restoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: "", //restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, - } - err = rstr.Restore(restoreOptions) - Expect(err).ShouldNot(HaveOccurred()) - + err = rstr.Restore(restoreOpts) + Expect(err).Should(HaveOccurred()) }) }) Context("with invalid snapdir and snapname", func() { It("should fail to restore", func() { + restoreOpts.BaseSnapshot.SnapDir = "test" + restoreOpts.BaseSnapshot.SnapName = "test" - restoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, - } - restoreOptions.BaseSnapshot.SnapDir = "test" - restoreOptions.BaseSnapshot.SnapName = "test" - err := rstr.Restore(restoreOptions) + err := rstr.Restore(restoreOpts) Expect(err).Should(HaveOccurred()) + }) + }) + + Context("with zero fetchers", func() { + It("should return error", func() { + restoreOpts.Config.MaxFetchers = 0 + err = restoreOpts.Config.Validate() + Expect(err).Should(HaveOccurred()) }) }) Context("with maximum of one fetcher allowed", func() { - const maxFetchers = 1 - It("should restore etcd data directory", func() { - logger.Infof("Testing for max-fetchers: %d", maxFetchers) - restoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, - } - err = rstr.Restore(restoreOptions) + restoreOpts.Config.MaxFetchers = 1 + err = rstr.Restore(restoreOpts) Expect(err).ShouldNot(HaveOccurred()) - err = checkDataConsistency(testCtx, restoreDataDir, logger) + err = checkDataConsistency(testCtx, restoreOpts.Config.RestoreDataDir, logger) Expect(err).ShouldNot(HaveOccurred()) }) }) Context("with maximum of four fetchers allowed", func() { - const maxFetchers = 4 - It("should restore etcd data directory", func() { - logger.Infof("Testing for max-fetchers: %d", maxFetchers) - restoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, - } - err = rstr.Restore(restoreOptions) + restoreOpts.Config.MaxFetchers = 4 + + err = rstr.Restore(restoreOpts) Expect(err).ShouldNot(HaveOccurred()) - err = checkDataConsistency(testCtx, restoreDataDir, logger) + err = checkDataConsistency(testCtx, restoreOpts.Config.RestoreDataDir, logger) Expect(err).ShouldNot(HaveOccurred()) }) }) Context("with maximum of hundred fetchers allowed", func() { - const maxFetchers = 100 - It("should restore etcd data directory", func() { - logger.Infof("Testing for max-fetchers: %d", maxFetchers) - restoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, - } - err = rstr.Restore(restoreOptions) + restoreOpts.Config.MaxFetchers = 100 + + err = rstr.Restore(restoreOpts) Expect(err).ShouldNot(HaveOccurred()) - err = checkDataConsistency(testCtx, restoreDataDir, logger) + err = checkDataConsistency(testCtx, restoreOpts.Config.RestoreDataDir, logger) Expect(err).ShouldNot(HaveOccurred()) }) }) @@ -280,6 +186,7 @@ var _ = Describe("Running Restorer", func() { store snapstore.SnapStore deltaSnapshotPeriod time.Duration endpoints []string + restorationConfig *RestorationConfig ) BeforeEach(func() { @@ -290,6 +197,17 @@ var _ = Describe("Running Restorer", func() { store, err = snapstore.GetSnapstore(&snapstore.Config{Container: snapstoreDir, Provider: "Local"}) Expect(err).ShouldNot(HaveOccurred()) + + restorationConfig = &RestorationConfig{ + RestoreDataDir: etcdDir, + InitialClusterToken: restoreClusterToken, + InitialCluster: restoreCluster, + Name: restoreName, + InitialAdvertisePeerURLs: restorePeerURLs, + SkipHashCheck: skipHashCheck, + MaxFetchers: maxFetchers, + EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, + } }) AfterEach(func() { @@ -324,23 +242,20 @@ var _ = Describe("Running Restorer", func() { logger.Infof("Base snapshot is %v", baseSnapshot) rstr = NewRestorer(store, logger) - restoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - DeltaSnapList: deltaSnapList, + restoreOpts := RestoreOptions{ + Config: restorationConfig, + DeltaSnapList: deltaSnapList, + ClusterURLs: clusterUrlsMap, + PeerURLs: peerUrls, } - restoreOptions.BaseSnapshot.SnapDir = "" - restoreOptions.BaseSnapshot.SnapName = "" - err := rstr.Restore(restoreOptions) + + restoreOpts.BaseSnapshot.SnapDir = "" + restoreOpts.BaseSnapshot.SnapName = "" + + err := rstr.Restore(restoreOpts) Expect(err).ShouldNot(HaveOccurred()) - err = checkDataConsistency(testCtx, restoreDataDir, logger) + err = checkDataConsistency(testCtx, restoreOpts.Config.RestoreDataDir, logger) Expect(err).ShouldNot(HaveOccurred()) }) }) @@ -368,19 +283,15 @@ var _ = Describe("Running Restorer", func() { rstr = NewRestorer(store, logger) - RestoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, + restoreOpts := RestoreOptions{ + Config: restorationConfig, + BaseSnapshot: *baseSnapshot, + DeltaSnapList: deltaSnapList, + ClusterURLs: clusterUrlsMap, + PeerURLs: peerUrls, } - err = rstr.Restore(RestoreOptions) + + err = rstr.Restore(restoreOpts) Expect(err).ShouldNot(HaveOccurred()) @@ -414,24 +325,19 @@ var _ = Describe("Running Restorer", func() { rstr = NewRestorer(store, logger) - RestoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, + restoreOpts := RestoreOptions{ + Config: restorationConfig, + BaseSnapshot: *baseSnapshot, + DeltaSnapList: deltaSnapList, + ClusterURLs: clusterUrlsMap, + PeerURLs: peerUrls, } - err = rstr.Restore(RestoreOptions) + err = rstr.Restore(restoreOpts) Expect(err).Should(HaveOccurred()) // the below consistency fails with index out of range error hence commented, // but the etcd directory is filled partially as part of the restore which should be relooked. - // err = checkDataConsistency(restoreDataDir, logger) + // err = checkDataConsistency(restoreOptions.Config.RestoreDataDir, logger) // Expect(err).Should(HaveOccurred()) }) @@ -454,20 +360,16 @@ var _ = Describe("Running Restorer", func() { rstr = NewRestorer(store, logger) - RestoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, + restoreOpts := RestoreOptions{ + Config: restorationConfig, + BaseSnapshot: *baseSnapshot, + DeltaSnapList: deltaSnapList, + ClusterURLs: clusterUrlsMap, + PeerURLs: peerUrls, } + logger.Infoln("starting restore, restore directory exists already") - err = rstr.Restore(RestoreOptions) + err = rstr.Restore(restoreOpts) logger.Infof("Failed to restore because :: %s", err) Expect(err).Should(HaveOccurred()) @@ -504,22 +406,18 @@ var _ = Describe("Running Restorer", func() { rstr = NewRestorer(store, logger) - RestoreOptions := RestoreOptions{ - ClusterURLs: clusterUrlsMap, - ClusterToken: restoreClusterToken, - RestoreDataDir: restoreDataDir, - PeerURLs: peerUrls, - SkipHashCheck: skipHashCheck, - Name: restoreName, - MaxFetchers: maxFetchers, - EmbeddedEtcdQuotaBytes: embeddedEtcdQuotaBytes, - BaseSnapshot: *baseSnapshot, - DeltaSnapList: deltaSnapList, + restoreOpts := RestoreOptions{ + Config: restorationConfig, + BaseSnapshot: *baseSnapshot, + DeltaSnapList: deltaSnapList, + ClusterURLs: clusterUrlsMap, + PeerURLs: peerUrls, } + logger.Infoln("starting restore while snapshotter is running") - err = rstr.Restore(RestoreOptions) + err = rstr.Restore(restoreOpts) Expect(err).ShouldNot(HaveOccurred()) - err = checkDataConsistency(testCtx, restoreDataDir, logger) + err = checkDataConsistency(testCtx, restoreOpts.Config.RestoreDataDir, logger) Expect(err).ShouldNot(HaveOccurred()) // Although the test has passed but the logic currently doesn't stop snapshotter explicitly but assumes that restore diff --git a/pkg/snapshot/restorer/types.go b/pkg/snapshot/restorer/types.go index 5b06e7cfc..04d18c0a3 100755 --- a/pkg/snapshot/restorer/types.go +++ b/pkg/snapshot/restorer/types.go @@ -36,19 +36,26 @@ type Restorer struct { // RestoreOptions hold all snapshot restore related fields type RestoreOptions struct { - ClusterURLs types.URLsMap - ClusterToken string - RestoreDataDir string - PeerURLs types.URLs - SkipHashCheck bool - Name string - MaxFetchers int - EmbeddedEtcdQuotaBytes int64 + Config *RestorationConfig + ClusterURLs types.URLsMap + PeerURLs types.URLs // Base full snapshot + delta snapshots to restore from BaseSnapshot snapstore.Snapshot DeltaSnapList snapstore.SnapList } +// RestorationConfig holds the restoration configuration. +type RestorationConfig struct { + InitialCluster string `json:"initialCluster"` + InitialClusterToken string `json:"initialClusterToken,omitempty"` + RestoreDataDir string `json:"restoreDataDir,omitempty"` + InitialAdvertisePeerURLs []string `json:"initialAdvertisePeerURLs"` + Name string `json:"name"` + SkipHashCheck bool `json:"skipHashCheck,omitempty"` + MaxFetchers uint `json:"maxFetchers,omitempty"` + EmbeddedEtcdQuotaBytes int64 `json:"embeddedEtcdQuotaBytes,omitempty"` +} + type initIndex int func (i *initIndex) ConsistentIndex() uint64 { diff --git a/pkg/snapshot/snapshotter/garbagecollector.go b/pkg/snapshot/snapshotter/garbagecollector.go index db354de26..33bbc1b30 100644 --- a/pkg/snapshot/snapshotter/garbagecollector.go +++ b/pkg/snapshot/snapshotter/garbagecollector.go @@ -27,8 +27,8 @@ import ( // RunGarbageCollector basically consider the older backups as garbage and deletes it func (ssr *Snapshotter) RunGarbageCollector(stopCh <-chan struct{}) { - if ssr.config.garbageCollectionPeriod <= time.Second { - ssr.logger.Infof("GC: Not running garbage collector since GarbageCollectionPeriod [%s] set to less than 1 second.", ssr.config.garbageCollectionPeriod) + if ssr.config.GarbageCollectionPeriod.Duration <= time.Second { + ssr.logger.Infof("GC: Not running garbage collector since GarbageCollectionPeriod [%s] set to less than 1 second.", ssr.config.GarbageCollectionPeriod) return } @@ -37,10 +37,10 @@ func (ssr *Snapshotter) RunGarbageCollector(stopCh <-chan struct{}) { case <-stopCh: ssr.logger.Info("GC: Stop signal received. Closing garbage collector.") return - case <-time.After(ssr.config.garbageCollectionPeriod): + case <-time.After(ssr.config.GarbageCollectionPeriod.Duration): total := 0 ssr.logger.Info("GC: Executing garbage collection...") - snapList, err := ssr.config.store.List() + snapList, err := ssr.store.List() if err != nil { ssr.logger.Warnf("GC: Failed to list snapshots: %v", err) continue @@ -48,7 +48,7 @@ func (ssr *Snapshotter) RunGarbageCollector(stopCh <-chan struct{}) { snapStreamIndexList := getSnapStreamIndexList(snapList) - switch ssr.config.garbageCollectionPolicy { + switch ssr.config.GarbageCollectionPolicy { case GarbageCollectionPolicyExponential: // Overall policy: // Delete delta snapshots in all snapStream but the latest one. @@ -121,14 +121,14 @@ func (ssr *Snapshotter) RunGarbageCollector(stopCh <-chan struct{}) { if deleteSnap { ssr.logger.Infof("GC: Deleting old full snapshot: %s %v", nextSnap.CreatedOn.UTC(), deleteSnap) - if err := ssr.config.store.Delete(*nextSnap); err != nil { + if err := ssr.store.Delete(*nextSnap); err != nil { ssr.logger.Warnf("GC: Failed to delete snapshot %s: %v", path.Join(nextSnap.SnapDir, nextSnap.SnapName), err) metrics.GCSnapshotCounter.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull, metrics.LabelSucceeded: metrics.ValueSucceededFalse}).Inc() continue } metrics.GCSnapshotCounter.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull, metrics.LabelSucceeded: metrics.ValueSucceededTrue}).Inc() total++ - garbageCollectChunks(ssr.config.store, snapList, snapStreamIndexList[snapStreamIndex-1]+1, snapStreamIndexList[snapStreamIndex]) + garbageCollectChunks(ssr.store, snapList, snapStreamIndexList[snapStreamIndex-1]+1, snapStreamIndexList[snapStreamIndex]) } } @@ -141,18 +141,18 @@ func (ssr *Snapshotter) RunGarbageCollector(stopCh <-chan struct{}) { if err != nil { continue } - if snapStreamIndex < len(snapStreamIndexList)-ssr.config.maxBackups { + if snapStreamIndex < len(snapStreamIndexList)-int(ssr.config.MaxBackups) { snap := snapList[snapStreamIndexList[snapStreamIndex]] snapPath := path.Join(snap.SnapDir, snap.SnapName) ssr.logger.Infof("GC: Deleting old full snapshot: %s", snapPath) - if err := ssr.config.store.Delete(*snap); err != nil { + if err := ssr.store.Delete(*snap); err != nil { ssr.logger.Warnf("GC: Failed to delete snapshot %s: %v", snapPath, err) metrics.GCSnapshotCounter.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull, metrics.LabelSucceeded: metrics.ValueSucceededFalse}).Inc() continue } metrics.GCSnapshotCounter.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull, metrics.LabelSucceeded: metrics.ValueSucceededTrue}).Inc() total++ - garbageCollectChunks(ssr.config.store, snapList, snapStreamIndexList[snapStreamIndex]+1, snapStreamIndexList[snapStreamIndex+1]) + garbageCollectChunks(ssr.store, snapList, snapStreamIndexList[snapStreamIndex]+1, snapStreamIndexList[snapStreamIndex+1]) } } } @@ -206,7 +206,7 @@ func (ssr *Snapshotter) garbageCollectDeltaSnapshots(snapStream snapstore.SnapLi } snapPath := path.Join(snapStream[i].SnapDir, snapStream[i].SnapName) ssr.logger.Infof("GC: Deleting old delta snapshot: %s", snapPath) - if err := ssr.config.store.Delete(*snapStream[i]); err != nil { + if err := ssr.store.Delete(*snapStream[i]); err != nil { ssr.logger.Warnf("GC: Failed to delete snapshot %s: %v", snapPath, err) metrics.GCSnapshotCounter.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindDelta, metrics.LabelSucceeded: metrics.ValueSucceededFalse}).Inc() return total, err diff --git a/pkg/snapshot/snapshotter/init.go b/pkg/snapshot/snapshotter/init.go new file mode 100644 index 000000000..610db54f3 --- /dev/null +++ b/pkg/snapshot/snapshotter/init.go @@ -0,0 +1,70 @@ +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 snapshotter + +import ( + "fmt" + "time" + + "github.com/gardener/etcd-backup-restore/pkg/wrappers" + cron "github.com/robfig/cron/v3" + "github.com/sirupsen/logrus" + flag "github.com/spf13/pflag" +) + +// NewSnapshotterConfig returns the snapshotter config. +func NewSnapshotterConfig() *Config { + return &Config{ + FullSnapshotSchedule: "0 */1 * * *", + DeltaSnapshotPeriod: wrappers.Duration{Duration: 20 * time.Second}, + DeltaSnapshotMemoryLimit: 10 * 1024 * 1024, //10Mib + GarbageCollectionPeriod: wrappers.Duration{Duration: time.Minute}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: 7, + } +} + +// AddFlags adds the flags to flagset. +func (c *Config) AddFlags(fs *flag.FlagSet) { + fs.StringVarP(&c.FullSnapshotSchedule, "schedule", "s", c.FullSnapshotSchedule, "schedule for snapshots") + fs.DurationVar(&c.DeltaSnapshotPeriod.Duration, "delta-snapshot-period", c.DeltaSnapshotPeriod.Duration, "Period after which delta snapshot will be persisted. If this value is set to be lesser than 1, delta snapshotting will be disabled.") + fs.UintVar(&c.DeltaSnapshotMemoryLimit, "delta-snapshot-memory-limit", c.DeltaSnapshotMemoryLimit, "memory limit after which delta snapshots will be taken") + fs.DurationVar(&c.GarbageCollectionPeriod.Duration, "garbage-collection-period", c.GarbageCollectionPeriod.Duration, "Period for garbage collecting old backups") + fs.StringVar(&c.GarbageCollectionPolicy, "garbage-collection-policy", c.GarbageCollectionPolicy, "Policy for garbage collecting old backups") + fs.UintVarP(&c.MaxBackups, "max-backups", "m", c.MaxBackups, "maximum number of previous backups to keep") +} + +// Validate validates the config. +func (c *Config) Validate() error { + if _, err := cron.ParseStandard(c.FullSnapshotSchedule); err != nil { + return err + } + if c.GarbageCollectionPolicy != GarbageCollectionPolicyLimitBased && c.GarbageCollectionPolicy != GarbageCollectionPolicyExponential { + return fmt.Errorf("invalid garbage collection policy: %s", c.GarbageCollectionPolicy) + } + if c.GarbageCollectionPolicy == GarbageCollectionPolicyLimitBased && c.MaxBackups <= 0 { + return fmt.Errorf("max backups should be greather than zero for garbage collection policy set to limit based") + } + + if c.DeltaSnapshotPeriod.Duration < deltaSnapshotIntervalThreshold { + logrus.Infof("Found delta snapshot interval %s less than 1 second. Disabling delta snapshotting. ", c.DeltaSnapshotPeriod) + } + + if c.DeltaSnapshotMemoryLimit < 1 { + logrus.Infof("Found delta snapshot memory limit %d bytes less than 1 byte. Setting it to default: %d ", c.DeltaSnapshotMemoryLimit, DefaultDeltaSnapMemoryLimit) + c.DeltaSnapshotMemoryLimit = DefaultDeltaSnapMemoryLimit + } + return nil +} diff --git a/pkg/snapshot/snapshotter/snapshotter.go b/pkg/snapshot/snapshotter/snapshotter.go index 634472357..1cfc056a1 100644 --- a/pkg/snapshot/snapshotter/snapshotter.go +++ b/pkg/snapshot/snapshotter/snapshotter.go @@ -25,11 +25,10 @@ import ( "sync" "time" - "github.com/gardener/etcd-backup-restore/pkg/metrics" - "github.com/coreos/etcd/clientv3" "github.com/gardener/etcd-backup-restore/pkg/errors" "github.com/gardener/etcd-backup-restore/pkg/etcdutil" + "github.com/gardener/etcd-backup-restore/pkg/metrics" "github.com/gardener/etcd-backup-restore/pkg/miscellaneous" "github.com/gardener/etcd-backup-restore/pkg/snapstore" "github.com/prometheus/client_golang/prometheus" @@ -37,44 +36,17 @@ import ( "github.com/sirupsen/logrus" ) -// NewSnapshotterConfig returns a config for the snapshotter. -func NewSnapshotterConfig(schedule string, store snapstore.SnapStore, maxBackups, deltaSnapshotMemoryLimit int, deltaSnapshotInterval, etcdConnectionTimeout, garbageCollectionPeriod time.Duration, garbageCollectionPolicy string, tlsConfig *etcdutil.TLSConfig) (*Config, error) { - logrus.Printf("Validating schedule... [%s]", schedule) - sdl, err := cron.ParseStandard(schedule) +// NewSnapshotter returns the snapshotter object. +func NewSnapshotter(logger *logrus.Entry, config *Config, store snapstore.SnapStore, etcdConnectionConfig *etcdutil.EtcdConnectionConfig) (*Snapshotter, error) { + sdl, err := cron.ParseStandard(config.FullSnapshotSchedule) if err != nil { - return nil, fmt.Errorf("invalid schedule provied %s : %v", schedule, err) - } - - if garbageCollectionPolicy == GarbageCollectionPolicyLimitBased && maxBackups < 1 { - logrus.Infof("Found garbage collection policy: [%s], and maximum backup value %d less than 1. Setting it to default: %d ", GarbageCollectionPolicyLimitBased, maxBackups, DefaultMaxBackups) - maxBackups = DefaultMaxBackups - } - if deltaSnapshotInterval < deltaSnapshotIntervalThreshold { - logrus.Infof("Found delta snapshot interval %s less than 1 second. Disabling delta snapshotting. ", deltaSnapshotInterval) - } - if deltaSnapshotMemoryLimit < 1 { - logrus.Infof("Found delta snapshot memory limit %d bytes less than 1 byte. Setting it to default: %d ", deltaSnapshotMemoryLimit, DefaultDeltaSnapMemoryLimit) - deltaSnapshotMemoryLimit = DefaultDeltaSnapMemoryLimit + // Ideally this should be validated before. + return nil, fmt.Errorf("invalid schedule provied %s : %v", config.FullSnapshotSchedule, err) } - return &Config{ - schedule: sdl, - store: store, - deltaSnapshotMemoryLimit: deltaSnapshotMemoryLimit, - deltaSnapshotInterval: deltaSnapshotInterval, - etcdConnectionTimeout: etcdConnectionTimeout, - garbageCollectionPeriod: garbageCollectionPeriod, - garbageCollectionPolicy: garbageCollectionPolicy, - maxBackups: maxBackups, - tlsConfig: tlsConfig, - }, nil -} - -// NewSnapshotter returns the snapshotter object. -func NewSnapshotter(logger *logrus.Entry, config *Config) *Snapshotter { // Create dummy previous snapshot var prevSnapshot *snapstore.Snapshot - fullSnap, deltaSnapList, err := miscellaneous.GetLatestFullSnapshotAndDeltaSnapList(config.store) + fullSnap, deltaSnapList, err := miscellaneous.GetLatestFullSnapshotAndDeltaSnapList(store) if err != nil || fullSnap == nil { prevSnapshot = snapstore.NewSnapshot(snapstore.SnapshotKindFull, 0, 0) } else if len(deltaSnapList) == 0 { @@ -88,10 +60,14 @@ func NewSnapshotter(logger *logrus.Entry, config *Config) *Snapshotter { metrics.LatestSnapshotRevision.With(prometheus.Labels{metrics.LabelKind: prevSnapshot.Kind}).Set(float64(prevSnapshot.LastRevision)) return &Snapshotter{ - logger: logger.WithField("actor", "snapshotter"), + logger: logger.WithField("actor", "snapshotter"), + store: store, + config: config, + etcdConnectionConfig: etcdConnectionConfig, + + schedule: sdl, prevSnapshot: prevSnapshot, PrevFullSnapshot: fullSnap, - config: config, SsrState: SnapshotterInactive, SsrStateMutex: &sync.Mutex{}, fullSnapshotReqCh: make(chan struct{}), @@ -99,7 +75,7 @@ func NewSnapshotter(logger *logrus.Entry, config *Config) *Snapshotter { fullSnapshotAckCh: make(chan error), deltaSnapshotAckCh: make(chan error), cancelWatch: func() {}, - } + }, nil } // Run process loop for scheduled backup @@ -129,9 +105,9 @@ func (ssr *Snapshotter) Run(stopCh <-chan struct{}, startWithFullSnapshot bool) } ssr.deltaSnapshotTimer = time.NewTimer(DefaultDeltaSnapshotInterval) - if ssr.config.deltaSnapshotInterval >= deltaSnapshotIntervalThreshold { + if ssr.config.DeltaSnapshotPeriod.Duration >= deltaSnapshotIntervalThreshold { ssr.deltaSnapshotTimer.Stop() - ssr.deltaSnapshotTimer.Reset(ssr.config.deltaSnapshotInterval) + ssr.deltaSnapshotTimer.Reset(ssr.config.DeltaSnapshotPeriod.Duration) } return ssr.snapshotEventHandler(stopCh) @@ -160,8 +136,8 @@ func (ssr *Snapshotter) TriggerDeltaSnapshot() error { if ssr.SsrState != SnapshotterActive { return fmt.Errorf("snapshotter is not active") } - if ssr.config.deltaSnapshotInterval < deltaSnapshotIntervalThreshold { - return fmt.Errorf("Found delta snapshot interval %s less than %v. Delta snapshotting is disabled. ", ssr.config.deltaSnapshotInterval, time.Duration(deltaSnapshotIntervalThreshold)) + if ssr.config.DeltaSnapshotPeriod.Duration < deltaSnapshotIntervalThreshold { + return fmt.Errorf("Found delta snapshot interval %s less than %v. Delta snapshotting is disabled. ", ssr.config.DeltaSnapshotPeriod.Duration, time.Duration(deltaSnapshotIntervalThreshold)) } ssr.logger.Info("Triggering out of schedule delta snapshot...") ssr.deltaSnapshotReqCh <- emptyStruct @@ -225,14 +201,14 @@ func (ssr *Snapshotter) takeFullSnapshot() error { // close previous watch and client. ssr.closeEtcdClient() - client, err := etcdutil.GetTLSClientForEtcd(ssr.config.tlsConfig) + client, err := etcdutil.GetTLSClientForEtcd(ssr.etcdConnectionConfig) if err != nil { return &errors.EtcdError{ Message: fmt.Sprintf("failed to create etcd client: %v", err), } } - ctx, cancel := context.WithTimeout(context.TODO(), ssr.config.etcdConnectionTimeout) + ctx, cancel := context.WithTimeout(context.TODO(), ssr.etcdConnectionConfig.ConnectionTimeout.Duration) // Note: Although Get and snapshot call are not atomic, so revision number in snapshot file // may be ahead of the revision found from GET call. But currently this is the only workaround available // Refer: https://github.com/coreos/etcd/issues/9037 @@ -248,7 +224,7 @@ func (ssr *Snapshotter) takeFullSnapshot() error { if ssr.prevSnapshot.Kind == snapstore.SnapshotKindFull && ssr.prevSnapshot.LastRevision == lastRevision { ssr.logger.Infof("There are no updates since last snapshot, skipping full snapshot.") } else { - ctx, cancel = context.WithTimeout(context.TODO(), ssr.config.etcdConnectionTimeout) + ctx, cancel = context.WithTimeout(context.TODO(), ssr.etcdConnectionConfig.ConnectionTimeout.Duration) defer cancel() rc, err := client.Snapshot(ctx) if err != nil { @@ -259,7 +235,7 @@ func (ssr *Snapshotter) takeFullSnapshot() error { ssr.logger.Infof("Successfully opened snapshot reader on etcd") s := snapstore.NewSnapshot(snapstore.SnapshotKindFull, 0, lastRevision) startTime := time.Now() - if err := ssr.config.store.Save(*s, rc); err != nil { + if err := ssr.store.Save(*s, rc); err != nil { timeTaken := time.Now().Sub(startTime).Seconds() metrics.SnapshotDurationSeconds.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull, metrics.LabelSucceeded: metrics.ValueSucceededFalse}).Observe(timeTaken) return &errors.SnapstoreError{ @@ -283,7 +259,7 @@ func (ssr *Snapshotter) takeFullSnapshot() error { metrics.SnapshotRequired.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull}).Set(0) metrics.SnapshotRequired.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindDelta}).Set(0) - if ssr.config.deltaSnapshotInterval < time.Second { + if ssr.config.DeltaSnapshotPeriod.Duration < time.Second { // return without creating a watch on events return nil } @@ -311,12 +287,12 @@ func (ssr *Snapshotter) takeDeltaSnapshotAndResetTimer() error { } if ssr.deltaSnapshotTimer == nil { - ssr.deltaSnapshotTimer = time.NewTimer(ssr.config.deltaSnapshotInterval) + ssr.deltaSnapshotTimer = time.NewTimer(ssr.config.DeltaSnapshotPeriod.Duration) } else { ssr.logger.Infof("Stopping delta snapshot...") ssr.deltaSnapshotTimer.Stop() - ssr.logger.Infof("Resetting delta snapshot to run after %s.", ssr.config.deltaSnapshotInterval.String()) - ssr.deltaSnapshotTimer.Reset(ssr.config.deltaSnapshotInterval) + ssr.logger.Infof("Resetting delta snapshot to run after %s.", ssr.config.DeltaSnapshotPeriod.Duration.String()) + ssr.deltaSnapshotTimer.Reset(ssr.config.DeltaSnapshotPeriod.Duration) } return nil } @@ -343,7 +319,7 @@ func (ssr *Snapshotter) TakeDeltaSnapshot() error { } ssr.events = hash.Sum(ssr.events) startTime := time.Now() - if err := ssr.config.store.Save(*snap, ioutil.NopCloser(bytes.NewReader(ssr.events))); err != nil { + if err := ssr.store.Save(*snap, ioutil.NopCloser(bytes.NewReader(ssr.events))); err != nil { timeTaken := time.Now().Sub(startTime).Seconds() metrics.SnapshotDurationSeconds.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindDelta, metrics.LabelSucceeded: metrics.ValueSucceededFalse}).Observe(timeTaken) ssr.logger.Errorf("Error saving delta snapshots. %v", err) @@ -364,15 +340,14 @@ func (ssr *Snapshotter) TakeDeltaSnapshot() error { func (ssr *Snapshotter) CollectEventsSincePrevSnapshot(stopCh <-chan struct{}) (bool, error) { // close any previous watch and client. ssr.closeEtcdClient() - - client, err := etcdutil.GetTLSClientForEtcd(ssr.config.tlsConfig) + client, err := etcdutil.GetTLSClientForEtcd(ssr.etcdConnectionConfig) if err != nil { return false, &errors.EtcdError{ Message: fmt.Sprintf("failed to create etcd client: %v", err), } } - ctx, cancel := context.WithTimeout(context.TODO(), ssr.config.etcdConnectionTimeout) + ctx, cancel := context.WithTimeout(context.TODO(), ssr.etcdConnectionConfig.ConnectionTimeout.Duration) resp, err := client.Get(ctx, "", clientv3.WithLastRev()...) cancel() if err != nil { @@ -391,6 +366,7 @@ func (ssr *Snapshotter) CollectEventsSincePrevSnapshot(stopCh <-chan struct{}) ( metrics.SnapshotRequired.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindFull}).Set(1) } + // TODO: Use parent context. Passing parent context here directly requires some additional management of error handling. watchCtx, cancelWatch := context.WithCancel(context.TODO()) ssr.cancelWatch = cancelWatch ssr.etcdClient = client @@ -451,7 +427,7 @@ func (ssr *Snapshotter) handleDeltaWatchEvents(wr clientv3.WatchResponse) error metrics.SnapshotRequired.With(prometheus.Labels{metrics.LabelKind: snapstore.SnapshotKindDelta}).Set(1) } ssr.logger.Debugf("Added events till revision: %d", ssr.lastEventRevision) - if len(ssr.events) >= ssr.config.deltaSnapshotMemoryLimit { + if len(ssr.events) >= int(ssr.config.DeltaSnapshotMemoryLimit) { ssr.logger.Infof("Delta events memory crossed the memory limit: %d Bytes", len(ssr.events)) return ssr.takeDeltaSnapshotAndResetTimer() } @@ -488,7 +464,7 @@ func (ssr *Snapshotter) snapshotEventHandler(stopCh <-chan struct{}) error { } case <-ssr.deltaSnapshotTimer.C: - if ssr.config.deltaSnapshotInterval >= time.Second { + if ssr.config.DeltaSnapshotPeriod.Duration >= time.Second { if err := ssr.takeDeltaSnapshotAndResetTimer(); err != nil { return err } @@ -511,7 +487,7 @@ func (ssr *Snapshotter) snapshotEventHandler(stopCh <-chan struct{}) error { func (ssr *Snapshotter) resetFullSnapshotTimer() error { now := time.Now() - effective := ssr.config.schedule.Next(now) + effective := ssr.schedule.Next(now) if effective.IsZero() { ssr.logger.Info("There are no backups scheduled for the future. Stopping now.") return fmt.Errorf("error in full snapshot schedule") diff --git a/pkg/snapshot/snapshotter/snapshotter_test.go b/pkg/snapshot/snapshotter/snapshotter_test.go index f6cd2c286..0ff42ed6e 100644 --- a/pkg/snapshot/snapshotter/snapshotter_test.go +++ b/pkg/snapshot/snapshotter/snapshotter_test.go @@ -27,6 +27,7 @@ import ( "github.com/gardener/etcd-backup-restore/pkg/etcdutil" . "github.com/gardener/etcd-backup-restore/pkg/snapshot/snapshotter" "github.com/gardener/etcd-backup-restore/pkg/snapstore" + "github.com/gardener/etcd-backup-restore/pkg/wrappers" "github.com/gardener/etcd-backup-restore/test/utils" . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" @@ -34,28 +35,19 @@ import ( var _ = Describe("Snapshotter", func() { var ( - endpoints []string store snapstore.SnapStore - etcdConnectionTimeout time.Duration garbageCollectionPeriod time.Duration - maxBackups int + maxBackups uint schedule string - certFile string - keyFile string - caFile string - insecureTransport bool - insecureSkipVerify bool - etcdUsername string - etcdPassword string + etcdConnectionConfig *etcdutil.EtcdConnectionConfig err error ) BeforeEach(func() { - endpoints = []string{etcd.Clients[0].Addr().String()} - etcdConnectionTimeout = 10 * time.Second + etcdConnectionConfig = etcdutil.NewEtcdConnectionConfig() + etcdConnectionConfig.Endpoints = []string{etcd.Clients[0].Addr().String()} + etcdConnectionConfig.ConnectionTimeout.Duration = 5 * time.Second garbageCollectionPeriod = 30 * time.Second schedule = "*/1 * * * *" - insecureTransport = true - insecureSkipVerify = true }) Describe("creating Snapshotter", func() { @@ -66,25 +58,16 @@ var _ = Describe("Snapshotter", func() { Context("With invalid schedule", func() { It("should return error", func() { schedule = "65 * * * 5" - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - _, err := NewSnapshotterConfig( - schedule, - store, - 1, - 10, - DefaultDeltaSnapMemoryLimit, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: 10}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: 1, + } + + _, err := NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) Expect(err).Should(HaveOccurred()) }) }) @@ -92,25 +75,16 @@ var _ = Describe("Snapshotter", func() { Context("With valid schedule", func() { It("should create snapshotter config", func() { schedule = "*/5 * * * *" - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - _, err := NewSnapshotterConfig( - schedule, - store, - 1, - 10, - DefaultDeltaSnapMemoryLimit, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: 10}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: 1, + } + + _, err := NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) Expect(err).ShouldNot(HaveOccurred()) }) }) @@ -118,43 +92,32 @@ var _ = Describe("Snapshotter", func() { Describe("running snapshotter", func() { Context("with etcd not running at configured endpoint", func() { - It("should timeout & not take any snapshot", func() { + BeforeEach(func() { validEndpoint := etcd.Clients[0].Addr().String() tokens := strings.Split(validEndpoint, ":") Expect(len(tokens)).Should(BeNumerically(">=", 2)) i, err := strconv.Atoi(tokens[len(tokens)-1]) Expect(err).ShouldNot(HaveOccurred()) invalidEndpoint := fmt.Sprintf("%s:%d", strings.Join(tokens[:len(tokens)-1], ":"), i+12) - endpoints = []string{invalidEndpoint} - etcdConnectionTimeout = 5 * time.Second + etcdConnectionConfig.Endpoints = []string{invalidEndpoint} + }) + + It("should timeout & not take any snapshot", func() { maxBackups = 2 testTimeout := time.Duration(time.Minute * time.Duration(maxBackups+1)) store, err = snapstore.GetSnapstore(&snapstore.Config{Container: path.Join(outputDir, "snapshotter_2.bkp")}) Expect(err).ShouldNot(HaveOccurred()) - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := NewSnapshotterConfig( - schedule, - store, - maxBackups, - 10, - DefaultDeltaSnapMemoryLimit, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) - Expect(err).ShouldNot(HaveOccurred()) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: 10}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: maxBackups, + } - ssr := NewSnapshotter( - logger, - snapshotterConfig) + ssr, err := NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) + Expect(err).ShouldNot(HaveOccurred()) ctx, cancel := context.WithTimeout(testCtx, testTimeout) defer cancel() @@ -168,42 +131,28 @@ var _ = Describe("Snapshotter", func() { Context("with etcd running at configured endpoint", func() { BeforeEach(func() { - endpoints = []string{etcd.Clients[0].Addr().String()} + etcdConnectionConfig.Endpoints = []string{etcd.Clients[0].Addr().String()} }) Context("with unreachable schedule", func() { var ssr *Snapshotter BeforeEach(func() { schedule = "* * 31 2 *" - etcdConnectionTimeout = 5 * time.Second maxBackups = 2 testTimeout := time.Duration(time.Minute * time.Duration(maxBackups+1)) store, err = snapstore.GetSnapstore(&snapstore.Config{Container: path.Join(outputDir, "snapshotter_3.bkp")}) Expect(err).ShouldNot(HaveOccurred()) - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := NewSnapshotterConfig( - schedule, - store, - maxBackups, - DefaultDeltaSnapMemoryLimit, - 10*time.Second, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) - Expect(err).ShouldNot(HaveOccurred()) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: 10 * time.Second}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: maxBackups, + } - ssr = NewSnapshotter( - logger, - snapshotterConfig) + ssr, err = NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) + Expect(err).ShouldNot(HaveOccurred()) ctx, cancel := context.WithTimeout(testCtx, testTimeout) defer cancel() err = ssr.Run(ctx.Done(), true) @@ -227,17 +176,15 @@ var _ = Describe("Snapshotter", func() { var ( ssr *Snapshotter schedule string - maxBackups int + maxBackups uint testTimeout time.Duration deltaSnapshotInterval time.Duration ) BeforeEach(func() { - endpoints = []string{etcd.Clients[0].Addr().String()} schedule = "*/1 * * * *" maxBackups = 2 // We will wait for maxBackups+1 times schedule period testTimeout = time.Duration(time.Minute * time.Duration(maxBackups+1)) - etcdConnectionTimeout = 5 * time.Second }) Context("with delta snapshot interval set to zero seconds", func() { @@ -245,33 +192,21 @@ var _ = Describe("Snapshotter", func() { deltaSnapshotInterval = 0 testTimeout = time.Duration(time.Minute * time.Duration(maxBackups)) }) + It("should take periodic backups without delta snapshots", func() { store, err = snapstore.GetSnapstore(&snapstore.Config{Container: path.Join(outputDir, "snapshotter_4.bkp")}) Expect(err).ShouldNot(HaveOccurred()) - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := NewSnapshotterConfig( - schedule, - store, - maxBackups, - DefaultDeltaSnapMemoryLimit, - deltaSnapshotInterval, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) - Expect(err).ShouldNot(HaveOccurred()) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: deltaSnapshotInterval}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: maxBackups, + } - ssr = NewSnapshotter( - logger, - snapshotterConfig) + ssr, err = NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) + Expect(err).ShouldNot(HaveOccurred()) ctx, cancel := context.WithTimeout(testCtx, testTimeout) defer cancel() @@ -288,30 +223,17 @@ var _ = Describe("Snapshotter", func() { It("should fail on triggering out-of-schedule delta snapshot", func() { store, err = snapstore.GetSnapstore(&snapstore.Config{Container: path.Join(outputDir, "snapshotter_4.bkp")}) Expect(err).ShouldNot(HaveOccurred()) - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := NewSnapshotterConfig( - schedule, - store, - maxBackups, - DefaultDeltaSnapMemoryLimit, - deltaSnapshotInterval, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) - Expect(err).ShouldNot(HaveOccurred()) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: deltaSnapshotInterval}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: maxBackups, + } - ssr = NewSnapshotter( - logger, - snapshotterConfig) + ssr, err = NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) + Expect(err).ShouldNot(HaveOccurred()) err = ssr.TriggerDeltaSnapshot() Expect(err).Should(HaveOccurred()) @@ -329,36 +251,23 @@ var _ = Describe("Snapshotter", func() { currentHour := time.Now().Hour() store, err = snapstore.GetSnapstore(&snapstore.Config{Container: path.Join(outputDir, "snapshotter_5.bkp")}) Expect(err).ShouldNot(HaveOccurred()) - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := NewSnapshotterConfig( - fmt.Sprintf("59 %d * * *", (currentHour+1)%24), // This make sure that full snapshot timer doesn't trigger full snapshot. - store, - maxBackups, - DefaultDeltaSnapMemoryLimit, - deltaSnapshotInterval, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) + snapshotterConfig := &Config{ + FullSnapshotSchedule: fmt.Sprintf("59 %d * * *", (currentHour+1)%24), // This make sure that full snapshot timer doesn't trigger full snapshot. + DeltaSnapshotPeriod: wrappers.Duration{Duration: deltaSnapshotInterval}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: maxBackups, + } + + ssr, err = NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) Expect(err).ShouldNot(HaveOccurred()) - - ssr = NewSnapshotter( - logger, - snapshotterConfig) populatorCtx, cancelPopulator := context.WithTimeout(testCtx, testTimeout) defer cancelPopulator() wg := &sync.WaitGroup{} wg.Add(1) // populating etcd so that snapshots will be taken - go utils.PopulateEtcdWithWaitGroup(populatorCtx, wg, logger, endpoints, nil) + go utils.PopulateEtcdWithWaitGroup(populatorCtx, wg, logger, etcdConnectionConfig.Endpoints, nil) ssrCtx := utils.ContextWithWaitGroup(testCtx, wg) err = ssr.Run(ssrCtx.Done(), false) Expect(err).ShouldNot(HaveOccurred()) @@ -373,37 +282,24 @@ var _ = Describe("Snapshotter", func() { It("should take periodic backups", func() { store, err = snapstore.GetSnapstore(&snapstore.Config{Container: path.Join(outputDir, "snapshotter_6.bkp")}) Expect(err).ShouldNot(HaveOccurred()) - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := NewSnapshotterConfig( - schedule, - store, - maxBackups, - DefaultDeltaSnapMemoryLimit, - deltaSnapshotInterval, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) - Expect(err).ShouldNot(HaveOccurred()) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: deltaSnapshotInterval}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: maxBackups, + } populatorCtx, cancelPopulator := context.WithTimeout(testCtx, testTimeout) defer cancelPopulator() wg := &sync.WaitGroup{} wg.Add(1) // populating etcd so that snapshots will be taken - go utils.PopulateEtcdWithWaitGroup(populatorCtx, wg, logger, endpoints, nil) + go utils.PopulateEtcdWithWaitGroup(populatorCtx, wg, logger, etcdConnectionConfig.Endpoints, nil) - ssr = NewSnapshotter( - logger, - snapshotterConfig) + ssr, err = NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) + Expect(err).ShouldNot(HaveOccurred()) ssrCtx := utils.ContextWithWaitGroup(testCtx, wg) err = ssr.Run(ssrCtx.Done(), true) @@ -423,12 +319,11 @@ var _ = Describe("Snapshotter", func() { testTimeout time.Duration ) BeforeEach(func() { - endpoints = []string{etcd.Clients[0].Addr().String()} + etcdConnectionConfig.Endpoints = []string{etcd.Clients[0].Addr().String()} schedule = "*/1 * * * *" maxBackups = 2 garbageCollectionPeriod = 5 * time.Second testTimeout = garbageCollectionPeriod * 2 - etcdConnectionTimeout = 5 * time.Second }) It("should garbage collect exponentially", func() { @@ -532,37 +427,21 @@ var _ = Describe("Snapshotter", func() { fmt.Println("Incremental snapshot list prepared") //start test - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := NewSnapshotterConfig( - schedule, - store, - maxBackups, - DefaultDeltaSnapMemoryLimit, - 10*time.Second, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyExponential, - tlsConfig) - Expect(err).ShouldNot(HaveOccurred()) - ssr := NewSnapshotter( - logger, - snapshotterConfig) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: 10 * time.Second}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyExponential, + MaxBackups: maxBackups, + } - gcStopCh := make(chan struct{}) + ssr, err := NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) + Expect(err).ShouldNot(HaveOccurred()) - go func() { - <-time.After(testTimeout) - close(gcStopCh) - }() - ssr.RunGarbageCollector(gcStopCh) + gcCtx, cancel := context.WithTimeout(testCtx, testTimeout) + defer cancel() + ssr.RunGarbageCollector(gcCtx.Done()) list, err := store.List() Expect(err).ShouldNot(HaveOccurred()) @@ -578,30 +457,17 @@ var _ = Describe("Snapshotter", func() { It("should garbage collect limitBased", func() { now := time.Now().UTC() store := prepareStoreForGarbageCollection(now, "garbagecollector_limit_based.bkp") - tlsConfig := etcdutil.NewTLSConfig( - certFile, - keyFile, - caFile, - insecureTransport, - insecureSkipVerify, - endpoints, - etcdUsername, - etcdPassword) - snapshotterConfig, err := NewSnapshotterConfig( - schedule, - store, - maxBackups, - DefaultDeltaSnapMemoryLimit, - 10*time.Second, - etcdConnectionTimeout, - garbageCollectionPeriod, - GarbageCollectionPolicyLimitBased, - tlsConfig) - Expect(err).ShouldNot(HaveOccurred()) + snapshotterConfig := &Config{ + FullSnapshotSchedule: schedule, + DeltaSnapshotPeriod: wrappers.Duration{Duration: 10 * time.Second}, + DeltaSnapshotMemoryLimit: DefaultDeltaSnapMemoryLimit, + GarbageCollectionPeriod: wrappers.Duration{Duration: garbageCollectionPeriod}, + GarbageCollectionPolicy: GarbageCollectionPolicyLimitBased, + MaxBackups: maxBackups, + } - ssr := NewSnapshotter( - logger, - snapshotterConfig) + ssr, err := NewSnapshotter(logger, snapshotterConfig, store, etcdConnectionConfig) + Expect(err).ShouldNot(HaveOccurred()) gcCtx, cancel := context.WithTimeout(testCtx, testTimeout) defer cancel() diff --git a/pkg/snapshot/snapshotter/types.go b/pkg/snapshot/snapshotter/types.go index 2b79426d5..0ed343cf5 100644 --- a/pkg/snapshot/snapshotter/types.go +++ b/pkg/snapshot/snapshotter/types.go @@ -19,6 +19,8 @@ import ( "sync" "time" + "github.com/gardener/etcd-backup-restore/pkg/wrappers" + "github.com/coreos/etcd/clientv3" "github.com/gardener/etcd-backup-restore/pkg/etcdutil" "github.com/gardener/etcd-backup-restore/pkg/snapstore" @@ -52,10 +54,14 @@ type State int // Snapshotter is a struct for etcd snapshot taker type Snapshotter struct { - logger *logrus.Entry + logger *logrus.Entry + etcdConnectionConfig *etcdutil.EtcdConnectionConfig + store snapstore.SnapStore + config *Config + + schedule cron.Schedule prevSnapshot *snapstore.Snapshot PrevFullSnapshot *snapstore.Snapshot - config *Config fullSnapshotReqCh chan struct{} deltaSnapshotReqCh chan struct{} fullSnapshotAckCh chan error @@ -71,17 +77,14 @@ type Snapshotter struct { lastEventRevision int64 } -// Config stores the configuration parameters for the snapshotter. +// Config holds the snapshotter config. type Config struct { - schedule cron.Schedule - store snapstore.SnapStore - maxBackups int - deltaSnapshotMemoryLimit int - deltaSnapshotInterval time.Duration - etcdConnectionTimeout time.Duration - garbageCollectionPeriod time.Duration - garbageCollectionPolicy string - tlsConfig *etcdutil.TLSConfig + FullSnapshotSchedule string `json:"schedule,omitempty"` + DeltaSnapshotPeriod wrappers.Duration `json:"deltaSnapshotPeriod,omitempty"` + DeltaSnapshotMemoryLimit uint `json:"deltaSnapshotMemoryLimit,omitempty"` + GarbageCollectionPeriod wrappers.Duration `json:"garbageCollectionPeriod,omitempty"` + GarbageCollectionPolicy string `json:"garbageCollectionPolicy,omitempty"` + MaxBackups uint `json:"maxBackups,omitempty"` } // event is wrapper over etcd event to keep track of time of event diff --git a/pkg/snapstore/abs_snapstore.go b/pkg/snapstore/abs_snapstore.go index f63782280..858200f3e 100644 --- a/pkg/snapstore/abs_snapstore.go +++ b/pkg/snapstore/abs_snapstore.go @@ -40,12 +40,12 @@ type ABSSnapStore struct { containerURL *azblob.ContainerURL prefix string // maxParallelChunkUploads hold the maximum number of parallel chunk uploads allowed. - maxParallelChunkUploads int + maxParallelChunkUploads uint tempDir string } // NewABSSnapStore create new ABSSnapStore from shared configuration with specified bucket -func NewABSSnapStore(container, prefix, tempDir string, maxParallelChunkUploads int) (*ABSSnapStore, error) { +func NewABSSnapStore(container, prefix, tempDir string, maxParallelChunkUploads uint) (*ABSSnapStore, error) { storageAccount, err := GetEnvVarOrError(absStorageAccount) if err != nil { return nil, err @@ -74,7 +74,7 @@ func NewABSSnapStore(container, prefix, tempDir string, maxParallelChunkUploads } // GetABSSnapstoreFromClient returns a new ABS object for a given container using the supplied storageClient -func GetABSSnapstoreFromClient(container, prefix, tempDir string, maxParallelChunkUploads int, containerURL *azblob.ContainerURL) (*ABSSnapStore, error) { +func GetABSSnapstoreFromClient(container, prefix, tempDir string, maxParallelChunkUploads uint, containerURL *azblob.ContainerURL) (*ABSSnapStore, error) { // Check if supplied container exists ctx, cancel := context.WithTimeout(context.TODO(), providerConnectionTimeout) defer cancel() @@ -168,7 +168,7 @@ func (a *ABSSnapStore) Save(snap Snapshot, rc io.ReadCloser) error { cancelCh = make(chan struct{}) ) - for i := 0; i < a.maxParallelChunkUploads; i++ { + for i := uint(0); i < a.maxParallelChunkUploads; i++ { wg.Add(1) go a.blockUploader(&wg, cancelCh, &snap, tmpfile, chunkUploadCh, resCh) } diff --git a/pkg/snapstore/gcs_snapstore.go b/pkg/snapstore/gcs_snapstore.go index 9f85ddb9e..0df7e1346 100644 --- a/pkg/snapstore/gcs_snapstore.go +++ b/pkg/snapstore/gcs_snapstore.go @@ -38,7 +38,7 @@ type GCSSnapStore struct { prefix string bucket string // maxParallelChunkUploads hold the maximum number of parallel chunk uploads allowed. - maxParallelChunkUploads int + maxParallelChunkUploads uint tempDir string } @@ -47,7 +47,7 @@ const ( ) // NewGCSSnapStore create new GCSSnapStore from shared configuration with specified bucket. -func NewGCSSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads int) (*GCSSnapStore, error) { +func NewGCSSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads uint) (*GCSSnapStore, error) { ctx := context.TODO() cli, err := storage.NewClient(ctx) if err != nil { @@ -59,7 +59,7 @@ func NewGCSSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads int } // NewGCSSnapStoreFromClient create new GCSSnapStore from shared configuration with specified bucket. -func NewGCSSnapStoreFromClient(bucket, prefix, tempDir string, maxParallelChunkUploads int, cli stiface.Client) *GCSSnapStore { +func NewGCSSnapStoreFromClient(bucket, prefix, tempDir string, maxParallelChunkUploads uint, cli stiface.Client) *GCSSnapStore { return &GCSSnapStore{ prefix: prefix, client: cli, @@ -108,7 +108,7 @@ func (s *GCSSnapStore) Save(snap Snapshot, rc io.ReadCloser) error { cancelCh = make(chan struct{}) ) - for i := 0; i < s.maxParallelChunkUploads; i++ { + for i := uint(0); i < s.maxParallelChunkUploads; i++ { wg.Add(1) go s.componentUploader(&wg, cancelCh, &snap, tmpfile, chunkUploadCh, resCh) } diff --git a/pkg/snapstore/init.go b/pkg/snapstore/init.go new file mode 100644 index 000000000..314e7837a --- /dev/null +++ b/pkg/snapstore/init.go @@ -0,0 +1,56 @@ +// Copyright (c) 2019 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 snapstore + +import ( + "fmt" + "path" + + flag "github.com/spf13/pflag" +) + +const ( + backupFormatVersion = "v1" +) + +// NewSnapstoreConfig returns the snapstore config. +func NewSnapstoreConfig() *Config { + return &Config{ + MaxParallelChunkUploads: 5, + TempDir: "/tmp", + } +} + +// AddFlags adds the flags to flagset. +func (c *Config) AddFlags(fs *flag.FlagSet) { + fs.StringVar(&c.Provider, "storage-provider", c.Provider, "snapshot storage provider") + fs.StringVar(&c.Container, "store-container", c.Container, "container which will be used as snapstore") + fs.StringVar(&c.Prefix, "store-prefix", c.Prefix, "prefix or directory inside container under which snapstore is created") + fs.UintVar(&c.MaxParallelChunkUploads, "max-parallel-chunk-uploads", c.MaxParallelChunkUploads, "maximum number of parallel chunk uploads allowed ") + fs.StringVar(&c.TempDir, "snapstore-temp-directory", c.TempDir, "temporary directory for processing") +} + +// Validate validates the config. +func (c *Config) Validate() error { + if c.MaxParallelChunkUploads <= 0 { + return fmt.Errorf("max parallel chunk uploads should be greater than zero") + } + return nil +} + +// Complete completes the config. +func (c *Config) Complete() { + c.Prefix = path.Join(c.Prefix, backupFormatVersion) +} diff --git a/pkg/snapstore/oss_snapstore.go b/pkg/snapstore/oss_snapstore.go index 4390aa98c..f73117944 100644 --- a/pkg/snapstore/oss_snapstore.go +++ b/pkg/snapstore/oss_snapstore.go @@ -57,12 +57,12 @@ type OSSSnapStore struct { prefix string bucket OSSBucket multiPart sync.Mutex - maxParallelChunkUploads int + maxParallelChunkUploads uint tempDir string } // NewOSSSnapStore create new OSSSnapStore from shared configuration with specified bucket -func NewOSSSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads int) (*OSSSnapStore, error) { +func NewOSSSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads uint) (*OSSSnapStore, error) { ao, err := authOptionsFromEnv() if err != nil { return nil, err @@ -70,7 +70,7 @@ func NewOSSSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads int return newOSSFromAuthOpt(bucket, prefix, tempDir, maxParallelChunkUploads, ao) } -func newOSSFromAuthOpt(bucket, prefix, tempDir string, maxParallelChunkUploads int, ao authOptions) (*OSSSnapStore, error) { +func newOSSFromAuthOpt(bucket, prefix, tempDir string, maxParallelChunkUploads uint, ao authOptions) (*OSSSnapStore, error) { client, err := oss.New(ao.endpoint, ao.accessID, ao.accessKey) if err != nil { return nil, err @@ -85,7 +85,7 @@ func newOSSFromAuthOpt(bucket, prefix, tempDir string, maxParallelChunkUploads i } // NewOSSFromBucket will create the new OSS snapstore object from OSS bucket -func NewOSSFromBucket(prefix, tempDir string, maxParallelChunkUploads int, bucket OSSBucket) *OSSSnapStore { +func NewOSSFromBucket(prefix, tempDir string, maxParallelChunkUploads uint, bucket OSSBucket) *OSSSnapStore { return &OSSSnapStore{ prefix: prefix, bucket: bucket, @@ -151,7 +151,7 @@ func (s *OSSSnapStore) Save(snap Snapshot, rc io.ReadCloser) error { wg sync.WaitGroup ) - for i := 0; i < s.maxParallelChunkUploads; i++ { + for i := uint(0); i < s.maxParallelChunkUploads; i++ { wg.Add(1) go s.partUploader(&wg, imur, tmpfile, completedParts, chunkUploadCh, cancelCh, resCh) } diff --git a/pkg/snapstore/s3_snapstore.go b/pkg/snapstore/s3_snapstore.go index 5e4504a3a..250a4d05c 100644 --- a/pkg/snapstore/s3_snapstore.go +++ b/pkg/snapstore/s3_snapstore.go @@ -44,12 +44,12 @@ type S3SnapStore struct { bucket string multiPart sync.Mutex // maxParallelChunkUploads hold the maximum number of parallel chunk uploads allowed. - maxParallelChunkUploads int + maxParallelChunkUploads uint tempDir string } // NewS3SnapStore create new S3SnapStore from shared configuration with specified bucket -func NewS3SnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads int) (*S3SnapStore, error) { +func NewS3SnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads uint) (*S3SnapStore, error) { return newS3FromSessionOpt(bucket, prefix, tempDir, maxParallelChunkUploads, session.Options{ // Setting this is equal to the AWS_SDK_LOAD_CONFIG environment variable was set. // We want to save the work to set AWS_SDK_LOAD_CONFIG=1 outside. @@ -58,7 +58,7 @@ func NewS3SnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads int) } // newS3FromSessionOpt will create the new S3 snapstore object from S3 session options -func newS3FromSessionOpt(bucket, prefix, tempDir string, maxParallelChunkUploads int, so session.Options) (*S3SnapStore, error) { +func newS3FromSessionOpt(bucket, prefix, tempDir string, maxParallelChunkUploads uint, so session.Options) (*S3SnapStore, error) { sess, err := session.NewSessionWithOptions(so) if err != nil { return nil, fmt.Errorf("new AWS session failed: %v", err) @@ -68,7 +68,7 @@ func newS3FromSessionOpt(bucket, prefix, tempDir string, maxParallelChunkUploads } // NewS3FromClient will create the new S3 snapstore object from S3 client -func NewS3FromClient(bucket, prefix, tempDir string, maxParallelChunkUploads int, cli s3iface.S3API) *S3SnapStore { +func NewS3FromClient(bucket, prefix, tempDir string, maxParallelChunkUploads uint, cli s3iface.S3API) *S3SnapStore { return &S3SnapStore{ bucket: bucket, prefix: prefix, @@ -139,7 +139,7 @@ func (s *S3SnapStore) Save(snap Snapshot, rc io.ReadCloser) error { cancelCh = make(chan struct{}) ) - for i := 0; i < s.maxParallelChunkUploads; i++ { + for i := uint(0); i < s.maxParallelChunkUploads; i++ { wg.Add(1) go s.partUploader(&wg, cancelCh, &snap, tmpfile, uploadOutput.UploadId, completedParts, chunkUploadCh, resCh) } diff --git a/pkg/snapstore/swift_snapstore.go b/pkg/snapstore/swift_snapstore.go index e4eff8cbc..89bb444e7 100644 --- a/pkg/snapstore/swift_snapstore.go +++ b/pkg/snapstore/swift_snapstore.go @@ -41,7 +41,7 @@ type SwiftSnapStore struct { client *gophercloud.ServiceClient bucket string // maxParallelChunkUploads hold the maximum number of parallel chunk uploads allowed. - maxParallelChunkUploads int + maxParallelChunkUploads uint tempDir string } @@ -50,7 +50,7 @@ const ( ) // NewSwiftSnapStore create new SwiftSnapStore from shared configuration with specified bucket -func NewSwiftSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads int) (*SwiftSnapStore, error) { +func NewSwiftSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads uint) (*SwiftSnapStore, error) { authOpts, err := clientconfig.AuthOptions(nil) if err != nil { return nil, err @@ -75,7 +75,7 @@ func NewSwiftSnapStore(bucket, prefix, tempDir string, maxParallelChunkUploads i } // NewSwiftSnapstoreFromClient will create the new Swift snapstore object from Swift client -func NewSwiftSnapstoreFromClient(bucket, prefix, tempDir string, maxParallelChunkUploads int, cli *gophercloud.ServiceClient) *SwiftSnapStore { +func NewSwiftSnapstoreFromClient(bucket, prefix, tempDir string, maxParallelChunkUploads uint, cli *gophercloud.ServiceClient) *SwiftSnapStore { return &SwiftSnapStore{ bucket: bucket, prefix: prefix, @@ -124,7 +124,7 @@ func (s *SwiftSnapStore) Save(snap Snapshot, rc io.ReadCloser) error { cancelCh = make(chan struct{}) ) - for i := 0; i < s.maxParallelChunkUploads; i++ { + for i := uint(0); i < s.maxParallelChunkUploads; i++ { wg.Add(1) go s.chunkUploader(&wg, cancelCh, &snap, tmpfile, chunkUploadCh, resCh) } diff --git a/pkg/snapstore/types.go b/pkg/snapstore/types.go index bff6ea6fc..240bc69a6 100644 --- a/pkg/snapstore/types.go +++ b/pkg/snapstore/types.go @@ -92,15 +92,15 @@ type SnapList []*Snapshot // Config defines the configuration to create snapshot store. type Config struct { // Provider indicated the cloud provider. - Provider string + Provider string `json:"provider,omitempty"` // Container holds the name of bucket or container to which snapshot will be stored. - Container string + Container string `json:"container"` // Prefix holds the prefix or directory under StorageContainer under which snapshot will be stored. - Prefix string + Prefix string `json:"prefix,omitempty"` // MaxParallelChunkUploads hold the maximum number of parallel chunk uploads allowed. - MaxParallelChunkUploads int + MaxParallelChunkUploads uint `json:"maxParallelChunkUploads,omitempty"` // Temporary Directory - TempDir string + TempDir string `json:"tempDir,omitempty"` } type chunk struct { diff --git a/pkg/wrappers/duration.go b/pkg/wrappers/duration.go new file mode 100644 index 000000000..21d0679da --- /dev/null +++ b/pkg/wrappers/duration.go @@ -0,0 +1,46 @@ +// Copyright (c) 2018 SAP SE or an SAP affiliate company. All rights reserved. This file is licensed under the Apache Software License, v. 2 except as noted otherwise in the LICENSE file. +// +// 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 wrappers + +import ( + "encoding/json" + "time" +) + +// Duration is a wrapper around time.Duration which supports correct +// marshaling to YAML and JSON. +type Duration struct { + time.Duration +} + +// UnmarshalJSON implements the json.Unmarshaller interface. +func (d *Duration) UnmarshalJSON(b []byte) error { + var str string + if err := json.Unmarshal(b, &str); err != nil { + return err + } + + pd, err := time.ParseDuration(str) + if err != nil { + return err + } + d.Duration = pd + return nil +} + +// MarshalJSON implements the json.Marshaler interface. +func (d *Duration) MarshalJSON() ([]byte, error) { + return json.Marshal(d.Duration.String()) +}