From a6b9d48d3b612720295f7c31c2c9f4c8f9e65392 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 10:10:01 +0200 Subject: [PATCH 01/25] RS Controller refactor fixed merge conflicts --- .gitignore | 4 + api/v1/mdb/mongodb_types.go | 2 +- controllers/om/process/om_process.go | 12 + controllers/om/replicaset/om_replicaset.go | 10 + controllers/operator/agents/agents.go | 13 + .../mongodbmultireplicaset_controller.go | 19 +- .../operator/mongodbreplicaset_controller.go | 363 ++++++++++++------ .../mongodbreplicaset_controller_test.go | 16 +- 8 files changed, 311 insertions(+), 128 deletions(-) diff --git a/.gitignore b/.gitignore index be38e569d..b6df00461 100644 --- a/.gitignore +++ b/.gitignore @@ -93,3 +93,7 @@ docs/**/test.sh.run.log dist logs *.run.log + +# AI +.mcp.json +CLAUDE.md diff --git a/api/v1/mdb/mongodb_types.go b/api/v1/mdb/mongodb_types.go index 91765cc81..6653203c2 100644 --- a/api/v1/mdb/mongodb_types.go +++ b/api/v1/mdb/mongodb_types.go @@ -1037,7 +1037,7 @@ func (a *Authentication) IsOIDCEnabled() bool { return stringutil.Contains(a.GetModes(), util.OIDC) } -// GetModes returns the modes of the Authentication instance of an empty +// GetModes returns the modes of the Authentication instance, or an empty // list if it is nil func (a *Authentication) GetModes() []string { if a == nil { diff --git a/controllers/om/process/om_process.go b/controllers/om/process/om_process.go index 20dc293ec..87dfb9df9 100644 --- a/controllers/om/process/om_process.go +++ b/controllers/om/process/om_process.go @@ -22,6 +22,18 @@ func CreateMongodProcessesWithLimit(mongoDBImage string, forceEnterprise bool, s return processes } +// CreateMongodProcessesFromMongoDB creates mongod processes directly from MongoDB resource without StatefulSet +func CreateMongodProcessesFromMongoDB(mongoDBImage string, forceEnterprise bool, mdb *mdbv1.MongoDB, limit int, fcv string, tlsCertPath string) []om.Process { + hostnames, names := dns.GetDNSNames(mdb.Name, mdb.ServiceName(), mdb.Namespace, mdb.Spec.GetClusterDomain(), limit, mdb.Spec.DbCommonSpec.GetExternalDomain()) + processes := make([]om.Process, len(hostnames)) + + for idx, hostname := range hostnames { + processes[idx] = om.NewMongodProcess(names[idx], hostname, mongoDBImage, forceEnterprise, mdb.Spec.GetAdditionalMongodConfig(), &mdb.Spec, tlsCertPath, mdb.Annotations, fcv) + } + + return processes +} + // CreateMongodProcessesWithLimitMulti creates the process array for automationConfig based on MultiCluster CR spec func CreateMongodProcessesWithLimitMulti(mongoDBImage string, forceEnterprise bool, mrs mdbmultiv1.MongoDBMultiCluster, certFileName string) ([]om.Process, error) { hostnames := make([]string, 0) diff --git a/controllers/om/replicaset/om_replicaset.go b/controllers/om/replicaset/om_replicaset.go index 2e72d2c3e..a30e86726 100644 --- a/controllers/om/replicaset/om_replicaset.go +++ b/controllers/om/replicaset/om_replicaset.go @@ -30,6 +30,16 @@ func BuildFromStatefulSetWithReplicas(mongoDBImage string, forceEnterprise bool, return rsWithProcesses } +// BuildFromMongoDBWithReplicas returns a replica set that can be set in the Automation Config +// based on the given MongoDB resource directly without requiring a StatefulSet. +func BuildFromMongoDBWithReplicas(mongoDBImage string, forceEnterprise bool, mdb *mdbv1.MongoDB, replicas int, fcv string, tlsCertPath string) om.ReplicaSetWithProcesses { + members := process.CreateMongodProcessesFromMongoDB(mongoDBImage, forceEnterprise, mdb, replicas, fcv, tlsCertPath) + replicaSet := om.NewReplicaSet(mdb.Name, mdb.Spec.GetMongoDBVersion()) + rsWithProcesses := om.NewReplicaSetWithProcesses(replicaSet, members, mdb.Spec.GetMemberOptions()) + rsWithProcesses.SetHorizons(mdb.Spec.GetHorizonConfig()) + return rsWithProcesses +} + // PrepareScaleDownFromMap performs additional steps necessary to make sure removed members are not primary (so no // election happens and replica set is available) (see // https://jira.mongodb.org/browse/HELP-3818?focusedCommentId=1548348 for more details) diff --git a/controllers/operator/agents/agents.go b/controllers/operator/agents/agents.go index c5eae2656..7366d6e83 100644 --- a/controllers/operator/agents/agents.go +++ b/controllers/operator/agents/agents.go @@ -96,6 +96,19 @@ func WaitForRsAgentsToRegister(set appsv1.StatefulSet, members int, clusterName return nil } +// WaitForRsAgentsToRegisterByResource waits for RS agents to register using MongoDB resource directly without StatefulSet +func WaitForRsAgentsToRegisterByResource(rs *mdbv1.MongoDB, members int, omConnection om.Connection, log *zap.SugaredLogger) error { + hostnames, _ := dns.GetDNSNames(rs.Name, rs.ServiceName(), rs.Namespace, rs.Spec.GetClusterDomain(), members, rs.Spec.DbCommonSpec.GetExternalDomain()) + + log = log.With("mongodb", rs.Name) + + ok, msg := waitUntilRegistered(omConnection, log, retryParams{retrials: 5, waitSeconds: 3}, hostnames...) + if !ok { + return getAgentRegisterError(msg) + } + return nil +} + // WaitForRsAgentsToRegisterSpecifiedHostnames waits for the specified agents to registry with Ops Manager. func WaitForRsAgentsToRegisterSpecifiedHostnames(omConnection om.Connection, hostnames []string, log *zap.SugaredLogger) error { ok, msg := waitUntilRegistered(omConnection, log, retryParams{retrials: 10, waitSeconds: 9}, hostnames...) diff --git a/controllers/operator/mongodbmultireplicaset_controller.go b/controllers/operator/mongodbmultireplicaset_controller.go index aad481b32..e1b82783c 100644 --- a/controllers/operator/mongodbmultireplicaset_controller.go +++ b/controllers/operator/mongodbmultireplicaset_controller.go @@ -152,6 +152,7 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request if err != nil { return r.updateStatus(ctx, &mrs, workflow.Failed(xerrors.Errorf("error establishing connection to Ops Manager: %w", err)), log) } + // TODO: need to ensure supported OM version ? log = log.With("MemberCluster Namespace", mrs.Namespace) @@ -165,11 +166,9 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request } r.SetupCommonWatchers(&mrs, nil, nil, mrs.Name) + // TODO: Need equivalent of validateMongoDBResource and checkIfHasExcessProcesses here ? - publishAutomationConfigFirst, err := r.publishAutomationConfigFirstMultiCluster(ctx, &mrs, log) - if err != nil { - return r.updateStatus(ctx, &mrs, workflow.Failed(err), log) - } + // TODO: ensure feature controls ? // If tls is enabled we need to configure the "processes" array in opsManager/Cloud Manager with the // correct tlsCertPath, with the new tls design, this path has the certHash in it(so that cert can be rotated @@ -192,6 +191,8 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request } } + // TODO: agentCertSecretSelector ? + agentCertSecretName := mrs.GetSecurity().AgentClientCertificateSecretName(mrs.GetName()) agentCertHash, agentCertPath := r.agentCertHashAndPath(ctx, log, mrs.Namespace, agentCertSecretName, "") @@ -210,6 +211,12 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request } } + // TODO: review conditions for publishing AC first + publishAutomationConfigFirst, err := r.publishAutomationConfigFirstMultiCluster(ctx, &mrs, log) + if err != nil { + return r.updateStatus(ctx, &mrs, workflow.Failed(err), log) + } + status := workflow.RunInGivenOrder(publishAutomationConfigFirst, func() workflow.Status { if err := r.updateOmDeploymentRs(ctx, conn, mrs, agentCertPath, tlsCertPath, internalClusterCertPath, false, log); err != nil { @@ -260,6 +267,10 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request return r.updateStatus(ctx, &mrs, workflow.OK(), log, mdbstatus.NewPVCsStatusOptionEmptyStatus()) } +// TODO: in multi cluster, the below function only checks a subset of conditions compared to the shared "publishAutomationConfigFirst" in common_controller.go, +// +// which is used for single cluster replica set. Why, and are these enough ? +// // publishAutomationConfigFirstMultiCluster returns a boolean indicating whether Ops Manager // needs to be updated before the StatefulSets are created for this resource. func (r *ReconcileMongoDbMultiReplicaSet) publishAutomationConfigFirstMultiCluster(ctx context.Context, mrs *mdbmultiv1.MongoDBMultiCluster, log *zap.SugaredLogger) (bool, error) { diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 470e56716..c6bbcb890 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -3,6 +3,8 @@ package operator import ( "context" "fmt" + kubernetesClient "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/kube/client" + "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/kube/container" "go.uber.org/zap" "golang.org/x/xerrors" @@ -89,6 +91,18 @@ func newReplicaSetReconciler(ctx context.Context, kubeClient client.Client, imag } } +// TODO: move above, and keep only relevant fields. Taken from sharded controller +type deploymentOptionsRS struct { + //podEnvVars *env.PodEnvVars + //currentAgentAuthMode string + //caFilePath string + agentCertPath string + //certTLSType map[string]bool + //finalizing bool + //processNames []string + prometheusCertHash string +} + // Generic Kubernetes Resources // +kubebuilder:rbac:groups=core,resources=namespaces,verbs=list;watch,namespace=placeholder // +kubebuilder:rbac:groups=core,resources=pods,verbs=get;list;watch,namespace=placeholder @@ -108,6 +122,7 @@ func newReplicaSetReconciler(ctx context.Context, kubeClient client.Client, imag // Reconcile reads that state of the cluster for a MongoDbReplicaSet object and makes changes based on the state read // and what is in the MongoDbReplicaSet.Spec func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reconcile.Request) (res reconcile.Result, e error) { + // === 1. Initial Checks and setup log := zap.S().With("ReplicaSet", request.NamespacedName) rs := &mdbv1.MongoDB{} @@ -155,153 +170,97 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, status, log) } - status := certs.EnsureSSLCertsForStatefulSet(ctx, r.SecretClient, r.SecretClient, *rs.Spec.Security, certs.ReplicaSetConfig(*rs), log) - if !status.IsOK() { - return r.updateStatus(ctx, rs, status, log) - } - - prometheusCertHash, err := certs.EnsureTLSCertsForPrometheus(ctx, r.SecretClient, rs.GetNamespace(), rs.GetPrometheus(), certs.Database, log) - if err != nil { - log.Infof("Could not generate certificates for Prometheus: %s", err) - return r.updateStatus(ctx, rs, workflow.Pending("%s", err.Error()), log) - } - if status := controlledfeature.EnsureFeatureControls(*rs, conn, conn.OpsManagerVersion(), log); !status.IsOK() { return r.updateStatus(ctx, rs, status, log) } - currentAgentAuthMode, err := conn.GetAgentAuthMode() - if err != nil { - return r.updateStatus(ctx, rs, workflow.Failed(err), log) - } - - certConfigurator := certs.ReplicaSetX509CertConfigurator{MongoDB: rs, SecretClient: r.SecretClient} - status = r.ensureX509SecretAndCheckTLSType(ctx, certConfigurator, currentAgentAuthMode, log) - if !status.IsOK() { - return r.updateStatus(ctx, rs, status, log) - } + // === 2. Auth and Certificates + // Get certificate paths for later use rsCertsConfig := certs.ReplicaSetConfig(*rs) - - var vaultConfig vault.VaultConfiguration var databaseSecretPath string if r.VaultClient != nil { - vaultConfig = r.VaultClient.VaultConfig databaseSecretPath = r.VaultClient.DatabaseSecretPath() } - - var automationAgentVersion string - if architectures.IsRunningStaticArchitecture(rs.Annotations) { - // In case the Agent *is* overridden, its version will be merged into the StatefulSet. The merging process - // happens after creating the StatefulSet definition. - if !rs.IsAgentImageOverridden() { - automationAgentVersion, err = r.getAgentVersion(conn, conn.OpsManagerVersion().VersionString, false, log) - if err != nil { - log.Errorf("Impossible to get agent version, please override the agent image by providing a pod template") - status := workflow.Failed(xerrors.Errorf("Failed to get agent version: %w", err)) - return r.updateStatus(ctx, rs, status, log) - } - } - } - tlsCertHash := enterprisepem.ReadHashFromSecret(ctx, r.SecretClient, rs.Namespace, rsCertsConfig.CertSecretName, databaseSecretPath, log) internalClusterCertHash := enterprisepem.ReadHashFromSecret(ctx, r.SecretClient, rs.Namespace, rsCertsConfig.InternalClusterSecretName, databaseSecretPath, log) agentCertSecretName := rs.GetSecurity().AgentClientCertificateSecretName(rs.Name) - agentCertHash, agentCertPath := r.agentCertHashAndPath(ctx, log, rs.Namespace, agentCertSecretName, databaseSecretPath) - rsConfig := construct.ReplicaSetOptions( - PodEnvVars(newPodVars(conn, projectConfig, rs.Spec.LogLevel)), - CurrentAgentAuthMechanism(currentAgentAuthMode), - CertificateHash(tlsCertHash), - AgentCertHash(agentCertHash), - InternalClusterHash(internalClusterCertHash), - PrometheusTLSCertHash(prometheusCertHash), - WithVaultConfig(vaultConfig), - WithLabels(rs.Labels), - WithAdditionalMongodConfig(rs.Spec.GetAdditionalMongodConfig()), - WithInitDatabaseNonStaticImage(images.ContainerImage(r.imageUrls, util.InitDatabaseImageUrlEnv, r.initDatabaseNonStaticImageVersion)), - WithDatabaseNonStaticImage(images.ContainerImage(r.imageUrls, util.NonStaticDatabaseEnterpriseImage, r.databaseNonStaticImageVersion)), - WithAgentImage(images.ContainerImage(r.imageUrls, architectures.MdbAgentImageRepo, automationAgentVersion)), - WithMongodbImage(images.GetOfficialImage(r.imageUrls, rs.Spec.Version, rs.GetAnnotations())), - ) - caFilePath := fmt.Sprintf("%s/ca-pem", util.TLSCaMountPath) + tlsCertPath := "" + internalClusterCertPath := "" + if internalClusterCertHash != "" { + internalClusterCertPath = fmt.Sprintf("%s%s", util.InternalClusterAuthMountPath, internalClusterCertHash) + } + if tlsCertHash != "" { + tlsCertPath = fmt.Sprintf("%s/%s", util.TLSCertMountPath, tlsCertHash) + } - if err := r.reconcileHostnameOverrideConfigMap(ctx, log, r.client, *rs); err != nil { - return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("Failed to reconcileHostnameOverrideConfigMap: %w", err)), log) + // TODO: (just as a note for current refactoring), we have the following limitation: The Kubernetes Operator doesn't + // support integration with Prometheus for *multi-cluster* replica sets. + prometheusCertHash, err := certs.EnsureTLSCertsForPrometheus(ctx, r.SecretClient, rs.GetNamespace(), rs.GetPrometheus(), certs.Database, log) + if err != nil { + log.Infof("Could not generate certificates for Prometheus: %s", err) + return r.updateStatus(ctx, rs, workflow.Failed(err), log) } - shouldMirrorKeyfile := r.applySearchOverrides(ctx, rs, log) + // TODO: commented to simplify refactoring, because it might not be needed (not done in MC RS reconciler) + //// Check if we need to prepare for scale-down + //if scale.ReplicasThisReconciliation(rs) < rs.Status.Members { + // if err := replicaset.PrepareScaleDownFromStatefulSet(conn, sts, rs, log); err != nil { + // return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("Failed to prepare Replica Set for scaling down using Ops Manager: %w", err)), log) + // } + //} - sts := construct.DatabaseStatefulSet(*rs, rsConfig, log) - if status := r.ensureRoles(ctx, rs.Spec.DbCommonSpec, r.enableClusterMongoDBRoles, conn, kube.ObjectKeyFromApiObject(rs), log); !status.IsOK() { - return r.updateStatus(ctx, rs, status, log) - } + agentCertSecretSelector := rs.GetSecurity().AgentClientCertificateSecretName(rs.Name) + agentCertSecretSelector.Name += certs.OperatorGeneratedCertSuffix - if scale.ReplicasThisReconciliation(rs) < rs.Status.Members { - if err := replicaset.PrepareScaleDownFromStatefulSet(conn, sts, rs, log); err != nil { - return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("Failed to prepare Replica Set for scaling down using Ops Manager: %w", err)), log) - } - } + agentCertHash, agentCertPath := r.agentCertHashAndPath(ctx, log, rs.Namespace, agentCertSecretName, databaseSecretPath) - internalClusterCertPath := "" - if internalClusterCertHash != "" { - internalClusterCertPath = fmt.Sprintf("%s%s", util.InternalClusterAuthMountPath, internalClusterCertHash) + deploymentOpts := deploymentOptionsRS{ + prometheusCertHash: prometheusCertHash, + agentCertPath: agentCertPath, } - tlsCertPath := "" - if tlsCertHash != "" { - tlsCertPath = fmt.Sprintf("%s/%s", util.TLSCertMountPath, tlsCertHash) - } + // 4. Recovery // Recovery prevents some deadlocks that can occur during reconciliation, e.g. the setting of an incorrect automation // configuration and a subsequent attempt to overwrite it later, the operator would be stuck in Pending phase. // See CLOUDP-189433 and CLOUDP-229222 for more details. if recovery.ShouldTriggerRecovery(rs.Status.Phase != mdbstatus.PhaseRunning, rs.Status.LastTransition) { log.Warnf("Triggering Automatic Recovery. The MongoDB resource %s/%s is in %s state since %s", rs.Namespace, rs.Name, rs.Status.Phase, rs.Status.LastTransition) - automationConfigStatus := r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, sts, log, agentCertPath, caFilePath, tlsCertPath, internalClusterCertPath, prometheusCertHash, true, shouldMirrorKeyfile).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") - deploymentError := create.DatabaseInKubernetes(ctx, r.client, *rs, sts, rsConfig, log) - if deploymentError != nil { - log.Errorf("Recovery failed because of deployment errors, %w", deploymentError) + automationConfigStatus := r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, true).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") + reconcileStatus := r.reconcileMemberResources(ctx, rs, conn, log, projectConfig, deploymentOpts) + if !reconcileStatus.IsOK() { + log.Errorf("Recovery failed because of reconcile errors, %v", reconcileStatus) } if !automationConfigStatus.IsOK() { log.Errorf("Recovery failed because of Automation Config update errors, %v", automationConfigStatus) } } + // 5. Actual kube resources deployment and OM updates lastSpec, err := rs.GetLastSpec() if err != nil { lastSpec = &mdbv1.MongoDbSpec{} } - status = workflow.RunInGivenOrder(publishAutomationConfigFirst(ctx, r.client, *rs, lastSpec, rsConfig, log), + + // TODO: review conditions for publishing AC first, get rid of need for rsConfig (opts) + // make a special function for rs ? This one is in the common controller + publishAutomationConfigFirst := publishAutomationConfigFirstRS(ctx, r.client, *rs, lastSpec, log) + status := workflow.RunInGivenOrder(publishAutomationConfigFirst, func() workflow.Status { - return r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, sts, log, agentCertPath, caFilePath, tlsCertPath, internalClusterCertPath, prometheusCertHash, false, shouldMirrorKeyfile).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") + return r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, false).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") }, func() workflow.Status { - workflowStatus := create.HandlePVCResize(ctx, r.client, &sts, log) - if !workflowStatus.IsOK() { - return workflowStatus - } - if workflow.ContainsPVCOption(workflowStatus.StatusOptions()) { - _, _ = r.updateStatus(ctx, rs, workflow.Pending(""), log, workflowStatus.StatusOptions()...) - } - - if err := create.DatabaseInKubernetes(ctx, r.client, *rs, sts, rsConfig, log); err != nil { - return workflow.Failed(xerrors.Errorf("Failed to create/update (Kubernetes reconciliation phase): %w", err)) - } - - if status := statefulset.GetStatefulSetStatus(ctx, rs.Namespace, rs.Name, r.client); !status.IsOK() { - return status - } - - log.Info("Updated StatefulSet for replica set") - return workflow.OK() + return r.reconcileMemberResources(ctx, rs, conn, log, projectConfig, deploymentOpts) }) if !status.IsOK() { return r.updateStatus(ctx, rs, status, log) } + // === 6. Final steps if scale.IsStillScaling(rs) { return r.updateStatus(ctx, rs, workflow.Pending("Continuing scaling operation for ReplicaSet %s, desiredMembers=%d, currentMembers=%d", rs.ObjectKey(), rs.DesiredReplicas(), scale.ReplicasThisReconciliation(rs)), log, mdbstatus.MembersOption(rs)) } @@ -333,6 +292,60 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, workflow.OK(), log, mdbstatus.NewBaseUrlOption(deployment.Link(conn.BaseURL(), conn.GroupID())), mdbstatus.MembersOption(rs), mdbstatus.NewPVCsStatusOptionEmptyStatus()) } +func publishAutomationConfigFirstRS(ctx context.Context, getter kubernetesClient.Client, mdb mdbv1.MongoDB, lastSpec *mdbv1.MongoDbSpec, log *zap.SugaredLogger) bool { + namespacedName := kube.ObjectKey(mdb.Namespace, mdb.Name) + currentSts, err := getter.GetStatefulSet(ctx, namespacedName) + if err != nil { + if errors.IsNotFound(err) { + // No need to publish state as this is a new StatefulSet + log.Debugf("New StatefulSet %s", namespacedName) + return false + } + + log.Debugw(fmt.Sprintf("Error getting StatefulSet %s", namespacedName), "error", err) + return false + } + + databaseContainer := container.GetByName(util.DatabaseContainerName, currentSts.Spec.Template.Spec.Containers) + volumeMounts := databaseContainer.VolumeMounts + + if !mdb.Spec.Security.IsTLSEnabled() && wasTLSSecretMounted(ctx, getter, currentSts, mdb, log) { + log.Debug(automationConfigFirstMsg("security.tls.enabled", "false")) + return true + } + + if mdb.Spec.Security.TLSConfig.CA == "" && wasCAConfigMapMounted(ctx, getter, currentSts, mdb, log) { + log.Debug(automationConfigFirstMsg("security.tls.CA", "empty")) + return true + + } + + // TODO: for now removing the need for sslmmscaconfigmap. Check for regression, and clarify for the need later + //if opts.PodVars.SSLMMSCAConfigMap == "" && statefulset.VolumeMountWithNameExists(volumeMounts, construct.CaCertName) { + // log.Debug(automationConfigFirstMsg("SSLMMSCAConfigMap", "empty")) + // return true + //} + + // TODO: passed empty string to GetAgentMechanism for now, how to know the current one without relying on `opts` ? + if mdb.Spec.Security.GetAgentMechanism("") != util.X509 && statefulset.VolumeMountWithNameExists(volumeMounts, util.AgentSecretName) { + log.Debug(automationConfigFirstMsg("project.AuthMode", "empty")) + return true + } + + if mdb.Spec.Members < int(*currentSts.Spec.Replicas) { + log.Debug("Scaling down operation. automationConfig needs to be updated first") + return true + } + + if architectures.IsRunningStaticArchitecture(mdb.GetAnnotations()) { + if mdb.Spec.IsInChangeVersion(lastSpec) { + return true + } + } + + return false +} + func getHostnameOverrideConfigMapForReplicaset(mdb mdbv1.MongoDB) corev1.ConfigMap { data := make(map[string]string) @@ -368,6 +381,125 @@ func (r *ReconcileMongoDbReplicaSet) reconcileHostnameOverrideConfigMap(ctx cont return nil } +// reconcileMemberResources handles the synchronization of kubernetes resources, which can be statefulsets, services etc. +// All the resources required in the k8s cluster (as opposed to the automation config) for creating the replicaset +// should be reconciled in this method. +func (r *ReconcileMongoDbReplicaSet) reconcileMemberResources(ctx context.Context, rs *mdbv1.MongoDB, conn om.Connection, + log *zap.SugaredLogger, projectConfig mdbv1.ProjectConfig, deploymentOptions deploymentOptionsRS) workflow.Status { + // Reconcile hostname override ConfigMap + if err := r.reconcileHostnameOverrideConfigMap(ctx, log, r.client, *rs); err != nil { + return workflow.Failed(xerrors.Errorf("Failed to reconcileHostnameOverrideConfigMap: %w", err)) + } + + // Ensure roles are properly configured + if status := r.ensureRoles(ctx, rs.Spec.DbCommonSpec, r.enableClusterMongoDBRoles, conn, kube.ObjectKeyFromApiObject(rs), log); !status.IsOK() { + return status + } + + if status := r.reconcileStatefulSet(ctx, rs, log, conn, projectConfig, deploymentOptions); !status.IsOK() { + return status + } + + return workflow.OK() + +} + +func (r *ReconcileMongoDbReplicaSet) reconcileStatefulSet(ctx context.Context, rs *mdbv1.MongoDB, + log *zap.SugaredLogger, conn om.Connection, projectConfig mdbv1.ProjectConfig, deploymentOptions deploymentOptionsRS) workflow.Status { + + currentAgentAuthMode, err := conn.GetAgentAuthMode() + if err != nil { + return workflow.Failed(xerrors.Errorf("failed to get agent auth mode: %w", err)) + } + + certConfigurator := certs.ReplicaSetX509CertConfigurator{MongoDB: rs, SecretClient: r.SecretClient} + status := r.ensureX509SecretAndCheckTLSType(ctx, certConfigurator, currentAgentAuthMode, log) + if !status.IsOK() { + return status + } + + status = certs.EnsureSSLCertsForStatefulSet(ctx, r.SecretClient, r.SecretClient, *rs.Spec.Security, certs.ReplicaSetConfig(*rs), log) + if !status.IsOK() { + return status + } + + // Build the replica set config + rsConfig, err := r.buildStatefulSetOptions(ctx, rs, conn, projectConfig, currentAgentAuthMode, deploymentOptions.prometheusCertHash, log) + if err != nil { + return workflow.Failed(xerrors.Errorf("failed to build StatefulSet options: %w", err)) + } + + sts := construct.DatabaseStatefulSet(*rs, rsConfig, log) + + // Handle PVC resize if needed + workflowStatus := create.HandlePVCResize(ctx, r.client, &sts, log) + if !workflowStatus.IsOK() { + return workflowStatus + } + if workflow.ContainsPVCOption(workflowStatus.StatusOptions()) { + _, _ = r.updateStatus(ctx, rs, workflow.Pending(""), log, workflowStatus.StatusOptions()...) + } + + // Create or update the StatefulSet in Kubernetes + if err := create.DatabaseInKubernetes(ctx, r.client, *rs, sts, rsConfig, log); err != nil { + return workflow.Failed(xerrors.Errorf("Failed to create/update (Kubernetes reconciliation phase): %w", err)) + } + + // Check StatefulSet status + if status := statefulset.GetStatefulSetStatus(ctx, rs.Namespace, rs.Name, r.client); !status.IsOK() { + return status + } + + log.Info("Updated StatefulSet for replica set") + return workflow.OK() +} + +// buildStatefulSetOptions creates the options needed for constructing the StatefulSet +func (r *ReconcileMongoDbReplicaSet) buildStatefulSetOptions(ctx context.Context, rs *mdbv1.MongoDB, conn om.Connection, projectConfig mdbv1.ProjectConfig, currentAgentAuthMode string, prometheusCertHash string, log *zap.SugaredLogger) (func(mdb mdbv1.MongoDB) construct.DatabaseStatefulSetOptions, error) { + rsCertsConfig := certs.ReplicaSetConfig(*rs) + + var vaultConfig vault.VaultConfiguration + var databaseSecretPath string + if r.VaultClient != nil { + vaultConfig = r.VaultClient.VaultConfig + databaseSecretPath = r.VaultClient.DatabaseSecretPath() + } + + // Determine automation agent version for static architecture + var automationAgentVersion string + if architectures.IsRunningStaticArchitecture(rs.Annotations) { + // In case the Agent *is* overridden, its version will be merged into the StatefulSet. The merging process + // happens after creating the StatefulSet definition. + if !rs.IsAgentImageOverridden() { + var err error + automationAgentVersion, err = r.getAgentVersion(conn, conn.OpsManagerVersion().VersionString, false, log) + if err != nil { + return nil, xerrors.Errorf("Impossible to get agent version, please override the agent image by providing a pod template: %w", err) + } + } + } + + tlsCertHash := enterprisepem.ReadHashFromSecret(ctx, r.SecretClient, rs.Namespace, rsCertsConfig.CertSecretName, databaseSecretPath, log) + internalClusterCertHash := enterprisepem.ReadHashFromSecret(ctx, r.SecretClient, rs.Namespace, rsCertsConfig.InternalClusterSecretName, databaseSecretPath, log) + + rsConfig := construct.ReplicaSetOptions( + PodEnvVars(newPodVars(conn, projectConfig, rs.Spec.LogLevel)), + CurrentAgentAuthMechanism(currentAgentAuthMode), + CertificateHash(tlsCertHash), + InternalClusterHash(internalClusterCertHash), + PrometheusTLSCertHash(prometheusCertHash), + WithVaultConfig(vaultConfig), + WithLabels(rs.Labels), + WithAdditionalMongodConfig(rs.Spec.GetAdditionalMongodConfig()), + WithInitDatabaseNonStaticImage(images.ContainerImage(r.imageUrls, util.InitDatabaseImageUrlEnv, r.initDatabaseNonStaticImageVersion)), + WithDatabaseNonStaticImage(images.ContainerImage(r.imageUrls, util.NonStaticDatabaseEnterpriseImage, r.databaseNonStaticImageVersion)), + WithAgentImage(images.ContainerImage(r.imageUrls, architectures.MdbAgentImageRepo, automationAgentVersion)), + WithMongodbImage(images.GetOfficialImage(r.imageUrls, rs.Spec.Version, rs.GetAnnotations())), + ) + + return rsConfig, nil +} + // AddReplicaSetController creates a new MongoDbReplicaset Controller and adds it to the Manager. The Manager will set fields on the Controller // and Start it when the Manager is Started. func AddReplicaSetController(ctx context.Context, mgr manager.Manager, imageUrls images.ImageUrls, initDatabaseNonStaticImageVersion, databaseNonStaticImageVersion string, forceEnterprise bool, enableClusterMongoDBRoles bool) error { @@ -449,19 +581,21 @@ func AddReplicaSetController(ctx context.Context, mgr manager.Manager, imageUrls // updateOmDeploymentRs performs OM registration operation for the replicaset. So the changes will be finally propagated // to automation agents in containers -func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, conn om.Connection, membersNumberBefore int, rs *mdbv1.MongoDB, set appsv1.StatefulSet, log *zap.SugaredLogger, agentCertPath, caFilePath, tlsCertPath, internalClusterCertPath string, prometheusCertHash string, isRecovering bool, shouldMirrorKeyfileForMongot bool) workflow.Status { +func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, conn om.Connection, membersNumberBefore int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, tlsCertPath, internalClusterCertPath string, deploymentOptionsRS deploymentOptionsRS, isRecovering bool) workflow.Status { log.Debug("Entering UpdateOMDeployments") // Only "concrete" RS members should be observed // - if scaling down, let's observe only members that will remain after scale-down operation // - if scaling up, observe only current members, because new ones might not exist yet - err := agents.WaitForRsAgentsToRegister(set, util_int.Min(membersNumberBefore, int(*set.Spec.Replicas)), rs.Spec.GetClusterDomain(), conn, log, rs) + replicasTarget := scale.ReplicasThisReconciliation(rs) + err := agents.WaitForRsAgentsToRegisterByResource(rs, util_int.Min(membersNumberBefore, replicasTarget), conn, log) if err != nil && !isRecovering { return workflow.Failed(err) } + caFilePath := fmt.Sprintf("%s/ca-pem", util.TLSCaMountPath) // If current operation is to Disable TLS, then we should the current members of the Replica Set, // this is, do not scale them up or down util TLS disabling has completed. - shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(conn, r.imageUrls[mcoConstruct.MongodbImageEnv], r.forceEnterprise, membersNumberBefore, rs, set, log, caFilePath, tlsCertPath) + shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(conn, r.imageUrls[mcoConstruct.MongodbImageEnv], r.forceEnterprise, membersNumberBefore, rs, log, caFilePath, tlsCertPath) if err != nil && !isRecovering { return workflow.Failed(err) } @@ -472,13 +606,13 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c // TLS to be completely disabled first. updatedMembers = membersNumberBefore } else { - updatedMembers = int(*set.Spec.Replicas) + updatedMembers = replicasTarget } - replicaSet := replicaset.BuildFromStatefulSetWithReplicas(r.imageUrls[mcoConstruct.MongodbImageEnv], r.forceEnterprise, set, rs.GetSpec(), updatedMembers, rs.CalculateFeatureCompatibilityVersion(), tlsCertPath) + replicaSet := replicaset.BuildFromMongoDBWithReplicas(r.imageUrls[mcoConstruct.MongodbImageEnv], r.forceEnterprise, rs, updatedMembers, rs.CalculateFeatureCompatibilityVersion(), tlsCertPath) processNames := replicaSet.GetProcessNames() - status, additionalReconciliationRequired := r.updateOmAuthentication(ctx, conn, processNames, rs, agentCertPath, caFilePath, internalClusterCertPath, isRecovering, log) + status, additionalReconciliationRequired := r.updateOmAuthentication(ctx, conn, processNames, rs, deploymentOptionsRS.agentCertPath, caFilePath, internalClusterCertPath, isRecovering, log) if !status.IsOK() && !isRecovering { return status } @@ -488,12 +622,12 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c return workflow.Failed(err) } - p := PrometheusConfiguration{ + prometheusConfiguration := PrometheusConfiguration{ prometheus: rs.GetPrometheus(), conn: conn, secretsClient: r.SecretClient, namespace: rs.GetNamespace(), - prometheusCertHash: prometheusCertHash, + prometheusCertHash: deploymentOptionsRS.prometheusCertHash, } err = conn.ReadUpdateDeployment( @@ -503,7 +637,7 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c return err } } - return ReconcileReplicaSetAC(ctx, d, rs.Spec.DbCommonSpec, lastRsConfig.ToMap(), rs.Name, replicaSet, caFilePath, internalClusterCertPath, &p, log) + return ReconcileReplicaSetAC(ctx, d, rs.Spec.DbCommonSpec, lastRsConfig.ToMap(), rs.Name, replicaSet, caFilePath, internalClusterCertPath, &prometheusConfiguration, log) }, log, ) @@ -525,9 +659,8 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c return workflow.Pending("Performing multi stage reconciliation") } - externalDomain := rs.Spec.DbCommonSpec.GetExternalDomain() - hostsBefore := getAllHostsRs(set, rs.Spec.GetClusterDomain(), membersNumberBefore, externalDomain) - hostsAfter := getAllHostsRs(set, rs.Spec.GetClusterDomain(), scale.ReplicasThisReconciliation(rs), externalDomain) + hostsBefore := getAllHostsForReplicas(rs, membersNumberBefore) + hostsAfter := getAllHostsForReplicas(rs, scale.ReplicasThisReconciliation(rs)) if err := host.CalculateDiffAndStopMonitoring(conn, hostsBefore, hostsAfter, log); err != nil && !isRecovering { return workflow.Failed(err) @@ -544,7 +677,7 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c // updateOmDeploymentDisableTLSConfiguration checks if TLS configuration needs // to be disabled. In which case it will disable it and inform to the calling // function. -func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage string, forceEnterprise bool, membersNumberBefore int, rs *mdbv1.MongoDB, set appsv1.StatefulSet, log *zap.SugaredLogger, caFilePath, tlsCertPath string) (bool, error) { +func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage string, forceEnterprise bool, membersNumberBefore int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, caFilePath, tlsCertPath string) (bool, error) { tlsConfigWasDisabled := false err := conn.ReadUpdateDeployment( @@ -558,7 +691,7 @@ func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage // configure as many agents/Pods as we currently have, no more (in case // there's a scale up change at the same time). - replicaSet := replicaset.BuildFromStatefulSetWithReplicas(mongoDBImage, forceEnterprise, set, rs.GetSpec(), membersNumberBefore, rs.CalculateFeatureCompatibilityVersion(), tlsCertPath) + replicaSet := replicaset.BuildFromMongoDBWithReplicas(mongoDBImage, forceEnterprise, rs, membersNumberBefore, rs.CalculateFeatureCompatibilityVersion(), tlsCertPath) lastConfig, err := rs.GetLastAdditionalMongodConfigByType(mdbv1.ReplicaSetConfig) if err != nil { @@ -639,8 +772,8 @@ func (r *ReconcileMongoDbReplicaSet) OnDelete(ctx context.Context, obj runtime.O return nil } -func getAllHostsRs(set appsv1.StatefulSet, clusterName string, membersCount int, externalDomain *string) []string { - hostnames, _ := dns.GetDnsForStatefulSetReplicasSpecified(set, clusterName, membersCount, externalDomain) +func getAllHostsForReplicas(rs *mdbv1.MongoDB, membersCount int) []string { + hostnames, _ := dns.GetDNSNames(rs.Name, rs.ServiceName(), rs.Namespace, rs.Spec.GetClusterDomain(), membersCount, rs.Spec.DbCommonSpec.GetExternalDomain()) return hostnames } diff --git a/controllers/operator/mongodbreplicaset_controller_test.go b/controllers/operator/mongodbreplicaset_controller_test.go index e85e7050d..b9ad0ddab 100644 --- a/controllers/operator/mongodbreplicaset_controller_test.go +++ b/controllers/operator/mongodbreplicaset_controller_test.go @@ -3,6 +3,7 @@ package operator import ( "context" "fmt" + "github.com/mongodb/mongodb-kubernetes/controllers/operator/controlledfeature" "reflect" "testing" "time" @@ -30,8 +31,7 @@ import ( "github.com/mongodb/mongodb-kubernetes/controllers/om/backup" "github.com/mongodb/mongodb-kubernetes/controllers/om/deployment" "github.com/mongodb/mongodb-kubernetes/controllers/operator/authentication" - "github.com/mongodb/mongodb-kubernetes/controllers/operator/construct" - "github.com/mongodb/mongodb-kubernetes/controllers/operator/controlledfeature" + //"github.com/mongodb/mongodb-kubernetes/controllers/operator/controlledfeature" "github.com/mongodb/mongodb-kubernetes/controllers/operator/create" "github.com/mongodb/mongodb-kubernetes/controllers/operator/mock" "github.com/mongodb/mongodb-kubernetes/controllers/operator/pem" @@ -396,26 +396,26 @@ func TestUpdateDeploymentTLSConfiguration(t *testing.T) { rsNoTLS := mdbv1.NewReplicaSetBuilder().Build() deploymentWithTLS := deployment.CreateFromReplicaSet("fake-mongoDBImage", false, rsWithTLS) deploymentNoTLS := deployment.CreateFromReplicaSet("fake-mongoDBImage", false, rsNoTLS) - stsWithTLS := construct.DatabaseStatefulSet(*rsWithTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) - stsNoTLS := construct.DatabaseStatefulSet(*rsNoTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) + //stsWithTLS := construct.DatabaseStatefulSet(*rsWithTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) + //stsNoTLS := construct.DatabaseStatefulSet(*rsNoTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) // TLS Disabled -> TLS Disabled - shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentNoTLS), "fake-mongoDBImage", false, 3, rsNoTLS, stsNoTLS, zap.S(), util.CAFilePathInContainer, "") + shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentNoTLS), "fake-mongoDBImage", false, 3, rsNoTLS, zap.S(), util.CAFilePathInContainer, "") assert.NoError(t, err) assert.False(t, shouldLockMembers) // TLS Disabled -> TLS Enabled - shouldLockMembers, err = updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentNoTLS), "fake-mongoDBImage", false, 3, rsWithTLS, stsWithTLS, zap.S(), util.CAFilePathInContainer, "") + shouldLockMembers, err = updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentNoTLS), "fake-mongoDBImage", false, 3, rsWithTLS, zap.S(), util.CAFilePathInContainer, "") assert.NoError(t, err) assert.False(t, shouldLockMembers) // TLS Enabled -> TLS Enabled - shouldLockMembers, err = updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentWithTLS), "fake-mongoDBImage", false, 3, rsWithTLS, stsWithTLS, zap.S(), util.CAFilePathInContainer, "") + shouldLockMembers, err = updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentWithTLS), "fake-mongoDBImage", false, 3, rsWithTLS, zap.S(), util.CAFilePathInContainer, "") assert.NoError(t, err) assert.False(t, shouldLockMembers) // TLS Enabled -> TLS Disabled - shouldLockMembers, err = updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentWithTLS), "fake-mongoDBImage", false, 3, rsNoTLS, stsNoTLS, zap.S(), util.CAFilePathInContainer, "") + shouldLockMembers, err = updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentWithTLS), "fake-mongoDBImage", false, 3, rsNoTLS, zap.S(), util.CAFilePathInContainer, "") assert.NoError(t, err) assert.True(t, shouldLockMembers) } From 41965291532cedf3c4706e393b655cf7e6075c5e Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 10:34:55 +0200 Subject: [PATCH 02/25] Fix and improve certificates handling --- .../operator/mongodbreplicaset_controller.go | 38 +++++++++---------- 1 file changed, 17 insertions(+), 21 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index c6bbcb890..3a32e9af0 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -97,6 +97,7 @@ type deploymentOptionsRS struct { //currentAgentAuthMode string //caFilePath string agentCertPath string + agentCertHash string //certTLSType map[string]bool //finalizing bool //processNames []string @@ -184,8 +185,6 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco } tlsCertHash := enterprisepem.ReadHashFromSecret(ctx, r.SecretClient, rs.Namespace, rsCertsConfig.CertSecretName, databaseSecretPath, log) internalClusterCertHash := enterprisepem.ReadHashFromSecret(ctx, r.SecretClient, rs.Namespace, rsCertsConfig.InternalClusterSecretName, databaseSecretPath, log) - agentCertSecretName := rs.GetSecurity().AgentClientCertificateSecretName(rs.Name) - tlsCertPath := "" internalClusterCertPath := "" @@ -196,6 +195,9 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco tlsCertPath = fmt.Sprintf("%s/%s", util.TLSCertMountPath, tlsCertHash) } + agentCertSecretName := rs.GetSecurity().AgentClientCertificateSecretName(rs.Name) + agentCertHash, agentCertPath := r.agentCertHashAndPath(ctx, log, rs.Namespace, agentCertSecretName, databaseSecretPath) + // TODO: (just as a note for current refactoring), we have the following limitation: The Kubernetes Operator doesn't // support integration with Prometheus for *multi-cluster* replica sets. prometheusCertHash, err := certs.EnsureTLSCertsForPrometheus(ctx, r.SecretClient, rs.GetNamespace(), rs.GetPrometheus(), certs.Database, log) @@ -204,6 +206,12 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, workflow.Failed(err), log) } + deploymentOpts := deploymentOptionsRS{ + prometheusCertHash: prometheusCertHash, + agentCertPath: agentCertPath, + agentCertHash: agentCertHash, + } + // TODO: commented to simplify refactoring, because it might not be needed (not done in MC RS reconciler) //// Check if we need to prepare for scale-down //if scale.ReplicasThisReconciliation(rs) < rs.Status.Members { @@ -212,16 +220,6 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco // } //} - agentCertSecretSelector := rs.GetSecurity().AgentClientCertificateSecretName(rs.Name) - agentCertSecretSelector.Name += certs.OperatorGeneratedCertSuffix - - agentCertHash, agentCertPath := r.agentCertHashAndPath(ctx, log, rs.Namespace, agentCertSecretName, databaseSecretPath) - - deploymentOpts := deploymentOptionsRS{ - prometheusCertHash: prometheusCertHash, - agentCertPath: agentCertPath, - } - // 4. Recovery // Recovery prevents some deadlocks that can occur during reconciliation, e.g. the setting of an incorrect automation @@ -396,12 +394,7 @@ func (r *ReconcileMongoDbReplicaSet) reconcileMemberResources(ctx context.Contex return status } - if status := r.reconcileStatefulSet(ctx, rs, log, conn, projectConfig, deploymentOptions); !status.IsOK() { - return status - } - - return workflow.OK() - + return r.reconcileStatefulSet(ctx, rs, log, conn, projectConfig, deploymentOptions) } func (r *ReconcileMongoDbReplicaSet) reconcileStatefulSet(ctx context.Context, rs *mdbv1.MongoDB, @@ -424,7 +417,7 @@ func (r *ReconcileMongoDbReplicaSet) reconcileStatefulSet(ctx context.Context, r } // Build the replica set config - rsConfig, err := r.buildStatefulSetOptions(ctx, rs, conn, projectConfig, currentAgentAuthMode, deploymentOptions.prometheusCertHash, log) + rsConfig, err := r.buildStatefulSetOptions(ctx, rs, conn, projectConfig, currentAgentAuthMode, deploymentOptions.prometheusCertHash, deploymentOptions.agentCertHash, log) if err != nil { return workflow.Failed(xerrors.Errorf("failed to build StatefulSet options: %w", err)) } @@ -455,7 +448,7 @@ func (r *ReconcileMongoDbReplicaSet) reconcileStatefulSet(ctx context.Context, r } // buildStatefulSetOptions creates the options needed for constructing the StatefulSet -func (r *ReconcileMongoDbReplicaSet) buildStatefulSetOptions(ctx context.Context, rs *mdbv1.MongoDB, conn om.Connection, projectConfig mdbv1.ProjectConfig, currentAgentAuthMode string, prometheusCertHash string, log *zap.SugaredLogger) (func(mdb mdbv1.MongoDB) construct.DatabaseStatefulSetOptions, error) { +func (r *ReconcileMongoDbReplicaSet) buildStatefulSetOptions(ctx context.Context, rs *mdbv1.MongoDB, conn om.Connection, projectConfig mdbv1.ProjectConfig, currentAgentAuthMode string, prometheusCertHash string, agentCertHash string, log *zap.SugaredLogger) (func(mdb mdbv1.MongoDB) construct.DatabaseStatefulSetOptions, error) { rsCertsConfig := certs.ReplicaSetConfig(*rs) var vaultConfig vault.VaultConfiguration @@ -486,6 +479,7 @@ func (r *ReconcileMongoDbReplicaSet) buildStatefulSetOptions(ctx context.Context PodEnvVars(newPodVars(conn, projectConfig, rs.Spec.LogLevel)), CurrentAgentAuthMechanism(currentAgentAuthMode), CertificateHash(tlsCertHash), + AgentCertHash(agentCertHash), InternalClusterHash(internalClusterCertHash), PrometheusTLSCertHash(prometheusCertHash), WithVaultConfig(vaultConfig), @@ -630,9 +624,11 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c prometheusCertHash: deploymentOptionsRS.prometheusCertHash, } + shouldMirrorKeyfile := r.applySearchOverrides(ctx, rs, log) + err = conn.ReadUpdateDeployment( func(d om.Deployment) error { - if shouldMirrorKeyfileForMongot { + if shouldMirrorKeyfile { if err := r.mirrorKeyfileIntoSecretForMongot(ctx, d, rs, log); err != nil { return err } From 1eead9439bb4a1d556e47d40b2f249aad4a48ce6 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 11:22:47 +0200 Subject: [PATCH 03/25] Pass current agent auth mode --- .../operator/mongodbreplicaset_controller.go | 25 ++++++------------- 1 file changed, 8 insertions(+), 17 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 3a32e9af0..f4fc75be2 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -243,9 +243,12 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco lastSpec = &mdbv1.MongoDbSpec{} } - // TODO: review conditions for publishing AC first, get rid of need for rsConfig (opts) - // make a special function for rs ? This one is in the common controller - publishAutomationConfigFirst := publishAutomationConfigFirstRS(ctx, r.client, *rs, lastSpec, log) + currentAgentAuthMode, err := conn.GetAgentAuthMode() + if err != nil { + return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("failed to get agent auth mode: %w", err)), log) + } + + publishAutomationConfigFirst := publishAutomationConfigFirstRS(ctx, r.client, *rs, lastSpec, currentAgentAuthMode, log) status := workflow.RunInGivenOrder(publishAutomationConfigFirst, func() workflow.Status { return r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, false).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") @@ -290,7 +293,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, workflow.OK(), log, mdbstatus.NewBaseUrlOption(deployment.Link(conn.BaseURL(), conn.GroupID())), mdbstatus.MembersOption(rs), mdbstatus.NewPVCsStatusOptionEmptyStatus()) } -func publishAutomationConfigFirstRS(ctx context.Context, getter kubernetesClient.Client, mdb mdbv1.MongoDB, lastSpec *mdbv1.MongoDbSpec, log *zap.SugaredLogger) bool { +func publishAutomationConfigFirstRS(ctx context.Context, getter kubernetesClient.Client, mdb mdbv1.MongoDB, lastSpec *mdbv1.MongoDbSpec, currentAgentAuthMode string, log *zap.SugaredLogger) bool { namespacedName := kube.ObjectKey(mdb.Namespace, mdb.Name) currentSts, err := getter.GetStatefulSet(ctx, namespacedName) if err != nil { @@ -318,14 +321,7 @@ func publishAutomationConfigFirstRS(ctx context.Context, getter kubernetesClient } - // TODO: for now removing the need for sslmmscaconfigmap. Check for regression, and clarify for the need later - //if opts.PodVars.SSLMMSCAConfigMap == "" && statefulset.VolumeMountWithNameExists(volumeMounts, construct.CaCertName) { - // log.Debug(automationConfigFirstMsg("SSLMMSCAConfigMap", "empty")) - // return true - //} - - // TODO: passed empty string to GetAgentMechanism for now, how to know the current one without relying on `opts` ? - if mdb.Spec.Security.GetAgentMechanism("") != util.X509 && statefulset.VolumeMountWithNameExists(volumeMounts, util.AgentSecretName) { + if mdb.Spec.Security.GetAgentMechanism(currentAgentAuthMode) != util.X509 && statefulset.VolumeMountWithNameExists(volumeMounts, util.AgentSecretName) { log.Debug(automationConfigFirstMsg("project.AuthMode", "empty")) return true } @@ -400,11 +396,6 @@ func (r *ReconcileMongoDbReplicaSet) reconcileMemberResources(ctx context.Contex func (r *ReconcileMongoDbReplicaSet) reconcileStatefulSet(ctx context.Context, rs *mdbv1.MongoDB, log *zap.SugaredLogger, conn om.Connection, projectConfig mdbv1.ProjectConfig, deploymentOptions deploymentOptionsRS) workflow.Status { - currentAgentAuthMode, err := conn.GetAgentAuthMode() - if err != nil { - return workflow.Failed(xerrors.Errorf("failed to get agent auth mode: %w", err)) - } - certConfigurator := certs.ReplicaSetX509CertConfigurator{MongoDB: rs, SecretClient: r.SecretClient} status := r.ensureX509SecretAndCheckTLSType(ctx, certConfigurator, currentAgentAuthMode, log) if !status.IsOK() { From e913144efd463a87d799efcc314c7ca8786b30e2 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 11:48:52 +0200 Subject: [PATCH 04/25] currentAgentAuthMode in deploymentOptions --- .../operator/mongodbreplicaset_controller.go | 37 ++++++++----------- 1 file changed, 16 insertions(+), 21 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index f4fc75be2..ed946b61f 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -93,15 +93,10 @@ func newReplicaSetReconciler(ctx context.Context, kubeClient client.Client, imag // TODO: move above, and keep only relevant fields. Taken from sharded controller type deploymentOptionsRS struct { - //podEnvVars *env.PodEnvVars - //currentAgentAuthMode string - //caFilePath string - agentCertPath string - agentCertHash string - //certTLSType map[string]bool - //finalizing bool - //processNames []string - prometheusCertHash string + agentCertPath string + agentCertHash string + prometheusCertHash string + currentAgentAuthMode string } // Generic Kubernetes Resources @@ -206,10 +201,9 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, workflow.Failed(err), log) } - deploymentOpts := deploymentOptionsRS{ - prometheusCertHash: prometheusCertHash, - agentCertPath: agentCertPath, - agentCertHash: agentCertHash, + currentAgentAuthMode, err := conn.GetAgentAuthMode() + if err != nil { + return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("failed to get agent auth mode: %w", err)), log) } // TODO: commented to simplify refactoring, because it might not be needed (not done in MC RS reconciler) @@ -219,6 +213,12 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco // return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("Failed to prepare Replica Set for scaling down using Ops Manager: %w", err)), log) // } //} + deploymentOpts := deploymentOptionsRS{ + prometheusCertHash: prometheusCertHash, + agentCertPath: agentCertPath, + agentCertHash: agentCertHash, + currentAgentAuthMode: currentAgentAuthMode, + } // 4. Recovery @@ -243,12 +243,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco lastSpec = &mdbv1.MongoDbSpec{} } - currentAgentAuthMode, err := conn.GetAgentAuthMode() - if err != nil { - return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("failed to get agent auth mode: %w", err)), log) - } - - publishAutomationConfigFirst := publishAutomationConfigFirstRS(ctx, r.client, *rs, lastSpec, currentAgentAuthMode, log) + publishAutomationConfigFirst := publishAutomationConfigFirstRS(ctx, r.client, *rs, lastSpec, deploymentOpts.currentAgentAuthMode, log) status := workflow.RunInGivenOrder(publishAutomationConfigFirst, func() workflow.Status { return r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, false).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") @@ -397,7 +392,7 @@ func (r *ReconcileMongoDbReplicaSet) reconcileStatefulSet(ctx context.Context, r log *zap.SugaredLogger, conn om.Connection, projectConfig mdbv1.ProjectConfig, deploymentOptions deploymentOptionsRS) workflow.Status { certConfigurator := certs.ReplicaSetX509CertConfigurator{MongoDB: rs, SecretClient: r.SecretClient} - status := r.ensureX509SecretAndCheckTLSType(ctx, certConfigurator, currentAgentAuthMode, log) + status := r.ensureX509SecretAndCheckTLSType(ctx, certConfigurator, deploymentOptions.currentAgentAuthMode, log) if !status.IsOK() { return status } @@ -408,7 +403,7 @@ func (r *ReconcileMongoDbReplicaSet) reconcileStatefulSet(ctx context.Context, r } // Build the replica set config - rsConfig, err := r.buildStatefulSetOptions(ctx, rs, conn, projectConfig, currentAgentAuthMode, deploymentOptions.prometheusCertHash, deploymentOptions.agentCertHash, log) + rsConfig, err := r.buildStatefulSetOptions(ctx, rs, conn, projectConfig, deploymentOptions.currentAgentAuthMode, deploymentOptions.prometheusCertHash, deploymentOptions.agentCertHash, log) if err != nil { return workflow.Failed(xerrors.Errorf("failed to build StatefulSet options: %w", err)) } From 74cc89a986ecd7dcf2f55ff74d2ed4908d61d5d4 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 13:03:16 +0200 Subject: [PATCH 05/25] PrepareScaleDown without need for sts --- controllers/om/replicaset/om_replicaset.go | 4 ++-- .../operator/mongodbreplicaset_controller.go | 13 ++++++------- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/controllers/om/replicaset/om_replicaset.go b/controllers/om/replicaset/om_replicaset.go index a30e86726..9a9804adc 100644 --- a/controllers/om/replicaset/om_replicaset.go +++ b/controllers/om/replicaset/om_replicaset.go @@ -97,8 +97,8 @@ func PrepareScaleDownFromMap(omClient om.Connection, rsMembers map[string][]stri return nil } -func PrepareScaleDownFromStatefulSet(omClient om.Connection, statefulSet appsv1.StatefulSet, rs *mdbv1.MongoDB, log *zap.SugaredLogger) error { - _, podNames := dns.GetDnsForStatefulSetReplicasSpecified(statefulSet, rs.Spec.GetClusterDomain(), rs.Status.Members, nil) +func PrepareScaleDownFromMongoDB(omClient om.Connection, rs *mdbv1.MongoDB, log *zap.SugaredLogger) error { + _, podNames := dns.GetDNSNames(rs.Name, rs.ServiceName(), rs.Namespace, rs.Spec.GetClusterDomain(), rs.Status.Members, rs.Spec.DbCommonSpec.GetExternalDomain()) podNames = podNames[scale.ReplicasThisReconciliation(rs):rs.Status.Members] if len(podNames) != 1 { diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index ed946b61f..4a5910d2f 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -206,13 +206,12 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("failed to get agent auth mode: %w", err)), log) } - // TODO: commented to simplify refactoring, because it might not be needed (not done in MC RS reconciler) - //// Check if we need to prepare for scale-down - //if scale.ReplicasThisReconciliation(rs) < rs.Status.Members { - // if err := replicaset.PrepareScaleDownFromStatefulSet(conn, sts, rs, log); err != nil { - // return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("Failed to prepare Replica Set for scaling down using Ops Manager: %w", err)), log) - // } - //} + // Check if we need to prepare for scale-down + if scale.ReplicasThisReconciliation(rs) < rs.Status.Members { + if err := replicaset.PrepareScaleDownFromMongoDB(conn, rs, log); err != nil { + return r.updateStatus(ctx, rs, workflow.Failed(xerrors.Errorf("Failed to prepare Replica Set for scaling down using Ops Manager: %w", err)), log) + } + } deploymentOpts := deploymentOptionsRS{ prometheusCertHash: prometheusCertHash, agentCertPath: agentCertPath, From dd4756f7f4c585e17c66edfb8b07b0fc153be94d Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 13:56:19 +0200 Subject: [PATCH 06/25] Pass configmap to publishAutomationConfigFirstRS directly --- controllers/operator/mongodbreplicaset_controller.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 4a5910d2f..50bc55239 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -242,7 +242,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco lastSpec = &mdbv1.MongoDbSpec{} } - publishAutomationConfigFirst := publishAutomationConfigFirstRS(ctx, r.client, *rs, lastSpec, deploymentOpts.currentAgentAuthMode, log) + publishAutomationConfigFirst := publishAutomationConfigFirstRS(ctx, r.client, *rs, lastSpec, deploymentOpts.currentAgentAuthMode, projectConfig.SSLMMSCAConfigMap, log) status := workflow.RunInGivenOrder(publishAutomationConfigFirst, func() workflow.Status { return r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, false).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") @@ -287,7 +287,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, workflow.OK(), log, mdbstatus.NewBaseUrlOption(deployment.Link(conn.BaseURL(), conn.GroupID())), mdbstatus.MembersOption(rs), mdbstatus.NewPVCsStatusOptionEmptyStatus()) } -func publishAutomationConfigFirstRS(ctx context.Context, getter kubernetesClient.Client, mdb mdbv1.MongoDB, lastSpec *mdbv1.MongoDbSpec, currentAgentAuthMode string, log *zap.SugaredLogger) bool { +func publishAutomationConfigFirstRS(ctx context.Context, getter kubernetesClient.Client, mdb mdbv1.MongoDB, lastSpec *mdbv1.MongoDbSpec, currentAgentAuthMode string, sslMMSCAConfigMap string, log *zap.SugaredLogger) bool { namespacedName := kube.ObjectKey(mdb.Namespace, mdb.Name) currentSts, err := getter.GetStatefulSet(ctx, namespacedName) if err != nil { @@ -312,7 +312,11 @@ func publishAutomationConfigFirstRS(ctx context.Context, getter kubernetesClient if mdb.Spec.Security.TLSConfig.CA == "" && wasCAConfigMapMounted(ctx, getter, currentSts, mdb, log) { log.Debug(automationConfigFirstMsg("security.tls.CA", "empty")) return true + } + if sslMMSCAConfigMap == "" && statefulset.VolumeMountWithNameExists(volumeMounts, construct.CaCertName) { + log.Debug(automationConfigFirstMsg("SSLMMSCAConfigMap", "empty")) + return true } if mdb.Spec.Security.GetAgentMechanism(currentAgentAuthMode) != util.X509 && statefulset.VolumeMountWithNameExists(volumeMounts, util.AgentSecretName) { From c3ef579d76a36cb5f1b754674414a3ecf5ea4dc3 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 14:08:38 +0200 Subject: [PATCH 07/25] Lint --- controllers/operator/mongodbreplicaset_controller.go | 11 ++++++----- .../operator/mongodbreplicaset_controller_test.go | 7 +++---- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 50bc55239..c376479cd 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -3,8 +3,6 @@ package operator import ( "context" "fmt" - kubernetesClient "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/kube/client" - "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/kube/container" "go.uber.org/zap" "golang.org/x/xerrors" @@ -48,7 +46,9 @@ import ( "github.com/mongodb/mongodb-kubernetes/controllers/searchcontroller" mcoConstruct "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/controllers/construct" "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/kube/annotations" + kubernetesClient "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/kube/client" "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/kube/configmap" + "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/kube/container" "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/util/merge" "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/util/scale" "github.com/mongodb/mongodb-kubernetes/pkg/dns" @@ -377,7 +377,8 @@ func (r *ReconcileMongoDbReplicaSet) reconcileHostnameOverrideConfigMap(ctx cont // All the resources required in the k8s cluster (as opposed to the automation config) for creating the replicaset // should be reconciled in this method. func (r *ReconcileMongoDbReplicaSet) reconcileMemberResources(ctx context.Context, rs *mdbv1.MongoDB, conn om.Connection, - log *zap.SugaredLogger, projectConfig mdbv1.ProjectConfig, deploymentOptions deploymentOptionsRS) workflow.Status { + log *zap.SugaredLogger, projectConfig mdbv1.ProjectConfig, deploymentOptions deploymentOptionsRS, +) workflow.Status { // Reconcile hostname override ConfigMap if err := r.reconcileHostnameOverrideConfigMap(ctx, log, r.client, *rs); err != nil { return workflow.Failed(xerrors.Errorf("Failed to reconcileHostnameOverrideConfigMap: %w", err)) @@ -392,8 +393,8 @@ func (r *ReconcileMongoDbReplicaSet) reconcileMemberResources(ctx context.Contex } func (r *ReconcileMongoDbReplicaSet) reconcileStatefulSet(ctx context.Context, rs *mdbv1.MongoDB, - log *zap.SugaredLogger, conn om.Connection, projectConfig mdbv1.ProjectConfig, deploymentOptions deploymentOptionsRS) workflow.Status { - + log *zap.SugaredLogger, conn om.Connection, projectConfig mdbv1.ProjectConfig, deploymentOptions deploymentOptionsRS, +) workflow.Status { certConfigurator := certs.ReplicaSetX509CertConfigurator{MongoDB: rs, SecretClient: r.SecretClient} status := r.ensureX509SecretAndCheckTLSType(ctx, certConfigurator, deploymentOptions.currentAgentAuthMode, log) if !status.IsOK() { diff --git a/controllers/operator/mongodbreplicaset_controller_test.go b/controllers/operator/mongodbreplicaset_controller_test.go index b9ad0ddab..5633faf62 100644 --- a/controllers/operator/mongodbreplicaset_controller_test.go +++ b/controllers/operator/mongodbreplicaset_controller_test.go @@ -3,7 +3,6 @@ package operator import ( "context" "fmt" - "github.com/mongodb/mongodb-kubernetes/controllers/operator/controlledfeature" "reflect" "testing" "time" @@ -31,7 +30,7 @@ import ( "github.com/mongodb/mongodb-kubernetes/controllers/om/backup" "github.com/mongodb/mongodb-kubernetes/controllers/om/deployment" "github.com/mongodb/mongodb-kubernetes/controllers/operator/authentication" - //"github.com/mongodb/mongodb-kubernetes/controllers/operator/controlledfeature" + "github.com/mongodb/mongodb-kubernetes/controllers/operator/controlledfeature" "github.com/mongodb/mongodb-kubernetes/controllers/operator/create" "github.com/mongodb/mongodb-kubernetes/controllers/operator/mock" "github.com/mongodb/mongodb-kubernetes/controllers/operator/pem" @@ -396,8 +395,8 @@ func TestUpdateDeploymentTLSConfiguration(t *testing.T) { rsNoTLS := mdbv1.NewReplicaSetBuilder().Build() deploymentWithTLS := deployment.CreateFromReplicaSet("fake-mongoDBImage", false, rsWithTLS) deploymentNoTLS := deployment.CreateFromReplicaSet("fake-mongoDBImage", false, rsNoTLS) - //stsWithTLS := construct.DatabaseStatefulSet(*rsWithTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) - //stsNoTLS := construct.DatabaseStatefulSet(*rsNoTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) + // stsWithTLS := construct.DatabaseStatefulSet(*rsWithTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) + // stsNoTLS := construct.DatabaseStatefulSet(*rsNoTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) // TLS Disabled -> TLS Disabled shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentNoTLS), "fake-mongoDBImage", false, 3, rsNoTLS, zap.S(), util.CAFilePathInContainer, "") From e41d24a08b84da4f7711712b7f853bcf0444742e Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 14:14:20 +0200 Subject: [PATCH 08/25] Commit TODOs --- controllers/operator/mongodbmultireplicaset_controller.go | 1 + controllers/operator/mongodbreplicaset_controller.go | 1 + 2 files changed, 2 insertions(+) diff --git a/controllers/operator/mongodbmultireplicaset_controller.go b/controllers/operator/mongodbmultireplicaset_controller.go index e1b82783c..a854568bf 100644 --- a/controllers/operator/mongodbmultireplicaset_controller.go +++ b/controllers/operator/mongodbmultireplicaset_controller.go @@ -530,6 +530,7 @@ func (r *ReconcileMongoDbMultiReplicaSet) reconcileStatefulSets(ctx context.Cont } } + // TODO: make a separate function similar to `buildStatefulSetOptions` in single rs controller? opts := mconstruct.MultiClusterReplicaSetOptions( mconstruct.WithClusterNum(clusterNum), Replicas(replicasThisReconciliation), diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index c376479cd..3461dd8e6 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -265,6 +265,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, workflow.Failed(err), log) } + // TODO: note for refactoring - vault is not supported on multi cluster RS yet if vault.IsVaultSecretBackend() { secrets := rs.GetSecretsMountedIntoDBPod() vaultMap := make(map[string]string) From c89617b7b12224715a1ee1014f720dff393b112a Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 14:27:13 +0200 Subject: [PATCH 09/25] Remove TODOs --- .../operator/mongodbmultireplicaset_controller.go | 12 ------------ controllers/operator/mongodbreplicaset_controller.go | 4 ---- 2 files changed, 16 deletions(-) diff --git a/controllers/operator/mongodbmultireplicaset_controller.go b/controllers/operator/mongodbmultireplicaset_controller.go index a854568bf..4ea1ee832 100644 --- a/controllers/operator/mongodbmultireplicaset_controller.go +++ b/controllers/operator/mongodbmultireplicaset_controller.go @@ -152,7 +152,6 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request if err != nil { return r.updateStatus(ctx, &mrs, workflow.Failed(xerrors.Errorf("error establishing connection to Ops Manager: %w", err)), log) } - // TODO: need to ensure supported OM version ? log = log.With("MemberCluster Namespace", mrs.Namespace) @@ -166,9 +165,6 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request } r.SetupCommonWatchers(&mrs, nil, nil, mrs.Name) - // TODO: Need equivalent of validateMongoDBResource and checkIfHasExcessProcesses here ? - - // TODO: ensure feature controls ? // If tls is enabled we need to configure the "processes" array in opsManager/Cloud Manager with the // correct tlsCertPath, with the new tls design, this path has the certHash in it(so that cert can be rotated @@ -191,8 +187,6 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request } } - // TODO: agentCertSecretSelector ? - agentCertSecretName := mrs.GetSecurity().AgentClientCertificateSecretName(mrs.GetName()) agentCertHash, agentCertPath := r.agentCertHashAndPath(ctx, log, mrs.Namespace, agentCertSecretName, "") @@ -211,7 +205,6 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request } } - // TODO: review conditions for publishing AC first publishAutomationConfigFirst, err := r.publishAutomationConfigFirstMultiCluster(ctx, &mrs, log) if err != nil { return r.updateStatus(ctx, &mrs, workflow.Failed(err), log) @@ -267,10 +260,6 @@ func (r *ReconcileMongoDbMultiReplicaSet) Reconcile(ctx context.Context, request return r.updateStatus(ctx, &mrs, workflow.OK(), log, mdbstatus.NewPVCsStatusOptionEmptyStatus()) } -// TODO: in multi cluster, the below function only checks a subset of conditions compared to the shared "publishAutomationConfigFirst" in common_controller.go, -// -// which is used for single cluster replica set. Why, and are these enough ? -// // publishAutomationConfigFirstMultiCluster returns a boolean indicating whether Ops Manager // needs to be updated before the StatefulSets are created for this resource. func (r *ReconcileMongoDbMultiReplicaSet) publishAutomationConfigFirstMultiCluster(ctx context.Context, mrs *mdbmultiv1.MongoDBMultiCluster, log *zap.SugaredLogger) (bool, error) { @@ -530,7 +519,6 @@ func (r *ReconcileMongoDbMultiReplicaSet) reconcileStatefulSets(ctx context.Cont } } - // TODO: make a separate function similar to `buildStatefulSetOptions` in single rs controller? opts := mconstruct.MultiClusterReplicaSetOptions( mconstruct.WithClusterNum(clusterNum), Replicas(replicasThisReconciliation), diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 3461dd8e6..aec6f09ad 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -91,7 +91,6 @@ func newReplicaSetReconciler(ctx context.Context, kubeClient client.Client, imag } } -// TODO: move above, and keep only relevant fields. Taken from sharded controller type deploymentOptionsRS struct { agentCertPath string agentCertHash string @@ -193,8 +192,6 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco agentCertSecretName := rs.GetSecurity().AgentClientCertificateSecretName(rs.Name) agentCertHash, agentCertPath := r.agentCertHashAndPath(ctx, log, rs.Namespace, agentCertSecretName, databaseSecretPath) - // TODO: (just as a note for current refactoring), we have the following limitation: The Kubernetes Operator doesn't - // support integration with Prometheus for *multi-cluster* replica sets. prometheusCertHash, err := certs.EnsureTLSCertsForPrometheus(ctx, r.SecretClient, rs.GetNamespace(), rs.GetPrometheus(), certs.Database, log) if err != nil { log.Infof("Could not generate certificates for Prometheus: %s", err) @@ -265,7 +262,6 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco return r.updateStatus(ctx, rs, workflow.Failed(err), log) } - // TODO: note for refactoring - vault is not supported on multi cluster RS yet if vault.IsVaultSecretBackend() { secrets := rs.GetSecretsMountedIntoDBPod() vaultMap := make(map[string]string) From eb78918ed27cbeaeeaa2f6ad72d320e3e56bd1de Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 14:56:14 +0200 Subject: [PATCH 10/25] Lint --- api/v1/search/zz_generated.deepcopy.go | 8 +- .../mongodb.com_clustermongodbroles.yaml | 2 +- config/crd/bases/mongodb.com_mongodb.yaml | 48 +------- .../mongodb.com_mongodbmulticluster.yaml | 48 +------- .../crd/bases/mongodb.com_mongodbsearch.yaml | 86 +------------- .../crd/bases/mongodb.com_mongodbusers.yaml | 2 +- config/crd/bases/mongodb.com_opsmanagers.yaml | 112 ++---------------- ...ommunity.mongodb.com_mongodbcommunity.yaml | 54 +-------- controllers/om/deployment/testing_utils.go | 4 +- 9 files changed, 36 insertions(+), 328 deletions(-) diff --git a/api/v1/search/zz_generated.deepcopy.go b/api/v1/search/zz_generated.deepcopy.go index c66322146..d18d025f8 100644 --- a/api/v1/search/zz_generated.deepcopy.go +++ b/api/v1/search/zz_generated.deepcopy.go @@ -159,7 +159,7 @@ func (in *MongoDBSearchSpec) DeepCopyInto(out *MongoDBSearchSpec) { *out = new(v1.ResourceRequirements) (*in).DeepCopyInto(*out) } - out.Security = in.Security + in.Security.DeepCopyInto(&out.Security) } // DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new MongoDBSearchSpec. @@ -231,7 +231,11 @@ func (in *MongoDBSource) DeepCopy() *MongoDBSource { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *Security) DeepCopyInto(out *Security) { *out = *in - out.TLS = in.TLS + if in.TLS != nil { + in, out := &in.TLS, &out.TLS + *out = new(TLS) + **out = **in + } } // DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new Security. diff --git a/config/crd/bases/mongodb.com_clustermongodbroles.yaml b/config/crd/bases/mongodb.com_clustermongodbroles.yaml index 3d583bcfd..9241b7dad 100644 --- a/config/crd/bases/mongodb.com_clustermongodbroles.yaml +++ b/config/crd/bases/mongodb.com_clustermongodbroles.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.18.0 + controller-gen.kubebuilder.io/version: v0.15.0 name: clustermongodbroles.mongodb.com spec: group: mongodb.com diff --git a/config/crd/bases/mongodb.com_mongodb.yaml b/config/crd/bases/mongodb.com_mongodb.yaml index d421d8837..2a7076877 100644 --- a/config/crd/bases/mongodb.com_mongodb.yaml +++ b/config/crd/bases/mongodb.com_mongodb.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.18.0 + controller-gen.kubebuilder.io/version: v0.15.0 name: mongodb.mongodb.com spec: group: mongodb.com @@ -1410,29 +1410,7 @@ spec: automationLdapGroupDN: type: string automationPasswordSecretRef: - description: SecretKeySelector selects a key of a Secret. - properties: - key: - description: The key of the secret to select from. Must - be a valid secret key. - type: string - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - optional: - description: Specify whether the Secret or its key - must be defined - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic + $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' automationUserName: type: string clientCertificateSecretRef: @@ -1467,29 +1445,9 @@ spec: bindQueryUser: type: string caConfigMapRef: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ConfigMapKeySelector' description: Allows to point at a ConfigMap/key with a CA file to mount on the Pod - properties: - key: - description: The key to select. - type: string - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - optional: - description: Specify whether the ConfigMap or its - key must be defined - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic servers: items: type: string diff --git a/config/crd/bases/mongodb.com_mongodbmulticluster.yaml b/config/crd/bases/mongodb.com_mongodbmulticluster.yaml index 3f1fa05c9..01fe3f2e6 100644 --- a/config/crd/bases/mongodb.com_mongodbmulticluster.yaml +++ b/config/crd/bases/mongodb.com_mongodbmulticluster.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.18.0 + controller-gen.kubebuilder.io/version: v0.15.0 name: mongodbmulticluster.mongodb.com spec: group: mongodb.com @@ -670,29 +670,7 @@ spec: automationLdapGroupDN: type: string automationPasswordSecretRef: - description: SecretKeySelector selects a key of a Secret. - properties: - key: - description: The key of the secret to select from. Must - be a valid secret key. - type: string - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - optional: - description: Specify whether the Secret or its key - must be defined - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic + $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' automationUserName: type: string clientCertificateSecretRef: @@ -727,29 +705,9 @@ spec: bindQueryUser: type: string caConfigMapRef: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ConfigMapKeySelector' description: Allows to point at a ConfigMap/key with a CA file to mount on the Pod - properties: - key: - description: The key to select. - type: string - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - optional: - description: Specify whether the ConfigMap or its - key must be defined - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic servers: items: type: string diff --git a/config/crd/bases/mongodb.com_mongodbsearch.yaml b/config/crd/bases/mongodb.com_mongodbsearch.yaml index 15153ba25..096cb7bb7 100644 --- a/config/crd/bases/mongodb.com_mongodbsearch.yaml +++ b/config/crd/bases/mongodb.com_mongodbsearch.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.18.0 + controller-gen.kubebuilder.io/version: v0.15.0 name: mongodbsearch.mongodb.com spec: group: mongodb.com @@ -107,65 +107,9 @@ spec: type: object type: object resourceRequirements: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ResourceRequirements' description: Configure resource requests and limits for the MongoDB Search pods. - properties: - claims: - description: |- - Claims lists the names of resources, defined in spec.resourceClaims, - that are used by this container. - - This is an alpha field and requires enabling the - DynamicResourceAllocation feature gate. - - This field is immutable. It can only be set for containers. - items: - description: ResourceClaim references one entry in PodSpec.ResourceClaims. - properties: - name: - description: |- - Name must match the name of one entry in pod.spec.resourceClaims of - the Pod where this field is used. It makes that resource available - inside a container. - type: string - request: - description: |- - Request is the name chosen for a request in the referenced claim. - If empty, everything from the claim is made available, otherwise - only the result of this request. - type: string - required: - - name - type: object - type: array - x-kubernetes-list-map-keys: - - name - x-kubernetes-list-type: map - limits: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - description: |- - Limits describes the maximum amount of compute resources allowed. - More info: https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/ - type: object - requests: - additionalProperties: - anyOf: - - type: integer - - type: string - pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ - x-kubernetes-int-or-string: true - description: |- - Requests describes the minimum amount of compute resources required. - If Requests is omitted for a container, it defaults to Limits if that is explicitly specified, - otherwise to an implementation-defined value. Requests cannot exceed Limits. - More info: https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/ - type: object - type: object security: description: Configure security settings of the MongoDB Search server that MongoDB database is connecting to when performing search queries. @@ -173,22 +117,11 @@ spec: tls: properties: certificateKeySecretRef: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CertificateKeySecret is a reference to a Secret containing a private key and certificate to use for TLS. The key and cert are expected to be PEM encoded and available at "tls.key" and "tls.crt". This is the same format used for the standard "kubernetes.io/tls" Secret type, but no specific type is required. - properties: - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - type: object - x-kubernetes-map-type: atomic required: - certificateKeySecretRef type: object @@ -218,21 +151,10 @@ spec: description: TLS configuration for the external MongoDB deployment properties: ca: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CA is a reference to a Secret containing the CA certificate that issued mongod's TLS certificate. The CA certificate is expected to be PEM encoded and available at the "ca.crt" key. - properties: - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - type: object - x-kubernetes-map-type: atomic required: - ca type: object diff --git a/config/crd/bases/mongodb.com_mongodbusers.yaml b/config/crd/bases/mongodb.com_mongodbusers.yaml index 89713ce7f..a81f0d449 100644 --- a/config/crd/bases/mongodb.com_mongodbusers.yaml +++ b/config/crd/bases/mongodb.com_mongodbusers.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.18.0 + controller-gen.kubebuilder.io/version: v0.15.0 name: mongodbusers.mongodb.com spec: group: mongodb.com diff --git a/config/crd/bases/mongodb.com_opsmanagers.yaml b/config/crd/bases/mongodb.com_opsmanagers.yaml index 3ace001da..c830b9a24 100644 --- a/config/crd/bases/mongodb.com_opsmanagers.yaml +++ b/config/crd/bases/mongodb.com_opsmanagers.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.18.0 + controller-gen.kubebuilder.io/version: v0.15.0 name: opsmanagers.mongodb.com spec: group: mongodb.com @@ -730,30 +730,7 @@ spec: automationLdapGroupDN: type: string automationPasswordSecretRef: - description: SecretKeySelector selects a key of a - Secret. - properties: - key: - description: The key of the secret to select from. Must - be a valid secret key. - type: string - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - optional: - description: Specify whether the Secret or its - key must be defined - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic + $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' automationUserName: type: string clientCertificateSecretRef: @@ -789,29 +766,9 @@ spec: bindQueryUser: type: string caConfigMapRef: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ConfigMapKeySelector' description: Allows to point at a ConfigMap/key with a CA file to mount on the Pod - properties: - key: - description: The key to select. - type: string - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - optional: - description: Specify whether the ConfigMap or - its key must be defined - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic servers: items: type: string @@ -1236,29 +1193,7 @@ spec: CustomCertificateSecretRefs is a list of valid Certificate Authority certificate secrets that apply to the associated S3 bucket. items: - description: SecretKeySelector selects a key of a Secret. - properties: - key: - description: The key of the secret to select from. Must - be a valid secret key. - type: string - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - optional: - description: Specify whether the Secret or its key - must be defined - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic + $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' type: array irsaEnabled: description: |- @@ -1328,29 +1263,7 @@ spec: CustomCertificateSecretRefs is a list of valid Certificate Authority certificate secrets that apply to the associated S3 bucket. items: - description: SecretKeySelector selects a key of a Secret. - properties: - key: - description: The key of the secret to select from. Must - be a valid secret key. - type: string - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - optional: - description: Specify whether the Secret or its key - must be defined - type: boolean - required: - - key - type: object - x-kubernetes-map-type: atomic + $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' type: array irsaEnabled: description: |- @@ -1495,8 +1408,6 @@ spec: required: - spec type: object - required: - - members type: object clusterDomain: description: Cluster domain to override the default *.svc.cluster.local @@ -1536,13 +1447,13 @@ spec: Service when creating a ClusterIP type Service type: string externalTrafficPolicy: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceExternalTrafficPolicy' description: |- ExternalTrafficPolicy mechanism to preserve the client source IP. Only supported on GCE and Google Kubernetes Engine. enum: - Cluster - Local - type: string loadBalancerIP: description: LoadBalancerIP IP that will be assigned to this LoadBalancer. @@ -1553,12 +1464,12 @@ spec: format: int32 type: integer type: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceType' description: Type of the `Service` to be created. enum: - LoadBalancer - NodePort - ClusterIP - type: string required: - type type: object @@ -1600,7 +1511,6 @@ spec: - spec type: object required: - - clusterName - members type: object type: array @@ -1626,13 +1536,13 @@ spec: when creating a ClusterIP type Service type: string externalTrafficPolicy: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceExternalTrafficPolicy' description: |- ExternalTrafficPolicy mechanism to preserve the client source IP. Only supported on GCE and Google Kubernetes Engine. enum: - Cluster - Local - type: string loadBalancerIP: description: LoadBalancerIP IP that will be assigned to this LoadBalancer. type: string @@ -1642,12 +1552,12 @@ spec: format: int32 type: integer type: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceType' description: Type of the `Service` to be created. enum: - LoadBalancer - NodePort - ClusterIP - type: string required: - type type: object @@ -1667,13 +1577,13 @@ spec: when creating a ClusterIP type Service type: string externalTrafficPolicy: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceExternalTrafficPolicy' description: |- ExternalTrafficPolicy mechanism to preserve the client source IP. Only supported on GCE and Google Kubernetes Engine. enum: - Cluster - Local - type: string loadBalancerIP: description: LoadBalancerIP IP that will be assigned to this LoadBalancer. type: string @@ -1683,12 +1593,12 @@ spec: format: int32 type: integer type: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceType' description: Type of the `Service` to be created. enum: - LoadBalancer - NodePort - ClusterIP - type: string required: - type type: object diff --git a/config/crd/bases/mongodbcommunity.mongodb.com_mongodbcommunity.yaml b/config/crd/bases/mongodbcommunity.mongodb.com_mongodbcommunity.yaml index 36d5c892d..a0004e22e 100644 --- a/config/crd/bases/mongodbcommunity.mongodb.com_mongodbcommunity.yaml +++ b/config/crd/bases/mongodbcommunity.mongodb.com_mongodbcommunity.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.18.0 + controller-gen.kubebuilder.io/version: v0.15.0 service.binding: path={.metadata.name}-{.spec.users[0].db}-{.spec.users[0].name},objectType=Secret service.binding/connectionString: path={.metadata.name}-{.spec.users[0].db}-{.spec.users[0].name},objectType=Secret,sourceKey=connectionString.standardSrv service.binding/password: path={.metadata.name}-{.spec.users[0].db}-{.spec.users[0].name},objectType=Secret,sourceKey=password @@ -330,24 +330,13 @@ spec: authentication: properties: agentCertificateSecretRef: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- AgentCertificateSecret is a reference to a Secret containing the certificate and the key for the automation agent The secret needs to have available: - certificate under key: "tls.crt" - private key under key: "tls.key" If additionally, tls.pem is present, then it needs to be equal to the concatenation of tls.crt and tls.key - properties: - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - type: object - x-kubernetes-map-type: atomic agentMode: description: AgentMode contains the authentication mode used by the automation agent. @@ -466,57 +455,24 @@ spec: communication properties: caCertificateSecretRef: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CaCertificateSecret is a reference to a Secret containing the certificate for the CA which signed the server certificates The certificate is expected to be available under the key "ca.crt" - properties: - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - type: object - x-kubernetes-map-type: atomic caConfigMapRef: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CaConfigMap is a reference to a ConfigMap containing the certificate for the CA which signed the server certificates The certificate is expected to be available under the key "ca.crt" This field is ignored when CaCertificateSecretRef is configured - properties: - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - type: object - x-kubernetes-map-type: atomic certificateKeySecretRef: + $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CertificateKeySecret is a reference to a Secret containing a private key and certificate to use for TLS. The key and cert are expected to be PEM encoded and available at "tls.key" and "tls.crt". This is the same format used for the standard "kubernetes.io/tls" Secret type, but no specific type is required. Alternatively, an entry tls.pem, containing the concatenation of cert and key, can be provided. If all of tls.pem, tls.crt and tls.key are present, the tls.pem one needs to be equal to the concatenation of tls.crt and tls.key - properties: - name: - default: "" - description: |- - Name of the referent. - This field is effectively required, but due to backwards compatibility is - allowed to be empty. Instances of this type with an empty value here are - almost certainly wrong. - More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names - type: string - type: object - x-kubernetes-map-type: atomic enabled: type: boolean optional: diff --git a/controllers/om/deployment/testing_utils.go b/controllers/om/deployment/testing_utils.go index d62a1aca2..a4f948488 100644 --- a/controllers/om/deployment/testing_utils.go +++ b/controllers/om/deployment/testing_utils.go @@ -1,14 +1,14 @@ package deployment import ( + "github.com/mongodb/mongodb-kubernetes/controllers/operator/construct" + "github.com/mongodb/mongodb-kubernetes/pkg/util/env" "go.uber.org/zap" "github.com/mongodb/mongodb-kubernetes/api/v1/mdb" "github.com/mongodb/mongodb-kubernetes/controllers/om" "github.com/mongodb/mongodb-kubernetes/controllers/om/replicaset" - "github.com/mongodb/mongodb-kubernetes/controllers/operator/construct" "github.com/mongodb/mongodb-kubernetes/pkg/util" - "github.com/mongodb/mongodb-kubernetes/pkg/util/env" ) // CreateFromReplicaSet builds the replica set for the automation config From 073032bea1b9f657ba46b3a6bb8792c9f5eae26d Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 15:50:02 +0200 Subject: [PATCH 11/25] Edge case TLS disabled and rs scaled --- .../operator/mongodbreplicaset_controller.go | 27 ++++++++++++++----- .../tests/tls/e2e_tls_disable_and_scale_up.py | 10 ++----- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index aec6f09ad..41d4c8d6c 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -576,7 +576,7 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c caFilePath := fmt.Sprintf("%s/ca-pem", util.TLSCaMountPath) // If current operation is to Disable TLS, then we should the current members of the Replica Set, // this is, do not scale them up or down util TLS disabling has completed. - shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(conn, r.imageUrls[mcoConstruct.MongodbImageEnv], r.forceEnterprise, membersNumberBefore, rs, log, caFilePath, tlsCertPath) + shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(conn, r.imageUrls[mcoConstruct.MongodbImageEnv], r.forceEnterprise, replicasTarget, rs, log, caFilePath, tlsCertPath) if err != nil && !isRecovering { return workflow.Failed(err) } @@ -585,7 +585,12 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c if shouldLockMembers { // We should not add or remove members during this run, we'll wait for // TLS to be completely disabled first. - updatedMembers = membersNumberBefore + // However, on first reconciliation when membersNumberBefore=0, we need to use replicasTarget + if membersNumberBefore == 0 { + updatedMembers = replicasTarget + } else { + updatedMembers = membersNumberBefore + } } else { updatedMembers = replicasTarget } @@ -657,10 +662,18 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c return workflow.OK() } -// updateOmDeploymentDisableTLSConfiguration checks if TLS configuration needs -// to be disabled. In which case it will disable it and inform to the calling -// function. -func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage string, forceEnterprise bool, membersNumberBefore int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, caFilePath, tlsCertPath string) (bool, error) { +// updateOmDeploymentDisableTLSConfiguration handles the edge case where TLS is disabled while +// simultaneously scaling the replica set. Without this safeguard, automation agents could fail during the +// transition, or new pods might join with inconsistent TLS configuration. +// +// This function implements a two-phase reconciliation pattern: +// 1. First reconciliation: Disable TLS on existing members (returns shouldLockMembers=true to prevent scaling) +// 2. Second reconciliation: Once TLS is fully disabled, allow scaling operations to proceed +// +// Related ticket: CLOUDP-80768 (March 2021) +// See also: e2e_tls_disable_and_scale_up.py test +// See also: e2e_tls_disable_and_scale_down.py test +func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage string, forceEnterprise bool, currentMemberCount int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, caFilePath, tlsCertPath string) (bool, error) { tlsConfigWasDisabled := false err := conn.ReadUpdateDeployment( @@ -674,7 +687,7 @@ func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage // configure as many agents/Pods as we currently have, no more (in case // there's a scale up change at the same time). - replicaSet := replicaset.BuildFromMongoDBWithReplicas(mongoDBImage, forceEnterprise, rs, membersNumberBefore, rs.CalculateFeatureCompatibilityVersion(), tlsCertPath) + replicaSet := replicaset.BuildFromMongoDBWithReplicas(mongoDBImage, forceEnterprise, rs, currentMemberCount, rs.CalculateFeatureCompatibilityVersion(), tlsCertPath) lastConfig, err := rs.GetLastAdditionalMongodConfigByType(mdbv1.ReplicaSetConfig) if err != nil { diff --git a/docker/mongodb-kubernetes-tests/tests/tls/e2e_tls_disable_and_scale_up.py b/docker/mongodb-kubernetes-tests/tests/tls/e2e_tls_disable_and_scale_up.py index 3eae37813..154b91bff 100644 --- a/docker/mongodb-kubernetes-tests/tests/tls/e2e_tls_disable_and_scale_up.py +++ b/docker/mongodb-kubernetes-tests/tests/tls/e2e_tls_disable_and_scale_up.py @@ -37,19 +37,13 @@ def test_rs_is_running(replica_set: MongoDB): @pytest.mark.e2e_disable_tls_scale_up def test_tls_is_disabled_and_scaled_up(replica_set: MongoDB): replica_set.load() + # Parameters must be changed at the same time replica_set["spec"]["members"] = 5 - - replica_set.update() - - -@pytest.mark.e2e_disable_tls_scale_up -def test_tls_is_disabled_and_scaled_up(replica_set: MongoDB): - replica_set.load() replica_set["spec"]["security"]["tls"]["enabled"] = False del replica_set["spec"]["additionalMongodConfig"] replica_set.update() # timeout is longer because the operator first needs to - # disable TLS and then, scale down one by one. + # disable TLS on existing members, then scale up one by one. replica_set.assert_reaches_phase(Phase.Running, timeout=800) From 8c4f229e01c7341421fcca8f70de8ee6ea08f367 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 15:53:33 +0200 Subject: [PATCH 12/25] Remove unused lines --- controllers/operator/mongodbreplicaset_controller_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller_test.go b/controllers/operator/mongodbreplicaset_controller_test.go index 5633faf62..669285d64 100644 --- a/controllers/operator/mongodbreplicaset_controller_test.go +++ b/controllers/operator/mongodbreplicaset_controller_test.go @@ -395,8 +395,6 @@ func TestUpdateDeploymentTLSConfiguration(t *testing.T) { rsNoTLS := mdbv1.NewReplicaSetBuilder().Build() deploymentWithTLS := deployment.CreateFromReplicaSet("fake-mongoDBImage", false, rsWithTLS) deploymentNoTLS := deployment.CreateFromReplicaSet("fake-mongoDBImage", false, rsNoTLS) - // stsWithTLS := construct.DatabaseStatefulSet(*rsWithTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) - // stsNoTLS := construct.DatabaseStatefulSet(*rsNoTLS, construct.ReplicaSetOptions(construct.GetPodEnvOptions()), zap.S()) // TLS Disabled -> TLS Disabled shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(om.NewMockedOmConnection(deploymentNoTLS), "fake-mongoDBImage", false, 3, rsNoTLS, zap.S(), util.CAFilePathInContainer, "") From 691d804c1e29427f57ff994b2dff1a4e5b7bb492 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 16:00:09 +0200 Subject: [PATCH 13/25] Import order --- controllers/om/deployment/testing_utils.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/controllers/om/deployment/testing_utils.go b/controllers/om/deployment/testing_utils.go index a4f948488..d62a1aca2 100644 --- a/controllers/om/deployment/testing_utils.go +++ b/controllers/om/deployment/testing_utils.go @@ -1,14 +1,14 @@ package deployment import ( - "github.com/mongodb/mongodb-kubernetes/controllers/operator/construct" - "github.com/mongodb/mongodb-kubernetes/pkg/util/env" "go.uber.org/zap" "github.com/mongodb/mongodb-kubernetes/api/v1/mdb" "github.com/mongodb/mongodb-kubernetes/controllers/om" "github.com/mongodb/mongodb-kubernetes/controllers/om/replicaset" + "github.com/mongodb/mongodb-kubernetes/controllers/operator/construct" "github.com/mongodb/mongodb-kubernetes/pkg/util" + "github.com/mongodb/mongodb-kubernetes/pkg/util/env" ) // CreateFromReplicaSet builds the replica set for the automation config From 31daeae682b2b1e6ca5909e00df3db909d62bb53 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 16:01:19 +0200 Subject: [PATCH 14/25] Fix comment --- controllers/operator/mongodbreplicaset_controller.go | 1 - 1 file changed, 1 deletion(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 41d4c8d6c..3afe3de7c 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -672,7 +672,6 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c // // Related ticket: CLOUDP-80768 (March 2021) // See also: e2e_tls_disable_and_scale_up.py test -// See also: e2e_tls_disable_and_scale_down.py test func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage string, forceEnterprise bool, currentMemberCount int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, caFilePath, tlsCertPath string) (bool, error) { tlsConfigWasDisabled := false From 310030e2057ebe5d16d83a99c914b3fbf5eb3614 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Wed, 1 Oct 2025 16:39:11 +0200 Subject: [PATCH 15/25] Run applySearchOverrides early --- controllers/operator/mongodbreplicaset_controller.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 3afe3de7c..e1355f6f8 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -216,6 +216,10 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco currentAgentAuthMode: currentAgentAuthMode, } + // Apply search overrides early so searchCoordinator role is present before ensureRoles runs + // This must happen before the ordering logic to ensure roles are synced regardless of order + shouldMirrorKeyfile := r.applySearchOverrides(ctx, rs, log) + // 4. Recovery // Recovery prevents some deadlocks that can occur during reconciliation, e.g. the setting of an incorrect automation @@ -223,7 +227,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco // See CLOUDP-189433 and CLOUDP-229222 for more details. if recovery.ShouldTriggerRecovery(rs.Status.Phase != mdbstatus.PhaseRunning, rs.Status.LastTransition) { log.Warnf("Triggering Automatic Recovery. The MongoDB resource %s/%s is in %s state since %s", rs.Namespace, rs.Name, rs.Status.Phase, rs.Status.LastTransition) - automationConfigStatus := r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, true).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") + automationConfigStatus := r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, shouldMirrorKeyfile, true).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") reconcileStatus := r.reconcileMemberResources(ctx, rs, conn, log, projectConfig, deploymentOpts) if !reconcileStatus.IsOK() { log.Errorf("Recovery failed because of reconcile errors, %v", reconcileStatus) @@ -242,7 +246,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco publishAutomationConfigFirst := publishAutomationConfigFirstRS(ctx, r.client, *rs, lastSpec, deploymentOpts.currentAgentAuthMode, projectConfig.SSLMMSCAConfigMap, log) status := workflow.RunInGivenOrder(publishAutomationConfigFirst, func() workflow.Status { - return r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, false).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") + return r.updateOmDeploymentRs(ctx, conn, rs.Status.Members, rs, log, tlsCertPath, internalClusterCertPath, deploymentOpts, shouldMirrorKeyfile, false).OnErrorPrepend("Failed to create/update (Ops Manager reconciliation phase):") }, func() workflow.Status { return r.reconcileMemberResources(ctx, rs, conn, log, projectConfig, deploymentOpts) @@ -562,7 +566,7 @@ func AddReplicaSetController(ctx context.Context, mgr manager.Manager, imageUrls // updateOmDeploymentRs performs OM registration operation for the replicaset. So the changes will be finally propagated // to automation agents in containers -func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, conn om.Connection, membersNumberBefore int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, tlsCertPath, internalClusterCertPath string, deploymentOptionsRS deploymentOptionsRS, isRecovering bool) workflow.Status { +func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, conn om.Connection, membersNumberBefore int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, tlsCertPath, internalClusterCertPath string, deploymentOptionsRS deploymentOptionsRS, shouldMirrorKeyfile bool, isRecovering bool) workflow.Status { log.Debug("Entering UpdateOMDeployments") // Only "concrete" RS members should be observed // - if scaling down, let's observe only members that will remain after scale-down operation @@ -616,8 +620,6 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c prometheusCertHash: deploymentOptionsRS.prometheusCertHash, } - shouldMirrorKeyfile := r.applySearchOverrides(ctx, rs, log) - err = conn.ReadUpdateDeployment( func(d om.Deployment) error { if shouldMirrorKeyfile { From 82467d61b9496ac7dd59b4a85d3e1d57bb59b535 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Fri, 3 Oct 2025 14:24:14 +0200 Subject: [PATCH 16/25] Revert "Edge case TLS disabled and rs scaled" This reverts commit 073032bea1b9f657ba46b3a6bb8792c9f5eae26d. --- .../operator/mongodbreplicaset_controller.go | 26 +++++-------------- .../tests/tls/e2e_tls_disable_and_scale_up.py | 10 +++++-- 2 files changed, 15 insertions(+), 21 deletions(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index e1355f6f8..216516890 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -580,7 +580,7 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c caFilePath := fmt.Sprintf("%s/ca-pem", util.TLSCaMountPath) // If current operation is to Disable TLS, then we should the current members of the Replica Set, // this is, do not scale them up or down util TLS disabling has completed. - shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(conn, r.imageUrls[mcoConstruct.MongodbImageEnv], r.forceEnterprise, replicasTarget, rs, log, caFilePath, tlsCertPath) + shouldLockMembers, err := updateOmDeploymentDisableTLSConfiguration(conn, r.imageUrls[mcoConstruct.MongodbImageEnv], r.forceEnterprise, membersNumberBefore, rs, log, caFilePath, tlsCertPath) if err != nil && !isRecovering { return workflow.Failed(err) } @@ -589,12 +589,7 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c if shouldLockMembers { // We should not add or remove members during this run, we'll wait for // TLS to be completely disabled first. - // However, on first reconciliation when membersNumberBefore=0, we need to use replicasTarget - if membersNumberBefore == 0 { - updatedMembers = replicasTarget - } else { - updatedMembers = membersNumberBefore - } + updatedMembers = membersNumberBefore } else { updatedMembers = replicasTarget } @@ -664,17 +659,10 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c return workflow.OK() } -// updateOmDeploymentDisableTLSConfiguration handles the edge case where TLS is disabled while -// simultaneously scaling the replica set. Without this safeguard, automation agents could fail during the -// transition, or new pods might join with inconsistent TLS configuration. -// -// This function implements a two-phase reconciliation pattern: -// 1. First reconciliation: Disable TLS on existing members (returns shouldLockMembers=true to prevent scaling) -// 2. Second reconciliation: Once TLS is fully disabled, allow scaling operations to proceed -// -// Related ticket: CLOUDP-80768 (March 2021) -// See also: e2e_tls_disable_and_scale_up.py test -func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage string, forceEnterprise bool, currentMemberCount int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, caFilePath, tlsCertPath string) (bool, error) { +// updateOmDeploymentDisableTLSConfiguration checks if TLS configuration needs +// to be disabled. In which case it will disable it and inform to the calling +// function. +func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage string, forceEnterprise bool, membersNumberBefore int, rs *mdbv1.MongoDB, log *zap.SugaredLogger, caFilePath, tlsCertPath string) (bool, error) { tlsConfigWasDisabled := false err := conn.ReadUpdateDeployment( @@ -688,7 +676,7 @@ func updateOmDeploymentDisableTLSConfiguration(conn om.Connection, mongoDBImage // configure as many agents/Pods as we currently have, no more (in case // there's a scale up change at the same time). - replicaSet := replicaset.BuildFromMongoDBWithReplicas(mongoDBImage, forceEnterprise, rs, currentMemberCount, rs.CalculateFeatureCompatibilityVersion(), tlsCertPath) + replicaSet := replicaset.BuildFromMongoDBWithReplicas(mongoDBImage, forceEnterprise, rs, membersNumberBefore, rs.CalculateFeatureCompatibilityVersion(), tlsCertPath) lastConfig, err := rs.GetLastAdditionalMongodConfigByType(mdbv1.ReplicaSetConfig) if err != nil { diff --git a/docker/mongodb-kubernetes-tests/tests/tls/e2e_tls_disable_and_scale_up.py b/docker/mongodb-kubernetes-tests/tests/tls/e2e_tls_disable_and_scale_up.py index 154b91bff..3eae37813 100644 --- a/docker/mongodb-kubernetes-tests/tests/tls/e2e_tls_disable_and_scale_up.py +++ b/docker/mongodb-kubernetes-tests/tests/tls/e2e_tls_disable_and_scale_up.py @@ -37,13 +37,19 @@ def test_rs_is_running(replica_set: MongoDB): @pytest.mark.e2e_disable_tls_scale_up def test_tls_is_disabled_and_scaled_up(replica_set: MongoDB): replica_set.load() - # Parameters must be changed at the same time replica_set["spec"]["members"] = 5 + + replica_set.update() + + +@pytest.mark.e2e_disable_tls_scale_up +def test_tls_is_disabled_and_scaled_up(replica_set: MongoDB): + replica_set.load() replica_set["spec"]["security"]["tls"]["enabled"] = False del replica_set["spec"]["additionalMongodConfig"] replica_set.update() # timeout is longer because the operator first needs to - # disable TLS on existing members, then scale up one by one. + # disable TLS and then, scale down one by one. replica_set.assert_reaches_phase(Phase.Running, timeout=800) From c7268d1597e98371c07b02b4b04dd72616f692ec Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Fri, 3 Oct 2025 14:26:40 +0200 Subject: [PATCH 17/25] Revert to controller gen 0.18 --- .../mongodb.com_clustermongodbroles.yaml | 2 +- config/crd/bases/mongodb.com_mongodb.yaml | 48 +++++++- .../mongodb.com_mongodbmulticluster.yaml | 48 +++++++- .../crd/bases/mongodb.com_mongodbsearch.yaml | 86 +++++++++++++- .../crd/bases/mongodb.com_mongodbusers.yaml | 2 +- config/crd/bases/mongodb.com_opsmanagers.yaml | 112 ++++++++++++++++-- ...ommunity.mongodb.com_mongodbcommunity.yaml | 54 ++++++++- 7 files changed, 324 insertions(+), 28 deletions(-) diff --git a/config/crd/bases/mongodb.com_clustermongodbroles.yaml b/config/crd/bases/mongodb.com_clustermongodbroles.yaml index 9241b7dad..3d583bcfd 100644 --- a/config/crd/bases/mongodb.com_clustermongodbroles.yaml +++ b/config/crd/bases/mongodb.com_clustermongodbroles.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.15.0 + controller-gen.kubebuilder.io/version: v0.18.0 name: clustermongodbroles.mongodb.com spec: group: mongodb.com diff --git a/config/crd/bases/mongodb.com_mongodb.yaml b/config/crd/bases/mongodb.com_mongodb.yaml index 2a7076877..d421d8837 100644 --- a/config/crd/bases/mongodb.com_mongodb.yaml +++ b/config/crd/bases/mongodb.com_mongodb.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.15.0 + controller-gen.kubebuilder.io/version: v0.18.0 name: mongodb.mongodb.com spec: group: mongodb.com @@ -1410,7 +1410,29 @@ spec: automationLdapGroupDN: type: string automationPasswordSecretRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' + description: SecretKeySelector selects a key of a Secret. + properties: + key: + description: The key of the secret to select from. Must + be a valid secret key. + type: string + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + optional: + description: Specify whether the Secret or its key + must be defined + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic automationUserName: type: string clientCertificateSecretRef: @@ -1445,9 +1467,29 @@ spec: bindQueryUser: type: string caConfigMapRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ConfigMapKeySelector' description: Allows to point at a ConfigMap/key with a CA file to mount on the Pod + properties: + key: + description: The key to select. + type: string + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + optional: + description: Specify whether the ConfigMap or its + key must be defined + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic servers: items: type: string diff --git a/config/crd/bases/mongodb.com_mongodbmulticluster.yaml b/config/crd/bases/mongodb.com_mongodbmulticluster.yaml index 01fe3f2e6..3f1fa05c9 100644 --- a/config/crd/bases/mongodb.com_mongodbmulticluster.yaml +++ b/config/crd/bases/mongodb.com_mongodbmulticluster.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.15.0 + controller-gen.kubebuilder.io/version: v0.18.0 name: mongodbmulticluster.mongodb.com spec: group: mongodb.com @@ -670,7 +670,29 @@ spec: automationLdapGroupDN: type: string automationPasswordSecretRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' + description: SecretKeySelector selects a key of a Secret. + properties: + key: + description: The key of the secret to select from. Must + be a valid secret key. + type: string + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + optional: + description: Specify whether the Secret or its key + must be defined + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic automationUserName: type: string clientCertificateSecretRef: @@ -705,9 +727,29 @@ spec: bindQueryUser: type: string caConfigMapRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ConfigMapKeySelector' description: Allows to point at a ConfigMap/key with a CA file to mount on the Pod + properties: + key: + description: The key to select. + type: string + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + optional: + description: Specify whether the ConfigMap or its + key must be defined + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic servers: items: type: string diff --git a/config/crd/bases/mongodb.com_mongodbsearch.yaml b/config/crd/bases/mongodb.com_mongodbsearch.yaml index 096cb7bb7..15153ba25 100644 --- a/config/crd/bases/mongodb.com_mongodbsearch.yaml +++ b/config/crd/bases/mongodb.com_mongodbsearch.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.15.0 + controller-gen.kubebuilder.io/version: v0.18.0 name: mongodbsearch.mongodb.com spec: group: mongodb.com @@ -107,9 +107,65 @@ spec: type: object type: object resourceRequirements: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ResourceRequirements' description: Configure resource requests and limits for the MongoDB Search pods. + properties: + claims: + description: |- + Claims lists the names of resources, defined in spec.resourceClaims, + that are used by this container. + + This is an alpha field and requires enabling the + DynamicResourceAllocation feature gate. + + This field is immutable. It can only be set for containers. + items: + description: ResourceClaim references one entry in PodSpec.ResourceClaims. + properties: + name: + description: |- + Name must match the name of one entry in pod.spec.resourceClaims of + the Pod where this field is used. It makes that resource available + inside a container. + type: string + request: + description: |- + Request is the name chosen for a request in the referenced claim. + If empty, everything from the claim is made available, otherwise + only the result of this request. + type: string + required: + - name + type: object + type: array + x-kubernetes-list-map-keys: + - name + x-kubernetes-list-type: map + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: |- + Limits describes the maximum amount of compute resources allowed. + More info: https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/ + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: |- + Requests describes the minimum amount of compute resources required. + If Requests is omitted for a container, it defaults to Limits if that is explicitly specified, + otherwise to an implementation-defined value. Requests cannot exceed Limits. + More info: https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/ + type: object + type: object security: description: Configure security settings of the MongoDB Search server that MongoDB database is connecting to when performing search queries. @@ -117,11 +173,22 @@ spec: tls: properties: certificateKeySecretRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CertificateKeySecret is a reference to a Secret containing a private key and certificate to use for TLS. The key and cert are expected to be PEM encoded and available at "tls.key" and "tls.crt". This is the same format used for the standard "kubernetes.io/tls" Secret type, but no specific type is required. + properties: + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + type: object + x-kubernetes-map-type: atomic required: - certificateKeySecretRef type: object @@ -151,10 +218,21 @@ spec: description: TLS configuration for the external MongoDB deployment properties: ca: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CA is a reference to a Secret containing the CA certificate that issued mongod's TLS certificate. The CA certificate is expected to be PEM encoded and available at the "ca.crt" key. + properties: + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + type: object + x-kubernetes-map-type: atomic required: - ca type: object diff --git a/config/crd/bases/mongodb.com_mongodbusers.yaml b/config/crd/bases/mongodb.com_mongodbusers.yaml index a81f0d449..89713ce7f 100644 --- a/config/crd/bases/mongodb.com_mongodbusers.yaml +++ b/config/crd/bases/mongodb.com_mongodbusers.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.15.0 + controller-gen.kubebuilder.io/version: v0.18.0 name: mongodbusers.mongodb.com spec: group: mongodb.com diff --git a/config/crd/bases/mongodb.com_opsmanagers.yaml b/config/crd/bases/mongodb.com_opsmanagers.yaml index c830b9a24..3ace001da 100644 --- a/config/crd/bases/mongodb.com_opsmanagers.yaml +++ b/config/crd/bases/mongodb.com_opsmanagers.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.15.0 + controller-gen.kubebuilder.io/version: v0.18.0 name: opsmanagers.mongodb.com spec: group: mongodb.com @@ -730,7 +730,30 @@ spec: automationLdapGroupDN: type: string automationPasswordSecretRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' + description: SecretKeySelector selects a key of a + Secret. + properties: + key: + description: The key of the secret to select from. Must + be a valid secret key. + type: string + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + optional: + description: Specify whether the Secret or its + key must be defined + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic automationUserName: type: string clientCertificateSecretRef: @@ -766,9 +789,29 @@ spec: bindQueryUser: type: string caConfigMapRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ConfigMapKeySelector' description: Allows to point at a ConfigMap/key with a CA file to mount on the Pod + properties: + key: + description: The key to select. + type: string + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + optional: + description: Specify whether the ConfigMap or + its key must be defined + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic servers: items: type: string @@ -1193,7 +1236,29 @@ spec: CustomCertificateSecretRefs is a list of valid Certificate Authority certificate secrets that apply to the associated S3 bucket. items: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' + description: SecretKeySelector selects a key of a Secret. + properties: + key: + description: The key of the secret to select from. Must + be a valid secret key. + type: string + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + optional: + description: Specify whether the Secret or its key + must be defined + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic type: array irsaEnabled: description: |- @@ -1263,7 +1328,29 @@ spec: CustomCertificateSecretRefs is a list of valid Certificate Authority certificate secrets that apply to the associated S3 bucket. items: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0SecretKeySelector' + description: SecretKeySelector selects a key of a Secret. + properties: + key: + description: The key of the secret to select from. Must + be a valid secret key. + type: string + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + optional: + description: Specify whether the Secret or its key + must be defined + type: boolean + required: + - key + type: object + x-kubernetes-map-type: atomic type: array irsaEnabled: description: |- @@ -1408,6 +1495,8 @@ spec: required: - spec type: object + required: + - members type: object clusterDomain: description: Cluster domain to override the default *.svc.cluster.local @@ -1447,13 +1536,13 @@ spec: Service when creating a ClusterIP type Service type: string externalTrafficPolicy: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceExternalTrafficPolicy' description: |- ExternalTrafficPolicy mechanism to preserve the client source IP. Only supported on GCE and Google Kubernetes Engine. enum: - Cluster - Local + type: string loadBalancerIP: description: LoadBalancerIP IP that will be assigned to this LoadBalancer. @@ -1464,12 +1553,12 @@ spec: format: int32 type: integer type: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceType' description: Type of the `Service` to be created. enum: - LoadBalancer - NodePort - ClusterIP + type: string required: - type type: object @@ -1511,6 +1600,7 @@ spec: - spec type: object required: + - clusterName - members type: object type: array @@ -1536,13 +1626,13 @@ spec: when creating a ClusterIP type Service type: string externalTrafficPolicy: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceExternalTrafficPolicy' description: |- ExternalTrafficPolicy mechanism to preserve the client source IP. Only supported on GCE and Google Kubernetes Engine. enum: - Cluster - Local + type: string loadBalancerIP: description: LoadBalancerIP IP that will be assigned to this LoadBalancer. type: string @@ -1552,12 +1642,12 @@ spec: format: int32 type: integer type: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceType' description: Type of the `Service` to be created. enum: - LoadBalancer - NodePort - ClusterIP + type: string required: - type type: object @@ -1577,13 +1667,13 @@ spec: when creating a ClusterIP type Service type: string externalTrafficPolicy: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceExternalTrafficPolicy' description: |- ExternalTrafficPolicy mechanism to preserve the client source IP. Only supported on GCE and Google Kubernetes Engine. enum: - Cluster - Local + type: string loadBalancerIP: description: LoadBalancerIP IP that will be assigned to this LoadBalancer. type: string @@ -1593,12 +1683,12 @@ spec: format: int32 type: integer type: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0ServiceType' description: Type of the `Service` to be created. enum: - LoadBalancer - NodePort - ClusterIP + type: string required: - type type: object diff --git a/config/crd/bases/mongodbcommunity.mongodb.com_mongodbcommunity.yaml b/config/crd/bases/mongodbcommunity.mongodb.com_mongodbcommunity.yaml index a0004e22e..36d5c892d 100644 --- a/config/crd/bases/mongodbcommunity.mongodb.com_mongodbcommunity.yaml +++ b/config/crd/bases/mongodbcommunity.mongodb.com_mongodbcommunity.yaml @@ -3,7 +3,7 @@ apiVersion: apiextensions.k8s.io/v1 kind: CustomResourceDefinition metadata: annotations: - controller-gen.kubebuilder.io/version: v0.15.0 + controller-gen.kubebuilder.io/version: v0.18.0 service.binding: path={.metadata.name}-{.spec.users[0].db}-{.spec.users[0].name},objectType=Secret service.binding/connectionString: path={.metadata.name}-{.spec.users[0].db}-{.spec.users[0].name},objectType=Secret,sourceKey=connectionString.standardSrv service.binding/password: path={.metadata.name}-{.spec.users[0].db}-{.spec.users[0].name},objectType=Secret,sourceKey=password @@ -330,13 +330,24 @@ spec: authentication: properties: agentCertificateSecretRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- AgentCertificateSecret is a reference to a Secret containing the certificate and the key for the automation agent The secret needs to have available: - certificate under key: "tls.crt" - private key under key: "tls.key" If additionally, tls.pem is present, then it needs to be equal to the concatenation of tls.crt and tls.key + properties: + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + type: object + x-kubernetes-map-type: atomic agentMode: description: AgentMode contains the authentication mode used by the automation agent. @@ -455,24 +466,57 @@ spec: communication properties: caCertificateSecretRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CaCertificateSecret is a reference to a Secret containing the certificate for the CA which signed the server certificates The certificate is expected to be available under the key "ca.crt" + properties: + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + type: object + x-kubernetes-map-type: atomic caConfigMapRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CaConfigMap is a reference to a ConfigMap containing the certificate for the CA which signed the server certificates The certificate is expected to be available under the key "ca.crt" This field is ignored when CaCertificateSecretRef is configured + properties: + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + type: object + x-kubernetes-map-type: atomic certificateKeySecretRef: - $ref: '#/definitions/k8s.io~1api~1core~1v1~0LocalObjectReference' description: |- CertificateKeySecret is a reference to a Secret containing a private key and certificate to use for TLS. The key and cert are expected to be PEM encoded and available at "tls.key" and "tls.crt". This is the same format used for the standard "kubernetes.io/tls" Secret type, but no specific type is required. Alternatively, an entry tls.pem, containing the concatenation of cert and key, can be provided. If all of tls.pem, tls.crt and tls.key are present, the tls.pem one needs to be equal to the concatenation of tls.crt and tls.key + properties: + name: + default: "" + description: |- + Name of the referent. + This field is effectively required, but due to backwards compatibility is + allowed to be empty. Instances of this type with an empty value here are + almost certainly wrong. + More info: https://kubernetes.io/docs/concepts/overview/working-with-objects/names/#names + type: string + type: object + x-kubernetes-map-type: atomic enabled: type: boolean optional: From e302990338b0218164fc23ecf31faf2a9a763db9 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Fri, 3 Oct 2025 15:24:39 +0200 Subject: [PATCH 18/25] Fix edge case --- controllers/operator/mongodbreplicaset_controller.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 216516890..01e5e8b9c 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -586,10 +586,19 @@ func (r *ReconcileMongoDbReplicaSet) updateOmDeploymentRs(ctx context.Context, c } var updatedMembers int + // This lock member logic will be removed soon, we should rather block possibility to disable tls + scale + // Tracked in CLOUDP-349087 if shouldLockMembers { // We should not add or remove members during this run, we'll wait for // TLS to be completely disabled first. - updatedMembers = membersNumberBefore + // However, on first reconciliation (membersNumberBefore=0), we need to use replicasTarget + // because the OM deployment is initialized with TLS enabled by default. + log.Debugf("locking members for this reconciliation because TLS was disabled") + if membersNumberBefore == 0 { + updatedMembers = replicasTarget + } else { + updatedMembers = membersNumberBefore + } } else { updatedMembers = replicasTarget } From 02920f80b030c3751f1bfc7230e7dd0bcf03f0da Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Tue, 7 Oct 2025 10:44:13 +0200 Subject: [PATCH 19/25] TestPublishAutomationConfigFirstRS Builder functions Test works --- .../mongodbreplicaset_controller_test.go | 202 ++++++++++++++++++ 1 file changed, 202 insertions(+) diff --git a/controllers/operator/mongodbreplicaset_controller_test.go b/controllers/operator/mongodbreplicaset_controller_test.go index 669285d64..787883ffb 100644 --- a/controllers/operator/mongodbreplicaset_controller_test.go +++ b/controllers/operator/mongodbreplicaset_controller_test.go @@ -30,6 +30,7 @@ import ( "github.com/mongodb/mongodb-kubernetes/controllers/om/backup" "github.com/mongodb/mongodb-kubernetes/controllers/om/deployment" "github.com/mongodb/mongodb-kubernetes/controllers/operator/authentication" + "github.com/mongodb/mongodb-kubernetes/controllers/operator/construct" "github.com/mongodb/mongodb-kubernetes/controllers/operator/controlledfeature" "github.com/mongodb/mongodb-kubernetes/controllers/operator/create" "github.com/mongodb/mongodb-kubernetes/controllers/operator/mock" @@ -1199,3 +1200,204 @@ func (b *ReplicaSetBuilder) Build() *mdbv1.MongoDB { b.InitDefaults() return b.DeepCopy() } + +// Helper functions for TestPublishAutomationConfigFirstRS + +func baseTestStatefulSet(name string, replicas int32) *appsv1.StatefulSet { + return &appsv1.StatefulSet{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + Namespace: mock.TestNamespace, + }, + Spec: appsv1.StatefulSetSpec{ + Replicas: ptr.To(replicas), + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + {Name: util.DatabaseContainerName}, + }, + }, + }, + }, + } +} + +func baseTestMongoDB(name string, members int) mdbv1.MongoDB { + return mdbv1.MongoDB{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + Namespace: mock.TestNamespace, + }, + Spec: mdbv1.MongoDbSpec{ + DbCommonSpec: mdbv1.DbCommonSpec{ + Security: &mdbv1.Security{ + TLSConfig: &mdbv1.TLSConfig{}, + Authentication: &mdbv1.Authentication{}, + }, + }, + Members: members, + }, + } +} + +// TestPublishAutomationConfigFirstRS tests the publishAutomationConfigFirstRS function which determines +// whether the OM automation config should be updated before the StatefulSet in certain scenarios +// (e.g., TLS disabled, CA removed, scaling down, agent auth changes, version changes). +func TestPublishAutomationConfigFirstRS(t *testing.T) { + ctx := context.Background() + + testCases := []struct { + name string + existingSts *appsv1.StatefulSet + mdb mdbv1.MongoDB + lastSpec *mdbv1.MongoDbSpec + currentAgentAuthMode string + sslMMSCAConfigMap string + extraObjects []client.Object + expectedPublishACFirst bool + }{ + { + name: "New StatefulSet", + existingSts: nil, + mdb: func() mdbv1.MongoDB { + m := baseTestMongoDB("test-rs", 3) + m.Spec.Security = nil // Simple case without security + return m + }(), + expectedPublishACFirst: false, + }, + { + name: "Scaling down", + existingSts: baseTestStatefulSet("test-rs", 5), + mdb: baseTestMongoDB("test-rs", 3), + expectedPublishACFirst: true, + }, + { + name: "TLS disabled with mounted cert", + existingSts: func() *appsv1.StatefulSet { + sts := baseTestStatefulSet("test-rs", 3) + sts.Spec.Template.Spec.Volumes = []corev1.Volume{ + { + Name: util.SecretVolumeName, + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{SecretName: "test-rs-cert"}, + }, + }, + } + sts.Spec.Template.Spec.Containers[0].VolumeMounts = []corev1.VolumeMount{ + {Name: util.SecretVolumeName, MountPath: "/tls"}, + } + return sts + }(), + mdb: func() mdbv1.MongoDB { + m := baseTestMongoDB("test-rs", 3) + m.Spec.Security.TLSConfig.Enabled = false + return m + }(), + extraObjects: []client.Object{ + &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test-rs-cert", Namespace: mock.TestNamespace}, + }, + }, + expectedPublishACFirst: true, + }, + { + name: "CA configmap removed", + existingSts: func() *appsv1.StatefulSet { + sts := baseTestStatefulSet("test-rs", 3) + sts.Spec.Template.Spec.Volumes = []corev1.Volume{ + { + Name: util.ConfigMapVolumeCAMountPath, + VolumeSource: corev1.VolumeSource{ + ConfigMap: &corev1.ConfigMapVolumeSource{ + LocalObjectReference: corev1.LocalObjectReference{Name: "custom-ca-configmap"}, + }, + }, + }, + } + sts.Spec.Template.Spec.Containers[0].VolumeMounts = []corev1.VolumeMount{ + {Name: util.ConfigMapVolumeCAMountPath, MountPath: "/ca"}, + } + return sts + }(), + mdb: func() mdbv1.MongoDB { + m := baseTestMongoDB("test-rs", 3) + m.Spec.Security.TLSConfig.Enabled = true + m.Spec.Security.TLSConfig.CA = "" + return m + }(), + extraObjects: []client.Object{ + &corev1.ConfigMap{ + ObjectMeta: metav1.ObjectMeta{Name: "custom-ca-configmap", Namespace: mock.TestNamespace}, + }, + }, + expectedPublishACFirst: true, + }, + { + name: "SSL MMS CA removed", + existingSts: func() *appsv1.StatefulSet { + sts := baseTestStatefulSet("test-rs", 3) + sts.Spec.Template.Spec.Containers[0].VolumeMounts = []corev1.VolumeMount{ + {Name: construct.CaCertName, MountPath: "/mms-ca"}, + } + return sts + }(), + mdb: baseTestMongoDB("test-rs", 3), + sslMMSCAConfigMap: "", + expectedPublishACFirst: true, + }, + { + name: "Agent X509 disabled", + existingSts: func() *appsv1.StatefulSet { + sts := baseTestStatefulSet("test-rs", 3) + sts.Spec.Template.Spec.Containers[0].VolumeMounts = []corev1.VolumeMount{ + {Name: util.AgentSecretName, MountPath: "/agent-certs"}, + } + return sts + }(), + mdb: func() mdbv1.MongoDB { + m := baseTestMongoDB("test-rs", 3) + m.Spec.Security.Authentication.Agents = mdbv1.AgentAuthentication{Mode: "SCRAM"} + return m + }(), + currentAgentAuthMode: "X509", + expectedPublishACFirst: true, + }, + { + name: "Version change in static architecture", + existingSts: baseTestStatefulSet("test-rs", 3), + mdb: func() mdbv1.MongoDB { + m := baseTestMongoDB("test-rs", 3) + m.Annotations = map[string]string{ + architectures.ArchitectureAnnotation: string(architectures.Static), + } + m.Spec.Version = "8.0.0" + return m + }(), + lastSpec: &mdbv1.MongoDbSpec{ + DbCommonSpec: mdbv1.DbCommonSpec{Version: "7.0.0"}, + }, + expectedPublishACFirst: true, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + objects := []client.Object{} + if tc.existingSts != nil { + objects = append(objects, tc.existingSts) + } + objects = append(objects, tc.extraObjects...) + + fakeClient := mock.NewEmptyFakeClientBuilder(). + WithObjects(objects...). + WithObjects(mock.GetDefaultResources()...). + Build() + kubeClient := kubernetesClient.NewClient(fakeClient) + + result := publishAutomationConfigFirstRS(ctx, kubeClient, tc.mdb, tc.lastSpec, tc.currentAgentAuthMode, tc.sslMMSCAConfigMap, zap.S()) + + assert.Equal(t, tc.expectedPublishACFirst, result) + }) + } +} From 580927f836db36cd22c5092568d836fafa9fcc44 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Tue, 7 Oct 2025 15:14:30 +0200 Subject: [PATCH 20/25] Remove old comment --- controllers/om/replicaset/om_replicaset.go | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/controllers/om/replicaset/om_replicaset.go b/controllers/om/replicaset/om_replicaset.go index 9a9804adc..e9535fad8 100644 --- a/controllers/om/replicaset/om_replicaset.go +++ b/controllers/om/replicaset/om_replicaset.go @@ -75,23 +75,6 @@ func PrepareScaleDownFromMap(omClient om.Connection, rsMembers map[string][]stri log.Debugw("Marked replica set members as non-voting", "replica set with members", rsMembers) } - // TODO practice shows that automation agents can get stuck on setting db to "disabled" also it seems that this process - // works correctly without explicit disabling - feel free to remove this code after some time when it is clear - // that everything works correctly without disabling - - // Stage 2. Set disabled to true - //err = omClient.ReadUpdateDeployment( - // func(d om.Deployment) error { - // d.DisableProcesses(allProcesses) - // return nil - // }, - //) - // - //if err != nil { - // return errors.New(fmt.Sprintf("Unable to set disabled to true, hosts: %v, err: %w", allProcesses, err)) - //} - //log.Debugw("Disabled processes", "processes", allProcesses) - log.Infow("Performed some preliminary steps to support scale down", "hosts", processes) return nil From ea14853b90b069b9fd4ef8abee57070195282d7c Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Tue, 7 Oct 2025 16:14:29 +0200 Subject: [PATCH 21/25] TestCreateMongodProcessesFromMongoDB --- controllers/om/process/om_process_test.go | 230 ++++++++++++++++++++++ 1 file changed, 230 insertions(+) create mode 100644 controllers/om/process/om_process_test.go diff --git a/controllers/om/process/om_process_test.go b/controllers/om/process/om_process_test.go new file mode 100644 index 000000000..818cfd0cd --- /dev/null +++ b/controllers/om/process/om_process_test.go @@ -0,0 +1,230 @@ +package process + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + mdbv1 "github.com/mongodb/mongodb-kubernetes/api/v1/mdb" + "github.com/mongodb/mongodb-kubernetes/pkg/util/maputil" +) + +const ( + defaultMongoDBImage = "mongodb/mongodb-enterprise-server:7.0.0" + defaultFCV = "7.0" + defaultNamespace = "test-namespace" +) + +func TestCreateMongodProcessesFromMongoDB(t *testing.T) { + tests := []struct { + name string + mdb *mdbv1.MongoDB + limit int + mongoDBImage string + forceEnterprise bool + fcv string + tlsCertPath string + expectedCount int + expectedHostnames []string + expectedNames []string + }{ + { + name: "3-member replica set", + mdb: baseReplicaSet("test-rs", 3), + limit: 3, + mongoDBImage: defaultMongoDBImage, + forceEnterprise: false, + fcv: defaultFCV, + expectedCount: 3, + expectedHostnames: []string{ + "test-rs-0.test-rs-svc.test-namespace.svc.cluster.local", + "test-rs-1.test-rs-svc.test-namespace.svc.cluster.local", + "test-rs-2.test-rs-svc.test-namespace.svc.cluster.local", + }, + expectedNames: []string{"test-rs-0", "test-rs-1", "test-rs-2"}, + }, + { + name: "Single member replica set", + mdb: baseReplicaSet("single-rs", 1), + limit: 1, + mongoDBImage: defaultMongoDBImage, + forceEnterprise: false, + fcv: defaultFCV, + expectedCount: 1, + expectedHostnames: []string{ + "single-rs-0.single-rs-svc.test-namespace.svc.cluster.local", + }, + expectedNames: []string{"single-rs-0"}, + }, + { + name: "Limit less than members (scale up in progress)", + mdb: baseReplicaSet("scale-up-rs", 5), + limit: 3, + mongoDBImage: defaultMongoDBImage, + forceEnterprise: false, + fcv: defaultFCV, + expectedCount: 3, + expectedHostnames: []string{ + "scale-up-rs-0.scale-up-rs-svc.test-namespace.svc.cluster.local", + "scale-up-rs-1.scale-up-rs-svc.test-namespace.svc.cluster.local", + "scale-up-rs-2.scale-up-rs-svc.test-namespace.svc.cluster.local", + }, + expectedNames: []string{"scale-up-rs-0", "scale-up-rs-1", "scale-up-rs-2"}, + }, + { + name: "Limit greater than members (scale down in progress)", + mdb: baseReplicaSet("scale-down-rs", 3), + limit: 5, + mongoDBImage: defaultMongoDBImage, + forceEnterprise: false, + fcv: defaultFCV, + expectedCount: 5, + expectedHostnames: []string{ + "scale-down-rs-0.scale-down-rs-svc.test-namespace.svc.cluster.local", + "scale-down-rs-1.scale-down-rs-svc.test-namespace.svc.cluster.local", + "scale-down-rs-2.scale-down-rs-svc.test-namespace.svc.cluster.local", + "scale-down-rs-3.scale-down-rs-svc.test-namespace.svc.cluster.local", + "scale-down-rs-4.scale-down-rs-svc.test-namespace.svc.cluster.local", + }, + expectedNames: []string{"scale-down-rs-0", "scale-down-rs-1", "scale-down-rs-2", "scale-down-rs-3", "scale-down-rs-4"}, + }, + { + name: "Limit zero creates empty slice", + mdb: baseReplicaSet("empty-rs", 3), + limit: 0, + mongoDBImage: defaultMongoDBImage, + forceEnterprise: false, + fcv: defaultFCV, + expectedCount: 0, + expectedHostnames: []string{}, + expectedNames: []string{}, + }, + { + name: "Custom cluster domain", + mdb: func() *mdbv1.MongoDB { + rs := baseReplicaSet("custom-domain-rs", 2) + rs.Spec.ClusterDomain = "my-cluster.local" + return rs + }(), + + limit: 2, + mongoDBImage: defaultMongoDBImage, + forceEnterprise: false, + fcv: defaultFCV, + expectedCount: 2, + expectedHostnames: []string{ + "custom-domain-rs-0.custom-domain-rs-svc.test-namespace.svc.my-cluster.local", + "custom-domain-rs-1.custom-domain-rs-svc.test-namespace.svc.my-cluster.local", + }, + expectedNames: []string{"custom-domain-rs-0", "custom-domain-rs-1"}, + }, + { + name: "Different namespace", + mdb: func() *mdbv1.MongoDB { + rs := baseReplicaSet("other-ns-rs", 2) + rs.Namespace = "production" + return rs + }(), + limit: 2, + mongoDBImage: defaultMongoDBImage, + forceEnterprise: false, + fcv: defaultFCV, + expectedCount: 2, + expectedHostnames: []string{ + "other-ns-rs-0.other-ns-rs-svc.production.svc.cluster.local", + "other-ns-rs-1.other-ns-rs-svc.production.svc.cluster.local", + }, + expectedNames: []string{"other-ns-rs-0", "other-ns-rs-1"}, + }, + { + name: "With TLS cert path", + mdb: func() *mdbv1.MongoDB { + rs := baseReplicaSet("tls-rs", 2) + rs.Spec.Security = &mdbv1.Security{ + TLSConfig: &mdbv1.TLSConfig{Enabled: true}, + } + return rs + }(), + limit: 2, + mongoDBImage: defaultMongoDBImage, + forceEnterprise: false, + fcv: defaultFCV, + tlsCertPath: "/path/to/cert.pem", + expectedCount: 2, + expectedHostnames: []string{ + "tls-rs-0.tls-rs-svc.test-namespace.svc.cluster.local", + "tls-rs-1.tls-rs-svc.test-namespace.svc.cluster.local", + }, + expectedNames: []string{"tls-rs-0", "tls-rs-1"}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + processes := CreateMongodProcessesFromMongoDB( + tt.mongoDBImage, + tt.forceEnterprise, + tt.mdb, + tt.limit, + tt.fcv, + tt.tlsCertPath, + ) + + assert.Equal(t, tt.expectedCount, len(processes), "Process count mismatch") + + for i, process := range processes { + assert.Equal(t, tt.expectedNames[i], process.Name(), "Process name mismatch at index %d", i) + assert.Equal(t, tt.expectedHostnames[i], process.HostName(), "Hostname mismatch at index %d", i) + assert.Equal(t, tt.fcv, process.FeatureCompatibilityVersion(), "FCV mismatch at index %d", i) + + if tt.tlsCertPath != "" { + tlsConfig := process.TLSConfig() + assert.NotNil(t, tlsConfig, "TLS config should not be nil when cert path is provided") + assert.Equal(t, tt.tlsCertPath, tlsConfig["certificateKeyFile"], "TLS cert path mismatch at index %d", i) + } + } + }) + } +} + +func TestCreateMongodProcessesFromMongoDB_AdditionalConfig(t *testing.T) { + config := mdbv1.NewAdditionalMongodConfig("storage.engine", "inMemory"). + AddOption("replication.oplogSizeMB", 2048) + + mdb := mdbv1.NewReplicaSetBuilder(). + SetName("config-rs"). + SetNamespace(defaultNamespace). + SetMembers(2). + SetVersion("7.0.0"). + SetFCVersion(defaultFCV). + SetAdditionalConfig(config). + Build() + + processes := CreateMongodProcessesFromMongoDB( + defaultMongoDBImage, + false, + mdb, + 2, + defaultFCV, + "", + ) + + assert.Len(t, processes, 2) + + for i, process := range processes { + assert.Equal(t, "inMemory", maputil.ReadMapValueAsInterface(process.Args(), "storage", "engine"), + "Storage engine mismatch at index %d", i) + assert.Equal(t, 2048, maputil.ReadMapValueAsInterface(process.Args(), "replication", "oplogSizeMB"), + "OplogSizeMB mismatch at index %d", i) + } +} + +func baseReplicaSet(name string, members int) *mdbv1.MongoDB { + return mdbv1.NewReplicaSetBuilder(). + SetName(name). + SetNamespace(defaultNamespace). + SetMembers(members). + SetVersion("7.0.0"). + SetFCVersion(defaultFCV). + Build() +} From 1376b5d17490c87079b22b8d94f840cafe34f9cf Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Tue, 7 Oct 2025 16:42:02 +0200 Subject: [PATCH 22/25] TestBuildFromMongoDBWithReplicas --- .../om/replicaset/om_replicaset_test.go | 94 +++++++++++++++++++ 1 file changed, 94 insertions(+) create mode 100644 controllers/om/replicaset/om_replicaset_test.go diff --git a/controllers/om/replicaset/om_replicaset_test.go b/controllers/om/replicaset/om_replicaset_test.go new file mode 100644 index 000000000..cc7f91c39 --- /dev/null +++ b/controllers/om/replicaset/om_replicaset_test.go @@ -0,0 +1,94 @@ +package replicaset + +import ( + "testing" + + "github.com/stretchr/testify/assert" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/utils/ptr" + + mdbv1 "github.com/mongodb/mongodb-kubernetes/api/v1/mdb" + "github.com/mongodb/mongodb-kubernetes/controllers/om" + "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/automationconfig" +) + +// This test focuses on the integration/glue logic, not re-testing components. +func TestBuildFromMongoDBWithReplicas(t *testing.T) { + memberOptions := []automationconfig.MemberOptions{ + {Votes: ptr.To(1), Priority: ptr.To("1.0")}, + {Votes: ptr.To(1), Priority: ptr.To("0.5")}, + {Votes: ptr.To(0), Priority: ptr.To("0")}, // Non-voting member + } + + mdb := &mdbv1.MongoDB{ + ObjectMeta: metav1.ObjectMeta{ + Name: "test-rs", + Namespace: "test-namespace", + }, + Spec: mdbv1.MongoDbSpec{ + DbCommonSpec: mdbv1.DbCommonSpec{ + Version: "7.0.5", + Security: &mdbv1.Security{ + TLSConfig: &mdbv1.TLSConfig{}, + Authentication: &mdbv1.Authentication{}, + }, + Connectivity: &mdbv1.MongoDBConnectivity{ + ReplicaSetHorizons: []mdbv1.MongoDBHorizonConfig{}, + }, + }, + Members: 5, // Spec (target) is 5 members + MemberConfig: memberOptions, + }, + } + + // 3 replicas is less than spec.Members, scale up scenario + replicas := 3 + rsWithProcesses := BuildFromMongoDBWithReplicas( + "mongodb/mongodb-enterprise-server:7.0.5", + false, + mdb, + replicas, + "7.0", + "", + ) + + // Assert: ReplicaSet structure + assert.Equal(t, "test-rs", rsWithProcesses.Rs.Name(), "ReplicaSet ID should match MongoDB name") + assert.Equal(t, "1", rsWithProcesses.Rs["protocolVersion"], "Protocol version should be set to 1 for this MongoDB version") + + // Assert: Member count is controlled by replicas parameter, NOT mdb.Spec.Members + members := rsWithProcesses.Rs["members"].([]om.ReplicaSetMember) + assert.Len(t, members, replicas, "Member count should match replicas parameter (3), not mdb.Spec.Members (5)") + assert.Equal(t, 3, len(members), "Should have exactly 3 members") + + // Assert: Processes are created correctly + assert.Len(t, rsWithProcesses.Processes, replicas, "Process count should match replicas parameter") + expectedProcessNames := []string{"test-rs-0", "test-rs-1", "test-rs-2"} + expectedHostnames := []string{ + "test-rs-0.test-rs-svc.test-namespace.svc.cluster.local", + "test-rs-1.test-rs-svc.test-namespace.svc.cluster.local", + "test-rs-2.test-rs-svc.test-namespace.svc.cluster.local", + } + + for i := 0; i < replicas; i++ { + assert.Equal(t, expectedProcessNames[i], rsWithProcesses.Processes[i].Name(), + "Process name mismatch at index %d", i) + assert.Equal(t, expectedHostnames[i], rsWithProcesses.Processes[i].HostName(), + "Process hostname mismatch at index %d", i) + } + + // Assert: Member options are propagated + assert.Equal(t, 1, members[0].Votes(), "Member 0 should have 1 vote") + assert.Equal(t, float32(1.0), members[0].Priority(), "Member 0 should have priority 1.0") + assert.Equal(t, 1, members[1].Votes(), "Member 1 should have 1 vote") + assert.Equal(t, float32(0.5), members[1].Priority(), "Member 1 should have priority 0.5") + assert.Equal(t, 0, members[2].Votes(), "Member 2 should have 0 votes (non-voting)") + assert.Equal(t, float32(0), members[2].Priority(), "Member 2 should have priority 0") + + // Assert: Member host field contains process name (not full hostname) + // Note: ReplicaSetMember["host"] is the process name, not the full hostname + for i := 0; i < replicas; i++ { + assert.Equal(t, expectedProcessNames[i], members[i].Name(), + "Member host should match process name at index %d", i) + } +} From 3c3075887f3b589da2198904512701f5a6d54601 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Tue, 7 Oct 2025 16:48:10 +0200 Subject: [PATCH 23/25] Revert unrelated changed --- .gitignore | 4 ---- api/v1/search/zz_generated.deepcopy.go | 8 ++------ 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/.gitignore b/.gitignore index b6df00461..be38e569d 100644 --- a/.gitignore +++ b/.gitignore @@ -93,7 +93,3 @@ docs/**/test.sh.run.log dist logs *.run.log - -# AI -.mcp.json -CLAUDE.md diff --git a/api/v1/search/zz_generated.deepcopy.go b/api/v1/search/zz_generated.deepcopy.go index d18d025f8..c66322146 100644 --- a/api/v1/search/zz_generated.deepcopy.go +++ b/api/v1/search/zz_generated.deepcopy.go @@ -159,7 +159,7 @@ func (in *MongoDBSearchSpec) DeepCopyInto(out *MongoDBSearchSpec) { *out = new(v1.ResourceRequirements) (*in).DeepCopyInto(*out) } - in.Security.DeepCopyInto(&out.Security) + out.Security = in.Security } // DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new MongoDBSearchSpec. @@ -231,11 +231,7 @@ func (in *MongoDBSource) DeepCopy() *MongoDBSource { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *Security) DeepCopyInto(out *Security) { *out = *in - if in.TLS != nil { - in, out := &in.TLS, &out.TLS - *out = new(TLS) - **out = **in - } + out.TLS = in.TLS } // DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new Security. From e2d2e33b17da3e4d753b755508dfa211aef779db Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Tue, 7 Oct 2025 16:48:53 +0200 Subject: [PATCH 24/25] Lint` --- controllers/om/replicaset/om_replicaset_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/controllers/om/replicaset/om_replicaset_test.go b/controllers/om/replicaset/om_replicaset_test.go index cc7f91c39..721c70efa 100644 --- a/controllers/om/replicaset/om_replicaset_test.go +++ b/controllers/om/replicaset/om_replicaset_test.go @@ -4,9 +4,10 @@ import ( "testing" "github.com/stretchr/testify/assert" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/utils/ptr" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + mdbv1 "github.com/mongodb/mongodb-kubernetes/api/v1/mdb" "github.com/mongodb/mongodb-kubernetes/controllers/om" "github.com/mongodb/mongodb-kubernetes/mongodb-community-operator/pkg/automationconfig" From 96e6dc0194b4e3c2dda87a8c0aa4d10134905301 Mon Sep 17 00:00:00 2001 From: Julien Benhaim Date: Tue, 7 Oct 2025 18:21:49 +0200 Subject: [PATCH 25/25] Update doc --- controllers/operator/mongodbreplicaset_controller.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/controllers/operator/mongodbreplicaset_controller.go b/controllers/operator/mongodbreplicaset_controller.go index 01e5e8b9c..a16ac249b 100644 --- a/controllers/operator/mongodbreplicaset_controller.go +++ b/controllers/operator/mongodbreplicaset_controller.go @@ -216,6 +216,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco currentAgentAuthMode: currentAgentAuthMode, } + // 3. Search Overrides // Apply search overrides early so searchCoordinator role is present before ensureRoles runs // This must happen before the ordering logic to ensure roles are synced regardless of order shouldMirrorKeyfile := r.applySearchOverrides(ctx, rs, log) @@ -237,7 +238,7 @@ func (r *ReconcileMongoDbReplicaSet) Reconcile(ctx context.Context, request reco } } - // 5. Actual kube resources deployment and OM updates + // 5. Actual reconciliation execution, Ops Manager and kubernetes resources update lastSpec, err := rs.GetLastSpec() if err != nil { lastSpec = &mdbv1.MongoDbSpec{}