From c405594562b91e06248ed369c81cd5626f49a01c Mon Sep 17 00:00:00 2001 From: Rafal Korepta Date: Wed, 15 Jan 2025 18:38:21 +0100 Subject: [PATCH] Synchronize operator source code with kubebuilder scafolding project Clean the kustomization. Remove unused kustomization `default-with-metrics`. Remove `kube-rbac-proxy` container from kustomization and helm chart. --- acceptance/features/operator.feature | 26 + acceptance/go.mod | 6 +- acceptance/steps/helpers.go | 122 + acceptance/steps/operator.go | 78 + acceptance/steps/register.go | 7 + charts/operator/deployment.go | 52 +- charts/operator/templates/_deployment.go.tpl | 36 +- .../testdata/template-cases.golden.txtar | 2048 ++--------------- licenses/third_party.md | 15 +- operator/cmd/run/run.go | 138 +- .../default-with-metrics/kustomization.yaml | 44 - .../manager_auth_proxy_patch.yaml | 28 - .../manager_config_patch.yaml | 20 - .../manager_webhook_patch.yaml | 23 - .../webhookcainjection_patch.yaml | 15 - operator/config/default/kustomization.yaml | 298 ++- .../default/manager_auth_proxy_patch.yaml | 28 - .../config/default/manager_config_patch.yaml | 20 - .../config/default/manager_metrics_patch.yaml | 4 + .../config/default/manager_webhook_patch.yaml | 54 +- .../mutatingwebhookcainjection_patch.yaml | 8 - .../validatingwebhookcainjection_patch.yaml | 8 - .../default/webhookcainjection_patch.yaml | 15 - operator/config/e2e-tests/manager.yaml | 3 +- operator/config/manager/manager.yaml | 3 + .../rbac/bases/auth_proxy/kustomization.yaml | 8 +- ...proxy_role.yaml => metrics_auth_role.yaml} | 2 +- ...ng.yaml => metrics_auth_role_binding.yaml} | 4 +- ...service.yaml => metrics_auth_service.yaml} | 3 +- ...sterrole.yaml => metrics_reader_role.yaml} | 0 operator/config/rbac/kustomization.yaml | 2 + .../config/without-webhook/kustomization.yaml | 6 - .../manager_auth_proxy_patch.yaml | 27 - operator/go.mod | 13 + operator/go.sum | 17 +- 35 files changed, 877 insertions(+), 2304 deletions(-) create mode 100644 acceptance/features/operator.feature create mode 100644 acceptance/steps/operator.go delete mode 100644 operator/config/default-with-metrics/kustomization.yaml delete mode 100644 operator/config/default-with-metrics/manager_auth_proxy_patch.yaml delete mode 100644 operator/config/default-with-metrics/manager_config_patch.yaml delete mode 100644 operator/config/default-with-metrics/manager_webhook_patch.yaml delete mode 100644 operator/config/default-with-metrics/webhookcainjection_patch.yaml delete mode 100644 operator/config/default/manager_auth_proxy_patch.yaml delete mode 100644 operator/config/default/manager_config_patch.yaml create mode 100644 operator/config/default/manager_metrics_patch.yaml delete mode 100644 operator/config/default/mutatingwebhookcainjection_patch.yaml delete mode 100644 operator/config/default/validatingwebhookcainjection_patch.yaml delete mode 100644 operator/config/default/webhookcainjection_patch.yaml rename operator/config/rbac/bases/auth_proxy/{auth_proxy_role.yaml => metrics_auth_role.yaml} (90%) rename operator/config/rbac/bases/auth_proxy/{auth_proxy_role_binding.yaml => metrics_auth_role_binding.yaml} (76%) rename operator/config/rbac/bases/auth_proxy/{auth_proxy_service.yaml => metrics_auth_service.yaml} (86%) rename operator/config/rbac/bases/auth_proxy/{auth_proxy_client_clusterrole.yaml => metrics_reader_role.yaml} (100%) delete mode 100644 operator/config/without-webhook/manager_auth_proxy_patch.yaml diff --git a/acceptance/features/operator.feature b/acceptance/features/operator.feature new file mode 100644 index 000000000..2079b0eb6 --- /dev/null +++ b/acceptance/features/operator.feature @@ -0,0 +1,26 @@ +Feature: Metrics endpoint has authentication and authorization + + @skip:gke @skip:aks @skip:eks + Scenario: Reject request without TLS + Given operator is running + Then metrics endpoint should reject http request with status code "400" + + @skip:gke @skip:aks @skip:eks + Scenario: Reject unauthenticated token + Given operator is running + Then metrics endpoint should reject authorization random token request with status code "500" + + @skip:gke @skip:aks @skip:eks + Scenario: Accept request + Given operator is running + When I apply Kubernetes manifest: + """ + apiVersion: v1 + kind: ServiceAccount + metadata: + name: testing + """ + And "testing" service account has bounded "redpanda-operator-metrics-reader" cluster role + Then metrics endpoint should accept https request with "testing" service account token + + diff --git a/acceptance/go.mod b/acceptance/go.mod index ad800adac..bdee685a0 100644 --- a/acceptance/go.mod +++ b/acceptance/go.mod @@ -5,6 +5,7 @@ go 1.23.2 require ( github.com/cucumber/godog v0.14.1 github.com/go-logr/logr v1.4.2 + github.com/prometheus/common v0.55.0 github.com/quasilyte/go-ruleguard/dsl v0.3.22 github.com/redpanda-data/common-go/rpadmin v0.1.13-0.20250109154132-12ac78a58f95 github.com/redpanda-data/redpanda-operator/harpoon v0.0.0-00010101000000-000000000000 @@ -14,8 +15,10 @@ require ( github.com/twmb/franz-go v1.18.0 github.com/twmb/franz-go/pkg/kadm v1.12.0 github.com/twmb/franz-go/pkg/sr v1.2.0 + golang.org/x/exp v0.0.0-20240904232852-e7e105dedf7e k8s.io/api v0.30.3 k8s.io/apimachinery v0.30.3 + k8s.io/client-go v0.30.3 k8s.io/utils v0.0.0-20240711033017-18e509b52bc8 sigs.k8s.io/controller-runtime v0.18.5 ) @@ -166,7 +169,6 @@ require ( github.com/prometheus-operator/prometheus-operator/pkg/apis/monitoring v0.76.2 // indirect github.com/prometheus/client_golang v1.20.0 // indirect github.com/prometheus/client_model v0.6.1 // indirect - github.com/prometheus/common v0.55.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/redpanda-data/common-go/net v0.1.0 // indirect github.com/redpanda-data/console/backend v0.0.0-20240303221210-05d5d9e85f20 // indirect @@ -203,7 +205,6 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect golang.org/x/crypto v0.32.0 // indirect - golang.org/x/exp v0.0.0-20240904232852-e7e105dedf7e // indirect golang.org/x/mod v0.22.0 // indirect golang.org/x/net v0.34.0 // indirect golang.org/x/oauth2 v0.22.0 // indirect @@ -227,7 +228,6 @@ require ( k8s.io/apiextensions-apiserver v0.30.3 // indirect k8s.io/apiserver v0.30.3 // indirect k8s.io/cli-runtime v0.30.3 // indirect - k8s.io/client-go v0.30.3 // indirect k8s.io/component-base v0.30.3 // indirect k8s.io/klog/v2 v2.130.1 // indirect k8s.io/kube-openapi v0.0.0-20240709000822-3c01b740850f // indirect diff --git a/acceptance/steps/helpers.go b/acceptance/steps/helpers.go index 77a93679c..db7a3679d 100644 --- a/acceptance/steps/helpers.go +++ b/acceptance/steps/helpers.go @@ -11,21 +11,30 @@ package steps import ( "context" + "crypto/tls" "encoding/json" "fmt" + "net/http" "slices" + "strconv" "strings" "time" "github.com/cucumber/godog" + "github.com/prometheus/common/expfmt" "github.com/redpanda-data/common-go/rpadmin" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/twmb/franz-go/pkg/kadm" "github.com/twmb/franz-go/pkg/kgo" "github.com/twmb/franz-go/pkg/sr" + "golang.org/x/exp/rand" + appsv1 "k8s.io/api/apps/v1" + authenticationv1 "k8s.io/api/authentication/v1" corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/labels" + "k8s.io/client-go/kubernetes" "k8s.io/utils/ptr" runtimeclient "sigs.k8s.io/controller-runtime/pkg/client" @@ -354,3 +363,116 @@ func checkStableResource(ctx context.Context, t framework.TestingT, o runtimecli }, 30*time.Second, 1*time.Second, "Resource never stabilized") t.Logf("Resource %q has been stable for 5 seconds", key.String()) } + +type operatorClients struct { + client http.Client + operatorPodName string + namespace string + schema string + token string + expectedStatusCode int +} + +func (c *operatorClients) ExpectRequestRejected(ctx context.Context) { + t := framework.T(ctx) + + url := fmt.Sprintf("%s://%s.%s:8443/metrics", c.schema, c.operatorPodName, c.namespace) + + t.Logf("Request %s to operator", url) + req, err := http.NewRequestWithContext(ctx, "GET", url, nil) + require.NoError(t, err) + + req.Header.Set("Authorization", "Bearer "+c.token) + resp, err := c.client.Do(req) + require.NoError(t, err) + + require.Equal(t, c.expectedStatusCode, resp.StatusCode) + + defer resp.Body.Close() +} + +func (c *operatorClients) ExpectCorrectMetricsResponse(ctx context.Context) { + t := framework.T(ctx) + + url := fmt.Sprintf("%s://%s.%s:8443/metrics", c.schema, c.operatorPodName, c.namespace) + + t.Logf("Request %s to operator", url) + req, err := http.NewRequestWithContext(ctx, "GET", url, nil) + require.NoError(t, err) + + req.Header.Set("Authorization", "Bearer "+c.token) + resp, err := c.client.Do(req) + require.NoError(t, err) + + require.Equal(t, http.StatusOK, resp.StatusCode) + + defer resp.Body.Close() + + var parser expfmt.TextParser + _, err = parser.TextToMetricFamilies(resp.Body) + require.NoError(t, err) +} + +func clientsForOperator(ctx context.Context, includeTLS bool, serviceAccountName, expectedStatusCode string) *operatorClients { + t := framework.T(ctx) + + var dep appsv1.Deployment + require.NoError(t, t.Get(ctx, t.ResourceKey("redpanda-operator"), &dep)) + + var podList corev1.PodList + + require.NoError(t, t.List(ctx, &podList, &runtimeclient.ListOptions{ + LabelSelector: labels.SelectorFromSet(dep.Spec.Selector.MatchLabels), + })) + + require.Len(t, podList.Items, 1, "expected 1 pod, got %d", len(podList.Items)) + + var tlsCfg tls.Config + schema := "http" + if includeTLS { + tlsCfg = tls.Config{InsecureSkipVerify: includeTLS} // nolint:gosec + schema = "https" + } + + token := randomString(20) + if serviceAccountName != "" { + cs, err := kubernetes.NewForConfig(t.RestConfig()) + require.NoError(t, err) + tokenResponse, err := cs.CoreV1().ServiceAccounts(t.Namespace()).CreateToken(ctx, serviceAccountName, &authenticationv1.TokenRequest{}, metav1.CreateOptions{}) + require.NoError(t, err) + token = tokenResponse.Status.Token + } + + statusCode := http.StatusOK + if expectedStatusCode != "" { + var err error + statusCode, err = strconv.Atoi(expectedStatusCode) + require.NoError(t, err) + } + + return &operatorClients{ + expectedStatusCode: statusCode, + token: token, + schema: schema, + namespace: t.Namespace(), + operatorPodName: podList.Items[0].Name, + client: http.Client{Transport: &http.Transport{ + TLSClientConfig: &tlsCfg, + DialContext: kube.NewPodDialer(t.RestConfig()).DialContext, + }}, + } +} + +func init() { + rand.Seed(uint64(time.Now().UnixNano())) +} + +var letters = "0123456789abcdefghijklmnopqrstuvwxyz" + +func randomString(n int) string { + b := make([]byte, n) + for i := range b { + b[i] = letters[rand.Intn(len(letters))] + } + return string(b) +} diff --git a/acceptance/steps/operator.go b/acceptance/steps/operator.go new file mode 100644 index 000000000..b32a7e43f --- /dev/null +++ b/acceptance/steps/operator.go @@ -0,0 +1,78 @@ +// 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 steps + +import ( + "context" + + "github.com/stretchr/testify/require" + appsv1 "k8s.io/api/apps/v1" + rbacv1 "k8s.io/api/rbac/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + framework "github.com/redpanda-data/redpanda-operator/harpoon" +) + +func operatorIsRunning(ctx context.Context, t framework.TestingT) { + var dep appsv1.Deployment + require.NoError(t, t.Get(ctx, t.ResourceKey("redpanda-operator"), &dep)) + + // make sure the resource is stable + checkStableResource(ctx, t, &dep) + + require.Equal(t, dep.Status.AvailableReplicas, int32(1)) + require.Equal(t, dep.Status.Replicas, int32(1)) + require.Equal(t, dep.Status.ReadyReplicas, int32(1)) + require.Equal(t, dep.Status.UnavailableReplicas, int32(0)) +} + +func requestMetricsEndpointPlainHTTP(ctx context.Context, statusCode string) { + clientsForOperator(ctx, false, "", statusCode).ExpectRequestRejected(ctx) +} + +func requestMetricsEndpointWithTLSAndRandomToken(ctx context.Context, statusCode string) { + clientsForOperator(ctx, true, "", statusCode).ExpectRequestRejected(ctx) +} + +func acceptServiceAccountMetricsRequest(ctx context.Context, serviceAccountName string) { + clientsForOperator(ctx, true, serviceAccountName, "").ExpectCorrectMetricsResponse(ctx) +} + +func createClusterRoleBinding(ctx context.Context, serviceAccountName, clusterRoleName string) { + t := framework.T(ctx) + + require.NoError(t, t.Create(ctx, &rbacv1.ClusterRoleBinding{ + ObjectMeta: metav1.ObjectMeta{ + Name: serviceAccountName, + Namespace: t.Namespace(), + }, + RoleRef: rbacv1.RoleRef{ + APIGroup: rbacv1.GroupName, + Kind: "ClusterRole", + Name: clusterRoleName, + }, + Subjects: []rbacv1.Subject{ + { + Kind: "ServiceAccount", + Name: serviceAccountName, + Namespace: t.Namespace(), + }, + }, + })) + + t.Cleanup(func(ctx context.Context) { + require.NoError(t, t.Delete(ctx, &rbacv1.ClusterRoleBinding{ + ObjectMeta: metav1.ObjectMeta{ + Name: serviceAccountName, + Namespace: t.Namespace(), + }, + })) + }) +} diff --git a/acceptance/steps/register.go b/acceptance/steps/register.go index 9e79dfc47..861e268d8 100644 --- a/acceptance/steps/register.go +++ b/acceptance/steps/register.go @@ -41,4 +41,11 @@ func init() { framework.RegisterStep(`^"([^"]*)" should exist and be able to authenticate to the "([^"]*)" cluster$`, shouldExistAndBeAbleToAuthenticateToTheCluster) framework.RegisterStep(`^"([^"]*)" should be able to authenticate to the "([^"]*)" cluster with password "([^"]*)" and mechanism "([^"]*)"$`, shouldBeAbleToAuthenticateToTheClusterWithPasswordAndMechanism) framework.RegisterStep(`^there should be ACLs in the cluster "([^"]*)" for user "([^"]*)"$`, thereShouldBeACLsInTheClusterForUser) + + // Operator scenario steps + framework.RegisterStep(`^operator is running$`, operatorIsRunning) + framework.RegisterStep(`^metrics endpoint should reject http request with status code "([^"]*)"$`, requestMetricsEndpointPlainHTTP) + framework.RegisterStep(`^metrics endpoint should reject authorization random token request with status code "([^"]*)"$`, requestMetricsEndpointWithTLSAndRandomToken) + framework.RegisterStep(`^"([^"]*)" service account has bounded "([^"]*)" cluster role$`, createClusterRoleBinding) + framework.RegisterStep(`^metrics endpoint should accept https request with "([^"]*)" service account token$`, acceptServiceAccountMetricsRequest) } diff --git a/charts/operator/deployment.go b/charts/operator/deployment.go index dad7396a2..dff9521cc 100644 --- a/charts/operator/deployment.go +++ b/charts/operator/deployment.go @@ -119,48 +119,9 @@ func operatorContainers(dot *helmette.Dot, podTerminationGracePeriodSeconds *int ReadinessProbe: readinessProbe(dot, podTerminationGracePeriodSeconds), Resources: values.Resources, }, - { - Name: "kube-rbac-proxy", - Args: []string{ - "--secure-listen-address=0.0.0.0:8443", - "--upstream=http://127.0.0.1:8080/", - "--logtostderr=true", - fmt.Sprintf("--v=%d", values.KubeRBACProxy.LogLevel), - }, - Image: fmt.Sprintf("%s:%s", values.KubeRBACProxy.Image.Repository, *values.KubeRBACProxy.Image.Tag), - ImagePullPolicy: values.KubeRBACProxy.Image.PullPolicy, - Ports: []corev1.ContainerPort{ - { - ContainerPort: 8443, - Name: "https", - }, - }, - VolumeMounts: kubeRBACProxyVolumeMounts(dot), - }, } } -func kubeRBACProxyVolumeMounts(dot *helmette.Dot) []corev1.VolumeMount { - values := helmette.Unwrap[Values](dot.Values) - - if !values.ServiceAccount.Create { - return nil - } - - mountName := ServiceAccountVolumeName - for _, vol := range operatorPodVolumes(dot) { - if strings.HasPrefix(ServiceAccountVolumeName+"-", vol.Name) { - mountName = vol.Name - } - } - - return []corev1.VolumeMount{{ - Name: mountName, - ReadOnly: true, - MountPath: DefaultAPITokenMountPath, - }} -} - func livenessProbe(dot *helmette.Dot, podTerminationGracePeriodSeconds *int64) *corev1.Probe { values := helmette.Unwrap[Values](dot.Values) @@ -250,7 +211,7 @@ func operatorPodVolumes(dot *helmette.Dot) []corev1.Volume { vol = append(vol, kubeTokenAPIVolume(ServiceAccountVolumeName)) } - if !values.Webhook.Enabled { + if !isWebhookEnabled(dot) { return vol } @@ -337,7 +298,7 @@ func operatorPodVolumesMounts(dot *helmette.Dot) []corev1.VolumeMount { }) } - if !values.Webhook.Enabled { + if !isWebhookEnabled(dot) { return volMount } @@ -355,11 +316,18 @@ func operatorArguments(dot *helmette.Dot) []string { args := []string{ "--health-probe-bind-address=:8081", - "--metrics-bind-address=127.0.0.1:8080", + "--metrics-bind-address=:8443", "--leader-elect", fmt.Sprintf("--webhook-enabled=%t", isWebhookEnabled(dot)), } + if isWebhookEnabled(dot) { + args = append(args, + "--webhook-enabled=true", + "--webhook-cert-path=/tmp/k8s-webhook-server/serving-certs", + ) + } + if values.Scope == Namespace { args = append(args, fmt.Sprintf("--namespace=%s", dot.Release.Namespace), diff --git a/charts/operator/templates/_deployment.go.tpl b/charts/operator/templates/_deployment.go.tpl index 6801f31e0..fd937efc7 100644 --- a/charts/operator/templates/_deployment.go.tpl +++ b/charts/operator/templates/_deployment.go.tpl @@ -22,32 +22,7 @@ {{- $_is_returning := false -}} {{- $values := $dot.Values.AsMap -}} {{- $_is_returning = true -}} -{{- (dict "r" (list (mustMergeOverwrite (dict "name" "" "resources" (dict ) ) (dict "name" "manager" "image" (get (fromJson (include "operator.containerImage" (dict "a" (list $dot) ))) "r") "imagePullPolicy" $values.image.pullPolicy "command" (list "/manager") "args" (get (fromJson (include "operator.operatorArguments" (dict "a" (list $dot) ))) "r") "securityContext" (mustMergeOverwrite (dict ) (dict "allowPrivilegeEscalation" false )) "ports" (list (mustMergeOverwrite (dict "containerPort" 0 ) (dict "name" "webhook-server" "containerPort" (9443 | int) "protocol" "TCP" ))) "volumeMounts" (get (fromJson (include "operator.operatorPodVolumesMounts" (dict "a" (list $dot) ))) "r") "livenessProbe" (get (fromJson (include "operator.livenessProbe" (dict "a" (list $dot $podTerminationGracePeriodSeconds) ))) "r") "readinessProbe" (get (fromJson (include "operator.readinessProbe" (dict "a" (list $dot $podTerminationGracePeriodSeconds) ))) "r") "resources" $values.resources )) (mustMergeOverwrite (dict "name" "" "resources" (dict ) ) (dict "name" "kube-rbac-proxy" "args" (list "--secure-listen-address=0.0.0.0:8443" "--upstream=http://127.0.0.1:8080/" "--logtostderr=true" (printf "--v=%d" ($values.kubeRbacProxy.logLevel | int))) "image" (printf "%s:%s" $values.kubeRbacProxy.image.repository $values.kubeRbacProxy.image.tag) "imagePullPolicy" $values.kubeRbacProxy.image.pullPolicy "ports" (list (mustMergeOverwrite (dict "containerPort" 0 ) (dict "containerPort" (8443 | int) "name" "https" ))) "volumeMounts" (get (fromJson (include "operator.kubeRBACProxyVolumeMounts" (dict "a" (list $dot) ))) "r") )))) | toJson -}} -{{- break -}} -{{- end -}} -{{- end -}} - -{{- define "operator.kubeRBACProxyVolumeMounts" -}} -{{- $dot := (index .a 0) -}} -{{- range $_ := (list 1) -}} -{{- $_is_returning := false -}} -{{- $values := $dot.Values.AsMap -}} -{{- if (not $values.serviceAccount.create) -}} -{{- $_is_returning = true -}} -{{- (dict "r" (coalesce nil)) | toJson -}} -{{- break -}} -{{- end -}} -{{- $mountName := "kube-api-access" -}} -{{- range $_, $vol := (get (fromJson (include "operator.operatorPodVolumes" (dict "a" (list $dot) ))) "r") -}} -{{- if (hasPrefix $vol.name (printf "%s%s" "kube-api-access" "-")) -}} -{{- $mountName = $vol.name -}} -{{- end -}} -{{- end -}} -{{- if $_is_returning -}} -{{- break -}} -{{- end -}} -{{- $_is_returning = true -}} -{{- (dict "r" (list (mustMergeOverwrite (dict "name" "" "mountPath" "" ) (dict "name" $mountName "readOnly" true "mountPath" "/var/run/secrets/kubernetes.io/serviceaccount" )))) | toJson -}} +{{- (dict "r" (list (mustMergeOverwrite (dict "name" "" "resources" (dict ) ) (dict "name" "manager" "image" (get (fromJson (include "operator.containerImage" (dict "a" (list $dot) ))) "r") "imagePullPolicy" $values.image.pullPolicy "command" (list "/manager") "args" (get (fromJson (include "operator.operatorArguments" (dict "a" (list $dot) ))) "r") "securityContext" (mustMergeOverwrite (dict ) (dict "allowPrivilegeEscalation" false )) "ports" (list (mustMergeOverwrite (dict "containerPort" 0 ) (dict "name" "webhook-server" "containerPort" (9443 | int) "protocol" "TCP" ))) "volumeMounts" (get (fromJson (include "operator.operatorPodVolumesMounts" (dict "a" (list $dot) ))) "r") "livenessProbe" (get (fromJson (include "operator.livenessProbe" (dict "a" (list $dot $podTerminationGracePeriodSeconds) ))) "r") "readinessProbe" (get (fromJson (include "operator.readinessProbe" (dict "a" (list $dot $podTerminationGracePeriodSeconds) ))) "r") "resources" $values.resources )))) | toJson -}} {{- break -}} {{- end -}} {{- end -}} @@ -121,7 +96,7 @@ {{- if $values.serviceAccount.create -}} {{- $vol = (concat (default (list ) $vol) (list (get (fromJson (include "operator.kubeTokenAPIVolume" (dict "a" (list "kube-api-access") ))) "r"))) -}} {{- end -}} -{{- if (not $values.webhook.enabled) -}} +{{- if (not (get (fromJson (include "operator.isWebhookEnabled" (dict "a" (list $dot) ))) "r")) -}} {{- $_is_returning = true -}} {{- (dict "r" $vol) | toJson -}} {{- break -}} @@ -161,7 +136,7 @@ {{- end -}} {{- $volMount = (concat (default (list ) $volMount) (list (mustMergeOverwrite (dict "name" "" "mountPath" "" ) (dict "name" $mountName "readOnly" true "mountPath" "/var/run/secrets/kubernetes.io/serviceaccount" )))) -}} {{- end -}} -{{- if (not $values.webhook.enabled) -}} +{{- if (not (get (fromJson (include "operator.isWebhookEnabled" (dict "a" (list $dot) ))) "r")) -}} {{- $_is_returning = true -}} {{- (dict "r" $volMount) | toJson -}} {{- break -}} @@ -178,7 +153,10 @@ {{- range $_ := (list 1) -}} {{- $_is_returning := false -}} {{- $values := $dot.Values.AsMap -}} -{{- $args := (list "--health-probe-bind-address=:8081" "--metrics-bind-address=127.0.0.1:8080" "--leader-elect" (printf "--webhook-enabled=%t" (get (fromJson (include "operator.isWebhookEnabled" (dict "a" (list $dot) ))) "r"))) -}} +{{- $args := (list "--health-probe-bind-address=:8081" "--metrics-bind-address=:8443" "--leader-elect" (printf "--webhook-enabled=%t" (get (fromJson (include "operator.isWebhookEnabled" (dict "a" (list $dot) ))) "r"))) -}} +{{- if (get (fromJson (include "operator.isWebhookEnabled" (dict "a" (list $dot) ))) "r") -}} +{{- $args = (concat (default (list ) $args) (list "--webhook-enabled=true" "--webhook-cert-path=/tmp/k8s-webhook-server/serving-certs")) -}} +{{- end -}} {{- if (eq $values.scope "Namespace") -}} {{- $args = (concat (default (list ) $args) (list (printf "--namespace=%s" $dot.Release.Namespace) (printf "--log-level=%s" $values.logLevel))) -}} {{- end -}} diff --git a/charts/operator/testdata/template-cases.golden.txtar b/charts/operator/testdata/template-cases.golden.txtar index d3345f7de..3ae247864 100644 --- a/charts/operator/testdata/template-cases.golden.txtar +++ b/charts/operator/testdata/template-cases.golden.txtar @@ -810,7 +810,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -847,23 +847,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -1962,7 +1945,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -2005,23 +1988,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: ʣĔ湲 enableServiceLinks: true ephemeralContainers: null @@ -3165,7 +3131,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -3210,23 +3176,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -4367,7 +4316,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -4410,23 +4359,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: - name: fAr @@ -5533,7 +5465,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -5584,23 +5516,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -6726,7 +6641,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -6769,23 +6684,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=500 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -7929,7 +7827,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -7973,23 +7871,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=445 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: 蚱跆ni仩(y·!Sžǩ ephemeralContainers: null hostAliases: @@ -9119,7 +9000,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -9162,23 +9043,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -9608,7 +9472,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -9651,23 +9515,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -10835,7 +10682,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -10878,23 +10725,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -12018,7 +11848,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -12067,23 +11897,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -13312,7 +13125,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -13356,23 +13169,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=310 - env: [] - image: :qDuS - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: 橍Ǎ!ɻ麳蠊寢ĥ)×Xʟ羄鳤 enableServiceLinks: true ephemeralContainers: null @@ -14533,7 +14329,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -14577,23 +14373,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -15686,7 +15465,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -15729,23 +15508,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=339 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: Ƣ熨f ephemeralContainers: null hostAliases: @@ -16923,7 +16685,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -16972,23 +16734,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=109 - env: [] - image: BEuqFP:fel7iI4 - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -17412,7 +17157,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -17461,23 +17206,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: Êĕƹ糁Ř舛ě勒 ephemeralContainers: null hostNetwork: true @@ -18623,7 +18351,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -18669,20 +18397,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=263 - env: [] - image: wMt:KUgVbNQmz - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsPolicy: 禉ȎÝ汱 ephemeralContainers: null hostAliases: @@ -19168,7 +18882,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -19219,23 +18933,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=277 - env: [] - image: DiySuZhqL:8Z - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: 峻Ő輇 ephemeralContainers: null hostNetwork: true @@ -20473,7 +20170,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -20516,23 +20213,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - z4 @@ -21805,7 +21485,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -21848,23 +21528,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: - name: P0TDZL8he @@ -23060,7 +22723,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -23103,23 +22766,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=229 - env: [] - image: WwSCIw:ABSHPzN - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -26419,7 +26065,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -26465,20 +26111,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=362 - env: [] - image: PCYk:AnH7 - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - QXyBi4x @@ -33778,7 +33410,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -33829,23 +33461,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=145 - env: [] - image: fLJ:aG - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -39253,7 +38868,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -39302,23 +38917,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: searches: - P5YqgV3Moz @@ -40793,7 +40391,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -40833,20 +40431,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=33 - env: [] - image: 3vVEuSY:LX - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - uFFDQ @@ -44221,7 +43805,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -44271,23 +43855,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=245 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: - name: 2di49JY @@ -51043,7 +50610,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -51092,23 +50659,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=307 - env: [] - image: qn:kT - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: options: - name: Wf0HvUtqF @@ -58545,7 +58095,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -58595,23 +58145,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=291 - env: [] - image: lOU:9Z - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - tZ @@ -66189,7 +65722,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -66238,23 +65771,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=109 - env: [] - image: QmP:LRmkId - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - oW5sc9 @@ -72941,7 +72457,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -72987,20 +72503,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=367 - env: [] - image: 5Aj:Et1 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - 9uF4n @@ -78422,7 +77924,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -78468,20 +77970,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=125 - env: [] - image: Q9SKJed:Pm5 - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - bnxKu @@ -81342,7 +80830,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -81391,23 +80879,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=64 - env: [] - image: :WEMN - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - a @@ -89079,7 +88550,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -89129,23 +88600,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=330 - env: [] - image: WPPKhIlvmxeug:VMn7v9B - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - CDHKmOg2 @@ -95857,7 +95311,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -95906,23 +95360,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=68 - env: [] - image: 8eQInSt:rhfUTa - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - KryE @@ -103570,7 +103007,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -103617,20 +103054,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=64 - env: [] - image: qPSB7h6jS:qr - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - Nr @@ -108375,7 +107798,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -108424,23 +107847,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=384 - env: [] - image: TY:Xk - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: options: - name: 6Fq @@ -109153,7 +108559,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -109202,23 +108608,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=7 - env: [] - image: eF:Tnx - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - E4iX @@ -110029,7 +109418,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -110076,20 +109465,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=168 - env: [] - image: ze9Osk6tOx2:nWuywKVlk - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: searches: - "" @@ -112877,7 +112252,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -112924,20 +112299,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=186 - env: [] - image: MLG8:Fj4j - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - YISghsOm @@ -118496,7 +117857,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -118547,23 +117908,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=278 - env: [] - image: 4jdbu:THCADXet - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - KNDuF @@ -121348,7 +120692,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -121396,20 +120740,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=186 - env: [] - image: R:B9 - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: searches: - 1M9d @@ -128087,7 +127417,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -128138,23 +127468,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=294 - env: [] - image: amuilGh:LzDC4Xxr - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - 1n @@ -133708,7 +133021,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -133754,20 +133067,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=62 - env: [] - image: mZ:cpGiTs039e - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - "34" @@ -137436,7 +136735,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -137483,20 +136782,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=464 - env: [] - image: DydZz:IJ - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - g9 @@ -142272,7 +141557,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -142323,23 +141608,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=254 - env: [] - image: :8ET6P - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - Xm @@ -144003,7 +143271,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -144049,20 +143317,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=427 - env: [] - image: PS9rQIXzEM:huSub - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - fjX4eE @@ -144909,7 +144163,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -144960,23 +144214,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=51 - env: [] - image: Oa40e8D:9U5s2 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - uu5wuW @@ -147762,7 +146999,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -147811,23 +147048,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=255 - env: [] - image: gGB:NOt - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - MlS @@ -153441,7 +152661,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -153487,20 +152707,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=339 - env: [] - image: 7OdjR:C7SY - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - XQQvkKFB7z @@ -155023,7 +154229,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -155069,20 +154275,6 @@ spec: securityContext: allowPrivilegeEscalation: false volumeMounts: [] - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=230 - env: [] - image: I6mqY:z - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - OKrRLUXo1z @@ -158641,7 +157833,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -158691,23 +157883,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=487 - env: [] - image: SJbGg9G:TerHhXnk - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: options: - name: HHHVB5Q @@ -163803,9 +162978,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -163847,23 +163024,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=388 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -164796,9 +163956,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - l4dHBtic0cX0cs7 - "" command: @@ -164847,23 +164009,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: xť]\ĕȨ宂鄈 ephemeralContainers: null hostIPC: true @@ -165802,9 +164947,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -165846,23 +164993,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: {} dnsPolicy: 刲ʑ|ʖl狉J enableServiceLinks: true @@ -166828,9 +165958,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -166878,23 +166010,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -167861,9 +166976,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -167905,23 +167022,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: - name: eNCbsc @@ -168881,9 +167981,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - D4rks - p7p - 1kSb @@ -168927,23 +168029,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -169405,9 +168490,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -169455,23 +168542,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: 旃9乚ɸ蓔2嘷! ephemeralContainers: null hostPID: true @@ -171995,9 +171065,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - JOd - JV6w command: @@ -172046,23 +171118,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=177 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: {} dnsPolicy: ±訇邓/釉Ï捓ʐ ephemeralContainers: null @@ -177130,9 +176185,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -177180,23 +176237,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsPolicy: 荧ʇNJ亍/-ʇƬQ{Ǯ/ ephemeralContainers: null hostAliases: @@ -178158,9 +177198,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -178202,23 +177244,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=456 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -181449,9 +180474,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - 8TZ - c - cpUq @@ -181501,23 +180528,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=232 - env: [] - image: VXepc3:zN - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: - name: Qzv @@ -188596,9 +187606,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -188646,23 +187658,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -195948,9 +194943,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -195998,23 +194995,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=144 - env: [] - image: RE:re - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: - name: GI3C2 @@ -201096,9 +200076,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - r4LII - m5622dBz6A command: @@ -201144,20 +200126,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=190 - env: [] - image: yP:KhfSU - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - IVuNkr4z @@ -204296,9 +203264,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - "" - QL9X command: @@ -204347,23 +203317,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=183 - env: [] - image: ntF:AnS1f - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: - name: Qi @@ -208958,9 +207911,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - Q9 - JLCZc - c @@ -209007,20 +207962,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=288 - env: [] - image: j3r:XPG6J - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - QiG @@ -211678,9 +210619,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -211728,23 +210671,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=22 - env: [] - image: J67dgqgI:k4c - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: - name: T @@ -216887,9 +215813,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -216934,20 +215862,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=63 - env: [] - image: q1szt2:tmKv - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - mXmDRel @@ -217607,9 +216521,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - dWzxr3xXRM - vEF - DAM @@ -217656,20 +216572,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=43 - env: [] - image: e04Gnjsm:6lY4KM - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - iJM @@ -218348,9 +217250,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - oxdTcKZmy5 command: - /manager @@ -218398,23 +217302,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=123 - env: [] - image: DCg5AwUni:xIDQv4 - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - w @@ -221827,9 +220714,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - ReK6Upw - 7Pu0AA - 0yNEqlggmA @@ -221876,20 +220765,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=239 - env: [] - image: PnVf8:2 - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - D8E @@ -227257,9 +226132,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - LlHz1 - 4qYzSV command: @@ -227305,20 +226182,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=420 - env: [] - image: VNg6D2sx8T:9e - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: searches: - 0IRYIXHfLfR @@ -228171,9 +227034,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - zTsg - Sv - 9M23O7X @@ -228220,20 +227085,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=240 - env: [] - image: FjPCP:sTRN33j - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - "" @@ -231106,9 +229957,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - 5orCeu - v7kr command: @@ -231157,23 +230010,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=67 - env: [] - image: W7Gm:CwGlxr - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - s7h5u7RQdK @@ -236005,9 +234841,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - "n" - H7B3A1m command: @@ -236053,20 +234891,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=392 - env: [] - image: c:U - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - bga8iMz @@ -237265,9 +236089,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - eyxGhI - 9YL - 6HEr @@ -237317,23 +236143,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=139 - env: [] - image: 582AO:S8bAQfhT9M - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - 4mEBMGUN @@ -240956,9 +239765,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -241003,20 +239814,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=152 - env: [] - image: oULd:yPOGFghUo - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - 9vmm6l @@ -246383,9 +245180,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -246433,23 +245232,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=403 - env: [] - image: sftJ:JajgrDP - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - EC @@ -247753,9 +246535,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - Sb4JTueXASpBtQ - og - GaoUIcOT @@ -247805,23 +246589,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=429 - env: [] - image: yW4j:6qlKjn - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - ujfhX4IQ @@ -250687,9 +249454,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -250737,23 +249506,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=295 - env: [] - image: omQS:o9JkuA - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - RGKmo @@ -256102,9 +254854,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -256152,23 +254906,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=78 - env: [] - image: NA:VBjy5N - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - dVU @@ -259661,9 +258398,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - Gc3q48w - JNFXPzkrj - hYzBm @@ -259713,23 +258452,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=292 - env: [] - image: h1:d0sg - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - "" @@ -264431,9 +263153,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -264478,20 +263202,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=296 - env: [] - image: Zt:lZcBojk - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - h9KdM @@ -267893,9 +266603,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - QpxE2 command: - /manager @@ -267940,20 +266652,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=47 - env: [] - image: u:e8Pqlld - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - dSXcDS @@ -273371,9 +272069,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - bv4LbFjFG - X5vvak7u command: @@ -273422,23 +272122,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=267 - env: [] - image: RPN0vgCLR:Tf5R - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: searches: - H1Ls9 @@ -274694,9 +273377,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - amoUVan command: - /manager @@ -274741,20 +273426,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=356 - env: [] - image: Ng:5qq - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - puVsXD @@ -275509,9 +274180,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -275559,23 +274232,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=27 - env: [] - image: B:iTS - imagePullPolicy: Always - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - nex087njHP @@ -276836,9 +275492,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -276883,20 +275541,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=386 - env: [] - image: WVH:EUjjP - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - HE4 @@ -278305,9 +276949,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -278352,20 +276998,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=445 - env: [] - image: UZO6:OZLi6ZK - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - HrhMd @@ -279139,9 +277771,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -279189,23 +277823,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=405 - env: [] - image: 4ZneBG:wMF1 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - 41sorB33 @@ -280518,9 +279135,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -280568,23 +279187,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=477 - env: [] - image: L1M6F:QOyMi - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - FNqDGX @@ -284126,9 +282728,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - oMtn - Q - 3z6Z0vrvjWuC @@ -284178,23 +282782,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=489 - env: [] - image: zkjPJE:udvp - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - "" @@ -289100,9 +287687,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - IoP - oEPSF - w659VcIIlO @@ -289152,23 +287741,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=226 - env: [] - image: nYU:3WGWloTA - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: options: - name: rHF @@ -290471,9 +289043,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - "" - 2BwL4Tf6d - LY5cEH @@ -290523,23 +289097,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=231 - env: [] - image: Mzt:YjH8Y - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - 0l7aqP @@ -293962,9 +292519,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - 7G3Tgu - Oh command: @@ -294010,20 +292569,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=153 - env: [] - image: UETwk:L - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - lFF4bR @@ -301422,9 +299967,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -301472,23 +300019,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=125 - env: [] - image: mbREx:jpy - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: options: - name: kUN @@ -308341,9 +306871,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -308391,23 +306923,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=353 - env: [] - image: H:D - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - o2YA @@ -315866,9 +314381,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - l9a - Arx - bLHpq0J3 @@ -315915,20 +314432,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=43 - env: [] - image: O50k5k:kkIJqNV - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: null dnsConfig: nameservers: - vD1Q @@ -320804,9 +319307,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - P - WM7Be - JLya @@ -320856,23 +319361,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=372 - env: [] - image: BCQC:I7 - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - qpD5w @@ -324358,9 +322846,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -324408,23 +322898,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=432 - env: [] - image: x:fwIj - imagePullPolicy: Never - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true dnsConfig: nameservers: - 2CpyVbnvo @@ -329691,7 +328164,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -329726,23 +328199,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -330834,7 +329290,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -330868,23 +329324,6 @@ spec: volumeMounts: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access-overwrite - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -331994,7 +330433,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -332029,23 +330468,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -333137,7 +331559,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -333172,23 +331594,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -334106,9 +332511,11 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=true + - --webhook-enabled=true + - --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs - --configurator-tag=v2.3.6-24.3.3 command: - /manager @@ -334142,23 +332549,6 @@ spec: - mountPath: /tmp/k8s-webhook-server/serving-certs name: cert readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] @@ -335254,7 +333644,7 @@ spec: containers: - args: - --health-probe-bind-address=:8081 - - --metrics-bind-address=127.0.0.1:8080 + - --metrics-bind-address=:8443 - --leader-elect - --webhook-enabled=false - --namespace=default @@ -335290,26 +333680,6 @@ spec: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount name: kube-api-access readOnly: true - - args: - - --secure-listen-address=0.0.0.0:8443 - - --upstream=http://127.0.0.1:8080/ - - --logtostderr=true - - --v=10 - env: [] - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.0 - imagePullPolicy: IfNotPresent - name: kube-rbac-proxy - ports: - - containerPort: 8443 - name: https - resources: {} - securityContext: - allowPrivilegeEscalation: false - runAsNonRoot: true - volumeMounts: - - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access - readOnly: true ephemeralContainers: null imagePullSecrets: [] initContainers: [] diff --git a/licenses/third_party.md b/licenses/third_party.md index 060631a99..e5261d2de 100644 --- a/licenses/third_party.md +++ b/licenses/third_party.md @@ -58,6 +58,7 @@ run `task generate:third-party-licenses-list` | github.com/alibabacloud-go/tea-utils/service | [Apache-2.0](https://github.com/alibabacloud-go/tea-utils/blob/v1.4.5/LICENSE) | | github.com/alibabacloud-go/tea-xml/service | [Apache-2.0](https://github.com/alibabacloud-go/tea-xml/blob/v1.1.3/LICENSE) | | github.com/aliyun/credentials-go/credentials | [Apache-2.0](https://github.com/aliyun/credentials-go/blob/v1.3.2/LICENSE) | +| github.com/antlr/antlr4/runtime/Go/antlr/v4 | [BSD-3-Clause](https://github.com/antlr/antlr4/blob/8188dc5388df/runtime/Go/antlr/v4/LICENSE) | | github.com/asaskevich/govalidator | [MIT](https://github.com/asaskevich/govalidator/blob/a9d515a09cc2/LICENSE) | | github.com/aws/aws-sdk-go-v2 | [Apache-2.0](https://github.com/aws/aws-sdk-go-v2/blob/v1.24.0/LICENSE.txt) | | github.com/aws/aws-sdk-go-v2/config | [Apache-2.0](https://github.com/aws/aws-sdk-go-v2/blob/config/v1.26.1/config/LICENSE.txt) | @@ -80,7 +81,9 @@ run `task generate:third-party-licenses-list` | github.com/bahlo/generic-list-go | [BSD-3-Clause](https://github.com/bahlo/generic-list-go/blob/v0.2.0/LICENSE) | | github.com/beorn7/perks/quantile | [MIT](https://github.com/beorn7/perks/blob/v1.0.1/LICENSE) | | github.com/blang/semver | [MIT](https://github.com/blang/semver/blob/v3.5.1/LICENSE) | +| github.com/blang/semver/v4 | [MIT](https://github.com/blang/semver/blob/v4.0.0/v4/LICENSE) | | github.com/buger/jsonparser | [MIT](https://github.com/buger/jsonparser/blob/v1.1.1/LICENSE) | +| github.com/cenkalti/backoff/v4 | [MIT](https://github.com/cenkalti/backoff/blob/v4.3.0/LICENSE) | | github.com/cert-manager/cert-manager/pkg/apis | [Apache-2.0](https://github.com/cert-manager/cert-manager/blob/v1.14.5/LICENSE) | | github.com/cespare/xxhash/v2 | [MIT](https://github.com/cespare/xxhash/blob/v2.3.0/LICENSE.txt) | | github.com/chai2010/gettext-go | [BSD-3-Clause](https://github.com/chai2010/gettext-go/blob/v1.0.2/LICENSE) | @@ -182,6 +185,7 @@ run `task generate:third-party-licenses-list` | github.com/gonvenience/wrap | [MIT](https://github.com/gonvenience/wrap/blob/v1.2.0/LICENSE) | | github.com/gonvenience/ytbx | [MIT](https://github.com/gonvenience/ytbx/blob/v1.4.4/LICENSE) | | github.com/google/btree | [Apache-2.0](https://github.com/google/btree/blob/v1.1.2/LICENSE) | +| github.com/google/cel-go | [Apache-2.0](https://github.com/google/cel-go/blob/v0.17.8/LICENSE) | | github.com/google/certificate-transparency-go | [Apache-2.0](https://github.com/google/certificate-transparency-go/blob/v1.1.7/LICENSE) | | github.com/google/gnostic-models | [Apache-2.0](https://github.com/google/gnostic-models/blob/c7be7c783f49/LICENSE) | | github.com/google/go-cmp/cmp | [BSD-3-Clause](https://github.com/google/go-cmp/blob/v0.6.0/LICENSE) | @@ -201,6 +205,7 @@ run `task generate:third-party-licenses-list` | github.com/gosuri/uitable | [MIT](https://github.com/gosuri/uitable/blob/v0.0.4/LICENSE) | | github.com/gosuri/uitable/util/wordwrap | [MIT](https://github.com/gosuri/uitable/blob/v0.0.4/util/wordwrap/LICENSE.md) | | github.com/gregjones/httpcache | [MIT](https://github.com/gregjones/httpcache/blob/901d90724c79/LICENSE.txt) | +| github.com/grpc-ecosystem/grpc-gateway/v2 | [BSD-3-Clause](https://github.com/grpc-ecosystem/grpc-gateway/blob/v2.19.1/LICENSE) | | github.com/hashicorp/errwrap | [MPL-2.0](https://github.com/hashicorp/errwrap/blob/v1.1.0/LICENSE) | | github.com/hashicorp/go-cleanhttp | [MPL-2.0](https://github.com/hashicorp/go-cleanhttp/blob/v0.5.2/LICENSE) | | github.com/hashicorp/go-multierror | [MPL-2.0](https://github.com/hashicorp/go-multierror/blob/v1.1.1/LICENSE) | @@ -324,6 +329,7 @@ run `task generate:third-party-licenses-list` | github.com/spf13/cobra | [Apache-2.0](https://github.com/spf13/cobra/blob/v1.8.1/LICENSE.txt) | | github.com/spf13/pflag | [BSD-3-Clause](https://github.com/spf13/pflag/blob/v1.0.5/LICENSE) | | github.com/spf13/viper | [MIT](https://github.com/spf13/viper/blob/v1.18.1/LICENSE) | +| github.com/stoewer/go-strcase | [MIT](https://github.com/stoewer/go-strcase/blob/v1.3.0/LICENSE) | | github.com/stretchr/testify | [MIT](https://github.com/stretchr/testify/blob/v1.10.0/LICENSE) | | github.com/subosito/gotenv | [MIT](https://github.com/subosito/gotenv/blob/v1.6.0/LICENSE) | | github.com/syndtr/goleveldb/leveldb | [BSD-2-Clause](https://github.com/syndtr/goleveldb/blob/126854af5e6d/LICENSE) | @@ -358,8 +364,12 @@ run `task generate:third-party-licenses-list` | go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-go-contrib/blob/instrumentation/google.golang.org/grpc/otelgrpc/v0.49.0/instrumentation/google.golang.org/grpc/otelgrpc/LICENSE) | | go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-go-contrib/blob/instrumentation/net/http/otelhttp/v0.53.0/instrumentation/net/http/otelhttp/LICENSE) | | go.opentelemetry.io/otel | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-go/blob/v1.28.0/LICENSE) | +| go.opentelemetry.io/otel/exporters/otlp/otlptrace | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-go/blob/exporters/otlp/otlptrace/v1.24.0/exporters/otlp/otlptrace/LICENSE) | +| go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-go/blob/exporters/otlp/otlptrace/otlptracegrpc/v1.21.0/exporters/otlp/otlptrace/otlptracegrpc/LICENSE) | | go.opentelemetry.io/otel/metric | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-go/blob/metric/v1.28.0/metric/LICENSE) | +| go.opentelemetry.io/otel/sdk | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-go/blob/sdk/v1.28.0/sdk/LICENSE) | | go.opentelemetry.io/otel/trace | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-go/blob/trace/v1.28.0/trace/LICENSE) | +| go.opentelemetry.io/proto/otlp | [Apache-2.0](https://github.com/open-telemetry/opentelemetry-proto-go/blob/otlp/v1.1.0/otlp/LICENSE) | | go.starlark.net | [BSD-3-Clause](https://github.com/google/starlark-go/blob/90ade8b19d09/LICENSE) | | go.step.sm/crypto | [Apache-2.0](https://github.com/smallstep/crypto/blob/v0.40.0/LICENSE) | | go.step.sm/crypto/internal/bcrypt_pbkdf | [BSD-2-Clause](https://github.com/smallstep/crypto/blob/v0.40.0/internal/bcrypt_pbkdf/LICENSE) | @@ -397,7 +407,7 @@ run `task generate:third-party-licenses-list` | k8s.io/apiextensions-apiserver/pkg | [Apache-2.0](https://github.com/kubernetes/apiextensions-apiserver/blob/v0.30.3/LICENSE) | | k8s.io/apimachinery/pkg | [Apache-2.0](https://github.com/kubernetes/apimachinery/blob/v0.30.3/LICENSE) | | k8s.io/apimachinery/third_party/forked/golang | [BSD-3-Clause](https://github.com/kubernetes/apimachinery/blob/v0.30.3/third_party/forked/golang/LICENSE) | -| k8s.io/apiserver/pkg/endpoints/deprecation | [Apache-2.0](https://github.com/kubernetes/apiserver/blob/v0.30.3/LICENSE) | +| k8s.io/apiserver | [Apache-2.0](https://github.com/kubernetes/apiserver/blob/v0.30.3/LICENSE) | | k8s.io/cli-runtime/pkg | [Apache-2.0](https://github.com/kubernetes/cli-runtime/blob/v0.30.3/LICENSE) | | k8s.io/client-go | [Apache-2.0](https://github.com/kubernetes/client-go/blob/v0.30.3/LICENSE) | | k8s.io/client-go/third_party/forked/golang/template | [BSD-3-Clause](https://github.com/kubernetes/client-go/blob/v0.30.3/third_party/forked/golang/LICENSE) | @@ -409,8 +419,9 @@ run `task generate:third-party-licenses-list` | k8s.io/kube-openapi/pkg/validation/spec | [Apache-2.0](https://github.com/kubernetes/kube-openapi/blob/3c01b740850f/pkg/validation/spec/LICENSE) | | k8s.io/kubectl/pkg | [Apache-2.0](https://github.com/kubernetes/kubectl/blob/v0.30.3/LICENSE) | | k8s.io/utils | [Apache-2.0](https://github.com/kubernetes/utils/blob/18e509b52bc8/LICENSE) | -| k8s.io/utils/internal/third_party/forked/golang/net | [BSD-3-Clause](https://github.com/kubernetes/utils/blob/18e509b52bc8/internal/third_party/forked/golang/LICENSE) | +| k8s.io/utils/internal/third_party/forked/golang | [BSD-3-Clause](https://github.com/kubernetes/utils/blob/18e509b52bc8/internal/third_party/forked/golang/LICENSE) | | oras.land/oras-go/pkg | [Apache-2.0](https://github.com/oras-project/oras-go/blob/v1.2.5/LICENSE) | +| sigs.k8s.io/apiserver-network-proxy/konnectivity-client | [Apache-2.0](https://github.com/kubernetes-sigs/apiserver-network-proxy/blob/konnectivity-client/v0.29.0/konnectivity-client/LICENSE) | | sigs.k8s.io/controller-runtime | [Apache-2.0](https://github.com/kubernetes-sigs/controller-runtime/blob/v0.18.5/LICENSE) | | sigs.k8s.io/gateway-api/apis/v1 | [Apache-2.0](https://github.com/kubernetes-sigs/gateway-api/blob/v1.0.0/LICENSE) | | sigs.k8s.io/json | [Apache-2.0](https://github.com/kubernetes-sigs/json/blob/9aa6b5e7a4b3/LICENSE) | diff --git a/operator/cmd/run/run.go b/operator/cmd/run/run.go index f39c33814..2979e698b 100644 --- a/operator/cmd/run/run.go +++ b/operator/cmd/run/run.go @@ -13,8 +13,11 @@ package run import ( "context" + "crypto/tls" "errors" "fmt" + "os" + "path/filepath" "strings" "time" @@ -27,7 +30,9 @@ import ( _ "k8s.io/client-go/plugin/pkg/client/auth" ctrl "sigs.k8s.io/controller-runtime" "sigs.k8s.io/controller-runtime/pkg/cache" + "sigs.k8s.io/controller-runtime/pkg/certwatcher" "sigs.k8s.io/controller-runtime/pkg/healthz" + "sigs.k8s.io/controller-runtime/pkg/metrics/filters" metricsserver "sigs.k8s.io/controller-runtime/pkg/metrics/server" "sigs.k8s.io/controller-runtime/pkg/webhook" "sigs.k8s.io/controller-runtime/pkg/webhook/admission" @@ -116,6 +121,8 @@ func Command() *cobra.Command { var ( clusterDomain string metricsAddr string + secureMetrics bool + enableHTTP2 bool probeAddr string pprofAddr string enableLeaderElection bool @@ -136,6 +143,12 @@ func Command() *cobra.Command { autoDeletePVCs bool forceDefluxedMode bool helmRepositoryURL string + webhookCertPath string + webhookCertName string + webhookCertKey string + metricsCertPath string + metricsCertName string + metricsCertKey string ) cmd := &cobra.Command{ @@ -148,6 +161,8 @@ func Command() *cobra.Command { ctx, clusterDomain, metricsAddr, + secureMetrics, + enableHTTP2, probeAddr, enableLeaderElection, webhookEnabled, @@ -168,11 +183,20 @@ func Command() *cobra.Command { forceDefluxedMode, pprofAddr, helmRepositoryURL, + webhookCertPath, + webhookCertName, + webhookCertKey, + metricsCertPath, + metricsCertName, + metricsCertKey, ) }, } - cmd.Flags().StringVar(&metricsAddr, "metrics-bind-address", ":8080", "The address the metric endpoint binds to.") + cmd.Flags().StringVar(&metricsAddr, "metrics-bind-address", "0", "The address the metrics endpoint binds to. "+ + "Use :8443 for HTTPS or :8080 for HTTP, or leave as 0 to disable the metrics service.") + cmd.Flags().BoolVar(&secureMetrics, "metrics-secure", true, "If set, the metrics endpoint is served securely via HTTPS. Use --metrics-secure=false to use HTTP instead.") + cmd.Flags().BoolVar(&enableHTTP2, "enable-http2", false, "If set, HTTP/2 will be enabled for the metrics and webhook servers") cmd.Flags().StringVar(&probeAddr, "health-probe-bind-address", ":8081", "The address the probe endpoint binds to.") cmd.Flags().StringVar(&pprofAddr, "pprof-bind-address", ":8082", "The address the metric endpoint binds to. Set to '' or 0 to disable") cmd.Flags().StringVar(&clusterDomain, "cluster-domain", "cluster.local", "Set the Kubernetes local domain (Kubelet's --cluster-domain)") @@ -202,6 +226,14 @@ func Command() *cobra.Command { cmd.Flags().BoolVar(&forceDefluxedMode, "force-defluxed-mode", false, "specifies the default value for useFlux of Redpanda CRs if not specified. May be used in conjunction with enable-helm-controllers=false") cmd.Flags().StringVar(&helmRepositoryURL, "helm-repository-url", "https://charts.redpanda.com/", "URL to overwrite official `https://charts.redpanda.com/` Redpanda Helm chart repository") + cmd.Flags().StringVar(&webhookCertPath, "webhook-cert-path", "", "The directory that contains the webhook certificate.") + cmd.Flags().StringVar(&webhookCertName, "webhook-cert-name", "tls.crt", "The name of the webhook certificate file.") + cmd.Flags().StringVar(&webhookCertKey, "webhook-cert-key", "tls.key", "The name of the webhook key file.") + cmd.Flags().StringVar(&metricsCertPath, "metrics-cert-path", "", + "The directory that contains the metrics server certificate.") + cmd.Flags().StringVar(&metricsCertName, "metrics-cert-name", "tls.crt", "The name of the metrics server certificate file.") + cmd.Flags().StringVar(&metricsCertKey, "metrics-cert-key", "tls.key", "The name of the metrics server key file.") + // 3rd party flags. clientOptions.BindFlags(cmd.Flags()) kubeConfigOpts.BindFlags(cmd.Flags()) @@ -218,6 +250,8 @@ func Run( ctx context.Context, clusterDomain string, metricsAddr string, + secureMetrics bool, + enableHTTP2 bool, probeAddr string, enableLeaderElection bool, webhookEnabled bool, @@ -238,19 +272,119 @@ func Run( forceDefluxedMode bool, pprofAddr string, helmRepositoryURL string, + webhookCertPath string, + webhookCertName string, + webhookCertKey string, + metricsCertPath string, + metricsCertName string, + metricsCertKey string, ) error { setupLog := ctrl.LoggerFrom(ctx).WithName("setup") // set the managedFields owner for resources reconciled from Helm charts kube.ManagedFieldsManager = controllerName + // if the enable-http2 flag is false (the default), http/2 should be disabled + // due to its vulnerabilities. More specifically, disabling http/2 will + // prevent from being vulnerable to the HTTP/2 Stream Cancellation and + // Rapid Reset CVEs. For more information see: + // - https://github.com/advisories/GHSA-qppj-fm5r-hxr3 + // - https://github.com/advisories/GHSA-4374-p667-p6c8 + disableHTTP2 := func(c *tls.Config) { + setupLog.Info("disabling http/2") + c.NextProtos = []string{"http/1.1"} + } + + var tlsOpts []func(*tls.Config) + if !enableHTTP2 { + tlsOpts = append(tlsOpts, disableHTTP2) + } + + // Create watchers for metrics and webhooks certificates + var metricsCertWatcher, webhookCertWatcher *certwatcher.CertWatcher + + var webhookServer webhook.Server + if webhookEnabled { + // Initial webhook TLS options + webhookTLSOpts := tlsOpts + + if len(webhookCertPath) > 0 { + setupLog.Info("Initializing webhook certificate watcher using provided certificates", + "webhook-cert-path", webhookCertPath, "webhook-cert-name", webhookCertName, "webhook-cert-key", webhookCertKey) + + var err error + webhookCertWatcher, err = certwatcher.New( + filepath.Join(webhookCertPath, webhookCertName), + filepath.Join(webhookCertPath, webhookCertKey), + ) + if err != nil { + setupLog.Error(err, "Failed to initialize webhook certificate watcher") + os.Exit(1) + } + + webhookTLSOpts = append(webhookTLSOpts, func(config *tls.Config) { + config.GetCertificate = webhookCertWatcher.GetCertificate + }) + } + + webhookServer = webhook.NewServer(webhook.Options{ + TLSOpts: webhookTLSOpts, + }) + } + + // Metrics endpoint is enabled in 'config/default/kustomization.yaml'. The Metrics options configure the server. + // More info: + // - https://pkg.go.dev/sigs.k8s.io/controller-runtime@v0.19.4/pkg/metrics/server + // - https://book.kubebuilder.io/reference/metrics.html + metricsServerOptions := metricsserver.Options{ + BindAddress: metricsAddr, + SecureServing: secureMetrics, + TLSOpts: tlsOpts, + } + + if secureMetrics { + // FilterProvider is used to protect the metrics endpoint with authn/authz. + // These configurations ensure that only authorized users and service accounts + // can access the metrics endpoint. The RBAC are configured in 'config/rbac/kustomization.yaml'. More info: + // https://pkg.go.dev/sigs.k8s.io/controller-runtime@v0.19.4/pkg/metrics/filters#WithAuthenticationAndAuthorization + metricsServerOptions.FilterProvider = filters.WithAuthenticationAndAuthorization + } + + // If the certificate is not specified, controller-runtime will automatically + // generate self-signed certificates for the metrics server. While convenient for development and testing, + // this setup is not recommended for production. + // + // TODO(user): If you enable certManager, uncomment the following lines: + // - [METRICS-WITH-CERTS] at config/default/kustomization.yaml to generate and use certificates + // managed by cert-manager for the metrics server. + // - [PROMETHEUS-WITH-CERTS] at config/prometheus/kustomization.yaml for TLS certification. + if len(metricsCertPath) > 0 { + setupLog.Info("Initializing metrics certificate watcher using provided certificates", + "metrics-cert-path", metricsCertPath, "metrics-cert-name", metricsCertName, "metrics-cert-key", metricsCertKey) + + var err error + metricsCertWatcher, err = certwatcher.New( + filepath.Join(metricsCertPath, metricsCertName), + filepath.Join(metricsCertPath, metricsCertKey), + ) + if err != nil { + setupLog.Error(err, "to initialize metrics certificate watcher", "error", err) + os.Exit(1) + } + + metricsServerOptions.TLSOpts = append(metricsServerOptions.TLSOpts, func(config *tls.Config) { + config.GetCertificate = metricsCertWatcher.GetCertificate + }) + } + mgrOptions := ctrl.Options{ HealthProbeBindAddress: probeAddr, LeaderElection: enableLeaderElection, LeaderElectionID: "aa9fc693.vectorized.io", LeaderElectionNamespace: namespace, - Metrics: metricsserver.Options{BindAddress: metricsAddr}, + Metrics: metricsServerOptions, PprofBindAddress: pprofAddr, + WebhookServer: webhookServer, } if namespace != "" { mgrOptions.Cache.DefaultNamespaces = map[string]cache.Config{namespace: {}} diff --git a/operator/config/default-with-metrics/kustomization.yaml b/operator/config/default-with-metrics/kustomization.yaml deleted file mode 100644 index ebdfbf5c6..000000000 --- a/operator/config/default-with-metrics/kustomization.yaml +++ /dev/null @@ -1,44 +0,0 @@ -# Adds namespace to all resources. -namespace: redpanda-system - -# Value of this field is prepended to the -# names of all resources, e.g. a deployment named -# "wordpress" becomes "alices-wordpress". -# Note that it should also match with the prefix (text before '-') of the namespace -# field above. -namePrefix: redpanda- - -# Labels to add to all resources and selectors. -#commonLabels: -# someName: someValue - -resources: -- ../crd -- ../rbac -- ../manager -# [WEBHOOK] To enable webhook, uncomment all the sections with [WEBHOOK] prefix including the one in -# crd/kustomization.yaml -- ../webhook -# [CERTMANAGER] To enable cert-manager, uncomment all sections with 'CERTMANAGER'. 'WEBHOOK' components are required. -- ../certmanager -# [PROMETHEUS] To enable prometheus monitor, uncomment all sections with 'PROMETHEUS'. -- ../prometheus - -patchesStrategicMerge: -# Protect the /metrics endpoint by putting it behind auth. -# If you want your controller-manager to expose the /metrics -# endpoint w/o any authn/z, please comment the following line. -- manager_auth_proxy_patch.yaml - -# Mount the controller config file for loading manager configurations -# through a ComponentConfig type -#- manager_config_patch.yaml - -# [WEBHOOK] To enable webhook, uncomment all the sections with [WEBHOOK] prefix including the one in -# crd/kustomization.yaml -- manager_webhook_patch.yaml - -# [CERTMANAGER] To enable cert-manager, uncomment all sections with 'CERTMANAGER'. -# Uncomment 'CERTMANAGER' sections in crd/kustomization.yaml to enable the CA injection in the admission webhooks. -# 'CERTMANAGER' needs to be enabled to use ca injection -- webhookcainjection_patch.yaml diff --git a/operator/config/default-with-metrics/manager_auth_proxy_patch.yaml b/operator/config/default-with-metrics/manager_auth_proxy_patch.yaml deleted file mode 100644 index 9d923a31b..000000000 --- a/operator/config/default-with-metrics/manager_auth_proxy_patch.yaml +++ /dev/null @@ -1,28 +0,0 @@ -# This patch inject a sidecar container which is a HTTP proxy for the -# controller manager, it performs RBAC authorization against the Kubernetes API using SubjectAccessReviews. -apiVersion: apps/v1 -kind: Deployment -metadata: - name: controller-manager - namespace: system -spec: - template: - spec: - containers: - - name: kube-rbac-proxy - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.1 - args: - - "--secure-listen-address=0.0.0.0:8443" - - "--upstream=http://127.0.0.1:8080/" - - "--logtostderr=true" - - "--v=10" - ports: - - containerPort: 8443 - name: https - - name: manager - imagePullPolicy: IfNotPresent - args: - - "--health-probe-bind-address=:8081" - - "--metrics-bind-address=127.0.0.1:8080" - - "--leader-elect" - - "--webhook-enabled=true" diff --git a/operator/config/default-with-metrics/manager_config_patch.yaml b/operator/config/default-with-metrics/manager_config_patch.yaml deleted file mode 100644 index 6c400155c..000000000 --- a/operator/config/default-with-metrics/manager_config_patch.yaml +++ /dev/null @@ -1,20 +0,0 @@ -apiVersion: apps/v1 -kind: Deployment -metadata: - name: controller-manager - namespace: system -spec: - template: - spec: - containers: - - name: manager - args: - - "--config=controller_manager_config.yaml" - volumeMounts: - - name: manager-config - mountPath: /controller_manager_config.yaml - subPath: controller_manager_config.yaml - volumes: - - name: manager-config - configMap: - name: manager-config diff --git a/operator/config/default-with-metrics/manager_webhook_patch.yaml b/operator/config/default-with-metrics/manager_webhook_patch.yaml deleted file mode 100644 index 738de350b..000000000 --- a/operator/config/default-with-metrics/manager_webhook_patch.yaml +++ /dev/null @@ -1,23 +0,0 @@ -apiVersion: apps/v1 -kind: Deployment -metadata: - name: controller-manager - namespace: system -spec: - template: - spec: - containers: - - name: manager - ports: - - containerPort: 9443 - name: webhook-server - protocol: TCP - volumeMounts: - - mountPath: /tmp/k8s-webhook-server/serving-certs - name: cert - readOnly: true - volumes: - - name: cert - secret: - defaultMode: 420 - secretName: webhook-server-cert diff --git a/operator/config/default-with-metrics/webhookcainjection_patch.yaml b/operator/config/default-with-metrics/webhookcainjection_patch.yaml deleted file mode 100644 index 93c0c65aa..000000000 --- a/operator/config/default-with-metrics/webhookcainjection_patch.yaml +++ /dev/null @@ -1,15 +0,0 @@ -# This patch add annotation to admission webhook config and -# the variables $(CERTIFICATE_NAMESPACE) and $(CERTIFICATE_NAME) will be substituted by kustomize. -apiVersion: admissionregistration.k8s.io/v1 -kind: MutatingWebhookConfiguration -metadata: - name: mutating-webhook-configuration - annotations: - cert-manager.io/inject-ca-from: CERTIFICATE_NAMESPACE/CERTIFICATE_NAME ---- -apiVersion: admissionregistration.k8s.io/v1 -kind: ValidatingWebhookConfiguration -metadata: - name: validating-webhook-configuration - annotations: - cert-manager.io/inject-ca-from: CERTIFICATE_NAMESPACE/CERTIFICATE_NAME diff --git a/operator/config/default/kustomization.yaml b/operator/config/default/kustomization.yaml index 63af6a5a9..48d0400fe 100644 --- a/operator/config/default/kustomization.yaml +++ b/operator/config/default/kustomization.yaml @@ -24,119 +24,197 @@ resources: # [PROMETHEUS] To enable prometheus monitor, uncomment all sections with 'PROMETHEUS'. # - ../prometheus -patchesStrategicMerge: -# Protect the /metrics endpoint by putting it behind auth. -# If you want your controller-manager to expose the /metrics -# endpoint w/o any authn/z, please comment the following line. -- manager_auth_proxy_patch.yaml +# Uncomment the patches line if you enable Metrics +patches: +# [METRICS] The following patch will enable the metrics endpoint using HTTPS and the port :8443. +# More info: https://book.kubebuilder.io/reference/metrics +- path: manager_metrics_patch.yaml + target: + kind: Deployment -# Mount the controller config file for loading manager configurations -# through a ComponentConfig type -#- manager_config_patch.yaml +# Uncomment the patches line if you enable Metrics and CertManager +# [METRICS-WITH-CERTS] To enable metrics protected with certManager, uncomment the following line. +# This patch will protect the metrics with certManager self-signed certs. +#- path: cert_metrics_manager_patch.yaml +# target: +# kind: Deployment # [WEBHOOK] To enable webhook, uncomment all the sections with [WEBHOOK] prefix including the one in # crd/kustomization.yaml -- manager_webhook_patch.yaml - -# [CERTMANAGER] To enable cert-manager, uncomment all sections with 'CERTMANAGER'. -# Uncomment 'CERTMANAGER' sections in crd/kustomization.yaml to enable the CA injection in the admission webhooks. -# 'CERTMANAGER' needs to be enabled to use ca injection -- webhookcainjection_patch.yaml +- path: manager_webhook_patch.yaml + target: + kind: Deployment +# [CERTMANAGER] To enable cert-manager, uncomment all sections with 'CERTMANAGER' prefix. +# Uncomment the following replacements to add the cert-manager CA injection annotations replacements: - - source: # Add cert-manager annotation to ValidatingWebhookConfiguration, MutatingWebhookConfiguration and CRDs - kind: Certificate - group: cert-manager.io - version: v1 - name: serving-cert # this name should match the one in certificate.yaml - fieldPath: .metadata.namespace # namespace of the certificate CR - targets: - - select: - kind: ValidatingWebhookConfiguration - fieldPaths: - - .metadata.annotations.[cert-manager.io/inject-ca-from] - options: - delimiter: '/' - index: 0 - create: true - - select: - kind: MutatingWebhookConfiguration - fieldPaths: - - .metadata.annotations.[cert-manager.io/inject-ca-from] - options: - delimiter: '/' - index: 0 - create: true - - select: - kind: CustomResourceDefinition - fieldPaths: - - .metadata.annotations.[cert-manager.io/inject-ca-from] - options: - delimiter: '/' - index: 0 - create: true - - source: - kind: Certificate - group: cert-manager.io - version: v1 - name: serving-cert # this name should match the one in certificate.yaml - fieldPath: .metadata.name - targets: - - select: - kind: ValidatingWebhookConfiguration - fieldPaths: - - .metadata.annotations.[cert-manager.io/inject-ca-from] - options: - delimiter: '/' - index: 1 - create: true - - select: - kind: MutatingWebhookConfiguration - fieldPaths: - - .metadata.annotations.[cert-manager.io/inject-ca-from] - options: - delimiter: '/' - index: 1 - create: true - - select: - kind: CustomResourceDefinition - fieldPaths: - - .metadata.annotations.[cert-manager.io/inject-ca-from] - options: - delimiter: '/' - index: 1 - create: true - - source: # Add cert-manager annotation to the webhook Service - kind: Service - version: v1 - name: webhook-service - fieldPath: .metadata.name # namespace of the service - targets: - - select: - kind: Certificate - group: cert-manager.io - version: v1 - fieldPaths: - - .spec.dnsNames.0 - - .spec.dnsNames.1 - options: - delimiter: '.' - index: 0 - create: true - - source: - kind: Service - version: v1 - name: webhook-service - fieldPath: .metadata.namespace # namespace of the service - targets: - - select: - kind: Certificate - group: cert-manager.io - version: v1 - fieldPaths: - - .spec.dnsNames.0 - - .spec.dnsNames.1 - options: - delimiter: '.' - index: 1 - create: true +- source: # Uncomment the following block to enable certificates for metrics + kind: Service + version: v1 + name: controller-manager-metrics-service + fieldPath: metadata.name + targets: + - select: + kind: Certificate + group: cert-manager.io + version: v1 + name: metrics-certs + fieldPaths: + - spec.dnsNames.0 + - spec.dnsNames.1 + options: + delimiter: '.' + index: 0 + create: true + +- source: + kind: Service + version: v1 + name: controller-manager-metrics-service + fieldPath: metadata.namespace + targets: + - select: + kind: Certificate + group: cert-manager.io + version: v1 + name: metrics-certs + fieldPaths: + - spec.dnsNames.0 + - spec.dnsNames.1 + options: + delimiter: '.' + index: 1 + create: true + +- source: # Uncomment the following block if you have any webhook + kind: Service + version: v1 + name: webhook-service + fieldPath: .metadata.name # Name of the service + targets: + - select: + kind: Certificate + group: cert-manager.io + version: v1 + name: serving-cert + fieldPaths: + - .spec.dnsNames.0 + - .spec.dnsNames.1 + options: + delimiter: '.' + index: 0 + create: true +- source: + kind: Service + version: v1 + name: webhook-service + fieldPath: .metadata.namespace # Namespace of the service + targets: + - select: + kind: Certificate + group: cert-manager.io + version: v1 + name: serving-cert + fieldPaths: + - .spec.dnsNames.0 + - .spec.dnsNames.1 + options: + delimiter: '.' + index: 1 + create: true + +- source: # Uncomment the following block if you have a ValidatingWebhook (--programmatic-validation) + kind: Certificate + group: cert-manager.io + version: v1 + name: serving-cert # This name should match the one in certificate.yaml + fieldPath: .metadata.namespace # Namespace of the certificate CR + targets: + - select: + kind: ValidatingWebhookConfiguration + fieldPaths: + - .metadata.annotations.[cert-manager.io/inject-ca-from] + options: + delimiter: '/' + index: 0 + create: true +- source: + kind: Certificate + group: cert-manager.io + version: v1 + name: serving-cert + fieldPath: .metadata.name + targets: + - select: + kind: ValidatingWebhookConfiguration + fieldPaths: + - .metadata.annotations.[cert-manager.io/inject-ca-from] + options: + delimiter: '/' + index: 1 + create: true + +- source: # Uncomment the following block if you have a DefaultingWebhook (--defaulting ) + kind: Certificate + group: cert-manager.io + version: v1 + name: serving-cert + fieldPath: .metadata.namespace # Namespace of the certificate CR + targets: + - select: + kind: MutatingWebhookConfiguration + fieldPaths: + - .metadata.annotations.[cert-manager.io/inject-ca-from] + options: + delimiter: '/' + index: 0 + create: true +- source: + kind: Certificate + group: cert-manager.io + version: v1 + name: serving-cert + fieldPath: .metadata.name + targets: + - select: + kind: MutatingWebhookConfiguration + fieldPaths: + - .metadata.annotations.[cert-manager.io/inject-ca-from] + options: + delimiter: '/' + index: 1 + create: true + +- source: # Uncomment the following block if you have a ConversionWebhook (--conversion) + kind: Certificate + group: cert-manager.io + version: v1 + name: serving-cert + fieldPath: .metadata.namespace # Namespace of the certificate CR + targets: # Do not remove or uncomment the following scaffold marker; required to generate code for target CRD. + - select: + kind: CustomResourceDefinition + fieldPaths: + - .metadata.annotations.[cert-manager.io/inject-ca-from] + options: + delimiter: '/' + index: 0 + create: true +# +kubebuilder:scaffold:crdkustomizecainjectionns +- source: + kind: Certificate + group: cert-manager.io + version: v1 + name: serving-cert + fieldPath: .metadata.name + targets: # Do not remove or uncomment the following scaffold marker; required to generate code for target CRD. + - select: + kind: CustomResourceDefinition + name: firstmates.crew.testproject.org + fieldPaths: + - .metadata.annotations.[cert-manager.io/inject-ca-from] + options: + delimiter: '/' + index: 1 + create: true +# +kubebuilder:scaffold:crdkustomizecainjectionname diff --git a/operator/config/default/manager_auth_proxy_patch.yaml b/operator/config/default/manager_auth_proxy_patch.yaml deleted file mode 100644 index 9d923a31b..000000000 --- a/operator/config/default/manager_auth_proxy_patch.yaml +++ /dev/null @@ -1,28 +0,0 @@ -# This patch inject a sidecar container which is a HTTP proxy for the -# controller manager, it performs RBAC authorization against the Kubernetes API using SubjectAccessReviews. -apiVersion: apps/v1 -kind: Deployment -metadata: - name: controller-manager - namespace: system -spec: - template: - spec: - containers: - - name: kube-rbac-proxy - image: gcr.io/kubebuilder/kube-rbac-proxy:v0.14.1 - args: - - "--secure-listen-address=0.0.0.0:8443" - - "--upstream=http://127.0.0.1:8080/" - - "--logtostderr=true" - - "--v=10" - ports: - - containerPort: 8443 - name: https - - name: manager - imagePullPolicy: IfNotPresent - args: - - "--health-probe-bind-address=:8081" - - "--metrics-bind-address=127.0.0.1:8080" - - "--leader-elect" - - "--webhook-enabled=true" diff --git a/operator/config/default/manager_config_patch.yaml b/operator/config/default/manager_config_patch.yaml deleted file mode 100644 index 6c400155c..000000000 --- a/operator/config/default/manager_config_patch.yaml +++ /dev/null @@ -1,20 +0,0 @@ -apiVersion: apps/v1 -kind: Deployment -metadata: - name: controller-manager - namespace: system -spec: - template: - spec: - containers: - - name: manager - args: - - "--config=controller_manager_config.yaml" - volumeMounts: - - name: manager-config - mountPath: /controller_manager_config.yaml - subPath: controller_manager_config.yaml - volumes: - - name: manager-config - configMap: - name: manager-config diff --git a/operator/config/default/manager_metrics_patch.yaml b/operator/config/default/manager_metrics_patch.yaml new file mode 100644 index 000000000..2aaef6536 --- /dev/null +++ b/operator/config/default/manager_metrics_patch.yaml @@ -0,0 +1,4 @@ +# This patch adds the args to allow exposing the metrics endpoint using HTTPS +- op: add + path: /spec/template/spec/containers/0/args/0 + value: --metrics-bind-address=:8443 diff --git a/operator/config/default/manager_webhook_patch.yaml b/operator/config/default/manager_webhook_patch.yaml index 738de350b..ba94a2d35 100644 --- a/operator/config/default/manager_webhook_patch.yaml +++ b/operator/config/default/manager_webhook_patch.yaml @@ -1,23 +1,31 @@ -apiVersion: apps/v1 -kind: Deployment -metadata: - name: controller-manager - namespace: system -spec: - template: - spec: - containers: - - name: manager - ports: - - containerPort: 9443 - name: webhook-server - protocol: TCP - volumeMounts: - - mountPath: /tmp/k8s-webhook-server/serving-certs - name: cert - readOnly: true - volumes: - - name: cert - secret: - defaultMode: 420 - secretName: webhook-server-cert +# This patch ensures the webhook certificates are properly mounted in the manager container. +# It configures the necessary arguments, volumes, volume mounts, and container ports. + +# Add the --webhook-cert-path argument for configuring the webhook certificate path +- op: add + path: /spec/template/spec/containers/0/args/- + value: --webhook-cert-path=/tmp/k8s-webhook-server/serving-certs + +# Add the volumeMount for the webhook certificates +- op: add + path: /spec/template/spec/containers/0/volumeMounts/- + value: + mountPath: /tmp/k8s-webhook-server/serving-certs + name: webhook-certs + readOnly: true + +# Add the port configuration for the webhook server +- op: add + path: /spec/template/spec/containers/0/ports/- + value: + containerPort: 9443 + name: webhook-server + protocol: TCP + +# Add the volume configuration for the webhook certificates +- op: add + path: /spec/template/spec/volumes/- + value: + name: webhook-certs + secret: + secretName: webhook-server-cert \ No newline at end of file diff --git a/operator/config/default/mutatingwebhookcainjection_patch.yaml b/operator/config/default/mutatingwebhookcainjection_patch.yaml deleted file mode 100644 index 10f29d592..000000000 --- a/operator/config/default/mutatingwebhookcainjection_patch.yaml +++ /dev/null @@ -1,8 +0,0 @@ -# This patch add annotation to admission webhook config -# CERTIFICATE_NAMESPACE and CERTIFICATE_NAME will be substituted by kustomize. -apiVersion: admissionregistration.k8s.io/v1 -kind: MutatingWebhookConfiguration -metadata: - name: mutating-webhook-configuration - annotations: - cert-manager.io/inject-ca-from: CERTIFICATE_NAMESPACE/CERTIFICATE_NAME diff --git a/operator/config/default/validatingwebhookcainjection_patch.yaml b/operator/config/default/validatingwebhookcainjection_patch.yaml deleted file mode 100644 index 72cce394d..000000000 --- a/operator/config/default/validatingwebhookcainjection_patch.yaml +++ /dev/null @@ -1,8 +0,0 @@ -# This patch add annotation to admission webhook config -# CERTIFICATE_NAMESPACE and CERTIFICATE_NAME will be substituted by kustomize. -apiVersion: admissionregistration.k8s.io/v1 -kind: ValidatingWebhookConfiguration -metadata: - name: validating-webhook-configuration - annotations: - cert-manager.io/inject-ca-from: CERTIFICATE_NAMESPACE/CERTIFICATE_NAME diff --git a/operator/config/default/webhookcainjection_patch.yaml b/operator/config/default/webhookcainjection_patch.yaml deleted file mode 100644 index 93c0c65aa..000000000 --- a/operator/config/default/webhookcainjection_patch.yaml +++ /dev/null @@ -1,15 +0,0 @@ -# This patch add annotation to admission webhook config and -# the variables $(CERTIFICATE_NAMESPACE) and $(CERTIFICATE_NAME) will be substituted by kustomize. -apiVersion: admissionregistration.k8s.io/v1 -kind: MutatingWebhookConfiguration -metadata: - name: mutating-webhook-configuration - annotations: - cert-manager.io/inject-ca-from: CERTIFICATE_NAMESPACE/CERTIFICATE_NAME ---- -apiVersion: admissionregistration.k8s.io/v1 -kind: ValidatingWebhookConfiguration -metadata: - name: validating-webhook-configuration - annotations: - cert-manager.io/inject-ca-from: CERTIFICATE_NAMESPACE/CERTIFICATE_NAME diff --git a/operator/config/e2e-tests/manager.yaml b/operator/config/e2e-tests/manager.yaml index d589b27a9..eb74a5fec 100644 --- a/operator/config/e2e-tests/manager.yaml +++ b/operator/config/e2e-tests/manager.yaml @@ -11,9 +11,10 @@ spec: imagePullPolicy: IfNotPresent args: - "--health-probe-bind-address=:8081" - - "--metrics-bind-address=127.0.0.1:8080" + - "--metrics-bind-address=:8443" - "--leader-elect" - "--webhook-enabled=true" + - "--webhook-cert-path=/tmp/k8s-webhook-server/serving-certs" - "--configurator-base-image=localhost/redpanda-operator" - "--configurator-tag=dev" - "--configurator-image-pull-policy=Never" diff --git a/operator/config/manager/manager.yaml b/operator/config/manager/manager.yaml index 3c7319f4d..be518b1ce 100644 --- a/operator/config/manager/manager.yaml +++ b/operator/config/manager/manager.yaml @@ -31,6 +31,7 @@ spec: - --leader-elect image: vectorized/redpanda-operator:2021022-e853bf3 name: manager + ports: [] securityContext: allowPrivilegeEscalation: false livenessProbe: @@ -54,4 +55,6 @@ spec: requests: cpu: 100m memory: 128Mi + volumeMounts: [] terminationGracePeriodSeconds: 10 + volumes: [] \ No newline at end of file diff --git a/operator/config/rbac/bases/auth_proxy/kustomization.yaml b/operator/config/rbac/bases/auth_proxy/kustomization.yaml index efc81334f..e8bb7a2b9 100644 --- a/operator/config/rbac/bases/auth_proxy/kustomization.yaml +++ b/operator/config/rbac/bases/auth_proxy/kustomization.yaml @@ -1,7 +1,7 @@ apiVersion: kustomize.config.k8s.io/v1beta1 kind: Kustomization resources: -- auth_proxy_client_clusterrole.yaml -- auth_proxy_role.yaml -- auth_proxy_role_binding.yaml -- auth_proxy_service.yaml +- metrics_reader_role.yaml +- metrics_auth_role.yaml +- metrics_auth_role_binding.yaml +- metrics_auth_service.yaml diff --git a/operator/config/rbac/bases/auth_proxy/auth_proxy_role.yaml b/operator/config/rbac/bases/auth_proxy/metrics_auth_role.yaml similarity index 90% rename from operator/config/rbac/bases/auth_proxy/auth_proxy_role.yaml rename to operator/config/rbac/bases/auth_proxy/metrics_auth_role.yaml index 618f5e417..5fffcf571 100644 --- a/operator/config/rbac/bases/auth_proxy/auth_proxy_role.yaml +++ b/operator/config/rbac/bases/auth_proxy/metrics_auth_role.yaml @@ -1,7 +1,7 @@ apiVersion: rbac.authorization.k8s.io/v1 kind: ClusterRole metadata: - name: proxy-role + name: metrics-auth-role rules: - apiGroups: ["authentication.k8s.io"] resources: diff --git a/operator/config/rbac/bases/auth_proxy/auth_proxy_role_binding.yaml b/operator/config/rbac/bases/auth_proxy/metrics_auth_role_binding.yaml similarity index 76% rename from operator/config/rbac/bases/auth_proxy/auth_proxy_role_binding.yaml rename to operator/config/rbac/bases/auth_proxy/metrics_auth_role_binding.yaml index 48ed1e4b8..8052a1bc8 100644 --- a/operator/config/rbac/bases/auth_proxy/auth_proxy_role_binding.yaml +++ b/operator/config/rbac/bases/auth_proxy/metrics_auth_role_binding.yaml @@ -1,11 +1,11 @@ apiVersion: rbac.authorization.k8s.io/v1 kind: ClusterRoleBinding metadata: - name: proxy-rolebinding + name: metrics-auth-role-rolebinding roleRef: apiGroup: rbac.authorization.k8s.io kind: ClusterRole - name: proxy-role + name: metrics-auth-role subjects: - kind: ServiceAccount name: default diff --git a/operator/config/rbac/bases/auth_proxy/auth_proxy_service.yaml b/operator/config/rbac/bases/auth_proxy/metrics_auth_service.yaml similarity index 86% rename from operator/config/rbac/bases/auth_proxy/auth_proxy_service.yaml rename to operator/config/rbac/bases/auth_proxy/metrics_auth_service.yaml index 6cf656be1..ef4147424 100644 --- a/operator/config/rbac/bases/auth_proxy/auth_proxy_service.yaml +++ b/operator/config/rbac/bases/auth_proxy/metrics_auth_service.yaml @@ -9,6 +9,7 @@ spec: ports: - name: https port: 8443 - targetPort: https + protocol: TCP + targetPort: 8443 selector: control-plane: controller-manager diff --git a/operator/config/rbac/bases/auth_proxy/auth_proxy_client_clusterrole.yaml b/operator/config/rbac/bases/auth_proxy/metrics_reader_role.yaml similarity index 100% rename from operator/config/rbac/bases/auth_proxy/auth_proxy_client_clusterrole.yaml rename to operator/config/rbac/bases/auth_proxy/metrics_reader_role.yaml diff --git a/operator/config/rbac/kustomization.yaml b/operator/config/rbac/kustomization.yaml index 5c650b162..f5f6f44ab 100644 --- a/operator/config/rbac/kustomization.yaml +++ b/operator/config/rbac/kustomization.yaml @@ -1,3 +1,5 @@ +apiVersion: kustomize.config.k8s.io/v1beta1 +kind: Kustomization resources: - bases/operator/role.yaml - bases/operator/role_binding.yaml diff --git a/operator/config/without-webhook/kustomization.yaml b/operator/config/without-webhook/kustomization.yaml index d61445382..f0a1fe75f 100644 --- a/operator/config/without-webhook/kustomization.yaml +++ b/operator/config/without-webhook/kustomization.yaml @@ -16,9 +16,3 @@ resources: - ../crd - ../rbac - ../manager - -patchesStrategicMerge: -# Protect the /metrics endpoint by putting it behind auth. -# If you want your controller-manager to expose the /metrics -# endpoint w/o any authn/z, please comment the following line. -- manager_auth_proxy_patch.yaml diff --git a/operator/config/without-webhook/manager_auth_proxy_patch.yaml b/operator/config/without-webhook/manager_auth_proxy_patch.yaml deleted file mode 100644 index 79bf6ae47..000000000 --- a/operator/config/without-webhook/manager_auth_proxy_patch.yaml +++ /dev/null @@ -1,27 +0,0 @@ -# This patch inject a sidecar container which is a HTTP proxy for the -# controller manager, it performs RBAC authorization against the Kubernetes API using SubjectAccessReviews. -apiVersion: apps/v1 -kind: Deployment -metadata: - name: controller-manager - namespace: system -spec: - template: - spec: - containers: - - name: kube-rbac-proxy - image: quay.io/brancz/kube-rbac-proxy:v0.14.1 - args: - - "--secure-listen-address=0.0.0.0:8443" - - "--upstream=http://127.0.0.1:8080/" - - "--logtostderr=true" - - "--v=7" - ports: - - containerPort: 8443 - name: https - - name: manager - imagePullPolicy: IfNotPresent - args: - - "--health-probe-bind-address=:8081" - - "--metrics-bind-address=127.0.0.1:8080" - - "--leader-elect" diff --git a/operator/go.mod b/operator/go.mod index bc3056d25..91223d84d 100644 --- a/operator/go.mod +++ b/operator/go.mod @@ -121,6 +121,7 @@ require ( github.com/alibabacloud-go/tea-utils v1.4.5 // indirect github.com/alibabacloud-go/tea-xml v1.1.3 // indirect github.com/aliyun/credentials-go v1.3.2 // indirect + github.com/antlr/antlr4/runtime/Go/antlr/v4 v4.0.0-20230305170008-8188dc5388df // indirect github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 // indirect github.com/aws/aws-sdk-go-v2 v1.24.0 // indirect github.com/aws/aws-sdk-go-v2/config v1.26.1 // indirect @@ -141,6 +142,7 @@ require ( github.com/bahlo/generic-list-go v0.2.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/blang/semver v3.5.1+incompatible // indirect + github.com/blang/semver/v4 v4.0.0 // indirect github.com/buger/jsonparser v1.1.1 // indirect github.com/cenkalti/backoff/v4 v4.3.0 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect @@ -234,6 +236,7 @@ require ( github.com/gonvenience/wrap v1.2.0 // indirect github.com/gonvenience/ytbx v1.4.4 // indirect github.com/google/btree v1.1.2 // indirect + github.com/google/cel-go v0.18.2 // indirect github.com/google/certificate-transparency-go v1.1.7 // indirect github.com/google/gnostic-models v0.6.9-0.20230804172637-c7be7c783f49 // indirect github.com/google/go-cmp v0.6.0 // indirect @@ -253,6 +256,7 @@ require ( github.com/gorilla/websocket v1.5.0 // indirect github.com/gosuri/uitable v0.0.4 // indirect github.com/gregjones/httpcache v0.0.0-20190611155906-901d90724c79 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1 // indirect github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-cleanhttp v0.5.2 // indirect github.com/hashicorp/go-multierror v1.1.1 // indirect @@ -372,6 +376,7 @@ require ( github.com/sourcegraph/conc v0.3.0 // indirect github.com/spf13/cast v1.7.0 // indirect github.com/spf13/viper v1.18.1 // indirect + github.com/stoewer/go-strcase v1.3.0 // indirect github.com/subosito/gotenv v1.6.0 // indirect github.com/syndtr/goleveldb v1.0.1-0.20220721030215-126854af5e6d // indirect github.com/texttheater/golang-levenshtein v1.0.1 // indirect @@ -405,7 +410,9 @@ require ( go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.53.0 // indirect go.opentelemetry.io/otel v1.28.0 // indirect go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.24.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0 // indirect go.opentelemetry.io/otel/metric v1.28.0 // indirect + go.opentelemetry.io/otel/sdk v1.24.0 // indirect go.opentelemetry.io/otel/trace v1.28.0 // indirect go.opentelemetry.io/proto/otlp v1.1.0 // indirect go.starlark.net v0.0.0-20231121155337-90ade8b19d09 // indirect @@ -438,6 +445,7 @@ require ( k8s.io/klog/v2 v2.130.1 // indirect k8s.io/kube-openapi v0.0.0-20240709000822-3c01b740850f // indirect oras.land/oras-go v1.2.5 // indirect + sigs.k8s.io/apiserver-network-proxy/konnectivity-client v0.29.0 // indirect sigs.k8s.io/gateway-api v1.0.0 // indirect sigs.k8s.io/json v0.0.0-20241010143419-9aa6b5e7a4b3 // indirect sigs.k8s.io/kustomize/api v0.16.0 // indirect @@ -459,3 +467,8 @@ replace ( pgregory.net/rapid => github.com/chrisseto/rapid v0.0.0-20240815210052-cdeef406c65c ) + +replace ( + github.com/google/cel-go => github.com/google/cel-go v0.17.8 + go.opentelemetry.io/otel/sdk => go.opentelemetry.io/otel/sdk v1.28.0 +) diff --git a/operator/go.sum b/operator/go.sum index 201232e23..86bdc56fb 100644 --- a/operator/go.sum +++ b/operator/go.sum @@ -173,6 +173,8 @@ github.com/aliyun/credentials-go v1.3.2/go.mod h1:tlpz4uys4Rn7Ik4/piGRrTbXy2uLKv github.com/anmitsu/go-shlex v0.0.0-20200514113438-38f4b401e2be h1:9AeTilPcZAjCFIImctFaOjnTIavg87rW78vTPkQqLI8= github.com/anmitsu/go-shlex v0.0.0-20200514113438-38f4b401e2be/go.mod h1:ySMOLuWl6zY27l47sB3qLNK6tF2fkHG55UZxx8oIVo4= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= +github.com/antlr/antlr4/runtime/Go/antlr/v4 v4.0.0-20230305170008-8188dc5388df h1:7RFfzj4SSt6nnvCPbCqijJi1nWCd+TqAT3bYCStRC18= +github.com/antlr/antlr4/runtime/Go/antlr/v4 v4.0.0-20230305170008-8188dc5388df/go.mod h1:pSwJ0fSY5KhvocuWSx4fz3BA8OrA1bQn+K1Eli3BRwM= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= @@ -237,6 +239,8 @@ github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6r github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/blang/semver v3.5.1+incompatible h1:cQNTCjp13qL8KC3Nbxr/y2Bqb63oX6wdnnjpJbkM4JQ= github.com/blang/semver v3.5.1+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnwebNt5EWlYSAyrTnjyyk= +github.com/blang/semver/v4 v4.0.0 h1:1PFHFE6yCCTv8C1TeyNNarDzntLi7wMI5i/pzqYIsAM= +github.com/blang/semver/v4 v4.0.0/go.mod h1:IbckMUScFkM3pff0VJDNKRiT6TG/YpiHIM2yvyW5YoQ= github.com/bshuster-repo/logrus-logstash-hook v1.0.0 h1:e+C0SB5R1pu//O4MQ3f9cFuPGoOVeF2fE4Og9otCc70= github.com/bshuster-repo/logrus-logstash-hook v1.0.0/go.mod h1:zsTqEiSzDgAa/8GZR7E1qaXrhYNDKBYy5/dWPTIflbk= github.com/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= @@ -600,6 +604,8 @@ github.com/gonvenience/ytbx v1.4.4/go.mod h1:w37+MKCPcCMY/jpPNmEklD4xKqrOAVBO6kI github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.1.2 h1:xf4v41cLI2Z6FxbKm+8Bu+m8ifhj15JuZ9sa0jZCMUU= github.com/google/btree v1.1.2/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= +github.com/google/cel-go v0.17.8 h1:j9m730pMZt1Fc4oKhCLUHfjj6527LuhYcYw0Rl8gqto= +github.com/google/cel-go v0.17.8/go.mod h1:HXZKzB0LXqer5lHHgfWAnlYwJaQBDKMjxjulNQzhwhY= github.com/google/certificate-transparency-go v1.1.7 h1:IASD+NtgSTJLPdzkthwvAG1ZVbF2WtFg4IvoA68XGSw= github.com/google/certificate-transparency-go v1.1.7/go.mod h1:FSSBo8fyMVgqptbfF6j5p/XNdgQftAhSmXcIxV9iphE= github.com/google/gnostic-models v0.6.9-0.20230804172637-c7be7c783f49 h1:0VpGH+cDhbDtdcweoyCVsF3fhN8kejK6rFe/2FFX2nU= @@ -667,7 +673,6 @@ github.com/gosuri/uitable v0.0.4/go.mod h1:tKR86bXuXPZazfOTG1FIzvjIdXzd0mo4Vtn16 github.com/gregjones/httpcache v0.0.0-20190611155906-901d90724c79 h1:+ngKgrYPPJrOjhax5N+uePQ0Fh1Z7PheYoUI/0nzkPA= github.com/gregjones/httpcache v0.0.0-20190611155906-901d90724c79/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= -github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1 h1:/c3QmbOGMGTOumP2iT/rCwB7b0QDGLKzqOmktBjT+Is= github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1/go.mod h1:5SN9VR2LTsRFsrEC6FHgRbTWrTHu6tqPeKxEQv15giM= @@ -1200,6 +1205,8 @@ github.com/spf13/viper v1.18.1 h1:rmuU42rScKWlhhJDyXZRKJQHXFX02chSVW1IvkPGiVM= github.com/spf13/viper v1.18.1/go.mod h1:EKmWIqdnk5lOcmR72yw6hS+8OPYcwD0jteitLMVB+yk= github.com/spiffe/go-spiffe/v2 v2.1.6 h1:4SdizuQieFyL9eNU+SPiCArH4kynzaKOOj0VvM8R7Xo= github.com/spiffe/go-spiffe/v2 v2.1.6/go.mod h1:eVDqm9xFvyqao6C+eQensb9ZPkyNEeaUbqbBpOhBnNk= +github.com/stoewer/go-strcase v1.3.0 h1:g0eASXYtp+yvN9fK8sH94oCIk0fau9uV1/ZdJ0AVEzs= +github.com/stoewer/go-strcase v1.3.0/go.mod h1:fAH5hQ5pehh+j3nZfvwdk2RgEgQjAoM8wodgtPmh1xo= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= @@ -1347,12 +1354,14 @@ go.opentelemetry.io/otel v1.28.0 h1:/SqNcYk+idO0CxKEUOtKQClMK/MimZihKYMruSMViUo= go.opentelemetry.io/otel v1.28.0/go.mod h1:q68ijF8Fc8CnMHKyzqL6akLO46ePnjkgfIMIjUIX9z4= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.24.0 h1:t6wl9SPayj+c7lEIFgm4ooDBZVb01IhLB4InpomhRw8= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.24.0/go.mod h1:iSDOcsnSA5INXzZtwaBPrKp/lWu/V14Dd+llD0oI2EA= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0 h1:tIqheXEFWAZ7O8A7m+J0aPTmpJN3YQ7qetUAdkkkKpk= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.21.0/go.mod h1:nUeKExfxAQVbiVFn32YXpXZZHZ61Cc3s3Rn1pDBGAb0= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.24.0 h1:Xw8U6u2f8DK2XAkGRFV7BBLENgnTGX9i4rQRxJf+/vs= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.24.0/go.mod h1:6KW1Fm6R/s6Z3PGXwSJN2K4eT6wQB3vXX6CVnYX9NmM= go.opentelemetry.io/otel/metric v1.28.0 h1:f0HGvSl1KRAU1DLgLGFjrwVyismPlnuU6JD6bOeuA5Q= go.opentelemetry.io/otel/metric v1.28.0/go.mod h1:Fb1eVBFZmLVTMb6PPohq3TO9IIhUisDsbJoL/+uQW4s= -go.opentelemetry.io/otel/sdk v1.24.0 h1:YMPPDNymmQN3ZgczicBY3B6sf9n62Dlj9pWD3ucgoDw= -go.opentelemetry.io/otel/sdk v1.24.0/go.mod h1:KVrIYw6tEubO9E96HQpcmpTKDVn9gdv35HoYiQWGDFg= +go.opentelemetry.io/otel/sdk v1.28.0 h1:b9d7hIry8yZsgtbmM0DKyPWMMUMlK9NEKuIG4aBqWyE= +go.opentelemetry.io/otel/sdk v1.28.0/go.mod h1:oYj7ClPUA7Iw3m+r7GeEjz0qckQRJK2B8zjcZEfu7Pg= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= go.opentelemetry.io/proto/otlp v1.1.0 h1:2Di21piLrCqJ3U3eXGCTPHE9R8Nh+0uglSnOyxikMeI= @@ -1698,6 +1707,8 @@ k8s.io/utils v0.0.0-20240711033017-18e509b52bc8 h1:pUdcCO1Lk/tbT5ztQWOBi5HBgbBP1 k8s.io/utils v0.0.0-20240711033017-18e509b52bc8/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= oras.land/oras-go v1.2.5 h1:XpYuAwAb0DfQsunIyMfeET92emK8km3W4yEzZvUbsTo= oras.land/oras-go v1.2.5/go.mod h1:PuAwRShRZCsZb7g8Ar3jKKQR/2A/qN+pkYxIOd/FAoo= +sigs.k8s.io/apiserver-network-proxy/konnectivity-client v0.29.0 h1:/U5vjBbQn3RChhv7P11uhYvCSm5G2GaIi5AIGBS6r4c= +sigs.k8s.io/apiserver-network-proxy/konnectivity-client v0.29.0/go.mod h1:z7+wmGM2dfIiLRfrC6jb5kV2Mq/sK1ZP303cxzkV5Y4= sigs.k8s.io/controller-runtime v0.18.5 h1:nTHio/W+Q4aBlQMgbnC5hZb4IjIidyrizMai9P6n4Rk= sigs.k8s.io/controller-runtime v0.18.5/go.mod h1:TVoGrfdpbA9VRFaRnKgk9P5/atA0pMwq+f+msb9M8Sg= sigs.k8s.io/gateway-api v1.0.0 h1:iPTStSv41+d9p0xFydll6d7f7MOBGuqXM6p2/zVYMAs=