mirror of
https://github.com/edgelesssys/constellation.git
synced 2025-05-23 08:21:18 -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,23 @@
|
|||
load("@io_bazel_rules_go//go:def.bzl", "go_library")
|
||||
load("//bazel/go:go_test.bzl", "go_test")
|
||||
|
||||
go_library(
|
||||
name = "executor",
|
||||
srcs = ["executor.go"],
|
||||
importpath = "github.com/edgelesssys/constellation/v2/operators/constellation-node-operator/v2/internal/executor",
|
||||
visibility = ["//operators/constellation-node-operator:__subpackages__"],
|
||||
deps = [
|
||||
"@io_k8s_client_go//util/workqueue",
|
||||
"@io_k8s_sigs_controller_runtime//pkg/log",
|
||||
],
|
||||
)
|
||||
|
||||
go_test(
|
||||
name = "executor_test",
|
||||
srcs = ["executor_test.go"],
|
||||
embed = [":executor"],
|
||||
deps = [
|
||||
"@com_github_stretchr_testify//assert",
|
||||
"@org_uber_go_goleak//:goleak",
|
||||
],
|
||||
)
|
|
@ -0,0 +1,218 @@
|
|||
/*
|
||||
Copyright (c) Edgeless Systems GmbH
|
||||
|
||||
SPDX-License-Identifier: AGPL-3.0-only
|
||||
*/
|
||||
|
||||
// Package executor contains a task executor / scheduler for the constellation node operator.
|
||||
// It is used to execute tasks (outside of the k8s specific operator controllers) with regular intervals and
|
||||
// based of external triggers.
|
||||
package executor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"k8s.io/client-go/util/workqueue"
|
||||
"sigs.k8s.io/controller-runtime/pkg/log"
|
||||
)
|
||||
|
||||
const (
|
||||
defaultPollingFrequency = 15 * time.Minute
|
||||
// rateLimiterItem is the key used to rate limit the reconciliation loop.
|
||||
// since we don't have a reconcile request, we use a constant key.
|
||||
rateLimiterItem = "reconcile"
|
||||
)
|
||||
|
||||
// Controller is a type with a reconcile method.
|
||||
// It is modeled after the controller-runtime reconcile method,
|
||||
// but reconciles on external resources instead of k8s resources.
|
||||
type Controller interface {
|
||||
Reconcile(ctx context.Context) (Result, error)
|
||||
}
|
||||
|
||||
// Executor is a task executor / scheduler.
|
||||
// It will call the reconcile method of the given controller with a regular interval
|
||||
// or when triggered externally.
|
||||
type Executor struct {
|
||||
running atomic.Bool
|
||||
|
||||
// controller is the controller to be reconciled.
|
||||
controller Controller
|
||||
|
||||
// pollingFrequency is the default frequency with which the controller is reconciled
|
||||
// if no external trigger is received and no requeue is requested by the controller.
|
||||
pollingFrequency time.Duration
|
||||
// rateLimiter is used to rate limit the reconciliation loop.
|
||||
rateLimiter RateLimiter
|
||||
// externalTrigger is used to trigger a reconciliation immediately from the outside.
|
||||
// multiple triggers in a short time will be coalesced into one externalTrigger.
|
||||
externalTrigger chan struct{}
|
||||
// stop is used to stop the reconciliation loop.
|
||||
stop chan struct{}
|
||||
}
|
||||
|
||||
// New creates a new Executor.
|
||||
func New(controller Controller, cfg Config) *Executor {
|
||||
cfg.applyDefaults()
|
||||
return &Executor{
|
||||
controller: controller,
|
||||
pollingFrequency: cfg.PollingFrequency,
|
||||
rateLimiter: cfg.RateLimiter,
|
||||
}
|
||||
}
|
||||
|
||||
// StopWaitFn is a function that can be called to stop the executor and wait for it to stop.
|
||||
type StopWaitFn func()
|
||||
|
||||
// Start starts the executor in a separate go routine.
|
||||
// Call Stop to stop the executor.
|
||||
func (e *Executor) Start(ctx context.Context) StopWaitFn {
|
||||
wg := &sync.WaitGroup{}
|
||||
logr := log.FromContext(ctx)
|
||||
stopWait := func() {
|
||||
defer wg.Wait()
|
||||
e.Stop()
|
||||
}
|
||||
|
||||
// this will return early if the executor is already running
|
||||
// if the executor is not running, set the running flag to true
|
||||
// and continue
|
||||
if !e.running.CompareAndSwap(false, true) {
|
||||
return stopWait
|
||||
}
|
||||
|
||||
e.externalTrigger = make(chan struct{}, 1)
|
||||
e.stop = make(chan struct{}, 1)
|
||||
// execute is used by the go routines below to communicate
|
||||
// that a reconciliation should happen
|
||||
execute := make(chan struct{}, 1)
|
||||
// nextScheduledReconcile is used to communicate the next scheduled reconciliation time
|
||||
nextScheduledReconcile := make(chan time.Duration, 1)
|
||||
// trigger a reconciliation on startup
|
||||
nextScheduledReconcile <- 0
|
||||
|
||||
wg.Add(2)
|
||||
|
||||
// timer routine is responsible for triggering the reconciliation after the timer expires
|
||||
// or when triggered externally
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
defer close(execute)
|
||||
defer logr.Info("Timer stopped")
|
||||
for {
|
||||
nextScheduledReconcileAfter := <-nextScheduledReconcile
|
||||
timer := *time.NewTimer(nextScheduledReconcileAfter)
|
||||
select {
|
||||
case <-e.stop:
|
||||
return
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-e.externalTrigger:
|
||||
case <-timer.C:
|
||||
}
|
||||
execute <- struct{}{}
|
||||
}
|
||||
}()
|
||||
|
||||
// executor routine is responsible for executing the reconciliation
|
||||
go func() {
|
||||
defer func() {
|
||||
e.running.Store(false)
|
||||
}()
|
||||
defer wg.Done()
|
||||
defer close(nextScheduledReconcile)
|
||||
defer logr.Info("Executor stopped")
|
||||
|
||||
for {
|
||||
_, ok := <-execute
|
||||
// execute channel closed. executor should stop
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
res, err := e.controller.Reconcile(ctx)
|
||||
var requeueAfter time.Duration
|
||||
switch {
|
||||
case err != nil:
|
||||
logr.Error(err, "reconciliation failed")
|
||||
requeueAfter = e.rateLimiter.When(rateLimiterItem) // requeue with rate limiter
|
||||
case res.Requeue && res.RequeueAfter != 0:
|
||||
e.rateLimiter.Forget(rateLimiterItem) // reset the rate limiter
|
||||
requeueAfter = res.RequeueAfter // requeue after the given duration
|
||||
case res.Requeue:
|
||||
requeueAfter = e.rateLimiter.When(rateLimiterItem) // requeue with rate limiter
|
||||
default:
|
||||
e.rateLimiter.Forget(rateLimiterItem) // reset the rate limiter
|
||||
requeueAfter = e.pollingFrequency // default polling frequency
|
||||
}
|
||||
|
||||
nextScheduledReconcile <- requeueAfter
|
||||
}
|
||||
}()
|
||||
|
||||
return stopWait
|
||||
}
|
||||
|
||||
// Stop stops the executor.
|
||||
// It does not block until the executor is stopped.
|
||||
func (e *Executor) Stop() {
|
||||
select {
|
||||
case e.stop <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
close(e.stop)
|
||||
}
|
||||
|
||||
// Running returns true if the executor is running.
|
||||
// When the executor is stopped, it is not running anymore.
|
||||
func (e *Executor) Running() bool {
|
||||
return e.running.Load()
|
||||
}
|
||||
|
||||
// Trigger triggers a reconciliation.
|
||||
// If a reconciliation is already pending, this call is a no-op.
|
||||
func (e *Executor) Trigger() {
|
||||
select {
|
||||
case e.externalTrigger <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
// Config is the configuration for the executor.
|
||||
type Config struct {
|
||||
PollingFrequency time.Duration
|
||||
RateLimiter RateLimiter
|
||||
}
|
||||
|
||||
// NewDefaultConfig creates a new default configuration.
|
||||
func NewDefaultConfig() Config {
|
||||
cfg := Config{}
|
||||
cfg.applyDefaults()
|
||||
return cfg
|
||||
}
|
||||
|
||||
func (c *Config) applyDefaults() {
|
||||
if c.PollingFrequency == 0 {
|
||||
c.PollingFrequency = defaultPollingFrequency
|
||||
}
|
||||
if c.RateLimiter == nil {
|
||||
c.RateLimiter = workqueue.DefaultControllerRateLimiter()
|
||||
}
|
||||
}
|
||||
|
||||
// Result is the result of a reconciliation.
|
||||
type Result struct {
|
||||
Requeue bool
|
||||
RequeueAfter time.Duration
|
||||
}
|
||||
|
||||
// RateLimiter is a stripped down version of the controller-runtime ratelimiter.RateLimiter interface.
|
||||
type RateLimiter interface {
|
||||
// When gets an item and gets to decide how long that item should wait
|
||||
When(item any) time.Duration
|
||||
// Forget indicates that an item is finished being retried. Doesn't matter whether its for perm failing
|
||||
// or for success, we'll stop tracking it
|
||||
Forget(item any)
|
||||
}
|
|
@ -0,0 +1,306 @@
|
|||
/*
|
||||
Copyright (c) Edgeless Systems GmbH
|
||||
|
||||
SPDX-License-Identifier: AGPL-3.0-only
|
||||
*/
|
||||
|
||||
package executor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.uber.org/goleak"
|
||||
)
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
goleak.VerifyTestMain(m)
|
||||
}
|
||||
|
||||
func TestNew(t *testing.T) {
|
||||
testCases := map[string]struct {
|
||||
config Config
|
||||
wantPollingFrequency time.Duration
|
||||
}{
|
||||
"applies default polling frequency": {
|
||||
config: Config{},
|
||||
wantPollingFrequency: defaultPollingFrequency,
|
||||
},
|
||||
"custom polling frequency": {
|
||||
config: Config{
|
||||
PollingFrequency: time.Hour,
|
||||
},
|
||||
wantPollingFrequency: time.Hour,
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range testCases {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
exec := New(nil, tc.config)
|
||||
assert.Equal(tc.wantPollingFrequency, exec.pollingFrequency)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestStartTriggersImmediateReconciliation(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
ctrl := newStubController(Result{}, nil)
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
rateLimiter: &stubRateLimiter{}, // no rate limiting
|
||||
}
|
||||
// on start, the executor should trigger a reconciliation
|
||||
stopAndWait := exec.Start(context.Background())
|
||||
<-ctrl.waitUntilReconciled // makes sure to wait until reconcile was called
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
stopAndWait()
|
||||
|
||||
// initial trigger
|
||||
assert.Equal(1, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
func TestStartMultipleTimesIsCoalesced(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
ctrl := newStubController(Result{}, nil)
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
rateLimiter: &stubRateLimiter{}, // no rate limiting
|
||||
}
|
||||
// start once
|
||||
stopAndWait := exec.Start(context.Background())
|
||||
// start again multiple times
|
||||
for i := 0; i < 10; i++ {
|
||||
_ = exec.Start(context.Background())
|
||||
}
|
||||
|
||||
<-ctrl.waitUntilReconciled // makes sure to wait until reconcile was called
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
stopAndWait()
|
||||
|
||||
// initial trigger. extra start calls should be coalesced
|
||||
assert.Equal(1, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
func TestErrorTriggersImmediateReconciliation(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
// returning an error should trigger a reconciliation immediately
|
||||
ctrl := newStubController(Result{}, errors.New("reconciler error"))
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
rateLimiter: &stubRateLimiter{}, // no rate limiting
|
||||
}
|
||||
stopAndWait := exec.Start(context.Background())
|
||||
for i := 0; i < 10; i++ {
|
||||
<-ctrl.waitUntilReconciled // makes sure to wait until reconcile was called
|
||||
}
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
stopAndWait()
|
||||
|
||||
// we cannot assert the exact number of reconciliations here, because the executor might
|
||||
// select the stop case or the timer case first.
|
||||
assertBetween(assert, 10, 11, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
func TestErrorTriggersRateLimiting(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
// returning an error should trigger a reconciliation immediately
|
||||
ctrl := newStubController(Result{}, errors.New("reconciler error"))
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
rateLimiter: &stubRateLimiter{
|
||||
whenRes: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
},
|
||||
}
|
||||
stopAndWait := exec.Start(context.Background())
|
||||
<-ctrl.waitUntilReconciled // makes sure to wait until reconcile was called once to trigger rate limiting
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
stopAndWait()
|
||||
|
||||
// initial trigger. error triggers are rate limited to 1 per year
|
||||
assert.Equal(1, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
func TestRequeueAfterResultRequeueInterval(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
// setting a requeue result should trigger a reconciliation after the specified delay
|
||||
ctrl := newStubController(Result{
|
||||
Requeue: true,
|
||||
RequeueAfter: time.Microsecond,
|
||||
}, nil)
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
rateLimiter: &stubRateLimiter{
|
||||
whenRes: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
},
|
||||
}
|
||||
stopAndWait := exec.Start(context.Background())
|
||||
for i := 0; i < 10; i++ {
|
||||
<-ctrl.waitUntilReconciled // makes sure to wait until reconcile was called
|
||||
}
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
stopAndWait()
|
||||
|
||||
// we cannot assert the exact number of reconciliations here, because the executor might
|
||||
// select the stop case or the timer case first.
|
||||
assertBetween(assert, 10, 11, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
func TestExternalTrigger(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
ctrl := newStubController(Result{}, nil)
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
rateLimiter: &stubRateLimiter{
|
||||
whenRes: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
},
|
||||
}
|
||||
stopAndWait := exec.Start(context.Background())
|
||||
<-ctrl.waitUntilReconciled // initial trigger
|
||||
for i := 0; i < 10; i++ {
|
||||
exec.Trigger()
|
||||
<-ctrl.waitUntilReconciled // external trigger
|
||||
}
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
stopAndWait()
|
||||
|
||||
// initial trigger + 10 external triggers
|
||||
assert.Equal(11, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
func TestSimultaneousExternalTriggers(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
ctrl := newStubController(Result{}, nil)
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
rateLimiter: &stubRateLimiter{
|
||||
whenRes: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
},
|
||||
}
|
||||
stopAndWait := exec.Start(context.Background())
|
||||
<-ctrl.waitUntilReconciled // initial trigger
|
||||
for i := 0; i < 100; i++ {
|
||||
exec.Trigger() // extra trigger calls are coalesced
|
||||
}
|
||||
<-ctrl.waitUntilReconciled // external trigger
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
stopAndWait()
|
||||
|
||||
// we cannot assert the exact number of reconciliations here, because the executor might
|
||||
// select the stop case or the next manual trigger case first.
|
||||
assertBetween(assert, 2, 3, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
func TestContextCancel(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
ctrl := newStubController(Result{}, nil)
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
rateLimiter: &stubRateLimiter{}, // no rate limiting
|
||||
}
|
||||
_ = exec.Start(ctx) // no need to explicitly stop the executor, it will stop when the context is canceled
|
||||
<-ctrl.waitUntilReconciled // initial trigger
|
||||
|
||||
// canceling the context should stop the executor without blocking
|
||||
cancel()
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
// poll for the executor stop running
|
||||
// this is necessary since the executor doesn't expose
|
||||
// a pure wait method
|
||||
assert.Eventually(func() bool {
|
||||
return !exec.Running()
|
||||
}, 5*time.Second, 10*time.Millisecond)
|
||||
|
||||
// initial trigger
|
||||
assert.Equal(1, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
func TestRequeueAfterPollingFrequency(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
ctrl := newStubController(Result{}, nil)
|
||||
exec := Executor{
|
||||
controller: ctrl,
|
||||
pollingFrequency: time.Microsecond, // basically no delay
|
||||
rateLimiter: &stubRateLimiter{
|
||||
whenRes: time.Hour * 24 * 365, // 1 year. Should be high enough to not trigger the timer in the test.
|
||||
},
|
||||
}
|
||||
stopAndWait := exec.Start(context.Background())
|
||||
for i := 0; i < 10; i++ {
|
||||
<-ctrl.waitUntilReconciled // makes sure to wait until reconcile was called
|
||||
}
|
||||
ctrl.stop <- struct{}{}
|
||||
|
||||
stopAndWait()
|
||||
|
||||
// we cannot assert the exact number of reconciliations here, because the executor might
|
||||
// select the stop case or the timer case first.
|
||||
assertBetween(assert, 10, 11, ctrl.reconciliationCounter)
|
||||
}
|
||||
|
||||
type stubController struct {
|
||||
stopped bool
|
||||
stop chan struct{}
|
||||
waitUntilReconciled chan struct{}
|
||||
reconciliationCounter int
|
||||
result Result
|
||||
err error
|
||||
}
|
||||
|
||||
func newStubController(result Result, err error) *stubController {
|
||||
return &stubController{
|
||||
waitUntilReconciled: make(chan struct{}),
|
||||
stop: make(chan struct{}, 1),
|
||||
result: result,
|
||||
err: err,
|
||||
}
|
||||
}
|
||||
|
||||
func (s *stubController) Reconcile(_ context.Context) (Result, error) {
|
||||
if s.stopped {
|
||||
return Result{}, errors.New("controller stopped")
|
||||
}
|
||||
s.reconciliationCounter++
|
||||
select {
|
||||
case <-s.stop:
|
||||
s.stopped = true
|
||||
case s.waitUntilReconciled <- struct{}{}:
|
||||
}
|
||||
|
||||
return s.result, s.err
|
||||
}
|
||||
|
||||
type stubRateLimiter struct {
|
||||
whenRes time.Duration
|
||||
}
|
||||
|
||||
func (s *stubRateLimiter) When(_ any) time.Duration {
|
||||
return s.whenRes
|
||||
}
|
||||
|
||||
func (s *stubRateLimiter) Forget(_ any) {}
|
||||
|
||||
func assertBetween(assert *assert.Assertions, min, max, actual int) {
|
||||
assert.GreaterOrEqual(actual, min)
|
||||
assert.LessOrEqual(actual, max)
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue