diff --git a/operator/config/rbac/bases/operator/role.yaml b/operator/config/rbac/bases/operator/role.yaml index e6549fefe..af57b57b9 100644 --- a/operator/config/rbac/bases/operator/role.yaml +++ b/operator/config/rbac/bases/operator/role.yaml @@ -155,6 +155,7 @@ rules: - clusterroles verbs: - create + - delete - get - list - patch @@ -394,6 +395,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..b9e008296 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 @@ -53,6 +55,19 @@ rules: - get - patch - update +- 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 +78,13 @@ rules: - apiGroups: - "" resources: - - events + - configmaps + - pods + - secrets + - serviceaccounts + - services verbs: - create - - patch -- apiGroups: - - "" - resources: - - persistentvolumeclaims - verbs: - delete - get - list @@ -81,12 +94,15 @@ rules: - apiGroups: - "" resources: - - pods - - secrets - - serviceaccounts - - services + - events verbs: - create + - patch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: - delete - get - list @@ -231,6 +247,7 @@ rules: - apiGroups: - monitoring.coreos.com resources: + - podmonitors - servicemonitors verbs: - create diff --git a/operator/internal/controller/redpanda/redpanda_controller.go b/operator/internal/controller/redpanda/redpanda_controller.go index 43b556d50..2561a3595 100644 --- a/operator/internal/controller/redpanda/redpanda_controller.go +++ b/operator/internal/controller/redpanda/redpanda_controller.go @@ -104,22 +104,20 @@ 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 diff --git a/operator/internal/controller/redpanda/redpanda_controller_test.go b/operator/internal/controller/redpanda/redpanda_controller_test.go index f1d7c0d73..74ac03f79 100644 --- a/operator/internal/controller/redpanda/redpanda_controller_test.go +++ b/operator/internal/controller/redpanda/redpanda_controller_test.go @@ -11,9 +11,11 @@ package redpanda_test import ( "context" + _ "embed" "encoding/json" "fmt" "math/rand" + "slices" "sort" "strings" "testing" @@ -36,6 +38,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 +50,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) { @@ -65,7 +74,10 @@ type RedpandaControllerSuite struct { clientFactory internalclient.ClientFactory } -var _ suite.SetupAllSuite = (*RedpandaControllerSuite)(nil) +var ( + _ suite.SetupAllSuite = (*RedpandaControllerSuite)(nil) + _ suite.TearDownAllSuite = (*RedpandaControllerSuite)(nil) +) // TestStableUIDAndGeneration asserts that UIDs, Generations, Labels, and // Annotations of all objects created by the controller are stable across flux @@ -254,7 +266,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 +275,7 @@ func (s *RedpandaControllerSuite) TestManagedDecommission() { } } return false - }) + }, rp) s.waitFor(rp, func(o client.Object) bool { rp := o.(*redpandav1alpha2.Redpanda) @@ -318,7 +330,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 +338,7 @@ func (s *RedpandaControllerSuite) TestClusterSettings() { } } return false - }) + }, rp) } s.applyAndWait(&corev1.Secret{ ObjectMeta: metav1.ObjectMeta{ @@ -426,14 +438,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 { @@ -462,18 +474,94 @@ func (s *RedpandaControllerSuite) SetupSuite() { }) } -func (s *RedpandaControllerSuite) minimalRP(useFlux bool) *redpandav1alpha2.Redpanda { +func (s *RedpandaControllerSuite) TearDownSuite() { + // 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) + } +} + +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{ @@ -541,45 +629,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 +779,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..33550fe5d 100644 --- a/operator/internal/controller/redpanda/redpanda_decommission_controller.go +++ b/operator/internal/controller/redpanda/redpanda_decommission_controller.go @@ -39,6 +39,9 @@ import ( // +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..bc5e6d823 100644 --- a/operator/internal/controller/redpanda/redpanda_node_pvc_controller.go +++ b/operator/internal/controller/redpanda/redpanda_node_pvc_controller.go @@ -29,6 +29,9 @@ import ( // +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..b9e008296 --- /dev/null +++ b/operator/internal/controller/redpanda/role.yaml @@ -0,0 +1,334 @@ +--- +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: + - schemas + - topics + - users + verbs: + - get + - list + - patch + - update + - watch +- apiGroups: + - cluster.redpanda.com + resources: + - schemas/finalizers + - topics/finalizers + - users/finalizers + verbs: + - update +- apiGroups: + - cluster.redpanda.com + resources: + - schemas/status + - topics/status + - users/status + verbs: + - get + - patch + - update +- 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: + - 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: + - 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/testenv/testenv.go b/operator/internal/testenv/testenv.go index dedd8878c..af84d8a9d 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. @@ -180,13 +191,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), 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/...' \