mirror of
https://github.com/edgelesssys/constellation.git
synced 2025-07-25 16:25:21 -04:00
operators: infrastructure autodiscovery (#1958)
* helm: configure GCP cloud controller manager to search in all zones of a region
See also: d716fdd452/providers/gce/gce.go (L376-L380)
* operators: add nodeGroupName to ScalingGroup CRD
NodeGroupName is the human friendly name of the node group that will be exposed to customers via the Constellation config in the future.
* operators: support simple executor / scheduler to reconcile on non-k8s resources
* operators: add new return type for ListScalingGroups to support arbitrary node groups
* operators: ListScalingGroups should return additionally created node groups on AWS
* operators: ListScalingGroups should return additionally created node groups on Azure
* operators: ListScalingGroups should return additionally created node groups on GCP
* operators: ListScalingGroups should return additionally created node groups on unsupported CSPs
* operators: implement external scaling group reconciler
This controller scans the cloud provider infrastructure and changes k8s resources accordingly.
It creates ScaleSet resources when new node groups are created and deletes them if the node groups are removed.
* operators: no longer create scale sets when the operator starts
In the future, scale sets are created dynamically.
* operators: watch for node join/leave events using a controller
* operators: deploy new controllers
* docs: update auto scaling documentation with support for node groups
This commit is contained in:
parent
10a540c290
commit
388ff011a3
36 changed files with 1836 additions and 232 deletions
|
@ -0,0 +1,51 @@
|
|||
load("@io_bazel_rules_go//go:def.bzl", "go_library")
|
||||
load("//bazel/go:go_test.bzl", "go_test")
|
||||
|
||||
go_library(
|
||||
name = "sgreconciler",
|
||||
srcs = [
|
||||
"nodejoin_watcher.go",
|
||||
"scalinggroup_controller.go",
|
||||
"sgreconciler.go",
|
||||
],
|
||||
importpath = "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/sgreconciler",
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
"//internal/constants",
|
||||
"//operators/constellation-node-operator/api/v1alpha1",
|
||||
"//operators/constellation-node-operator/internal/cloud/api",
|
||||
"//operators/constellation-node-operator/internal/executor",
|
||||
"@io_k8s_api//core/v1:core",
|
||||
"@io_k8s_apimachinery//pkg/api/errors",
|
||||
"@io_k8s_apimachinery//pkg/apis/meta/v1:meta",
|
||||
"@io_k8s_apimachinery//pkg/runtime",
|
||||
"@io_k8s_apimachinery//pkg/types",
|
||||
"@io_k8s_client_go//util/retry",
|
||||
"@io_k8s_sigs_controller_runtime//:controller-runtime",
|
||||
"@io_k8s_sigs_controller_runtime//pkg/builder",
|
||||
"@io_k8s_sigs_controller_runtime//pkg/client",
|
||||
"@io_k8s_sigs_controller_runtime//pkg/event",
|
||||
"@io_k8s_sigs_controller_runtime//pkg/handler",
|
||||
"@io_k8s_sigs_controller_runtime//pkg/log",
|
||||
"@io_k8s_sigs_controller_runtime//pkg/predicate",
|
||||
],
|
||||
)
|
||||
|
||||
go_test(
|
||||
name = "sgreconciler_test",
|
||||
srcs = ["scalinggroup_controller_test.go"],
|
||||
embed = [":sgreconciler"],
|
||||
deps = [
|
||||
"//internal/constants",
|
||||
"//operators/constellation-node-operator/api/v1alpha1",
|
||||
"//operators/constellation-node-operator/internal/cloud/api",
|
||||
"//operators/constellation-node-operator/internal/constants",
|
||||
"@com_github_stretchr_testify//assert",
|
||||
"@com_github_stretchr_testify//require",
|
||||
"@io_k8s_apimachinery//pkg/api/errors",
|
||||
"@io_k8s_apimachinery//pkg/apis/meta/v1:meta",
|
||||
"@io_k8s_apimachinery//pkg/runtime/schema",
|
||||
"@io_k8s_apimachinery//pkg/types",
|
||||
"@io_k8s_sigs_controller_runtime//pkg/client",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
Copyright (c) Edgeless Systems GmbH
|
||||
|
||||
SPDX-License-Identifier: AGPL-3.0-only
|
||||
*/
|
||||
|
||||
package sgreconciler
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/apimachinery/pkg/runtime"
|
||||
"k8s.io/apimachinery/pkg/types"
|
||||
ctrl "sigs.k8s.io/controller-runtime"
|
||||
"sigs.k8s.io/controller-runtime/pkg/builder"
|
||||
"sigs.k8s.io/controller-runtime/pkg/client"
|
||||
"sigs.k8s.io/controller-runtime/pkg/event"
|
||||
"sigs.k8s.io/controller-runtime/pkg/handler"
|
||||
"sigs.k8s.io/controller-runtime/pkg/log"
|
||||
"sigs.k8s.io/controller-runtime/pkg/predicate"
|
||||
)
|
||||
|
||||
// NodeJoinWatcher watches node join / leave events.
|
||||
type NodeJoinWatcher struct {
|
||||
trigger func()
|
||||
client.Client
|
||||
Scheme *runtime.Scheme
|
||||
}
|
||||
|
||||
// NewNodeJoinWatcher creates a new NodeJoinWatcher.
|
||||
func NewNodeJoinWatcher(trigger func(), client client.Client, scheme *runtime.Scheme) *NodeJoinWatcher {
|
||||
return &NodeJoinWatcher{
|
||||
trigger: trigger,
|
||||
Client: client,
|
||||
Scheme: scheme,
|
||||
}
|
||||
}
|
||||
|
||||
//+kubebuilder:rbac:groups="",resources=nodes,verbs=get;list;watch;create;update;patch;delete
|
||||
//+kubebuilder:rbac:groups="",resources=nodes/status,verbs=get
|
||||
|
||||
// Reconcile reconciles node join / leave events.
|
||||
func (w *NodeJoinWatcher) Reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) {
|
||||
logr := log.FromContext(ctx)
|
||||
logr.Info("node has joined or left the cluster", "node", req.Name)
|
||||
w.trigger()
|
||||
|
||||
return ctrl.Result{}, nil
|
||||
}
|
||||
|
||||
// SetupWithManager sets up the controller with the Manager.
|
||||
func (w *NodeJoinWatcher) SetupWithManager(mgr ctrl.Manager) error {
|
||||
return ctrl.NewControllerManagedBy(mgr).
|
||||
Named("node-join-watcher").
|
||||
Watches(
|
||||
client.Object(&corev1.Node{}),
|
||||
handler.EnqueueRequestsFromMapFunc(func(ctx context.Context, obj client.Object) []ctrl.Request {
|
||||
return []ctrl.Request{{
|
||||
NamespacedName: types.NamespacedName{Name: obj.GetName()},
|
||||
}}
|
||||
}),
|
||||
builder.WithPredicates(nodeJoinLeavePredicate()),
|
||||
).
|
||||
Complete(w)
|
||||
}
|
||||
|
||||
// nodeJoinLeavePredicate returns a predicate that returns true if a node joins or leaves the cluster.
|
||||
func nodeJoinLeavePredicate() predicate.Predicate {
|
||||
return predicate.Funcs{
|
||||
// CreateFunc is not specified => never filter out create events
|
||||
// DeleteFunc is not specified => never filter out delete events
|
||||
UpdateFunc: func(e event.UpdateEvent) bool { return false },
|
||||
GenericFunc: func(e event.GenericEvent) bool { return false },
|
||||
}
|
||||
}
|
|
@ -0,0 +1,185 @@
|
|||
/*
|
||||
Copyright (c) Edgeless Systems GmbH
|
||||
|
||||
SPDX-License-Identifier: AGPL-3.0-only
|
||||
*/
|
||||
|
||||
package sgreconciler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"strings"
|
||||
|
||||
mainconstants "github.com/edgelesssys/constellation/v2/internal/constants"
|
||||
updatev1alpha1 "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/api/v1alpha1"
|
||||
cspapi "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/internal/cloud/api"
|
||||
"github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/internal/executor"
|
||||
k8sErrors "k8s.io/apimachinery/pkg/api/errors"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/client-go/util/retry"
|
||||
"sigs.k8s.io/controller-runtime/pkg/client"
|
||||
"sigs.k8s.io/controller-runtime/pkg/log"
|
||||
)
|
||||
|
||||
const (
|
||||
// defaultScalingGroupMin is the default minimum number of nodes in a scaling group.
|
||||
// This value is used if the scaling group is created by the operator.
|
||||
// If a user modifies the scaling group, the operator will not overwrite the user's configuration.
|
||||
defaultScalingGroupMin = 1
|
||||
// defaultScalingGroupMax is the default maximum number of nodes in a scaling group.
|
||||
// This value is used if the scaling group is created by the operator.
|
||||
// If a user modifies the scaling group, the operator will not overwrite the user's configuration.
|
||||
defaultScalingGroupMax = 10
|
||||
)
|
||||
|
||||
// ExternalScalingGroupReconciler reconciles on scaling groups in CSP infrastructure.
|
||||
// It does NOT reconcile on k8s resources.
|
||||
// Instead, it scans the cloud provider infrastructure and changes k8s resources accordingly.
|
||||
type ExternalScalingGroupReconciler struct {
|
||||
// uid is the unique identifier of the Constellation cluster.
|
||||
uid string
|
||||
scalingGroupDiscoverer scalingGroupDiscoverer
|
||||
k8sClient k8sReadWriter
|
||||
}
|
||||
|
||||
// NewExternalScalingGroupReconciler creates a new InfrastructureReconciler.
|
||||
func NewExternalScalingGroupReconciler(uid string, discoverer scalingGroupDiscoverer, k8sClient k8sReadWriter) *ExternalScalingGroupReconciler {
|
||||
return &ExternalScalingGroupReconciler{
|
||||
uid: uid,
|
||||
scalingGroupDiscoverer: discoverer,
|
||||
k8sClient: k8sClient,
|
||||
}
|
||||
}
|
||||
|
||||
// Reconcile reconciles on scaling groups in CSP infrastructure.
|
||||
func (r *ExternalScalingGroupReconciler) Reconcile(ctx context.Context) (executor.Result, error) {
|
||||
logr := log.FromContext(ctx)
|
||||
logr.Info("reconciling external scaling groups")
|
||||
|
||||
nodeGroups, err := r.scalingGroupDiscoverer.ListScalingGroups(ctx, r.uid)
|
||||
if err != nil {
|
||||
return executor.Result{}, err
|
||||
}
|
||||
|
||||
existingNodeGroups := map[string]struct{}{}
|
||||
|
||||
// create all scaling groups that are newly discovered
|
||||
for _, group := range nodeGroups {
|
||||
exists, err := patchNodeGroupName(ctx, r.k8sClient, group.Name, group.NodeGroupName)
|
||||
if err != nil {
|
||||
return executor.Result{}, err
|
||||
}
|
||||
if exists {
|
||||
// scaling group already exists
|
||||
existingNodeGroups[group.Name] = struct{}{}
|
||||
continue
|
||||
}
|
||||
err = createScalingGroupIfNotExists(ctx, newScalingGroupConfig{
|
||||
k8sClient: r.k8sClient,
|
||||
resourceName: group.Name,
|
||||
groupID: group.GroupID,
|
||||
nodeGroupName: group.NodeGroupName,
|
||||
autoscalingGroupName: group.AutoscalingGroupName,
|
||||
role: group.Role,
|
||||
})
|
||||
if err != nil {
|
||||
return executor.Result{}, err
|
||||
}
|
||||
existingNodeGroups[group.Name] = struct{}{}
|
||||
}
|
||||
|
||||
logr.Info("ensured scaling groups are created", "count", len(nodeGroups))
|
||||
|
||||
// delete all scaling groups that no longer exist
|
||||
var scalingGroups updatev1alpha1.ScalingGroupList
|
||||
if err := r.k8sClient.List(ctx, &scalingGroups); err != nil {
|
||||
return executor.Result{}, err
|
||||
}
|
||||
for _, group := range scalingGroups.Items {
|
||||
if _, ok := existingNodeGroups[group.Name]; !ok {
|
||||
logr.Info("deleting scaling group", "name", group.Name)
|
||||
err := r.k8sClient.Delete(ctx, &group)
|
||||
if err != nil {
|
||||
return executor.Result{}, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
logr.Info("external scaling groups reconciled")
|
||||
|
||||
return executor.Result{}, nil
|
||||
}
|
||||
|
||||
// patchNodeGroupName patches the node group name of a scaling group resource (if necessary and it exists).
|
||||
func patchNodeGroupName(ctx context.Context, k8sClient k8sReadWriter, resourceName, nodeGroupName string) (exists bool, err error) {
|
||||
logr := log.FromContext(ctx)
|
||||
var scalingGroup updatev1alpha1.ScalingGroup
|
||||
err = k8sClient.Get(ctx, client.ObjectKey{Name: resourceName}, &scalingGroup)
|
||||
if k8sErrors.IsNotFound(err) {
|
||||
// scaling group does not exist
|
||||
// no need to patch
|
||||
return false /* doesn't exist */, nil
|
||||
}
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if scalingGroup.Spec.NodeGroupName == nodeGroupName {
|
||||
// scaling group already has the correct node group name
|
||||
return true /* exists */, nil
|
||||
}
|
||||
logr.Info("patching node group name", "resourceName", resourceName, "nodeGroupName", nodeGroupName)
|
||||
return true, retry.RetryOnConflict(retry.DefaultRetry, func() error {
|
||||
if err := k8sClient.Get(ctx, client.ObjectKey{Name: resourceName}, &scalingGroup); err != nil {
|
||||
return err
|
||||
}
|
||||
scalingGroup.Spec.NodeGroupName = nodeGroupName
|
||||
return k8sClient.Update(ctx, &scalingGroup)
|
||||
})
|
||||
}
|
||||
|
||||
func createScalingGroupIfNotExists(ctx context.Context, config newScalingGroupConfig) error {
|
||||
logr := log.FromContext(ctx)
|
||||
err := config.k8sClient.Create(ctx, &updatev1alpha1.ScalingGroup{
|
||||
TypeMeta: metav1.TypeMeta{APIVersion: "update.edgeless.systems/v1alpha1", Kind: "ScalingGroup"},
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: strings.ToLower(config.resourceName),
|
||||
},
|
||||
Spec: updatev1alpha1.ScalingGroupSpec{
|
||||
NodeVersion: mainconstants.NodeVersionResourceName,
|
||||
GroupID: config.groupID,
|
||||
AutoscalerGroupName: config.autoscalingGroupName,
|
||||
NodeGroupName: config.nodeGroupName,
|
||||
Min: defaultScalingGroupMin,
|
||||
Max: defaultScalingGroupMax,
|
||||
Role: config.role,
|
||||
},
|
||||
})
|
||||
if k8sErrors.IsAlreadyExists(err) {
|
||||
return nil
|
||||
} else if err == nil {
|
||||
logr.Info("created scaling group", "name", config.resourceName, "nodeGroupName", config.nodeGroupName)
|
||||
} else {
|
||||
logr.Error(err, "failed to create scaling group", "name", config.resourceName, "nodeGroupName", config.nodeGroupName)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// scalingGroupDiscoverer is used to discover scaling groups in the cloud provider infrastructure.
|
||||
type scalingGroupDiscoverer interface {
|
||||
ListScalingGroups(ctx context.Context, uid string,
|
||||
) ([]cspapi.ScalingGroup, error)
|
||||
}
|
||||
|
||||
type k8sReadWriter interface {
|
||||
client.Reader
|
||||
client.Writer
|
||||
}
|
||||
|
||||
type newScalingGroupConfig struct {
|
||||
k8sClient client.Writer
|
||||
resourceName string
|
||||
groupID string
|
||||
nodeGroupName string
|
||||
autoscalingGroupName string
|
||||
role updatev1alpha1.NodeRole
|
||||
}
|
|
@ -0,0 +1,98 @@
|
|||
//go:build integration
|
||||
|
||||
/*
|
||||
Copyright (c) Edgeless Systems GmbH
|
||||
|
||||
SPDX-License-Identifier: AGPL-3.0-only
|
||||
*/
|
||||
|
||||
package sgreconciler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
cspapi "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/internal/cloud/api"
|
||||
. "github.com/onsi/ginkgo/v2"
|
||||
. "github.com/onsi/gomega"
|
||||
"k8s.io/apimachinery/pkg/types"
|
||||
|
||||
updatev1alpha1 "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/api/v1alpha1"
|
||||
)
|
||||
|
||||
var _ = Describe("ExternalScalingGroup controller", func() {
|
||||
AfterEach(func() {
|
||||
Eventually(func() error {
|
||||
return resetEnv()
|
||||
}, 30*time.Second, 1*time.Second).Should(Succeed())
|
||||
})
|
||||
|
||||
// Define utility constants for object names and testing timeouts/durations and intervals.
|
||||
const (
|
||||
nodeVersionName = "node-version"
|
||||
scalingGroupName = "test-group"
|
||||
nodeGroupName = "test-node-group"
|
||||
groupID = "test-group-id"
|
||||
autoscalingGroupName = "test-autoscaling-group-name"
|
||||
|
||||
timeout = time.Second * 20
|
||||
interval = time.Millisecond * 250
|
||||
)
|
||||
|
||||
Context("When creating a scaling group externally", func() {
|
||||
It("Should create a corresponding scaling group resource", func() {
|
||||
By("creating an external scaling group")
|
||||
ctx := context.Background()
|
||||
fakes.scalingGroupDiscoverer.set([]cspapi.ScalingGroup{
|
||||
{
|
||||
Name: scalingGroupName,
|
||||
NodeGroupName: nodeGroupName,
|
||||
GroupID: groupID,
|
||||
AutoscalingGroupName: autoscalingGroupName,
|
||||
Role: updatev1alpha1.ControlPlaneRole,
|
||||
},
|
||||
})
|
||||
|
||||
By("checking that the scaling group resource was created")
|
||||
triggerReconcile()
|
||||
createdScalingGroup := &updatev1alpha1.ScalingGroup{}
|
||||
Eventually(func() error {
|
||||
return k8sClient.Get(ctx, types.NamespacedName{Name: scalingGroupName}, createdScalingGroup)
|
||||
}, timeout, interval).Should(Succeed())
|
||||
Expect(createdScalingGroup.Spec.GroupID).Should(Equal(groupID))
|
||||
Expect(createdScalingGroup.Spec.AutoscalerGroupName).Should(Equal(autoscalingGroupName))
|
||||
Expect(createdScalingGroup.Spec.NodeGroupName).Should(Equal(nodeGroupName))
|
||||
Expect(createdScalingGroup.Spec.Role).Should(Equal(updatev1alpha1.ControlPlaneRole))
|
||||
})
|
||||
|
||||
It("Should update if external scaling groups are added/removed", func() {
|
||||
By("changing the external scaling groups")
|
||||
fakes.scalingGroupDiscoverer.set([]cspapi.ScalingGroup{
|
||||
{
|
||||
Name: "other-scaling-group",
|
||||
NodeGroupName: "other-node-group",
|
||||
GroupID: "other-group-id",
|
||||
AutoscalingGroupName: "other-autoscaling-group-name",
|
||||
Role: updatev1alpha1.WorkerRole,
|
||||
},
|
||||
})
|
||||
|
||||
By("checking that the scaling group resource was created")
|
||||
triggerReconcile()
|
||||
createdScalingGroup := &updatev1alpha1.ScalingGroup{}
|
||||
Eventually(func() error {
|
||||
return k8sClient.Get(ctx, types.NamespacedName{Name: "other-scaling-group"}, createdScalingGroup)
|
||||
}, timeout, interval).Should(Succeed())
|
||||
Expect(createdScalingGroup.Spec.GroupID).Should(Equal("other-group-id"))
|
||||
Expect(createdScalingGroup.Spec.AutoscalerGroupName).Should(Equal("other-autoscaling-group-name"))
|
||||
Expect(createdScalingGroup.Spec.NodeGroupName).Should(Equal("other-node-group"))
|
||||
Expect(createdScalingGroup.Spec.Role).Should(Equal(updatev1alpha1.WorkerRole))
|
||||
|
||||
By("checking that the old scaling group resource was deleted")
|
||||
deletedScalingGroup := &updatev1alpha1.ScalingGroup{}
|
||||
Eventually(func() error {
|
||||
return k8sClient.Get(ctx, types.NamespacedName{Name: scalingGroupName}, deletedScalingGroup)
|
||||
}, timeout, interval).ShouldNot(Succeed())
|
||||
})
|
||||
})
|
||||
})
|
|
@ -0,0 +1,265 @@
|
|||
/*
|
||||
Copyright (c) Edgeless Systems GmbH
|
||||
|
||||
SPDX-License-Identifier: AGPL-3.0-only
|
||||
*/
|
||||
|
||||
package sgreconciler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"sync"
|
||||
"testing"
|
||||
|
||||
mainconstants "github.com/edgelesssys/constellation/v2/internal/constants"
|
||||
updatev1alpha1 "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/api/v1alpha1"
|
||||
cspapi "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/internal/cloud/api"
|
||||
"github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/internal/constants"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
k8sErrors "k8s.io/apimachinery/pkg/api/errors"
|
||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||
"k8s.io/apimachinery/pkg/runtime/schema"
|
||||
"k8s.io/apimachinery/pkg/types"
|
||||
"sigs.k8s.io/controller-runtime/pkg/client"
|
||||
)
|
||||
|
||||
func TestCreateScalingGroupIfNotExists(t *testing.T) {
|
||||
testCases := map[string]struct {
|
||||
createErr error
|
||||
wantScalingGroup *updatev1alpha1.ScalingGroup
|
||||
wantErr bool
|
||||
}{
|
||||
"create works": {
|
||||
wantScalingGroup: &updatev1alpha1.ScalingGroup{
|
||||
TypeMeta: metav1.TypeMeta{APIVersion: "update.edgeless.systems/v1alpha1", Kind: "ScalingGroup"},
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "resource-name",
|
||||
},
|
||||
Spec: updatev1alpha1.ScalingGroupSpec{
|
||||
NodeVersion: mainconstants.NodeVersionResourceName,
|
||||
GroupID: "group-id",
|
||||
AutoscalerGroupName: "autoscaling-group-name",
|
||||
NodeGroupName: "node-group-name",
|
||||
Min: 1,
|
||||
Max: 10,
|
||||
Role: updatev1alpha1.WorkerRole,
|
||||
},
|
||||
},
|
||||
},
|
||||
"create fails": {
|
||||
createErr: errors.New("create failed"),
|
||||
wantErr: true,
|
||||
},
|
||||
"scaling group exists": {
|
||||
createErr: k8sErrors.NewAlreadyExists(schema.GroupResource{}, constants.AutoscalingStrategyResourceName),
|
||||
wantScalingGroup: &updatev1alpha1.ScalingGroup{
|
||||
TypeMeta: metav1.TypeMeta{APIVersion: "update.edgeless.systems/v1alpha1", Kind: "ScalingGroup"},
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "resource-name",
|
||||
},
|
||||
Spec: updatev1alpha1.ScalingGroupSpec{
|
||||
NodeVersion: mainconstants.NodeVersionResourceName,
|
||||
GroupID: "group-id",
|
||||
AutoscalerGroupName: "autoscaling-group-name",
|
||||
NodeGroupName: "node-group-name",
|
||||
Min: 1,
|
||||
Max: 10,
|
||||
Role: updatev1alpha1.WorkerRole,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range testCases {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
require := require.New(t)
|
||||
|
||||
k8sClient := &fakeK8sClient{createErr: tc.createErr}
|
||||
newScalingGroupConfig := newScalingGroupConfig{
|
||||
k8sClient: k8sClient,
|
||||
resourceName: "resource-name",
|
||||
groupID: "group-id",
|
||||
nodeGroupName: "node-group-name",
|
||||
autoscalingGroupName: "autoscaling-group-name",
|
||||
role: updatev1alpha1.WorkerRole,
|
||||
}
|
||||
err := createScalingGroupIfNotExists(context.Background(), newScalingGroupConfig)
|
||||
if tc.wantErr {
|
||||
assert.Error(err)
|
||||
return
|
||||
}
|
||||
require.NoError(err)
|
||||
assert.Len(k8sClient.createdObjects, 1)
|
||||
assert.Equal(tc.wantScalingGroup, k8sClient.createdObjects[0])
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestPatchNodeGroupName(t *testing.T) {
|
||||
testCases := map[string]struct {
|
||||
getRes client.Object
|
||||
getErr error
|
||||
updateErr error
|
||||
wantExists bool
|
||||
wantErr bool
|
||||
}{
|
||||
"patching works": {
|
||||
getRes: &updatev1alpha1.ScalingGroup{
|
||||
TypeMeta: metav1.TypeMeta{APIVersion: "update.edgeless.systems/v1alpha1", Kind: "ScalingGroup"},
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "resource-name",
|
||||
},
|
||||
Spec: updatev1alpha1.ScalingGroupSpec{
|
||||
NodeVersion: mainconstants.NodeVersionResourceName,
|
||||
GroupID: "group-id",
|
||||
AutoscalerGroupName: "autoscaling-group-name",
|
||||
Autoscaling: true,
|
||||
Min: 1,
|
||||
Max: 10,
|
||||
Role: updatev1alpha1.ControlPlaneRole,
|
||||
},
|
||||
},
|
||||
wantExists: true,
|
||||
},
|
||||
"name already set": {
|
||||
getRes: &updatev1alpha1.ScalingGroup{
|
||||
TypeMeta: metav1.TypeMeta{APIVersion: "update.edgeless.systems/v1alpha1", Kind: "ScalingGroup"},
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "resource-name",
|
||||
},
|
||||
Spec: updatev1alpha1.ScalingGroupSpec{
|
||||
NodeVersion: mainconstants.NodeVersionResourceName,
|
||||
GroupID: "group-id",
|
||||
NodeGroupName: "node-group-name",
|
||||
AutoscalerGroupName: "autoscaling-group-name",
|
||||
Autoscaling: true,
|
||||
Min: 1,
|
||||
Max: 10,
|
||||
Role: updatev1alpha1.ControlPlaneRole,
|
||||
},
|
||||
},
|
||||
wantExists: true,
|
||||
},
|
||||
"does not exist": {
|
||||
getErr: k8sErrors.NewNotFound(schema.GroupResource{}, "resource-name"),
|
||||
wantExists: false,
|
||||
},
|
||||
"getting fails": {
|
||||
getErr: errors.New("get failed"),
|
||||
wantErr: true,
|
||||
},
|
||||
"patching fails": {
|
||||
getRes: &updatev1alpha1.ScalingGroup{
|
||||
TypeMeta: metav1.TypeMeta{APIVersion: "update.edgeless.systems/v1alpha1", Kind: "ScalingGroup"},
|
||||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "resource-name",
|
||||
},
|
||||
Spec: updatev1alpha1.ScalingGroupSpec{
|
||||
NodeVersion: mainconstants.NodeVersionResourceName,
|
||||
GroupID: "group-id",
|
||||
AutoscalerGroupName: "autoscaling-group-name",
|
||||
Autoscaling: true,
|
||||
Min: 1,
|
||||
Max: 10,
|
||||
Role: updatev1alpha1.ControlPlaneRole,
|
||||
},
|
||||
},
|
||||
updateErr: errors.New("patch failed"),
|
||||
wantErr: true,
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range testCases {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
require := require.New(t)
|
||||
|
||||
k8sClient := &fakeK8sClient{
|
||||
getRes: tc.getRes,
|
||||
getErr: tc.getErr,
|
||||
updateErr: tc.updateErr,
|
||||
}
|
||||
gotExists, gotErr := patchNodeGroupName(context.Background(), k8sClient, "resource-name", "node-group-name")
|
||||
if tc.wantErr {
|
||||
assert.Error(gotErr)
|
||||
return
|
||||
}
|
||||
require.NoError(gotErr)
|
||||
assert.Equal(tc.wantExists, gotExists)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type fakeK8sClient struct {
|
||||
getRes client.Object
|
||||
createdObjects []client.Object
|
||||
createErr error
|
||||
listErr error
|
||||
getErr error
|
||||
updateErr error
|
||||
client.Client
|
||||
}
|
||||
|
||||
func (s *fakeK8sClient) Create(_ context.Context, obj client.Object, _ ...client.CreateOption) error {
|
||||
for _, o := range s.createdObjects {
|
||||
if obj.GetName() == o.GetName() {
|
||||
return k8sErrors.NewAlreadyExists(schema.GroupResource{}, obj.GetName())
|
||||
}
|
||||
}
|
||||
|
||||
s.createdObjects = append(s.createdObjects, obj.DeepCopyObject().(client.Object))
|
||||
return s.createErr
|
||||
}
|
||||
|
||||
func (s *fakeK8sClient) Get(_ context.Context, _ types.NamespacedName, out client.Object, _ ...client.GetOption) error {
|
||||
if s.getErr != nil {
|
||||
return s.getErr
|
||||
}
|
||||
obj := s.getRes.DeepCopyObject()
|
||||
outVal := reflect.ValueOf(out)
|
||||
objVal := reflect.ValueOf(obj)
|
||||
if !objVal.Type().AssignableTo(outVal.Type()) {
|
||||
return fmt.Errorf("fake had type %s, but %s was asked for", objVal.Type(), outVal.Type())
|
||||
}
|
||||
reflect.Indirect(outVal).Set(reflect.Indirect(objVal))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *fakeK8sClient) Update(_ context.Context, _ client.Object, _ ...client.UpdateOption) error {
|
||||
return s.updateErr
|
||||
}
|
||||
|
||||
func (s *fakeK8sClient) List(_ context.Context, _ client.ObjectList, _ ...client.ListOption) error {
|
||||
return s.listErr
|
||||
}
|
||||
|
||||
type stubScalingGroupDiscoverer struct {
|
||||
sync.RWMutex
|
||||
groups []cspapi.ScalingGroup
|
||||
}
|
||||
|
||||
func (d *stubScalingGroupDiscoverer) ListScalingGroups(_ context.Context, _ string,
|
||||
) ([]cspapi.ScalingGroup, error) {
|
||||
d.RLock()
|
||||
defer d.RUnlock()
|
||||
ret := make([]cspapi.ScalingGroup, len(d.groups))
|
||||
copy(ret, d.groups)
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
func (d *stubScalingGroupDiscoverer) set(groups []cspapi.ScalingGroup) {
|
||||
d.Lock()
|
||||
defer d.Unlock()
|
||||
d.groups = groups
|
||||
}
|
||||
|
||||
func (d *stubScalingGroupDiscoverer) reset() {
|
||||
d.Lock()
|
||||
defer d.Unlock()
|
||||
d.groups = nil
|
||||
}
|
|
@ -0,0 +1,11 @@
|
|||
/*
|
||||
Copyright (c) Edgeless Systems GmbH
|
||||
|
||||
SPDX-License-Identifier: AGPL-3.0-only
|
||||
*/
|
||||
|
||||
// Package sgreconciler contains a reconciler that reconciles on cloud provider infrastructure.
|
||||
// It is used to create, delete and update the spec of infrastructure-related k8s resources based on the
|
||||
// actual state of the infrastructure.
|
||||
// It uses polling (with additional triggers) to check the state of the infrastructure.
|
||||
package sgreconciler
|
194
operators/constellation-node-operator/sgreconciler/suite_test.go
Normal file
194
operators/constellation-node-operator/sgreconciler/suite_test.go
Normal file
|
@ -0,0 +1,194 @@
|
|||
//go:build integration
|
||||
|
||||
/*
|
||||
Copyright (c) Edgeless Systems GmbH
|
||||
|
||||
SPDX-License-Identifier: AGPL-3.0-only
|
||||
*/
|
||||
|
||||
package sgreconciler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"path/filepath"
|
||||
"testing"
|
||||
|
||||
. "github.com/onsi/ginkgo/v2"
|
||||
. "github.com/onsi/gomega"
|
||||
appsv1 "k8s.io/api/apps/v1"
|
||||
corev1 "k8s.io/api/core/v1"
|
||||
"k8s.io/client-go/kubernetes/scheme"
|
||||
"k8s.io/client-go/rest"
|
||||
"sigs.k8s.io/controller-runtime/pkg/client"
|
||||
"sigs.k8s.io/controller-runtime/pkg/envtest"
|
||||
logf "sigs.k8s.io/controller-runtime/pkg/log"
|
||||
"sigs.k8s.io/controller-runtime/pkg/log/zap"
|
||||
|
||||
updatev1alpha1 "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/api/v1alpha1"
|
||||
"github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/internal/executor"
|
||||
//+kubebuilder:scaffold:imports
|
||||
)
|
||||
|
||||
// These tests use Ginkgo (BDD-style Go testing framework). Refer to
|
||||
// http://onsi.github.io/ginkgo/ to learn more about Ginkgo.
|
||||
|
||||
var (
|
||||
cfg *rest.Config
|
||||
k8sClient client.Client
|
||||
testEnv *envtest.Environment
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
fakes = newFakes()
|
||||
triggerReconcile func()
|
||||
stopAndWaitForExecutor func()
|
||||
)
|
||||
|
||||
func TestAPIs(t *testing.T) {
|
||||
RegisterFailHandler(Fail)
|
||||
suiteConfig, reporterConfig := GinkgoConfiguration()
|
||||
// If you want to debug a specific seed, set it here.
|
||||
// suiteConfig.RandomSeed = 1679587116
|
||||
reporterConfig.VeryVerbose = true
|
||||
RunSpecs(t, "Controller Suite", suiteConfig, reporterConfig)
|
||||
}
|
||||
|
||||
var _ = BeforeSuite(func() {
|
||||
logf.SetLogger(zap.New(zap.WriteTo(GinkgoWriter), zap.UseDevMode(true)))
|
||||
ctx, cancel = context.WithCancel(context.Background())
|
||||
|
||||
By("bootstrapping test environment")
|
||||
testEnv = &envtest.Environment{
|
||||
CRDDirectoryPaths: []string{
|
||||
filepath.Join("..", "config", "crd", "bases"),
|
||||
},
|
||||
ErrorIfCRDPathMissing: true,
|
||||
}
|
||||
|
||||
var err error
|
||||
// cfg is defined in this file globally.
|
||||
cfg, err = testEnv.Start()
|
||||
Expect(err).NotTo(HaveOccurred())
|
||||
Expect(cfg).NotTo(BeNil())
|
||||
|
||||
err = updatev1alpha1.AddToScheme(scheme.Scheme)
|
||||
Expect(err).NotTo(HaveOccurred())
|
||||
|
||||
//+kubebuilder:scaffold:scheme
|
||||
|
||||
k8sClient, err = client.New(cfg, client.Options{Scheme: scheme.Scheme})
|
||||
Expect(err).NotTo(HaveOccurred())
|
||||
Expect(k8sClient).NotTo(BeNil())
|
||||
|
||||
extScalingGroupReconciler := NewExternalScalingGroupReconciler(
|
||||
"uid",
|
||||
fakes.scalingGroupDiscoverer,
|
||||
k8sClient,
|
||||
)
|
||||
|
||||
exec := executor.New(extScalingGroupReconciler, executor.Config{})
|
||||
triggerReconcile = exec.Trigger
|
||||
|
||||
Expect(err).ToNot(HaveOccurred())
|
||||
|
||||
go func() {
|
||||
defer GinkgoRecover()
|
||||
stopAndWaitForExecutor = exec.Start(ctx)
|
||||
}()
|
||||
})
|
||||
|
||||
var _ = AfterSuite(func() {
|
||||
By("tearing down the test environment")
|
||||
cancel()
|
||||
defer stopAndWaitForExecutor()
|
||||
err := testEnv.Stop()
|
||||
Expect(err).NotTo(HaveOccurred())
|
||||
})
|
||||
|
||||
type fakeCollection struct {
|
||||
scalingGroupDiscoverer *stubScalingGroupDiscoverer
|
||||
}
|
||||
|
||||
func (c *fakeCollection) reset() {
|
||||
c.scalingGroupDiscoverer.reset()
|
||||
}
|
||||
|
||||
func newFakes() *fakeCollection {
|
||||
return &fakeCollection{
|
||||
scalingGroupDiscoverer: &stubScalingGroupDiscoverer{},
|
||||
}
|
||||
}
|
||||
|
||||
func resetEnv() error {
|
||||
// cleanup all nodes
|
||||
nodeList := &corev1.NodeList{}
|
||||
if err := k8sClient.List(context.Background(), nodeList); err != nil {
|
||||
return err
|
||||
}
|
||||
for _, node := range nodeList.Items {
|
||||
if err := k8sClient.Delete(context.Background(), &node); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// cleanup all node versions
|
||||
nodeVersionList := &updatev1alpha1.NodeVersionList{}
|
||||
if err := k8sClient.List(context.Background(), nodeVersionList); err != nil {
|
||||
return err
|
||||
}
|
||||
for _, nodeVersion := range nodeVersionList.Items {
|
||||
if err := k8sClient.Delete(context.Background(), &nodeVersion); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// cleanup all scaling groups
|
||||
scalingGroupList := &updatev1alpha1.ScalingGroupList{}
|
||||
if err := k8sClient.List(context.Background(), scalingGroupList); err != nil {
|
||||
return err
|
||||
}
|
||||
for _, scalingGroup := range scalingGroupList.Items {
|
||||
if err := k8sClient.Delete(context.Background(), &scalingGroup); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// cleanup all pending nodes
|
||||
pendingNodeList := &updatev1alpha1.PendingNodeList{}
|
||||
if err := k8sClient.List(context.Background(), pendingNodeList); err != nil {
|
||||
return err
|
||||
}
|
||||
for _, pendingNode := range pendingNodeList.Items {
|
||||
if err := k8sClient.Delete(context.Background(), &pendingNode); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// cleanup all joining nodes
|
||||
joiningNodeList := &updatev1alpha1.JoiningNodeList{}
|
||||
if err := k8sClient.List(context.Background(), joiningNodeList); err != nil {
|
||||
return err
|
||||
}
|
||||
for _, joiningNode := range joiningNodeList.Items {
|
||||
if err := k8sClient.Delete(context.Background(), &joiningNode); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// cleanup all autoscaling strategies
|
||||
autoscalingStrategyList := &updatev1alpha1.AutoscalingStrategyList{}
|
||||
if err := k8sClient.List(context.Background(), autoscalingStrategyList); err != nil {
|
||||
return err
|
||||
}
|
||||
for _, autoscalingStrategy := range autoscalingStrategyList.Items {
|
||||
if err := k8sClient.Delete(context.Background(), &autoscalingStrategy); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
// cleanup all deployments
|
||||
deploymentList := &appsv1.DeploymentList{}
|
||||
if err := k8sClient.List(context.Background(), deploymentList); err != nil {
|
||||
return err
|
||||
}
|
||||
for _, deployment := range deploymentList.Items {
|
||||
if err := k8sClient.Delete(context.Background(), &deployment); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
fakes.reset()
|
||||
return nil
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue