Skip to content

Commit

Permalink
Merge pull request #1080 from openshift-kni/leader-election-extras
Browse files Browse the repository at this point in the history
sched: always set the leader election parameters
  • Loading branch information
ffromani authored Nov 20, 2024
2 parents b972636 + d83cc70 commit ae8a5c4
Show file tree
Hide file tree
Showing 4 changed files with 98 additions and 29 deletions.
46 changes: 26 additions & 20 deletions controllers/numaresourcesscheduler_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,22 +45,14 @@ import (
"github.com/openshift-kni/numaresources-operator/pkg/apply"
"github.com/openshift-kni/numaresources-operator/pkg/hash"
"github.com/openshift-kni/numaresources-operator/pkg/loglevel"
nrosched "github.com/openshift-kni/numaresources-operator/pkg/numaresourcesscheduler"
schedmanifests "github.com/openshift-kni/numaresources-operator/pkg/numaresourcesscheduler/manifests/sched"
schedstate "github.com/openshift-kni/numaresources-operator/pkg/numaresourcesscheduler/objectstate/sched"
"github.com/openshift-kni/numaresources-operator/pkg/objectnames"
schedupdate "github.com/openshift-kni/numaresources-operator/pkg/objectupdate/sched"
"github.com/openshift-kni/numaresources-operator/pkg/status"
)

const (
leaderElectionResourceName = "numa-scheduler-leader"
schedulerPriorityClassName = "system-node-critical"
)

const (
conditionTypeIncorrectNUMAResourcesSchedulerResourceName = "IncorrectNUMAResourcesSchedulerResourceName"
)

// NUMAResourcesSchedulerReconciler reconciles a NUMAResourcesScheduler object
type NUMAResourcesSchedulerReconciler struct {
client.Client
Expand Down Expand Up @@ -107,7 +99,7 @@ func (r *NUMAResourcesSchedulerReconciler) Reconcile(ctx context.Context, req ct

if req.Name != objectnames.DefaultNUMAResourcesSchedulerCrName {
message := fmt.Sprintf("incorrect NUMAResourcesScheduler resource name: %s", instance.Name)
return ctrl.Result{}, r.updateStatus(ctx, instance, status.ConditionDegraded, conditionTypeIncorrectNUMAResourcesSchedulerResourceName, message)
return ctrl.Result{}, r.updateStatus(ctx, instance, status.ConditionDegraded, status.ConditionTypeIncorrectNUMAResourcesSchedulerResourceName, message)
}

result, condition, err := r.reconcileResource(ctx, instance)
Expand Down Expand Up @@ -215,7 +207,7 @@ func (r *NUMAResourcesSchedulerReconciler) syncNUMASchedulerResources(ctx contex
// should set a degraded state

// node-critical so the pod won't be preempted by pods having the most critical priority class
r.SchedulerManifests.Deployment.Spec.Template.Spec.PriorityClassName = schedulerPriorityClassName
r.SchedulerManifests.Deployment.Spec.Template.Spec.PriorityClassName = nrosched.SchedulerPriorityClassName

schedupdate.DeploymentImageSettings(r.SchedulerManifests.Deployment, schedSpec.SchedulerImage)
cmHash := hash.ConfigMapData(r.SchedulerManifests.ConfigMap)
Expand All @@ -226,7 +218,7 @@ func (r *NUMAResourcesSchedulerReconciler) syncNUMASchedulerResources(ctx contex

schedupdate.DeploymentEnvVarSettings(r.SchedulerManifests.Deployment, schedSpec)

k8swgrbacupdate.RoleForLeaderElection(r.SchedulerManifests.Role, r.Namespace, leaderElectionResourceName)
k8swgrbacupdate.RoleForLeaderElection(r.SchedulerManifests.Role, r.Namespace, nrosched.LeaderElectionResourceName)

existing := schedstate.FromClient(ctx, r.Client, r.SchedulerManifests)
for _, objState := range existing.State(r.SchedulerManifests) {
Expand Down Expand Up @@ -264,23 +256,29 @@ func unpackAPIResyncPeriod(reconcilePeriod *metav1.Duration) time.Duration {

func configParamsFromSchedSpec(schedSpec nropv1.NUMAResourcesSchedulerSpec, cacheResyncPeriod time.Duration, namespace string) k8swgmanifests.ConfigParams {
resyncPeriod := int64(cacheResyncPeriod.Seconds())
// if no actual replicas are required, leader election is unnecessary, so
// we force it to off to reduce the background noise.
// note: the api validation/normalization layer must ensure this value is != nil
leaderElect := (*schedSpec.Replicas > 1)

params := k8swgmanifests.ConfigParams{
ProfileName: schedSpec.SchedulerName,
Cache: &k8swgmanifests.ConfigCacheParams{
ResyncPeriodSeconds: &resyncPeriod,
},
ScoringStrategy: &k8swgmanifests.ScoringStrategyParams{},
}

if schedSpec.Replicas != nil && *schedSpec.Replicas > 1 {
params.LeaderElection = &k8swgmanifests.LeaderElectionParams{
LeaderElect: true,
LeaderElection: &k8swgmanifests.LeaderElectionParams{
// Make sure to always set explicitly the value and override the configmap defaults.
LeaderElect: leaderElect,
// unconditionally set those to make sure
// to play nice with the cluster and the main scheduler
ResourceNamespace: namespace,
ResourceName: leaderElectionResourceName,
}
ResourceName: nrosched.LeaderElectionResourceName,
},
}

klog.V(2).InfoS("setting leader election parameters", dumpLeaderElectionParams(params.LeaderElection)...)

var foreignPodsDetect string
var resyncMethod string = k8swgmanifests.CacheResyncAutodetect
var informerMode string
Expand Down Expand Up @@ -317,7 +315,7 @@ func configParamsFromSchedSpec(schedSpec nropv1.NUMAResourcesSchedulerSpec, cach
params.Cache.ResyncMethod = &resyncMethod
params.Cache.ForeignPodsDetectMode = &foreignPodsDetect
params.Cache.InformerMode = &informerMode
klog.InfoS("setting cache parameters", dumpConfigCacheParams(params.Cache)...)
klog.V(2).InfoS("setting cache parameters", dumpConfigCacheParams(params.Cache)...)

return params
}
Expand All @@ -331,6 +329,14 @@ func dumpConfigCacheParams(ccp *k8swgmanifests.ConfigCacheParams) []interface{}
}
}

func dumpLeaderElectionParams(lep *k8swgmanifests.LeaderElectionParams) []interface{} {
return []interface{}{
"leaderElect", lep.LeaderElect,
"resourceNamespace", lep.ResourceNamespace,
"resourceName", lep.ResourceName,
}
}

func strInt64Ptr(ip *int64) string {
if ip == nil {
return "N/A"
Expand Down
55 changes: 46 additions & 9 deletions controllers/numaresourcesscheduler_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
depobjupdate "github.com/k8stopologyawareschedwg/deployer/pkg/objectupdate"
nropv1 "github.com/openshift-kni/numaresources-operator/api/numaresourcesoperator/v1"
"github.com/openshift-kni/numaresources-operator/pkg/hash"
nrosched "github.com/openshift-kni/numaresources-operator/pkg/numaresourcesscheduler"
schedmanifests "github.com/openshift-kni/numaresources-operator/pkg/numaresourcesscheduler/manifests/sched"
"github.com/openshift-kni/numaresources-operator/pkg/numaresourcesscheduler/objectstate/sched"
schedupdate "github.com/openshift-kni/numaresources-operator/pkg/objectupdate/sched"
Expand Down Expand Up @@ -85,7 +86,7 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
ginkgo.Context("with unexpected NRS CR name", func() {
ginkgo.It("should updated the CR condition to degraded", func() {
nrs := testobjs.NewNUMAResourcesScheduler("test", "some/url:latest", testSchedulerName, 9*time.Second)
verifyDegradedCondition(nrs, conditionTypeIncorrectNUMAResourcesSchedulerResourceName)
verifyDegradedCondition(nrs, status.ConditionTypeIncorrectNUMAResourcesSchedulerResourceName)
})
})

Expand Down Expand Up @@ -221,7 +222,7 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
dp := &appsv1.Deployment{}
gomega.Expect(reconciler.Client.Get(context.TODO(), key, dp)).ToNot(gomega.HaveOccurred())

gomega.Expect(dp.Spec.Template.Spec.PriorityClassName).To(gomega.BeEquivalentTo(schedulerPriorityClassName))
gomega.Expect(dp.Spec.Template.Spec.PriorityClassName).To(gomega.BeEquivalentTo(nrosched.SchedulerPriorityClassName))
})

ginkgo.It("should have a config hash annotation under deployment", func() {
Expand Down Expand Up @@ -434,7 +435,6 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
_, err := reconciler.Reconcile(context.TODO(), reconcile.Request{NamespacedName: key})
gomega.Expect(err).ToNot(gomega.HaveOccurred())
expectCacheParams(reconciler.Client, depmanifests.CacheResyncAutodetect, depmanifests.CacheResyncOnlyExclusiveResources, depmanifests.CacheInformerDedicated)

})

ginkgo.It("should allow to change the informerMode to Shared", func() {
Expand All @@ -454,7 +454,6 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
gomega.Expect(err).ToNot(gomega.HaveOccurred())

expectCacheParams(reconciler.Client, depmanifests.CacheResyncAutodetect, depmanifests.CacheResyncOnlyExclusiveResources, depmanifests.CacheInformerShared)

})

ginkgo.It("should allow to change the informerMode to Dedicated", func() {
Expand All @@ -474,7 +473,6 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
gomega.Expect(err).ToNot(gomega.HaveOccurred())

expectCacheParams(reconciler.Client, depmanifests.CacheResyncAutodetect, depmanifests.CacheResyncOnlyExclusiveResources, depmanifests.CacheInformerDedicated)

})

ginkgo.It("should configure by default the ScoringStrategy to be LeastAllocated", func() {
Expand Down Expand Up @@ -504,7 +502,6 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
gomega.Expect(err).ToNot(gomega.HaveOccurred())
resources := []depmanifests.ResourceSpecParams{{Name: "cpu", Weight: 10}, {Name: "memory", Weight: 5}}
expectScoringStrategyParams(reconciler.Client, depmanifests.ScoringStrategyLeastAllocated, resources)

})

ginkgo.It("should allow to change the ScoringStrategy to BalancedAllocation", func() {
Expand All @@ -524,7 +521,6 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
gomega.Expect(err).ToNot(gomega.HaveOccurred())
var resources []depmanifests.ResourceSpecParams
expectScoringStrategyParams(reconciler.Client, depmanifests.ScoringStrategyBalancedAllocation, resources)

})

ginkgo.It("should allow to change the ScoringStrategy to MostAllocated", func() {
Expand All @@ -544,7 +540,6 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
gomega.Expect(err).ToNot(gomega.HaveOccurred())
var resources []depmanifests.ResourceSpecParams
expectScoringStrategyParams(reconciler.Client, depmanifests.ScoringStrategyMostAllocated, resources)

})

ginkgo.It("should allow to change the ScoringStrategy to BalancedAllocation with resources", func() {
Expand All @@ -566,7 +561,6 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
gomega.Expect(err).ToNot(gomega.HaveOccurred())
resources := []depmanifests.ResourceSpecParams{{Name: "cpu", Weight: 10}, {Name: "memory", Weight: 5}}
expectScoringStrategyParams(reconciler.Client, depmanifests.ScoringStrategyBalancedAllocation, resources)

})

ginkgo.It("should allow to change the ScoringStrategy to MostAllocated with resources", func() {
Expand All @@ -588,8 +582,29 @@ var _ = ginkgo.Describe("Test NUMAResourcesScheduler Reconcile", func() {
gomega.Expect(err).ToNot(gomega.HaveOccurred())
resources := []depmanifests.ResourceSpecParams{{Name: "cpu", Weight: 10}, {Name: "memory", Weight: 5}}
expectScoringStrategyParams(reconciler.Client, depmanifests.ScoringStrategyMostAllocated, resources)
})

ginkgo.It("should set the leader election resource parameters by default", func() {
key := client.ObjectKeyFromObject(nrs)
_, err := reconciler.Reconcile(context.TODO(), reconcile.Request{NamespacedName: key})
gomega.Expect(err).ToNot(gomega.HaveOccurred())
expectLeaderElectParams(reconciler.Client, false, testNamespace, nrosched.LeaderElectionResourceName)
})

ginkgo.DescribeTable("should set the leader election resource parameters depending on replica count", func(replicas int32, expectedEnabled bool) {
nrs := nrs.DeepCopy()
nrs.Spec.Replicas = &replicas
gomega.Eventually(reconciler.Client.Update(context.TODO(), nrs)).WithPolling(30 * time.Second).WithTimeout(5 * time.Minute).Should(gomega.Succeed())

key := client.ObjectKeyFromObject(nrs)
_, err := reconciler.Reconcile(context.TODO(), reconcile.Request{NamespacedName: key})
gomega.Expect(err).ToNot(gomega.HaveOccurred())
expectLeaderElectParams(reconciler.Client, expectedEnabled, testNamespace, nrosched.LeaderElectionResourceName)
},
ginkgo.Entry("replicas=0", int32(0), false),
ginkgo.Entry("replicas=1", int32(1), false),
ginkgo.Entry("replicas=3", int32(3), true),
)
})
})

Expand Down Expand Up @@ -658,3 +673,25 @@ func expectScoringStrategyParams(cli client.Client, ScoringStrategyType string,
gomega.Expect(cfg.ScoringStrategy.Type).To(gomega.Equal(ScoringStrategyType))
gomega.Expect(cfg.ScoringStrategy.Resources).To(gomega.Equal(resources))
}

func expectLeaderElectParams(cli client.Client, enabled bool, resourceNamespace, resourceName string) {
ginkgo.GinkgoHelper()

key := client.ObjectKey{
Name: "topo-aware-scheduler-config",
Namespace: testNamespace,
}

cm := corev1.ConfigMap{}
gomega.Expect(cli.Get(context.TODO(), key, &cm)).To(gomega.Succeed())

confRaw := cm.Data[sched.SchedulerConfigFileName]
cfgs, err := depmanifests.DecodeSchedulerProfilesFromData([]byte(confRaw))
gomega.Expect(err).ToNot(gomega.HaveOccurred())
gomega.Expect(cfgs).To(gomega.HaveLen(1), "unexpected config params count: %d", len(cfgs))
cfg := cfgs[0]

gomega.Expect(cfg.LeaderElection.LeaderElect).To(gomega.Equal(enabled))
gomega.Expect(cfg.LeaderElection.ResourceNamespace).To(gomega.Equal(resourceNamespace))
gomega.Expect(cfg.LeaderElection.ResourceName).To(gomega.Equal(resourceName))
}
22 changes: 22 additions & 0 deletions pkg/numaresourcesscheduler/consts.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
/*
* Copyright 2021 Red Hat, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package numaresourcesscheduler

const (
LeaderElectionResourceName = "numa-scheduler-leader"
SchedulerPriorityClassName = "system-node-critical"
)
4 changes: 4 additions & 0 deletions pkg/status/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,10 @@ const (
ConditionTypeIncorrectNUMAResourcesOperatorResourceName = "IncorrectNUMAResourcesOperatorResourceName"
)

const (
ConditionTypeIncorrectNUMAResourcesSchedulerResourceName = "IncorrectNUMAResourcesSchedulerResourceName"
)

func IsUpdatedNUMAResourcesOperator(oldStatus, newStatus *nropv1.NUMAResourcesOperatorStatus) bool {
options := []cmp.Option{
cmpopts.IgnoreFields(metav1.Condition{}, "LastTransitionTime"),
Expand Down

0 comments on commit ae8a5c4

Please sign in to comment.