Skip to content

Commit

Permalink
Validate whether Scylla container is tuned before Pod is unblocked.
Browse files Browse the repository at this point in the history
Scylla Pod was unblocked even when container wasn't yet tuned.
E2E validating node optimizations wasn't validating it either giving us a false positive.
  • Loading branch information
zimnx committed Mar 20, 2024
1 parent fefabc3 commit 943f9ca
Show file tree
Hide file tree
Showing 7 changed files with 159 additions and 3 deletions.
4 changes: 4 additions & 0 deletions pkg/controller/nodeconfigpod/sync_configmaps.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,10 @@ func (ncpc *Controller) makeConfigMap(ctx context.Context, pod *corev1.Pod) (*co
continue
}

if !controllerhelpers.IsPodTunable(pod) {
continue
}

if controllerhelpers.IsNodeTunedForContainer(nc, node.Name, containerID) {
continue
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/controller/nodetune/sync_jobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,7 @@ func (ncdc *Controller) makeJobForContainers(ctx context.Context) (*batchv1.Job,
for i := range localScyllaPods {
scyllaPod := localScyllaPods[i]

if scyllaPod.Status.QOSClass != corev1.PodQOSGuaranteed {
if !controllerhelpers.IsPodTunable(scyllaPod) {
klog.V(4).Infof("Pod %q isn't a subject for optimizations", naming.ObjRef(scyllaPod))
continue
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/controller/nodetune/tune.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
func getIRQCPUs(ctx context.Context, kubeletPodResourcesClient kubelet.PodResourcesClient, scyllaPods []*corev1.Pod, hostFullCpuset cpuset.CPUSet) (cpuset.CPUSet, error) {
scyllaCPUs := cpuset.CPUSet{}
for _, scyllaPod := range scyllaPods {
if scyllaPod.Status.QOSClass != corev1.PodQOSGuaranteed {
if !controllerhelpers.IsPodTunable(scyllaPod) {
continue
}

Expand Down
12 changes: 11 additions & 1 deletion pkg/controllerhelpers/scylla.go
Original file line number Diff line number Diff line change
Expand Up @@ -318,7 +318,17 @@ func IsNodeTunedForContainer(nc *scyllav1alpha1.NodeConfig, nodeName string, con
return false
}

return true
for _, cid := range ns.TunedContainers {
if cid == containerID {
return true
}
}

return false
}

func IsPodTunable(pod *corev1.Pod) bool {
return pod.Status.QOSClass == corev1.PodQOSGuaranteed
}

func IsNodeTuned(ncnss []scyllav1alpha1.NodeConfigNodeStatus, nodeName string) bool {
Expand Down
100 changes: 100 additions & 0 deletions pkg/controllerhelpers/scylla_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -781,3 +781,103 @@ func TestGetRequiredScyllaHosts(t *testing.T) {
})
}
}

func TestIsNodeTunedForContainer(t *testing.T) {
t.Parallel()

tt := []struct {
name string
nodeConfig *scyllav1alpha1.NodeConfig
nodeName string
containerID string
expectedTuned bool
}{
{
name: "empty nodeConfig status isn't considered to be tuned",
nodeConfig: &scyllav1alpha1.NodeConfig{},
nodeName: "node1",
containerID: "container-id",
expectedTuned: false,
},
{
name: "nodeConfig of different node",
nodeConfig: &scyllav1alpha1.NodeConfig{
Status: scyllav1alpha1.NodeConfigStatus{
NodeStatuses: []scyllav1alpha1.NodeConfigNodeStatus{
{
Name: "different-node",
TunedNode: true,
TunedContainers: []string{"container-id"},
},
},
},
},
nodeName: "node1",
containerID: "container-id",
expectedTuned: false,
},
{
name: "not tuned node having no tuned containers isn't considered to be tuned",
nodeConfig: &scyllav1alpha1.NodeConfig{
Status: scyllav1alpha1.NodeConfigStatus{
NodeStatuses: []scyllav1alpha1.NodeConfigNodeStatus{
{
Name: "node1",
TunedNode: false,
TunedContainers: []string{},
},
},
},
},
nodeName: "node1",
containerID: "container-id",
expectedTuned: false,
},
{
name: "tuned node but with different tuned container isn't considered to be tuned",
nodeConfig: &scyllav1alpha1.NodeConfig{
Status: scyllav1alpha1.NodeConfigStatus{
NodeStatuses: []scyllav1alpha1.NodeConfigNodeStatus{
{
Name: "node1",
TunedNode: true,
TunedContainers: []string{"different-container-id"},
},
},
},
},
nodeName: "node1",
containerID: "container-id",
expectedTuned: false,
},
{
name: "tuned node having matching tuned container is considered to be tuned",
nodeConfig: &scyllav1alpha1.NodeConfig{
Status: scyllav1alpha1.NodeConfigStatus{
NodeStatuses: []scyllav1alpha1.NodeConfigNodeStatus{
{
Name: "node1",
TunedNode: true,
TunedContainers: []string{"different-container-id", "container-id"},
},
},
},
},
nodeName: "node1",
containerID: "container-id",
expectedTuned: true,
},
}

for i := range tt {
tc := tt[i]
t.Run(tc.name, func(t *testing.T) {
t.Parallel()

tuned := IsNodeTunedForContainer(tc.nodeConfig, tc.nodeName, tc.containerID)
if tuned != tc.expectedTuned {
t.Errorf("expected %v, got %v", tc.expectedTuned, tuned)
}
})
}
}
35 changes: 35 additions & 0 deletions test/e2e/set/nodeconfig/nodeconfig_optimizations.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
o "github.com/onsi/gomega"
scyllav1alpha1 "github.com/scylladb/scylla-operator/pkg/api/scylla/v1alpha1"
"github.com/scylladb/scylla-operator/pkg/controllerhelpers"
"github.com/scylladb/scylla-operator/pkg/helpers/slices"
"github.com/scylladb/scylla-operator/pkg/internalapi"
"github.com/scylladb/scylla-operator/pkg/naming"
scyllafixture "github.com/scylladb/scylla-operator/test/e2e/fixture/scylla"
Expand Down Expand Up @@ -185,6 +186,26 @@ var _ = g.Describe("NodeConfig Optimizations", framework.Serial, func() {
o.Expect(err).NotTo(o.HaveOccurred())

sc := f.GetDefaultScyllaCluster()
sc.Spec.Datacenter.Racks[0].AgentResources = corev1.ResourceRequirements{
Requests: map[corev1.ResourceName]resource.Quantity{
corev1.ResourceCPU: resource.MustParse("50m"),
corev1.ResourceMemory: resource.MustParse("50Mi"),
},
Limits: map[corev1.ResourceName]resource.Quantity{
corev1.ResourceCPU: resource.MustParse("50m"),
corev1.ResourceMemory: resource.MustParse("50Mi"),
},
}
sc.Spec.Datacenter.Racks[0].Resources = corev1.ResourceRequirements{
Requests: map[corev1.ResourceName]resource.Quantity{
corev1.ResourceCPU: resource.MustParse("500m"),
corev1.ResourceMemory: resource.MustParse("1Gi"),
},
Limits: map[corev1.ResourceName]resource.Quantity{
corev1.ResourceCPU: resource.MustParse("500m"),
corev1.ResourceMemory: resource.MustParse("1Gi"),
},
}

framework.By("Creating a ScyllaCluster")
sc, err = f.ScyllaClient().ScyllaV1().ScyllaClusters(f.Namespace()).Create(ctx, sc, metav1.CreateOptions{})
Expand All @@ -198,6 +219,7 @@ var _ = g.Describe("NodeConfig Optimizations", framework.Serial, func() {
return true, nil
})
o.Expect(err).NotTo(o.HaveOccurred())
o.Expect(pod.Status.QOSClass).To(o.Equal(corev1.PodQOSGuaranteed))

cmName := naming.GetTuningConfigMapNameForPod(pod)
ctx2, ctx2Cancel := context.WithTimeout(ctx, 30*time.Second)
Expand Down Expand Up @@ -241,6 +263,18 @@ var _ = g.Describe("NodeConfig Optimizations", framework.Serial, func() {
)
o.Expect(err).NotTo(o.HaveOccurred())

pod, err = f.KubeClient().CoreV1().Pods(f.Namespace()).Get(
ctx,
utils.GetNodeName(sc, 0),
metav1.GetOptions{},
)
o.Expect(err).NotTo(o.HaveOccurred())

scyllaContainerStatus, _, ok := slices.Find(pod.Status.ContainerStatuses, func(status corev1.ContainerStatus) bool {
return status.Name == naming.ScyllaContainerName
})
o.Expect(ok).To(o.BeTrue())

framework.By("Waiting for the NodeConfig to deploy")
ctx3, ctx3Cancel := context.WithTimeout(ctx, nodeConfigRolloutTimeout)
defer ctx3Cancel()
Expand All @@ -251,6 +285,7 @@ var _ = g.Describe("NodeConfig Optimizations", framework.Serial, func() {
controllerhelpers.WaitForStateOptions{TolerateDelete: false},
utils.IsNodeConfigRolledOut,
utils.IsNodeConfigDoneWithNodeTuningFunc(matchingNodes),
utils.IsNodeConfigDoneWithContainerTuningFunc(pod.Spec.NodeName, scyllaContainerStatus.ContainerID),
)
o.Expect(err).NotTo(o.HaveOccurred())

Expand Down
7 changes: 7 additions & 0 deletions test/e2e/utils/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,13 @@ func IsNodeConfigDoneWithNodes(nodes []*corev1.Node) func(nc *scyllav1alpha1.Nod
}
}

func IsNodeConfigDoneWithContainerTuningFunc(nodeName, containerID string) func(nc *scyllav1alpha1.NodeConfig) (bool, error) {
return func(nc *scyllav1alpha1.NodeConfig) (bool, error) {
containerTuned := controllerhelpers.IsNodeTunedForContainer(nc, nodeName, containerID)
return containerTuned, nil
}
}

func IsNodeConfigDoneWithNodeTuningFunc(nodes []*corev1.Node) func(nc *scyllav1alpha1.NodeConfig) (bool, error) {
return func(nc *scyllav1alpha1.NodeConfig) (bool, error) {
for _, node := range nodes {
Expand Down

0 comments on commit 943f9ca

Please sign in to comment.