diff --git a/cmd/backup-manager/app/backup/manager.go b/cmd/backup-manager/app/backup/manager.go index 3678f3a0bd6..8a49ca1b25f 100644 --- a/cmd/backup-manager/app/backup/manager.go +++ b/cmd/backup-manager/app/backup/manager.go @@ -198,7 +198,7 @@ func (bm *Manager) performBackup(ctx context.Context, backup *v1alpha1.Backup, d var errs []error - backupFullPath, err := util.GetStoragePath(backup) + backupFullPath, err := util.GetStoragePath(&backup.Spec.StorageProvider) if err != nil { errs = append(errs, err) uerr := bm.StatusUpdater.Update(backup, &v1alpha1.BackupCondition{ @@ -506,7 +506,7 @@ func (bm *Manager) performLogBackup(ctx context.Context, backup *v1alpha1.Backup // startLogBackup starts log backup. func (bm *Manager) startLogBackup(ctx context.Context, backup *v1alpha1.Backup) (*controller.BackupUpdateStatus, string, error) { started := time.Now() - backupFullPath, err := util.GetStoragePath(backup) + backupFullPath, err := util.GetStoragePath(&backup.Spec.StorageProvider) if err != nil { klog.Errorf("Get backup full path of cluster %s failed, err: %s", bm, err) return nil, "GetBackupRemotePathFailed", err diff --git a/cmd/backup-manager/app/clean/clean_test.go b/cmd/backup-manager/app/clean/clean_test.go index 7207565d164..c1fb71a1b55 100644 --- a/cmd/backup-manager/app/clean/clean_test.go +++ b/cmd/backup-manager/app/clean/clean_test.go @@ -23,11 +23,10 @@ import ( "github.com/agiledragon/gomonkey/v2" "github.com/aws/aws-sdk-go/service/s3/s3iface" . "github.com/onsi/gomega" - "gocloud.dev/blob" - "gocloud.dev/blob/driver" - "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" "github.com/pingcap/tidb-operator/pkg/backup/util" + "gocloud.dev/blob" + "gocloud.dev/blob/driver" ) func TestCleanBRRemoteBackupDataOnce(t *testing.T) { diff --git a/cmd/backup-manager/app/cmd/cmd.go b/cmd/backup-manager/app/cmd/cmd.go index c2a1b7a457f..24c20e10d14 100644 --- a/cmd/backup-manager/app/cmd/cmd.go +++ b/cmd/backup-manager/app/cmd/cmd.go @@ -34,6 +34,7 @@ func NewBackupMgrCommand() *cobra.Command { cmds.AddCommand(NewRestoreCommand()) cmds.AddCommand(NewImportCommand()) cmds.AddCommand(NewCleanCommand()) + cmds.AddCommand(NewCompactCommand()) return cmds } diff --git a/cmd/backup-manager/app/cmd/compact.go b/cmd/backup-manager/app/cmd/compact.go new file mode 100644 index 00000000000..f4fab17ed7f --- /dev/null +++ b/cmd/backup-manager/app/cmd/compact.go @@ -0,0 +1,70 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cmd + +import ( + "context" + + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/compact" + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/compact/options" + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/constants" + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/util" + informers "github.com/pingcap/tidb-operator/pkg/client/informers/externalversions" + "github.com/pingcap/tidb-operator/pkg/controller" + "github.com/spf13/cobra" + "k8s.io/client-go/tools/cache" + cmdutil "k8s.io/kubectl/pkg/cmd/util" +) + +func NewCompactCommand() *cobra.Command { + opts := options.CompactOpts{} + + cmd := &cobra.Command{ + Use: "compact", + Short: "Compact log backup.", + Run: func(cmd *cobra.Command, args []string) { + util.ValidCmdFlags(cmd.CommandPath(), cmd.LocalFlags()) + cmdutil.CheckErr(runCompact(opts, kubecfg)) + }, + } + + cmd.Flags().StringVar(&opts.Namespace, "namespace", "", "Backup CR's namespace") + cmd.Flags().StringVar(&opts.ResourceName, "resourceName", "", "Backup CRD object name") + return cmd +} + +func runCompact(compactOpts options.CompactOpts, kubecfg string) error { + kubeCli, cli, err := util.NewKubeAndCRCli(kubecfg) + if err != nil { + return err + } + options := []informers.SharedInformerOption{ + informers.WithNamespace(compactOpts.Namespace), + } + informerFactory := informers.NewSharedInformerFactoryWithOptions(cli, constants.ResyncDuration, options...) + recorder := util.NewEventRecorder(kubeCli, "compact-manager") + compactInformer := informerFactory.Pingcap().V1alpha1().CompactBackups() + statusUpdater := controller.NewCompactStatusUpdater(recorder, compactInformer.Lister(), cli) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go informerFactory.Start(ctx.Done()) + + // waiting for the shared informer's store has synced. + cache.WaitForCacheSync(ctx.Done(), compactInformer.Informer().HasSynced) + + // klog.Infof("start to process backup %s", compactOpts.String()) + cm := compact.NewManager(compactInformer.Lister(), statusUpdater, compactOpts) + return cm.ProcessCompact() +} diff --git a/cmd/backup-manager/app/compact/manager.go b/cmd/backup-manager/app/compact/manager.go new file mode 100644 index 00000000000..d916c11590b --- /dev/null +++ b/cmd/backup-manager/app/compact/manager.go @@ -0,0 +1,229 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package compact + +import ( + "bytes" + "context" + "encoding/json" + "io" + "os" + "os/exec" + "path/filepath" + "strconv" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/compact/options" + backuputil "github.com/pingcap/tidb-operator/cmd/backup-manager/app/util" + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + pkgutil "github.com/pingcap/tidb-operator/pkg/backup/util" + listers "github.com/pingcap/tidb-operator/pkg/client/listers/pingcap/v1alpha1" + "github.com/pingcap/tidb-operator/pkg/controller" + "github.com/pingcap/tidb-operator/pkg/util" + "k8s.io/klog/v2" +) + +const ( + messageCompactionDone = "Finishing compaction." + messageCompactAborted = "Compaction aborted." +) + +// logLine is line of JSON log. +// It just extracted the message from the JSON and keeps the origin json bytes. +// So you may extract fields from it by `json.Unmarshal(l.Raw, ...)`. +type logLine struct { + Message string `json:"Message"` + Raw json.RawMessage `json:"-"` +} + +// Manager mainly used to manage backup related work +type Manager struct { + compact *v1alpha1.CompactBackup + resourceLister listers.CompactBackupLister + statusUpdater controller.CompactStatusUpdaterInterface + options options.CompactOpts +} + +// NewManager return a Manager +func NewManager( + lister listers.CompactBackupLister, + statusUpdater controller.CompactStatusUpdaterInterface, + compactOpts options.CompactOpts) *Manager { + compact, err := lister.CompactBackups(compactOpts.Namespace).Get(compactOpts.ResourceName) + if err != nil { + klog.Errorf("can't find compact %s:%s CRD object, err: %v", compactOpts.Namespace, compactOpts.ResourceName, err) + return nil + } + return &Manager{ + compact, + lister, + statusUpdater, + compactOpts, + } +} + +func (cm *Manager) brBin() string { + return filepath.Join(util.BRBinPath, "br") +} + +func (cm *Manager) kvCtlBin() string { + return filepath.Join(util.KVCTLBinPath, "tikv-ctl") +} + +// ProcessBackup used to process the backup logic +func (cm *Manager) ProcessCompact() error { + var err error + ctx, cancel := backuputil.GetContextForTerminationSignals(cm.options.ResourceName) + defer cancel() + + compact, err := cm.resourceLister.CompactBackups(cm.options.Namespace).Get(cm.options.ResourceName) + defer func() { + cm.statusUpdater.OnFinish(ctx, cm.compact, err) + }() + if err != nil { + return errors.New("backup not found") + } + if err = options.ParseCompactOptions(compact, &cm.options); err != nil { + return errors.Annotate(err, "failed to parse compact options") + } + + b64, err := cm.base64ifyStorage(ctx) + if err != nil { + return errors.Annotate(err, "failed to base64ify storage") + } + return cm.runCompaction(ctx, b64) +} + +func (cm *Manager) base64ifyStorage(ctx context.Context) (string, error) { + brCmd, err := cm.base64ifyCmd(ctx) + if err != nil { + return "", err + } + out, err := brCmd.Output() + if err != nil { + eerr, ok := err.(*exec.ExitError) + if !ok { + return "", errors.Annotatef(err, "failed to execute BR with args %v", brCmd.Args) + } + klog.Warningf("Failed to execute base64ify; stderr = %s", string(eerr.Stderr)) + return "", errors.Annotatef(err, "failed to execute BR with args %v", brCmd.Args) + } + out = bytes.Trim(out, "\r\n \t") + return string(out), nil +} + +func (cm *Manager) base64ifyCmd(ctx context.Context) (*exec.Cmd, error) { + br := cm.brBin() + args := []string{ + "operator", + "base64ify", + } + StorageOpts, err := pkgutil.GenStorageArgsForFlag(cm.compact.Spec.StorageProvider, "storage") + if err != nil { + return nil, err + } + args = append(args, StorageOpts...) + return exec.CommandContext(ctx, br, args...), nil +} + +func (cm *Manager) runCompaction(ctx context.Context, base64Storage string) (err error) { + cmd := cm.compactCmd(ctx, base64Storage) + + // tikvLog is used to capture the log from tikv-ctl, which is sent to stderr by default + tikvLog, err := cmd.StderrPipe() + if err != nil { + return errors.Annotate(err, "failed to create stderr pipe for compact") + } + if err := cmd.Start(); err != nil { + return errors.Annotate(err, "failed to start compact") + } + + cm.statusUpdater.OnStart(ctx, cm.compact) + err = cm.processCompactionLogs(ctx, io.TeeReader(tikvLog, os.Stdout)) + if err != nil { + return err + } + + return cmd.Wait() +} + +func (cm *Manager) compactCmd(ctx context.Context, base64Storage string) *exec.Cmd { + ctl := cm.kvCtlBin() + // You should not change the log configuration here, it should sync with the upstream setting + args := []string{ + "--log-level", + "INFO", + "--log-format", + "json", + "compact-log-backup", + "--storage-base64", + base64Storage, + "--from", + strconv.FormatUint(cm.options.FromTS, 10), + "--until", + strconv.FormatUint(cm.options.UntilTS, 10), + "-N", + strconv.FormatUint(cm.options.Concurrency, 10), + } + return exec.CommandContext(ctx, ctl, args...) +} + +func (cm *Manager) processCompactionLogs(ctx context.Context, logStream io.Reader) error { + dec := json.NewDecoder(logStream) + + for dec.More() { + if ctx.Err() != nil { + return ctx.Err() + } + + var raw json.RawMessage + if err := dec.Decode(&raw); err != nil { + return errors.Annotate(err, "failed to decode raw log line") + } + + var line logLine + if err := json.Unmarshal(raw, &line); err != nil { + return errors.Annotate(err, "failed to decode the line of log") + } + line.Raw = raw + + if err := cm.processLogLine(ctx, line); err != nil { + return err + } + } + + return nil +} + +func (cm *Manager) processLogLine(ctx context.Context, l logLine) error { + switch l.Message { + case messageCompactionDone: + var prog controller.Progress + if err := json.Unmarshal(l.Raw, &prog); err != nil { + return errors.Annotatef(err, "failed to decode progress message: %s", string(l.Raw)) + } + cm.statusUpdater.OnProgress(ctx, cm.compact, prog) + return nil + case messageCompactAborted: + errContainer := struct { + Err string `json:"err"` + }{} + if err := json.Unmarshal(l.Raw, &errContainer); err != nil { + return errors.Annotatef(err, "failed to decode error message: %s", string(l.Raw)) + } + return errors.New(errContainer.Err) + default: + return nil + } +} diff --git a/cmd/backup-manager/app/compact/options/options.go b/cmd/backup-manager/app/compact/options/options.go new file mode 100644 index 00000000000..56e2afe9308 --- /dev/null +++ b/cmd/backup-manager/app/compact/options/options.go @@ -0,0 +1,76 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package options + +import ( + "math" + + "github.com/juju/errors" + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + "github.com/pingcap/tidb-operator/pkg/apis/util/config" +) + +const ( + fromTSUnset = math.MaxUint64 + untilTSUnset = 0 +) + +type CompactOpts struct { + FromTS uint64 + UntilTS uint64 + Name string + Concurrency uint64 + Namespace string `json:"namespace"` + ResourceName string `json:"resourceName"` + TikvVersion string `json:"tikvVersion"` +} + +func ParseCompactOptions(compact *v1alpha1.CompactBackup, opts *CompactOpts) error { + + startTs, err := config.ParseTSString(compact.Spec.StartTs) + if err != nil { + return errors.Annotatef(err, "failed to parse startTs %s", compact.Spec.StartTs) + } + endTs, err := config.ParseTSString(compact.Spec.EndTs) + if err != nil { + return errors.Annotatef(err, "failed to parse endTs %s", compact.Spec.EndTs) + } + opts.FromTS = startTs + opts.UntilTS = endTs + + opts.Name = compact.ObjectMeta.Name + opts.Concurrency = uint64(compact.Spec.Concurrency) + + if err := opts.Verify(); err != nil { + return err + } + + return nil +} + +func (c *CompactOpts) Verify() error { + if c.UntilTS < c.FromTS { + if c.UntilTS == untilTSUnset { + return errors.New("until-ts must be set") + } + if c.FromTS == fromTSUnset { + return errors.New("from-ts must be set") + } + return errors.Errorf("until-ts %d must be greater than from-ts %d", c.UntilTS, c.FromTS) + } + if c.Concurrency <= 0 { + return errors.Errorf("concurrency %d must be greater than 0", c.Concurrency) + } + return nil +} diff --git a/cmd/backup-manager/app/util/util.go b/cmd/backup-manager/app/util/util.go index b00b1eaa0c2..2b5b9102ae3 100644 --- a/cmd/backup-manager/app/util/util.go +++ b/cmd/backup-manager/app/util/util.go @@ -103,28 +103,28 @@ func EnsureDirectoryExist(dirName string) error { } // GetStoragePath generate the path of a specific storage -func GetStoragePath(backup *v1alpha1.Backup) (string, error) { +func GetStoragePath(StorageProvider *v1alpha1.StorageProvider) (string, error) { var url, bucket, prefix string - st := util.GetStorageType(backup.Spec.StorageProvider) + st := util.GetStorageType(*StorageProvider) switch st { case v1alpha1.BackupStorageTypeS3: - prefix = backup.Spec.StorageProvider.S3.Prefix - bucket = backup.Spec.StorageProvider.S3.Bucket + prefix = StorageProvider.S3.Prefix + bucket = StorageProvider.S3.Bucket url = fmt.Sprintf("s3://%s", path.Join(bucket, prefix)) return url, nil case v1alpha1.BackupStorageTypeGcs: - prefix = backup.Spec.StorageProvider.Gcs.Prefix - bucket = backup.Spec.StorageProvider.Gcs.Bucket + prefix = StorageProvider.Gcs.Prefix + bucket = StorageProvider.Gcs.Bucket url = fmt.Sprintf("gcs://%s/", path.Join(bucket, prefix)) return url, nil case v1alpha1.BackupStorageTypeAzblob: - prefix = backup.Spec.StorageProvider.Azblob.Prefix - bucket = backup.Spec.StorageProvider.Azblob.Container + prefix = StorageProvider.Azblob.Prefix + bucket = StorageProvider.Azblob.Container url = fmt.Sprintf("azure://%s/", path.Join(bucket, prefix)) return url, nil case v1alpha1.BackupStorageTypeLocal: - prefix = backup.Spec.StorageProvider.Local.Prefix - mountPath := backup.Spec.StorageProvider.Local.VolumeMount.MountPath + prefix = StorageProvider.Local.Prefix + mountPath := StorageProvider.Local.VolumeMount.MountPath url = fmt.Sprintf("local://%s", path.Join(mountPath, prefix)) return url, nil default: @@ -535,7 +535,9 @@ func ReadAllStdErrToChannel(stdErr io.Reader, errMsgCh chan []byte) { func GracefullyShutDownSubProcess(ctx context.Context, cmd *exec.Cmd) { <-ctx.Done() klog.Errorf("context done, err: %s. start to shut down sub process gracefully", ctx.Err().Error()) - if err := cmd.Process.Signal(syscall.SIGTERM); err != nil { + if cmd.Process == nil { + klog.Infof("sub process not started, won't send SIGTERM") + } else if err := cmd.Process.Signal(syscall.SIGTERM); err != nil { klog.Errorf("send SIGTERM to sub process error: %s", err.Error()) } else { klog.Infof("send SIGTERM to sub process successfully") diff --git a/cmd/backup-manager/app/util/util_test.go b/cmd/backup-manager/app/util/util_test.go index 0a25c93876a..0848efaf202 100644 --- a/cmd/backup-manager/app/util/util_test.go +++ b/cmd/backup-manager/app/util/util_test.go @@ -260,7 +260,7 @@ func TestGetRemotePath(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - p, err := GetStoragePath(tt.backup) + p, err := GetStoragePath(&tt.backup.Spec.StorageProvider) if tt.err { g.Expect(err).To(HaveOccurred()) return diff --git a/cmd/backup-manager/main.go b/cmd/backup-manager/main.go index 37b70321944..8a0f64f2f8d 100644 --- a/cmd/backup-manager/main.go +++ b/cmd/backup-manager/main.go @@ -16,12 +16,10 @@ package main import ( "os" - "k8s.io/klog/v2" - "github.com/pingcap/tidb-operator/cmd/backup-manager/app" - // Enable FIPS when necessary _ "github.com/pingcap/tidb-operator/pkg/fips" + "k8s.io/klog/v2" ) func main() { diff --git a/cmd/controller-manager/main.go b/cmd/controller-manager/main.go index 991de3e2eb6..283de5583c9 100644 --- a/cmd/controller-manager/main.go +++ b/cmd/controller-manager/main.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb-operator/pkg/controller/autoscaler" "github.com/pingcap/tidb-operator/pkg/controller/backup" "github.com/pingcap/tidb-operator/pkg/controller/backupschedule" + compact "github.com/pingcap/tidb-operator/pkg/controller/compactbackup" "github.com/pingcap/tidb-operator/pkg/controller/dmcluster" "github.com/pingcap/tidb-operator/pkg/controller/restore" "github.com/pingcap/tidb-operator/pkg/controller/tidbcluster" @@ -182,6 +183,7 @@ func main() { tidbcluster.NewPodController(deps), dmcluster.NewController(deps), backup.NewController(deps), + compact.NewController(deps), restore.NewController(deps), backupschedule.NewController(deps), tidbinitializer.NewController(deps), diff --git a/docs/api-references/docs.md b/docs/api-references/docs.md index 3b737983140..0b47c517f1a 100644 --- a/docs/api-references/docs.md +++ b/docs/api-references/docs.md @@ -3447,6 +3447,7 @@ string
(Appears on: BackupSpec, +CompactSpec, RestoreSpec)
@@ -5335,6 +5336,695 @@ FlashSecurity +
+
+Field | +Description | +||||||||||||||||||||||||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
+metadata
+
+
+Kubernetes meta/v1.ObjectMeta
+
+
+ |
+
+Refer to the Kubernetes API documentation for the fields of the
+metadata field.
+ |
+||||||||||||||||||||||||||||||||||||
+spec
+
+
+CompactSpec
+
+
+ |
+
+ + +
|
+||||||||||||||||||||||||||||||||||||
+status
+
+
+CompactStatus
+
+
+ |
++ | +
+(Appears on: +CompactStatus) +
++
CompactRetryRecord is the record of compact backoff retry
+ +Field | +Description | +
---|---|
+retryNum
+
+int
+
+ |
+
+ RetryNum is the number of retry + |
+
+detectFailedAt
+
+
+Kubernetes meta/v1.Time
+
+
+ |
+
+ DetectFailedAt is the time when detect failure + |
+
+retryReason
+
+string
+
+ |
+
+ Reason is the reason of retry + |
+
+(Appears on: +CompactBackup) +
++
CompactSpec contains the backup specification for a tidb cluster.
+ +Field | +Description | +
---|---|
+resources
+
+
+Kubernetes core/v1.ResourceRequirements
+
+
+ |
++ | +
+env
+
+
+[]Kubernetes core/v1.EnvVar
+
+
+ |
+
+(Optional)
+ List of environment variables to set in the container, like v1.Container.Env. +Note that the following builtin env vars will be overwritten by values set here +- S3_PROVIDER +- S3_ENDPOINT +- AWS_REGION +- AWS_ACL +- AWS_STORAGE_CLASS +- AWS_DEFAULT_REGION +- AWS_ACCESS_KEY_ID +- AWS_SECRET_ACCESS_KEY +- GCS_PROJECT_ID +- GCS_OBJECT_ACL +- GCS_BUCKET_ACL +- GCS_LOCATION +- GCS_STORAGE_CLASS +- GCS_SERVICE_ACCOUNT_JSON_KEY +- BR_LOG_TO_TERM + |
+
+StorageProvider
+
+
+StorageProvider
+
+
+ |
+
+
+(Members of StorageProvider configures where and how backups should be stored. + |
+
+startTs
+
+string
+
+ |
+
+ StartTs is the start ts of the compact backup. +Format supports TSO or datetime, e.g. ‘400036290571534337’, ‘2018-05-11 01:42:23’. + |
+
+endTs
+
+string
+
+ |
+
+(Optional)
+ EndTs is the end ts of the compact backup. +Format supports TSO or datetime, e.g. ‘400036290571534337’, ‘2018-05-11 01:42:23’. +Default is current timestamp. + |
+
+concurrency
+
+int
+
+ |
+
+ Concurrency is the concurrency of compact backup job + |
+
+tolerations
+
+
+[]Kubernetes core/v1.Toleration
+
+
+ |
+
+(Optional)
+ Base tolerations of backup Pods, components may add more tolerations upon this respectively + |
+
+toolImage
+
+string
+
+ |
+
+(Optional)
+ BrImage specifies the br image used in compact |
+
+br
+
+
+BRConfig
+
+
+ |
+
+ BRConfig is the configs for BR + |
+
+imagePullSecrets
+
+
+[]Kubernetes core/v1.LocalObjectReference
+
+
+ |
+
+(Optional)
+ ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images. + |
+
+affinity
+
+
+Kubernetes core/v1.Affinity
+
+
+ |
+
+(Optional)
+ Affinity of backup Pods + |
+
+useKMS
+
+bool
+
+ |
+
+ Use KMS to decrypt the secrets + |
+
+serviceAccount
+
+string
+
+ |
+
+ Specify service account of backup + |
+
+podSecurityContext
+
+
+Kubernetes core/v1.PodSecurityContext
+
+
+ |
+
+(Optional)
+ PodSecurityContext of the component + |
+
+priorityClassName
+
+string
+
+ |
+
+ PriorityClassName of Backup Job Pods + |
+
+maxRetryTimes
+
+int32
+
+ |
+
+ BackoffRetryPolicy the backoff retry policy, currently only valid for snapshot backup + |
+
+additionalVolumes
+
+
+[]Kubernetes core/v1.Volume
+
+
+ |
+
+(Optional)
+ Additional volumes of component pod. + |
+
+additionalVolumeMounts
+
+
+[]Kubernetes core/v1.VolumeMount
+
+
+ |
+
+(Optional)
+ Additional volume mounts of component pod. + |
+
+(Appears on: +CompactBackup) +
++
+Field | +Description | +
---|---|
+state
+
+string
+
+ |
+
+ State is the current state of the backup + |
+
+progress
+
+string
+
+ |
+
+ Progress is the progress of the backup + |
+
+message
+
+string
+
+ |
+
+ Message is the error message of the backup + |
+
+backoffRetryStatus
+
+
+[]CompactRetryRecord
+
+
+ |
+
+ RetryStatus is status of the backoff retry, it will be used when backup pod or job exited unexpectedly + |
+
ComponentAccessor is the interface to access component details, which respects the cluster-level properties @@ -15882,6 +16572,7 @@ More info: BackupSpec, +CompactSpec, RestoreSpec)
diff --git a/go.mod b/go.mod index 123156cbca4..56991247f27 100644 --- a/go.mod +++ b/go.mod @@ -155,6 +155,7 @@ require ( github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect + github.com/juju/errors v1.0.0 github.com/klauspost/compress v1.17.4 // indirect github.com/klauspost/cpuid v1.2.3 // indirect github.com/klauspost/pgzip v1.2.5 // indirect @@ -227,6 +228,7 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect + k8s.io/klog v1.0.0 k8s.io/kms v0.28.14 // indirect k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9 // indirect sigs.k8s.io/apiserver-network-proxy/konnectivity-client v0.1.2 // indirect diff --git a/go.sum b/go.sum index 0956cbbd9c5..8f8b46c8252 100644 --- a/go.sum +++ b/go.sum @@ -515,6 +515,8 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/juju/errors v1.0.0 h1:yiq7kjCLll1BiaRuNY53MGI0+EQ3rF6GB+wvboZDefM= +github.com/juju/errors v1.0.0/go.mod h1:B5x9thDqx0wIMH3+aLIMP9HjItInYWObRovoCFM5Qe8= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/keybase/go-keychain v0.0.0-20231219164618-57a3676c3af6 h1:IsMZxCuZqKuao2vNdfD82fjjgPLfyHLpR41Z88viRWs= github.com/keybase/go-keychain v0.0.0-20231219164618-57a3676c3af6/go.mod h1:3VeWNIJaW+O5xpRQbPp0Ybqu1vJd/pm7s2F473HRrkw= @@ -1201,6 +1203,8 @@ k8s.io/component-base v0.28.14 h1:sJowHyRY166hBfBQ4cOKjkSvUo4bUdeuePtEOQfSNRY= k8s.io/component-base v0.28.14/go.mod h1:DgYlfHNvP1yeBb4L+UIzMsWNtOl0yqTk+4dGGc79H0w= k8s.io/gengo v0.0.0-20200413195148-3a45101e95ac/go.mod h1:ezvh/TsK7cY6rbqRK0oQQ8IAqLxYwwyPxAX1Pzy0ii0= k8s.io/gengo v0.0.0-20200428234225-8167cfdcfc14/go.mod h1:ezvh/TsK7cY6rbqRK0oQQ8IAqLxYwwyPxAX1Pzy0ii0= +k8s.io/klog v1.0.0 h1:Pt+yjF5aB1xDSVbau4VsWe+dQNzA0qv1LlXdC2dF6Q8= +k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= k8s.io/klog/v2 v2.0.0/go.mod h1:PBfzABfn139FHAV07az/IF9Wp1bkk3vpT2XSJ76fSDE= k8s.io/klog/v2 v2.2.0/go.mod h1:Od+F08eJP+W3HUb4pSrPpgp9DGU4GzlpG/TmITuYh/Y= k8s.io/klog/v2 v2.110.1 h1:U/Af64HJf7FcwMcXyKm2RPM22WZzyR7OSpYj5tg3cL0= diff --git a/images/tidb-backup-manager/e2e-entrypoint.sh b/images/tidb-backup-manager/e2e-entrypoint.sh index c239805e20c..1a9f1fb5fcb 100755 --- a/images/tidb-backup-manager/e2e-entrypoint.sh +++ b/images/tidb-backup-manager/e2e-entrypoint.sh @@ -93,6 +93,11 @@ case "$1" in echo "$BACKUP_BIN $E2E_ARGS clean $@" exec $BACKUP_BIN $E2E_ARGS clean "$@" ;; + compact) + shift 1 + echo "$BACKUP_BIN $E2E_ARGS compact $@" + exec $BACKUP_BIN $E2E_ARGS compact "$@" + ;; *) echo "Usage: $0 {backup|restore|clean}" echo "Now runs your command." diff --git a/images/tidb-backup-manager/entrypoint.sh b/images/tidb-backup-manager/entrypoint.sh index dbc87227840..6804abdc288 100755 --- a/images/tidb-backup-manager/entrypoint.sh +++ b/images/tidb-backup-manager/entrypoint.sh @@ -105,6 +105,11 @@ case "$1" in echo "$BACKUP_BIN import $@" $EXEC_COMMAND $BACKUP_BIN import "$@" ;; + compact) + shift 1 + echo "$BACKUP_BIN compact $@" + $EXEC_COMMAND $BACKUP_BIN compact "$@" + ;; clean) shift 1 echo "$BACKUP_BIN clean $@" @@ -113,7 +118,7 @@ case "$1" in $EXEC_COMMAND $BACKUP_BIN clean "$@" ;; *) - echo "Usage: $0 {backup|restore|clean}" + echo "Usage: $0 {backup|restore|import|compact|clean}" echo "Now runs your command." echo "$@" diff --git a/manifests/backup/backup-rbac.yaml b/manifests/backup/backup-rbac.yaml index 9b59576c630..efd7722a0d9 100644 --- a/manifests/backup/backup-rbac.yaml +++ b/manifests/backup/backup-rbac.yaml @@ -10,7 +10,7 @@ rules: resources: ["events"] verbs: ["*"] - apiGroups: ["pingcap.com"] - resources: ["backups", "restores"] + resources: ["backups", "restores", "compactbackups"] verbs: ["get", "watch", "list", "update"] --- diff --git a/manifests/crd.yaml b/manifests/crd.yaml index dbead2f6775..1637c9155ec 100644 --- a/manifests/crd.yaml +++ b/manifests/crd.yaml @@ -4,68 +4,41 @@ kind: CustomResourceDefinition metadata: annotations: controller-gen.kubebuilder.io/version: v0.15.0 - name: backups.pingcap.com + name: backupschedules.pingcap.com spec: group: pingcap.com names: - kind: Backup - listKind: BackupList - plural: backups + kind: BackupSchedule + listKind: BackupScheduleList + plural: backupschedules shortNames: - - bk - singular: backup + - bks + singular: backupschedule scope: Namespaced versions: - additionalPrinterColumns: - - description: the type of backup, such as full, db, table. Only used when Mode - = snapshot. - jsonPath: .spec.backupType - name: Type - type: string - - description: the mode of backup, such as snapshot, log. - jsonPath: .spec.backupMode - name: Mode - type: string - - description: The current status of the backup - jsonPath: .status.phase - name: Status - type: string - - description: The full path of backup data - jsonPath: .status.backupPath - name: BackupPath - type: string - - description: The data size of the backup - jsonPath: .status.backupSizeReadable - name: BackupSize - type: string - - description: The real size of volume snapshot backup, only valid to volume snapshot - backup - jsonPath: .status.incrementalBackupSizeReadable - name: IncrementalBackupSize - priority: 10 - type: string - - description: The commit ts of the backup - jsonPath: .status.commitTs - name: CommitTS + - description: The cron format string used for backup scheduling + jsonPath: .spec.schedule + name: Schedule type: string - - description: The log backup truncate until ts - jsonPath: .status.logSuccessTruncateUntil - name: LogTruncateUntil + - description: The max number of backups we want to keep + jsonPath: .spec.maxBackups + name: MaxBackups + type: integer + - description: How long backups we want to keep + jsonPath: .spec.maxReservedTime + name: MaxReservedTime type: string - - description: The time at which the backup was started - jsonPath: .status.timeStarted - name: Started + - description: The last backup CR name + jsonPath: .status.lastBackup + name: LastBackup priority: 1 - type: date - - description: The time at which the backup was completed - jsonPath: .status.timeCompleted - name: Completed + type: string + - description: The last time the backup was successfully created + jsonPath: .status.lastBackupTime + name: LastBackupTime priority: 1 type: date - - description: The time that the backup takes - jsonPath: .status.timeTaken - name: TimeTaken - type: string - jsonPath: .metadata.creationTimestamp name: Age type: date @@ -81,841 +54,747 @@ spec: type: object spec: properties: - additionalVolumeMounts: - items: - properties: - mountPath: - type: string - mountPropagation: - type: string - name: - type: string - readOnly: - type: boolean - subPath: - type: string - subPathExpr: - type: string - required: - - mountPath - - name - type: object - type: array - additionalVolumes: - items: - properties: - awsElasticBlockStore: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - azureDisk: + backupTemplate: + properties: + additionalVolumeMounts: + items: properties: - cachingMode: - type: string - diskName: - type: string - diskURI: + mountPath: type: string - fsType: + mountPropagation: type: string - kind: + name: type: string readOnly: type: boolean - required: - - diskName - - diskURI - type: object - azureFile: - properties: - readOnly: - type: boolean - secretName: + subPath: type: string - shareName: + subPathExpr: type: string required: - - secretName - - shareName + - mountPath + - name type: object - cephfs: + type: array + additionalVolumes: + items: properties: - monitors: - items: - type: string - type: array - path: - type: string - readOnly: - type: boolean - secretFile: - type: string - secretRef: + awsElasticBlockStore: properties: - name: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: type: string + required: + - volumeID type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - monitors - type: object - cinder: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: + azureDisk: properties: - name: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: type: string + readOnly: + type: boolean + required: + - diskName + - diskURI type: object - x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - csi: - properties: - driver: - type: string - fsType: - type: string - nodePublishSecretRef: + azureFile: properties: - name: + readOnly: + type: boolean + secretName: type: string + shareName: + type: string + required: + - secretName + - shareName type: object - x-kubernetes-map-type: atomic - readOnly: - type: boolean - volumeAttributes: - additionalProperties: - type: string - type: object - required: - - driver - type: object - downwardAPI: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - fieldRef: + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - apiVersion: + key: type: string - fieldPath: + mode: + format: int32 + type: integer + path: type: string required: - - fieldPath + - key + - path type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: type: string - resourceFieldRef: + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic required: - - resource + - path type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: - properties: - volumeClaimTemplate: + type: array + type: object + emptyDir: properties: - metadata: - type: object - spec: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: properties: - accessModes: - items: - type: string - type: array - dataSource: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - required: - - kind - - name - type: object - x-kubernetes-map-type: atomic - dataSourceRef: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - namespace: - type: string - required: - - kind - - name + metadata: type: object - resources: + spec: properties: - claims: + accessModes: items: - properties: - name: - type: string - required: - - name - type: object + type: string type: array - x-kubernetes-list-map-keys: + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true type: object - type: object - selector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string + type: object type: object + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string type: object - x-kubernetes-map-type: atomic - storageClassName: - type: string - volumeMode: - type: string - volumeName: - type: string + required: + - spec type: object - required: - - spec - type: object - type: object - fc: - properties: - fsType: - type: string - lun: - format: int32 - type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: - items: - type: string - type: array - type: object - flexVolume: - properties: - driver: - type: string - fsType: - type: string - options: - additionalProperties: - type: string type: object - readOnly: - type: boolean - secretRef: + fc: properties: - name: + fsType: type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array type: object - x-kubernetes-map-type: atomic - required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: - type: string - type: object - gcePersistentDisk: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - pdName: - type: string - readOnly: - type: boolean - required: - - pdName - type: object - gitRepo: - properties: - directory: - type: string - repository: - type: string - revision: - type: string - required: - - repository - type: object - glusterfs: - properties: - endpoints: - type: string - path: - type: string - readOnly: - type: boolean - required: - - endpoints - - path - type: object - hostPath: - properties: - path: - type: string - type: - type: string - required: - - path - type: object - iscsi: - properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: - type: string - lun: - format: int32 - type: integer - portals: - items: - type: string - type: array - readOnly: - type: boolean - secretRef: + flexVolume: properties: - name: + driver: type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + required: + - driver type: object - x-kubernetes-map-type: atomic - targetPortal: - type: string - required: - - iqn - - lun - - targetPortal - type: object - name: - type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName - type: object - photonPersistentDisk: - properties: - fsType: - type: string - pdID: - type: string - required: - - pdID - type: object - portworxVolume: - properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: type: string - required: - - volumeID - type: object - projected: - properties: - defaultMode: - format: int32 - type: integer - sources: - items: - properties: - configMap: - properties: - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - downwardAPI: + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: properties: - items: - items: - properties: - fieldRef: + configMap: + properties: + items: + items: properties: - apiVersion: + key: type: string - fieldPath: + mode: + format: int32 + type: integer + path: type: string required: - - fieldPath + - key + - path type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: - type: string - resourceFieldRef: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic required: - - resource + - path type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - secret: - properties: - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object type: object - x-kubernetes-map-type: atomic - serviceAccountToken: + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - audience: + key: type: string - expirationSeconds: - format: int64 + mode: + format: int32 type: integer path: type: string required: + - key - path type: object - type: object - type: array - type: object - quobyte: - properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: - type: string - required: - - registry - - volume - type: object - rbd: - properties: - fsType: - type: string - image: - type: string - keyring: - type: string - monitors: - items: - type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: properties: - name: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: type: string type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - image - - monitors - type: object - scaleIO: - properties: - fsType: - type: string - gateway: - type: string - protectionDomain: - type: string - readOnly: - type: boolean - secretRef: + vsphereVolume: properties: - name: + fsType: type: string - type: object - x-kubernetes-map-type: atomic - sslEnabled: - type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string - required: - - gateway - - secretRef - - system - type: object - secret: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - optional: - type: boolean - secretName: - type: string - type: object - storageos: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: + storagePolicyID: type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath type: object - x-kubernetes-map-type: atomic - volumeName: - type: string - volumeNamespace: - type: string - type: object - vsphereVolume: - properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: - type: string - volumePath: - type: string required: - - volumePath + - name type: object - required: - - name - type: object - type: array - affinity: - properties: - nodeAffinity: + type: array + affinity: properties: - preferredDuringSchedulingIgnoredDuringExecution: - items: - properties: - preference: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchFields: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - type: object - x-kubernetes-map-type: atomic - weight: - format: int32 - type: integer - required: - - preference - - weight - type: object - type: array - requiredDuringSchedulingIgnoredDuringExecution: + nodeAffinity: properties: - nodeSelectorTerms: + preferredDuringSchedulingIgnoredDuringExecution: items: properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchFields: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - type: object - x-kubernetes-map-type: atomic - type: array - required: - - nodeSelectorTerms - type: object - x-kubernetes-map-type: atomic - type: object - podAffinity: - properties: - preferredDuringSchedulingIgnoredDuringExecution: - items: - properties: - podAffinityTerm: - properties: - labelSelector: + preference: properties: matchExpressions: items: @@ -933,15 +812,7 @@ spec: - operator type: object type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - namespaceSelector: - properties: - matchExpressions: + matchFields: items: properties: key: @@ -957,97 +828,135 @@ spec: - operator type: object type: array - matchLabels: - additionalProperties: - type: string - type: object type: object x-kubernetes-map-type: atomic - namespaces: - items: - type: string - type: array - topologyKey: - type: string + weight: + format: int32 + type: integer required: - - topologyKey + - preference + - weight type: object - weight: - format: int32 - type: integer - required: - - podAffinityTerm - - weight - type: object - type: array - requiredDuringSchedulingIgnoredDuringExecution: - items: - properties: - labelSelector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array type: object - type: object - x-kubernetes-map-type: atomic - namespaceSelector: + x-kubernetes-map-type: atomic + type: array + required: + - nodeSelectorTerms + type: object + x-kubernetes-map-type: atomic + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: properties: - matchExpressions: - items: - properties: - key: - type: string - operator: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string + type: array + topologyKey: + type: string + required: + - topologyKey type: object + weight: + format: int32 + type: integer + required: + - podAffinityTerm + - weight type: object - x-kubernetes-map-type: atomic - namespaces: - items: - type: string - type: array - topologyKey: - type: string - required: - - topologyKey - type: object - type: array - type: object - podAntiAffinity: - properties: - preferredDuringSchedulingIgnoredDuringExecution: - items: - properties: - podAffinityTerm: + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: properties: labelSelector: properties: @@ -1106,1418 +1015,1297 @@ spec: required: - topologyKey type: object - weight: - format: int32 - type: integer - required: - - podAffinityTerm - - weight - type: object - type: array - requiredDuringSchedulingIgnoredDuringExecution: - items: - properties: - labelSelector: + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: properties: - matchExpressions: - items: - properties: - key: - type: string - operator: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string + type: array + topologyKey: + type: string + required: + - topologyKey type: object - type: object - x-kubernetes-map-type: atomic - namespaceSelector: + weight: + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: properties: - matchExpressions: - items: - properties: - key: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: type: string - operator: + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string + type: object type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey type: object - x-kubernetes-map-type: atomic - namespaces: - items: - type: string - type: array - topologyKey: - type: string - required: - - topologyKey - type: object - type: array + type: array + type: object type: object - type: object - azblob: - properties: - accessTier: - type: string - container: - type: string - path: - type: string - prefix: - type: string - sasToken: - type: string - secretName: - type: string - storageAccount: - type: string - type: object - backoffRetryPolicy: - properties: - maxRetryTimes: - default: 2 - type: integer - minRetryDuration: - default: 300s - type: string - retryTimeout: - default: 30m + azblob: + properties: + accessTier: + type: string + container: + type: string + path: + type: string + prefix: + type: string + sasToken: + type: string + secretName: + type: string + storageAccount: + type: string + type: object + backoffRetryPolicy: + properties: + maxRetryTimes: + default: 2 + type: integer + minRetryDuration: + default: 300s + type: string + retryTimeout: + default: 30m + type: string + type: object + backupMode: + default: snapshot type: string - type: object - backupMode: - default: snapshot - type: string - backupType: - type: string - br: - properties: - checkRequirements: - type: boolean - checksum: - type: boolean - cluster: + backupType: type: string - clusterNamespace: + br: + properties: + checkRequirements: + type: boolean + checksum: + type: boolean + cluster: + type: string + clusterNamespace: + type: string + concurrency: + format: int32 + type: integer + db: + type: string + logLevel: + type: string + onLine: + type: boolean + options: + items: + type: string + type: array + rateLimit: + type: integer + sendCredToTikv: + type: boolean + statusAddr: + type: string + table: + type: string + timeAgo: + type: string + required: + - cluster + type: object + calcSizeLevel: + default: all type: string - concurrency: - format: int32 - type: integer - db: + cleanOption: + properties: + backoffEnabled: + type: boolean + batchConcurrency: + format: int32 + type: integer + disableBatchConcurrency: + type: boolean + pageSize: + format: int64 + type: integer + retryCount: + default: 5 + type: integer + routineConcurrency: + format: int32 + type: integer + snapshotsDeleteRatio: + default: 1 + type: number + type: object + cleanPolicy: + default: Retain + enum: + - Retain + - OnFailure + - Delete type: string - logLevel: + commitTs: type: string - onLine: - type: boolean - options: + dumpling: + properties: + options: + items: + type: string + type: array + tableFilter: + items: + type: string + type: array + type: object + env: items: - type: string - type: array - rateLimit: - type: integer - sendCredToTikv: - type: boolean - statusAddr: - type: string - table: - type: string - timeAgo: - type: string - required: - - cluster - type: object - calcSizeLevel: - default: all - type: string - cleanOption: - properties: - backoffEnabled: - type: boolean - batchConcurrency: - format: int32 - type: integer - disableBatchConcurrency: - type: boolean - pageSize: - format: int64 - type: integer - retryCount: - default: 5 - type: integer - routineConcurrency: - format: int32 - type: integer - snapshotsDeleteRatio: - default: 1 - type: number - type: object - cleanPolicy: - default: Retain - enum: - - Retain - - OnFailure - - Delete - type: string - commitTs: - type: string - dumpling: - properties: - options: - items: - type: string - type: array - tableFilter: - items: - type: string - type: array - type: object - env: - items: - properties: - name: - type: string - value: - type: string - valueFrom: properties: - configMapKeyRef: - properties: - key: - type: string - name: - type: string - optional: - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic - secretKeyRef: + name: + type: string + value: + type: string + valueFrom: properties: - key: - type: string - name: - type: string - optional: - type: boolean - required: - - key + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic type: object - x-kubernetes-map-type: atomic + required: + - name type: object - required: - - name - type: object - type: array - federalVolumeBackupPhase: - type: string - from: - properties: - host: - type: string - port: - format: int32 - type: integer - secretName: - type: string - tlsClientSecretName: - type: string - user: - type: string - required: - - host - - secretName - type: object - gcs: - properties: - bucket: - type: string - bucketAcl: - type: string - location: - type: string - objectAcl: - type: string - path: - type: string - prefix: - type: string - projectId: - type: string - secretName: - type: string - storageClass: - type: string - required: - - projectId - type: object - imagePullSecrets: - items: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - type: array - local: - properties: - prefix: + type: array + federalVolumeBackupPhase: type: string - volume: + from: properties: - awsElasticBlockStore: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - azureDisk: - properties: - cachingMode: - type: string - diskName: - type: string - diskURI: - type: string - fsType: - type: string - kind: - type: string - readOnly: - type: boolean - required: - - diskName - - diskURI - type: object - azureFile: - properties: - readOnly: - type: boolean - secretName: - type: string - shareName: - type: string - required: - - secretName - - shareName - type: object - cephfs: + host: + type: string + port: + format: int32 + type: integer + secretName: + type: string + tlsClientSecretName: + type: string + user: + type: string + required: + - host + - secretName + type: object + gcs: + properties: + bucket: + type: string + bucketAcl: + type: string + location: + type: string + objectAcl: + type: string + path: + type: string + prefix: + type: string + projectId: + type: string + secretName: + type: string + storageClass: + type: string + required: + - projectId + type: object + imagePullSecrets: + items: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + type: array + local: + properties: + prefix: + type: string + volume: properties: - monitors: - items: - type: string - type: array - path: - type: string - readOnly: - type: boolean - secretFile: - type: string - secretRef: + awsElasticBlockStore: properties: - name: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: type: string + required: + - volumeID type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - monitors - type: object - cinder: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: + azureDisk: properties: - name: + cachingMode: type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI type: object - x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - csi: - properties: - driver: - type: string - fsType: - type: string - nodePublishSecretRef: + azureFile: properties: - name: + readOnly: + type: boolean + secretName: + type: string + shareName: type: string + required: + - secretName + - shareName type: object - x-kubernetes-map-type: atomic - readOnly: - type: boolean - volumeAttributes: - additionalProperties: - type: string + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors type: object - required: - - driver - type: object - downwardAPI: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - fieldRef: + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - apiVersion: + key: type: string - fieldPath: + mode: + format: int32 + type: integer + path: type: string required: - - fieldPath + - key + - path type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: - type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: - properties: - volumeClaimTemplate: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: properties: - metadata: - type: object - spec: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: properties: - accessModes: - items: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string - type: array - dataSource: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - required: - - kind - - name - type: object - x-kubernetes-map-type: atomic - dataSourceRef: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - namespace: - type: string - required: - - kind - - name - type: object - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: + resourceFieldRef: + properties: + containerName: + type: string + divisor: anyOf: - type: integer - type: string pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ x-kubernetes-int-or-string: true - type: object - type: object - selector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: + resource: type: string - type: object - type: object - x-kubernetes-map-type: atomic - storageClassName: - type: string - volumeMode: - type: string - volumeName: - type: string - type: object - required: - - spec - type: object - type: object - fc: - properties: - fsType: - type: string - lun: - format: int32 - type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: - items: - type: string - type: array - type: object - flexVolume: - properties: - driver: - type: string - fsType: - type: string - options: - additionalProperties: - type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array type: object - readOnly: - type: boolean - secretRef: + emptyDir: properties: - name: + medium: type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true type: object - x-kubernetes-map-type: atomic - required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: - type: string - type: object - gcePersistentDisk: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - pdName: - type: string - readOnly: - type: boolean - required: - - pdName - type: object - gitRepo: - properties: - directory: - type: string - repository: - type: string - revision: - type: string - required: - - repository - type: object - glusterfs: - properties: - endpoints: - type: string - path: - type: string - readOnly: - type: boolean - required: - - endpoints - - path - type: object - hostPath: - properties: - path: - type: string - type: - type: string - required: - - path - type: object - iscsi: - properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: - type: string - lun: - format: int32 - type: integer - portals: - items: - type: string - type: array - readOnly: - type: boolean - secretRef: + ephemeral: properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - targetPortal: - type: string - required: - - iqn - - lun - - targetPortal - type: object - name: - type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName - type: object - photonPersistentDisk: - properties: - fsType: - type: string - pdID: - type: string - required: - - pdID - type: object - portworxVolume: - properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - projected: - properties: - defaultMode: - format: int32 - type: integer - sources: - items: - properties: - configMap: - properties: - items: - items: + volumeClaimTemplate: + properties: + metadata: + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: properties: - key: + apiGroup: type: string - mode: - format: int32 - type: integer - path: + kind: type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - downwardAPI: - properties: - items: - items: - properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + name: type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic required: - - path + - kind + - name type: object - type: array - type: object - secret: - properties: - items: - items: + x-kubernetes-map-type: atomic + dataSourceRef: properties: - key: + apiGroup: type: string - mode: - format: int32 - type: integer - path: + kind: + type: string + name: + type: string + namespace: type: string required: - - key - - path + - kind + - name type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - serviceAccountToken: - properties: - audience: - type: string - expirationSeconds: - format: int64 - type: integer - path: - type: string - required: - - path - type: object - type: object - type: array - type: object - quobyte: - properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: - type: string - required: - - registry - - volume - type: object - rbd: - properties: - fsType: - type: string - image: - type: string - keyring: - type: string - monitors: - items: - type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + required: + - spec + type: object + type: object + fc: properties: - name: + fsType: type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - image - - monitors - type: object - scaleIO: - properties: - fsType: - type: string - gateway: - type: string - protectionDomain: - type: string - readOnly: - type: boolean - secretRef: + flexVolume: properties: - name: + driver: type: string - type: object - x-kubernetes-map-type: atomic - sslEnabled: - type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string - required: - - gateway - - secretRef - - system - type: object - secret: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: + fsType: + type: string + options: + additionalProperties: type: string - required: - - key - - path - type: object - type: array - optional: - type: boolean - secretName: - type: string - type: object - storageos: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: properties: - name: + datasetName: + type: string + datasetUUID: type: string type: object - x-kubernetes-map-type: atomic - volumeName: - type: string - volumeNamespace: - type: string - type: object - vsphereVolume: - properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: - type: string - volumePath: - type: string - required: - - volumePath - type: object - required: - - name - type: object - volumeMount: - properties: - mountPath: - type: string - mountPropagation: - type: string - name: - type: string - readOnly: - type: boolean - subPath: - type: string - subPathExpr: - type: string - required: - - mountPath - - name - type: object - required: - - volume - - volumeMount - type: object - logStop: - type: boolean - logSubcommand: - enum: - - log-start - - log-stop - - log-pause - type: string - logTruncateUntil: - type: string - podSecurityContext: - properties: - fsGroup: - format: int64 - type: integer - fsGroupChangePolicy: - type: string - runAsGroup: - format: int64 - type: integer - runAsNonRoot: - type: boolean - runAsUser: - format: int64 - type: integer - seLinuxOptions: - properties: - level: - type: string - role: - type: string - type: - type: string - user: - type: string - type: object - seccompProfile: - properties: - localhostProfile: - type: string - type: - type: string - required: - - type - type: object - supplementalGroups: - items: - format: int64 - type: integer - type: array - sysctls: - items: - properties: - name: - type: string - value: - type: string - required: - - name - - value - type: object - type: array - windowsOptions: - properties: - gmsaCredentialSpec: - type: string - gmsaCredentialSpecName: - type: string - hostProcess: - type: boolean - runAsUserName: - type: string - type: object - type: object - priorityClassName: - type: string - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - type: object - resumeGcSchedule: - type: boolean - s3: - properties: - acl: - type: string - bucket: - type: string - endpoint: - type: string - options: - items: - type: string - type: array - path: - type: string - prefix: - type: string - provider: - type: string - region: - type: string - secretName: - type: string - sse: - type: string - storageClass: - type: string - required: - - provider - type: object - serviceAccount: - type: string - storageClassName: - type: string - storageSize: - type: string - tableFilter: - items: - type: string - type: array - tikvGCLifeTime: - type: string - tolerations: - items: - properties: - effect: - type: string - key: - type: string - operator: - type: string - tolerationSeconds: - format: int64 - type: integer - value: - type: string - type: object - type: array - toolImage: - type: string - useKMS: - type: boolean - volumeBackupInitJobMaxActiveSeconds: - default: 600 - type: integer - type: object - x-kubernetes-validations: - - message: Field `logStop` is the old version field, please use `logSubcommand` - instead - rule: 'has(self.logSubcommand) ? !has(self.logStop) : true' - - message: Field `logStop` is the old version field, please use `logSubcommand` - instead - rule: 'has(self.logStop) ? !has(self.logSubcommand) : true' - status: - properties: - backoffRetryStatus: - items: - properties: - detectFailedAt: - format: date-time - type: string - expectedRetryAt: - format: date-time - type: string - originalReason: - type: string - realRetryAt: - format: date-time - type: string - retryNum: - type: integer - retryReason: - type: string - type: object - type: array - backupPath: - type: string - backupSize: - format: int64 - type: integer - backupSizeReadable: - type: string - commitTs: - type: string - conditions: - items: - properties: - command: - type: string - lastTransitionTime: - format: date-time - nullable: true - type: string - message: - type: string - reason: - type: string - status: - type: string - type: - type: string - required: - - status - - type - type: object - nullable: true - type: array - incrementalBackupSize: - format: int64 - type: integer - incrementalBackupSizeReadable: - type: string - logCheckpointTs: - type: string - logSubCommandStatuses: - additionalProperties: - properties: - command: - type: string - conditions: - items: + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: + properties: + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath + type: object + required: + - name + type: object + volumeMount: properties: - command: + mountPath: type: string - lastTransitionTime: - format: date-time - nullable: true + mountPropagation: type: string - message: + name: type: string - reason: + readOnly: + type: boolean + subPath: type: string - status: + subPathExpr: + type: string + required: + - mountPath + - name + type: object + required: + - volume + - volumeMount + type: object + logStop: + type: boolean + logSubcommand: + enum: + - log-start + - log-stop + - log-pause + type: string + logTruncateUntil: + type: string + podSecurityContext: + properties: + fsGroup: + format: int64 + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + format: int64 + type: integer + runAsNonRoot: + type: boolean + runAsUser: + format: int64 + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: type: string type: type: string required: - - status - type type: object - nullable: true - type: array - logTruncatingUntil: - type: string - phase: - type: string - timeCompleted: - format: date-time - nullable: true - type: string - timeStarted: - format: date-time - nullable: true - type: string - type: object - type: object - logSuccessTruncateUntil: - type: string - phase: - type: string - progresses: - items: - properties: - lastTransitionTime: - format: date-time - nullable: true - type: string - progress: - type: number - step: - type: string - type: object - nullable: true - type: array - timeCompleted: - format: date-time - nullable: true - type: string - timeStarted: - format: date-time - nullable: true - type: string - timeTaken: - type: string - type: object - required: - - metadata - - spec - type: object - served: true - storage: true - subresources: {} ---- -apiVersion: apiextensions.k8s.io/v1 -kind: CustomResourceDefinition -metadata: - annotations: - controller-gen.kubebuilder.io/version: v0.15.0 - name: backupschedules.pingcap.com -spec: - group: pingcap.com - names: - kind: BackupSchedule - listKind: BackupScheduleList - plural: backupschedules - shortNames: - - bks - singular: backupschedule - scope: Namespaced - versions: - - additionalPrinterColumns: - - description: The cron format string used for backup scheduling - jsonPath: .spec.schedule - name: Schedule - type: string - - description: The max number of backups we want to keep - jsonPath: .spec.maxBackups - name: MaxBackups - type: integer - - description: How long backups we want to keep - jsonPath: .spec.maxReservedTime - name: MaxReservedTime - type: string - - description: The last backup CR name - jsonPath: .status.lastBackup - name: LastBackup - priority: 1 - type: string - - description: The last time the backup was successfully created - jsonPath: .status.lastBackupTime - name: LastBackupTime - priority: 1 - type: date - - jsonPath: .metadata.creationTimestamp - name: Age - type: date - name: v1alpha1 - schema: - openAPIV3Schema: - properties: - apiVersion: - type: string - kind: - type: string - metadata: - type: object - spec: - properties: - backupTemplate: + supplementalGroups: + items: + format: int64 + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + required: + - name + - value + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + priorityClassName: + type: string + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + resumeGcSchedule: + type: boolean + s3: + properties: + acl: + type: string + bucket: + type: string + endpoint: + type: string + options: + items: + type: string + type: array + path: + type: string + prefix: + type: string + provider: + type: string + region: + type: string + secretName: + type: string + sse: + type: string + storageClass: + type: string + required: + - provider + type: object + serviceAccount: + type: string + storageClassName: + type: string + storageSize: + type: string + tableFilter: + items: + type: string + type: array + tikvGCLifeTime: + type: string + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + format: int64 + type: integer + value: + type: string + type: object + type: array + toolImage: + type: string + useKMS: + type: boolean + volumeBackupInitJobMaxActiveSeconds: + default: 600 + type: integer + type: object + x-kubernetes-validations: + - message: Field `logStop` is the old version field, please use `logSubcommand` + instead + rule: 'has(self.logSubcommand) ? !has(self.logStop) : true' + - message: Field `logStop` is the old version field, please use `logSubcommand` + instead + rule: 'has(self.logStop) ? !has(self.logSubcommand) : true' + imagePullSecrets: + items: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + type: array + logBackupTemplate: properties: additionalVolumeMounts: items: @@ -3781,82 +3569,530 @@ spec: properties: key: type: string - name: + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic + type: object + required: + - name + type: object + type: array + federalVolumeBackupPhase: + type: string + from: + properties: + host: + type: string + port: + format: int32 + type: integer + secretName: + type: string + tlsClientSecretName: + type: string + user: + type: string + required: + - host + - secretName + type: object + gcs: + properties: + bucket: + type: string + bucketAcl: + type: string + location: + type: string + objectAcl: + type: string + path: + type: string + prefix: + type: string + projectId: + type: string + secretName: + type: string + storageClass: + type: string + required: + - projectId + type: object + imagePullSecrets: + items: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + type: array + local: + properties: + prefix: + type: string + volume: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + required: + - secretName + - shareName + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + required: + - spec + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: type: string - optional: - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic - type: object - required: - - name - type: object - type: array - federalVolumeBackupPhase: - type: string - from: - properties: - host: - type: string - port: - format: int32 - type: integer - secretName: - type: string - tlsClientSecretName: - type: string - user: - type: string - required: - - host - - secretName - type: object - gcs: - properties: - bucket: - type: string - bucketAcl: - type: string - location: - type: string - objectAcl: - type: string - path: - type: string - prefix: - type: string - projectId: - type: string - secretName: - type: string - storageClass: - type: string - required: - - projectId - type: object - imagePullSecrets: - items: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - type: array - local: - properties: - prefix: - type: string - volume: - properties: - awsElasticBlockStore: + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: properties: fsType: type: string partition: format: int32 type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string readOnly: type: boolean volumeID: @@ -3864,48 +4100,150 @@ spec: required: - volumeID type: object - azureDisk: + projected: properties: - cachingMode: + defaultMode: + format: int32 + type: integer + sources: + items: + properties: + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: + properties: + group: type: string - diskName: + readOnly: + type: boolean + registry: type: string - diskURI: + tenant: type: string - fsType: + user: type: string - kind: + volume: type: string - readOnly: - type: boolean required: - - diskName - - diskURI + - registry + - volume type: object - azureFile: + rbd: properties: - readOnly: - type: boolean - secretName: + fsType: type: string - shareName: + image: + type: string + keyring: type: string - required: - - secretName - - shareName - type: object - cephfs: - properties: monitors: items: type: string type: array - path: + pool: type: string readOnly: type: boolean - secretFile: - type: string secretRef: properties: name: @@ -3915,1922 +4253,3124 @@ spec: user: type: string required: + - image - monitors type: object - cinder: + scaleIO: properties: fsType: type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - csi: - properties: - driver: + gateway: type: string - fsType: + protectionDomain: type: string - nodePublishSecretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic readOnly: type: boolean - volumeAttributes: - additionalProperties: - type: string - type: object - required: - - driver - type: object - downwardAPI: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: - type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: - properties: - volumeClaimTemplate: - properties: - metadata: - type: object - spec: - properties: - accessModes: - items: - type: string - type: array - dataSource: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - required: - - kind - - name - type: object - x-kubernetes-map-type: atomic - dataSourceRef: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - namespace: - type: string - required: - - kind - - name - type: object - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - type: object - selector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - storageClassName: - type: string - volumeMode: - type: string - volumeName: - type: string - type: object - required: - - spec + secretRef: + properties: + name: + type: string type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system type: object - fc: + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath + type: object + required: + - name + type: object + volumeMount: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + subPath: + type: string + subPathExpr: + type: string + required: + - mountPath + - name + type: object + required: + - volume + - volumeMount + type: object + logStop: + type: boolean + logSubcommand: + enum: + - log-start + - log-stop + - log-pause + type: string + logTruncateUntil: + type: string + podSecurityContext: + properties: + fsGroup: + format: int64 + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + format: int64 + type: integer + runAsNonRoot: + type: boolean + runAsUser: + format: int64 + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + required: + - type + type: object + supplementalGroups: + items: + format: int64 + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + required: + - name + - value + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + priorityClassName: + type: string + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + resumeGcSchedule: + type: boolean + s3: + properties: + acl: + type: string + bucket: + type: string + endpoint: + type: string + options: + items: + type: string + type: array + path: + type: string + prefix: + type: string + provider: + type: string + region: + type: string + secretName: + type: string + sse: + type: string + storageClass: + type: string + required: + - provider + type: object + serviceAccount: + type: string + storageClassName: + type: string + storageSize: + type: string + tableFilter: + items: + type: string + type: array + tikvGCLifeTime: + type: string + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + format: int64 + type: integer + value: + type: string + type: object + type: array + toolImage: + type: string + useKMS: + type: boolean + volumeBackupInitJobMaxActiveSeconds: + default: 600 + type: integer + type: object + x-kubernetes-validations: + - message: Field `logStop` is the old version field, please use `logSubcommand` + instead + rule: 'has(self.logSubcommand) ? !has(self.logStop) : true' + - message: Field `logStop` is the old version field, please use `logSubcommand` + instead + rule: 'has(self.logStop) ? !has(self.logSubcommand) : true' + maxBackups: + format: int32 + type: integer + maxReservedTime: + type: string + pause: + type: boolean + schedule: + type: string + storageClassName: + type: string + storageSize: + type: string + required: + - backupTemplate + - schedule + type: object + status: + properties: + allBackupCleanTime: + format: date-time + type: string + lastBackup: + type: string + lastBackupTime: + format: date-time + type: string + logBackup: + type: string + type: object + required: + - metadata + - spec + type: object + served: true + storage: true + subresources: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + annotations: + controller-gen.kubebuilder.io/version: v0.15.0 + name: backups.pingcap.com +spec: + group: pingcap.com + names: + kind: Backup + listKind: BackupList + plural: backups + shortNames: + - bk + singular: backup + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: the type of backup, such as full, db, table. Only used when Mode + = snapshot. + jsonPath: .spec.backupType + name: Type + type: string + - description: the mode of backup, such as snapshot, log. + jsonPath: .spec.backupMode + name: Mode + type: string + - description: The current status of the backup + jsonPath: .status.phase + name: Status + type: string + - description: The full path of backup data + jsonPath: .status.backupPath + name: BackupPath + type: string + - description: The data size of the backup + jsonPath: .status.backupSizeReadable + name: BackupSize + type: string + - description: The real size of volume snapshot backup, only valid to volume snapshot + backup + jsonPath: .status.incrementalBackupSizeReadable + name: IncrementalBackupSize + priority: 10 + type: string + - description: The commit ts of the backup + jsonPath: .status.commitTs + name: CommitTS + type: string + - description: The log backup truncate until ts + jsonPath: .status.logSuccessTruncateUntil + name: LogTruncateUntil + type: string + - description: The time at which the backup was started + jsonPath: .status.timeStarted + name: Started + priority: 1 + type: date + - description: The time at which the backup was completed + jsonPath: .status.timeCompleted + name: Completed + priority: 1 + type: date + - description: The time that the backup takes + jsonPath: .status.timeTaken + name: TimeTaken + type: string + - jsonPath: .metadata.creationTimestamp + name: Age + type: date + name: v1alpha1 + schema: + openAPIV3Schema: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + type: object + spec: + properties: + additionalVolumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + subPath: + type: string + subPathExpr: + type: string + required: + - mountPath + - name + type: object + type: array + additionalVolumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + required: + - secretName + - shareName + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - fsType: + key: type: string - lun: + mode: format: int32 type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: - items: - type: string - type: array + path: + type: string + required: + - key + - path type: object - flexVolume: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - driver: - type: string - fsType: - type: string - options: - additionalProperties: - type: string - type: object - readOnly: - type: boolean - secretRef: + fieldRef: properties: - name: + apiVersion: type: string + fieldPath: + type: string + required: + - fieldPath type: object x-kubernetes-map-type: atomic - required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: - type: string - type: object - gcePersistentDisk: - properties: - fsType: - type: string - partition: + mode: format: int32 type: integer - pdName: - type: string - readOnly: - type: boolean - required: - - pdName - type: object - gitRepo: - properties: - directory: - type: string - repository: - type: string - revision: - type: string - required: - - repository - type: object - glusterfs: - properties: - endpoints: - type: string - path: - type: string - readOnly: - type: boolean - required: - - endpoints - - path - type: object - hostPath: - properties: path: type: string - type: - type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic required: - path type: object - iscsi: - properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: - type: string - lun: - format: int32 - type: integer - portals: - items: + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + storageClassName: type: string - type: array - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - targetPortal: - type: string - required: - - iqn - - lun - - targetPortal - type: object - name: + volumeMode: + type: string + volumeName: + type: string + type: object + required: + - spec + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName - type: object - photonPersistentDisk: - properties: - fsType: - type: string - pdID: - type: string - required: - - pdID - type: object - portworxVolume: - properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - projected: + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: properties: - defaultMode: - format: int32 - type: integer - sources: - items: - properties: - configMap: - properties: - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - downwardAPI: - properties: - items: - items: - properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: - type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - secret: - properties: - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - serviceAccountToken: + configMap: + properties: + items: + items: properties: - audience: + key: type: string - expirationSeconds: - format: int64 + mode: + format: int32 type: integer path: type: string required: + - key - path type: object - type: object - type: array - type: object - quobyte: - properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: - type: string - required: - - registry - - volume - type: object - rbd: - properties: - fsType: - type: string - image: - type: string - keyring: - type: string - monitors: - items: - type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: - properties: + type: array name: type: string + optional: + type: boolean type: object x-kubernetes-map-type: atomic - user: - type: string - required: - - image - - monitors - type: object - scaleIO: - properties: - fsType: - type: string - gateway: - type: string - protectionDomain: - type: string - readOnly: - type: boolean - secretRef: + downwardAPI: properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - sslEnabled: - type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string - required: - - gateway - - secretRef - - system - type: object - secret: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - optional: - type: boolean - secretName: - type: string - type: object - storageos: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array name: type: string + optional: + type: boolean type: object x-kubernetes-map-type: atomic - volumeName: - type: string - volumeNamespace: - type: string - type: object - vsphereVolume: - properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: - type: string - volumePath: - type: string - required: - - volumePath + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object type: object - required: - - name - type: object - volumeMount: - properties: - mountPath: - type: string - mountPropagation: - type: string - name: - type: string - readOnly: - type: boolean - subPath: - type: string - subPathExpr: - type: string - required: - - mountPath - - name - type: object - required: - - volume - - volumeMount - type: object - logStop: - type: boolean - logSubcommand: - enum: - - log-start - - log-stop - - log-pause - type: string - logTruncateUntil: - type: string - podSecurityContext: - properties: - fsGroup: - format: int64 - type: integer - fsGroupChangePolicy: - type: string - runAsGroup: - format: int64 - type: integer - runAsNonRoot: - type: boolean - runAsUser: - format: int64 - type: integer - seLinuxOptions: - properties: - level: - type: string - role: - type: string - type: - type: string - user: - type: string - type: object - seccompProfile: - properties: - localhostProfile: - type: string - type: - type: string - required: - - type - type: object - supplementalGroups: - items: - format: int64 - type: integer - type: array - sysctls: - items: - properties: - name: - type: string - value: - type: string - required: - - name - - value - type: object - type: array - windowsOptions: - properties: - gmsaCredentialSpec: - type: string - gmsaCredentialSpecName: - type: string - hostProcess: - type: boolean - runAsUserName: + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: type: string - type: object - type: object - priorityClassName: - type: string - resources: - properties: - claims: - items: + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: properties: name: type: string - required: - - name type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - type: object - resumeGcSchedule: - type: boolean - s3: - properties: - acl: - type: string - bucket: - type: string - endpoint: - type: string - options: - items: - type: string - type: array - path: - type: string - prefix: - type: string - provider: - type: string - region: - type: string - secretName: - type: string - sse: - type: string - storageClass: - type: string - required: - - provider - type: object - serviceAccount: - type: string - storageClassName: - type: string - storageSize: - type: string - tableFilter: - items: - type: string - type: array - tikvGCLifeTime: - type: string - tolerations: - items: + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: properties: - effect: + fsType: type: string - key: + gateway: type: string - operator: + protectionDomain: type: string - tolerationSeconds: - format: int64 - type: integer - value: + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: type: string + required: + - gateway + - secretRef + - system type: object - type: array - toolImage: - type: string - useKMS: - type: boolean - volumeBackupInitJobMaxActiveSeconds: - default: 600 - type: integer - type: object - x-kubernetes-validations: - - message: Field `logStop` is the old version field, please use `logSubcommand` - instead - rule: 'has(self.logSubcommand) ? !has(self.logStop) : true' - - message: Field `logStop` is the old version field, please use `logSubcommand` - instead - rule: 'has(self.logStop) ? !has(self.logSubcommand) : true' - imagePullSecrets: - items: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - type: array - logBackupTemplate: - properties: - additionalVolumeMounts: - items: + secret: properties: - mountPath: - type: string - mountPropagation: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: + type: boolean + secretName: type: string - name: + type: object + storageos: + properties: + fsType: type: string readOnly: type: boolean - subPath: + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: type: string - subPathExpr: + volumeNamespace: type: string - required: - - mountPath - - name type: object - type: array - additionalVolumes: - items: + vsphereVolume: properties: - awsElasticBlockStore: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath + type: object + required: + - name + type: object + type: array + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: properties: - fsType: - type: string - partition: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + type: object + x-kubernetes-map-type: atomic + weight: + format: int32 + type: integer + required: + - preference + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + type: object + x-kubernetes-map-type: atomic + type: array + required: + - nodeSelectorTerms + type: object + x-kubernetes-map-type: atomic + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + weight: format: int32 type: integer - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - azureDisk: - properties: - cachingMode: - type: string - diskName: - type: string - diskURI: - type: string - fsType: - type: string - kind: - type: string - readOnly: - type: boolean - required: - - diskName - - diskURI - type: object - azureFile: - properties: - readOnly: - type: boolean - secretName: - type: string - shareName: - type: string required: - - secretName - - shareName + - podAffinityTerm + - weight type: object - cephfs: + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: properties: - monitors: - items: - type: string - type: array - path: - type: string - readOnly: - type: boolean - secretFile: - type: string - secretRef: + labelSelector: properties: - name: - type: string + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object type: object x-kubernetes-map-type: atomic - user: - type: string - required: - - monitors - type: object - cinder: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: + namespaceSelector: properties: - name: - type: string + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object type: object x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: - properties: - defaultMode: - format: int32 - type: integer - items: + namespaces: items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object + type: string type: array - name: + topologyKey: type: string - optional: - type: boolean + required: + - topologyKey type: object - x-kubernetes-map-type: atomic - csi: + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: properties: - driver: - type: string - fsType: - type: string - nodePublishSecretRef: + podAffinityTerm: properties: - name: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: type: string + required: + - topologyKey type: object - x-kubernetes-map-type: atomic - readOnly: - type: boolean - volumeAttributes: - additionalProperties: - type: string - type: object + weight: + format: int32 + type: integer required: - - driver + - podAffinityTerm + - weight type: object - downwardAPI: + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - fieldRef: + labelSelector: + properties: + matchExpressions: + items: properties: - apiVersion: + key: type: string - fieldPath: + operator: type: string + values: + items: + type: string + type: array required: - - fieldPath + - key + - operator type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + type: array + matchLabels: + additionalProperties: type: string - resourceFieldRef: + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: properties: - containerName: + key: type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + operator: type: string + values: + items: + type: string + type: array required: - - resource + - key + - operator type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: - properties: - volumeClaimTemplate: - properties: - metadata: - type: object - spec: - properties: - accessModes: - items: - type: string - type: array - dataSource: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - required: - - kind - - name - type: object - x-kubernetes-map-type: atomic - dataSourceRef: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - namespace: - type: string - required: - - kind - - name - type: object - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - type: object - selector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - storageClassName: - type: string - volumeMode: - type: string - volumeName: - type: string + type: array + matchLabels: + additionalProperties: + type: string type: object - required: - - spec type: object - type: object - fc: - properties: - fsType: - type: string - lun: - format: int32 - type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: + x-kubernetes-map-type: atomic + namespaces: items: type: string type: array + topologyKey: + type: string + required: + - topologyKey type: object - flexVolume: + type: array + type: object + type: object + azblob: + properties: + accessTier: + type: string + container: + type: string + path: + type: string + prefix: + type: string + sasToken: + type: string + secretName: + type: string + storageAccount: + type: string + type: object + backoffRetryPolicy: + properties: + maxRetryTimes: + default: 2 + type: integer + minRetryDuration: + default: 300s + type: string + retryTimeout: + default: 30m + type: string + type: object + backupMode: + default: snapshot + type: string + backupType: + type: string + br: + properties: + checkRequirements: + type: boolean + checksum: + type: boolean + cluster: + type: string + clusterNamespace: + type: string + concurrency: + format: int32 + type: integer + db: + type: string + logLevel: + type: string + onLine: + type: boolean + options: + items: + type: string + type: array + rateLimit: + type: integer + sendCredToTikv: + type: boolean + statusAddr: + type: string + table: + type: string + timeAgo: + type: string + required: + - cluster + type: object + calcSizeLevel: + default: all + type: string + cleanOption: + properties: + backoffEnabled: + type: boolean + batchConcurrency: + format: int32 + type: integer + disableBatchConcurrency: + type: boolean + pageSize: + format: int64 + type: integer + retryCount: + default: 5 + type: integer + routineConcurrency: + format: int32 + type: integer + snapshotsDeleteRatio: + default: 1 + type: number + type: object + cleanPolicy: + default: Retain + enum: + - Retain + - OnFailure + - Delete + type: string + commitTs: + type: string + dumpling: + properties: + options: + items: + type: string + type: array + tableFilter: + items: + type: string + type: array + type: object + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: properties: - driver: + key: type: string - fsType: + name: type: string - options: - additionalProperties: - type: string - type: object - readOnly: + optional: type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: - type: string + - key type: object - gcePersistentDisk: + x-kubernetes-map-type: atomic + fieldRef: properties: - fsType: + apiVersion: type: string - partition: - format: int32 - type: integer - pdName: + fieldPath: type: string - readOnly: - type: boolean required: - - pdName + - fieldPath type: object - gitRepo: + x-kubernetes-map-type: atomic + resourceFieldRef: properties: - directory: - type: string - repository: + containerName: type: string - revision: + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: type: string required: - - repository + - resource type: object - glusterfs: + x-kubernetes-map-type: atomic + secretKeyRef: properties: - endpoints: + key: type: string - path: + name: type: string - readOnly: + optional: type: boolean required: - - endpoints - - path - type: object - hostPath: - properties: - path: - type: string - type: - type: string - required: - - path + - key type: object - iscsi: - properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: + x-kubernetes-map-type: atomic + type: object + required: + - name + type: object + type: array + federalVolumeBackupPhase: + type: string + from: + properties: + host: + type: string + port: + format: int32 + type: integer + secretName: + type: string + tlsClientSecretName: + type: string + user: + type: string + required: + - host + - secretName + type: object + gcs: + properties: + bucket: + type: string + bucketAcl: + type: string + location: + type: string + objectAcl: + type: string + path: + type: string + prefix: + type: string + projectId: + type: string + secretName: + type: string + storageClass: + type: string + required: + - projectId + type: object + imagePullSecrets: + items: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + type: array + local: + properties: + prefix: + type: string + volume: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + required: + - secretName + - shareName + type: object + cephfs: + properties: + monitors: + items: type: string - lun: - format: int32 - type: integer - portals: - items: + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: type: string - type: array - readOnly: - type: boolean - secretRef: + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - name: + key: + type: string + mode: + format: int32 + type: integer + path: type: string + required: + - key + - path type: object - x-kubernetes-map-type: atomic - targetPortal: - type: string - required: - - iqn - - lun - - targetPortal - type: object - name: - type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName - type: object - photonPersistentDisk: - properties: - fsType: - type: string - pdID: - type: string - required: - - pdID - type: object - portworxVolume: - properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: type: string - required: - - volumeID - type: object - projected: - properties: - defaultMode: - format: int32 - type: integer - sources: - items: + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + type: object + spec: properties: - configMap: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: properties: - items: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: items: properties: key: type: string - mode: - format: int32 - type: integer - path: + operator: type: string + values: + items: + type: string + type: array required: - key - - path + - operator type: object type: array - name: - type: string - optional: - type: boolean + matchLabels: + additionalProperties: + type: string + type: object type: object x-kubernetes-map-type: atomic - downwardAPI: - properties: + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + required: + - spec + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: + properties: + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: items: - items: - properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: - type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - secret: - properties: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: + properties: + items: items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - serviceAccountToken: - properties: - audience: - type: string - expirationSeconds: - format: int64 - type: integer - path: - type: string - required: - - path - type: object - type: object - type: array - type: object - quobyte: - properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: - type: string - required: - - registry - - volume - type: object - rbd: - properties: - fsType: - type: string - image: - type: string - keyring: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: type: string - monitors: - items: + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - name: + key: type: string - type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - image - - monitors - type: object - scaleIO: - properties: - fsType: - type: string - gateway: - type: string - protectionDomain: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: + mode: + format: int32 + type: integer + path: type: string + required: + - key + - path type: object - x-kubernetes-map-type: atomic - sslEnabled: - type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string - required: - - gateway - - secretRef - - system - type: object - secret: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - optional: - type: boolean - secretName: - type: string - type: object - storageos: + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath + type: object + required: + - name + type: object + volumeMount: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + subPath: + type: string + subPathExpr: + type: string + required: + - mountPath + - name + type: object + required: + - volume + - volumeMount + type: object + logStop: + type: boolean + logSubcommand: + enum: + - log-start + - log-stop + - log-pause + type: string + logTruncateUntil: + type: string + podSecurityContext: + properties: + fsGroup: + format: int64 + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + format: int64 + type: integer + runAsNonRoot: + type: boolean + runAsUser: + format: int64 + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + required: + - type + type: object + supplementalGroups: + items: + format: int64 + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + required: + - name + - value + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + priorityClassName: + type: string + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + resumeGcSchedule: + type: boolean + s3: + properties: + acl: + type: string + bucket: + type: string + endpoint: + type: string + options: + items: + type: string + type: array + path: + type: string + prefix: + type: string + provider: + type: string + region: + type: string + secretName: + type: string + sse: + type: string + storageClass: + type: string + required: + - provider + type: object + serviceAccount: + type: string + storageClassName: + type: string + storageSize: + type: string + tableFilter: + items: + type: string + type: array + tikvGCLifeTime: + type: string + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + format: int64 + type: integer + value: + type: string + type: object + type: array + toolImage: + type: string + useKMS: + type: boolean + volumeBackupInitJobMaxActiveSeconds: + default: 600 + type: integer + type: object + x-kubernetes-validations: + - message: Field `logStop` is the old version field, please use `logSubcommand` + instead + rule: 'has(self.logSubcommand) ? !has(self.logStop) : true' + - message: Field `logStop` is the old version field, please use `logSubcommand` + instead + rule: 'has(self.logStop) ? !has(self.logSubcommand) : true' + status: + properties: + backoffRetryStatus: + items: + properties: + detectFailedAt: + format: date-time + type: string + expectedRetryAt: + format: date-time + type: string + originalReason: + type: string + realRetryAt: + format: date-time + type: string + retryNum: + type: integer + retryReason: + type: string + type: object + type: array + backupPath: + type: string + backupSize: + format: int64 + type: integer + backupSizeReadable: + type: string + commitTs: + type: string + conditions: + items: + properties: + command: + type: string + lastTransitionTime: + format: date-time + nullable: true + type: string + message: + type: string + reason: + type: string + status: + type: string + type: + type: string + required: + - status + - type + type: object + nullable: true + type: array + incrementalBackupSize: + format: int64 + type: integer + incrementalBackupSizeReadable: + type: string + logCheckpointTs: + type: string + logSubCommandStatuses: + additionalProperties: + properties: + command: + type: string + conditions: + items: + properties: + command: + type: string + lastTransitionTime: + format: date-time + nullable: true + type: string + message: + type: string + reason: + type: string + status: + type: string + type: + type: string + required: + - status + - type + type: object + nullable: true + type: array + logTruncatingUntil: + type: string + phase: + type: string + timeCompleted: + format: date-time + nullable: true + type: string + timeStarted: + format: date-time + nullable: true + type: string + type: object + type: object + logSuccessTruncateUntil: + type: string + phase: + type: string + progresses: + items: + properties: + lastTransitionTime: + format: date-time + nullable: true + type: string + progress: + type: number + step: + type: string + type: object + nullable: true + type: array + timeCompleted: + format: date-time + nullable: true + type: string + timeStarted: + format: date-time + nullable: true + type: string + timeTaken: + type: string + type: object + required: + - metadata + - spec + type: object + served: true + storage: true + subresources: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + annotations: + controller-gen.kubebuilder.io/version: v0.15.0 + name: compactbackups.pingcap.com +spec: + group: pingcap.com + names: + kind: CompactBackup + listKind: CompactBackupList + plural: compactbackups + shortNames: + - cpbk + singular: compactbackup + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The current status of the compact backup + jsonPath: .status.state + name: Status + type: string + - description: The progress of the compact backup + jsonPath: .status.progress + name: Progress + type: string + - description: The message of the compact backup + jsonPath: .status.message + name: Message + type: string + name: v1alpha1 + schema: + openAPIV3Schema: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + type: object + spec: + properties: + additionalVolumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + subPath: + type: string + subPathExpr: + type: string + required: + - mountPath + - name + type: object + type: array + additionalVolumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + required: + - secretName + - shareName + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - volumeName: - type: string - volumeNamespace: + name: type: string type: object - vsphereVolume: + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: - type: string - volumePath: + name: type: string - required: - - volumePath type: object + x-kubernetes-map-type: atomic + volumeID: + type: string required: - - name + - volumeID type: object - type: array - affinity: - properties: - nodeAffinity: - properties: - preferredDuringSchedulingIgnoredDuringExecution: - items: - properties: - preference: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchFields: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - type: object - x-kubernetes-map-type: atomic - weight: - format: int32 - type: integer - required: - - preference - - weight - type: object - type: array - requiredDuringSchedulingIgnoredDuringExecution: + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - nodeSelectorTerms: - items: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchFields: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - type: object - x-kubernetes-map-type: atomic - type: array + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic required: - - nodeSelectorTerms + - path type: object - x-kubernetes-map-type: atomic - type: object - podAffinity: - properties: - preferredDuringSchedulingIgnoredDuringExecution: - items: - properties: - podAffinityTerm: - properties: - labelSelector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - namespaceSelector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - namespaces: - items: - type: string - type: array - topologyKey: - type: string - required: - - topologyKey - type: object - weight: - format: int32 - type: integer - required: - - podAffinityTerm - - weight + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: type: object - type: array - requiredDuringSchedulingIgnoredDuringExecution: - items: + spec: properties: - labelSelector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - namespaceSelector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - namespaces: + accessModes: items: type: string type: array - topologyKey: - type: string - required: - - topologyKey - type: object - type: array - type: object - podAntiAffinity: - properties: - preferredDuringSchedulingIgnoredDuringExecution: - items: - properties: - podAffinityTerm: - properties: - labelSelector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - namespaceSelector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - namespaces: - items: - type: string - type: array - topologyKey: + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: type: string required: - - topologyKey + - kind + - name type: object - weight: - format: int32 - type: integer - required: - - podAffinityTerm - - weight - type: object - type: array - requiredDuringSchedulingIgnoredDuringExecution: - items: - properties: - labelSelector: + x-kubernetes-map-type: atomic + dataSourceRef: properties: - matchExpressions: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: items: properties: - key: - type: string - operator: + name: type: string - values: - items: - type: string - type: array required: - - key - - operator + - name type: object type: array - matchLabels: + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: additionalProperties: - type: string + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true type: object type: object - x-kubernetes-map-type: atomic - namespaceSelector: + selector: properties: matchExpressions: items: @@ -5840,1205 +7380,1875 @@ spec: operator: type: string values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string - type: object - type: object - x-kubernetes-map-type: atomic - namespaces: - items: - type: string - type: array - topologyKey: - type: string - required: - - topologyKey - type: object - type: array - type: object - type: object - azblob: - properties: - accessTier: - type: string - container: - type: string - path: - type: string - prefix: - type: string - sasToken: - type: string - secretName: - type: string - storageAccount: - type: string - type: object - backoffRetryPolicy: - properties: - maxRetryTimes: - default: 2 - type: integer - minRetryDuration: - default: 300s - type: string - retryTimeout: - default: 30m - type: string - type: object - backupMode: - default: snapshot - type: string - backupType: - type: string - br: - properties: - checkRequirements: - type: boolean - checksum: - type: boolean - cluster: - type: string - clusterNamespace: - type: string - concurrency: - format: int32 - type: integer - db: - type: string - logLevel: - type: string - onLine: - type: boolean - options: - items: - type: string - type: array - rateLimit: - type: integer - sendCredToTikv: - type: boolean - statusAddr: - type: string - table: - type: string - timeAgo: - type: string - required: - - cluster - type: object - calcSizeLevel: - default: all - type: string - cleanOption: - properties: - backoffEnabled: - type: boolean - batchConcurrency: - format: int32 - type: integer - disableBatchConcurrency: - type: boolean - pageSize: - format: int64 - type: integer - retryCount: - default: 5 - type: integer - routineConcurrency: - format: int32 - type: integer - snapshotsDeleteRatio: - default: 1 - type: number - type: object - cleanPolicy: - default: Retain - enum: - - Retain - - OnFailure - - Delete - type: string - commitTs: - type: string - dumpling: - properties: - options: - items: - type: string - type: array - tableFilter: - items: - type: string - type: array - type: object - env: - items: - properties: - name: - type: string - value: - type: string - valueFrom: - properties: - configMapKeyRef: - properties: - key: - type: string - name: - type: string - optional: - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + storageClassName: type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic - secretKeyRef: - properties: - key: + volumeMode: type: string - name: + volumeName: type: string - optional: - type: boolean - required: - - key type: object - x-kubernetes-map-type: atomic + required: + - spec + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string type: object + x-kubernetes-map-type: atomic required: - - name + - driver type: object - type: array - federalVolumeBackupPhase: - type: string - from: - properties: - host: - type: string - port: - format: int32 - type: integer - secretName: - type: string - tlsClientSecretName: - type: string - user: - type: string - required: - - host - - secretName - type: object - gcs: - properties: - bucket: - type: string - bucketAcl: - type: string - location: - type: string - objectAcl: - type: string - path: - type: string - prefix: - type: string - projectId: - type: string - secretName: - type: string - storageClass: - type: string - required: - - projectId - type: object - imagePullSecrets: - items: + flocker: properties: - name: + datasetName: + type: string + datasetUUID: type: string type: object - x-kubernetes-map-type: atomic - type: array - local: - properties: - prefix: - type: string - volume: - properties: - awsElasticBlockStore: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - azureDisk: - properties: - cachingMode: - type: string - diskName: - type: string - diskURI: - type: string - fsType: - type: string - kind: - type: string - readOnly: - type: boolean - required: - - diskName - - diskURI - type: object - azureFile: - properties: - readOnly: - type: boolean - secretName: - type: string - shareName: - type: string - required: - - secretName - - shareName - type: object - cephfs: + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: properties: - monitors: - items: - type: string - type: array - path: - type: string - readOnly: - type: boolean - secretFile: - type: string - secretRef: + configMap: properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array name: type: string + optional: + type: boolean type: object x-kubernetes-map-type: atomic - user: - type: string - required: - - monitors - type: object - cinder: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array name: type: string + optional: + type: boolean type: object x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object type: object - x-kubernetes-map-type: atomic - csi: + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - driver: + key: type: string - fsType: + mode: + format: int32 + type: integer + path: type: string - nodePublishSecretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - readOnly: - type: boolean - volumeAttributes: - additionalProperties: - type: string - type: object required: - - driver + - key + - path type: object - downwardAPI: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - fieldRef: - properties: - apiVersion: + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath + type: object + required: + - name + type: object + type: array + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: type: string - fieldPath: + type: array + required: + - key + - operator + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: - type: string - resourceFieldRef: - properties: - containerName: + type: array + required: + - key + - operator + type: object + type: array + type: object + x-kubernetes-map-type: atomic + weight: + format: int32 + type: integer + required: + - preference + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + type: array + required: + - key + - operator + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: type: string - required: - - resource + type: array + required: + - key + - operator + type: object + type: array + type: object + x-kubernetes-map-type: atomic + type: array + required: + - nodeSelectorTerms + type: object + x-kubernetes-map-type: atomic + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string type: object - x-kubernetes-map-type: atomic - required: - - path type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: - properties: - volumeClaimTemplate: - properties: - metadata: + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + weight: + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator type: object - spec: + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: properties: - accessModes: + key: + type: string + operator: + type: string + values: items: type: string type: array - dataSource: + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: properties: - apiGroup: - type: string - kind: + key: type: string - name: + operator: type: string + values: + items: + type: string + type: array required: - - kind - - name + - key + - operator type: object - x-kubernetes-map-type: atomic - dataSourceRef: + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: properties: - apiGroup: - type: string - kind: - type: string - name: + key: type: string - namespace: + operator: type: string - required: - - kind - - name - type: object - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - type: object - selector: - properties: - matchExpressions: + values: items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: type: string - type: object + type: array + required: + - key + - operator type: object - x-kubernetes-map-type: atomic - storageClassName: + type: array + matchLabels: + additionalProperties: type: string - volumeMode: + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + weight: + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: type: string - volumeName: + operator: type: string + values: + items: + type: string + type: array + required: + - key + - operator type: object - required: - - spec - type: object - type: object - fc: - properties: - fsType: - type: string - lun: - format: int32 - type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: - items: - type: string - type: array - type: object - flexVolume: - properties: - driver: - type: string - fsType: - type: string - options: - additionalProperties: - type: string - type: object - readOnly: - type: boolean - secretRef: - properties: - name: + type: array + matchLabels: + additionalProperties: type: string - type: object - x-kubernetes-map-type: atomic - required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: type: string - type: object - gcePersistentDisk: + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + type: array + type: object + type: object + azblob: + properties: + accessTier: + type: string + container: + type: string + path: + type: string + prefix: + type: string + sasToken: + type: string + secretName: + type: string + storageAccount: + type: string + type: object + br: + properties: + checkRequirements: + type: boolean + checksum: + type: boolean + cluster: + type: string + clusterNamespace: + type: string + concurrency: + format: int32 + type: integer + db: + type: string + logLevel: + type: string + onLine: + type: boolean + options: + items: + type: string + type: array + rateLimit: + type: integer + sendCredToTikv: + type: boolean + statusAddr: + type: string + table: + type: string + timeAgo: + type: string + required: + - cluster + type: object + concurrency: + default: 4 + type: integer + endTs: + type: string + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic + type: object + required: + - name + type: object + type: array + gcs: + properties: + bucket: + type: string + bucketAcl: + type: string + location: + type: string + objectAcl: + type: string + path: + type: string + prefix: + type: string + projectId: + type: string + secretName: + type: string + storageClass: + type: string + required: + - projectId + type: object + imagePullSecrets: + items: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + type: array + local: + properties: + prefix: + type: string + volume: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + required: + - secretName + - shareName + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: properties: - fsType: - type: string - partition: - format: int32 - type: integer - pdName: + name: type: string - readOnly: - type: boolean - required: - - pdName type: object - gitRepo: + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: properties: - directory: - type: string - repository: - type: string - revision: + name: type: string - required: - - repository type: object - glusterfs: + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: properties: - endpoints: - type: string - path: + name: type: string - readOnly: - type: boolean - required: - - endpoints - - path type: object - hostPath: - properties: - path: - type: string - type: - type: string - required: - - path + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string type: object - iscsi: - properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: - type: string - lun: - format: int32 - type: integer - portals: - items: + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string - type: array - readOnly: - type: boolean - secretRef: + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + type: object + spec: properties: - name: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: type: string type: object - x-kubernetes-map-type: atomic - targetPortal: - type: string required: - - iqn - - lun - - targetPortal + - spec type: object - name: + type: object + fc: + properties: + fsType: type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string type: object - photonPersistentDisk: + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - pdID: + name: type: string - required: - - pdID type: object - portworxVolume: + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: + name: type: string - required: - - volumeID type: object - projected: - properties: - defaultMode: - format: int32 - type: integer - sources: - items: + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: + properties: + configMap: properties: - configMap: - properties: - items: - items: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: + properties: + fieldRef: properties: - key: - type: string - mode: - format: int32 - type: integer - path: + apiVersion: type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - downwardAPI: - properties: - items: - items: - properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + fieldPath: type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic required: - - path + - fieldPath type: object - type: array - type: object - secret: - properties: - items: - items: + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: properties: - key: + containerName: type: string - mode: - format: int32 - type: integer - path: + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: type: string required: - - key - - path + - resource type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - serviceAccountToken: - properties: - audience: - type: string - expirationSeconds: - format: int64 - type: integer - path: - type: string - required: - - path - type: object - type: object - type: array - type: object - quobyte: - properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: - type: string - required: - - registry - - volume - type: object - rbd: - properties: - fsType: - type: string - image: - type: string - keyring: - type: string - monitors: - items: - type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - image - - monitors - type: object - scaleIO: - properties: - fsType: - type: string - gateway: - type: string - protectionDomain: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - sslEnabled: - type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string - required: - - gateway - - secretRef - - system - type: object - secret: - properties: - defaultMode: - format: int32 - type: integer - items: - items: + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: properties: - key: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: type: string - mode: - format: int32 + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 type: integer path: type: string required: - - key - path type: object - type: array - optional: - type: boolean - secretName: - type: string - type: object - storageos: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - volumeName: - type: string - volumeNamespace: - type: string - type: object - vsphereVolume: - properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: - type: string - volumePath: - type: string - required: - - volumePath - type: object - required: - - name + type: object + type: array type: object - volumeMount: + quobyte: properties: - mountPath: - type: string - mountPropagation: - type: string - name: + group: type: string readOnly: type: boolean - subPath: + registry: type: string - subPathExpr: + tenant: + type: string + user: + type: string + volume: type: string required: - - mountPath - - name + - registry + - volume type: object - required: - - volume - - volumeMount - type: object - logStop: - type: boolean - logSubcommand: - enum: - - log-start - - log-stop - - log-pause - type: string - logTruncateUntil: - type: string - podSecurityContext: - properties: - fsGroup: - format: int64 - type: integer - fsGroupChangePolicy: - type: string - runAsGroup: - format: int64 - type: integer - runAsNonRoot: - type: boolean - runAsUser: - format: int64 - type: integer - seLinuxOptions: + rbd: properties: - level: + fsType: type: string - role: + image: type: string - type: + keyring: + type: string + monitors: + items: + type: string + type: array + pool: type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic user: type: string + required: + - image + - monitors type: object - seccompProfile: + scaleIO: properties: - localhostProfile: + fsType: type: string - type: + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: type: string required: - - type + - gateway + - secretRef + - system type: object - supplementalGroups: - items: - format: int64 - type: integer - type: array - sysctls: - items: - properties: - name: - type: string - value: - type: string - required: - - name - - value - type: object - type: array - windowsOptions: + secret: properties: - gmsaCredentialSpec: - type: string - gmsaCredentialSpecName: - type: string - hostProcess: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: type: boolean - runAsUserName: + secretName: type: string type: object - type: object - priorityClassName: - type: string - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: + type: string type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath type: object + required: + - name type: object - resumeGcSchedule: - type: boolean - s3: + volumeMount: properties: - acl: + mountPath: type: string - bucket: + mountPropagation: type: string - endpoint: + name: type: string - options: - items: - type: string - type: array - path: + readOnly: + type: boolean + subPath: type: string - prefix: + subPathExpr: type: string - provider: + required: + - mountPath + - name + type: object + required: + - volume + - volumeMount + type: object + maxRetryTimes: + default: 6 + format: int32 + type: integer + podSecurityContext: + properties: + fsGroup: + format: int64 + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + format: int64 + type: integer + runAsNonRoot: + type: boolean + runAsUser: + format: int64 + type: integer + seLinuxOptions: + properties: + level: type: string - region: + role: type: string - secretName: + type: type: string - sse: + user: type: string - storageClass: + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: type: string required: - - provider + - type type: object - serviceAccount: - type: string - storageClassName: - type: string - storageSize: - type: string - tableFilter: + supplementalGroups: items: - type: string + format: int64 + type: integer type: array - tikvGCLifeTime: - type: string - tolerations: + sysctls: items: properties: - effect: - type: string - key: - type: string - operator: + name: type: string - tolerationSeconds: - format: int64 - type: integer value: type: string + required: + - name + - value type: object type: array - toolImage: - type: string - useKMS: - type: boolean - volumeBackupInitJobMaxActiveSeconds: - default: 600 - type: integer + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object type: object - x-kubernetes-validations: - - message: Field `logStop` is the old version field, please use `logSubcommand` - instead - rule: 'has(self.logSubcommand) ? !has(self.logStop) : true' - - message: Field `logStop` is the old version field, please use `logSubcommand` - instead - rule: 'has(self.logStop) ? !has(self.logSubcommand) : true' - maxBackups: - format: int32 - type: integer - maxReservedTime: + priorityClassName: type: string - pause: - type: boolean - schedule: + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + s3: + properties: + acl: + type: string + bucket: + type: string + endpoint: + type: string + options: + items: + type: string + type: array + path: + type: string + prefix: + type: string + provider: + type: string + region: + type: string + secretName: + type: string + sse: + type: string + storageClass: + type: string + required: + - provider + type: object + serviceAccount: type: string - storageClassName: + startTs: type: string - storageSize: + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + format: int64 + type: integer + value: + type: string + type: object + type: array + toolImage: type: string - required: - - backupTemplate - - schedule + useKMS: + type: boolean type: object status: properties: - allBackupCleanTime: - format: date-time - type: string - lastBackup: + backoffRetryStatus: + items: + properties: + detectFailedAt: + format: date-time + type: string + retryNum: + type: integer + retryReason: + type: string + type: object + type: array + message: type: string - lastBackupTime: - format: date-time + progress: type: string - logBackup: + state: type: string type: object required: diff --git a/manifests/crd/v1/pingcap.com_compactbackups.yaml b/manifests/crd/v1/pingcap.com_compactbackups.yaml new file mode 100644 index 00000000000..44bbdfff51f --- /dev/null +++ b/manifests/crd/v1/pingcap.com_compactbackups.yaml @@ -0,0 +1,2210 @@ +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + annotations: + controller-gen.kubebuilder.io/version: v0.15.0 + name: compactbackups.pingcap.com +spec: + group: pingcap.com + names: + kind: CompactBackup + listKind: CompactBackupList + plural: compactbackups + shortNames: + - cpbk + singular: compactbackup + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The current status of the compact backup + jsonPath: .status.state + name: Status + type: string + - description: The progress of the compact backup + jsonPath: .status.progress + name: Progress + type: string + - description: The message of the compact backup + jsonPath: .status.message + name: Message + type: string + name: v1alpha1 + schema: + openAPIV3Schema: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + type: object + spec: + properties: + additionalVolumeMounts: + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + subPath: + type: string + subPathExpr: + type: string + required: + - mountPath + - name + type: object + type: array + additionalVolumes: + items: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + required: + - secretName + - shareName + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + required: + - spec + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: + properties: + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath + type: object + required: + - name + type: object + type: array + affinity: + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + type: object + x-kubernetes-map-type: atomic + weight: + format: int32 + type: integer + required: + - preference + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + type: object + x-kubernetes-map-type: atomic + type: array + required: + - nodeSelectorTerms + type: object + x-kubernetes-map-type: atomic + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + weight: + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + weight: + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + namespaces: + items: + type: string + type: array + topologyKey: + type: string + required: + - topologyKey + type: object + type: array + type: object + type: object + azblob: + properties: + accessTier: + type: string + container: + type: string + path: + type: string + prefix: + type: string + sasToken: + type: string + secretName: + type: string + storageAccount: + type: string + type: object + br: + properties: + checkRequirements: + type: boolean + checksum: + type: boolean + cluster: + type: string + clusterNamespace: + type: string + concurrency: + format: int32 + type: integer + db: + type: string + logLevel: + type: string + onLine: + type: boolean + options: + items: + type: string + type: array + rateLimit: + type: integer + sendCredToTikv: + type: boolean + statusAddr: + type: string + table: + type: string + timeAgo: + type: string + required: + - cluster + type: object + concurrency: + default: 4 + type: integer + endTs: + type: string + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic + type: object + required: + - name + type: object + type: array + gcs: + properties: + bucket: + type: string + bucketAcl: + type: string + location: + type: string + objectAcl: + type: string + path: + type: string + prefix: + type: string + projectId: + type: string + secretName: + type: string + storageClass: + type: string + required: + - projectId + type: object + imagePullSecrets: + items: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + type: array + local: + properties: + prefix: + type: string + volume: + properties: + awsElasticBlockStore: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + required: + - secretName + - shareName + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + required: + - spec + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: + properties: + configMap: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath + type: object + required: + - name + type: object + volumeMount: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + subPath: + type: string + subPathExpr: + type: string + required: + - mountPath + - name + type: object + required: + - volume + - volumeMount + type: object + maxRetryTimes: + default: 6 + format: int32 + type: integer + podSecurityContext: + properties: + fsGroup: + format: int64 + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + format: int64 + type: integer + runAsNonRoot: + type: boolean + runAsUser: + format: int64 + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + required: + - type + type: object + supplementalGroups: + items: + format: int64 + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + required: + - name + - value + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + priorityClassName: + type: string + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + s3: + properties: + acl: + type: string + bucket: + type: string + endpoint: + type: string + options: + items: + type: string + type: array + path: + type: string + prefix: + type: string + provider: + type: string + region: + type: string + secretName: + type: string + sse: + type: string + storageClass: + type: string + required: + - provider + type: object + serviceAccount: + type: string + startTs: + type: string + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + format: int64 + type: integer + value: + type: string + type: object + type: array + toolImage: + type: string + useKMS: + type: boolean + type: object + status: + properties: + backoffRetryStatus: + items: + properties: + detectFailedAt: + format: date-time + type: string + retryNum: + type: integer + retryReason: + type: string + type: object + type: array + message: + type: string + progress: + type: string + state: + type: string + type: object + required: + - metadata + - spec + type: object + served: true + storage: true + subresources: {} diff --git a/manifests/federation-crd.yaml b/manifests/federation-crd.yaml index 0a6ae36d687..742c6627d4e 100644 --- a/manifests/federation-crd.yaml +++ b/manifests/federation-crd.yaml @@ -4,35 +4,41 @@ kind: CustomResourceDefinition metadata: annotations: controller-gen.kubebuilder.io/version: v0.15.0 - name: volumebackups.federation.pingcap.com + name: volumebackupschedules.federation.pingcap.com spec: group: federation.pingcap.com names: - kind: VolumeBackup - listKind: VolumeBackupList - plural: volumebackups + kind: VolumeBackupSchedule + listKind: VolumeBackupScheduleList + plural: volumebackupschedules shortNames: - - vbk - singular: volumebackup + - vbks + singular: volumebackupschedule scope: Namespaced versions: - additionalPrinterColumns: - - description: The current status of the backup - jsonPath: .status.phase - name: Status - type: string - - description: The data size of the backup - jsonPath: .status.backupSizeReadable - name: BackupSize + - description: The cron format string used for backup scheduling + jsonPath: .spec.schedule + name: Schedule type: string - - description: The commit ts of the backup - jsonPath: .status.commitTs - name: CommitTS + - description: The max number of backups we want to keep + jsonPath: .spec.maxBackups + name: MaxBackups + type: integer + - description: How long backups we want to keep + jsonPath: .spec.maxReservedTime + name: MaxReservedTime type: string - - description: The time that volume backup federation takes - jsonPath: .status.timeTaken - name: TimeTaken + - description: The last backup CR name + jsonPath: .status.lastBackup + name: LastBackup + priority: 1 type: string + - description: The last time the backup was successfully created + jsonPath: .status.lastBackupTime + name: LastBackupTime + priority: 1 + type: date - jsonPath: .metadata.creationTimestamp name: Age type: date @@ -46,1781 +52,1732 @@ spec: type: string metadata: type: object - skipGC: - default: false - type: boolean spec: properties: - clusters: - items: - properties: - k8sClusterName: - type: string - tcName: - type: string - tcNamespace: - type: string - type: object - type: array - template: + backupTemplate: properties: - additionalVolumeMounts: + clusters: items: properties: - mountPath: - type: string - mountPropagation: - type: string - name: + k8sClusterName: type: string - readOnly: - type: boolean - subPath: + tcName: type: string - subPathExpr: + tcNamespace: type: string - required: - - mountPath - - name type: object type: array - additionalVolumes: - items: - properties: - awsElasticBlockStore: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - azureDisk: + template: + properties: + additionalVolumeMounts: + items: properties: - cachingMode: - type: string - diskName: - type: string - diskURI: + mountPath: type: string - fsType: + mountPropagation: type: string - kind: + name: type: string readOnly: type: boolean - required: - - diskName - - diskURI - type: object - azureFile: - properties: - readOnly: - type: boolean - secretName: + subPath: type: string - shareName: + subPathExpr: type: string required: - - secretName - - shareName + - mountPath + - name type: object - cephfs: + type: array + additionalVolumes: + items: properties: - monitors: - items: - type: string - type: array - path: - type: string - readOnly: - type: boolean - secretFile: - type: string - secretRef: + awsElasticBlockStore: properties: - name: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: type: string + required: + - volumeID type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - monitors - type: object - cinder: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: + azureDisk: properties: - name: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: type: string + readOnly: + type: boolean + required: + - diskName + - diskURI type: object - x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - csi: - properties: - driver: - type: string - fsType: - type: string - nodePublishSecretRef: + azureFile: properties: - name: + readOnly: + type: boolean + secretName: type: string + shareName: + type: string + required: + - secretName + - shareName type: object - x-kubernetes-map-type: atomic - readOnly: - type: boolean - volumeAttributes: - additionalProperties: - type: string - type: object - required: - - driver - type: object - downwardAPI: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + cephfs: + properties: + monitors: + items: type: string - resourceFieldRef: + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - containerName: + key: type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + mode: + format: int32 + type: integer + path: type: string required: - - resource + - key + - path type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: - properties: - volumeClaimTemplate: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: properties: - metadata: - type: object - spec: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: properties: - accessModes: - items: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string - type: array - dataSource: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - required: - - kind - - name - type: object - x-kubernetes-map-type: atomic - dataSourceRef: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - namespace: - type: string - required: - - kind - - name - type: object - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: + resourceFieldRef: + properties: + containerName: + type: string + divisor: anyOf: - type: integer - type: string pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ x-kubernetes-int-or-string: true - type: object + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: + properties: + metadata: type: object - selector: + spec: properties: - matchExpressions: + accessModes: items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: type: string - type: object - type: object - x-kubernetes-map-type: atomic - storageClassName: - type: string - volumeMode: - type: string - volumeName: - type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + required: + - spec + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: + type: string type: object + x-kubernetes-map-type: atomic required: - - spec + - driver type: object - type: object - fc: - properties: - fsType: - type: string - lun: - format: int32 - type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: - items: - type: string - type: array - type: object - flexVolume: - properties: - driver: - type: string - fsType: - type: string - options: - additionalProperties: - type: string + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string type: object - readOnly: - type: boolean - secretRef: + gcePersistentDisk: properties: - name: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: type: string + readOnly: + type: boolean + required: + - pdName type: object - x-kubernetes-map-type: atomic - required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: - type: string - type: object - gcePersistentDisk: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - pdName: - type: string - readOnly: - type: boolean - required: - - pdName - type: object - gitRepo: - properties: - directory: - type: string - repository: - type: string - revision: - type: string - required: - - repository - type: object - glusterfs: - properties: - endpoints: - type: string - path: - type: string - readOnly: - type: boolean - required: - - endpoints - - path - type: object - hostPath: - properties: - path: - type: string - type: - type: string - required: - - path - type: object - iscsi: - properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: - type: string - lun: - format: int32 - type: integer - portals: - items: - type: string - type: array - readOnly: - type: boolean - secretRef: + gitRepo: properties: - name: + directory: + type: string + repository: + type: string + revision: type: string + required: + - repository type: object - x-kubernetes-map-type: atomic - targetPortal: - type: string - required: - - iqn - - lun - - targetPortal - type: object - name: - type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName - type: object - photonPersistentDisk: - properties: - fsType: - type: string - pdID: - type: string - required: - - pdID - type: object - portworxVolume: - properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - projected: - properties: - defaultMode: - format: int32 - type: integer - sources: - items: - properties: - configMap: - properties: - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - downwardAPI: + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: properties: - items: - items: - properties: - fieldRef: + configMap: + properties: + items: + items: properties: - apiVersion: + key: type: string - fieldPath: + mode: + format: int32 + type: integer + path: type: string required: - - fieldPath + - key + - path type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: - type: string - resourceFieldRef: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: properties: - containerName: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: type: string required: - - resource + - key + - path type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - secret: - properties: - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - serviceAccountToken: - properties: - audience: - type: string - expirationSeconds: - format: int64 - type: integer - path: - type: string - required: - - path - type: object - type: object - type: array - type: object - quobyte: - properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: - type: string - required: - - registry - - volume - type: object - rbd: - properties: - fsType: - type: string - image: - type: string - keyring: - type: string - monitors: - items: - type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: properties: - name: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: type: string + required: + - registry + - volume type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - image - - monitors - type: object - scaleIO: - properties: - fsType: - type: string - gateway: - type: string - protectionDomain: - type: string - readOnly: - type: boolean - secretRef: + rbd: properties: - name: + fsType: type: string - type: object - x-kubernetes-map-type: atomic - sslEnabled: - type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string - required: - - gateway - - secretRef - - system - type: object - secret: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: + image: + type: string + keyring: + type: string + monitors: + items: type: string - required: - - key - - path - type: object - type: array - optional: - type: boolean - secretName: - type: string - type: object - storageos: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: type: string + required: + - image + - monitors type: object - x-kubernetes-map-type: atomic - volumeName: - type: string - volumeNamespace: - type: string - type: object - vsphereVolume: - properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: - type: string - volumePath: - type: string - required: - - volumePath - type: object - required: - - name - type: object - type: array - azblob: - properties: - accessTier: - type: string - container: - type: string - path: - type: string - prefix: - type: string - sasToken: - type: string - secretName: - type: string - storageAccount: - type: string - type: object - br: - properties: - checkRequirements: - type: boolean - concurrency: - format: int32 - type: integer - options: - items: - type: string - type: array - sendCredToTikv: - type: boolean - type: object - calcSizeLevel: - default: all - type: string - cleanPolicy: - type: string - env: - items: - properties: - name: - type: string - value: - type: string - valueFrom: - properties: - configMapKeyRef: + scaleIO: properties: - key: + fsType: type: string - name: + gateway: type: string - optional: + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string required: - - key + - gateway + - secretRef + - system type: object - x-kubernetes-map-type: atomic - fieldRef: + secret: properties: - apiVersion: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: type: string - fieldPath: + volumeNamespace: type: string - required: - - fieldPath type: object - x-kubernetes-map-type: atomic - resourceFieldRef: + vsphereVolume: properties: - containerName: + fsType: type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + storagePolicyID: type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic - secretKeyRef: - properties: - key: + storagePolicyName: type: string - name: + volumePath: type: string - optional: - type: boolean required: - - key + - volumePath type: object - x-kubernetes-map-type: atomic + required: + - name type: object - required: - - name - type: object - type: array - gcs: - properties: - bucket: - type: string - bucketAcl: - type: string - location: - type: string - objectAcl: - type: string - path: - type: string - prefix: - type: string - projectId: - type: string - secretName: - type: string - storageClass: + type: array + azblob: + properties: + accessTier: + type: string + container: + type: string + path: + type: string + prefix: + type: string + sasToken: + type: string + secretName: + type: string + storageAccount: + type: string + type: object + br: + properties: + checkRequirements: + type: boolean + concurrency: + format: int32 + type: integer + options: + items: + type: string + type: array + sendCredToTikv: + type: boolean + type: object + calcSizeLevel: + default: all type: string - required: - - projectId - type: object - imagePullSecrets: - items: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - type: array - local: - properties: - prefix: + cleanPolicy: type: string - volume: + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + secretKeyRef: + properties: + key: + type: string + name: + type: string + optional: + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic + type: object + required: + - name + type: object + type: array + gcs: properties: - awsElasticBlockStore: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - azureDisk: - properties: - cachingMode: - type: string - diskName: - type: string - diskURI: - type: string - fsType: - type: string - kind: - type: string - readOnly: - type: boolean - required: - - diskName - - diskURI - type: object - azureFile: - properties: - readOnly: - type: boolean - secretName: - type: string - shareName: - type: string - required: - - secretName - - shareName - type: object - cephfs: + bucket: + type: string + bucketAcl: + type: string + location: + type: string + objectAcl: + type: string + path: + type: string + prefix: + type: string + projectId: + type: string + secretName: + type: string + storageClass: + type: string + required: + - projectId + type: object + imagePullSecrets: + items: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + type: array + local: + properties: + prefix: + type: string + volume: properties: - monitors: - items: - type: string - type: array - path: - type: string - readOnly: - type: boolean - secretFile: - type: string - secretRef: + awsElasticBlockStore: properties: - name: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: type: string + required: + - volumeID type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - monitors - type: object - cinder: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: + azureDisk: properties: - name: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI type: object - x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - csi: - properties: - driver: - type: string - fsType: - type: string - nodePublishSecretRef: + azureFile: properties: - name: + readOnly: + type: boolean + secretName: + type: string + shareName: type: string + required: + - secretName + - shareName type: object - x-kubernetes-map-type: atomic - readOnly: - type: boolean - volumeAttributes: - additionalProperties: - type: string + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors type: object - required: - - driver - type: object - downwardAPI: - properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - fieldRef: + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - apiVersion: + key: type: string - fieldPath: + mode: + format: int32 + type: integer + path: type: string required: - - fieldPath + - key + - path type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + readOnly: + type: boolean + volumeAttributes: + additionalProperties: type: string - resourceFieldRef: + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic required: - - resource + - path type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: - properties: - volumeClaimTemplate: + type: array + type: object + emptyDir: properties: - metadata: - type: object - spec: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: properties: - accessModes: - items: - type: string - type: array - dataSource: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - required: - - kind - - name - type: object - x-kubernetes-map-type: atomic - dataSourceRef: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - namespace: - type: string - required: - - kind - - name + metadata: type: object - resources: + spec: properties: - claims: + accessModes: items: - properties: - name: - type: string - required: - - name - type: object + type: string type: array - x-kubernetes-list-map-keys: + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true + x-kubernetes-map-type: atomic + dataSourceRef: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name type: object - type: object - selector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: + resources: + properties: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: - type: string + type: object type: object + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string type: object - x-kubernetes-map-type: atomic - storageClassName: - type: string - volumeMode: - type: string - volumeName: + required: + - spec + type: object + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string + type: object + readOnly: + type: boolean + secretRef: + properties: + name: type: string type: object + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean required: - - spec + - pdName type: object - type: object - fc: - properties: - fsType: - type: string - lun: - format: int32 - type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: - items: - type: string - type: array - type: object - flexVolume: - properties: - driver: - type: string - fsType: - type: string - options: - additionalProperties: - type: string + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository type: object - readOnly: - type: boolean - secretRef: + glusterfs: properties: - name: + endpoints: + type: string + path: type: string + readOnly: + type: boolean + required: + - endpoints + - path type: object - x-kubernetes-map-type: atomic - required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: - type: string - type: object - gcePersistentDisk: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - pdName: - type: string - readOnly: - type: boolean - required: - - pdName - type: object - gitRepo: - properties: - directory: - type: string - repository: - type: string - revision: - type: string - required: - - repository - type: object - glusterfs: - properties: - endpoints: - type: string - path: - type: string - readOnly: - type: boolean - required: - - endpoints - - path - type: object - hostPath: - properties: - path: - type: string - type: - type: string - required: - - path - type: object - iscsi: - properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: - type: string - lun: - format: int32 - type: integer - portals: - items: - type: string - type: array - readOnly: - type: boolean - secretRef: + hostPath: properties: - name: + path: + type: string + type: type: string + required: + - path type: object - x-kubernetes-map-type: atomic - targetPortal: - type: string - required: - - iqn - - lun - - targetPortal - type: object - name: - type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName - type: object - photonPersistentDisk: - properties: - fsType: - type: string - pdID: - type: string - required: - - pdID - type: object - portworxVolume: - properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - projected: - properties: - defaultMode: - format: int32 - type: integer - sources: - items: - properties: - configMap: - properties: - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - downwardAPI: + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: properties: - items: - items: - properties: - fieldRef: + configMap: + properties: + items: + items: properties: - apiVersion: + key: type: string - fieldPath: + mode: + format: int32 + type: integer + path: type: string required: - - fieldPath + - key + - path type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: - type: string - resourceFieldRef: + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: properties: - containerName: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: type: string required: - - resource + - key + - path type: object - x-kubernetes-map-type: atomic - required: - - path - type: object - type: array - type: object - secret: - properties: - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object type: object - x-kubernetes-map-type: atomic - serviceAccountToken: + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: properties: - audience: + key: type: string - expirationSeconds: - format: int64 + mode: + format: int32 type: integer path: type: string required: + - key - path type: object - type: object - type: array - type: object - quobyte: - properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: - type: string - required: - - registry - - volume - type: object - rbd: - properties: - fsType: - type: string - image: - type: string - keyring: - type: string - monitors: - items: - type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: properties: - name: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: type: string type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - image - - monitors - type: object - scaleIO: - properties: - fsType: - type: string - gateway: - type: string - protectionDomain: - type: string - readOnly: - type: boolean - secretRef: + vsphereVolume: properties: - name: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: type: string + volumePath: + type: string + required: + - volumePath type: object - x-kubernetes-map-type: atomic - sslEnabled: - type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string required: - - gateway - - secretRef - - system + - name type: object - secret: + volumeMount: properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - optional: - type: boolean - secretName: + mountPath: type: string - type: object - storageos: - properties: - fsType: + mountPropagation: + type: string + name: type: string readOnly: type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - volumeName: - type: string - volumeNamespace: - type: string - type: object - vsphereVolume: - properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: + subPath: type: string - volumePath: + subPathExpr: type: string required: - - volumePath + - mountPath + - name type: object required: - - name + - volume + - volumeMount type: object - volumeMount: + priorityClassName: + type: string + resources: properties: - mountPath: + claims: + items: + properties: + name: + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + s3: + properties: + acl: + type: string + bucket: + type: string + endpoint: + type: string + options: + items: + type: string + type: array + path: + type: string + prefix: + type: string + provider: type: string - mountPropagation: + region: type: string - name: + secretName: type: string - readOnly: - type: boolean - subPath: + sse: type: string - subPathExpr: + storageClass: type: string required: - - mountPath - - name + - provider type: object - required: - - volume - - volumeMount - type: object - priorityClassName: - type: string - resources: - properties: - claims: + serviceAccount: + type: string + snapshotsDeleteRatio: + default: 1 + type: number + tolerations: items: properties: - name: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + format: int64 + type: integer + value: type: string - required: - - name type: object type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - type: object - s3: - properties: - acl: - type: string - bucket: - type: string - endpoint: - type: string - options: - items: - type: string - type: array - path: - type: string - prefix: - type: string - provider: - type: string - region: - type: string - secretName: - type: string - sse: - type: string - storageClass: + toolImage: type: string - required: - - provider + volumeBackupInitJobMaxActiveSeconds: + default: 600 + type: integer type: object - serviceAccount: - type: string - snapshotsDeleteRatio: - default: 1 - type: number - tolerations: - items: - properties: - effect: - type: string - key: - type: string - operator: - type: string - tolerationSeconds: - format: int64 - type: integer - value: - type: string - type: object - type: array - toolImage: - type: string - volumeBackupInitJobMaxActiveSeconds: - default: 600 - type: integer type: object - type: object - status: - properties: - backupSize: - format: int64 + maxBackups: + format: int32 type: integer - backupSizeReadable: - type: string - backups: - items: - properties: - backupName: - type: string - backupPath: - type: string - backupSize: - format: int64 - type: integer - commitTs: - type: string - k8sClusterName: - type: string - message: - type: string - phase: - type: string - reason: - type: string - tcName: - type: string - tcNamespace: - type: string - required: - - backupName - - phase - type: object - type: array - commitTs: + maxReservedTime: type: string - conditions: - items: - properties: - lastTransitionTime: - format: date-time - nullable: true - type: string - message: - type: string - reason: - type: string - status: - type: string - type: - type: string - required: - - status - - type - type: object - nullable: true - type: array - phase: + pause: + type: boolean + schedule: type: string - timeCompleted: + required: + - backupTemplate + - schedule + type: object + status: + properties: + allBackupCleanTime: format: date-time - nullable: true type: string - timeStarted: - format: date-time - nullable: true + lastBackup: type: string - timeTaken: + lastBackupTime: + format: date-time type: string type: object required: @@ -1836,41 +1793,35 @@ kind: CustomResourceDefinition metadata: annotations: controller-gen.kubebuilder.io/version: v0.15.0 - name: volumebackupschedules.federation.pingcap.com + name: volumebackups.federation.pingcap.com spec: group: federation.pingcap.com names: - kind: VolumeBackupSchedule - listKind: VolumeBackupScheduleList - plural: volumebackupschedules + kind: VolumeBackup + listKind: VolumeBackupList + plural: volumebackups shortNames: - - vbks - singular: volumebackupschedule + - vbk + singular: volumebackup scope: Namespaced versions: - additionalPrinterColumns: - - description: The cron format string used for backup scheduling - jsonPath: .spec.schedule - name: Schedule + - description: The current status of the backup + jsonPath: .status.phase + name: Status type: string - - description: The max number of backups we want to keep - jsonPath: .spec.maxBackups - name: MaxBackups - type: integer - - description: How long backups we want to keep - jsonPath: .spec.maxReservedTime - name: MaxReservedTime + - description: The data size of the backup + jsonPath: .status.backupSizeReadable + name: BackupSize type: string - - description: The last backup CR name - jsonPath: .status.lastBackup - name: LastBackup - priority: 1 + - description: The commit ts of the backup + jsonPath: .status.commitTs + name: CommitTS + type: string + - description: The time that volume backup federation takes + jsonPath: .status.timeTaken + name: TimeTaken type: string - - description: The last time the backup was successfully created - jsonPath: .status.lastBackupTime - name: LastBackupTime - priority: 1 - type: date - jsonPath: .metadata.creationTimestamp name: Age type: date @@ -1884,1732 +1835,1781 @@ spec: type: string metadata: type: object + skipGC: + default: false + type: boolean spec: properties: - backupTemplate: + clusters: + items: + properties: + k8sClusterName: + type: string + tcName: + type: string + tcNamespace: + type: string + type: object + type: array + template: properties: - clusters: + additionalVolumeMounts: items: properties: - k8sClusterName: + mountPath: type: string - tcName: + mountPropagation: type: string - tcNamespace: + name: + type: string + readOnly: + type: boolean + subPath: + type: string + subPathExpr: type: string + required: + - mountPath + - name type: object type: array - template: - properties: - additionalVolumeMounts: - items: + additionalVolumes: + items: + properties: + awsElasticBlockStore: properties: - mountPath: + fsType: type: string - mountPropagation: + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: type: string - name: + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: type: string readOnly: type: boolean - subPath: + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: type: string - subPathExpr: + shareName: type: string required: - - mountPath - - name + - secretName + - shareName type: object - type: array - additionalVolumes: - items: + cephfs: properties: - awsElasticBlockStore: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - azureDisk: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: properties: - cachingMode: - type: string - diskName: - type: string - diskURI: - type: string - fsType: - type: string - kind: + name: type: string - readOnly: - type: boolean - required: - - diskName - - diskURI type: object - azureFile: + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: properties: - readOnly: - type: boolean - secretName: - type: string - shareName: + name: type: string - required: - - secretName - - shareName type: object - cephfs: - properties: - monitors: - items: + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: type: string - type: array - path: - type: string - readOnly: - type: boolean - secretFile: - type: string - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - monitors - type: object - cinder: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array name: type: string - optional: - type: boolean type: object x-kubernetes-map-type: atomic - csi: - properties: - driver: - type: string - fsType: - type: string - nodePublishSecretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - readOnly: - type: boolean - volumeAttributes: - additionalProperties: - type: string - type: object - required: - - driver + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string type: object - downwardAPI: - properties: - defaultMode: - format: int32 - type: integer - items: - items: + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + apiVersion: + type: string + fieldPath: type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic required: - - path + - fieldPath type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: + properties: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + type: string + required: + - resource + type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: properties: - volumeClaimTemplate: + metadata: + type: object + spec: properties: - metadata: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name type: object - spec: + x-kubernetes-map-type: atomic + dataSourceRef: properties: - accessModes: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: items: - type: string + properties: + name: + type: string + required: + - name + type: object type: array - dataSource: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - required: - - kind - - name - type: object - x-kubernetes-map-type: atomic - dataSourceRef: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - namespace: - type: string - required: - - kind + x-kubernetes-list-map-keys: - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true type: object - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true type: object - selector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: type: string - type: object + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string type: object - x-kubernetes-map-type: atomic - storageClassName: - type: string - volumeMode: - type: string - volumeName: - type: string type: object - required: - - spec - type: object - type: object - fc: - properties: - fsType: - type: string - lun: - format: int32 - type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: - items: - type: string - type: array - type: object - flexVolume: - properties: - driver: - type: string - fsType: - type: string - options: - additionalProperties: - type: string - type: object - readOnly: - type: boolean - secretRef: - properties: - name: + x-kubernetes-map-type: atomic + storageClassName: + type: string + volumeMode: + type: string + volumeName: type: string type: object - x-kubernetes-map-type: atomic - required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: - type: string - type: object - gcePersistentDisk: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - pdName: - type: string - readOnly: - type: boolean - required: - - pdName - type: object - gitRepo: - properties: - directory: - type: string - repository: - type: string - revision: - type: string - required: - - repository - type: object - glusterfs: - properties: - endpoints: - type: string - path: - type: string - readOnly: - type: boolean required: - - endpoints - - path + - spec type: object - hostPath: - properties: - path: - type: string - type: - type: string - required: - - path + type: object + fc: + properties: + fsType: + type: string + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string type: object - iscsi: + readOnly: + type: boolean + secretRef: properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: - type: string - lun: - format: int32 - type: integer - portals: - items: - type: string - type: array - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - targetPortal: + name: type: string - required: - - iqn - - lun - - targetPortal type: object - name: + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName - type: object - photonPersistentDisk: - properties: - fsType: - type: string - pdID: - type: string - required: - - pdID - type: object - portworxVolume: + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: + name: type: string - required: - - volumeID type: object - projected: - properties: - defaultMode: - format: int32 - type: integer - sources: - items: + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: + properties: + configMap: properties: - configMap: - properties: - items: - items: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: + properties: + fieldRef: properties: - key: - type: string - mode: - format: int32 - type: integer - path: + apiVersion: type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - downwardAPI: - properties: - items: - items: - properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + fieldPath: type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic required: - - path + - fieldPath type: object - type: array - type: object - secret: - properties: - items: - items: + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: properties: - key: + containerName: type: string - mode: - format: int32 - type: integer - path: + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: type: string required: - - key - - path + - resource type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - serviceAccountToken: - properties: - audience: - type: string - expirationSeconds: - format: int64 - type: integer - path: - type: string - required: - - path - type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array type: object - type: array - type: object - quobyte: + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + serviceAccountToken: + properties: + audience: + type: string + expirationSeconds: + format: int64 + type: integer + path: + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: + name: type: string - required: - - registry - - volume type: object - rbd: + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - image: - type: string - keyring: + name: type: string - monitors: - items: + type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - user: + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + storageos: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: + properties: + name: type: string - required: - - image - - monitors type: object - scaleIO: + x-kubernetes-map-type: atomic + volumeName: + type: string + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: + type: string + required: + - volumePath + type: object + required: + - name + type: object + type: array + azblob: + properties: + accessTier: + type: string + container: + type: string + path: + type: string + prefix: + type: string + sasToken: + type: string + secretName: + type: string + storageAccount: + type: string + type: object + br: + properties: + checkRequirements: + type: boolean + concurrency: + format: int32 + type: integer + options: + items: + type: string + type: array + sendCredToTikv: + type: boolean + type: object + calcSizeLevel: + default: all + type: string + cleanPolicy: + type: string + env: + items: + properties: + name: + type: string + value: + type: string + valueFrom: + properties: + configMapKeyRef: properties: - fsType: - type: string - gateway: + key: type: string - protectionDomain: + name: type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - sslEnabled: + optional: type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string required: - - gateway - - secretRef - - system + - key type: object - secret: + x-kubernetes-map-type: atomic + fieldRef: properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array - optional: - type: boolean - secretName: + apiVersion: + type: string + fieldPath: type: string + required: + - fieldPath type: object - storageos: + x-kubernetes-map-type: atomic + resourceFieldRef: properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - volumeName: + containerName: type: string - volumeNamespace: + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: type: string + required: + - resource type: object - vsphereVolume: + x-kubernetes-map-type: atomic + secretKeyRef: properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: + key: type: string - volumePath: + name: type: string + optional: + type: boolean required: - - volumePath + - key type: object - required: - - name + x-kubernetes-map-type: atomic type: object - type: array - azblob: - properties: - accessTier: - type: string - container: - type: string - path: - type: string - prefix: - type: string - sasToken: - type: string - secretName: - type: string - storageAccount: - type: string - type: object - br: - properties: - checkRequirements: - type: boolean - concurrency: - format: int32 - type: integer - options: - items: - type: string - type: array - sendCredToTikv: - type: boolean - type: object - calcSizeLevel: - default: all + required: + - name + type: object + type: array + gcs: + properties: + bucket: type: string - cleanPolicy: + bucketAcl: type: string - env: - items: - properties: - name: - type: string - value: - type: string - valueFrom: - properties: - configMapKeyRef: - properties: - key: - type: string - name: - type: string - optional: - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic - secretKeyRef: - properties: - key: - type: string - name: - type: string - optional: - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic - type: object - required: - - name - type: object - type: array - gcs: - properties: - bucket: - type: string - bucketAcl: - type: string - location: - type: string - objectAcl: - type: string - path: - type: string - prefix: - type: string - projectId: - type: string - secretName: - type: string - storageClass: - type: string - required: - - projectId - type: object - imagePullSecrets: - items: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - type: array - local: + location: + type: string + objectAcl: + type: string + path: + type: string + prefix: + type: string + projectId: + type: string + secretName: + type: string + storageClass: + type: string + required: + - projectId + type: object + imagePullSecrets: + items: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + type: array + local: + properties: + prefix: + type: string + volume: properties: - prefix: - type: string - volume: + awsElasticBlockStore: properties: - awsElasticBlockStore: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - readOnly: - type: boolean - volumeID: - type: string - required: - - volumeID - type: object - azureDisk: + fsType: + type: string + partition: + format: int32 + type: integer + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + azureDisk: + properties: + cachingMode: + type: string + diskName: + type: string + diskURI: + type: string + fsType: + type: string + kind: + type: string + readOnly: + type: boolean + required: + - diskName + - diskURI + type: object + azureFile: + properties: + readOnly: + type: boolean + secretName: + type: string + shareName: + type: string + required: + - secretName + - shareName + type: object + cephfs: + properties: + monitors: + items: + type: string + type: array + path: + type: string + readOnly: + type: boolean + secretFile: + type: string + secretRef: properties: - cachingMode: - type: string - diskName: - type: string - diskURI: - type: string - fsType: - type: string - kind: + name: type: string - readOnly: - type: boolean - required: - - diskName - - diskURI type: object - azureFile: + x-kubernetes-map-type: atomic + user: + type: string + required: + - monitors + type: object + cinder: + properties: + fsType: + type: string + readOnly: + type: boolean + secretRef: properties: - readOnly: - type: boolean - secretName: - type: string - shareName: + name: type: string - required: - - secretName - - shareName type: object - cephfs: - properties: - monitors: - items: + x-kubernetes-map-type: atomic + volumeID: + type: string + required: + - volumeID + type: object + configMap: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: type: string - type: array - path: - type: string - readOnly: - type: boolean - secretFile: - type: string - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - monitors - type: object - cinder: - properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - volumeID: - type: string - required: - - volumeID - type: object - configMap: + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + csi: + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: properties: - defaultMode: - format: int32 - type: integer - items: - items: - properties: - key: - type: string - mode: - format: int32 - type: integer - path: - type: string - required: - - key - - path - type: object - type: array name: type: string - optional: - type: boolean type: object x-kubernetes-map-type: atomic - csi: - properties: - driver: - type: string - fsType: - type: string - nodePublishSecretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - readOnly: - type: boolean - volumeAttributes: - additionalProperties: + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + required: + - driver + type: object + downwardAPI: + properties: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + fieldRef: + properties: + apiVersion: + type: string + fieldPath: + type: string + required: + - fieldPath + type: object + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: type: string - type: object - required: - - driver - type: object - downwardAPI: - properties: - defaultMode: - format: int32 - type: integer - items: - items: + resourceFieldRef: properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + containerName: + type: string + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic required: - - path + - resource type: object - type: array - type: object - emptyDir: - properties: - medium: - type: string - sizeLimit: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - ephemeral: + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + emptyDir: + properties: + medium: + type: string + sizeLimit: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + ephemeral: + properties: + volumeClaimTemplate: properties: - volumeClaimTemplate: + metadata: + type: object + spec: properties: - metadata: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name type: object - spec: + x-kubernetes-map-type: atomic + dataSourceRef: properties: - accessModes: + apiGroup: + type: string + kind: + type: string + name: + type: string + namespace: + type: string + required: + - kind + - name + type: object + resources: + properties: + claims: items: - type: string + properties: + name: + type: string + required: + - name + type: object type: array - dataSource: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - required: - - kind - - name - type: object - x-kubernetes-map-type: atomic - dataSourceRef: - properties: - apiGroup: - type: string - kind: - type: string - name: - type: string - namespace: - type: string - required: - - kind + x-kubernetes-list-map-keys: - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true type: object - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true type: object - selector: - properties: - matchExpressions: - items: - properties: - key: - type: string - operator: - type: string - values: - items: - type: string - type: array - required: - - key - - operator - type: object - type: array - matchLabels: - additionalProperties: + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: type: string - type: object + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string type: object - x-kubernetes-map-type: atomic - storageClassName: - type: string - volumeMode: - type: string - volumeName: - type: string type: object - required: - - spec - type: object - type: object - fc: - properties: - fsType: - type: string - lun: - format: int32 - type: integer - readOnly: - type: boolean - targetWWNs: - items: - type: string - type: array - wwids: - items: - type: string - type: array - type: object - flexVolume: - properties: - driver: - type: string - fsType: - type: string - options: - additionalProperties: - type: string - type: object - readOnly: - type: boolean - secretRef: - properties: - name: + x-kubernetes-map-type: atomic + storageClassName: type: string - type: object - x-kubernetes-map-type: atomic - required: - - driver - type: object - flocker: - properties: - datasetName: - type: string - datasetUUID: - type: string - type: object - gcePersistentDisk: - properties: - fsType: - type: string - partition: - format: int32 - type: integer - pdName: - type: string - readOnly: - type: boolean - required: - - pdName - type: object - gitRepo: - properties: - directory: - type: string - repository: - type: string - revision: - type: string - required: - - repository - type: object - glusterfs: - properties: - endpoints: - type: string - path: - type: string - readOnly: - type: boolean - required: - - endpoints - - path - type: object - hostPath: - properties: - path: - type: string - type: - type: string - required: - - path - type: object - iscsi: - properties: - chapAuthDiscovery: - type: boolean - chapAuthSession: - type: boolean - fsType: - type: string - initiatorName: - type: string - iqn: - type: string - iscsiInterface: - type: string - lun: - format: int32 - type: integer - portals: - items: - type: string - type: array - readOnly: - type: boolean - secretRef: - properties: - name: + volumeMode: + type: string + volumeName: type: string type: object - x-kubernetes-map-type: atomic - targetPortal: - type: string required: - - iqn - - lun - - targetPortal + - spec type: object - name: + type: object + fc: + properties: + fsType: type: string - nfs: - properties: - path: - type: string - readOnly: - type: boolean - server: - type: string - required: - - path - - server - type: object - persistentVolumeClaim: - properties: - claimName: - type: string - readOnly: - type: boolean - required: - - claimName + lun: + format: int32 + type: integer + readOnly: + type: boolean + targetWWNs: + items: + type: string + type: array + wwids: + items: + type: string + type: array + type: object + flexVolume: + properties: + driver: + type: string + fsType: + type: string + options: + additionalProperties: + type: string type: object - photonPersistentDisk: + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - pdID: + name: type: string - required: - - pdID type: object - portworxVolume: + x-kubernetes-map-type: atomic + required: + - driver + type: object + flocker: + properties: + datasetName: + type: string + datasetUUID: + type: string + type: object + gcePersistentDisk: + properties: + fsType: + type: string + partition: + format: int32 + type: integer + pdName: + type: string + readOnly: + type: boolean + required: + - pdName + type: object + gitRepo: + properties: + directory: + type: string + repository: + type: string + revision: + type: string + required: + - repository + type: object + glusterfs: + properties: + endpoints: + type: string + path: + type: string + readOnly: + type: boolean + required: + - endpoints + - path + type: object + hostPath: + properties: + path: + type: string + type: + type: string + required: + - path + type: object + iscsi: + properties: + chapAuthDiscovery: + type: boolean + chapAuthSession: + type: boolean + fsType: + type: string + initiatorName: + type: string + iqn: + type: string + iscsiInterface: + type: string + lun: + format: int32 + type: integer + portals: + items: + type: string + type: array + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - readOnly: - type: boolean - volumeID: + name: type: string - required: - - volumeID type: object - projected: - properties: - defaultMode: - format: int32 - type: integer - sources: - items: + x-kubernetes-map-type: atomic + targetPortal: + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + type: string + nfs: + properties: + path: + type: string + readOnly: + type: boolean + server: + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + properties: + claimName: + type: string + readOnly: + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + properties: + fsType: + type: string + pdID: + type: string + required: + - pdID + type: object + portworxVolume: + properties: + fsType: + type: string + readOnly: + type: boolean + volumeID: + type: string + required: + - volumeID + type: object + projected: + properties: + defaultMode: + format: int32 + type: integer + sources: + items: + properties: + configMap: properties: - configMap: - properties: - items: - items: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + x-kubernetes-map-type: atomic + downwardAPI: + properties: + items: + items: + properties: + fieldRef: properties: - key: - type: string - mode: - format: int32 - type: integer - path: + apiVersion: type: string - required: - - key - - path - type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - downwardAPI: - properties: - items: - items: - properties: - fieldRef: - properties: - apiVersion: - type: string - fieldPath: - type: string - required: - - fieldPath - type: object - x-kubernetes-map-type: atomic - mode: - format: int32 - type: integer - path: + fieldPath: type: string - resourceFieldRef: - properties: - containerName: - type: string - divisor: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - resource: - type: string - required: - - resource - type: object - x-kubernetes-map-type: atomic required: - - path + - fieldPath type: object - type: array - type: object - secret: - properties: - items: - items: + x-kubernetes-map-type: atomic + mode: + format: int32 + type: integer + path: + type: string + resourceFieldRef: properties: - key: + containerName: type: string - mode: - format: int32 - type: integer - path: + divisor: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: type: string required: - - key - - path + - resource type: object - type: array - name: - type: string - optional: - type: boolean - type: object - x-kubernetes-map-type: atomic - serviceAccountToken: - properties: - audience: - type: string - expirationSeconds: - format: int64 - type: integer - path: - type: string - required: - - path - type: object + x-kubernetes-map-type: atomic + required: + - path + type: object + type: array + type: object + secret: + properties: + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + name: + type: string + optional: + type: boolean type: object - type: array - type: object - quobyte: - properties: - group: - type: string - readOnly: - type: boolean - registry: - type: string - tenant: - type: string - user: - type: string - volume: - type: string - required: - - registry - - volume - type: object - rbd: - properties: - fsType: - type: string - image: - type: string - keyring: - type: string - monitors: - items: - type: string - type: array - pool: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - user: - type: string - required: - - image - - monitors - type: object - scaleIO: - properties: - fsType: - type: string - gateway: - type: string - protectionDomain: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - sslEnabled: - type: boolean - storageMode: - type: string - storagePool: - type: string - system: - type: string - volumeName: - type: string - required: - - gateway - - secretRef - - system - type: object - secret: - properties: - defaultMode: - format: int32 - type: integer - items: - items: + x-kubernetes-map-type: atomic + serviceAccountToken: properties: - key: + audience: type: string - mode: - format: int32 + expirationSeconds: + format: int64 type: integer path: type: string required: - - key - path type: object - type: array - optional: - type: boolean - secretName: - type: string - type: object - storageos: + type: object + type: array + type: object + quobyte: + properties: + group: + type: string + readOnly: + type: boolean + registry: + type: string + tenant: + type: string + user: + type: string + volume: + type: string + required: + - registry + - volume + type: object + rbd: + properties: + fsType: + type: string + image: + type: string + keyring: + type: string + monitors: + items: + type: string + type: array + pool: + type: string + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - readOnly: - type: boolean - secretRef: - properties: - name: - type: string - type: object - x-kubernetes-map-type: atomic - volumeName: - type: string - volumeNamespace: + name: type: string type: object - vsphereVolume: + x-kubernetes-map-type: atomic + user: + type: string + required: + - image + - monitors + type: object + scaleIO: + properties: + fsType: + type: string + gateway: + type: string + protectionDomain: + type: string + readOnly: + type: boolean + secretRef: properties: - fsType: - type: string - storagePolicyID: - type: string - storagePolicyName: - type: string - volumePath: + name: type: string - required: - - volumePath type: object + x-kubernetes-map-type: atomic + sslEnabled: + type: boolean + storageMode: + type: string + storagePool: + type: string + system: + type: string + volumeName: + type: string required: - - name + - gateway + - secretRef + - system type: object - volumeMount: + secret: properties: - mountPath: - type: string - mountPropagation: + defaultMode: + format: int32 + type: integer + items: + items: + properties: + key: + type: string + mode: + format: int32 + type: integer + path: + type: string + required: + - key + - path + type: object + type: array + optional: + type: boolean + secretName: type: string - name: + type: object + storageos: + properties: + fsType: type: string readOnly: type: boolean - subPath: + secretRef: + properties: + name: + type: string + type: object + x-kubernetes-map-type: atomic + volumeName: type: string - subPathExpr: + volumeNamespace: + type: string + type: object + vsphereVolume: + properties: + fsType: + type: string + storagePolicyID: + type: string + storagePolicyName: + type: string + volumePath: type: string required: - - mountPath - - name + - volumePath type: object required: - - volume - - volumeMount - type: object - priorityClassName: - type: string - resources: - properties: - claims: - items: - properties: - name: - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - type: object + - name type: object - s3: + volumeMount: properties: - acl: - type: string - bucket: - type: string - endpoint: - type: string - options: - items: - type: string - type: array - path: - type: string - prefix: - type: string - provider: + mountPath: type: string - region: + mountPropagation: type: string - secretName: + name: type: string - sse: + readOnly: + type: boolean + subPath: type: string - storageClass: + subPathExpr: type: string required: - - provider + - mountPath + - name type: object - serviceAccount: - type: string - snapshotsDeleteRatio: - default: 1 - type: number - tolerations: + required: + - volume + - volumeMount + type: object + priorityClassName: + type: string + resources: + properties: + claims: items: properties: - effect: - type: string - key: - type: string - operator: - type: string - tolerationSeconds: - format: int64 - type: integer - value: + name: type: string + required: + - name type: object type: array - toolImage: + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + s3: + properties: + acl: type: string - volumeBackupInitJobMaxActiveSeconds: - default: 600 - type: integer + bucket: + type: string + endpoint: + type: string + options: + items: + type: string + type: array + path: + type: string + prefix: + type: string + provider: + type: string + region: + type: string + secretName: + type: string + sse: + type: string + storageClass: + type: string + required: + - provider type: object + serviceAccount: + type: string + snapshotsDeleteRatio: + default: 1 + type: number + tolerations: + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + format: int64 + type: integer + value: + type: string + type: object + type: array + toolImage: + type: string + volumeBackupInitJobMaxActiveSeconds: + default: 600 + type: integer type: object - maxBackups: - format: int32 - type: integer - maxReservedTime: - type: string - pause: - type: boolean - schedule: - type: string - required: - - backupTemplate - - schedule type: object status: properties: - allBackupCleanTime: - format: date-time + backupSize: + format: int64 + type: integer + backupSizeReadable: type: string - lastBackup: + backups: + items: + properties: + backupName: + type: string + backupPath: + type: string + backupSize: + format: int64 + type: integer + commitTs: + type: string + k8sClusterName: + type: string + message: + type: string + phase: + type: string + reason: + type: string + tcName: + type: string + tcNamespace: + type: string + required: + - backupName + - phase + type: object + type: array + commitTs: type: string - lastBackupTime: + conditions: + items: + properties: + lastTransitionTime: + format: date-time + nullable: true + type: string + message: + type: string + reason: + type: string + status: + type: string + type: + type: string + required: + - status + - type + type: object + nullable: true + type: array + phase: + type: string + timeCompleted: + format: date-time + nullable: true + type: string + timeStarted: format: date-time + nullable: true + type: string + timeTaken: type: string type: object required: diff --git a/pkg/apis/pingcap/v1alpha1/openapi_generated.go b/pkg/apis/pingcap/v1alpha1/openapi_generated.go index ee8ef1dad71..50ddb3e2fe4 100644 --- a/pkg/apis/pingcap/v1alpha1/openapi_generated.go +++ b/pkg/apis/pingcap/v1alpha1/openapi_generated.go @@ -46,6 +46,9 @@ func GetOpenAPIDefinitions(ref common.ReferenceCallback) map[string]common.OpenA "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CleanOption": schema_pkg_apis_pingcap_v1alpha1_CleanOption(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.ClusterRef": schema_pkg_apis_pingcap_v1alpha1_ClusterRef(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CommonConfig": schema_pkg_apis_pingcap_v1alpha1_CommonConfig(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CompactBackup": schema_pkg_apis_pingcap_v1alpha1_CompactBackup(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CompactBackupList": schema_pkg_apis_pingcap_v1alpha1_CompactBackupList(ref), + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CompactSpec": schema_pkg_apis_pingcap_v1alpha1_CompactSpec(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.ComponentSpec": schema_pkg_apis_pingcap_v1alpha1_ComponentSpec(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.ConfigMapRef": schema_pkg_apis_pingcap_v1alpha1_ConfigMapRef(ref), "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.DMCluster": schema_pkg_apis_pingcap_v1alpha1_DMCluster(ref), @@ -1621,6 +1624,269 @@ func schema_pkg_apis_pingcap_v1alpha1_CommonConfig(ref common.ReferenceCallback) } } +func schema_pkg_apis_pingcap_v1alpha1_CompactBackup(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "kind": { + SchemaProps: spec.SchemaProps{ + Description: "Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds", + Type: []string{"string"}, + Format: "", + }, + }, + "apiVersion": { + SchemaProps: spec.SchemaProps{ + Description: "APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources", + Type: []string{"string"}, + Format: "", + }, + }, + "spec": { + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CompactSpec"), + }, + }, + }, + Required: []string{"spec"}, + }, + }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CompactSpec"}, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_CompactBackupList(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "CompactList contains a list of Compact Backup.", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "kind": { + SchemaProps: spec.SchemaProps{ + Description: "Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds", + Type: []string{"string"}, + Format: "", + }, + }, + "apiVersion": { + SchemaProps: spec.SchemaProps{ + Description: "APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources", + Type: []string{"string"}, + Format: "", + }, + }, + "items": { + SchemaProps: spec.SchemaProps{ + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CompactBackup"), + }, + }, + }, + }, + }, + }, + Required: []string{"items"}, + }, + }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.CompactBackup"}, + } +} + +func schema_pkg_apis_pingcap_v1alpha1_CompactSpec(ref common.ReferenceCallback) common.OpenAPIDefinition { + return common.OpenAPIDefinition{ + Schema: spec.Schema{ + SchemaProps: spec.SchemaProps{ + Description: "CompactSpec contains the backup specification for a tidb cluster.", + Type: []string{"object"}, + Properties: map[string]spec.Schema{ + "resources": { + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("k8s.io/api/core/v1.ResourceRequirements"), + }, + }, + "env": { + SchemaProps: spec.SchemaProps{ + Description: "List of environment variables to set in the container, like v1.Container.Env. Note that the following builtin env vars will be overwritten by values set here - S3_PROVIDER - S3_ENDPOINT - AWS_REGION - AWS_ACL - AWS_STORAGE_CLASS - AWS_DEFAULT_REGION - AWS_ACCESS_KEY_ID - AWS_SECRET_ACCESS_KEY - GCS_PROJECT_ID - GCS_OBJECT_ACL - GCS_BUCKET_ACL - GCS_LOCATION - GCS_STORAGE_CLASS - GCS_SERVICE_ACCOUNT_JSON_KEY - BR_LOG_TO_TERM", + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("k8s.io/api/core/v1.EnvVar"), + }, + }, + }, + }, + }, + "s3": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.S3StorageProvider"), + }, + }, + "gcs": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.GcsStorageProvider"), + }, + }, + "azblob": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.AzblobStorageProvider"), + }, + }, + "local": { + SchemaProps: spec.SchemaProps{ + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.LocalStorageProvider"), + }, + }, + "startTs": { + SchemaProps: spec.SchemaProps{ + Description: "StartTs is the start ts of the compact backup. Format supports TSO or datetime, e.g. '400036290571534337', '2018-05-11 01:42:23'.", + Type: []string{"string"}, + Format: "", + }, + }, + "endTs": { + SchemaProps: spec.SchemaProps{ + Description: "EndTs is the end ts of the compact backup. Format supports TSO or datetime, e.g. '400036290571534337', '2018-05-11 01:42:23'. Default is current timestamp.", + Type: []string{"string"}, + Format: "", + }, + }, + "concurrency": { + SchemaProps: spec.SchemaProps{ + Description: "Concurrency is the concurrency of compact backup job", + Type: []string{"integer"}, + Format: "int32", + }, + }, + "tolerations": { + SchemaProps: spec.SchemaProps{ + Description: "Base tolerations of backup Pods, components may add more tolerations upon this respectively", + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("k8s.io/api/core/v1.Toleration"), + }, + }, + }, + }, + }, + "toolImage": { + SchemaProps: spec.SchemaProps{ + Description: "BrImage specifies the br image used in compact `Backup`. For examples `spec.brImage: pingcap/br:v4.0.8` For BR image, if it does not contain tag, Pod will use image 'BrImage:${TiKV_Version}'.", + Type: []string{"string"}, + Format: "", + }, + }, + "br": { + SchemaProps: spec.SchemaProps{ + Description: "BRConfig is the configs for BR", + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.BRConfig"), + }, + }, + "imagePullSecrets": { + SchemaProps: spec.SchemaProps{ + Description: "ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images.", + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("k8s.io/api/core/v1.LocalObjectReference"), + }, + }, + }, + }, + }, + "affinity": { + SchemaProps: spec.SchemaProps{ + Description: "Affinity of backup Pods", + Ref: ref("k8s.io/api/core/v1.Affinity"), + }, + }, + "useKMS": { + SchemaProps: spec.SchemaProps{ + Description: "Use KMS to decrypt the secrets", + Type: []string{"boolean"}, + Format: "", + }, + }, + "serviceAccount": { + SchemaProps: spec.SchemaProps{ + Description: "Specify service account of backup", + Type: []string{"string"}, + Format: "", + }, + }, + "podSecurityContext": { + SchemaProps: spec.SchemaProps{ + Description: "PodSecurityContext of the component", + Ref: ref("k8s.io/api/core/v1.PodSecurityContext"), + }, + }, + "priorityClassName": { + SchemaProps: spec.SchemaProps{ + Description: "PriorityClassName of Backup Job Pods", + Type: []string{"string"}, + Format: "", + }, + }, + "maxRetryTimes": { + SchemaProps: spec.SchemaProps{ + Description: "BackoffRetryPolicy the backoff retry policy, currently only valid for snapshot backup", + Type: []string{"integer"}, + Format: "int32", + }, + }, + "additionalVolumes": { + SchemaProps: spec.SchemaProps{ + Description: "Additional volumes of component pod.", + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("k8s.io/api/core/v1.Volume"), + }, + }, + }, + }, + }, + "additionalVolumeMounts": { + SchemaProps: spec.SchemaProps{ + Description: "Additional volume mounts of component pod.", + Type: []string{"array"}, + Items: &spec.SchemaOrArray{ + Schema: &spec.Schema{ + SchemaProps: spec.SchemaProps{ + Default: map[string]interface{}{}, + Ref: ref("k8s.io/api/core/v1.VolumeMount"), + }, + }, + }, + }, + }, + }, + }, + }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.AzblobStorageProvider", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.BRConfig", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.GcsStorageProvider", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.LocalStorageProvider", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.S3StorageProvider", "k8s.io/api/core/v1.Affinity", "k8s.io/api/core/v1.EnvVar", "k8s.io/api/core/v1.LocalObjectReference", "k8s.io/api/core/v1.PodSecurityContext", "k8s.io/api/core/v1.ResourceRequirements", "k8s.io/api/core/v1.Toleration", "k8s.io/api/core/v1.Volume", "k8s.io/api/core/v1.VolumeMount"}, + } +} + func schema_pkg_apis_pingcap_v1alpha1_ComponentSpec(ref common.ReferenceCallback) common.OpenAPIDefinition { return common.OpenAPIDefinition{ Schema: spec.Schema{ diff --git a/pkg/apis/pingcap/v1alpha1/register.go b/pkg/apis/pingcap/v1alpha1/register.go index 2b98a786e6b..ac7de27a0ae 100644 --- a/pkg/apis/pingcap/v1alpha1/register.go +++ b/pkg/apis/pingcap/v1alpha1/register.go @@ -53,6 +53,8 @@ func addKnownTypes(scheme *runtime.Scheme) error { &TidbClusterList{}, &Backup{}, &BackupList{}, + &CompactBackup{}, + &CompactBackupList{}, &BackupSchedule{}, &BackupScheduleList{}, &Restore{}, diff --git a/pkg/apis/pingcap/v1alpha1/types.go b/pkg/apis/pingcap/v1alpha1/types.go index 1f204201e3a..6fb32f99e5f 100644 --- a/pkg/apis/pingcap/v1alpha1/types.go +++ b/pkg/apis/pingcap/v1alpha1/types.go @@ -3457,3 +3457,129 @@ type ScalePolicy struct { // +optional ScaleOutParallelism *int32 `json:"scaleOutParallelism,omitempty"` } + +// +genclient +// +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object +// +k8s:openapi-gen=true +// +kubebuilder:resource:shortName="cpbk" +// +kubebuilder:printcolumn:name="Status",type=string,JSONPath=`.status.state`,description="The current status of the compact backup" +// +kubebuilder:printcolumn:name="Progress",type=string,JSONPath=`.status.progress`,description="The progress of the compact backup" +// +kubebuilder:printcolumn:name="Message",type=string,JSONPath=`.status.message`,description="The message of the compact backup" +type CompactBackup struct { + metav1.TypeMeta `json:",inline"` + // +k8s:openapi-gen=false + metav1.ObjectMeta `json:"metadata"` + + Spec CompactSpec `json:"spec"` + // +k8s:openapi-gen=false + Status CompactStatus `json:"status,omitempty"` +} + +// CompactSpec contains the backup specification for a tidb cluster. +// +k8s:openapi-gen=true +type CompactSpec struct { + corev1.ResourceRequirements `json:"resources,omitempty"` + // List of environment variables to set in the container, like v1.Container.Env. + // Note that the following builtin env vars will be overwritten by values set here + // - S3_PROVIDER + // - S3_ENDPOINT + // - AWS_REGION + // - AWS_ACL + // - AWS_STORAGE_CLASS + // - AWS_DEFAULT_REGION + // - AWS_ACCESS_KEY_ID + // - AWS_SECRET_ACCESS_KEY + // - GCS_PROJECT_ID + // - GCS_OBJECT_ACL + // - GCS_BUCKET_ACL + // - GCS_LOCATION + // - GCS_STORAGE_CLASS + // - GCS_SERVICE_ACCOUNT_JSON_KEY + // - BR_LOG_TO_TERM + // +optional + Env []corev1.EnvVar `json:"env,omitempty"` + // StorageProvider configures where and how backups should be stored. + StorageProvider `json:",inline"` + // StartTs is the start ts of the compact backup. + // Format supports TSO or datetime, e.g. '400036290571534337', '2018-05-11 01:42:23'. + StartTs string `json:"startTs,omitempty"` + // EndTs is the end ts of the compact backup. + // Format supports TSO or datetime, e.g. '400036290571534337', '2018-05-11 01:42:23'. + // Default is current timestamp. + // +optional + EndTs string `json:"endTs,omitempty"` + // Concurrency is the concurrency of compact backup job + // +kubebuilder:default=4 + Concurrency int `json:"concurrency,omitempty"` + // Base tolerations of backup Pods, components may add more tolerations upon this respectively + // +optional + Tolerations []corev1.Toleration `json:"tolerations,omitempty"` + // BrImage specifies the br image used in compact `Backup`. + // For examples `spec.brImage: pingcap/br:v4.0.8` + // For BR image, if it does not contain tag, Pod will use image 'BrImage:${TiKV_Version}'. + // +optional + ToolImage string `json:"toolImage,omitempty"` + // BRConfig is the configs for BR + BR *BRConfig `json:"br,omitempty"` + // ImagePullSecrets is an optional list of references to secrets in the same namespace to use for pulling any of the images. + // +optional + ImagePullSecrets []corev1.LocalObjectReference `json:"imagePullSecrets,omitempty"` + // Affinity of backup Pods + // +optional + Affinity *corev1.Affinity `json:"affinity,omitempty"` + // Use KMS to decrypt the secrets + UseKMS bool `json:"useKMS,omitempty"` + // Specify service account of backup + ServiceAccount string `json:"serviceAccount,omitempty"` + + // PodSecurityContext of the component + // +optional + PodSecurityContext *corev1.PodSecurityContext `json:"podSecurityContext,omitempty"` + + // PriorityClassName of Backup Job Pods + PriorityClassName string `json:"priorityClassName,omitempty"` + + // BackoffRetryPolicy the backoff retry policy, currently only valid for snapshot backup + // +kubebuilder:default=6 + MaxRetryTimes int32 `json:"maxRetryTimes,omitempty"` + + // Additional volumes of component pod. + // +optional + AdditionalVolumes []corev1.Volume `json:"additionalVolumes,omitempty"` + // Additional volume mounts of component pod. + // +optional + AdditionalVolumeMounts []corev1.VolumeMount `json:"additionalVolumeMounts,omitempty"` +} + +// CompactRetryRecord is the record of compact backoff retry +type CompactRetryRecord struct { + // RetryNum is the number of retry + RetryNum int `json:"retryNum,omitempty"` + // DetectFailedAt is the time when detect failure + DetectFailedAt metav1.Time `json:"detectFailedAt,omitempty"` + // Reason is the reason of retry + RetryReason string `json:"retryReason,omitempty"` +} + +type CompactStatus struct { + // State is the current state of the backup + State string `json:"state,omitempty"` + // Progress is the progress of the backup + Progress string `json:"progress,omitempty"` + // Message is the error message of the backup + Message string `json:"message,omitempty"` + // RetryStatus is status of the backoff retry, it will be used when backup pod or job exited unexpectedly + RetryStatus []CompactRetryRecord `json:"backoffRetryStatus,omitempty"` +} + +// +k8s:deepcopy-gen:interfaces=k8s.io/apimachinery/pkg/runtime.Object + +// +k8s:openapi-gen=true +// CompactList contains a list of Compact Backup. +type CompactBackupList struct { + metav1.TypeMeta `json:",inline"` + // +k8s:openapi-gen=false + metav1.ListMeta `json:"metadata"` + + Items []CompactBackup `json:"items"` +} diff --git a/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go b/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go index 7cd8c5d5ef7..f472d006c05 100644 --- a/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go +++ b/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go @@ -865,6 +865,173 @@ func (in *CommonConfig) DeepCopy() *CommonConfig { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *CompactBackup) DeepCopyInto(out *CompactBackup) { + *out = *in + out.TypeMeta = in.TypeMeta + in.ObjectMeta.DeepCopyInto(&out.ObjectMeta) + in.Spec.DeepCopyInto(&out.Spec) + in.Status.DeepCopyInto(&out.Status) + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new CompactBackup. +func (in *CompactBackup) DeepCopy() *CompactBackup { + if in == nil { + return nil + } + out := new(CompactBackup) + in.DeepCopyInto(out) + return out +} + +// DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object. +func (in *CompactBackup) DeepCopyObject() runtime.Object { + if c := in.DeepCopy(); c != nil { + return c + } + return nil +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *CompactBackupList) DeepCopyInto(out *CompactBackupList) { + *out = *in + out.TypeMeta = in.TypeMeta + in.ListMeta.DeepCopyInto(&out.ListMeta) + if in.Items != nil { + in, out := &in.Items, &out.Items + *out = make([]CompactBackup, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new CompactBackupList. +func (in *CompactBackupList) DeepCopy() *CompactBackupList { + if in == nil { + return nil + } + out := new(CompactBackupList) + in.DeepCopyInto(out) + return out +} + +// DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object. +func (in *CompactBackupList) DeepCopyObject() runtime.Object { + if c := in.DeepCopy(); c != nil { + return c + } + return nil +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *CompactRetryRecord) DeepCopyInto(out *CompactRetryRecord) { + *out = *in + in.DetectFailedAt.DeepCopyInto(&out.DetectFailedAt) + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new CompactRetryRecord. +func (in *CompactRetryRecord) DeepCopy() *CompactRetryRecord { + if in == nil { + return nil + } + out := new(CompactRetryRecord) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *CompactSpec) DeepCopyInto(out *CompactSpec) { + *out = *in + in.ResourceRequirements.DeepCopyInto(&out.ResourceRequirements) + if in.Env != nil { + in, out := &in.Env, &out.Env + *out = make([]v1.EnvVar, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + in.StorageProvider.DeepCopyInto(&out.StorageProvider) + if in.Tolerations != nil { + in, out := &in.Tolerations, &out.Tolerations + *out = make([]v1.Toleration, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + if in.BR != nil { + in, out := &in.BR, &out.BR + *out = new(BRConfig) + (*in).DeepCopyInto(*out) + } + if in.ImagePullSecrets != nil { + in, out := &in.ImagePullSecrets, &out.ImagePullSecrets + *out = make([]v1.LocalObjectReference, len(*in)) + copy(*out, *in) + } + if in.Affinity != nil { + in, out := &in.Affinity, &out.Affinity + *out = new(v1.Affinity) + (*in).DeepCopyInto(*out) + } + if in.PodSecurityContext != nil { + in, out := &in.PodSecurityContext, &out.PodSecurityContext + *out = new(v1.PodSecurityContext) + (*in).DeepCopyInto(*out) + } + if in.AdditionalVolumes != nil { + in, out := &in.AdditionalVolumes, &out.AdditionalVolumes + *out = make([]v1.Volume, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + if in.AdditionalVolumeMounts != nil { + in, out := &in.AdditionalVolumeMounts, &out.AdditionalVolumeMounts + *out = make([]v1.VolumeMount, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new CompactSpec. +func (in *CompactSpec) DeepCopy() *CompactSpec { + if in == nil { + return nil + } + out := new(CompactSpec) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *CompactStatus) DeepCopyInto(out *CompactStatus) { + *out = *in + if in.RetryStatus != nil { + in, out := &in.RetryStatus, &out.RetryStatus + *out = make([]CompactRetryRecord, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + return +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new CompactStatus. +func (in *CompactStatus) DeepCopy() *CompactStatus { + if in == nil { + return nil + } + out := new(CompactStatus) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *ComponentSpec) DeepCopyInto(out *ComponentSpec) { *out = *in diff --git a/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/compactbackup.go b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/compactbackup.go new file mode 100644 index 00000000000..a42c9972de2 --- /dev/null +++ b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/compactbackup.go @@ -0,0 +1,192 @@ +// Copyright PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Code generated by client-gen. DO NOT EDIT. + +package v1alpha1 + +import ( + "context" + "time" + + v1alpha1 "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + scheme "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned/scheme" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + rest "k8s.io/client-go/rest" +) + +// CompactBackupsGetter has a method to return a CompactBackupInterface. +// A group's client should implement this interface. +type CompactBackupsGetter interface { + CompactBackups(namespace string) CompactBackupInterface +} + +// CompactBackupInterface has methods to work with CompactBackup resources. +type CompactBackupInterface interface { + Create(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.CreateOptions) (*v1alpha1.CompactBackup, error) + Update(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.UpdateOptions) (*v1alpha1.CompactBackup, error) + UpdateStatus(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.UpdateOptions) (*v1alpha1.CompactBackup, error) + Delete(ctx context.Context, name string, opts v1.DeleteOptions) error + DeleteCollection(ctx context.Context, opts v1.DeleteOptions, listOpts v1.ListOptions) error + Get(ctx context.Context, name string, opts v1.GetOptions) (*v1alpha1.CompactBackup, error) + List(ctx context.Context, opts v1.ListOptions) (*v1alpha1.CompactBackupList, error) + Watch(ctx context.Context, opts v1.ListOptions) (watch.Interface, error) + Patch(ctx context.Context, name string, pt types.PatchType, data []byte, opts v1.PatchOptions, subresources ...string) (result *v1alpha1.CompactBackup, err error) + CompactBackupExpansion +} + +// compactBackups implements CompactBackupInterface +type compactBackups struct { + client rest.Interface + ns string +} + +// newCompactBackups returns a CompactBackups +func newCompactBackups(c *PingcapV1alpha1Client, namespace string) *compactBackups { + return &compactBackups{ + client: c.RESTClient(), + ns: namespace, + } +} + +// Get takes name of the compactBackup, and returns the corresponding compactBackup object, and an error if there is any. +func (c *compactBackups) Get(ctx context.Context, name string, options v1.GetOptions) (result *v1alpha1.CompactBackup, err error) { + result = &v1alpha1.CompactBackup{} + err = c.client.Get(). + Namespace(c.ns). + Resource("compactbackups"). + Name(name). + VersionedParams(&options, scheme.ParameterCodec). + Do(ctx). + Into(result) + return +} + +// List takes label and field selectors, and returns the list of CompactBackups that match those selectors. +func (c *compactBackups) List(ctx context.Context, opts v1.ListOptions) (result *v1alpha1.CompactBackupList, err error) { + var timeout time.Duration + if opts.TimeoutSeconds != nil { + timeout = time.Duration(*opts.TimeoutSeconds) * time.Second + } + result = &v1alpha1.CompactBackupList{} + err = c.client.Get(). + Namespace(c.ns). + Resource("compactbackups"). + VersionedParams(&opts, scheme.ParameterCodec). + Timeout(timeout). + Do(ctx). + Into(result) + return +} + +// Watch returns a watch.Interface that watches the requested compactBackups. +func (c *compactBackups) Watch(ctx context.Context, opts v1.ListOptions) (watch.Interface, error) { + var timeout time.Duration + if opts.TimeoutSeconds != nil { + timeout = time.Duration(*opts.TimeoutSeconds) * time.Second + } + opts.Watch = true + return c.client.Get(). + Namespace(c.ns). + Resource("compactbackups"). + VersionedParams(&opts, scheme.ParameterCodec). + Timeout(timeout). + Watch(ctx) +} + +// Create takes the representation of a compactBackup and creates it. Returns the server's representation of the compactBackup, and an error, if there is any. +func (c *compactBackups) Create(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.CreateOptions) (result *v1alpha1.CompactBackup, err error) { + result = &v1alpha1.CompactBackup{} + err = c.client.Post(). + Namespace(c.ns). + Resource("compactbackups"). + VersionedParams(&opts, scheme.ParameterCodec). + Body(compactBackup). + Do(ctx). + Into(result) + return +} + +// Update takes the representation of a compactBackup and updates it. Returns the server's representation of the compactBackup, and an error, if there is any. +func (c *compactBackups) Update(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.UpdateOptions) (result *v1alpha1.CompactBackup, err error) { + result = &v1alpha1.CompactBackup{} + err = c.client.Put(). + Namespace(c.ns). + Resource("compactbackups"). + Name(compactBackup.Name). + VersionedParams(&opts, scheme.ParameterCodec). + Body(compactBackup). + Do(ctx). + Into(result) + return +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *compactBackups) UpdateStatus(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.UpdateOptions) (result *v1alpha1.CompactBackup, err error) { + result = &v1alpha1.CompactBackup{} + err = c.client.Put(). + Namespace(c.ns). + Resource("compactbackups"). + Name(compactBackup.Name). + SubResource("status"). + VersionedParams(&opts, scheme.ParameterCodec). + Body(compactBackup). + Do(ctx). + Into(result) + return +} + +// Delete takes name of the compactBackup and deletes it. Returns an error if one occurs. +func (c *compactBackups) Delete(ctx context.Context, name string, opts v1.DeleteOptions) error { + return c.client.Delete(). + Namespace(c.ns). + Resource("compactbackups"). + Name(name). + Body(&opts). + Do(ctx). + Error() +} + +// DeleteCollection deletes a collection of objects. +func (c *compactBackups) DeleteCollection(ctx context.Context, opts v1.DeleteOptions, listOpts v1.ListOptions) error { + var timeout time.Duration + if listOpts.TimeoutSeconds != nil { + timeout = time.Duration(*listOpts.TimeoutSeconds) * time.Second + } + return c.client.Delete(). + Namespace(c.ns). + Resource("compactbackups"). + VersionedParams(&listOpts, scheme.ParameterCodec). + Timeout(timeout). + Body(&opts). + Do(ctx). + Error() +} + +// Patch applies the patch and returns the patched compactBackup. +func (c *compactBackups) Patch(ctx context.Context, name string, pt types.PatchType, data []byte, opts v1.PatchOptions, subresources ...string) (result *v1alpha1.CompactBackup, err error) { + result = &v1alpha1.CompactBackup{} + err = c.client.Patch(pt). + Namespace(c.ns). + Resource("compactbackups"). + Name(name). + SubResource(subresources...). + VersionedParams(&opts, scheme.ParameterCodec). + Body(data). + Do(ctx). + Into(result) + return +} diff --git a/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/fake/fake_compactbackup.go b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/fake/fake_compactbackup.go new file mode 100644 index 00000000000..2a7e2d387c8 --- /dev/null +++ b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/fake/fake_compactbackup.go @@ -0,0 +1,138 @@ +// Copyright PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Code generated by client-gen. DO NOT EDIT. + +package fake + +import ( + "context" + + v1alpha1 "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + labels "k8s.io/apimachinery/pkg/labels" + types "k8s.io/apimachinery/pkg/types" + watch "k8s.io/apimachinery/pkg/watch" + testing "k8s.io/client-go/testing" +) + +// FakeCompactBackups implements CompactBackupInterface +type FakeCompactBackups struct { + Fake *FakePingcapV1alpha1 + ns string +} + +var compactbackupsResource = v1alpha1.SchemeGroupVersion.WithResource("compactbackups") + +var compactbackupsKind = v1alpha1.SchemeGroupVersion.WithKind("CompactBackup") + +// Get takes name of the compactBackup, and returns the corresponding compactBackup object, and an error if there is any. +func (c *FakeCompactBackups) Get(ctx context.Context, name string, options v1.GetOptions) (result *v1alpha1.CompactBackup, err error) { + obj, err := c.Fake. + Invokes(testing.NewGetAction(compactbackupsResource, c.ns, name), &v1alpha1.CompactBackup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.CompactBackup), err +} + +// List takes label and field selectors, and returns the list of CompactBackups that match those selectors. +func (c *FakeCompactBackups) List(ctx context.Context, opts v1.ListOptions) (result *v1alpha1.CompactBackupList, err error) { + obj, err := c.Fake. + Invokes(testing.NewListAction(compactbackupsResource, compactbackupsKind, c.ns, opts), &v1alpha1.CompactBackupList{}) + + if obj == nil { + return nil, err + } + + label, _, _ := testing.ExtractFromListOptions(opts) + if label == nil { + label = labels.Everything() + } + list := &v1alpha1.CompactBackupList{ListMeta: obj.(*v1alpha1.CompactBackupList).ListMeta} + for _, item := range obj.(*v1alpha1.CompactBackupList).Items { + if label.Matches(labels.Set(item.Labels)) { + list.Items = append(list.Items, item) + } + } + return list, err +} + +// Watch returns a watch.Interface that watches the requested compactBackups. +func (c *FakeCompactBackups) Watch(ctx context.Context, opts v1.ListOptions) (watch.Interface, error) { + return c.Fake. + InvokesWatch(testing.NewWatchAction(compactbackupsResource, c.ns, opts)) + +} + +// Create takes the representation of a compactBackup and creates it. Returns the server's representation of the compactBackup, and an error, if there is any. +func (c *FakeCompactBackups) Create(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.CreateOptions) (result *v1alpha1.CompactBackup, err error) { + obj, err := c.Fake. + Invokes(testing.NewCreateAction(compactbackupsResource, c.ns, compactBackup), &v1alpha1.CompactBackup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.CompactBackup), err +} + +// Update takes the representation of a compactBackup and updates it. Returns the server's representation of the compactBackup, and an error, if there is any. +func (c *FakeCompactBackups) Update(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.UpdateOptions) (result *v1alpha1.CompactBackup, err error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateAction(compactbackupsResource, c.ns, compactBackup), &v1alpha1.CompactBackup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.CompactBackup), err +} + +// UpdateStatus was generated because the type contains a Status member. +// Add a +genclient:noStatus comment above the type to avoid generating UpdateStatus(). +func (c *FakeCompactBackups) UpdateStatus(ctx context.Context, compactBackup *v1alpha1.CompactBackup, opts v1.UpdateOptions) (*v1alpha1.CompactBackup, error) { + obj, err := c.Fake. + Invokes(testing.NewUpdateSubresourceAction(compactbackupsResource, "status", c.ns, compactBackup), &v1alpha1.CompactBackup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.CompactBackup), err +} + +// Delete takes name of the compactBackup and deletes it. Returns an error if one occurs. +func (c *FakeCompactBackups) Delete(ctx context.Context, name string, opts v1.DeleteOptions) error { + _, err := c.Fake. + Invokes(testing.NewDeleteActionWithOptions(compactbackupsResource, c.ns, name, opts), &v1alpha1.CompactBackup{}) + + return err +} + +// DeleteCollection deletes a collection of objects. +func (c *FakeCompactBackups) DeleteCollection(ctx context.Context, opts v1.DeleteOptions, listOpts v1.ListOptions) error { + action := testing.NewDeleteCollectionAction(compactbackupsResource, c.ns, listOpts) + + _, err := c.Fake.Invokes(action, &v1alpha1.CompactBackupList{}) + return err +} + +// Patch applies the patch and returns the patched compactBackup. +func (c *FakeCompactBackups) Patch(ctx context.Context, name string, pt types.PatchType, data []byte, opts v1.PatchOptions, subresources ...string) (result *v1alpha1.CompactBackup, err error) { + obj, err := c.Fake. + Invokes(testing.NewPatchSubresourceAction(compactbackupsResource, c.ns, name, pt, data, subresources...), &v1alpha1.CompactBackup{}) + + if obj == nil { + return nil, err + } + return obj.(*v1alpha1.CompactBackup), err +} diff --git a/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/fake/fake_pingcap_client.go b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/fake/fake_pingcap_client.go index 80dcf442504..94f1896e3ef 100644 --- a/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/fake/fake_pingcap_client.go +++ b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/fake/fake_pingcap_client.go @@ -33,6 +33,10 @@ func (c *FakePingcapV1alpha1) BackupSchedules(namespace string) v1alpha1.BackupS return &FakeBackupSchedules{c, namespace} } +func (c *FakePingcapV1alpha1) CompactBackups(namespace string) v1alpha1.CompactBackupInterface { + return &FakeCompactBackups{c, namespace} +} + func (c *FakePingcapV1alpha1) DMClusters(namespace string) v1alpha1.DMClusterInterface { return &FakeDMClusters{c, namespace} } diff --git a/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/generated_expansion.go b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/generated_expansion.go index 6799967ce45..d5d8f8681e1 100644 --- a/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/generated_expansion.go +++ b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/generated_expansion.go @@ -19,6 +19,8 @@ type BackupExpansion interface{} type BackupScheduleExpansion interface{} +type CompactBackupExpansion interface{} + type DMClusterExpansion interface{} type DataResourceExpansion interface{} diff --git a/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/pingcap_client.go b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/pingcap_client.go index 126a8b75a6d..cb570a8b87d 100644 --- a/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/pingcap_client.go +++ b/pkg/client/clientset/versioned/typed/pingcap/v1alpha1/pingcap_client.go @@ -27,6 +27,7 @@ type PingcapV1alpha1Interface interface { RESTClient() rest.Interface BackupsGetter BackupSchedulesGetter + CompactBackupsGetter DMClustersGetter DataResourcesGetter RestoresGetter @@ -51,6 +52,10 @@ func (c *PingcapV1alpha1Client) BackupSchedules(namespace string) BackupSchedule return newBackupSchedules(c, namespace) } +func (c *PingcapV1alpha1Client) CompactBackups(namespace string) CompactBackupInterface { + return newCompactBackups(c, namespace) +} + func (c *PingcapV1alpha1Client) DMClusters(namespace string) DMClusterInterface { return newDMClusters(c, namespace) } diff --git a/pkg/client/informers/externalversions/generic.go b/pkg/client/informers/externalversions/generic.go index a5ccab2deba..6b07e7efe15 100644 --- a/pkg/client/informers/externalversions/generic.go +++ b/pkg/client/informers/externalversions/generic.go @@ -54,6 +54,8 @@ func (f *sharedInformerFactory) ForResource(resource schema.GroupVersionResource return &genericInformer{resource: resource.GroupResource(), informer: f.Pingcap().V1alpha1().Backups().Informer()}, nil case v1alpha1.SchemeGroupVersion.WithResource("backupschedules"): return &genericInformer{resource: resource.GroupResource(), informer: f.Pingcap().V1alpha1().BackupSchedules().Informer()}, nil + case v1alpha1.SchemeGroupVersion.WithResource("compactbackups"): + return &genericInformer{resource: resource.GroupResource(), informer: f.Pingcap().V1alpha1().CompactBackups().Informer()}, nil case v1alpha1.SchemeGroupVersion.WithResource("dmclusters"): return &genericInformer{resource: resource.GroupResource(), informer: f.Pingcap().V1alpha1().DMClusters().Informer()}, nil case v1alpha1.SchemeGroupVersion.WithResource("dataresources"): diff --git a/pkg/client/informers/externalversions/pingcap/v1alpha1/compactbackup.go b/pkg/client/informers/externalversions/pingcap/v1alpha1/compactbackup.go new file mode 100644 index 00000000000..47083cb76a1 --- /dev/null +++ b/pkg/client/informers/externalversions/pingcap/v1alpha1/compactbackup.go @@ -0,0 +1,87 @@ +// Copyright PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Code generated by informer-gen. DO NOT EDIT. + +package v1alpha1 + +import ( + "context" + time "time" + + pingcapv1alpha1 "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + versioned "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + internalinterfaces "github.com/pingcap/tidb-operator/pkg/client/informers/externalversions/internalinterfaces" + v1alpha1 "github.com/pingcap/tidb-operator/pkg/client/listers/pingcap/v1alpha1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + runtime "k8s.io/apimachinery/pkg/runtime" + watch "k8s.io/apimachinery/pkg/watch" + cache "k8s.io/client-go/tools/cache" +) + +// CompactBackupInformer provides access to a shared informer and lister for +// CompactBackups. +type CompactBackupInformer interface { + Informer() cache.SharedIndexInformer + Lister() v1alpha1.CompactBackupLister +} + +type compactBackupInformer struct { + factory internalinterfaces.SharedInformerFactory + tweakListOptions internalinterfaces.TweakListOptionsFunc + namespace string +} + +// NewCompactBackupInformer constructs a new informer for CompactBackup type. +// Always prefer using an informer factory to get a shared informer instead of getting an independent +// one. This reduces memory footprint and number of connections to the server. +func NewCompactBackupInformer(client versioned.Interface, namespace string, resyncPeriod time.Duration, indexers cache.Indexers) cache.SharedIndexInformer { + return NewFilteredCompactBackupInformer(client, namespace, resyncPeriod, indexers, nil) +} + +// NewFilteredCompactBackupInformer constructs a new informer for CompactBackup type. +// Always prefer using an informer factory to get a shared informer instead of getting an independent +// one. This reduces memory footprint and number of connections to the server. +func NewFilteredCompactBackupInformer(client versioned.Interface, namespace string, resyncPeriod time.Duration, indexers cache.Indexers, tweakListOptions internalinterfaces.TweakListOptionsFunc) cache.SharedIndexInformer { + return cache.NewSharedIndexInformer( + &cache.ListWatch{ + ListFunc: func(options v1.ListOptions) (runtime.Object, error) { + if tweakListOptions != nil { + tweakListOptions(&options) + } + return client.PingcapV1alpha1().CompactBackups(namespace).List(context.TODO(), options) + }, + WatchFunc: func(options v1.ListOptions) (watch.Interface, error) { + if tweakListOptions != nil { + tweakListOptions(&options) + } + return client.PingcapV1alpha1().CompactBackups(namespace).Watch(context.TODO(), options) + }, + }, + &pingcapv1alpha1.CompactBackup{}, + resyncPeriod, + indexers, + ) +} + +func (f *compactBackupInformer) defaultInformer(client versioned.Interface, resyncPeriod time.Duration) cache.SharedIndexInformer { + return NewFilteredCompactBackupInformer(client, f.namespace, resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc}, f.tweakListOptions) +} + +func (f *compactBackupInformer) Informer() cache.SharedIndexInformer { + return f.factory.InformerFor(&pingcapv1alpha1.CompactBackup{}, f.defaultInformer) +} + +func (f *compactBackupInformer) Lister() v1alpha1.CompactBackupLister { + return v1alpha1.NewCompactBackupLister(f.Informer().GetIndexer()) +} diff --git a/pkg/client/informers/externalversions/pingcap/v1alpha1/interface.go b/pkg/client/informers/externalversions/pingcap/v1alpha1/interface.go index 7ea63ea74ae..2e5e23b7a66 100644 --- a/pkg/client/informers/externalversions/pingcap/v1alpha1/interface.go +++ b/pkg/client/informers/externalversions/pingcap/v1alpha1/interface.go @@ -25,6 +25,8 @@ type Interface interface { Backups() BackupInformer // BackupSchedules returns a BackupScheduleInformer. BackupSchedules() BackupScheduleInformer + // CompactBackups returns a CompactBackupInformer. + CompactBackups() CompactBackupInformer // DMClusters returns a DMClusterInformer. DMClusters() DMClusterInformer // DataResources returns a DataResourceInformer. @@ -66,6 +68,11 @@ func (v *version) BackupSchedules() BackupScheduleInformer { return &backupScheduleInformer{factory: v.factory, namespace: v.namespace, tweakListOptions: v.tweakListOptions} } +// CompactBackups returns a CompactBackupInformer. +func (v *version) CompactBackups() CompactBackupInformer { + return &compactBackupInformer{factory: v.factory, namespace: v.namespace, tweakListOptions: v.tweakListOptions} +} + // DMClusters returns a DMClusterInformer. func (v *version) DMClusters() DMClusterInformer { return &dMClusterInformer{factory: v.factory, namespace: v.namespace, tweakListOptions: v.tweakListOptions} diff --git a/pkg/client/listers/pingcap/v1alpha1/compactbackup.go b/pkg/client/listers/pingcap/v1alpha1/compactbackup.go new file mode 100644 index 00000000000..0e4f26d86f1 --- /dev/null +++ b/pkg/client/listers/pingcap/v1alpha1/compactbackup.go @@ -0,0 +1,96 @@ +// Copyright PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Code generated by lister-gen. DO NOT EDIT. + +package v1alpha1 + +import ( + v1alpha1 "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/apimachinery/pkg/labels" + "k8s.io/client-go/tools/cache" +) + +// CompactBackupLister helps list CompactBackups. +// All objects returned here must be treated as read-only. +type CompactBackupLister interface { + // List lists all CompactBackups in the indexer. + // Objects returned here must be treated as read-only. + List(selector labels.Selector) (ret []*v1alpha1.CompactBackup, err error) + // CompactBackups returns an object that can list and get CompactBackups. + CompactBackups(namespace string) CompactBackupNamespaceLister + CompactBackupListerExpansion +} + +// compactBackupLister implements the CompactBackupLister interface. +type compactBackupLister struct { + indexer cache.Indexer +} + +// NewCompactBackupLister returns a new CompactBackupLister. +func NewCompactBackupLister(indexer cache.Indexer) CompactBackupLister { + return &compactBackupLister{indexer: indexer} +} + +// List lists all CompactBackups in the indexer. +func (s *compactBackupLister) List(selector labels.Selector) (ret []*v1alpha1.CompactBackup, err error) { + err = cache.ListAll(s.indexer, selector, func(m interface{}) { + ret = append(ret, m.(*v1alpha1.CompactBackup)) + }) + return ret, err +} + +// CompactBackups returns an object that can list and get CompactBackups. +func (s *compactBackupLister) CompactBackups(namespace string) CompactBackupNamespaceLister { + return compactBackupNamespaceLister{indexer: s.indexer, namespace: namespace} +} + +// CompactBackupNamespaceLister helps list and get CompactBackups. +// All objects returned here must be treated as read-only. +type CompactBackupNamespaceLister interface { + // List lists all CompactBackups in the indexer for a given namespace. + // Objects returned here must be treated as read-only. + List(selector labels.Selector) (ret []*v1alpha1.CompactBackup, err error) + // Get retrieves the CompactBackup from the indexer for a given namespace and name. + // Objects returned here must be treated as read-only. + Get(name string) (*v1alpha1.CompactBackup, error) + CompactBackupNamespaceListerExpansion +} + +// compactBackupNamespaceLister implements the CompactBackupNamespaceLister +// interface. +type compactBackupNamespaceLister struct { + indexer cache.Indexer + namespace string +} + +// List lists all CompactBackups in the indexer for a given namespace. +func (s compactBackupNamespaceLister) List(selector labels.Selector) (ret []*v1alpha1.CompactBackup, err error) { + err = cache.ListAllByNamespace(s.indexer, s.namespace, selector, func(m interface{}) { + ret = append(ret, m.(*v1alpha1.CompactBackup)) + }) + return ret, err +} + +// Get retrieves the CompactBackup from the indexer for a given namespace and name. +func (s compactBackupNamespaceLister) Get(name string) (*v1alpha1.CompactBackup, error) { + obj, exists, err := s.indexer.GetByKey(s.namespace + "/" + name) + if err != nil { + return nil, err + } + if !exists { + return nil, errors.NewNotFound(v1alpha1.Resource("compactbackup"), name) + } + return obj.(*v1alpha1.CompactBackup), nil +} diff --git a/pkg/client/listers/pingcap/v1alpha1/expansion_generated.go b/pkg/client/listers/pingcap/v1alpha1/expansion_generated.go index 5b0bb674123..f2cb82902bc 100644 --- a/pkg/client/listers/pingcap/v1alpha1/expansion_generated.go +++ b/pkg/client/listers/pingcap/v1alpha1/expansion_generated.go @@ -31,6 +31,14 @@ type BackupScheduleListerExpansion interface{} // BackupScheduleNamespaceLister. type BackupScheduleNamespaceListerExpansion interface{} +// CompactBackupListerExpansion allows custom methods to be added to +// CompactBackupLister. +type CompactBackupListerExpansion interface{} + +// CompactBackupNamespaceListerExpansion allows custom methods to be added to +// CompactBackupNamespaceLister. +type CompactBackupNamespaceListerExpansion interface{} + // DMClusterListerExpansion allows custom methods to be added to // DMClusterLister. type DMClusterListerExpansion interface{} diff --git a/pkg/controller/compact_backup_control.go b/pkg/controller/compact_backup_control.go new file mode 100644 index 00000000000..cd28fb5e627 --- /dev/null +++ b/pkg/controller/compact_backup_control.go @@ -0,0 +1,67 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License.i + +package controller + +import ( + "context" + + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/client-go/tools/record" +) + +// BackupControlInterface manages Backups used in BackupSchedule +type CompactBackupControlInterface interface { + CreateCompactBackup(compact *v1alpha1.CompactBackup) (*v1alpha1.CompactBackup, error) + DeleteCompactBackup(compact *v1alpha1.CompactBackup) error +} + +type realCompactControl struct { + cli versioned.Interface + recorder record.EventRecorder +} + +func NewRealCompactControl(cli versioned.Interface, recorder record.EventRecorder) CompactBackupControlInterface { + return &realCompactControl{ + cli: cli, + recorder: recorder, + } +} + +func (c *realCompactControl) CreateCompactBackup(compact *v1alpha1.CompactBackup) (*v1alpha1.CompactBackup, error) { + ns := compact.GetNamespace() + + return c.cli.PingcapV1alpha1().CompactBackups(ns).Create(context.TODO(), compact, metav1.CreateOptions{}) +} + +func (c *realCompactControl) DeleteCompactBackup(compact *v1alpha1.CompactBackup) error { + ns := compact.GetNamespace() + compactName := compact.GetName() + + return c.cli.PingcapV1alpha1().CompactBackups(ns).Delete(context.TODO(), compactName, metav1.DeleteOptions{}) +} diff --git a/pkg/controller/compact_status_updater.go b/pkg/controller/compact_status_updater.go new file mode 100644 index 00000000000..ec7adf565ae --- /dev/null +++ b/pkg/controller/compact_status_updater.go @@ -0,0 +1,207 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package controller + +import ( + "context" + "fmt" + "time" + + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + listers "github.com/pingcap/tidb-operator/pkg/client/listers/pingcap/v1alpha1" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + utilruntime "k8s.io/apimachinery/pkg/util/runtime" + "k8s.io/client-go/tools/record" + "k8s.io/client-go/util/retry" + "k8s.io/klog" +) + +const ( + // progressDebounceDuration is the minimum time interval between two progress updates for a backup + progressDebounceDuration = 3 * time.Second +) + +type Progress struct { + // MetaCompleted is the number of meta files compacted + MetaCompleted uint64 `json:"meta_completed"` + // MetaTotal is the total number of meta files + MetaTotal uint64 `json:"meta_total"` + // BytesToCompact is the number of bytes to compact + BytesToCompact uint64 `json:"bytes_to_compact"` + // BytesCompacted is the number of bytes compacted + BytesCompacted uint64 `json:"bytes_compacted"` +} + +type CompactStatusUpdaterInterface interface { + OnSchedule(ctx context.Context, compact *v1alpha1.CompactBackup, err error) error + OnCreateJob(ctx context.Context, compact *v1alpha1.CompactBackup, err error) error + OnStart(ctx context.Context, compact *v1alpha1.CompactBackup) error + OnProgress(ctx context.Context, compact *v1alpha1.CompactBackup, p Progress) error + OnFinish(ctx context.Context, compact *v1alpha1.CompactBackup, err error) error + OnJobFailed(ctx context.Context, compact *v1alpha1.CompactBackup, reason string) error +} + +type CompactStatusUpdater struct { + recorder record.EventRecorder + lister listers.CompactBackupLister + cli versioned.Interface + progressLastUpdate time.Time +} + +func NewCompactStatusUpdater(recorder record.EventRecorder, lister listers.CompactBackupLister, cli versioned.Interface) *CompactStatusUpdater { + return &CompactStatusUpdater{ + recorder: recorder, + lister: lister, + cli: cli, + } +} + +func (r *CompactStatusUpdater) Event(compact *v1alpha1.CompactBackup, ty, reason, msg string) { + r.recorder.Event(compact, ty, reason, msg) +} + +func (r *CompactStatusUpdater) UpdateStatus(compact *v1alpha1.CompactBackup, newStatus v1alpha1.CompactStatus) error { + ns := compact.GetNamespace() + compactName := compact.GetName() + + now := time.Now() + canUpdateProgress := true + if newStatus.Progress != "" { + if now.Sub(r.progressLastUpdate) < progressDebounceDuration { + canUpdateProgress = false + } + } + + // Update the status + err := retry.OnError(retry.DefaultRetry, func(e error) bool { return e != nil }, func() error { + // Always get the latest CompactBackup before updating + if updated, err := r.lister.CompactBackups(ns).Get(compactName); err == nil { + *compact = *(updated.DeepCopy()) + } else { + utilruntime.HandleError(fmt.Errorf("error getting updated compact %s/%s from lister: %v", ns, compactName, err)) + return err + } + + updated := false + if newStatus.State != "" && compact.Status.State != newStatus.State { + compact.Status.State = newStatus.State + updated = true + canUpdateProgress = true + } + if newStatus.Message != "" && compact.Status.Message != newStatus.Message { + compact.Status.Message = newStatus.Message + updated = true + } + if canUpdateProgress && newStatus.Progress != "" && compact.Status.Progress != newStatus.Progress { + compact.Status.Progress = newStatus.Progress + updated = true + r.progressLastUpdate = now + } + if newStatus.RetryStatus != nil && newStatus.RetryStatus[0].RetryNum == len(compact.Status.RetryStatus) { + compact.Status.RetryStatus = append(compact.Status.RetryStatus, newStatus.RetryStatus[0]) + updated = true + } + + // Apply the update if any field changed + if updated { + _, updateErr := r.cli.PingcapV1alpha1().CompactBackups(ns).Update(context.TODO(), compact, metav1.UpdateOptions{}) + if updateErr == nil { + klog.Infof("Compact: [%s/%s] updated successfully", ns, compactName) + return nil + } + klog.Errorf("Failed to update Compact [%s/%s], error: %v", ns, compactName, updateErr) + return updateErr + } + return nil + }) + return err +} + +func (r *CompactStatusUpdater) OnSchedule(ctx context.Context, compact *v1alpha1.CompactBackup, err error) error { + newStatus := v1alpha1.CompactStatus{} + if err != nil { + newStatus.State = string(v1alpha1.BackupFailed) + newStatus.Message = err.Error() + } else { + newStatus.State = string(v1alpha1.BackupScheduled) + } + return r.UpdateStatus(compact, newStatus) +} + +func (r *CompactStatusUpdater) OnCreateJob(ctx context.Context, compact *v1alpha1.CompactBackup, err error) error { + newStatus := v1alpha1.CompactStatus{} + if err != nil { + newStatus.State = string(v1alpha1.BackupRetryTheFailed) + newStatus.Message = err.Error() + newStatus.RetryStatus = []v1alpha1.CompactRetryRecord{ + { + RetryNum: len(compact.Status.RetryStatus), + DetectFailedAt: metav1.NewTime(time.Now()), + RetryReason: err.Error(), + }, + } + } else { + newStatus.State = string(v1alpha1.BackupPrepare) + } + return r.UpdateStatus(compact, newStatus) +} + +func (r *CompactStatusUpdater) OnStart(ctx context.Context, compact *v1alpha1.CompactBackup) error { + r.Event(compact, corev1.EventTypeNormal, "Started", "The compaction process has started successfully.") + + newStauts := v1alpha1.CompactStatus{ + State: string(v1alpha1.BackupRunning), + } + return r.UpdateStatus(compact, newStauts) +} + +func (r *CompactStatusUpdater) OnProgress(ctx context.Context, compact *v1alpha1.CompactBackup, p Progress) error { + progress := fmt.Sprintf("[READ_META(%d/%d),COMPACT_WORK(%d/%d)]", + p.MetaCompleted, p.MetaTotal, p.BytesCompacted, p.BytesToCompact) + + newStatus := v1alpha1.CompactStatus{ + Progress: progress, + } + return r.UpdateStatus(compact, newStatus) +} + +func (r *CompactStatusUpdater) OnFinish(ctx context.Context, compact *v1alpha1.CompactBackup, err error) error { + newStatus := v1alpha1.CompactStatus{} + if err != nil { + newStatus.State = string(v1alpha1.BackupRetryTheFailed) + newStatus.Message = err.Error() + newStatus.RetryStatus = []v1alpha1.CompactRetryRecord{ + { + RetryNum: len(compact.Status.RetryStatus), + DetectFailedAt: metav1.NewTime(time.Now()), + RetryReason: err.Error(), + }, + } + r.Event(compact, corev1.EventTypeWarning, "Failed(Retryable)", err.Error()) + } else { + newStatus.State = string(v1alpha1.BackupComplete) + r.Event(compact, corev1.EventTypeNormal, "Finished", "The compaction process has finished successfully.") + } + return r.UpdateStatus(compact, newStatus) +} + +func (r *CompactStatusUpdater) OnJobFailed(ctx context.Context, compact *v1alpha1.CompactBackup, reason string) error { + newStatus := v1alpha1.CompactStatus{ + State: string(v1alpha1.BackupFailed), + } + r.Event(compact, corev1.EventTypeWarning, "The compact job is failed.", reason) + return r.UpdateStatus(compact, newStatus) +} diff --git a/pkg/controller/compactbackup/compact_backup_controller.go b/pkg/controller/compactbackup/compact_backup_controller.go new file mode 100644 index 00000000000..ac3b3d37fe5 --- /dev/null +++ b/pkg/controller/compactbackup/compact_backup_controller.go @@ -0,0 +1,563 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package compact + +import ( + "context" + "fmt" + "math" + "strings" + "time" + + "github.com/pingcap/errors" + perrors "github.com/pingcap/errors" + "github.com/pingcap/tidb-operator/pkg/apis/label" + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + "github.com/pingcap/tidb-operator/pkg/backup/constants" + backuputil "github.com/pingcap/tidb-operator/pkg/backup/util" + "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + "github.com/pingcap/tidb-operator/pkg/controller" + "github.com/pingcap/tidb-operator/pkg/metrics" + "github.com/pingcap/tidb-operator/pkg/util" + batchv1 "k8s.io/api/batch/v1" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + utilruntime "k8s.io/apimachinery/pkg/util/runtime" + "k8s.io/apimachinery/pkg/util/wait" + "k8s.io/client-go/tools/cache" + "k8s.io/client-go/util/retry" + "k8s.io/client-go/util/workqueue" + "k8s.io/klog" + "k8s.io/utils/ptr" +) + +const ( + maxInterval = 6 * time.Minute +) + +// Controller controls backup. +type Controller struct { + deps *controller.Dependencies + // backups that need to be synced. + queue workqueue.RateLimitingInterface + cli versioned.Interface + statusUpdater controller.CompactStatusUpdaterInterface +} + +// NewController creates a backup controller. +func NewController(deps *controller.Dependencies) *Controller { + c := &Controller{ + deps: deps, + queue: workqueue.NewNamedRateLimitingQueue( + controller.NewControllerRateLimiter(1*time.Second, 100*time.Second), + "compactBackup", + ), + cli: deps.Clientset, + statusUpdater: controller.NewCompactStatusUpdater( + deps.Recorder, deps.CompactBackupLister, deps.Clientset, + ), + } + + compactInformer := deps.InformerFactory.Pingcap().V1alpha1().CompactBackups() + jobInformer := deps.KubeInformerFactory.Batch().V1().Jobs() + compactInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: c.updateCompact, + UpdateFunc: func(old, cur interface{}) { + c.updateCompact(cur) + }, + DeleteFunc: c.updateCompact, + }) + jobInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + DeleteFunc: c.deleteJob, + }) + + return c +} + +// Name returns compact backup controller name. +func (c *Controller) Name() string { + return "compactBackup" +} + +// Run runs the compact backup controller. +func (c *Controller) Run(workers int, stopCh <-chan struct{}) { + defer utilruntime.HandleCrash() + defer c.queue.ShutDown() + + klog.Info("Starting compact backup controller") + defer klog.Info("Shutting down compact backup controller") + + for i := 0; i < workers; i++ { + go wait.Until(c.worker, time.Second, stopCh) + } + + <-stopCh +} + +// worker runs a worker goroutine that invokes processNextWorkItem until the the controller's queue is closed +func (c *Controller) worker() { + for c.processNextWorkItem() { + } +} + +func (c *Controller) UpdateStatus(compact *v1alpha1.CompactBackup, newState string, message ...string) error { + ns := compact.GetNamespace() + compactName := compact.GetName() + // try best effort to guarantee backup is updated. + err := retry.OnError(retry.DefaultRetry, func(e error) bool { return e != nil }, func() error { + // Always get the latest backup before update. + if updated, err := c.deps.CompactBackupLister.CompactBackups(ns).Get(compactName); err == nil { + // make a copy so we don't mutate the shared cache + *compact = *(updated.DeepCopy()) + } else { + utilruntime.HandleError(fmt.Errorf("error getting updated compact %s/%s from lister: %v", ns, compactName, err)) + return err + } + if compact.Status.State != newState { + compact.Status.State = newState + if len(message) > 0 { + compact.Status.Message = message[0] + } + _, updateErr := c.cli.PingcapV1alpha1().CompactBackups(ns).Update(context.TODO(), compact, metav1.UpdateOptions{}) + if updateErr == nil { + klog.Infof("Compact: [%s/%s] updated successfully", ns, compactName) + return nil + } + klog.Errorf("Failed to update Compact [%s/%s], error: %v", ns, compactName, updateErr) + return updateErr + } + return nil + }) + return err +} + +func (c *Controller) resolveCompactBackupFromJob(namespace string, job *batchv1.Job) *v1alpha1.CompactBackup { + owner := metav1.GetControllerOf(job) + if owner == nil { + return nil + } + + if owner.Kind != controller.CompactBackupControllerKind.Kind { + return nil + } + + compact, err := c.deps.CompactBackupLister.CompactBackups(namespace).Get(owner.Name) + if err != nil { + return nil + } + if owner.UID != compact.UID { + return nil + } + return compact +} + +func (c *Controller) deleteJob(obj interface{}) { + job, ok := obj.(*batchv1.Job) + if !ok { + return + } + + ns := job.GetNamespace() + jobName := job.GetName() + compact := c.resolveCompactBackupFromJob(ns, job) + if compact == nil { + return + } + klog.V(4).Infof("Job %s/%s deleted through %v.", ns, jobName, utilruntime.GetCaller()) + c.updateCompact(compact) +} + +func (c *Controller) updateCompact(cur interface{}) { + newcompact := cur.(*v1alpha1.CompactBackup) + ns := newcompact.GetNamespace() + name := newcompact.GetName() + + if newcompact.Status.State == string(v1alpha1.BackupFailed) { + klog.Errorf("Compact %s/%s is failed, skip", ns, name) + return + } + + if newcompact.Status.State == string(v1alpha1.BackupComplete) { + klog.Errorf("Compact %s/%s is complete, skip", ns, name) + return + } + + klog.Infof("Compact object %s/%s enqueue", ns, name) + c.enqueueCompact(newcompact) +} + +// enqueueCompact enqueues the given compact in the work queue. +func (c *Controller) enqueueCompact(obj interface{}) { + key, err := cache.DeletionHandlingMetaNamespaceKeyFunc(obj) + if err != nil { + utilruntime.HandleError(fmt.Errorf("cound't get key for object %+v: %v", obj, err)) + return + } + c.queue.Add(key) +} + +// processNextWorkItem dequeues items, processes them, and marks them done. It enforces that the syncHandler is never +// invoked concurrently with the same key. +func (c *Controller) processNextWorkItem() bool { + metrics.ActiveWorkers.WithLabelValues(c.Name()).Add(1) + defer metrics.ActiveWorkers.WithLabelValues(c.Name()).Add(-1) + + key, quit := c.queue.Get() + if quit { + return false + } + defer c.queue.Done(key) + if err := c.sync(key.(string)); err != nil { + if perrors.Find(err, controller.IsRequeueError) != nil { + klog.Infof("Compact: %v, still need sync: %v, requeuing", key.(string), err) + c.queue.AddRateLimited(key) + } else if perrors.Find(err, controller.IsIgnoreError) != nil { + klog.Infof("Compact: %v, ignore err: %v", key.(string), err) + } else { + utilruntime.HandleError(fmt.Errorf("Compact: %v, sync failed, err: %v, requeuing", key.(string), err)) + c.queue.AddRateLimited(key) + } + } else { + c.queue.Forget(key) + } + return true +} + +func (c *Controller) sync(key string) (err error) { + startTime := time.Now() + defer func() { + duration := time.Since(startTime) + metrics.ReconcileTime.WithLabelValues(c.Name()).Observe(duration.Seconds()) + + if err == nil { + metrics.ReconcileTotal.WithLabelValues(c.Name(), metrics.LabelSuccess).Inc() + } else if perrors.Find(err, controller.IsRequeueError) != nil { + metrics.ReconcileTotal.WithLabelValues(c.Name(), metrics.LabelRequeue).Inc() + } else { + metrics.ReconcileTotal.WithLabelValues(c.Name(), metrics.LabelError).Inc() + metrics.ReconcileErrors.WithLabelValues(c.Name()).Inc() + } + + klog.V(4).Infof("Finished syncing Compact %q (%v)", key, duration) + }() + + ns, name, err := cache.SplitMetaNamespaceKey(key) + if err != nil { + return err + } + klog.Infof("Compact: [%s/%s] start to sync", ns, name) + compact, err := c.deps.CompactBackupLister.CompactBackups(ns).Get(name) + if err != nil { + if errors.IsNotFound(err) { + klog.Infof("Compact has been deleted %v", key) + return nil + } + klog.Infof("Compact get failed %v", err) + return err + } + + err = c.validate(compact) + if compact.Status.State == "" { + c.statusUpdater.OnSchedule(context.TODO(), compact, err) + } + if err != nil { + return err + } + + if compact.Status.State == string(v1alpha1.BackupComplete) { + klog.Infof("Compact %s/%s is complete, skip", ns, name) + return nil + } + + if compact.Status.State == string(v1alpha1.BackupFailed) { + klog.Infof("Compact %s/%s is failed, skip", ns, name) + return nil + } + + ok, err := c.checkJobStatus(compact) + if err != nil { + return err + } + if !ok { + klog.Infof("Compact %s/%s is not allowed to create new job, skip", ns, name) + return nil + } + + err = c.createCompactJob(compact.DeepCopy()) + c.statusUpdater.OnCreateJob(context.TODO(), compact, err) + return err +} + +func (c *Controller) createCompactJob(compact *v1alpha1.CompactBackup) error { + ns := compact.GetNamespace() + name := compact.GetName() + compactJobName := compact.GetName() + + // make compact job + var err error + var job *batchv1.Job + var reason string + if job, reason, err = c.makeCompactJob(compact); err != nil { + klog.Errorf("Compact %s/%s create job %s failed, reason is %s, error %v.", ns, name, compactJobName, reason, err) + return err + } + + // create k8s job + klog.Infof("Compact %s/%s creating job %s.", ns, name, compactJobName) + if err := c.deps.JobControl.CreateJob(compact, job); err != nil { + errMsg := fmt.Errorf("create Compact %s/%s job %s failed, err: %v", ns, name, compactJobName, err) + return errMsg + } + return nil +} + +func (c *Controller) makeCompactJob(compact *v1alpha1.CompactBackup) (*batchv1.Job, string, error) { + ns := compact.GetNamespace() + name := compact.GetName() + // Do we need a unique name for the job? + jobName := compact.GetName() + + var ( + envVars []corev1.EnvVar + reason string + err error + ) + + storageEnv, reason, err := backuputil.GenerateStorageCertEnv(ns, compact.Spec.UseKMS, compact.Spec.StorageProvider, c.deps.SecretLister) + if err != nil { + return nil, reason, fmt.Errorf("Compact %s/%s, %v", ns, name, err) + } + + envVars = append(envVars, storageEnv...) + + // set env vars specified in Compact.Spec.Env + envVars = util.AppendOverwriteEnv(envVars, compact.Spec.Env) + + args := []string{ + "compact", + fmt.Sprintf("--namespace=%s", ns), + fmt.Sprintf("--resourceName=%s", name), + } + + tc, err := c.deps.TiDBClusterLister.TidbClusters(compact.Spec.BR.ClusterNamespace).Get(compact.Spec.BR.Cluster) + if err != nil { + return nil, fmt.Sprintf("failed to fetch tidbcluster %s/%s", ns, compact.Spec.BR.Cluster), err + } + tikvImage := tc.TiKVImage() + _, tikvVersion := backuputil.ParseImage(tikvImage) + brImage := "pingcap/br:" + tikvVersion + if compact.Spec.ToolImage != "" { + toolImage := compact.Spec.ToolImage + if !strings.ContainsRune(compact.Spec.ToolImage, ':') { + toolImage = fmt.Sprintf("%s:%s", toolImage, tikvVersion) + } + + brImage = toolImage + } + klog.Infof("Compact %s/%s use br image %s and tikv image %s", ns, name, brImage, tikvImage) + + //TODO: (Ris)What is the instance here? + jobLabels := util.CombineStringMap(label.NewBackup().Instance("Compact-Backup").BackupJob().Backup(name), compact.Labels) + podLabels := jobLabels + jobAnnotations := compact.Annotations + podAnnotations := jobAnnotations + + volumeMounts := []corev1.VolumeMount{} + volumes := []corev1.Volume{} + + volumes = append(volumes, corev1.Volume{ + Name: "tool-bin", + VolumeSource: corev1.VolumeSource{ + EmptyDir: &corev1.EmptyDirVolumeSource{}, + }, + }) + + volumeMounts = append(volumeMounts, + corev1.VolumeMount{ + Name: "tool-bin", + MountPath: util.BRBinPath, + }, + corev1.VolumeMount{ + Name: "tool-bin", + MountPath: util.KVCTLBinPath, + }, + ) + + if len(compact.Spec.AdditionalVolumes) > 0 { + volumes = append(volumes, compact.Spec.AdditionalVolumes...) + } + if len(compact.Spec.AdditionalVolumeMounts) > 0 { + volumeMounts = append(volumeMounts, compact.Spec.AdditionalVolumeMounts...) + } + + // mount volumes if specified + if compact.Spec.Local != nil { + volumes = append(volumes, compact.Spec.Local.Volume) + volumeMounts = append(volumeMounts, compact.Spec.Local.VolumeMount) + } + + serviceAccount := constants.DefaultServiceAccountName + if compact.Spec.ServiceAccount != "" { + serviceAccount = compact.Spec.ServiceAccount + } + + podSpec := &corev1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: podLabels, + Annotations: podAnnotations, + }, + Spec: corev1.PodSpec{ + SecurityContext: compact.Spec.PodSecurityContext, + ServiceAccountName: serviceAccount, + InitContainers: []corev1.Container{ + { + Name: "br", + Image: brImage, + Command: []string{"/bin/sh", "-c"}, + Args: []string{fmt.Sprintf("cp /br %s/br; echo 'BR copy finished'", util.BRBinPath)}, + ImagePullPolicy: corev1.PullIfNotPresent, + VolumeMounts: volumeMounts, + Resources: compact.Spec.ResourceRequirements, + }, + { + Name: "tikv-ctl", + Image: tikvImage, + Command: []string{"/bin/sh", "-c"}, + Args: []string{fmt.Sprintf("cp /tikv-ctl %s/tikv-ctl; echo 'tikv-ctl copy finished'", util.KVCTLBinPath)}, + ImagePullPolicy: corev1.PullIfNotPresent, + VolumeMounts: volumeMounts, + Resources: compact.Spec.ResourceRequirements, + }, + }, + Containers: []corev1.Container{ + { + Name: "backup-manager", + Image: c.deps.CLIConfig.TiDBBackupManagerImage, + Args: args, + Env: envVars, + VolumeMounts: volumeMounts, + ImagePullPolicy: corev1.PullIfNotPresent, + }, + }, + RestartPolicy: corev1.RestartPolicyOnFailure, + Tolerations: compact.Spec.Tolerations, + ImagePullSecrets: compact.Spec.ImagePullSecrets, + Affinity: compact.Spec.Affinity, + Volumes: volumes, + PriorityClassName: compact.Spec.PriorityClassName, + }, + } + + job := &batchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: jobName, + Namespace: ns, + Labels: jobLabels, + Annotations: jobAnnotations, + OwnerReferences: []metav1.OwnerReference{ + controller.GetCompactBackupOwnerRef(compact), + }, + }, + Spec: batchv1.JobSpec{ + Template: *podSpec, + BackoffLimit: ptr.To(compact.Spec.MaxRetryTimes), + }, + } + + return job, "", nil +} + +// checkJobStatus checks if doCompact is allowed to run +// Only if there is no other compact job existing, doCompact is allowed +// If the existing job failed, update compact status +func (c *Controller) checkJobStatus(compact *v1alpha1.CompactBackup) (bool, error) { + ns := compact.GetNamespace() + name := compact.GetName() + + job, err := c.deps.KubeClientset.BatchV1().Jobs(ns).Get(context.TODO(), name, metav1.GetOptions{}) + if err != nil { + if errors.IsNotFound(err) { + return c.allowCompact(compact), nil + } + klog.Errorf("Failed to get job %s for compact %s/%s, error %v", name, ns, name, err) + return false, err + } + + for _, condition := range job.Status.Conditions { + if condition.Type == batchv1.JobFailed && condition.Status == corev1.ConditionTrue { + failReason := condition.Reason + failMessage := condition.Message + klog.Errorf("Compact: [%s/%s] compact job failed, reason: %s, message: %s", ns, name, failReason, failMessage) + c.statusUpdater.OnJobFailed(context.TODO(), compact, failMessage) + return false, nil + } + + if condition.Type == batchv1.JobComplete && condition.Status == corev1.ConditionTrue { + klog.Infof("Compact: [%s/%s] Job already completed successfully.", ns, name) + return false, nil + } + } + + klog.Infof("Compact: [%s/%s] compact job is running, skip.", ns, name) + return false, nil +} + +func (c *Controller) validate(compact *v1alpha1.CompactBackup) error { + spec := compact.Spec + if spec.StartTs == "" { + return errors.NewNoStackError("start-ts must be set") + } + if spec.EndTs == "" { + return errors.NewNoStackError("end-ts must be set") + } + if spec.Concurrency <= 0 { + return errors.NewNoStackError("concurrency must be greater than 0") + } + if spec.MaxRetryTimes < 0 { + return errors.NewNoStackError("maxRetryTimes must be greater than or equal to 0") + } + return nil +} + +func (c *Controller) allowCompact(compact *v1alpha1.CompactBackup) bool { + ns := compact.GetNamespace() + name := compact.GetName() + + // 10*2^(attempts-1) + expBackoff := func(attempts int) time.Duration { + if attempts <= 1 { + return 0 + } + interval := time.Duration(10*int(math.Pow(2, float64(attempts-1)))) * time.Second + if interval > maxInterval { + return maxInterval + } + return interval + } + + attempts := len(compact.Status.RetryStatus) + if attempts > 0 { + lastRetry := compact.Status.RetryStatus[attempts-1] + if lastRetry.RetryNum > int(compact.Spec.MaxRetryTimes) { + c.statusUpdater.OnJobFailed(context.TODO(), compact, "create job failed, reached max retry times") + return false + } + backoff := expBackoff(attempts) + if time.Since(lastRetry.DetectFailedAt.Time) < backoff { + klog.Infof("Compact: [%s/%s] backoff in effect, skipping retry.", ns, name) + return false + } + } + return true +} diff --git a/pkg/controller/controller_utils.go b/pkg/controller/controller_utils.go index e432802be8e..c321d0fc67a 100644 --- a/pkg/controller/controller_utils.go +++ b/pkg/controller/controller_utils.go @@ -49,6 +49,9 @@ var ( // BackupControllerKind contains the schema.GroupVersionKind for backup controller type. BackupControllerKind = v1alpha1.SchemeGroupVersion.WithKind("Backup") + // CompactBackupControllerKind contains the schema.GroupVersionKind for backup controller type. + CompactBackupControllerKind = v1alpha1.SchemeGroupVersion.WithKind("CompactBackup") + // RestoreControllerKind contains the schema.GroupVersionKind for restore controller type. RestoreControllerKind = v1alpha1.SchemeGroupVersion.WithKind("Restore") @@ -159,6 +162,20 @@ func GetBackupOwnerRef(backup *v1alpha1.Backup) metav1.OwnerReference { } } +// GetCompactBackupOwnerRef returns Backup's OwnerReference +func GetCompactBackupOwnerRef(backup *v1alpha1.CompactBackup) metav1.OwnerReference { + controller := true + blockOwnerDeletion := true + return metav1.OwnerReference{ + APIVersion: CompactBackupControllerKind.GroupVersion().String(), + Kind: CompactBackupControllerKind.Kind, + Name: backup.GetName(), + UID: backup.GetUID(), + Controller: &controller, + BlockOwnerDeletion: &blockOwnerDeletion, + } +} + // GetRestoreOwnerRef returns Restore's OwnerReference func GetRestoreOwnerRef(restore *v1alpha1.Restore) metav1.OwnerReference { controller := true diff --git a/pkg/controller/dependences.go b/pkg/controller/dependences.go index 6c22b8e658a..52f8d50ce29 100644 --- a/pkg/controller/dependences.go +++ b/pkg/controller/dependences.go @@ -195,6 +195,7 @@ type Controls struct { ProxyControl TiProxyControlInterface TiDBControl TiDBControlInterface BackupControl BackupControlInterface + CompactControl CompactBackupControlInterface RestoreControl RestoreControlInterface SecretControl SecretControlInterface } @@ -233,6 +234,7 @@ type Dependencies struct { TiDBClusterAutoScalerLister listers.TidbClusterAutoScalerLister DMClusterLister listers.DMClusterLister BackupLister listers.BackupLister + CompactBackupLister listers.CompactBackupLister RestoreLister listers.RestoreLister BackupScheduleLister listers.BackupScheduleLister TiDBInitializerLister listers.TidbInitializerLister @@ -378,6 +380,7 @@ func newDependencies( TiDBClusterAutoScalerLister: informerFactory.Pingcap().V1alpha1().TidbClusterAutoScalers().Lister(), DMClusterLister: informerFactory.Pingcap().V1alpha1().DMClusters().Lister(), BackupLister: informerFactory.Pingcap().V1alpha1().Backups().Lister(), + CompactBackupLister: informerFactory.Pingcap().V1alpha1().CompactBackups().Lister(), RestoreLister: informerFactory.Pingcap().V1alpha1().Restores().Lister(), BackupScheduleLister: informerFactory.Pingcap().V1alpha1().BackupSchedules().Lister(), TiDBInitializerLister: informerFactory.Pingcap().V1alpha1().TidbInitializers().Lister(), diff --git a/pkg/util/util.go b/pkg/util/util.go index 3df326f809a..909224a7be0 100644 --- a/pkg/util/util.go +++ b/pkg/util/util.go @@ -48,6 +48,7 @@ var ( ClusterAssetsTLSPath = "/var/lib/cluster-assets-tls" TiDBClientTLSPath = "/var/lib/tidb-client-tls" BRBinPath = "/var/lib/br-bin" + KVCTLBinPath = "/var/lib/kvctl-bin" DumplingBinPath = "/var/lib/dumpling-bin" LightningBinPath = "/var/lib/lightning-bin" ClusterClientVolName = "cluster-client-tls" diff --git a/tests/e2e/br/br.go b/tests/e2e/br/br.go index cab5c902e6b..18e3ecdbb48 100644 --- a/tests/e2e/br/br.go +++ b/tests/e2e/br/br.go @@ -1163,6 +1163,101 @@ var _ = ginkgo.Describe("Backup and Restore", func() { framework.ExpectEqual(cleaned, true, "storage should be cleaned") }) }) + + ginkgo.Context("Compact backup Test", func() { + ginkgo.It("test normal function", func() { + backupVersion := utilimage.TiDBLatest + enableTLS := false + skipCA := false + ns := f.Namespace.Name + dbName := "e2etest" + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + ginkgo.By("Create log-backup.enable TiDB cluster for pitr-master") + masterClusterName := "pitr-master" + err := createLogBackupEnableTidbCluster(f, masterClusterName, backupVersion, enableTLS, skipCA) + framework.ExpectNoError(err) + ginkgo.By("Wait for pitr-master TiDB cluster ready") + err = utiltidbcluster.WaitForTCConditionReady(f.ExtClient, ns, masterClusterName, tidbReadyTimeout, 0) + framework.ExpectNoError(err) + + ginkgo.By("Create RBAC for backup") + err = createRBAC(f) + framework.ExpectNoError(err) + + logBackupName := "log-backup" + typ := strings.ToLower(typeBR) + ginkgo.By("Start log backup") + logBackup, err := createBackupAndWaitForComplete(f, logBackupName, masterClusterName, typ, func(backup *v1alpha1.Backup) { + backup.Spec.CleanPolicy = v1alpha1.CleanPolicyTypeDelete + backup.Spec.Mode = v1alpha1.BackupModeLog + }) + framework.ExpectNoError(err) + framework.ExpectNotEqual(logBackup.Status.CommitTs, "") + + fullBackupName := "full-backup" + ginkgo.By("Start full backup") + fullBackup, err := createBackupAndWaitForComplete(f, fullBackupName, masterClusterName, typ, func(backup *v1alpha1.Backup) { + backup.Spec.CleanPolicy = v1alpha1.CleanPolicyTypeDelete + }) + framework.ExpectNoError(err) + framework.ExpectNotEqual(fullBackup.Status.CommitTs, "") + + ginkgo.By("Forward master TiDB cluster service") + masterHost, err := portforward.ForwardOnePort(ctx, f.PortForwarder, ns, getTiDBServiceResourceName(masterClusterName), int(v1alpha1.DefaultTiDBServerPort)) + framework.ExpectNoError(err) + err = initDatabase(masterHost, dbName) + framework.ExpectNoError(err) + + ginkgo.By("Write data into master TiDB cluster") + masterDSN := getDefaultDSN(masterHost, dbName) + err = blockwriter.New().Write(context.Background(), masterDSN) + framework.ExpectNoError(err) + + ginkgo.By("Forward master PD service") + masterPDHost, err := portforward.ForwardOnePort(ctx, f.PortForwarder, ns, getPDServiceResourceName(masterClusterName), int(v1alpha1.DefaultPDClientPort)) + framework.ExpectNoError(err) + ginkgo.By("Wait log backup reach current ts") + currentTS := strconv.FormatUint(config.GoTimeToTS(time.Now()), 10) + err = brutil.WaitForLogBackupReachTS(logBackupName, masterPDHost, currentTS, logbackupCatchUpTimeout) + framework.ExpectNoError(err) + + ginkgo.By("wait log backup progress reach current ts") + err = brutil.WaitForLogBackupProgressReachTS(f.ExtClient, ns, logBackupName, currentTS, logbackupCatchUpTimeout) + framework.ExpectNoError(err) + + compactName := "compact-backup" + ginkgo.By("Start a compact backup") + _, err = createCompactBackupAndWaitForComplete(f, compactName, masterClusterName, func(compact *v1alpha1.CompactBackup) { + compact.Spec.StartTs = fullBackup.Status.CommitTs + compact.Spec.EndTs = currentTS + compact.Spec.S3 = logBackup.Spec.S3 + compact.Spec.BR = logBackup.Spec.BR + compact.Spec.MaxRetryTimes = 2 + }) + framework.ExpectNoError(err) + }) + + ginkgo.It("test backoff when create job failed", func() { + _, cancel := context.WithCancel(context.Background()) + defer cancel() + + ginkgo.By("Create RBAC for backup") + err := createRBAC(f) + framework.ExpectNoError(err) + + compactName := "compact-backup" + ginkgo.By("Start a compact backup") + _, err = createCompactBackupAndWaitForComplete(f, compactName, "No_Such_Cluster", func(compact *v1alpha1.CompactBackup) { + compact.Spec.StartTs = "1" + compact.Spec.EndTs = "1" + compact.Spec.S3 = nil + compact.Spec.MaxRetryTimes = 2 + }) + framework.ExpectError(err, "create job failed, reached max retry times") + }) + }) }) func getTiDBServiceResourceName(tcName string) string { @@ -1703,3 +1798,36 @@ func getTableList(db *sql.DB) ([]string, error) { } return tables, nil } + +func createCompactBackupAndWaitForComplete(f *e2eframework.Framework, name, tcName string, configure func(*v1alpha1.CompactBackup)) (*v1alpha1.CompactBackup, error) { + ns := f.Namespace.Name + // secret to visit tidb cluster + s := brutil.GetSecret(ns, name, "") + // Check if the secret already exists + if _, err := f.ClientSet.CoreV1().Secrets(ns).Get(context.TODO(), name, metav1.GetOptions{}); err != nil { + if apierrors.IsNotFound(err) { + if _, err := f.ClientSet.CoreV1().Secrets(ns).Create(context.TODO(), s, metav1.CreateOptions{}); err != nil { + return nil, err + } + } else { + return nil, err + } + } + + backupFolder := time.Now().Format(time.RFC3339) + cfg := f.Storage.Config(ns, backupFolder) + compact := brutil.GetCompactBackup(ns, name, tcName, cfg) + + if configure != nil { + configure(compact) + } + + if _, err := f.ExtClient.PingcapV1alpha1().CompactBackups(ns).Create(context.TODO(), compact, metav1.CreateOptions{}); err != nil { + return nil, err + } + + if err := brutil.WaitForCompactComplete(f, ns, name, backupCompleteTimeout); err != nil { + return compact, err + } + return f.ExtClient.PingcapV1alpha1().CompactBackups(ns).Get(context.TODO(), name, metav1.GetOptions{}) +} diff --git a/tests/e2e/br/framework/br/data.go b/tests/e2e/br/framework/br/data.go index 92315fb4f03..1b1ece903f6 100644 --- a/tests/e2e/br/framework/br/data.go +++ b/tests/e2e/br/framework/br/data.go @@ -47,7 +47,7 @@ func GetRole(ns string) *rbacv1.Role { }, { APIGroups: []string{"pingcap.com"}, - Resources: []string{"backups", "restores"}, + Resources: []string{"backups", "restores", "compactbackups"}, Verbs: []string{"get", "watch", "list", "update"}, }, }, @@ -178,3 +178,24 @@ func GetRestore(ns, name, tcName, typ string, s3Config *v1alpha1.S3StorageProvid } return restore } + +func GetCompactBackup(ns, name, tcName string, s3Config *v1alpha1.S3StorageProvider) *v1alpha1.CompactBackup { + sendCredToTikv := true + compact := &v1alpha1.CompactBackup{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + Namespace: ns, + }, + Spec: v1alpha1.CompactSpec{ + StorageProvider: v1alpha1.StorageProvider{ + S3: s3Config, + }, + BR: &v1alpha1.BRConfig{ + Cluster: tcName, + ClusterNamespace: ns, + SendCredToTikv: &sendCredToTikv, + }, + }, + } + return compact +} diff --git a/tests/e2e/br/framework/br/wait.go b/tests/e2e/br/framework/br/wait.go index 7d90ed73351..aecd7759631 100644 --- a/tests/e2e/br/framework/br/wait.go +++ b/tests/e2e/br/framework/br/wait.go @@ -17,7 +17,9 @@ import ( "context" "encoding/binary" "fmt" + "io" "path" + "strings" "time" "github.com/pingcap/tidb-operator/pkg/apis/label" @@ -26,7 +28,9 @@ import ( "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" "github.com/pingcap/tidb-operator/pkg/pdapi" "github.com/pingcap/tidb-operator/tests/e2e/br/framework" + "github.com/pingcap/tidb-operator/tests/third_party/k8s/log" corev1 "k8s.io/api/core/v1" + v1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/util/wait" @@ -346,3 +350,70 @@ func WaitBackupPodOnPhase(f *framework.Framework, backup *v1alpha1.Backup, phase } return nil } + +func WaitForCompactComplete(f *framework.Framework, ns, name string, timeout time.Duration) error { + if err := wait.PollImmediate(poll, timeout, func() (bool, error) { + cpbk, err := f.ExtClient.PingcapV1alpha1().CompactBackups(ns).Get(context.TODO(), name, metav1.GetOptions{}) + if err != nil { + return false, err + } + + switch cpbk.Status.State { + case string(v1alpha1.BackupComplete): + return true, nil + case string(v1alpha1.BackupFailed): + return false, fmt.Errorf("Compact failed: %s", cpbk.Status.Message) + default: + log.Logf("the current status is: %s %s", cpbk.Status.State, cpbk.Status.Progress) + //do nothing + } + + return false, nil + }); err != nil { + printPodLogs(f, ns, name) + return fmt.Errorf("can't wait for backup complete: %v", err) + } + return nil +} + +func printPodLogs(f *framework.Framework, ns, name string) { + pods, err := f.ClientSet.CoreV1().Pods(ns).List(context.TODO(), metav1.ListOptions{}) + if err != nil { + log.Logf("Error listing pods: %v", err) + return + } + + var matchingPods []v1.Pod + for _, pod := range pods.Items { + if strings.Contains(pod.Name, name) { + matchingPods = append(matchingPods, pod) + } + } + + if len(matchingPods) == 0 { + fmt.Printf("No pods found containing '%s' in namespace '%s'\n", name, ns) + return + } + + for _, pod := range matchingPods { + req := f.ClientSet.CoreV1().Pods(ns).GetLogs(pod.Name, &v1.PodLogOptions{}) + + // Execute the log request and get the stream + logStream, err := req.Stream(context.TODO()) + if err != nil { + log.Logf("Error retrieving logs for pod %s: %v", pod.Name, err) + return + } + defer logStream.Close() + + // Read the log stream + logBytes, err := io.ReadAll(logStream) + if err != nil { + log.Logf("Error reading logs for pod %s: %v", pod.Name, err) + return + } + + // Print the logs as a string + log.Logf("Logs for pod %s in namespace %s: %s", pod.Name, ns, string(logBytes)) + } +}