From 6678387e4fd0a638e812988c7a5e46d1a655c1a8 Mon Sep 17 00:00:00 2001 From: Chris Seto Date: Mon, 11 Nov 2024 15:17:55 -0500 Subject: [PATCH] v2: correct and test controller RBAC Prior to this commit the declared permissions for the RedpandaReconciler had become out of date. This went unnoticed due to tests utilizing admin permissions or the inflated permissions required for executing `rpk debug bundle`. This commit corrects the permission declaration of the RedpandaReconciler, updates its tests to use the ClusterRole and Role generated by controller-gen, and adds a test to statically assert the correctness of the permissions. --- operator/config/rbac/bases/operator/role.yaml | 66 ++-- .../config/rbac/v2-manager-role/role.yaml | 103 +++--- .../managed_decommission_controller.go | 2 +- .../redpanda/redpanda_controller.go | 18 +- .../redpanda/redpanda_controller_test.go | 222 +++++++++++-- .../redpanda_decommission_controller.go | 5 +- .../redpanda/redpanda_node_pvc_controller.go | 5 +- .../internal/controller/redpanda/role.yaml | 304 ++++++++++++++++++ .../controller/redpanda/schema_controller.go | 6 - .../controller/redpanda/topic_controller.go | 6 - .../controller/redpanda/user_controller.go | 6 - operator/internal/testenv/testenv.go | 62 +++- taskfiles/k8s.yml | 3 + 13 files changed, 628 insertions(+), 180 deletions(-) create mode 100644 operator/internal/controller/redpanda/role.yaml diff --git a/operator/config/rbac/bases/operator/role.yaml b/operator/config/rbac/bases/operator/role.yaml index e6549fefe..9623a4391 100644 --- a/operator/config/rbac/bases/operator/role.yaml +++ b/operator/config/rbac/bases/operator/role.yaml @@ -97,10 +97,10 @@ rules: - apiGroups: - cluster.redpanda.com resources: - - schemas - - topics - - users + - redpandas verbs: + - create + - delete - get - list - patch @@ -109,6 +109,7 @@ rules: - apiGroups: - cluster.redpanda.com resources: + - redpandas/finalizers - schemas/finalizers - topics/finalizers - users/finalizers @@ -117,6 +118,7 @@ rules: - apiGroups: - cluster.redpanda.com resources: + - redpandas/status - schemas/status - topics/status - users/status @@ -124,6 +126,18 @@ rules: - get - patch - update +- apiGroups: + - cluster.redpanda.com + resources: + - schemas + - topics + - users + verbs: + - get + - list + - patch + - update + - watch - apiGroups: - networking.k8s.io resources: @@ -155,6 +169,7 @@ rules: - clusterroles verbs: - create + - delete - get - list - patch @@ -309,50 +324,6 @@ rules: - patch - update - watch -- apiGroups: - - cluster.redpanda.com - resources: - - redpandas - verbs: - - create - - delete - - get - - list - - patch - - update - - watch -- apiGroups: - - cluster.redpanda.com - resources: - - redpandas/finalizers - - schemas/finalizers - - topics/finalizers - - users/finalizers - verbs: - - update -- apiGroups: - - cluster.redpanda.com - resources: - - redpandas/status - - schemas/status - - topics/status - - users/status - verbs: - - get - - patch - - update -- apiGroups: - - cluster.redpanda.com - resources: - - schemas - - topics - - users - verbs: - - get - - list - - patch - - update - - watch - apiGroups: - coordination.k8s.io resources: @@ -394,6 +365,7 @@ rules: - apiGroups: - monitoring.coreos.com resources: + - podmonitors - servicemonitors verbs: - create diff --git a/operator/config/rbac/v2-manager-role/role.yaml b/operator/config/rbac/v2-manager-role/role.yaml index d798b5af5..b1ee6af5c 100644 --- a/operator/config/rbac/v2-manager-role/role.yaml +++ b/operator/config/rbac/v2-manager-role/role.yaml @@ -7,7 +7,9 @@ rules: - apiGroups: - "" resources: + - configmaps - nodes + - secrets verbs: - get - list @@ -26,10 +28,10 @@ rules: - apiGroups: - cluster.redpanda.com resources: - - schemas - - topics - - users + - redpandas verbs: + - create + - delete - get - list - patch @@ -38,6 +40,7 @@ rules: - apiGroups: - cluster.redpanda.com resources: + - redpandas/finalizers - schemas/finalizers - topics/finalizers - users/finalizers @@ -46,6 +49,7 @@ rules: - apiGroups: - cluster.redpanda.com resources: + - redpandas/status - schemas/status - topics/status - users/status @@ -53,6 +57,31 @@ rules: - get - patch - update +- apiGroups: + - cluster.redpanda.com + resources: + - schemas + - topics + - users + verbs: + - get + - list + - patch + - update + - watch +- apiGroups: + - rbac.authorization.k8s.io + resources: + - clusterrolebindings + - clusterroles + verbs: + - create + - delete + - get + - list + - patch + - update + - watch --- apiVersion: rbac.authorization.k8s.io/v1 kind: Role @@ -63,15 +92,13 @@ rules: - apiGroups: - "" resources: - - events + - configmaps + - pods + - secrets + - serviceaccounts + - services verbs: - create - - patch -- apiGroups: - - "" - resources: - - persistentvolumeclaims - verbs: - delete - get - list @@ -81,12 +108,15 @@ rules: - apiGroups: - "" resources: - - pods - - secrets - - serviceaccounts - - services + - events verbs: - create + - patch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: - delete - get - list @@ -158,50 +188,6 @@ rules: - patch - update - watch -- apiGroups: - - cluster.redpanda.com - resources: - - redpandas - verbs: - - create - - delete - - get - - list - - patch - - update - - watch -- apiGroups: - - cluster.redpanda.com - resources: - - redpandas/finalizers - - schemas/finalizers - - topics/finalizers - - users/finalizers - verbs: - - update -- apiGroups: - - cluster.redpanda.com - resources: - - redpandas/status - - schemas/status - - topics/status - - users/status - verbs: - - get - - patch - - update -- apiGroups: - - cluster.redpanda.com - resources: - - schemas - - topics - - users - verbs: - - get - - list - - patch - - update - - watch - apiGroups: - helm.toolkit.fluxcd.io resources: @@ -231,6 +217,7 @@ rules: - apiGroups: - monitoring.coreos.com resources: + - podmonitors - servicemonitors verbs: - create diff --git a/operator/internal/controller/redpanda/managed_decommission_controller.go b/operator/internal/controller/redpanda/managed_decommission_controller.go index 49d78ba41..8470ef55c 100644 --- a/operator/internal/controller/redpanda/managed_decommission_controller.go +++ b/operator/internal/controller/redpanda/managed_decommission_controller.go @@ -46,7 +46,7 @@ const ( var ErrZeroReplicas = errors.New("redpanda replicas is zero") -// +kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=redpandas,verbs=get;list;watch; +// +kubebuilder:rbac:groups=cluster.redpanda.com,resources=redpandas,verbs=get;list;watch; // +kubebuilder:rbac:groups=core,namespace=default,resources=pods,verbs=update;patch;delete;get;list;watch; // +kubebuilder:rbac:groups=core,namespace=default,resources=pods/status,verbs=update;patch // +kubebuilder:rbac:groups=core,namespace=default,resources=persistentvolumeclaims,verbs=get;list;update;patch;delete;watch diff --git a/operator/internal/controller/redpanda/redpanda_controller.go b/operator/internal/controller/redpanda/redpanda_controller.go index 43b556d50..b21f01ab0 100644 --- a/operator/internal/controller/redpanda/redpanda_controller.go +++ b/operator/internal/controller/redpanda/redpanda_controller.go @@ -104,27 +104,25 @@ type RedpandaReconciler struct { // any resource that Redpanda helm creates and flux controller needs to reconcile them // +kubebuilder:rbac:groups="",namespace=default,resources=pods,verbs=get;list;watch;create;update;patch;delete -// +kubebuilder:rbac:groups=rbac.authorization.k8s.io,namespace=default,resources=rolebindings,verbs=get;list;watch;create;update;patch;delete -// +kubebuilder:rbac:groups=rbac.authorization.k8s.io,namespace=default,resources=roles,verbs=get;list;watch;create;update;patch;delete +// +kubebuilder:rbac:groups=rbac.authorization.k8s.io,resources=clusterroles;clusterrolebindings,verbs=get;list;watch;create;update;patch;delete +// +kubebuilder:rbac:groups=rbac.authorization.k8s.io,namespace=default,resources=roles;rolebindings,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=batch,namespace=default,resources=jobs,verbs=get;list;watch;create;update;patch;delete -// +kubebuilder:rbac:groups=core,namespace=default,resources=secrets,verbs=get;list;watch;create;update;patch;delete -// +kubebuilder:rbac:groups=core,namespace=default,resources=services,verbs=get;list;watch;create;update;patch;delete -// +kubebuilder:rbac:groups=core,namespace=default,resources=serviceaccounts,verbs=get;list;watch;create;update;patch;delete +// +kubebuilder:rbac:groups=core,namespace=default,resources=configmaps;secrets;services;serviceaccounts,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=apps,namespace=default,resources=statefulsets,verbs=get;list;watch;create;update;patch;delete; // +kubebuilder:rbac:groups=policy,namespace=default,resources=poddisruptionbudgets,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=apps,namespace=default,resources=deployments,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=cert-manager.io,namespace=default,resources=certificates,verbs=get;create;update;patch;delete;list;watch // +kubebuilder:rbac:groups=cert-manager.io,namespace=default,resources=issuers,verbs=get;create;update;patch;delete;list;watch -// +kubebuilder:rbac:groups="monitoring.coreos.com",namespace=default,resources=servicemonitors,verbs=get;list;watch;create;update;patch;delete -// +kubebuilder:rbac:groups=networking.k8s.io,namespace=default,resources=ingresses,verbs=get;list;watch;create;update;patch;delete +// +kubebuilder:rbac:groups="monitoring.coreos.com",namespace=default,resources=podmonitors;servicemonitors,verbs=get;list;watch;create;update;patch;delete // Console chart // +kubebuilder:rbac:groups=autoscaling,namespace=default,resources=horizontalpodautoscalers,verbs=get;list;watch;create;update;patch;delete +// +kubebuilder:rbac:groups=networking.k8s.io,namespace=default,resources=ingresses,verbs=get;list;watch;create;update;patch;delete // redpanda resources -// +kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=redpandas,verbs=get;list;watch;create;update;patch;delete -// +kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=redpandas/status,verbs=get;update;patch -// +kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=redpandas/finalizers,verbs=update +// +kubebuilder:rbac:groups=cluster.redpanda.com,resources=redpandas,verbs=get;list;watch;create;update;patch;delete +// +kubebuilder:rbac:groups=cluster.redpanda.com,resources=redpandas/status,verbs=get;update;patch +// +kubebuilder:rbac:groups=cluster.redpanda.com,resources=redpandas/finalizers,verbs=update // +kubebuilder:rbac:groups=core,namespace=default,resources=events,verbs=create;patch // SetupWithManager sets up the controller with the Manager. diff --git a/operator/internal/controller/redpanda/redpanda_controller_test.go b/operator/internal/controller/redpanda/redpanda_controller_test.go index f1d7c0d73..73ff9e024 100644 --- a/operator/internal/controller/redpanda/redpanda_controller_test.go +++ b/operator/internal/controller/redpanda/redpanda_controller_test.go @@ -11,15 +11,18 @@ package redpanda_test import ( "context" + _ "embed" "encoding/json" "fmt" "math/rand" + "slices" "sort" "strings" "testing" "time" fluxclient "github.com/fluxcd/pkg/runtime/client" + sourcecontrollerv1beta2 "github.com/fluxcd/source-controller/api/v1beta2" "github.com/go-logr/logr/testr" redpandachart "github.com/redpanda-data/helm-charts/charts/redpanda" "github.com/redpanda-data/helm-charts/pkg/gotohelm/helmette" @@ -36,6 +39,7 @@ import ( "github.com/stretchr/testify/suite" appsv1 "k8s.io/api/apps/v1" corev1 "k8s.io/api/core/v1" + rbacv1 "k8s.io/api/rbac/v1" apierrors "k8s.io/apimachinery/pkg/api/errors" apimeta "k8s.io/apimachinery/pkg/api/meta" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -47,6 +51,12 @@ import ( "sigs.k8s.io/controller-runtime/pkg/client" ) +// operatorRBAC is the ClusterRole and Role generated via controller-gen and +// goembeded so it can be used for tests. +// +//go:embed role.yaml +var operatorRBAC []byte + // NB: This test setup is largely incompatible with webhooks. Though we might // be able to figure something freaky out. func TestRedpandaController(t *testing.T) { @@ -254,7 +264,7 @@ func (s *RedpandaControllerSuite) TestManagedDecommission() { "operator.redpanda.com/managed-decommission": "2999-12-31T00:00:00Z", } - s.applyAndWaitFor(rp, func(o client.Object) bool { + s.applyAndWaitFor(func(o client.Object) bool { rp := o.(*redpandav1alpha2.Redpanda) for _, cond := range rp.Status.Conditions { @@ -263,7 +273,7 @@ func (s *RedpandaControllerSuite) TestManagedDecommission() { } } return false - }) + }, rp) s.waitFor(rp, func(o client.Object) bool { rp := o.(*redpandav1alpha2.Redpanda) @@ -318,7 +328,7 @@ func (s *RedpandaControllerSuite) TestClusterSettings() { rp.Spec.ClusterSpec.Config.Cluster = &runtime.RawExtension{Raw: asJson} s.applyAndWait(rp) - s.applyAndWaitFor(rp, func(o client.Object) bool { + s.applyAndWaitFor(func(o client.Object) bool { rp := o.(*redpandav1alpha2.Redpanda) for _, cond := range rp.Status.Conditions { if cond.Type == redpandav1alpha2.ClusterConfigSynced { @@ -326,7 +336,7 @@ func (s *RedpandaControllerSuite) TestClusterSettings() { } } return false - }) + }, rp) } s.applyAndWait(&corev1.Secret{ ObjectMeta: metav1.ObjectMeta{ @@ -426,14 +436,14 @@ func (s *RedpandaControllerSuite) SetupSuite() { // rest config given to the manager. s.ctx = context.Background() s.env = testenv.New(t, testenv.Options{ - Scheme: controller.UnifiedScheme, + Scheme: controller.V2Scheme, CRDs: crds.All(), Logger: testr.New(t), }) s.client = s.env.Client() - s.env.SetupManager(func(mgr ctrl.Manager) error { + s.env.SetupManager(s.setupRBAC(), func(mgr ctrl.Manager) error { controllers := flux.NewFluxControllers(mgr, fluxclient.Options{}, fluxclient.KubeConfigOptions{}) for _, controller := range controllers { if err := controller.SetupWithManager(s.ctx, mgr); err != nil { @@ -460,20 +470,108 @@ func (s *RedpandaControllerSuite) SetupSuite() { ClientFactory: s.clientFactory, }).SetupWithManager(mgr) }) + + // NB: t.Cleanup is used here to properly order our shutdown logic with + // testenv which also uses t.Cleanup. Utilizing testify's TearDownAll makes + // reasoning about the ordering more complicated than need be. + t.Cleanup(func() { + // Due to a fun race condition in testenv, we need to clear out all the + // redpandas before we can let testenv shutdown. If we don't, the + // operator's ClusterRoles and Roles may get GC'd before all the Redpandas + // do which will prevent the operator from cleaning up said Redpandas. + var redpandas redpandav1alpha2.RedpandaList + s.NoError(s.env.Client().List(s.ctx, &redpandas)) + + for _, rp := range redpandas.Items { + s.deleteAndWait(&rp) + } + + // For some reason, it seems that HelmCharts can get abandoned. Clean + // them up to prevent hanging the NS deletion. + var helmCharts sourcecontrollerv1beta2.HelmChartList + s.NoError(s.env.Client().List(s.ctx, &helmCharts)) + + for _, chart := range helmCharts.Items { + s.deleteAndWait(&chart) + } + }) } -func (s *RedpandaControllerSuite) minimalRP(useFlux bool) *redpandav1alpha2.Redpanda { +func (s *RedpandaControllerSuite) setupRBAC() string { + roles, err := kube.DecodeYAML(operatorRBAC, s.client.Scheme()) + s.Require().NoError(err) + + role := roles[1].(*rbacv1.Role) + clusterRole := roles[0].(*rbacv1.ClusterRole) + + // Inject additional permissions required for running in testenv. + role.Rules = append(role.Rules, rbacv1.PolicyRule{ + APIGroups: []string{""}, + Resources: []string{"pods/portforward"}, + Verbs: []string{"*"}, + }) + + name := "testenv-" + s.randString(6) + + role.Name = name + role.Namespace = s.env.Namespace() + clusterRole.Name = name + clusterRole.Namespace = s.env.Namespace() + + s.applyAndWait(roles...) + s.applyAndWait( + &corev1.ServiceAccount{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + }, + }, + &rbacv1.RoleBinding{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + }, + Subjects: []rbacv1.Subject{ + {Kind: "ServiceAccount", Namespace: s.env.Namespace(), Name: name}, + }, + RoleRef: rbacv1.RoleRef{ + APIGroup: "rbac.authorization.k8s.io", + Kind: "Role", + Name: role.Name, + }, + }, + &rbacv1.ClusterRoleBinding{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + }, + Subjects: []rbacv1.Subject{ + {Kind: "ServiceAccount", Namespace: s.env.Namespace(), Name: name}, + }, + RoleRef: rbacv1.RoleRef{ + APIGroup: "rbac.authorization.k8s.io", + Kind: "ClusterRole", + Name: clusterRole.Name, + }, + }, + ) + + return name +} + +func (s *RedpandaControllerSuite) randString(length int) string { const alphabet = "abcdefghijklmnopqrstuvwxyz0123456789" - name := "rp-" + name := "" for i := 0; i < 6; i++ { //nolint:gosec // not meant to be a secure random string. name += string(alphabet[rand.Intn(len(alphabet))]) } + return name +} + +func (s *RedpandaControllerSuite) minimalRP(useFlux bool) *redpandav1alpha2.Redpanda { return &redpandav1alpha2.Redpanda{ ObjectMeta: metav1.ObjectMeta{ - Name: name, // GenerateName doesn't play nice with SSA. + Name: "rp-" + s.randString(6), // GenerateName doesn't play nice with SSA. }, Spec: redpandav1alpha2.RedpandaSpec{ ChartRef: redpandav1alpha2.ChartRef{ @@ -502,6 +600,11 @@ func (s *RedpandaControllerSuite) minimalRP(useFlux bool) *redpandav1alpha2.Redp // brokers. Type: ptr.To("soft"), }, + // Speeds up managed decommission tests. Decommissioned + // nodes will take the entirety of + // TerminationGracePeriodSeconds as the pre-stop hook + // doesn't account for decommissioned nodes. + TerminationGracePeriodSeconds: ptr.To(10), }, Resources: &redpandav1alpha2.Resources{ CPU: &redpandav1alpha2.CPU{ @@ -541,45 +644,50 @@ func (s *RedpandaControllerSuite) deleteAndWait(obj client.Object) { })) } -func (s *RedpandaControllerSuite) applyAndWait(obj client.Object) { - s.applyAndWaitFor(obj, func(o client.Object) bool { +func (s *RedpandaControllerSuite) applyAndWait(objs ...client.Object) { + s.applyAndWaitFor(func(obj client.Object) bool { switch obj := obj.(type) { case *redpandav1alpha2.Redpanda: ready := apimeta.IsStatusConditionTrue(obj.Status.Conditions, "Ready") upToDate := obj.Generation != 0 && obj.Generation == obj.Status.ObservedGeneration return upToDate && ready - case *corev1.Secret, *corev1.ConfigMap: + case *corev1.Secret, *corev1.ConfigMap, *corev1.ServiceAccount, + *rbacv1.ClusterRole, *rbacv1.Role, *rbacv1.RoleBinding, *rbacv1.ClusterRoleBinding: return true default: s.T().Fatalf("unhandled object %T in applyAndWait", obj) panic("unreachable") } - }) + }, objs...) } -func (s *RedpandaControllerSuite) applyAndWaitFor(obj client.Object, cond func(client.Object) bool) { - gvk, err := s.client.GroupVersionKindFor(obj) - s.NoError(err) +func (s *RedpandaControllerSuite) applyAndWaitFor(cond func(client.Object) bool, objs ...client.Object) { + for _, obj := range objs { + gvk, err := s.client.GroupVersionKindFor(obj) + s.NoError(err) - obj.SetManagedFields(nil) - obj.GetObjectKind().SetGroupVersionKind(gvk) + obj.SetManagedFields(nil) + obj.GetObjectKind().SetGroupVersionKind(gvk) - s.Require().NoError(s.client.Patch(s.ctx, obj, client.Apply, client.ForceOwnership, client.FieldOwner("tests"))) + s.Require().NoError(s.client.Patch(s.ctx, obj, client.Apply, client.ForceOwnership, client.FieldOwner("tests"))) + } - s.NoError(wait.PollUntilContextTimeout(s.ctx, 5*time.Second, 5*time.Minute, false, func(ctx context.Context) (done bool, err error) { - if err := s.client.Get(ctx, client.ObjectKeyFromObject(obj), obj); err != nil { - return false, err - } + for _, obj := range objs { + s.NoError(wait.PollUntilContextTimeout(s.ctx, 5*time.Second, 5*time.Minute, false, func(ctx context.Context) (done bool, err error) { + if err := s.client.Get(ctx, client.ObjectKeyFromObject(obj), obj); err != nil { + return false, err + } - if cond(obj) { - return true, nil - } + if cond(obj) { + return true, nil + } - s.T().Logf("waiting for %T %q to be ready", obj, obj.GetName()) - return false, nil - })) + s.T().Logf("waiting for %T %q to be ready", obj, obj.GetName()) + return false, nil + })) + } } func (s *RedpandaControllerSuite) waitFor(obj client.Object, cond func(client.Object) bool) { @@ -686,3 +794,59 @@ func TestPostInstallUpgradeJobIndex(t *testing.T) { // `clusterConfigfor` utilizes. require.Equal(t, "bootstrap-yaml-envsubst", job.Spec.Template.Spec.InitContainers[0].Name) } + +// TestControllerRBAC asserts that the declared Roles and ClusterRoles of the +// RedpandaReconciler line up with all the resource types it needs to manage. +func TestControllerRBAC(t *testing.T) { + scheme := controller.V2Scheme + + expectedVerbs := []string{"create", "delete", "get", "list", "patch", "update", "watch"} + + roles, err := kube.DecodeYAML(operatorRBAC, scheme) + require.NoError(t, err) + + role := roles[1].(*rbacv1.Role) + clusterRole := roles[0].(*rbacv1.ClusterRole) + + for _, typ := range redpandachart.Types() { + gkvs, _, err := scheme.ObjectKinds(typ) + require.NoError(t, err) + + require.Len(t, gkvs, 1) + gvk := gkvs[0] + + rules := role.Rules + if !isNamespaced(typ) { + rules = clusterRole.Rules + } + + group := gvk.Group + kind := pluralize(gvk.Kind) + + idx := slices.IndexFunc(rules, func(rule rbacv1.PolicyRule) bool { + return slices.Contains(rule.APIGroups, group) && slices.Contains(rule.Resources, kind) + }) + + require.NotEqual(t, -1, idx, "missing rules for %s %s", gvk.Group, kind) + require.EqualValues(t, expectedVerbs, rules[idx].Verbs, "incorrect verbs for %s %s", gvk.Group, kind) + } +} + +func isNamespaced(obj client.Object) bool { + switch obj.(type) { + case *corev1.Namespace, *rbacv1.ClusterRole, *rbacv1.ClusterRoleBinding: + return false + default: + return true + } +} + +func pluralize(kind string) string { + switch kind[len(kind)-1] { + case 's': + return strings.ToLower(kind) + "es" + + default: + return strings.ToLower(kind) + "s" + } +} diff --git a/operator/internal/controller/redpanda/redpanda_decommission_controller.go b/operator/internal/controller/redpanda/redpanda_decommission_controller.go index 737df0bc5..3476b82c7 100644 --- a/operator/internal/controller/redpanda/redpanda_decommission_controller.go +++ b/operator/internal/controller/redpanda/redpanda_decommission_controller.go @@ -34,11 +34,14 @@ import ( "github.com/redpanda-data/redpanda-operator/operator/pkg/collections" ) -// +kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=redpandas,verbs=get;list;watch; +// +kubebuilder:rbac:groups=cluster.redpanda.com,resources=redpandas,verbs=get;list;watch; // +kubebuilder:rbac:groups=core,namespace=default,resources=pods,verbs=get;list;watch; // +kubebuilder:rbac:groups=core,namespace=default,resources=persistentvolumeclaims,verbs=get;list;update;patch;delete;watch // +kubebuilder:rbac:groups=core,resources=persistentvolumes,verbs=get;list;update;patch;watch // +kubebuilder:rbac:groups=apps,namespace=default,resources=statefulsets/status,verbs=update;patch +// Cluster wide access to configmaps and secrets are required as long as we're +// fetching values from helm. +// +kubebuilder:rbac:groups=core,resources=configmaps;secrets,verbs=get;list;watch const ( DecommissionCondition = "DecommissionPhase" diff --git a/operator/internal/controller/redpanda/redpanda_node_pvc_controller.go b/operator/internal/controller/redpanda/redpanda_node_pvc_controller.go index ca4bc5b89..09c824b82 100644 --- a/operator/internal/controller/redpanda/redpanda_node_pvc_controller.go +++ b/operator/internal/controller/redpanda/redpanda_node_pvc_controller.go @@ -25,10 +25,13 @@ import ( "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha1" ) -// +kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=redpandas,verbs=get;list;watch; +// +kubebuilder:rbac:groups=cluster.redpanda.com,resources=redpandas,verbs=get;list;watch; // +kubebuilder:rbac:groups=core,namespace=default,resources=persistentvolumeclaims,verbs=get;list;update;patch;delete // +kubebuilder:rbac:groups=core,resources=persistentvolumes,verbs=get;list;update;patch;delete // +kubebuilder:rbac:groups=core,resources=nodes,verbs=get;list;watch +// Cluster wide access to configmaps and secrets are required as long as we're +// fetching values from helm. +// +kubebuilder:rbac:groups=core,resources=configmaps;secrets,verbs=get;list;watch // RedpandaNodePVCReconciler watches node objects, and sets annotation to PVC to mark them for deletion type RedpandaNodePVCReconciler struct { diff --git a/operator/internal/controller/redpanda/role.yaml b/operator/internal/controller/redpanda/role.yaml new file mode 100644 index 000000000..b1ee6af5c --- /dev/null +++ b/operator/internal/controller/redpanda/role.yaml @@ -0,0 +1,304 @@ +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + name: v2-manager-role +rules: +- apiGroups: + - "" + resources: + - configmaps + - nodes + - secrets + verbs: + - get + - list + - watch +- apiGroups: + - "" + resources: + - persistentvolumes + verbs: + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - cluster.redpanda.com + resources: + - redpandas + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - cluster.redpanda.com + resources: + - redpandas/finalizers + - schemas/finalizers + - topics/finalizers + - users/finalizers + verbs: + - update +- apiGroups: + - cluster.redpanda.com + resources: + - redpandas/status + - schemas/status + - topics/status + - users/status + verbs: + - get + - patch + - update +- apiGroups: + - cluster.redpanda.com + resources: + - schemas + - topics + - users + verbs: + - get + - list + - patch + - update + - watch +- apiGroups: + - rbac.authorization.k8s.io + resources: + - clusterrolebindings + - clusterroles + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: Role +metadata: + name: v2-manager-role + namespace: default +rules: +- apiGroups: + - "" + resources: + - configmaps + - pods + - secrets + - serviceaccounts + - services + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - "" + resources: + - events + verbs: + - create + - patch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - "" + resources: + - pods/status + verbs: + - patch + - update +- apiGroups: + - apps + resources: + - deployments + - replicasets + - statefulsets + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - apps + resources: + - statefulsets/status + verbs: + - patch + - update +- apiGroups: + - autoscaling + resources: + - horizontalpodautoscalers + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - batch + resources: + - jobs + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - cert-manager.io + resources: + - certificates + - issuers + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - helm.toolkit.fluxcd.io + resources: + - helmreleases + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - helm.toolkit.fluxcd.io + resources: + - helmreleases/finalizers + verbs: + - update +- apiGroups: + - helm.toolkit.fluxcd.io + resources: + - helmreleases/status + verbs: + - get + - patch + - update +- apiGroups: + - monitoring.coreos.com + resources: + - podmonitors + - servicemonitors + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - networking.k8s.io + resources: + - ingresses + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - policy + resources: + - poddisruptionbudgets + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - rbac.authorization.k8s.io + resources: + - rolebindings + - roles + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - source.toolkit.fluxcd.io + resources: + - buckets + - gitrepositories + - gitrepository + - helmcharts + - helmrepositories + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - source.toolkit.fluxcd.io + resources: + - gitrepository/finalizers + - helmcharts/finalizers + - helmrepositories/finalizers + verbs: + - create + - delete + - get + - patch + - update +- apiGroups: + - source.toolkit.fluxcd.io + resources: + - gitrepository/status + - helmcharts/status + - helmrepositories/status + verbs: + - get + - patch + - update diff --git a/operator/internal/controller/redpanda/schema_controller.go b/operator/internal/controller/redpanda/schema_controller.go index d12c5c910..bb2668523 100644 --- a/operator/internal/controller/redpanda/schema_controller.go +++ b/operator/internal/controller/redpanda/schema_controller.go @@ -24,12 +24,6 @@ import ( "sigs.k8s.io/controller-runtime/pkg/client" ) -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=schemas,verbs=get;list;watch;update;patch -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=schemas/status,verbs=get;update;patch -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=schemas/finalizers,verbs=update - -// For cluster scoped operator - //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=schemas,verbs=get;list;watch;update;patch //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=schemas/status,verbs=get;update;patch //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=schemas/finalizers,verbs=update diff --git a/operator/internal/controller/redpanda/topic_controller.go b/operator/internal/controller/redpanda/topic_controller.go index 74ebf07fa..296fcc4a8 100644 --- a/operator/internal/controller/redpanda/topic_controller.go +++ b/operator/internal/controller/redpanda/topic_controller.go @@ -63,12 +63,6 @@ type TopicReconciler struct { kuberecorder.EventRecorder } -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=topics,verbs=get;list;watch;update;patch -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=topics/status,verbs=get;update;patch -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=topics/finalizers,verbs=update - -// For cluster scoped operator - //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=topics,verbs=get;list;watch;update;patch //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=topics/status,verbs=get;update;patch //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=topics/finalizers,verbs=update diff --git a/operator/internal/controller/redpanda/user_controller.go b/operator/internal/controller/redpanda/user_controller.go index 897956efc..69801c25c 100644 --- a/operator/internal/controller/redpanda/user_controller.go +++ b/operator/internal/controller/redpanda/user_controller.go @@ -28,12 +28,6 @@ import ( "sigs.k8s.io/controller-runtime/pkg/client" ) -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=users,verbs=get;list;watch;update;patch -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=users/status,verbs=get;update;patch -//+kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=users/finalizers,verbs=update - -// For cluster scoped operator - //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=users,verbs=get;list;watch;update;patch //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=users/status,verbs=get;update;patch //+kubebuilder:rbac:groups=cluster.redpanda.com,resources=users/finalizers,verbs=update diff --git a/operator/internal/testenv/testenv.go b/operator/internal/testenv/testenv.go index dedd8878c..b11b3f664 100644 --- a/operator/internal/testenv/testenv.go +++ b/operator/internal/testenv/testenv.go @@ -11,6 +11,7 @@ package testenv import ( "context" + "fmt" "testing" "time" @@ -128,13 +129,23 @@ func (e *Env) Client() client.Client { return e.wrapClient(e.client()) } -func (e *Env) SetupManager(fn func(ctrl.Manager) error) { +func (e *Env) Namespace() string { + return e.namespace.Name +} + +func (e *Env) SetupManager(serviceAccount string, fn func(ctrl.Manager) error) { + // Bind the managers base config to a ServiceAccount via the "Impersonate" + // feature. This ensures that any permissions/RBAC issues get caught by + // theses tests as e.config has Admin permissions. + config := rest.CopyConfig(e.config) + config.Impersonate.UserName = fmt.Sprintf("system:serviceaccount:%s:%s", e.Namespace(), serviceAccount) + // TODO: Webhooks likely aren't going to place nicely with this method of // testing. The Kube API server will have to dial out of the cluster to the // local machine which could prove to be difficult across all docker/docker // in docker environments. // See also https://k3d.io/v5.4.6/faq/faq/?h=host#how-to-access-services-like-a-database-running-on-my-docker-host-machine - manager, err := ctrl.NewManager(e.config, ctrl.Options{ + manager, err := ctrl.NewManager(config, ctrl.Options{ Cache: cache.Options{ // Limit this manager to only interacting with objects within our // namespace. @@ -163,7 +174,10 @@ func (e *Env) SetupManager(fn func(ctrl.Manager) error) { require.NoError(e.t, fn(manager)) e.group.Go(func() error { - return manager.Start(e.ctx) + if err := manager.Start(e.ctx); err != nil && e.ctx.Err() != nil { + return err + } + return nil }) // No Without leader election enabled, this is just a wait for the manager @@ -180,13 +194,20 @@ func (e *Env) client() client.Client { } func (e *Env) wrapClient(c client.Client) client.Client { + gvk, err := c.GroupVersionKindFor(e.namespace) + if err != nil { + panic(err) + } + + apiVersion, kind := gvk.ToAPIVersionAndKind() + // Bind all operations to this namespace. We'll delete it at the end of this test. c = client.NewNamespacedClient(c, e.namespace.Name) // For any non-namespaced resources, we'll attach an OwnerReference to our // Namespace to ensure they get cleaned up as well. c = newOwnedClient(c, metav1.OwnerReference{ - APIVersion: e.namespace.GetObjectKind().GroupVersionKind().GroupVersion().String(), - Kind: e.namespace.GetObjectKind().GroupVersionKind().Kind, + APIVersion: apiVersion, + Kind: kind, UID: e.namespace.UID, Name: e.namespace.Name, BlockOwnerDeletion: ptr.To(true), @@ -195,22 +216,33 @@ func (e *Env) wrapClient(c client.Client) client.Client { } func (e *Env) shutdown() { + // Our shutdown logic is unfortunately complicated. The manager is using a + // ServiceAccount and roles that will be GC'd by the Namespace deletion due + // to the OwnedClient wrapper but the Namespace deletion could be blocked + // by finalizers that would need to be removed by the manager itself. + // + // For now, we'll assume that tests have cleaned all resources associated + // with the manager and shutdown the manager before cleaning out the + // namespace. + + // Shutdown the manger. + e.cancel() + assert.NoError(e.t, e.group.Wait()) + + // Clean up our namespace. if !testutil.Retain() { - // NB: Namespace deletion MUST happen before calling e.cancel. - // Otherwise we risk stopping controllers that would remove finalizers - // from various resources in the cluster which would hang the namespace - // deletion forever. + // Mint a new ctx for NS clean up as e.ctx has been canceled. + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + c := e.client() - assert.NoError(e.t, c.Delete(e.ctx, e.namespace, client.PropagationPolicy(metav1.DeletePropagationForeground))) + assert.NoError(e.t, c.Delete(ctx, e.namespace, client.PropagationPolicy(metav1.DeletePropagationForeground))) // Poll until the namespace is fully deleted. - assert.NoError(e.t, wait.PollUntilContextTimeout(e.ctx, time.Second, 5*time.Minute, false, func(ctx context.Context) (done bool, err error) { + assert.NoErrorf(e.t, wait.PollUntilContextTimeout(ctx, time.Second, 5*time.Minute, false, func(ctx context.Context) (done bool, err error) { err = c.Get(ctx, client.ObjectKeyFromObject(e.namespace), e.namespace) return apierrors.IsNotFound(err), client.IgnoreNotFound(err) - })) + }), "stalled waiting for Namespace %q to finish deleting", e.namespace.Name) } - - e.cancel() - assert.NoError(e.t, e.group.Wait()) } diff --git a/taskfiles/k8s.yml b/taskfiles/k8s.yml index 3210ba960..94a478fea 100644 --- a/taskfiles/k8s.yml +++ b/taskfiles/k8s.yml @@ -31,6 +31,9 @@ tasks: paths='./internal/controller/redpanda/...' \ rbac:roleName=v2-manager-role \ output:rbac:artifacts:config=config/rbac/v2-manager-role + # controller-gen won't output to multiple directories and we need a copy + # of resources for tests, so cp it is. + - cp ./config/rbac/v2-manager-role/*.yaml ./internal/controller/redpanda/ - | controller-gen \ paths='./internal/controller/vectorized/...' \