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/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..774335e4405 --- /dev/null +++ b/cmd/backup-manager/app/cmd/compact.go @@ -0,0 +1,47 @@ +// 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" + coptions "github.com/pingcap/tidb-operator/cmd/backup-manager/app/compact/options" + "github.com/spf13/cobra" +) + +func NewCompactCommand() *cobra.Command { + cmd := &cobra.Command{ + Use: "compact", + Short: "Compact log backup.", + RunE: func(cmd *cobra.Command, args []string) error { + opts := coptions.KubeOpts{} + if err := opts.ParseFromFlags(cmd.Flags()); err != nil { + return err + } + opts.Kubeconfig = kubecfg + + ctx := context.Background() + link, err := compact.NewKubelink(opts.Kubeconfig) + if err != nil { + return err + } + cx := compact.New(opts, link) + return cx.Run(ctx) + }, + } + + coptions.DefineFlags(cmd.Flags()) + return cmd +} diff --git a/cmd/backup-manager/app/compact/kubelink.go b/cmd/backup-manager/app/compact/kubelink.go new file mode 100644 index 00000000000..e2efbfd0a47 --- /dev/null +++ b/cmd/backup-manager/app/compact/kubelink.go @@ -0,0 +1,147 @@ +package compact + +import ( + "context" + "fmt" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/compact/options" + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/util" + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1" + "github.com/pingcap/tidb-operator/pkg/apis/util/config" + pkgutil "github.com/pingcap/tidb-operator/pkg/backup/util" + "github.com/pingcap/tidb-operator/pkg/client/clientset/versioned" + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/tools/record" + "k8s.io/klog/v2" +) + +type Kubelink struct { + kube kubernetes.Interface + cr versioned.Interface + + ref *CompactionRef + recorder record.EventRecorder +} + +func NewKubelink(kubeconfig string) (*Kubelink, error) { + kube, cr, err := util.NewKubeAndCRCli(kubeconfig) + if err != nil { + return nil, err + } + return &Kubelink{ + kube: kube, + cr: cr, + }, nil +} + +func (k *Kubelink) GetCompaction(ctx context.Context, opts CompactionRef) (options.CompactOpts, error) { + if k.ref != nil { + return options.CompactOpts{}, errors.New("GetCompaction called twice") + } + k.ref = &opts + k.recorder = util.NewEventRecorder(k.kube, "compact.Kubelink") + + cb, err := k.cr.PingcapV1alpha1(). + CompactBackups(opts.Namespace). + Get(ctx, opts.Name, v1.GetOptions{}) + if err != nil { + return options.CompactOpts{}, err + } + + out := options.CompactOpts{} + args, err := pkgutil.GenStorageArgsForFlag(cb.Spec.StorageProvider, "") + if err != nil { + return options.CompactOpts{}, err + } + out.StorageOpts = args + + startTs, err := config.ParseTSString(cb.Spec.StartTs) + if err != nil { + return options.CompactOpts{}, errors.Annotatef(err, "failed to parse startTs %s", cb.Spec.StartTs) + } + endTs, err := config.ParseTSString(cb.Spec.EndTs) + if err != nil { + return options.CompactOpts{}, errors.Annotatef(err, "failed to parse endTs %s", cb.Spec.EndTs) + } + out.FromTS = startTs + out.UntilTS = endTs + + out.Name = cb.ObjectMeta.Name + out.Concurrency = uint64(cb.Spec.Concurrency) + + if err := out.Verify(); err != nil { + return options.CompactOpts{}, err + } + + return out, nil +} + +type cOP func(*v1alpha1.CompactBackup) error + +func (k *Kubelink) setState(newState string) cOP { + return func(cb *v1alpha1.CompactBackup) error { + cb.Status.State = newState + return nil + } +} + +func (k *Kubelink) event(ty, reason, msg string) cOP { + return func(cb *v1alpha1.CompactBackup) error { + k.recorder.Event(cb, ty, reason, msg) + return nil + } +} + +func (k *Kubelink) edit(ctx context.Context, extraOps ...cOP) error { + lister := k.cr.PingcapV1alpha1(). + CompactBackups(k.ref.Namespace) + cb, err := lister. + Get(ctx, k.ref.Name, v1.GetOptions{}) + if err != nil { + return err + } + + for _, op := range extraOps { + if err := op(cb); err != nil { + return err + } + } + + _, err = lister.Update(ctx, cb, v1.UpdateOptions{}) + if err != nil { + return err + } + return nil +} + +func (k *Kubelink) editOrWarn(ctx context.Context, extraOps ...cOP) { + if err := k.edit(ctx, extraOps...); err != nil { + klog.Warningf( + "failed to edit state for %s/%s: %v", + k.ref.Namespace, + k.ref.Name, + err, + ) + } +} + +func (k *Kubelink) OnStart(ctx context.Context) { + k.editOrWarn(ctx, k.setState("STARTED"), k.event(corev1.EventTypeNormal, "Started", "CompactionStarted")) +} + +func (k *Kubelink) OnProgress(ctx context.Context, p Progress) { + message := fmt.Sprintf("RUNNING[READ_META(%d/%d),COMPACT_WORK(%d/%d)]", + p.MetaCompleted, p.MetaTotal, p.BytesCompacted, p.BytesToCompact) + k.editOrWarn(ctx, k.setState(message)) +} + +func (k *Kubelink) OnFinish(ctx context.Context, err error) { + if err != nil { + k.editOrWarn(ctx, k.setState(fmt.Sprintf("ERR[%s]", err)), k.event(corev1.EventTypeWarning, "Failed", err.Error())) + } else { + k.editOrWarn(ctx, k.setState("DONE"), k.event(corev1.EventTypeNormal, "Succeeded", "CompactionDone")) + } +} 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..280a9ea1228 --- /dev/null +++ b/cmd/backup-manager/app/compact/options/options.go @@ -0,0 +1,79 @@ +package options + +import ( + "math" + + "github.com/pingcap/errors" + + "github.com/spf13/pflag" +) + +const ( + fromTSUnset = math.MaxUint64 + untilTSUnset = 0 + + namespaceFlag = "namespace" + resourceNameFlag = "resourceName" + tiKVVersionFlag = "tikvVersion" + storageStringFlag = "storage-string" + fromTSFlag = "from-ts" + untilTSFlag = "until-ts" + nameFlag = "name" + concurrencyFlag = "concurrency" +) + +type KubeOpts struct { + // This should be fill by the caller. + Kubeconfig string `json:"-"` + Namespace string `json:"namespace"` + ResourceName string `json:"resourceName"` + TiKVVersion string `json:"tikvVersion"` +} + +type CompactOpts struct { + FromTS uint64 + UntilTS uint64 + Name string + Concurrency uint64 + StorageOpts []string +} + +func DefineFlags(fs *pflag.FlagSet) { + fs.String(tiKVVersionFlag, "", "TiKV version of the resource") + fs.String(namespaceFlag, "", "Namespace of the resource") + fs.String(resourceNameFlag, "", "Name of the resource") +} + +func (k *KubeOpts) ParseFromFlags(fs *pflag.FlagSet) error { + var err error + k.Namespace, err = fs.GetString(namespaceFlag) + if err != nil { + return errors.Trace(err) + } + k.ResourceName, err = fs.GetString(resourceNameFlag) + if err != nil { + return errors.Trace(err) + } + k.TiKVVersion, err = fs.GetString(tiKVVersionFlag) + if err != nil { + return errors.Trace(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/compact/run.go b/cmd/backup-manager/app/compact/run.go new file mode 100644 index 00000000000..8693ae5f9fc --- /dev/null +++ b/cmd/backup-manager/app/compact/run.go @@ -0,0 +1,254 @@ +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" + "github.com/pingcap/tidb-operator/pkg/util" + "k8s.io/klog/v2" +) + +// CompactCtx is the context for *a* compaction. +type CompactCtx struct { + kopts options.KubeOpts + opts options.CompactOpts + + // Delegate defines some hooks that allow you to listen over events and + // controls what compactions should be done. + Delegate Delegate + + /* Below fields are used for testing. */ + + // OverrideBase64ify overrides the shell command used for generate + // the storage base64. + OverrideBase64ify func(ctx context.Context, args []string) *exec.Cmd + // OverrideCompact overrides the shell command used for compacting. + OverrideCompact func(ctx context.Context, args []string) *exec.Cmd +} + +type Progress struct { + MetaCompleted uint64 `json:"meta_completed"` + MetaTotal uint64 `json:"meta_total"` + BytesToCompact uint64 `json:"bytes_to_compact"` + BytesCompacted uint64 `json:"bytes_compacted"` +} + +type CompactionRef struct { + Namespace string + Name string +} + +// Delegate describes the "environment" that a compaction runs. +type Delegate interface { + // GetCompaction will be called at the very beginning. + // This should return a set of parameter of a compaction, then a compaction + // defined by these parameters will be executed. + // + // The argument `ref` refs to a `CompactBackup` resource in the cluster, + // and the compaction should be generated according to it. + GetCompaction(ctx context.Context, ref CompactionRef) (options.CompactOpts, error) + + // OnStart will be called when the `tikv-ctl compact-log-backup` process is about to be spawned. + OnStart(ctx context.Context) + // OnPrgress will be called when the progress of compaction updated. + OnProgress(ctx context.Context, p Progress) + // OnFinish will be called when the `tikv-ctl` process exits. + OnFinish(ctx context.Context, err error) +} + +// BaseDelegate is an empty delegate. +// +// You shouldn't directly use this delegate, instead you may embed it into +// your delegate implementation and override the methods you need. +type BaseDelegate struct{} + +func (BaseDelegate) GetCompaction(ctx context.Context, ref CompactionRef) (options.CompactOpts, error) { + return options.CompactOpts{}, errors.New("override `GetCompaction` to provide a meanful execution") +} +func (BaseDelegate) OnStart(ctx context.Context) {} +func (BaseDelegate) OnProgress(ctx context.Context, p Progress) {} +func (BaseDelegate) OnFinish(ctx context.Context, err error) {} + +func (r *CompactCtx) brBin() string { + return filepath.Join(util.BRBinPath, "br") +} + +func (r *CompactCtx) kvCtlBin() string { + return filepath.Join(util.KVCTLBinPath, "tikv-ctl") +} + +func (r *CompactCtx) base64ifyCmd(ctx context.Context, extraArgs []string) *exec.Cmd { + br := r.brBin() + args := []string{ + "operator", + "base64ify", + } + args = append(args, extraArgs...) + + if r.OverrideBase64ify != nil { + return r.OverrideBase64ify(ctx, args) + } + return exec.CommandContext(ctx, br, args...) +} + +func (r *CompactCtx) base64ifyStorage(ctx context.Context) (string, error) { + brCmd := r.base64ifyCmd(ctx, r.opts.StorageOpts) + out, err := brCmd.Output() + if err != nil { + eerr := err.(*exec.ExitError) + 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 (r *CompactCtx) compactCmd(ctx context.Context, base64Storage string) *exec.Cmd { + ctl := r.kvCtlBin() + args := []string{ + "--log-level", + "INFO", + "--log-format", + "json", + "compact-log-backup", + "--storage-base64", + base64Storage, + "--from", + strconv.FormatUint(r.opts.FromTS, 10), + "--until", + strconv.FormatUint(r.opts.UntilTS, 10), + "-N", + strconv.FormatUint(r.opts.Concurrency, 10), + } + + if r.OverrideCompact != nil { + return r.OverrideCompact(ctx, args) + } + return exec.CommandContext(ctx, ctl, args...) +} + +func (r *CompactCtx) runCompaction(ctx context.Context, base64Storage string) (err error) { + cmd := r.compactCmd(ctx, base64Storage) + defer func() { r.Delegate.OnFinish(ctx, err) }() + + logs, 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") + } + + r.Delegate.OnStart(ctx) + err = r.processCompactionLogs(ctx, io.TeeReader(logs, os.Stdout)) + if err != nil { + return err + } + + return cmd.Wait() +} + +// 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 + + // Raw is the original log JSON. + Raw []byte +} + +var _ json.Unmarshaler = &logLine{} + +func (l *logLine) UnmarshalJSON(bytes []byte) error { + item := struct { + Message string `json:"message"` + }{} + if err := json.Unmarshal(bytes, &item); err != nil { + return err + } + l.Message = item.Message + l.Raw = bytes + + return nil +} + +func (r *CompactCtx) processLogLine(ctx context.Context, l logLine) error { + const ( + messageCompactionDone = "Finishing compaction." + messageCompactAborted = "Compaction aborted." + ) + + switch l.Message { + case messageCompactionDone: + var prog Progress + if err := json.Unmarshal(l.Raw, &prog); err != nil { + return errors.Annotate(err, "failed to decode progress") + } + r.Delegate.OnProgress(ctx, prog) + return nil + case messageCompactAborted: + errContainer := struct { + Err string `json:"err"` + }{} + if err := json.Unmarshal(l.Raw, &errContainer); err != nil { + return errors.Annotate(err, "failed to decode error message") + } + return errors.Errorf("compaction aborted: %s", errContainer.Err) + default: + return nil + } +} + +func (r *CompactCtx) processCompactionLogs(ctx context.Context, logStream io.Reader) error { + dec := json.NewDecoder(logStream) + + var line logLine + for dec.More() { + if ctx.Err() != nil { + return ctx.Err() + } + if err := dec.Decode(&line); err != nil { + return errors.Annotate(err, "failed to decode the line of log") + } + if err := r.processLogLine(ctx, line); err != nil { + return errors.Annotate(err, "error during processing log line") + } + } + + return nil +} + +func (r *CompactCtx) Run(ctx context.Context) error { + ref := CompactionRef{ + Namespace: r.kopts.Namespace, + Name: r.kopts.ResourceName, + } + opts, err := r.Delegate.GetCompaction(ctx, ref) + if err != nil { + return errors.Annotate(err, "failed to get storage string") + } + r.opts = opts + + b64, err := r.base64ifyStorage(ctx) + if err != nil { + return errors.Annotate(err, "failed to base64ify storage") + } + return r.runCompaction(ctx, b64) +} + +// New creates a new compaction context. +func New(opts options.KubeOpts, dele Delegate) *CompactCtx { + return &CompactCtx{ + kopts: opts, + Delegate: dele, + } +} diff --git a/cmd/backup-manager/app/compact/run_test.go b/cmd/backup-manager/app/compact/run_test.go new file mode 100644 index 00000000000..99d0d637392 --- /dev/null +++ b/cmd/backup-manager/app/compact/run_test.go @@ -0,0 +1,175 @@ +package compact_test + +import ( + "bytes" + "context" + "encoding/json" + "io" + "os/exec" + "testing" + "testing/iotest" + + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/compact" + "github.com/pingcap/tidb-operator/cmd/backup-manager/app/compact/options" + "github.com/stretchr/testify/require" +) + +type callRecorder []call + +type call struct { + cmd string + attrs map[string]any +} + +func (c *callRecorder) GetCompaction(ctx context.Context, ref compact.CompactionRef) (options.CompactOpts, error) { + *c = append(*c, call{cmd: "GetCompaction", attrs: map[string]any{ + "ref": ref, + }}) + return options.CompactOpts{}, nil +} + +func (c *callRecorder) OnStart(ctx context.Context) { + *c = append(*c, call{cmd: "OnStart"}) +} + +func (c *callRecorder) OnProgress(ctx context.Context, p compact.Progress) { + *c = append(*c, call{cmd: "OnProgress", attrs: map[string]any{ + "MetaCompleted": p.MetaCompleted, + "MetaTotal": p.MetaTotal, + "BytesToCompact": p.BytesToCompact, + "BytesCompacted": p.BytesCompacted, + }}) +} + +func (c *callRecorder) OnFinish(ctx context.Context, err error) { + *c = append(*c, call{cmd: "OnFinish", attrs: map[string]any{ + "error": err, + }}) +} + +type dummyCommands struct { + theBase64 string + t *testing.T +} + +func (d dummyCommands) OverrideBase64ify(context.Context, []string) *exec.Cmd { + cmd := exec.Command("cat", "-") + cmd.Stdin = bytes.NewBufferString(d.theBase64) + return cmd +} + +func (d dummyCommands) OverrideCompact(_ context.Context, args []string) *exec.Cmd { + storage := "" + for i, arg := range args { + if arg == "--storage-base64" && i+1 < len(args) { + storage = args[i+1] + break + } + } + + require.Equal(d.t, storage, d.theBase64, "%#v", args) + return exec.Command("true") +} + +type empty struct { + compact.BaseDelegate +} + +func (e empty) GetCompaction(context.Context, compact.CompactionRef) (options.CompactOpts, error) { + return options.CompactOpts{}, nil +} + +func TestNormal(t *testing.T) { + cx := compact.New(options.KubeOpts{}, empty{}) + dc := dummyCommands{ + theBase64: "was yae", + t: t, + } + cx.OverrideBase64ify = dc.OverrideBase64ify + cx.OverrideCompact = dc.OverrideCompact + + require.NoError(t, cx.Run(context.Background())) +} + +type dummyProgress struct { + progs []compact.Progress + t *testing.T +} + +func (d dummyProgress) OverrideBase64ify(context.Context, []string) *exec.Cmd { + return exec.Command("true") +} + +func toMap(t *testing.T, v any) (res map[string]any) { + data, err := json.Marshal(v) + require.NoError(t, err) + require.NoError(t, json.Unmarshal(data, &res)) + return +} + +func (d dummyProgress) OverrideCompact(_ context.Context, args []string) *exec.Cmd { + cmd := exec.Command("sh", "-c", "cat - >&2") + rx, tx := io.Pipe() + + enc := json.NewEncoder(tx) + go func() { + for _, prog := range d.progs { + m := toMap(d.t, prog) + m["message"] = "Finishing compaction." + require.NoError(d.t, enc.Encode(m)) + } + tx.Close() + }() + + // Simulating streaming progress. + cmd.Stdin = iotest.OneByteReader(rx) + return cmd +} + +func TestWithProgress(t *testing.T) { + rec := callRecorder{} + cx := compact.New(options.KubeOpts{Namespace: "foo", ResourceName: "bar"}, &rec) + dp := dummyProgress{ + progs: []compact.Progress{ + {MetaCompleted: 1, MetaTotal: 2, BytesToCompact: 3, BytesCompacted: 4}, + {MetaCompleted: 2, MetaTotal: 2, BytesToCompact: 7, BytesCompacted: 8}, + {MetaCompleted: 2, MetaTotal: 2, BytesToCompact: 8, BytesCompacted: 8}, + }, + t: t, + } + cx.OverrideBase64ify = dp.OverrideBase64ify + cx.OverrideCompact = dp.OverrideCompact + + require.NoError(t, cx.Run(context.Background())) + + require.Equal(t, []call(rec), []call{ + {cmd: "GetCompaction", attrs: map[string]any{ + "ref": compact.CompactionRef{ + Namespace: "foo", + Name: "bar", + }, + }}, + {cmd: "OnStart"}, + {cmd: "OnProgress", attrs: map[string]any{ + "MetaCompleted": uint64(1), + "MetaTotal": uint64(2), + "BytesToCompact": uint64(3), + "BytesCompacted": uint64(4), + }}, + {cmd: "OnProgress", attrs: map[string]any{ + "MetaCompleted": uint64(2), + "MetaTotal": uint64(2), + "BytesToCompact": uint64(7), + "BytesCompacted": uint64(8), + }}, + {cmd: "OnProgress", attrs: map[string]any{ + "MetaCompleted": uint64(2), + "MetaTotal": uint64(2), + "BytesToCompact": uint64(8), + "BytesCompacted": uint64(8), + }}, + {cmd: "OnFinish", attrs: map[string]any{ + "error": error(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/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 8367f02c8cf..57abb8805a9 100644 --- a/docs/api-references/docs.md +++ b/docs/api-references/docs.md @@ -3619,7 +3619,8 @@ bool
(Appears on: -BackupSpec) +BackupSpec, +CompactSpec)
BackoffRetryPolicy is the backoff retry policy, currently only valid for snapshot backup. @@ -5335,6 +5336,772 @@ 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: +CompactBackup) +
++
BackupSpec 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 + |
+
+from
+
+
+TiDBAccessConfig
+
+
+ |
+
+ From is the tidb cluster that needs to backup. + |
+
+tikvGCLifeTime
+
+string
+
+ |
++ | +
+StorageProvider
+
+
+StorageProvider
+
+
+ |
+
+
+(Members of StorageProvider configures where and how backups should be stored. + |
+
+storageClassName
+
+string
+
+ |
+
+(Optional)
+ The storageClassName of the persistent volume for Backup data storage. +Defaults to Kubernetes default storage class. + |
+
+storageSize
+
+string
+
+ |
+
+ StorageSize is the request storage size for backup job + |
+
+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
+
+ |
+
+(Optional)
+ ResumeGcSchedule indicates whether resume gc and pd scheduler for EBS volume snapshot backup +Concurrency is the concurrency of compact backup job + |
+
+resumeGcSchedule
+
+bool
+
+ |
++ | +
+tolerations
+
+
+[]Kubernetes core/v1.Toleration
+
+
+ |
+
+(Optional)
+ Base tolerations of backup Pods, components may add more tolerations upon this respectively + |
+
+version
+
+string
+
+ |
+
+ Version specifies the tool image version used in compact |
+
+brImage
+
+string
+
+ |
+
+(Optional)
+ BrImage specifies the br image used in compact |
+
+tikvImage
+
+string
+
+ |
+
+(Optional)
+ TiKVImage specifies the tikv image used in compact |
+
+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 + |
+
+backoffRetryPolicy
+
+
+BackoffRetryPolicy
+
+
+ |
+
+ 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. + |
+
+volumeBackupInitJobMaxActiveSeconds
+
+int
+
+ |
+
+ VolumeBackupInitJobMaxActiveSeconds represents the deadline (in seconds) of the vbk init job + |
+
+(Appears on: +CompactBackup) +
++
+Field | +Description | +
---|---|
+state
+
+string
+
+ |
++ | +
ComponentAccessor is the interface to access component details, which respects the cluster-level properties @@ -15861,6 +16628,7 @@ More info: BackupSpec, +CompactSpec, RestoreSpec)
@@ -16750,6 +17518,7 @@ map[github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.StorageVolumeName
(Appears on: BackupSpec, +CompactSpec, RestoreSpec)
diff --git a/go.mod b/go.mod index c9ad495768e..dcd301c2a82 100644 --- a/go.mod +++ b/go.mod @@ -214,6 +214,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 317846741e7..0ce9d0a1c79 100644 --- a/go.sum +++ b/go.sum @@ -1155,6 +1155,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/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/crd.yaml b/manifests/crd.yaml index 063308006fe..23946573609 100644 --- a/manifests/crd.yaml +++ b/manifests/crd.yaml @@ -7051,6 +7051,2195 @@ spec: --- 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 + 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 + backoffRetryPolicy: + properties: + maxRetryTimes: + default: 2 + type: integer + minRetryDuration: + default: 300s + type: string + retryTimeout: + default: 30m + type: string + type: object + brImage: + type: string + concurrency: + 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 + 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 + 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 + 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 + startTs: + type: string + storageClassName: + type: string + storageSize: + type: string + tikvGCLifeTime: + type: string + tikvImage: + 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 + useKMS: + type: boolean + version: + type: string + volumeBackupInitJobMaxActiveSeconds: + default: 600 + type: integer + type: object + status: + properties: + state: + 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 diff --git a/manifests/crd/v1/pingcap.com_compactbackups.yaml b/manifests/crd/v1/pingcap.com_compactbackups.yaml new file mode 100644 index 00000000000..e3b434ae2df --- /dev/null +++ b/manifests/crd/v1/pingcap.com_compactbackups.yaml @@ -0,0 +1,2189 @@ +--- +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 + 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 + backoffRetryPolicy: + properties: + maxRetryTimes: + default: 2 + type: integer + minRetryDuration: + default: 300s + type: string + retryTimeout: + default: 30m + type: string + type: object + brImage: + type: string + concurrency: + 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 + 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 + 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 + 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 + startTs: + type: string + storageClassName: + type: string + storageSize: + type: string + tikvGCLifeTime: + type: string + tikvImage: + 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 + useKMS: + type: boolean + version: + type: string + volumeBackupInitJobMaxActiveSeconds: + default: 600 + type: integer + type: object + status: + properties: + state: + type: string + type: object + required: + - metadata + - spec + type: object + served: true + storage: true + subresources: {} diff --git a/pkg/apis/pingcap/v1alpha1/openapi_generated.go b/pkg/apis/pingcap/v1alpha1/openapi_generated.go index 4f789a1a7e3..9c2de35e064 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,317 @@ 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: "BackupSpec 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"), + }, + }, + }, + }, + }, + "from": { + SchemaProps: spec.SchemaProps{ + Description: "From is the tidb cluster that needs to backup.", + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiDBAccessConfig"), + }, + }, + "tikvGCLifeTime": { + SchemaProps: spec.SchemaProps{ + Type: []string{"string"}, + Format: "", + }, + }, + "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"), + }, + }, + "storageClassName": { + SchemaProps: spec.SchemaProps{ + Description: "The storageClassName of the persistent volume for Backup data storage. Defaults to Kubernetes default storage class.", + Type: []string{"string"}, + Format: "", + }, + }, + "storageSize": { + SchemaProps: spec.SchemaProps{ + Description: "StorageSize is the request storage size for backup job", + Type: []string{"string"}, + Format: "", + }, + }, + "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: "ResumeGcSchedule indicates whether resume gc and pd scheduler for EBS volume snapshot backup Concurrency is the concurrency of compact backup job", + Default: 4, + Type: []string{"integer"}, + Format: "int32", + }, + }, + "resumeGcSchedule": { + SchemaProps: spec.SchemaProps{ + Type: []string{"boolean"}, + Format: "", + }, + }, + "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"), + }, + }, + }, + }, + }, + "version": { + SchemaProps: spec.SchemaProps{ + Description: "Version specifies the tool image version used in compact `Backup`.", + Type: []string{"string"}, + Format: "", + }, + }, + "brImage": { + 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: "", + }, + }, + "tikvImage": { + SchemaProps: spec.SchemaProps{ + Description: "TiKVImage specifies the tikv image used in compact `Backup`. For examples `spec.tikvImage: pingcap/tikv:v4.0.8`", + Type: []string{"string"}, + Format: "", + }, + }, + "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: "", + }, + }, + "backoffRetryPolicy": { + SchemaProps: spec.SchemaProps{ + Description: "BackoffRetryPolicy the backoff retry policy, currently only valid for snapshot backup", + Default: map[string]interface{}{}, + Ref: ref("github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.BackoffRetryPolicy"), + }, + }, + "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"), + }, + }, + }, + }, + }, + "volumeBackupInitJobMaxActiveSeconds": { + SchemaProps: spec.SchemaProps{ + Description: "VolumeBackupInitJobMaxActiveSeconds represents the deadline (in seconds) of the vbk init job", + Type: []string{"integer"}, + Format: "int32", + }, + }, + }, + }, + }, + Dependencies: []string{ + "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.AzblobStorageProvider", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.BackoffRetryPolicy", "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", "github.com/pingcap/tidb-operator/pkg/apis/pingcap/v1alpha1.TiDBAccessConfig", "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 5532fc5660f..8f7fa952803 100644 --- a/pkg/apis/pingcap/v1alpha1/types.go +++ b/pkg/apis/pingcap/v1alpha1/types.go @@ -3431,3 +3431,128 @@ 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" +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"` +} + +// BackupSpec 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"` + // From is the tidb cluster that needs to backup. + From *TiDBAccessConfig `json:"from,omitempty"` + TikvGCLifeTime *string `json:"tikvGCLifeTime,omitempty"` + // StorageProvider configures where and how backups should be stored. + StorageProvider `json:",inline"` + // The storageClassName of the persistent volume for Backup data storage. + // Defaults to Kubernetes default storage class. + // +optional + StorageClassName *string `json:"storageClassName,omitempty"` + // StorageSize is the request storage size for backup job + StorageSize string `json:"storageSize,omitempty"` + // 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"` + // ResumeGcSchedule indicates whether resume gc and pd scheduler for EBS volume snapshot backup + // +optional + // Concurrency is the concurrency of compact backup job + // +default=4 + Concurrency int `json:"concurrency,omitempty"` + ResumeGcSchedule bool `json:"resumeGcSchedule,omitempty"` + // Base tolerations of backup Pods, components may add more tolerations upon this respectively + // +optional + Tolerations []corev1.Toleration `json:"tolerations,omitempty"` + // Version specifies the tool image version used in compact `Backup`. + Version string `json:"version,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 + BrImage string `json:"brImage,omitempty"` + // TiKVImage specifies the tikv image used in compact `Backup`. + // For examples `spec.tikvImage: pingcap/tikv:v4.0.8` + // +optional + TiKVImage string `json:"tikvImage,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 + BackoffRetryPolicy BackoffRetryPolicy `json:"backoffRetryPolicy,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"` + // VolumeBackupInitJobMaxActiveSeconds represents the deadline (in seconds) of the vbk init job + // +kubebuilder:default=600 + VolumeBackupInitJobMaxActiveSeconds int `json:"volumeBackupInitJobMaxActiveSeconds,omitempty"` +} + +type CompactStatus struct { + State string `json:"state,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 b65b9dca1d3..51b722ada76 100644 --- a/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go +++ b/pkg/apis/pingcap/v1alpha1/zz_generated.deepcopy.go @@ -865,6 +865,160 @@ 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) + out.Status = in.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 *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]) + } + } + if in.From != nil { + in, out := &in.From, &out.From + *out = new(TiDBAccessConfig) + (*in).DeepCopyInto(*out) + } + if in.TikvGCLifeTime != nil { + in, out := &in.TikvGCLifeTime, &out.TikvGCLifeTime + *out = new(string) + **out = **in + } + in.StorageProvider.DeepCopyInto(&out.StorageProvider) + if in.StorageClassName != nil { + in, out := &in.StorageClassName, &out.StorageClassName + *out = new(string) + **out = **in + } + 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.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) + } + out.BackoffRetryPolicy = in.BackoffRetryPolicy + 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 + 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/compactbackup/compact_backup_controller.go b/pkg/controller/compactbackup/compact_backup_controller.go new file mode 100644 index 00000000000..e7ee80e2f3a --- /dev/null +++ b/pkg/controller/compactbackup/compact_backup_controller.go @@ -0,0 +1,437 @@ +package compact + +import ( + "context" + "fmt" + "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/pointer" +) + +// Controller controls backup. +type Controller struct { + deps *controller.Dependencies + // backups that need to be synced. + queue workqueue.RateLimitingInterface + cli versioned.Interface +} + +// 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, + } + + backupInformer := deps.InformerFactory.Pingcap().V1alpha1().CompactBackups() + jobInformer := deps.KubeInformerFactory.Batch().V1().Jobs() + backupInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: c.updateBackup, + UpdateFunc: func(old, cur interface{}) { + c.updateBackup(cur) + }, + DeleteFunc: c.updateBackup, + }) + jobInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{ + DeleteFunc: c.deleteJob, + }) + + return c +} + +// Name returns backup controller name. +func (c *Controller) Name() string { + return "compactBackup" +} + +// Run runs the 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(backup *v1alpha1.CompactBackup, newState string) error { + ns := backup.GetNamespace() + backupName := backup.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(backupName); err == nil { + // make a copy so we don't mutate the shared cache + *backup = *(updated.DeepCopy()) + } else { + utilruntime.HandleError(fmt.Errorf("error getting updated backup %s/%s from lister: %v", ns, backupName, err)) + return err + } + if backup.Status.State != newState { + backup.Status.State = newState + _, updateErr := c.cli.PingcapV1alpha1().CompactBackups(ns).Update(context.TODO(), backup, metav1.UpdateOptions{}) + if updateErr == nil { + klog.Infof("Backup: [%s/%s] updated successfully", ns, backupName) + return nil + } + klog.Errorf("Failed to update backup [%s/%s], error: %v", ns, backupName, 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 + } + + backup, err := c.deps.CompactBackupLister.CompactBackups(namespace).Get(owner.Name) + if err != nil { + return nil + } + if owner.UID != backup.UID { + return nil + } + return backup +} + +func (c *Controller) deleteJob(obj interface{}) { + job, ok := obj.(*batchv1.Job) + if !ok { + return + } + + ns := job.GetNamespace() + jobName := job.GetName() + backup := c.resolveCompactBackupFromJob(ns, job) + if backup == nil { + return + } + klog.V(4).Infof("Job %s/%s deleted through %v.", ns, jobName, utilruntime.GetCaller()) + c.updateBackup(backup) +} + +func (c *Controller) updateBackup(cur interface{}) { + newBackup := cur.(*v1alpha1.CompactBackup) + ns := newBackup.GetNamespace() + name := newBackup.GetName() + + klog.Infof("backup object %s/%s enqueue", ns, name) + c.enqueueBackup(newBackup) +} + +// enqueueBackup enqueues the given backup in the work queue. +func (c *Controller) enqueueBackup(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("Backup: %v, still need sync: %v, requeuing", key.(string), err) + c.queue.AddRateLimited(key) + } else if perrors.Find(err, controller.IsIgnoreError) != nil { + klog.Infof("Backup: %v, ignore err: %v", key.(string), err) + } else { + utilruntime.HandleError(fmt.Errorf("Backup: %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 Backup %q (%v)", key, duration) + }() + + ns, name, err := cache.SplitMetaNamespaceKey(key) + if err != nil { + return err + } + klog.Infof("Backup: [%s/%s] start to sync", ns, name) + backup, err := c.deps.CompactBackupLister.CompactBackups(ns).Get(name) + if err != nil { + if errors.IsNotFound(err) { + klog.Infof("Backup has been deleted %v", key) + return nil + } + klog.Infof("Backup get failed %v", err) + return err + } + + //Skip + if backup.Status.State != "" { + return nil + } + + c.UpdateStatus(backup, "Preparing") + + err = c.doCompact(backup.DeepCopy()) + + var newState string + if err != nil { + newState = "Failed" + } else { + newState = "Running" + } + c.UpdateStatus(backup, newState) + return err +} + +func (c *Controller) doCompact(backup *v1alpha1.CompactBackup) error { + ns := backup.GetNamespace() + name := backup.GetName() + backupJobName := backup.GetName() + + // make backup job + var err error + var job *batchv1.Job + var reason string + if job, reason, err = c.makeBackupJob(backup); err != nil { + klog.Errorf("backup %s/%s create job %s failed, reason is %s, error %v.", ns, name, backupJobName, reason, err) + return err + } + + // create k8s job + klog.Infof("backup %s/%s creating job %s.", ns, name, backupJobName) + if err := c.deps.JobControl.CreateJob(backup, job); err != nil { + errMsg := fmt.Errorf("create backup %s/%s job %s failed, err: %v", ns, name, backupJobName, err) + return errMsg + } + return nil +} + +func (c *Controller) makeBackupJob(backup *v1alpha1.CompactBackup) (*batchv1.Job, string, error) { + ns := backup.GetNamespace() + name := backup.GetName() + jobName := backup.GetName() + + var ( + envVars []corev1.EnvVar + reason string + err error + ) + + storageEnv, reason, err := backuputil.GenerateStorageCertEnv(ns, backup.Spec.UseKMS, backup.Spec.StorageProvider, c.deps.SecretLister) + if err != nil { + return nil, reason, fmt.Errorf("backup %s/%s, %v", ns, name, err) + } + + envVars = append(envVars, storageEnv...) + + // set env vars specified in backup.Spec.Env + envVars = util.AppendOverwriteEnv(envVars, backup.Spec.Env) + + args := []string{ + "compact", + fmt.Sprintf("--namespace=%s", ns), + fmt.Sprintf("--resourceName=%s", name), + } + + tikvImage := "pingcap/tikv" + if backup.Spec.TiKVImage != "" { + tikvImage = backup.Spec.TiKVImage + } + + tikvVersion := backup.Spec.Version + _, imageVersion := backuputil.ParseImage(tikvImage) + if imageVersion != "" { + tikvVersion = imageVersion + } + if tikvVersion != "" { + args = append(args, fmt.Sprintf("--tikvVersion=%s", tikvVersion)) + } else { + return nil, "tikv version is empty", fmt.Errorf("tikv version is empty") + } + + brImage := fmt.Sprintf("pingcap/br:%s", tikvVersion) + if backup.Spec.BrImage != "" { + brImage = backup.Spec.BrImage + if !strings.ContainsRune(brImage, ':') { + brImage = fmt.Sprintf("%s:%s", brImage, tikvVersion) + } + } + + //TODO: (Ris)What is the instance here? + jobLabels := util.CombineStringMap(label.NewBackup().Instance("Compact-Backup").BackupJob().Backup(name), backup.Labels) + podLabels := jobLabels + jobAnnotations := backup.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(backup.Spec.AdditionalVolumes) > 0 { + volumes = append(volumes, backup.Spec.AdditionalVolumes...) + } + if len(backup.Spec.AdditionalVolumeMounts) > 0 { + volumeMounts = append(volumeMounts, backup.Spec.AdditionalVolumeMounts...) + } + + // mount volumes if specified + if backup.Spec.Local != nil { + volumes = append(volumes, backup.Spec.Local.Volume) + volumeMounts = append(volumeMounts, backup.Spec.Local.VolumeMount) + } + + serviceAccount := constants.DefaultServiceAccountName + if backup.Spec.ServiceAccount != "" { + serviceAccount = backup.Spec.ServiceAccount + } + + podSpec := &corev1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: podLabels, + Annotations: podAnnotations, + }, + Spec: corev1.PodSpec{ + SecurityContext: backup.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: backup.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: backup.Spec.ResourceRequirements, + }, + }, + Containers: []corev1.Container{ + { + Name: "backup-manager", + Image: c.deps.CLIConfig.TiDBBackupManagerImage, + Args: args, + Env: envVars, + VolumeMounts: volumeMounts, + ImagePullPolicy: corev1.PullAlways, + }, + }, + RestartPolicy: corev1.RestartPolicyNever, + Tolerations: backup.Spec.Tolerations, + ImagePullSecrets: backup.Spec.ImagePullSecrets, + Affinity: backup.Spec.Affinity, + Volumes: volumes, + PriorityClassName: backup.Spec.PriorityClassName, + }, + } + + job := &batchv1.Job{ + ObjectMeta: metav1.ObjectMeta{ + Name: jobName, + Namespace: ns, + Labels: jobLabels, + Annotations: jobAnnotations, + OwnerReferences: []metav1.OwnerReference{ + controller.GetCompactBackupOwnerRef(backup), + }, + }, + Spec: batchv1.JobSpec{ + BackoffLimit: pointer.Int32Ptr(0), + Template: *podSpec, + }, + } + + return job, "", nil +} 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..34878c0c44b 100644 --- a/pkg/controller/dependences.go +++ b/pkg/controller/dependences.go @@ -233,6 +233,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 +379,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"