diff --git a/gen/deprecations/cmd.go b/gen/deprecations/cmd.go new file mode 100644 index 000000000..b4c167d88 --- /dev/null +++ b/gen/deprecations/cmd.go @@ -0,0 +1,120 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package deprecations + +import ( + _ "embed" + "fmt" + "go/ast" + "os" + "path/filepath" + "strings" + "text/template" + "time" + + "github.com/spf13/cobra" +) + +const DeprecationPrefix = "Deprecated" + +var ( + testGenerator *template.Template + //go:embed templates/test.tpl + testTemplate string +) + +func init() { + helpers := map[string]any{ + "year": func() string { + return time.Now().Format("2006") + }, + } + testGenerator = template.Must(template.New("tests").Funcs(helpers).Parse(testTemplate)) +} + +type DeprecationConfig struct { + Directory string + PackageName string + OutputFile string + Verbose bool +} + +type fieldRef struct { + GoPath []string // Go field names e.g. [Spec ClusterSpec DeprecatedFullNameOverride] + JsonPath []string // JSON names e.g. [spec clusterSpec fullNameOverride] + TypeExpr ast.Expr +} + +type objSpec struct { + Name string + Literal string + Warnings []string +} + +// Debugf emits debug output when the config's Verbose flag is set. +func (c DeprecationConfig) Debugf(format string, a ...interface{}) { + if !c.Verbose { + return + } + fmt.Fprintf(os.Stderr, format, a...) +} + +func Cmd() *cobra.Command { + var config DeprecationConfig + + cmd := &cobra.Command{ + Use: "deprecations", + Short: "Generate tests for deprecated API fields", + Example: "gen deprecations --directory ./operator/api/redpanda/v1alpha2", + RunE: func(cmd *cobra.Command, args []string) error { + return Render(config) + }, + } + + cmd.Flags().StringVar(&config.Directory, "directory", ".", "The directory to scan for deprecated fields") + cmd.Flags().StringVar(&config.PackageName, "package", "", "The name of the package, if not specified we try and figure it out dynamically") + cmd.Flags().StringVar(&config.OutputFile, "output-file", "zz_generated.deprecations_test.go", "The name of the file to output in the given directory") + cmd.Flags().BoolVarP(&config.Verbose, "verbose", "v", false, "Enable debug output.") + + return cmd +} + +func Render(config DeprecationConfig) error { + dir := config.Directory + + if config.PackageName == "" { + config.PackageName = strings.Trim(filepath.Base(dir), ".") + config.PackageName = strings.Trim(filepath.Base(dir), "/") + } + + if config.PackageName == "" { + return fmt.Errorf("could not determine package name") + } + + parser := NewParser(config) + + if err := parser.Parse(); err != nil { + return err + } + + contents, err := parser.Compile() + if err != nil { + return err + } + + outPath := filepath.Join(dir, config.OutputFile) + if err := os.WriteFile(outPath, contents, 0o644); err != nil { + return fmt.Errorf("failed to write output file: %w", err) + } + + fmt.Printf("wrote %s\n", outPath) + + return nil +} diff --git a/gen/deprecations/helpers.go b/gen/deprecations/helpers.go new file mode 100644 index 000000000..8c1d3a3f3 --- /dev/null +++ b/gen/deprecations/helpers.go @@ -0,0 +1,344 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package deprecations + +import ( + "fmt" + "go/ast" + "go/token" + "strconv" + "strings" +) + +// helpers for AST traversal +func isCRDRoot(st *ast.StructType) bool { + for _, f := range st.Fields.List { + if sel, ok := f.Type.(*ast.SelectorExpr); ok { + if id, ok := sel.X.(*ast.Ident); ok && id.Name == "metav1" && sel.Sel.Name == "TypeMeta" { + return true + } + } + } + return false +} + +func findFieldByGoName(st *ast.StructType, name string) *ast.Field { + for _, f := range st.Fields.List { + for _, n := range f.Names { + if n.Name == name { + return f + } + } + } + return nil +} + +func typeNameOf(expr ast.Expr) string { + switch t := expr.(type) { + case *ast.Ident: + return t.Name + case *ast.StarExpr: + return typeNameOf(t.X) + case *ast.SelectorExpr: + if id, ok := t.X.(*ast.Ident); ok { + return id.Name + "." + t.Sel.Name + } + return t.Sel.Name + default: + return "" + } +} + +func jsonNameForField(f *ast.Field) string { + if f.Tag == nil { + // use lowercased name if available + if len(f.Names) > 0 { + return strings.ToLower(f.Names[0].Name[:1]) + f.Names[0].Name[1:] + } + return "" + } + s, err := strconv.Unquote(f.Tag.Value) + if err != nil { + return "" + } + tag := reflectStructTag(s) + if j, ok := tag["json"]; ok { + parts := strings.Split(j, ",") + if parts[0] != "" && parts[0] != "-" { + return parts[0] + } + } + if len(f.Names) > 0 { + return strings.ToLower(f.Names[0].Name[:1]) + f.Names[0].Name[1:] + } + return "" +} + +// reflectStructTag extracts tag key/value pairs from a raw struct tag string +func reflectStructTag(tag string) map[string]string { + m := map[string]string{} + for tag != "" { + // skip leading space + tag = strings.TrimLeft(tag, " ") + if tag == "" { + break + } + // key + i := strings.Index(tag, ":") + if i < 0 { + break + } + key := tag[:i] + tag = tag[i+1:] + if tag == "" || tag[0] != '"' { + break + } + // quoted value + j := 1 + for j < len(tag) { + if tag[j] == '\\' { + j += 2 + continue + } + if tag[j] == '"' { + break + } + j++ + } + if j >= len(tag) { + break + } + val := tag[1:j] + m[key] = val + tag = tag[j+1:] + } + return m +} + +func findStructByFieldNames(structs map[string]*ast.StructType, names []string) string { + need := map[string]struct{}{} + for _, n := range names { + need[n] = struct{}{} + } + for typeName, st := range structs { + have := map[string]struct{}{} + for _, f := range st.Fields.List { + for _, id := range f.Names { + have[id.Name] = struct{}{} + } + } + ok := true + for n := range need { + if _, found := have[n]; !found { + ok = false + break + } + } + if ok { + return typeName + } + } + return "" +} + +func extractPoorlyNamedDeprecatedFields(files []*ast.File, structs map[string]*ast.StructType) map[string][]string { + result := make(map[string][]string) + + // Map to track fields with Deprecated prefix (these we already handle) + deprecatedPrefixed := make(map[string]map[string]bool) + for typeName, st := range structs { + deprecatedPrefixed[typeName] = make(map[string]bool) + for _, f := range st.Fields.List { + if len(f.Names) > 0 && strings.HasPrefix(f.Names[0].Name, DeprecationPrefix) { + deprecatedPrefixed[typeName][f.Names[0].Name] = true + } + } + } + + // Scan through all AST nodes looking for struct types with field comments containing "Deprecated" + for _, f := range files { + for _, decl := range f.Decls { + gd, ok := decl.(*ast.GenDecl) + if !ok || gd.Tok != token.TYPE { + continue + } + + for _, spec := range gd.Specs { + ts, ok := spec.(*ast.TypeSpec) + if !ok { + continue + } + + st, ok := ts.Type.(*ast.StructType) + if !ok { + continue + } + + typeName := ts.Name.Name + var foundFields []string + + for _, field := range st.Fields.List { + if len(field.Names) == 0 { + continue + } + + fieldName := field.Names[0].Name + + // Skip fields that already have Deprecated prefix + if strings.HasPrefix(fieldName, DeprecationPrefix) { + continue + } + + // Check if the field has a comment mentioning "Deprecated" or "deprecated" + if field.Comment != nil { + for _, comment := range field.Comment.List { + if strings.Contains(strings.ToLower(comment.Text), DeprecationPrefix) { + foundFields = append(foundFields, fieldName) + break + } + } + } + } + + // Also check doc comments attached to fields + if len(foundFields) == 0 && st.Fields != nil { + for _, field := range st.Fields.List { + if len(field.Names) == 0 { + continue + } + fieldName := field.Names[0].Name + + // Skip fields that already have Deprecated prefix + if strings.HasPrefix(fieldName, DeprecationPrefix) { + continue + } + + // Check doc comments + if field.Doc != nil { + for _, comment := range field.Doc.List { + if strings.Contains(strings.ToLower(comment.Text), strings.ToLower(DeprecationPrefix)) { + foundFields = append(foundFields, fieldName) + break + } + } + } + } + } + + if len(foundFields) > 0 { + result[typeName] = foundFields + } + } + } + } + + return result +} + +func literalForFieldType(fieldTypeName string, isPtr bool) string { + if fieldTypeName != "" { + if isPtr { + return fmt.Sprintf("ptr.To(%s)", literalForIdentity(fieldTypeName)) + } + return literalForIdentity(fieldTypeName) + } + return "struct{}{}" +} + +func literalForIdentity(name string) string { + switch name { + case "string": + return "\"deprecated\"" + case "bool": + return "true" + case "int", "int32", "int64": + return "1" + default: + return fmt.Sprintf("%s{}", name) + } +} + +func literalForTypeExpr(e ast.Expr) string { + switch t := e.(type) { + case *ast.StarExpr: + // pointer to some type + if id, ok := t.X.(*ast.Ident); ok { + name := id.Name + // default: non-nil pointer to empty struct + return fmt.Sprintf("ptr.To(%s)", literalForIdentity(name)) + } + if sel, ok := t.X.(*ast.SelectorExpr); ok { + return fmt.Sprintf("ptr.To(%s)", literalForIdentity(sel.X.(*ast.Ident).Name+"."+sel.Sel.Name)) + } + case *ast.Ident: + return literalForIdentity(t.Name) + case *ast.SelectorExpr: + // Qualified identifier like pkg.Type -> pkg.Type{} + if x, ok := t.X.(*ast.Ident); ok { + return literalForIdentity(x.Name + "." + t.Sel.Name) + } + return literalForIdentity(t.Sel.Name) + } + return "nil" +} + +func buildNodeFromFieldRefs(refs []fieldRef) *node { + tmp := &node{children: map[string]*node{}} + for _, dr := range refs { + cur := tmp + for i, seg := range dr.GoPath { + isLast := i == len(dr.GoPath)-1 + if cur.children[seg] == nil { + cur.children[seg] = &node{children: map[string]*node{}} + } + cur = cur.children[seg] + if isLast { + cur.typeExpr = dr.TypeExpr + } + } + } + return tmp +} + +func extractFieldType(st *ast.StructType, fname string) (typeName string, isPtr bool) { + if st == nil { + return "", false + } + f := findFieldByGoName(st, fname) + if f == nil { + return "", false + } + + if star, ok := f.Type.(*ast.StarExpr); ok { + isPtr = true + if id, ok := star.X.(*ast.Ident); ok { + typeName = id.Name + } else if sel, ok := star.X.(*ast.SelectorExpr); ok { + typeName = sel.Sel.Name + } + } else if id, ok := f.Type.(*ast.Ident); ok { + typeName = id.Name + } else if sel, ok := f.Type.(*ast.SelectorExpr); ok { + typeName = sel.Sel.Name + } + return typeName, isPtr +} + +func extractSpecType(specField *ast.Field) string { + if specField == nil { + return "" + } + specType := typeNameOf(specField.Type) + if strings.Contains(specType, ".") { + parts := strings.Split(specType, ".") + specType = parts[len(parts)-1] + } + return specType +} diff --git a/gen/deprecations/parser.go b/gen/deprecations/parser.go new file mode 100644 index 000000000..5ad21dc05 --- /dev/null +++ b/gen/deprecations/parser.go @@ -0,0 +1,441 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package deprecations + +import ( + "bytes" + "errors" + "fmt" + "go/ast" + "go/format" + "go/parser" + "go/token" + "path/filepath" + "sort" + "strings" + + "github.com/redpanda-data/redpanda-operator/gen/internal" +) + +type Parser struct { + config DeprecationConfig + structs map[string]*ast.StructType + files []*ast.File + crdDeprecations map[string][]fieldRef + allDeprecations map[string][]fieldRef +} + +func NewParser(config DeprecationConfig) *Parser { + return &Parser{ + config: config, + structs: make(map[string]*ast.StructType), + crdDeprecations: make(map[string][]fieldRef), + allDeprecations: make(map[string][]fieldRef), + } +} + +func (p *Parser) Parse() error { + fset := token.NewFileSet() + matches, err := filepath.Glob(filepath.Join(p.config.Directory, "*.go")) + if err != nil { + return fmt.Errorf("failed to glob files: %w", err) + } + + pkgName := "" + + for _, fname := range matches { + base := filepath.Base(fname) + + // skip any files that are test files or generated by us + if base == p.config.OutputFile || strings.HasSuffix(base, "_test.go") { + continue + } + + f, err := parser.ParseFile(fset, fname, nil, parser.ParseComments) + if err != nil { + return fmt.Errorf("failed to parse file %s: %w", fname, err) + } + + if pkgName == "" { + pkgName = f.Name.Name + } + p.files = append(p.files, f) + + // gather any structs we find + for _, decl := range f.Decls { + gd, ok := decl.(*ast.GenDecl) + if !ok || gd.Tok != token.TYPE { + continue + } + for _, spec := range gd.Specs { + ts := spec.(*ast.TypeSpec) + st, ok := ts.Type.(*ast.StructType) + if !ok { + continue + } + p.structs[ts.Name.Name] = st + } + } + } + + return nil +} + +func (p *Parser) Compile() ([]byte, error) { + p.compileAllDeprecations() + p.compileDeprecatedCRDs() + + var fileContents bytes.Buffer + if err := testGenerator.Execute(&fileContents, map[string]any{ + "Pkg": p.config.PackageName, + "Objs": p.buildCRDObjects(), + "TODOComment": p.compileTODO(), + }); err != nil { + return nil, fmt.Errorf("failed to execute template: %w", err) + } + + formatted, err := format.Source(fileContents.Bytes()) + if err != nil { + if contextualErrors := internal.ContextualizeFormatErrors(fileContents.Bytes(), err); contextualErrors != "" { + return nil, errors.New(contextualErrors) + } + } + + return formatted, err +} + +func (p *Parser) compileAllDeprecations() { + for tn := range p.structs { + refs := p.collectDeprecations(tn, []string{}, []string{}) + p.config.Debugf("[deprecations] type %s returned %d deprecated fields\n", tn, len(refs)) + if len(refs) > 0 { + p.allDeprecations[tn] = refs + } + } +} + +func (p *Parser) compileDeprecatedCRDs() { + for typeName, st := range p.structs { + if isCRDRoot(st) { + specField := findFieldByGoName(st, "Spec") + if specField == nil { + continue + } + specTypeName := typeNameOf(specField.Type) + if specTypeName == "" { + continue + } + refs := p.collectDeprecations(specTypeName, []string{"Spec"}, []string{jsonNameForField(specField)}) + p.config.Debugf("[deprecations] CRD root %q returned %d deprecated fields\n", typeName, len(refs)) + if len(refs) > 0 { + p.crdDeprecations[typeName] = refs + } + } + } +} + +// collectDeprecations recursively finds Deprecated-prefixed fields starting from a named type +func (p *Parser) collectDeprecations(typeName string, goPath []string, jsonPath []string) []fieldRef { + st, ok := p.structs[typeName] + if !ok { + return nil + } + var out []fieldRef + for _, f := range st.Fields.List { + // skip anonymous fields + if len(f.Names) == 0 { + continue + } + + name := f.Names[0].Name + jname := jsonNameForField(f) + if strings.HasPrefix(name, DeprecationPrefix) { + fr := fieldRef{ + GoPath: append(append([]string{}, goPath...), name), + JsonPath: append(append([]string{}, jsonPath...), jname), + TypeExpr: f.Type, + } + + p.config.Debugf("[deprecations] found deprecated %q at path %v\n", name, fr.GoPath) + + out = append(out, fr) + continue + } + + refType := typeNameOf(f.Type) + if strings.Contains(refType, ".") { + parts := strings.Split(refType, ".") + refType = parts[len(parts)-1] + } + + if _, exists := p.structs[refType]; exists { + out = append(out, p.collectDeprecations(refType, append(append([]string{}, goPath...), name), append(append([]string{}, jsonPath...), jname))...) + } + } + return out +} + +func (p *Parser) compileTODO() string { + poorlyNamed := extractPoorlyNamedDeprecatedFields(p.files, p.structs) + crdsWithPoorlyNamedFields := make(map[string][]string) + + for crdName := range p.crdDeprecations { + crdSt := p.structs[crdName] + if crdSt == nil { + continue + } + specField := findFieldByGoName(crdSt, "Spec") + if specField == nil { + continue + } + specTypeName := typeNameOf(specField.Type) + if specTypeName == "" { + continue + } + var fieldPaths []string + p.flattenPoorlyNamedFieldsToGoPaths(specTypeName, poorlyNamed, "", &fieldPaths) + if len(fieldPaths) > 0 { + crdsWithPoorlyNamedFields[crdName] = fieldPaths + } + } + + var todoCommentLines []string + todoCommentLines = append(todoCommentLines, "// TODO: The following fields are documented as deprecated in the API comments") + todoCommentLines = append(todoCommentLines, "// but their Go field names do not use the `Deprecated` prefix. Consider") + todoCommentLines = append(todoCommentLines, "// renaming them (or adding wrapper fields with the Deprecated prefix) so the") + todoCommentLines = append(todoCommentLines, "// reflective deprecation detector can find them uniformly. Examples found in") + todoCommentLines = append(todoCommentLines, "// this package:") + todoCommentLines = append(todoCommentLines, "//") + + var crdNames []string + for crdName := range crdsWithPoorlyNamedFields { + crdNames = append(crdNames, crdName) + } + sort.Strings(crdNames) + + for _, crdName := range crdNames { + fieldPaths := crdsWithPoorlyNamedFields[crdName] + todoCommentLines = append(todoCommentLines, fmt.Sprintf("// - %s:", crdName)) + for _, fieldPath := range fieldPaths { + todoCommentLines = append(todoCommentLines, fmt.Sprintf("// - %s", fieldPath)) + } + } + + return strings.Join(todoCommentLines, "\n") +} + +type node struct { + name string + children map[string]*node + warnings []string + typeExpr ast.Expr +} + +func (p *Parser) buildRoots() []*node { + nodes := []*node{} + for tn, refs := range p.crdDeprecations { + warnings := make([]string, len(refs)) + for i, ref := range refs { + warnings[i] = fmt.Sprintf("field '%s' is deprecated and set", strings.Join(ref.JsonPath, ".")) + } + + root := &node{name: tn, children: make(map[string]*node), warnings: warnings} + for _, r := range refs { + cur := root + for i, seg := range r.GoPath { + isLast := i == len(r.GoPath)-1 + if cur.children[seg] == nil { + cur.children[seg] = &node{children: make(map[string]*node)} + } + cur = cur.children[seg] + if isLast { + cur.typeExpr = r.TypeExpr + } + } + } + nodes = append(nodes, root) + } + + return nodes +} + +func (p *Parser) buildCRDObjects() []objSpec { + objs := []objSpec{} + for _, root := range p.buildRoots() { + objs = append(objs, objSpec{Name: root.name, Literal: p.renderCRDLiteral(root), Warnings: root.warnings}) + } + + sort.SliceStable(objs, func(i, j int) bool { + return objs[i].Name < objs[j].Name + }) + + return objs +} + +func (p *Parser) renderCRDLiteral(root *node) string { + return fmt.Sprintf("&%s{\nSpec: %s,\n}", root.name, p.renderSpecLiteral(root)) +} + +func (p *Parser) renderSpecLiteral(rootNode *node) string { + rootSt := p.structs[rootNode.name] + specField := findFieldByGoName(rootSt, "Spec") + specType := extractSpecType(specField) + + if specType != "" { + if rootNode.children["Spec"] == nil { + rootNode.children["Spec"] = &node{children: map[string]*node{}} + } + return p.renderLiteral(specType, rootNode.children["Spec"]) + } + return "struct{}{}" +} + +func (p *Parser) renderLiteral(typeName string, n *node) string { + children := 0 + if n != nil { + children = len(n.children) + } + + p.config.Debugf("[render] typeName=%s, children=%d\n", typeName, children) + if n == nil { + return fmt.Sprintf("%s{}", typeName) + } + + st := p.structs[typeName] + parts := p.renderFields(st, n) + + if len(parts) == 0 { + return fmt.Sprintf("%s{}", typeName) + } + return fmt.Sprintf("%s{\n%s,\n}", typeName, strings.Join(parts, ",\n")) +} + +func (p *Parser) renderFields(st *ast.StructType, n *node) []string { + var parts []string + + // Sort field names for stable output + fieldNames := make([]string, 0, len(n.children)) + for fname := range n.children { + fieldNames = append(fieldNames, fname) + } + sort.Strings(fieldNames) + + for _, fname := range fieldNames { + child := n.children[fname] + nested := p.renderField(st, fname, child) + parts = append(parts, fmt.Sprintf("%s: %s", fname, nested)) + } + return parts +} + +func (p *Parser) renderField(st *ast.StructType, fname string, child *node) string { + p.config.Debugf("[render] field=%s, children=%d\n", fname, len(child.children)) + + fieldTypeName, isPtr := extractFieldType(st, fname) + + if child.typeExpr != nil && len(child.children) == 0 { + return literalForTypeExpr(child.typeExpr) + } + + if fieldTypeName == "" && len(child.children) > 0 { + fieldTypeName = p.findTypeFromChildren(child) + } + + if len(child.children) == 0 && fieldTypeName != "" { + return p.renderFieldWithDeprecations(fieldTypeName, isPtr) + } + + if len(child.children) > 0 { + return p.renderNestedField(fieldTypeName, child, isPtr) + } + + return literalForFieldType(fieldTypeName, isPtr) +} + +func (p *Parser) findTypeFromChildren(child *node) string { + keys := make([]string, 0, len(child.children)) + for k := range child.children { + keys = append(keys, k) + } + return findStructByFieldNames(p.structs, keys) +} + +func (p *Parser) renderFieldWithDeprecations(fieldTypeName string, isPtr bool) string { + if deps, ok := p.allDeprecations[fieldTypeName]; ok && len(deps) > 0 { + tmp := buildNodeFromFieldRefs(deps) + nested := p.renderLiteral(fieldTypeName, tmp) + if isPtr && !strings.HasPrefix(nested, "&") { + nested = "&" + nested + } + return nested + } + return literalForFieldType(fieldTypeName, isPtr) +} + +func (p *Parser) renderNestedField(fieldTypeName string, child *node, isPtr bool) string { + if fieldTypeName == "" { + return fmt.Sprintf("struct{ %s }{}", "") + } + nested := p.renderLiteral(fieldTypeName, child) + if isPtr && !strings.HasPrefix(nested, "&") { + nested = "ptr.To(" + nested + ")" + } + return nested +} + +func (p *Parser) flattenPoorlyNamedFieldsToGoPaths(typeName string, poorlyNamed map[string][]string, prefix string, paths *[]string) { + st, ok := p.structs[typeName] + if !ok { + return + } + + // If this type has deprecated (no prefix) fields, add them + if fields, hasDeprecated := poorlyNamed[typeName]; hasDeprecated { + for _, field := range fields { + if prefix == "" { + *paths = append(*paths, field) + } else { + *paths = append(*paths, prefix+"."+field) + } + } + } + + // Recurse into nested struct fields + for _, f := range st.Fields.List { + if len(f.Names) == 0 { + continue + } + fieldName := f.Names[0].Name + + // Skip Deprecated-prefixed fields + if strings.HasPrefix(fieldName, DeprecationPrefix) { + continue + } + + nestedTypeName := typeNameOf(f.Type) + if nestedTypeName == "" { + continue + } + + // Handle selector types (pkg.Type -> just Type) + if strings.Contains(nestedTypeName, ".") { + parts := strings.Split(nestedTypeName, ".") + nestedTypeName = parts[len(parts)-1] + } + + if _, exists := p.structs[nestedTypeName]; exists { + newPrefix := fieldName + if prefix != "" { + newPrefix = prefix + "." + fieldName + } + p.flattenPoorlyNamedFieldsToGoPaths(nestedTypeName, poorlyNamed, newPrefix, paths) + } + } +} diff --git a/gen/deprecations/templates/test.tpl b/gen/deprecations/templates/test.tpl new file mode 100644 index 000000000..ad357358c --- /dev/null +++ b/gen/deprecations/templates/test.tpl @@ -0,0 +1,51 @@ +// Copyright {{ year }} Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package {{ .Pkg }} + +{{ .TODOComment }} + +import ( + "testing" + + "github.com/stretchr/testify/require" + "sigs.k8s.io/controller-runtime/pkg/client" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/utils/ptr" + + "github.com/redpanda-data/redpanda-operator/pkg/deprecations" +) + +func TestDeprecatedFieldWarnings(t *testing.T) { + tests := []struct { + name string + obj client.Object + wantWarnings []string + }{ + {{- range .Objs }} + { + name: "{{ .Name }}", + obj: {{ .Literal }}, + wantWarnings: []string{ + {{- range .Warnings }} + "{{ . }}", + {{- end }} + }, + }, + {{- end }} + } + + for _, tc := range tests { + tc := tc + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + require.ElementsMatch(t, tc.wantWarnings, deprecations.FindDeprecatedFieldWarnings(tc.obj)) + }) + } +} \ No newline at end of file diff --git a/gen/internal/contextualize.go b/gen/internal/contextualize.go new file mode 100644 index 000000000..6e76dc95d --- /dev/null +++ b/gen/internal/contextualize.go @@ -0,0 +1,42 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package internal + +import ( + "errors" + "go/scanner" + "strconv" + "strings" +) + +func ContextualizeFormatErrors(data []byte, err error) string { + var serr scanner.ErrorList + if errors.As(err, &serr) { + errContext := []string{} + lines := strings.Split(string(data), "\n") + + for i, err := range serr { + line := err.Pos.Line + + lineContext := []string{"[ERROR " + strconv.Itoa(i+1) + "]:\n"} + if line-2 >= 0 { + lineContext = append(lineContext, lines[line-2]) + } + lineContext = append(lineContext, lines[line-1]) + if line < len(lines) { + lineContext = append(lineContext, lines[line]) + } + errContext = append(errContext, strings.Join(lineContext, "\n")) + } + return strings.Join(errContext, "\n\n") + } + + return "" +} diff --git a/gen/main.go b/gen/main.go index f7b171bb4..30e28f81f 100644 --- a/gen/main.go +++ b/gen/main.go @@ -12,6 +12,7 @@ package main import ( "github.com/spf13/cobra" + "github.com/redpanda-data/redpanda-operator/gen/deprecations" "github.com/redpanda-data/redpanda-operator/gen/partial" "github.com/redpanda-data/redpanda-operator/gen/pipeline" "github.com/redpanda-data/redpanda-operator/gen/schema" @@ -29,6 +30,7 @@ func main() { pipeline.Cmd(), schema.Cmd(), status.Cmd(), + deprecations.Cmd(), ) if err := root.Execute(); err != nil { diff --git a/gen/status/status.go b/gen/status/status.go index 4d468d7c3..3da823d35 100644 --- a/gen/status/status.go +++ b/gen/status/status.go @@ -13,13 +13,11 @@ import ( "bufio" "bytes" _ "embed" - "errors" "fmt" "go/ast" "go/format" "go/parser" "go/printer" - "go/scanner" "go/token" "io" "os" @@ -27,12 +25,13 @@ import ( "regexp" "slices" "sort" - "strconv" "strings" "text/template" "time" "k8s.io/apimachinery/pkg/util/yaml" + + "github.com/redpanda-data/redpanda-operator/gen/internal" ) var ( @@ -148,7 +147,7 @@ func renderAndWrite(directory, file string, info *templateInfo, renderFn func(in } formatted, err := format.Source(data) if err != nil { - if contextualErrors := contextualizeFormatErrors(data, err); contextualErrors != "" { + if contextualErrors := internal.ContextualizeFormatErrors(data, err); contextualErrors != "" { fmt.Println(contextualErrors) } return err @@ -300,7 +299,7 @@ func replaceTombstones(r io.Reader, structs map[string]*status) ([]byte, error) data, err := format.Source(fileContents.Bytes()) if err != nil { - if contextualErrors := contextualizeFormatErrors(fileContents.Bytes(), err); contextualErrors != "" { + if contextualErrors := internal.ContextualizeFormatErrors(fileContents.Bytes(), err); contextualErrors != "" { fmt.Println(contextualErrors) } } @@ -368,28 +367,3 @@ func tombstoneComments(node ast.Node, structs map[string]*status) bool { return false } - -func contextualizeFormatErrors(data []byte, err error) string { - var serr scanner.ErrorList - if errors.As(err, &serr) { - errContext := []string{} - lines := strings.Split(string(data), "\n") - - for i, err := range serr { - line := err.Pos.Line - - lineContext := []string{"[ERROR " + strconv.Itoa(i+1) + "]:\n"} - if line-2 >= 0 { - lineContext = append(lineContext, lines[line-2]) - } - lineContext = append(lineContext, lines[line-1]) - if line < len(lines) { - lineContext = append(lineContext, lines[line]) - } - errContext = append(errContext, strings.Join(lineContext, "\n")) - } - return strings.Join(errContext, "\n\n") - } - - return "" -} diff --git a/operator/api/redpanda/v1alpha2/console_types.go b/operator/api/redpanda/v1alpha2/console_types.go index 1bd431350..e4c3f3935 100644 --- a/operator/api/redpanda/v1alpha2/console_types.go +++ b/operator/api/redpanda/v1alpha2/console_types.go @@ -301,8 +301,8 @@ func ConvertConsoleSubchartToConsoleValues(src *RedpandaConsole) (*ConsoleValues // Extract out .Console and .Config. .Console will be migrated and then // merged into .Config as Config is meant to house V3 configurations. var v2Config map[string]any - if src.Console != nil && len(src.Console.Raw) > 0 { - if err := json.Unmarshal(src.Console.Raw, &v2Config); err != nil { + if src.DeprecatedConsole != nil && len(src.DeprecatedConsole.Raw) > 0 { + if err := json.Unmarshal(src.DeprecatedConsole.Raw, &v2Config); err != nil { return nil, errors.WithStack(err) } } diff --git a/operator/api/redpanda/v1alpha2/conversion.go b/operator/api/redpanda/v1alpha2/conversion.go index 7c3dcbb13..fa2921b89 100644 --- a/operator/api/redpanda/v1alpha2/conversion.go +++ b/operator/api/redpanda/v1alpha2/conversion.go @@ -89,7 +89,7 @@ func convertConsoleLicenseSecretRef(src *RedpandaConsole) (*corev1.SecretKeySele } // Short circuit if Enterprise isn't specified. - if src.Enterprise == nil || len(src.Enterprise.Raw) != 0 { + if src.DeprecatedEnterprise == nil || len(src.DeprecatedEnterprise.Raw) != 0 { return nil, nil } @@ -98,7 +98,7 @@ func convertConsoleLicenseSecretRef(src *RedpandaConsole) (*corev1.SecretKeySele LicenseSecret *corev1.SecretKeySelector } - enterprise, err := convertRuntimeRawExtension[ConsoleEnterprise](src.Enterprise) + enterprise, err := convertRuntimeRawExtension[ConsoleEnterprise](src.DeprecatedEnterprise) if err != nil { return nil, err } diff --git a/operator/api/redpanda/v1alpha2/redpanda_clusterspec_types.go b/operator/api/redpanda/v1alpha2/redpanda_clusterspec_types.go index 989d99133..52da46a5e 100644 --- a/operator/api/redpanda/v1alpha2/redpanda_clusterspec_types.go +++ b/operator/api/redpanda/v1alpha2/redpanda_clusterspec_types.go @@ -61,10 +61,6 @@ type RedpandaClusterSpec struct { // Specifies credentials for a private image repository. For details, see https://kubernetes.io/docs/tasks/configure-pod-container/pull-image-private-registry/. ImagePullSecrets []corev1.LocalObjectReference `json:"imagePullSecrets,omitempty"` - // Deprecated: Use `Enterprise` instead. - LicenseKey *string `json:"license_key,omitempty"` - // Deprecated: Use `EnterpriseLicenseSecretRef` instead. - LicenseSecretRef *LicenseSecretRef `json:"license_secret_ref,omitempty"` // Defines an Enterprise license. Enterprise *Enterprise `json:"enterprise,omitempty"` @@ -140,10 +136,18 @@ type RedpandaClusterSpec struct { // documentation](https://kubernetes.io/docs/concepts/scheduling-eviction/assign-pod-node/#affinity-and-anti-affinity). Affinity *corev1.Affinity `json:"affinity,omitempty"` - Tests *Enablable `json:"tests,omitempty"` + // Deprecated fields below + + // Deprecated: Use `enterprise.license` instead. + DeprecatedLicenseKey *string `json:"license_key,omitempty"` + // Deprecated: Use `enterprise.licenseSecretRef` instead. + DeprecatedLicenseSecretRef *LicenseSecretRef `json:"license_secret_ref,omitempty"` + // Deprecated + DeprecatedTests *DeprecatedEnablable `json:"tests,omitempty"` } -type Enablable struct { +// This structure is deprecated and should be removed +type DeprecatedEnablable struct { Enabled *bool `json:"enabled,omitempty"` } @@ -282,10 +286,6 @@ type RedpandaConsole struct { // Specifies whether a Deployment should be created for Redpanda Console. // +kubebuilder:pruning:PreserveUnknownFields Deployment *runtime.RawExtension `json:"deployment,omitempty"` - // Deprecated: Use `config` instead - // `console` is available in Console chart version earlier or equal to v0.7.31 - // +kubebuilder:pruning:PreserveUnknownFields - Console *runtime.RawExtension `json:"console,omitempty"` // Configures custom settings for Redpanda Console. // `config` is available in Console chart version after v0.7.31 semver // +kubebuilder:pruning:PreserveUnknownFields @@ -293,10 +293,6 @@ type RedpandaConsole struct { // Configures console's Deployment's update strategy. // +kubebuilder:pruning:PreserveUnknownFields Strategy *runtime.RawExtension `json:"strategy,omitempty"` - // Deprecated: Use `licenseSecretRef` instead. - // `enterprise` is available in Console chart version earlier or equal to v0.7.31 - // +kubebuilder:pruning:PreserveUnknownFields - Enterprise *runtime.RawExtension `json:"enterprise,omitempty"` // Defines a reference to Kubernetes Secret that points to a Redpanda Enterprise license. // Please consider use Enterprise in RedpandaClusterSpec type. // `licenseSecretRef` is available in Console chart version after v0.7.31 semver @@ -307,8 +303,19 @@ type RedpandaConsole struct { ReadinessProbe *ReadinessProbe `json:"readinessProbe,omitempty"` // Settings for console's Deployment's liveness probe. LivenessProbe *LivenessProbe `json:"livenessProbe,omitempty"` - // Controls the creation of helm tests for console. - Tests *Enablable `json:"tests,omitempty"` + + // Deprecated fields below + + // Deprecated: Use `config` instead + // `console` is available in Console chart version earlier or equal to v0.7.31 + // +kubebuilder:pruning:PreserveUnknownFields + DeprecatedConsole *runtime.RawExtension `json:"console,omitempty"` + // Deprecated: Use `licenseSecretRef` instead. + // `enterprise` is available in Console chart version earlier or equal to v0.7.31 + // +kubebuilder:pruning:PreserveUnknownFields + DeprecatedEnterprise *runtime.RawExtension `json:"enterprise,omitempty"` + // Deprecated + DeprecatedTests *DeprecatedEnablable `json:"tests,omitempty"` } // ConsoleCreateObj represents configuration options for creating Kubernetes objects such as ConfigMaps, Secrets, and Deployments. diff --git a/operator/api/redpanda/v1alpha2/redpanda_types.go b/operator/api/redpanda/v1alpha2/redpanda_types.go index f45cc1d3b..4fd41a919 100644 --- a/operator/api/redpanda/v1alpha2/redpanda_types.go +++ b/operator/api/redpanda/v1alpha2/redpanda_types.go @@ -79,20 +79,17 @@ type RedpandaSpec struct { ChartRef ChartRef `json:"chartRef,omitempty"` // Defines the Helm values to use to deploy the cluster. ClusterSpec *RedpandaClusterSpec `json:"clusterSpec,omitempty"` + + // Deprecated fields + // Deprecated and Removed in v2.2.3-24.2.X. Downgrade to v2.2.2-24.2.4 perform the migration - Migration *Migration `json:"migration,omitempty"` + DeprecatedMigration *DeprecatedMigration `json:"migration,omitempty"` } -// Migration can configure old Cluster and Console custom resource that will be disabled. -// With Migration the ChartRef and ClusterSpec still need to be correctly configured. -type Migration struct { - Enabled bool `json:"enabled"` - // ClusterRef by default will not be able to reach different namespaces, but it can be - // overwritten by adding ClusterRole and ClusterRoleBinding to operator ServiceAccount. +// This struct is set to be removed and not in use, so it is no longer documented +type DeprecatedMigration struct { + Enabled bool `json:"enabled"` ClusterRef vectorizedv1alpha1.NamespaceNameRef `json:"clusterRef"` - - // ConsoleRef by default will not be able to reach different namespaces, but it can be - // overwritten by adding ClusterRole and ClusterRoleBinding to operator ServiceAccount. ConsoleRef vectorizedv1alpha1.NamespaceNameRef `json:"consoleRef"` } @@ -119,63 +116,44 @@ type RedpandaStatus struct { // everything below here is deprecated and should be removed - // Specifies the last observed generation. - // deprecated + // Deprecated // +optional - ObservedGeneration int64 `json:"observedGeneration,omitempty"` - - // LastHandledReconcileAt holds the value of the most recent - // reconcile request value, so a change of the annotation value - // can be detected. - // deprecated + DeprecatedObservedGeneration int64 `json:"observedGeneration,omitempty"` + // Deprecated // +optional - LastHandledReconcileAt string `json:"lastHandledReconcileAt,omitempty"` - - // LastAppliedRevision is the revision of the last successfully applied source. - // deprecated + DeprecatedLastHandledReconcileAt string `json:"lastHandledReconcileAt,omitempty"` + // Deprecated // +optional - LastAppliedRevision string `json:"lastAppliedRevision,omitempty"` - - // LastAttemptedRevision is the revision of the last reconciliation attempt. - // deprecated + DeprecatedLastAppliedRevision string `json:"lastAppliedRevision,omitempty"` + // Deprecated // +optional - LastAttemptedRevision string `json:"lastAttemptedRevision,omitempty"` - - // deprecated + DeprecatedLastAttemptedRevision string `json:"lastAttemptedRevision,omitempty"` + // Deprecated // +optional - HelmRelease string `json:"helmRelease,omitempty"` - - // deprecated + DeprecatedHelmRelease string `json:"helmRelease,omitempty"` + // Deprecated // +optional - HelmReleaseReady *bool `json:"helmReleaseReady,omitempty"` - - // deprecated + DeprecatedHelmReleaseReady *bool `json:"helmReleaseReady,omitempty"` + // Deprecated // +optional - HelmRepository string `json:"helmRepository,omitempty"` - - // deprecated + DeprecatedHelmRepository string `json:"helmRepository,omitempty"` + // Deprecated // +optional - HelmRepositoryReady *bool `json:"helmRepositoryReady,omitempty"` - - // deprecated + DeprecatedHelmRepositoryReady *bool `json:"helmRepositoryReady,omitempty"` + // Deprecated // +optional - UpgradeFailures int64 `json:"upgradeFailures,omitempty"` - + DeprecatedUpgradeFailures int64 `json:"upgradeFailures,omitempty"` // Failures is the reconciliation failure count against the latest desired // state. It is reset after a successful reconciliation. // deprecated // +optional - Failures int64 `json:"failures,omitempty"` - - // deprecated + DeprecatedFailures int64 `json:"failures,omitempty"` + // Deprecated // +optional - InstallFailures int64 `json:"installFailures,omitempty"` - - // ManagedDecommissioningNode indicates that a node is currently being - // decommissioned from the cluster and provides its ordinal number. - // deprecated + DeprecatedInstallFailures int64 `json:"installFailures,omitempty"` + // Deprecated // +optional - ManagedDecommissioningNode *int32 `json:"decommissioningNode,omitempty"` + DeprecatedManagedDecommissioningNode *int32 `json:"decommissioningNode,omitempty"` } type RedpandaLicenseStatus struct { @@ -248,7 +226,7 @@ func (in *Redpanda) ValuesJSON() (*apiextensionsv1.JSON, error) { } func (in *Redpanda) GenerationObserved() bool { - return in.Generation != 0 && in.Generation == in.Status.ObservedGeneration + return in.Generation != 0 && in.Generation == in.Status.DeprecatedObservedGeneration } // GetConditions returns the status conditions of the object. diff --git a/operator/api/redpanda/v1alpha2/testdata/crd-docs.adoc b/operator/api/redpanda/v1alpha2/testdata/crd-docs.adoc index eff88d415..ff811654f 100644 --- a/operator/api/redpanda/v1alpha2/testdata/crd-docs.adoc +++ b/operator/api/redpanda/v1alpha2/testdata/crd-docs.adoc @@ -1067,12 +1067,12 @@ CredentialSecretRef can be used to set cloud_storage_secret_key from referenced |=== -[id="{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-enablable"] -==== Enablable - +[id="{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-deprecatedenablable"] +==== DeprecatedEnablable +This structure is deprecated and should be removed @@ -2478,8 +2478,6 @@ RedpandaClusterSpec defines the desired state of a Redpanda cluster. These setti | *`tolerations`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#toleration-v1-core[$$Toleration$$] array__ | Specifies tolerations to allow Pods to be scheduled onto nodes where they otherwise wouldn’t. + | | | *`image`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-redpandaimage[$$RedpandaImage$$]__ | Defines the container image settings to use for the Redpanda cluster. + | | | *`imagePullSecrets`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#localobjectreference-v1-core[$$LocalObjectReference$$] array__ | Specifies credentials for a private image repository. For details, see https://kubernetes.io/docs/tasks/configure-pod-container/pull-image-private-registry/. + | | -| *`license_key`* __string__ | Deprecated: Use `Enterprise` instead. + | | -| *`license_secret_ref`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-licensesecretref[$$LicenseSecretRef$$]__ | Deprecated: Use `EnterpriseLicenseSecretRef` instead. + | | | *`enterprise`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-enterprise[$$Enterprise$$]__ | Defines an Enterprise license. + | | | *`rackAwareness`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-rackawareness[$$RackAwareness$$]__ | Defines rack awareness settings. + | | | *`console`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-redpandaconsole[$$RedpandaConsole$$]__ | Defines Redpanda Console settings. + @@ -2507,7 +2505,9 @@ Setting `force` to `true` will result in a short period of downtime. + | | | *`affinity`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#affinity-v1-core[$$Affinity$$]__ | Affinity constraints for scheduling Pods, can override this for + StatefulSets and Jobs. For details, see the [Kubernetes + documentation](https://kubernetes.io/docs/concepts/scheduling-eviction/assign-pod-node/#affinity-and-anti-affinity). + | | -| *`tests`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-enablable[$$Enablable$$]__ | | | +| *`license_key`* __string__ | Deprecated: Use `enterprise.license` instead. + | | +| *`license_secret_ref`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-licensesecretref[$$LicenseSecretRef$$]__ | Deprecated: Use `enterprise.licenseSecretRef` instead. + | | +| *`tests`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-deprecatedenablable[$$DeprecatedEnablable$$]__ | Deprecated + | | |=== @@ -2602,20 +2602,20 @@ never used. Prefer ConfigMap (configmap). + | | | *`configMap`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-consolecreateobj[$$ConsoleCreateObj$$]__ | Specifies whether a ConfigMap should be created for Redpanda Console. + | | | *`secret`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#rawextension-runtime-pkg[$$RawExtension$$]__ | Specifies whether a Secret should be created for Redpanda Console. + | | | *`deployment`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#rawextension-runtime-pkg[$$RawExtension$$]__ | Specifies whether a Deployment should be created for Redpanda Console. + | | -| *`console`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#rawextension-runtime-pkg[$$RawExtension$$]__ | Deprecated: Use `config` instead + -`console` is available in Console chart version earlier or equal to v0.7.31 + | | | *`config`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#rawextension-runtime-pkg[$$RawExtension$$]__ | Configures custom settings for Redpanda Console. + `config` is available in Console chart version after v0.7.31 semver + | | | *`strategy`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#rawextension-runtime-pkg[$$RawExtension$$]__ | Configures console's Deployment's update strategy. + | | -| *`enterprise`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#rawextension-runtime-pkg[$$RawExtension$$]__ | Deprecated: Use `licenseSecretRef` instead. + -`enterprise` is available in Console chart version earlier or equal to v0.7.31 + | | | *`licenseSecretRef`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#secretkeyselector-v1-core[$$SecretKeySelector$$]__ | Defines a reference to Kubernetes Secret that points to a Redpanda Enterprise license. + Please consider use Enterprise in RedpandaClusterSpec type. + `licenseSecretRef` is available in Console chart version after v0.7.31 semver + | | | *`automountServiceAccountToken`* __boolean__ | Automount API credentials for the Service Account into the pod. + | | | *`readinessProbe`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-readinessprobe[$$ReadinessProbe$$]__ | Settings for console's Deployment's readiness probe. + | | | *`livenessProbe`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-livenessprobe[$$LivenessProbe$$]__ | Settings for console's Deployment's liveness probe. + | | -| *`tests`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-enablable[$$Enablable$$]__ | Controls the creation of helm tests for console. + | | +| *`console`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#rawextension-runtime-pkg[$$RawExtension$$]__ | Deprecated: Use `config` instead + +`console` is available in Console chart version earlier or equal to v0.7.31 + | | +| *`enterprise`* __link:https://kubernetes.io/docs/reference/generated/kubernetes-api/v1.28/#rawextension-runtime-pkg[$$RawExtension$$]__ | Deprecated: Use `licenseSecretRef` instead. + +`enterprise` is available in Console chart version earlier or equal to v0.7.31 + | | +| *`tests`* __xref:{anchor_prefix}-github-com-redpanda-data-redpanda-operator-operator-api-redpanda-v1alpha2-deprecatedenablable[$$DeprecatedEnablable$$]__ | Deprecated + | | |=== @@ -2801,28 +2801,20 @@ RedpandaStatus defines the observed state of Redpanda installed license in the Redpanda cluster. + | | | *`configVersion`* __string__ | ConfigVersion contains the configuration version written in + Redpanda used for restarting broker nodes as necessary. + | | -| *`observedGeneration`* __integer__ | Specifies the last observed generation. + -deprecated + | | -| *`lastHandledReconcileAt`* __string__ | LastHandledReconcileAt holds the value of the most recent + -reconcile request value, so a change of the annotation value + -can be detected. + -deprecated + | | -| *`lastAppliedRevision`* __string__ | LastAppliedRevision is the revision of the last successfully applied source. + -deprecated + | | -| *`lastAttemptedRevision`* __string__ | LastAttemptedRevision is the revision of the last reconciliation attempt. + -deprecated + | | -| *`helmRelease`* __string__ | deprecated + | | -| *`helmReleaseReady`* __boolean__ | deprecated + | | -| *`helmRepository`* __string__ | deprecated + | | -| *`helmRepositoryReady`* __boolean__ | deprecated + | | -| *`upgradeFailures`* __integer__ | deprecated + | | +| *`observedGeneration`* __integer__ | Deprecated + | | +| *`lastHandledReconcileAt`* __string__ | Deprecated + | | +| *`lastAppliedRevision`* __string__ | Deprecated + | | +| *`lastAttemptedRevision`* __string__ | Deprecated + | | +| *`helmRelease`* __string__ | Deprecated + | | +| *`helmReleaseReady`* __boolean__ | Deprecated + | | +| *`helmRepository`* __string__ | Deprecated + | | +| *`helmRepositoryReady`* __boolean__ | Deprecated + | | +| *`upgradeFailures`* __integer__ | Deprecated + | | | *`failures`* __integer__ | Failures is the reconciliation failure count against the latest desired + state. It is reset after a successful reconciliation. + deprecated + | | -| *`installFailures`* __integer__ | deprecated + | | -| *`decommissioningNode`* __integer__ | ManagedDecommissioningNode indicates that a node is currently being + -decommissioned from the cluster and provides its ordinal number. + -deprecated + | | +| *`installFailures`* __integer__ | Deprecated + | | +| *`decommissioningNode`* __integer__ | Deprecated + | | |=== diff --git a/operator/api/redpanda/v1alpha2/zz_generated.deepcopy.go b/operator/api/redpanda/v1alpha2/zz_generated.deepcopy.go index d8611077b..3e124a78e 100644 --- a/operator/api/redpanda/v1alpha2/zz_generated.deepcopy.go +++ b/operator/api/redpanda/v1alpha2/zz_generated.deepcopy.go @@ -1307,6 +1307,43 @@ func (in *DeploymentConfig) DeepCopy() *DeploymentConfig { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *DeprecatedEnablable) DeepCopyInto(out *DeprecatedEnablable) { + *out = *in + if in.Enabled != nil { + in, out := &in.Enabled, &out.Enabled + *out = new(bool) + **out = **in + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new DeprecatedEnablable. +func (in *DeprecatedEnablable) DeepCopy() *DeprecatedEnablable { + if in == nil { + return nil + } + out := new(DeprecatedEnablable) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *DeprecatedMigration) DeepCopyInto(out *DeprecatedMigration) { + *out = *in + out.ClusterRef = in.ClusterRef + out.ConsoleRef = in.ConsoleRef +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new DeprecatedMigration. +func (in *DeprecatedMigration) DeepCopy() *DeprecatedMigration { + if in == nil { + return nil + } + out := new(DeprecatedMigration) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *EmbeddedNodePoolSpec) DeepCopyInto(out *EmbeddedNodePoolSpec) { *out = *in @@ -1379,26 +1416,6 @@ func (in *EmbeddedNodePoolStatus) DeepCopy() *EmbeddedNodePoolStatus { return out } -// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. -func (in *Enablable) DeepCopyInto(out *Enablable) { - *out = *in - if in.Enabled != nil { - in, out := &in.Enabled, &out.Enabled - *out = new(bool) - **out = **in - } -} - -// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new Enablable. -func (in *Enablable) DeepCopy() *Enablable { - if in == nil { - return nil - } - out := new(Enablable) - in.DeepCopyInto(out) - return out -} - // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *Enterprise) DeepCopyInto(out *Enterprise) { *out = *in @@ -2393,23 +2410,6 @@ func (in *MetadataTemplate) DeepCopy() *MetadataTemplate { return out } -// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. -func (in *Migration) DeepCopyInto(out *Migration) { - *out = *in - out.ClusterRef = in.ClusterRef - out.ConsoleRef = in.ConsoleRef -} - -// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new Migration. -func (in *Migration) DeepCopy() *Migration { - if in == nil { - return nil - } - out := new(Migration) - in.DeepCopyInto(out) - return out -} - // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *Monitoring) DeepCopyInto(out *Monitoring) { *out = *in @@ -3270,16 +3270,6 @@ func (in *RedpandaClusterSpec) DeepCopyInto(out *RedpandaClusterSpec) { *out = make([]v1.LocalObjectReference, len(*in)) copy(*out, *in) } - if in.LicenseKey != nil { - in, out := &in.LicenseKey, &out.LicenseKey - *out = new(string) - **out = **in - } - if in.LicenseSecretRef != nil { - in, out := &in.LicenseSecretRef, &out.LicenseSecretRef - *out = new(LicenseSecretRef) - (*in).DeepCopyInto(*out) - } if in.Enterprise != nil { in, out := &in.Enterprise, &out.Enterprise *out = new(Enterprise) @@ -3395,9 +3385,19 @@ func (in *RedpandaClusterSpec) DeepCopyInto(out *RedpandaClusterSpec) { *out = new(v1.Affinity) (*in).DeepCopyInto(*out) } - if in.Tests != nil { - in, out := &in.Tests, &out.Tests - *out = new(Enablable) + if in.DeprecatedLicenseKey != nil { + in, out := &in.DeprecatedLicenseKey, &out.DeprecatedLicenseKey + *out = new(string) + **out = **in + } + if in.DeprecatedLicenseSecretRef != nil { + in, out := &in.DeprecatedLicenseSecretRef, &out.DeprecatedLicenseSecretRef + *out = new(LicenseSecretRef) + (*in).DeepCopyInto(*out) + } + if in.DeprecatedTests != nil { + in, out := &in.DeprecatedTests, &out.DeprecatedTests + *out = new(DeprecatedEnablable) (*in).DeepCopyInto(*out) } } @@ -3734,11 +3734,6 @@ func (in *RedpandaConsole) DeepCopyInto(out *RedpandaConsole) { *out = new(runtime.RawExtension) (*in).DeepCopyInto(*out) } - if in.Console != nil { - in, out := &in.Console, &out.Console - *out = new(runtime.RawExtension) - (*in).DeepCopyInto(*out) - } if in.Config != nil { in, out := &in.Config, &out.Config *out = new(runtime.RawExtension) @@ -3749,11 +3744,6 @@ func (in *RedpandaConsole) DeepCopyInto(out *RedpandaConsole) { *out = new(runtime.RawExtension) (*in).DeepCopyInto(*out) } - if in.Enterprise != nil { - in, out := &in.Enterprise, &out.Enterprise - *out = new(runtime.RawExtension) - (*in).DeepCopyInto(*out) - } if in.LicenseSecretRef != nil { in, out := &in.LicenseSecretRef, &out.LicenseSecretRef *out = new(v1.SecretKeySelector) @@ -3774,9 +3764,19 @@ func (in *RedpandaConsole) DeepCopyInto(out *RedpandaConsole) { *out = new(LivenessProbe) (*in).DeepCopyInto(*out) } - if in.Tests != nil { - in, out := &in.Tests, &out.Tests - *out = new(Enablable) + if in.DeprecatedConsole != nil { + in, out := &in.DeprecatedConsole, &out.DeprecatedConsole + *out = new(runtime.RawExtension) + (*in).DeepCopyInto(*out) + } + if in.DeprecatedEnterprise != nil { + in, out := &in.DeprecatedEnterprise, &out.DeprecatedEnterprise + *out = new(runtime.RawExtension) + (*in).DeepCopyInto(*out) + } + if in.DeprecatedTests != nil { + in, out := &in.DeprecatedTests, &out.DeprecatedTests + *out = new(DeprecatedEnablable) (*in).DeepCopyInto(*out) } } @@ -4005,9 +4005,9 @@ func (in *RedpandaSpec) DeepCopyInto(out *RedpandaSpec) { *out = new(RedpandaClusterSpec) (*in).DeepCopyInto(*out) } - if in.Migration != nil { - in, out := &in.Migration, &out.Migration - *out = new(Migration) + if in.DeprecatedMigration != nil { + in, out := &in.DeprecatedMigration, &out.DeprecatedMigration + *out = new(DeprecatedMigration) **out = **in } } @@ -4042,18 +4042,18 @@ func (in *RedpandaStatus) DeepCopyInto(out *RedpandaStatus) { *out = make([]EmbeddedNodePoolStatus, len(*in)) copy(*out, *in) } - if in.HelmReleaseReady != nil { - in, out := &in.HelmReleaseReady, &out.HelmReleaseReady + if in.DeprecatedHelmReleaseReady != nil { + in, out := &in.DeprecatedHelmReleaseReady, &out.DeprecatedHelmReleaseReady *out = new(bool) **out = **in } - if in.HelmRepositoryReady != nil { - in, out := &in.HelmRepositoryReady, &out.HelmRepositoryReady + if in.DeprecatedHelmRepositoryReady != nil { + in, out := &in.DeprecatedHelmRepositoryReady, &out.DeprecatedHelmRepositoryReady *out = new(bool) **out = **in } - if in.ManagedDecommissioningNode != nil { - in, out := &in.ManagedDecommissioningNode, &out.ManagedDecommissioningNode + if in.DeprecatedManagedDecommissioningNode != nil { + in, out := &in.DeprecatedManagedDecommissioningNode, &out.DeprecatedManagedDecommissioningNode *out = new(int32) **out = **in } diff --git a/operator/api/redpanda/v1alpha2/zz_generated.deprecations_test.go b/operator/api/redpanda/v1alpha2/zz_generated.deprecations_test.go new file mode 100644 index 000000000..256c979b9 --- /dev/null +++ b/operator/api/redpanda/v1alpha2/zz_generated.deprecations_test.go @@ -0,0 +1,626 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package v1alpha2 + +// TODO: The following fields are documented as deprecated in the API comments +// but their Go field names do not use the `Deprecated` prefix. Consider +// renaming them (or adding wrapper fields with the Deprecated prefix) so the +// reflective deprecation detector can find them uniformly. Examples found in +// this package: +// +// - Redpanda: +// - ClusterSpec.Console +// - ClusterSpec.Connectors +// - ClusterSpec.Connectors.Test.Enabled +// - ClusterSpec.Logging.UsageStats.Organization +// - ClusterSpec.Storage.Tiered.Config.CloudStorageReconciliationIntervalMs +// - ClusterSpec.PostInstallJob.SecurityContext +// - ClusterSpec.PostUpgradeJob.SecurityContext +// - ClusterSpec.Statefulset.SideCars.ConfigWatcher.ExtraVolumeMounts +// - ClusterSpec.Statefulset.SideCars.ConfigWatcher.Resources +// - ClusterSpec.Statefulset.SideCars.ConfigWatcher.SecurityContext +// - ClusterSpec.Statefulset.SideCars.RpkStatus.Resources +// - ClusterSpec.Statefulset.SideCars.RpkStatus.SecurityContext +// - ClusterSpec.Statefulset.SideCars.Controllers.Resources +// - ClusterSpec.Statefulset.SideCars.Controllers.SecurityContext +// - ClusterSpec.Listeners.HTTP.KafkaEndpoint +// - ClusterSpec.Listeners.RPC.TLS.SecretRef +// - ClusterSpec.Listeners.SchemaRegistry.KafkaEndpoint +// - Topic: +// - KafkaAPISpec + +import ( + "testing" + + "github.com/stretchr/testify/require" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/utils/ptr" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/redpanda-data/redpanda-operator/pkg/deprecations" +) + +func TestDeprecatedFieldWarnings(t *testing.T) { + tests := []struct { + name string + obj client.Object + wantWarnings []string + }{ + { + name: "Console", + obj: &Console{ + Spec: ConsoleSpec{ + ClusterSource: ptr.To(ClusterSource{ + StaticConfiguration: ptr.To(StaticConfigurationSource{ + Admin: ptr.To(AdminAPISpec{ + SASL: ptr.To(AdminSASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + Kafka: ptr.To(KafkaAPISpec{ + SASL: ptr.To(KafkaSASL{ + AWSMskIam: ptr.To(KafkaSASLAWSMskIam{ + DeprecatedSecretKey: ptr.To(SecretKeyRef{}), + DeprecatedSessionToken: ptr.To(SecretKeyRef{}), + }), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + GSSAPIConfig: ptr.To(KafkaSASLGSSAPI{ + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + OAUth: ptr.To(KafkaSASLOAuthBearer{ + DeprecatedToken: ptr.To(SecretKeyRef{}), + }), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + SchemaRegistry: ptr.To(SchemaRegistrySpec{ + SASL: ptr.To(SchemaRegistrySASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + }), + }), + }, + }, + wantWarnings: []string{ + "field 'spec.cluster.staticConfiguration.kafka.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.oauth.tokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.gssapi.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.secretKeySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.sessionTokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.token' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.token' is deprecated and set", + }, + }, + { + name: "Redpanda", + obj: &Redpanda{ + Spec: RedpandaSpec{ + ClusterSpec: ptr.To(RedpandaClusterSpec{ + Console: ptr.To(RedpandaConsole{ + DeprecatedConfigMap: ptr.To(ConsoleCreateObj{}), + DeprecatedConsole: ptr.To(runtime.RawExtension{}), + DeprecatedEnterprise: ptr.To(runtime.RawExtension{}), + DeprecatedTests: ptr.To(DeprecatedEnablable{}), + }), + DeprecatedFullNameOverride: "deprecated", + DeprecatedLicenseKey: ptr.To("deprecated"), + DeprecatedLicenseSecretRef: ptr.To(LicenseSecretRef{}), + DeprecatedTests: ptr.To(DeprecatedEnablable{}), + }), + DeprecatedMigration: ptr.To(DeprecatedMigration{}), + }, + }, + wantWarnings: []string{ + "field 'spec.clusterSpec.fullNameOverride' is deprecated and set", + "field 'spec.clusterSpec.console.configmap' is deprecated and set", + "field 'spec.clusterSpec.console.console' is deprecated and set", + "field 'spec.clusterSpec.console.enterprise' is deprecated and set", + "field 'spec.clusterSpec.console.tests' is deprecated and set", + "field 'spec.clusterSpec.license_key' is deprecated and set", + "field 'spec.clusterSpec.license_secret_ref' is deprecated and set", + "field 'spec.clusterSpec.tests' is deprecated and set", + "field 'spec.migration' is deprecated and set", + }, + }, + { + name: "RedpandaRole", + obj: &RedpandaRole{ + Spec: RoleSpec{ + ClusterSource: ptr.To(ClusterSource{ + StaticConfiguration: ptr.To(StaticConfigurationSource{ + Admin: ptr.To(AdminAPISpec{ + SASL: ptr.To(AdminSASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + Kafka: ptr.To(KafkaAPISpec{ + SASL: ptr.To(KafkaSASL{ + AWSMskIam: ptr.To(KafkaSASLAWSMskIam{ + DeprecatedSecretKey: ptr.To(SecretKeyRef{}), + DeprecatedSessionToken: ptr.To(SecretKeyRef{}), + }), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + GSSAPIConfig: ptr.To(KafkaSASLGSSAPI{ + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + OAUth: ptr.To(KafkaSASLOAuthBearer{ + DeprecatedToken: ptr.To(SecretKeyRef{}), + }), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + SchemaRegistry: ptr.To(SchemaRegistrySpec{ + SASL: ptr.To(SchemaRegistrySASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + }), + }), + }, + }, + wantWarnings: []string{ + "field 'spec.cluster.staticConfiguration.kafka.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.oauth.tokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.gssapi.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.secretKeySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.sessionTokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.token' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.token' is deprecated and set", + }, + }, + { + name: "Schema", + obj: &Schema{ + Spec: SchemaSpec{ + ClusterSource: ptr.To(ClusterSource{ + StaticConfiguration: ptr.To(StaticConfigurationSource{ + Admin: ptr.To(AdminAPISpec{ + SASL: ptr.To(AdminSASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + Kafka: ptr.To(KafkaAPISpec{ + SASL: ptr.To(KafkaSASL{ + AWSMskIam: ptr.To(KafkaSASLAWSMskIam{ + DeprecatedSecretKey: ptr.To(SecretKeyRef{}), + DeprecatedSessionToken: ptr.To(SecretKeyRef{}), + }), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + GSSAPIConfig: ptr.To(KafkaSASLGSSAPI{ + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + OAUth: ptr.To(KafkaSASLOAuthBearer{ + DeprecatedToken: ptr.To(SecretKeyRef{}), + }), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + SchemaRegistry: ptr.To(SchemaRegistrySpec{ + SASL: ptr.To(SchemaRegistrySASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + }), + }), + }, + }, + wantWarnings: []string{ + "field 'spec.cluster.staticConfiguration.kafka.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.oauth.tokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.gssapi.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.secretKeySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.sessionTokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.token' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.token' is deprecated and set", + }, + }, + { + name: "ShadowLink", + obj: &ShadowLink{ + Spec: ShadowLinkSpec{ + ShadowCluster: ptr.To(ClusterSource{ + StaticConfiguration: ptr.To(StaticConfigurationSource{ + Admin: ptr.To(AdminAPISpec{ + SASL: ptr.To(AdminSASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + Kafka: ptr.To(KafkaAPISpec{ + SASL: ptr.To(KafkaSASL{ + AWSMskIam: ptr.To(KafkaSASLAWSMskIam{ + DeprecatedSecretKey: ptr.To(SecretKeyRef{}), + DeprecatedSessionToken: ptr.To(SecretKeyRef{}), + }), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + GSSAPIConfig: ptr.To(KafkaSASLGSSAPI{ + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + OAUth: ptr.To(KafkaSASLOAuthBearer{ + DeprecatedToken: ptr.To(SecretKeyRef{}), + }), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + SchemaRegistry: ptr.To(SchemaRegistrySpec{ + SASL: ptr.To(SchemaRegistrySASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + }), + }), + SourceCluster: ptr.To(ClusterSource{ + StaticConfiguration: ptr.To(StaticConfigurationSource{ + Admin: ptr.To(AdminAPISpec{ + SASL: ptr.To(AdminSASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + Kafka: ptr.To(KafkaAPISpec{ + SASL: ptr.To(KafkaSASL{ + AWSMskIam: ptr.To(KafkaSASLAWSMskIam{ + DeprecatedSecretKey: ptr.To(SecretKeyRef{}), + DeprecatedSessionToken: ptr.To(SecretKeyRef{}), + }), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + GSSAPIConfig: ptr.To(KafkaSASLGSSAPI{ + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + OAUth: ptr.To(KafkaSASLOAuthBearer{ + DeprecatedToken: ptr.To(SecretKeyRef{}), + }), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + SchemaRegistry: ptr.To(SchemaRegistrySpec{ + SASL: ptr.To(SchemaRegistrySASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + }), + }), + }, + }, + wantWarnings: []string{ + "field 'spec.shadowCluster.staticConfiguration.kafka.tls.caCertSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.kafka.tls.certSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.kafka.tls.keySecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.kafka.sasl.oauth.tokenSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.kafka.sasl.gssapi.passwordSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.kafka.sasl.awsMskIam.secretKeySecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.kafka.sasl.awsMskIam.sessionTokenSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.kafka.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.admin.tls.caCertSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.admin.tls.certSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.admin.tls.keySecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.admin.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.admin.sasl.token' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.schemaRegistry.tls.caCertSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.schemaRegistry.tls.certSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.schemaRegistry.tls.keySecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.schemaRegistry.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.shadowCluster.staticConfiguration.schemaRegistry.sasl.token' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.kafka.tls.caCertSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.kafka.tls.certSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.kafka.tls.keySecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.kafka.sasl.oauth.tokenSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.kafka.sasl.gssapi.passwordSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.kafka.sasl.awsMskIam.secretKeySecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.kafka.sasl.awsMskIam.sessionTokenSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.kafka.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.admin.tls.caCertSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.admin.tls.certSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.admin.tls.keySecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.admin.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.admin.sasl.token' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.schemaRegistry.tls.caCertSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.schemaRegistry.tls.certSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.schemaRegistry.tls.keySecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.schemaRegistry.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.sourceCluster.staticConfiguration.schemaRegistry.sasl.token' is deprecated and set", + }, + }, + { + name: "Topic", + obj: &Topic{ + Spec: TopicSpec{ + ClusterSource: ptr.To(ClusterSource{ + StaticConfiguration: ptr.To(StaticConfigurationSource{ + Admin: ptr.To(AdminAPISpec{ + SASL: ptr.To(AdminSASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + Kafka: ptr.To(KafkaAPISpec{ + SASL: ptr.To(KafkaSASL{ + AWSMskIam: ptr.To(KafkaSASLAWSMskIam{ + DeprecatedSecretKey: ptr.To(SecretKeyRef{}), + DeprecatedSessionToken: ptr.To(SecretKeyRef{}), + }), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + GSSAPIConfig: ptr.To(KafkaSASLGSSAPI{ + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + OAUth: ptr.To(KafkaSASLOAuthBearer{ + DeprecatedToken: ptr.To(SecretKeyRef{}), + }), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + SchemaRegistry: ptr.To(SchemaRegistrySpec{ + SASL: ptr.To(SchemaRegistrySASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + }), + }), + KafkaAPISpec: ptr.To(KafkaAPISpec{ + SASL: ptr.To(KafkaSASL{ + AWSMskIam: ptr.To(KafkaSASLAWSMskIam{ + DeprecatedSecretKey: ptr.To(SecretKeyRef{}), + DeprecatedSessionToken: ptr.To(SecretKeyRef{}), + }), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + GSSAPIConfig: ptr.To(KafkaSASLGSSAPI{ + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + OAUth: ptr.To(KafkaSASLOAuthBearer{ + DeprecatedToken: ptr.To(SecretKeyRef{}), + }), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + }, + }, + wantWarnings: []string{ + "field 'spec.cluster.staticConfiguration.kafka.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.oauth.tokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.gssapi.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.secretKeySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.sessionTokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.token' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.token' is deprecated and set", + "field 'spec.kafkaApiSpec.tls.caCertSecretRef' is deprecated and set", + "field 'spec.kafkaApiSpec.tls.certSecretRef' is deprecated and set", + "field 'spec.kafkaApiSpec.tls.keySecretRef' is deprecated and set", + "field 'spec.kafkaApiSpec.sasl.oauth.tokenSecretRef' is deprecated and set", + "field 'spec.kafkaApiSpec.sasl.gssapi.passwordSecretRef' is deprecated and set", + "field 'spec.kafkaApiSpec.sasl.awsMskIam.secretKeySecretRef' is deprecated and set", + "field 'spec.kafkaApiSpec.sasl.awsMskIam.sessionTokenSecretRef' is deprecated and set", + "field 'spec.kafkaApiSpec.sasl.passwordSecretRef' is deprecated and set", + }, + }, + { + name: "User", + obj: &User{ + Spec: UserSpec{ + ClusterSource: ptr.To(ClusterSource{ + StaticConfiguration: ptr.To(StaticConfigurationSource{ + Admin: ptr.To(AdminAPISpec{ + SASL: ptr.To(AdminSASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + Kafka: ptr.To(KafkaAPISpec{ + SASL: ptr.To(KafkaSASL{ + AWSMskIam: ptr.To(KafkaSASLAWSMskIam{ + DeprecatedSecretKey: ptr.To(SecretKeyRef{}), + DeprecatedSessionToken: ptr.To(SecretKeyRef{}), + }), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + GSSAPIConfig: ptr.To(KafkaSASLGSSAPI{ + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + OAUth: ptr.To(KafkaSASLOAuthBearer{ + DeprecatedToken: ptr.To(SecretKeyRef{}), + }), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + SchemaRegistry: ptr.To(SchemaRegistrySpec{ + SASL: ptr.To(SchemaRegistrySASL{ + DeprecatedAuthToken: ptr.To(SecretKeyRef{}), + DeprecatedPassword: ptr.To(SecretKeyRef{}), + }), + TLS: ptr.To(CommonTLS{ + DeprecatedCaCert: ptr.To(SecretKeyRef{}), + DeprecatedCert: ptr.To(SecretKeyRef{}), + DeprecatedKey: ptr.To(SecretKeyRef{}), + }), + }), + }), + }), + }, + }, + wantWarnings: []string{ + "field 'spec.cluster.staticConfiguration.kafka.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.oauth.tokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.gssapi.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.secretKeySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.awsMskIam.sessionTokenSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.kafka.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.admin.sasl.token' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.caCertSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.certSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.tls.keySecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.passwordSecretRef' is deprecated and set", + "field 'spec.cluster.staticConfiguration.schemaRegistry.sasl.token' is deprecated and set", + }, + }, + } + + for _, tc := range tests { + tc := tc + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + require.ElementsMatch(t, tc.wantWarnings, deprecations.FindDeprecatedFieldWarnings(tc.obj)) + }) + } +} diff --git a/operator/chart/chart_test.go b/operator/chart/chart_test.go index 55083356f..29a37504c 100644 --- a/operator/chart/chart_test.go +++ b/operator/chart/chart_test.go @@ -199,7 +199,7 @@ func TestIntegrationChart(t *testing.T) { } // We define "not reconciled" as not having an ObservedGeneration set for at least 5 seconds. - return time.Since(rp.CreationTimestamp.Time) >= 5*time.Second && rp.Generation != 0 && rp.Status.ObservedGeneration == 0, nil + return time.Since(rp.CreationTimestamp.Time) >= 5*time.Second && rp.Generation != 0 && rp.Status.DeprecatedObservedGeneration == 0, nil }, testRP("rp-2", "rp-2"), testRP("rp-3", "rp-3"))) diff --git a/operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml b/operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml index 4543e3b79..370014736 100644 --- a/operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml +++ b/operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml @@ -1776,7 +1776,7 @@ spec: type: object x-kubernetes-preserve-unknown-fields: true tests: - description: Controls the creation of helm tests for console. + description: Deprecated properties: enabled: type: boolean @@ -1929,10 +1929,10 @@ spec: x-kubernetes-map-type: atomic type: array license_key: - description: 'Deprecated: Use `Enterprise` instead.' + description: 'Deprecated: Use `enterprise.license` instead.' type: string license_secret_ref: - description: 'Deprecated: Use `EnterpriseLicenseSecretRef` instead.' + description: 'Deprecated: Use `enterprise.licenseSecretRef` instead.' properties: secret_key: description: Specifies the key that is contains the Enterprise @@ -20617,6 +20617,7 @@ spec: type: object type: object tests: + description: Deprecated properties: enabled: type: boolean @@ -20832,8 +20833,8 @@ spec: properties: clusterRef: description: |- - ClusterRef by default will not be able to reach different namespaces, but it can be - overwritten by adding ClusterRole and ClusterRoleBinding to operator ServiceAccount. + NamespaceNameRef contains namespace and name to inspect or modify the referred object + REF https://pkg.go.dev/k8s.io/api/core/v1#ObjectReference properties: name: description: |- @@ -20851,8 +20852,8 @@ spec: type: object consoleRef: description: |- - ConsoleRef by default will not be able to reach different namespaces, but it can be - overwritten by adding ClusterRole and ClusterRoleBinding to operator ServiceAccount. + NamespaceNameRef contains namespace and name to inspect or modify the referred object + REF https://pkg.go.dev/k8s.io/api/core/v1#ObjectReference properties: name: description: |- @@ -20979,10 +20980,7 @@ spec: Redpanda used for restarting broker nodes as necessary. type: string decommissioningNode: - description: |- - ManagedDecommissioningNode indicates that a node is currently being - decommissioned from the cluster and provides its ordinal number. - deprecated + description: Deprecated format: int32 type: integer failures: @@ -20993,37 +20991,29 @@ spec: format: int64 type: integer helmRelease: - description: deprecated + description: Deprecated type: string helmReleaseReady: - description: deprecated + description: Deprecated type: boolean helmRepository: - description: deprecated + description: Deprecated type: string helmRepositoryReady: - description: deprecated + description: Deprecated type: boolean installFailures: - description: deprecated + description: Deprecated format: int64 type: integer lastAppliedRevision: - description: |- - LastAppliedRevision is the revision of the last successfully applied source. - deprecated + description: Deprecated type: string lastAttemptedRevision: - description: |- - LastAttemptedRevision is the revision of the last reconciliation attempt. - deprecated + description: Deprecated type: string lastHandledReconcileAt: - description: |- - LastHandledReconcileAt holds the value of the most recent - reconcile request value, so a change of the annotation value - can be detected. - deprecated + description: Deprecated type: string license: description: |- @@ -21109,13 +21099,11 @@ spec: type: object type: array observedGeneration: - description: |- - Specifies the last observed generation. - deprecated + description: Deprecated format: int64 type: integer upgradeFailures: - description: deprecated + description: Deprecated format: int64 type: integer type: object @@ -22887,7 +22875,7 @@ spec: type: object x-kubernetes-preserve-unknown-fields: true tests: - description: Controls the creation of helm tests for console. + description: Deprecated properties: enabled: type: boolean @@ -23040,10 +23028,10 @@ spec: x-kubernetes-map-type: atomic type: array license_key: - description: 'Deprecated: Use `Enterprise` instead.' + description: 'Deprecated: Use `enterprise.license` instead.' type: string license_secret_ref: - description: 'Deprecated: Use `EnterpriseLicenseSecretRef` instead.' + description: 'Deprecated: Use `enterprise.licenseSecretRef` instead.' properties: secret_key: description: Specifies the key that is contains the Enterprise @@ -41728,6 +41716,7 @@ spec: type: object type: object tests: + description: Deprecated properties: enabled: type: boolean @@ -41943,8 +41932,8 @@ spec: properties: clusterRef: description: |- - ClusterRef by default will not be able to reach different namespaces, but it can be - overwritten by adding ClusterRole and ClusterRoleBinding to operator ServiceAccount. + NamespaceNameRef contains namespace and name to inspect or modify the referred object + REF https://pkg.go.dev/k8s.io/api/core/v1#ObjectReference properties: name: description: |- @@ -41962,8 +41951,8 @@ spec: type: object consoleRef: description: |- - ConsoleRef by default will not be able to reach different namespaces, but it can be - overwritten by adding ClusterRole and ClusterRoleBinding to operator ServiceAccount. + NamespaceNameRef contains namespace and name to inspect or modify the referred object + REF https://pkg.go.dev/k8s.io/api/core/v1#ObjectReference properties: name: description: |- @@ -42090,10 +42079,7 @@ spec: Redpanda used for restarting broker nodes as necessary. type: string decommissioningNode: - description: |- - ManagedDecommissioningNode indicates that a node is currently being - decommissioned from the cluster and provides its ordinal number. - deprecated + description: Deprecated format: int32 type: integer failures: @@ -42104,37 +42090,29 @@ spec: format: int64 type: integer helmRelease: - description: deprecated + description: Deprecated type: string helmReleaseReady: - description: deprecated + description: Deprecated type: boolean helmRepository: - description: deprecated + description: Deprecated type: string helmRepositoryReady: - description: deprecated + description: Deprecated type: boolean installFailures: - description: deprecated + description: Deprecated format: int64 type: integer lastAppliedRevision: - description: |- - LastAppliedRevision is the revision of the last successfully applied source. - deprecated + description: Deprecated type: string lastAttemptedRevision: - description: |- - LastAttemptedRevision is the revision of the last reconciliation attempt. - deprecated + description: Deprecated type: string lastHandledReconcileAt: - description: |- - LastHandledReconcileAt holds the value of the most recent - reconcile request value, so a change of the annotation value - can be detected. - deprecated + description: Deprecated type: string license: description: |- @@ -42220,13 +42198,11 @@ spec: type: object type: array observedGeneration: - description: |- - Specifies the last observed generation. - deprecated + description: Deprecated format: int64 type: integer upgradeFailures: - description: deprecated + description: Deprecated format: int64 type: integer type: object diff --git a/pkg/deprecations/warnings.go b/pkg/deprecations/warnings.go new file mode 100644 index 000000000..f0f77f02b --- /dev/null +++ b/pkg/deprecations/warnings.go @@ -0,0 +1,188 @@ +// Copyright 2025 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package deprecations + +import ( + "fmt" + "reflect" + "slices" + "strings" + + "sigs.k8s.io/controller-runtime/pkg/client" +) + +const DeprecatedPrefix = "Deprecated" + +// FindDeprecatedFieldWarnings inspects an arbitrary `client.Object` and returns a +// deprecation warning messages for any deeply nested struct fields that have a field +// prefixed with "Deprecated" and whose value is not the zero value. The name shown in the +// warning is taken from the field's full json path from the root of the CRD. +func FindDeprecatedFieldWarnings(obj client.Object) []string { + v := reflect.ValueOf(obj) + if v.Kind() == reflect.Pointer { + if v.IsNil() { + // nothing to do + return nil + } + v = v.Elem() + } + + if v.Kind() != reflect.Struct { + // if we don't have a struct, then it can't have fields + return nil + } + + // Only inspect the Spec field and its children + spec, ok := v.Type().FieldByName("Spec") + if !ok { + // we only warn on user-supplied input, which comes from + // Spec + return nil + } + + deprecations := deprecatedFields(v.FieldByName("Spec"), spec.Type, "spec", make(map[uintptr]struct{})) + + slices.Sort(deprecations) + + return deprecations +} + +func deprecatedFields(value reflect.Value, reflectType reflect.Type, path string, visited map[uintptr]struct{}) []string { + out := make([]string, 0) + + for value.Kind() == reflect.Pointer || value.Kind() == reflect.Interface { + if value.IsNil() { + return out + } + // protect against cycles + if value.Kind() == reflect.Pointer { + ptr := value.Pointer() + if _, ok := visited[ptr]; ok { + return out + } + visited[ptr] = struct{}{} + } + + // use concrete types + value = value.Elem() + reflectType = value.Type() + } + + if value.Kind() == reflect.Struct { + for i := 0; i < value.NumField(); i++ { + structField := reflectType.Field(i) + + // Skip unexported fields. + if structField.PkgPath != "" { + continue + } + + fieldValue := value.Field(i) + + nextPath := getNextPath(structField, path) + if warning := checkFieldUsage(structField, fieldValue, path); warning != "" { + out = append(out, warning) + } + + switch fieldValue.Kind() { + case reflect.Struct: + out = append(out, deprecatedFields(fieldValue, fieldValue.Type(), nextPath, visited)...) + case reflect.Pointer, reflect.Interface: + if fieldValue.IsNil() { + // the pointer is nil, so we don't need to traverse since that's the 0 value + } else { + out = append(out, deprecatedFields(fieldValue, fieldValue.Type(), nextPath, visited)...) + } + case reflect.Slice, reflect.Array: + elemKind := fieldValue.Type().Elem().Kind() + if elemKind == reflect.Struct || elemKind == reflect.Pointer || elemKind == reflect.Interface { + for j := 0; j < fieldValue.Len(); j++ { + value := fieldValue.Index(j) + out = append(out, deprecatedFields(value, value.Type(), nextPath, visited)...) + } + } + case reflect.Map: + elemKind := fieldValue.Type().Elem().Kind() + if elemKind == reflect.Struct || elemKind == reflect.Pointer || elemKind == reflect.Interface { + for _, key := range fieldValue.MapKeys() { + value := fieldValue.MapIndex(key) + out = append(out, deprecatedFields(value, value.Type(), nextPath, visited)...) + } + } + } + } + } + + return out +} + +func checkFieldUsage(field reflect.StructField, value reflect.Value, path string) string { + // If this field's name starts with Deprecated and it's not a + // zero value, emit a warning using the full json path. + if strings.HasPrefix(field.Name, DeprecatedPrefix) { + // Only check values we can interface with. + if value.IsValid() && value.CanInterface() { + isZero := value.IsZero() + if !isZero { + fullPath := getJSONName(field) + if path != "" { + fullPath = path + "." + fullPath + } + return fmt.Sprintf("field '%s' is deprecated and set", fullPath) + } + } + } + + return "" +} + +func getJSONName(field reflect.StructField) string { + fieldTag := field.Tag.Get("json") + if fieldTag == "" { + return field.Name + } + parts := strings.Split(fieldTag, ",") + if parts[0] == "-" || parts[0] == "" { + return field.Name + } + return parts[0] +} + +func getNextPath(field reflect.StructField, path string) string { + tag := field.Tag.Get("json") + inline := strings.Contains(tag, ",inline") + + nextPath := func(name string) string { + if inline { + return path + } + + if path == "" { + return name + } + return path + "." + name + } + + if tag == "" { + return nextPath(field.Name) + } + + parts := strings.Split(tag, ",") + tag = parts[0] + + switch tag { + case "-": + return nextPath(field.Name) + case "": + return nextPath(tag) + default: + return nextPath(tag) + } +} diff --git a/taskfiles/k8s.yml b/taskfiles/k8s.yml index 294f52c84..0ce243677 100644 --- a/taskfiles/k8s.yml +++ b/taskfiles/k8s.yml @@ -24,6 +24,7 @@ tasks: output:rbac:artifacts:config=config/rbac/bases/operator - task: generate:statuses + - task: generate:deprecations - task: generate:crd-docs # For each controller (or really itemizable RBAC source), generate an @@ -138,6 +139,14 @@ tasks: --output-path ./api/redpanda/v1alpha2/testdata/crd-docs.adoc \ --templates-dir=./crd-docs-templates/ + generate:deprecations: + label: "generate:deprecations" + desc: "Generate the deprecation tests for our deprecation warnings." + cmds: + - gen deprecations --directory ./operator/api/redpanda/v1alpha2 + deps: + - task: :build:gen + generate:statuses: run: when_changed internal: true