Merge pull request #856 from n3wscott/gate-until

Adding Gate and a CustomResourceGate reconciler
This commit is contained in:
Nic Cope 2025-07-11 15:08:17 -07:00 committed by GitHub
commit f5f608c93d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 1111 additions and 0 deletions

15
pkg/controller/gate.go Normal file
View File

@ -0,0 +1,15 @@
package controller
import (
"k8s.io/apimachinery/pkg/runtime/schema"
)
// A Gate is an interface to allow reconcilers to delay a callback until a set of GVKs are set to true inside the gate.
type Gate interface {
// Register to call a callback function when all given GVKs are marked true. If the callback is unblocked, the
// registration is removed.
Register(callback func(), gvks ...schema.GroupVersionKind)
// Set marks the associated condition to the given value. If the condition is already set as
// that value, then this is a no-op. Returns true if there was an update detected.
Set(gvk schema.GroupVersionKind, ready bool) bool
}

View File

@ -69,6 +69,9 @@ type Options struct {
// ChangeLogOptions for recording change logs.
ChangeLogOptions *ChangeLogOptions
// Gate implements a gated function callback pattern.
Gate Gate
}
// ForControllerRuntime extracts options for controller-runtime.

106
pkg/gate/gate.go Normal file
View File

@ -0,0 +1,106 @@
/*
Copyright 2025 The Crossplane Authors.
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 gate contains a gated function callback registration implementation.
package gate
import (
"slices"
"sync"
)
// Gate implements a gated function callback registration with comparable conditions.
type Gate[T comparable] struct {
mux sync.RWMutex
satisfied map[T]bool
fns []gated[T]
}
// gated is an internal tracking resource.
type gated[T comparable] struct {
// fn is the function callback we will invoke when all the dependent conditions are true.
fn func()
// depends is the list of conditions this gated function is waiting on. This is an AND.
depends []T
// released means the gated function has been invoked and we can garbage collect this gated function.
released bool
}
// Register a callback function that will be called when all the provided dependent conditions are true.
// After all conditions are true, the callback function is removed from the registration and will not be called again.
// Thread Safe.
func (g *Gate[T]) Register(fn func(), depends ...T) {
g.mux.Lock()
g.fns = append(g.fns, gated[T]{fn: fn, depends: depends})
g.mux.Unlock()
g.process()
}
// Set marks the associated condition to the given value. If the condition is already set as that value, then this is a
// no-op. Returns true if there was an update detected. Thread safe.
func (g *Gate[T]) Set(condition T, value bool) bool {
g.mux.Lock()
if g.satisfied == nil {
g.satisfied = make(map[T]bool)
}
old, found := g.satisfied[condition]
updated := false
if !found || old != value {
updated = true
g.satisfied[condition] = value
}
// process() would also like to lock the mux, so we must unlock here directly and not use defer.
g.mux.Unlock()
if updated {
g.process()
}
return updated
}
func (g *Gate[T]) process() {
g.mux.Lock()
defer g.mux.Unlock()
for i := range g.fns {
// release controls if we should release the function.
release := true
for _, dep := range g.fns[i].depends {
if !g.satisfied[dep] {
release = false
}
}
if release {
fn := g.fns[i].fn
// mark the function released so we can garbage collect after we are done with the loop.
g.fns[i].released = true
// Need to capture a copy of fn or else we would be accessing a deleted member when the go routine runs.
go fn()
}
}
// garbage collect released functions.
g.fns = slices.DeleteFunc(g.fns, func(a gated[T]) bool {
return a.released
})
}

299
pkg/gate/gate_test.go Normal file
View File

@ -0,0 +1,299 @@
/*
Copyright 2025 The Crossplane Authors.
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 gate_test
import (
"sync"
"testing"
"time"
"github.com/google/go-cmp/cmp"
"github.com/crossplane/crossplane-runtime/pkg/gate"
)
func TestGateRegister(t *testing.T) {
type args struct {
depends []string
}
type want struct {
called bool
}
cases := map[string]struct {
reason string
args args
want want
}{
"NoDependencies": {
reason: "Should immediately call function when no dependencies are required",
args: args{
depends: []string{},
},
want: want{
called: true,
},
},
"SingleDependency": {
reason: "Should not call function when dependency is not met",
args: args{
depends: []string{"condition1"},
},
want: want{
called: false,
},
},
"MultipleDependencies": {
reason: "Should not call function when multiple dependencies are not met",
args: args{
depends: []string{"condition1", "condition2"},
},
want: want{
called: false,
},
},
}
for name, tc := range cases {
t.Run(name, func(t *testing.T) {
g := new(gate.Gate[string])
called := false
g.Register(func() {
called = true
}, tc.args.depends...)
// Give some time for goroutine to execute
time.Sleep(10 * time.Millisecond)
if diff := cmp.Diff(tc.want.called, called); diff != "" {
t.Errorf("\n%s\nRegister(...): -want called, +got called:\n%s", tc.reason, diff)
}
})
}
}
func TestGateIntegration(t *testing.T) {
type want struct {
called bool
}
cases := map[string]struct {
reason string
setup func(g *gate.Gate[string]) chan bool
want want
}{
"SingleDependencyMet": {
reason: "Should call function when single dependency is met",
setup: func(g *gate.Gate[string]) chan bool {
called := make(chan bool, 1)
g.Register(func() {
called <- true
}, "condition1")
// Set condition to true (will be initialized as false first)
g.Set("condition1", true)
return called
},
want: want{
called: true,
},
},
"MultipleDependenciesMet": {
reason: "Should call function when all dependencies are met",
setup: func(g *gate.Gate[string]) chan bool {
called := make(chan bool, 1)
g.Register(func() {
called <- true
}, "condition1", "condition2")
// Set both conditions to true
g.Set("condition1", true)
g.Set("condition2", true)
return called
},
want: want{
called: true,
},
},
"PartialDependenciesMet": {
reason: "Should not call function when only some dependencies are met",
setup: func(g *gate.Gate[string]) chan bool {
called := make(chan bool, 1)
g.Register(func() {
called <- true
}, "condition1", "condition2")
// Set only one condition to true
g.Set("condition1", true)
return called
},
want: want{
called: false,
},
},
"DependenciesAlreadyMet": {
reason: "Should call function when dependencies are already met",
setup: func(g *gate.Gate[string]) chan bool {
called := make(chan bool, 1)
g.Set("condition1", true)
g.Set("condition2", true)
g.Register(func() {
called <- true
}, "condition1", "condition2")
return called
},
want: want{
called: true,
},
},
"DependencySetThenUnset": {
reason: "Should call function when dependency is met, even if unset later",
setup: func(g *gate.Gate[string]) chan bool {
called := make(chan bool, 1)
g.Register(func() {
called <- true
}, "condition1")
// Set condition to true then false (function already called when true)
g.Set("condition1", true)
g.Set("condition1", false)
return called
},
want: want{
called: true,
},
},
"FunctionCalledOnlyOnce": {
reason: "Should call function only once even if conditions change after",
setup: func(g *gate.Gate[string]) chan bool {
called := make(chan bool, 2) // Buffer for potential multiple calls
g.Register(func() {
called <- true
}, "condition1")
// Set condition multiple times
g.Set("condition1", true)
g.Set("condition1", false)
g.Set("condition1", true)
return called
},
want: want{
called: true,
},
},
}
for name, tc := range cases {
t.Run(name, func(t *testing.T) {
g := new(gate.Gate[string])
callChannel := tc.setup(g)
var got bool
select {
case got = <-callChannel:
case <-time.After(100 * time.Millisecond):
got = false
}
if diff := cmp.Diff(tc.want.called, got); diff != "" {
t.Errorf("\n%s\nIntegration test: -want called, +got called:\n%s", tc.reason, diff)
}
// For the "only once" test, ensure no additional calls
if name == "FunctionCalledOnlyOnce" && tc.want.called {
select {
case <-callChannel:
t.Errorf("\n%s\nFunction was called more than once", tc.reason)
case <-time.After(50 * time.Millisecond):
// Good - no additional calls
}
}
})
}
}
func TestGateConcurrency(t *testing.T) {
g := new(gate.Gate[string])
const numGoroutines = 100
var wg sync.WaitGroup
callCount := make(chan struct{}, numGoroutines)
// Register functions concurrently
for range numGoroutines {
wg.Add(1)
go func() {
defer wg.Done()
g.Register(func() {
callCount <- struct{}{}
}, "shared-condition")
}()
}
// Wait for all registrations
wg.Wait()
// Set condition to true once
g.Set("shared-condition", true)
// Give some time for goroutines to execute
time.Sleep(100 * time.Millisecond)
// Count how many functions were called
close(callCount)
count := 0
for range callCount {
count++
}
if count != numGoroutines {
t.Errorf("Expected %d function calls, got %d", numGoroutines, count)
}
}
func TestGateTypeSafety(t *testing.T) {
intGate := new(gate.Gate[int])
called := false
intGate.Register(func() {
called = true
}, 1, 2, 3)
intGate.Set(1, true)
intGate.Set(2, true)
intGate.Set(3, true)
// Give some time for goroutine to execute
time.Sleep(10 * time.Millisecond)
if !called {
t.Error("Function should have been called when all int conditions were met")
}
}

View File

@ -0,0 +1,85 @@
/*
Copyright 2025 The Crossplane Authors.
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 customresourcesgate implements a CustomResourceReconciler to report GKVs status to a Gate.
// This reconciler requires cluster scoped GET,LIST,WATCH on customresourcedefinitions[apiextensions.k8s.io]
package customresourcesgate
import (
"context"
apiextensionsv1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1"
"k8s.io/apimachinery/pkg/runtime/schema"
ctrl "sigs.k8s.io/controller-runtime"
"github.com/crossplane/crossplane-runtime/pkg/controller"
"github.com/crossplane/crossplane-runtime/pkg/logging"
)
// Reconciler reconciles a CustomResourceDefinitions in order to gate and wait
// on CRD readiness to start downstream controllers.
type Reconciler struct {
log logging.Logger
gate controller.Gate
}
// Reconcile reconciles CustomResourceDefinitions and reports ready and unready GVKs to the gate.
func (r *Reconciler) Reconcile(_ context.Context, crd *apiextensionsv1.CustomResourceDefinition) (ctrl.Result, error) {
established := isEstablished(crd)
gkvs := toGVKs(crd)
switch {
// CRD is not ready or being deleted.
case !established || !crd.GetDeletionTimestamp().IsZero():
for gvk := range gkvs {
r.log.Debug("gvk is not ready", "gvk", gvk)
r.gate.Set(gvk, false)
}
return ctrl.Result{}, nil
// CRD is ready.
default:
for gvk, served := range gkvs {
if served {
r.log.Debug("gvk is ready", "gvk", gvk)
r.gate.Set(gvk, true)
}
}
}
return ctrl.Result{}, nil
}
func toGVKs(crd *apiextensionsv1.CustomResourceDefinition) map[schema.GroupVersionKind]bool {
gvks := make(map[schema.GroupVersionKind]bool, len(crd.Spec.Versions))
for _, version := range crd.Spec.Versions {
gvks[schema.GroupVersionKind{Group: crd.Spec.Group, Version: version.Name, Kind: crd.Spec.Names.Kind}] = version.Served
}
return gvks
}
func isEstablished(crd *apiextensionsv1.CustomResourceDefinition) bool {
if len(crd.Status.Conditions) > 0 {
for _, cond := range crd.Status.Conditions {
if cond.Type == apiextensionsv1.Established {
return cond.Status == apiextensionsv1.ConditionTrue
}
}
}
return false
}

View File

@ -0,0 +1,557 @@
/*
Copyright 2025 The Crossplane Authors.
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 customresourcesgate
import (
"context"
"slices"
"strings"
"testing"
"github.com/google/go-cmp/cmp"
apiextensionsv1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime/schema"
ctrl "sigs.k8s.io/controller-runtime"
"github.com/crossplane/crossplane-runtime/pkg/logging"
"github.com/crossplane/crossplane-runtime/pkg/test"
)
func TestToGVKs(t *testing.T) {
type args struct {
crd *apiextensionsv1.CustomResourceDefinition
}
type want struct {
gvks map[schema.GroupVersionKind]bool
}
cases := map[string]struct {
reason string
args args
want want
}{
"SingleVersionServed": {
reason: "Should return single GVK for CRD with one served version",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Spec: apiextensionsv1.CustomResourceDefinitionSpec{
Group: "example.com",
Names: apiextensionsv1.CustomResourceDefinitionNames{
Kind: "TestResource",
},
Versions: []apiextensionsv1.CustomResourceDefinitionVersion{
{Name: "v1", Served: true},
},
},
},
},
want: want{
gvks: map[schema.GroupVersionKind]bool{
{Group: "example.com", Version: "v1", Kind: "TestResource"}: true,
},
},
},
"MultipleVersionsWithServedStatus": {
reason: "Should return GVKs with correct served status for multiple versions",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Spec: apiextensionsv1.CustomResourceDefinitionSpec{
Group: "example.com",
Names: apiextensionsv1.CustomResourceDefinitionNames{
Kind: "TestResource",
},
Versions: []apiextensionsv1.CustomResourceDefinitionVersion{
{Name: "v1alpha1", Served: false},
{Name: "v1beta1", Served: true},
{Name: "v1", Served: true},
},
},
},
},
want: want{
gvks: map[schema.GroupVersionKind]bool{
{Group: "example.com", Version: "v1alpha1", Kind: "TestResource"}: false,
{Group: "example.com", Version: "v1beta1", Kind: "TestResource"}: true,
{Group: "example.com", Version: "v1", Kind: "TestResource"}: true,
},
},
},
"NoVersions": {
reason: "Should return empty map for CRD with no versions",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Spec: apiextensionsv1.CustomResourceDefinitionSpec{
Group: "example.com",
Names: apiextensionsv1.CustomResourceDefinitionNames{
Kind: "TestResource",
},
Versions: []apiextensionsv1.CustomResourceDefinitionVersion{},
},
},
},
want: want{
gvks: map[schema.GroupVersionKind]bool{},
},
},
}
for name, tc := range cases {
t.Run(name, func(t *testing.T) {
got := toGVKs(tc.args.crd)
if diff := cmp.Diff(tc.want.gvks, got); diff != "" {
t.Errorf("\n%s\ntoGVKs(...): -want, +got:\n%s", tc.reason, diff)
}
})
}
}
func TestIsEstablished(t *testing.T) {
type args struct {
crd *apiextensionsv1.CustomResourceDefinition
}
type want struct {
established bool
}
cases := map[string]struct {
reason string
args args
want want
}{
"EstablishedTrue": {
reason: "Should return true when CRD has Established condition with True status",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionTrue,
},
},
},
},
},
want: want{
established: true,
},
},
"EstablishedFalse": {
reason: "Should return false when CRD has Established condition with False status",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionFalse,
},
},
},
},
},
want: want{
established: false,
},
},
"EstablishedUnknown": {
reason: "Should return false when CRD has Established condition with Unknown status",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionUnknown,
},
},
},
},
},
want: want{
established: false,
},
},
"NoEstablishedCondition": {
reason: "Should return false when CRD has no Established condition",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.NamesAccepted,
Status: apiextensionsv1.ConditionTrue,
},
},
},
},
},
want: want{
established: false,
},
},
"NoConditions": {
reason: "Should return false when CRD has no conditions",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{},
},
},
},
want: want{
established: false,
},
},
"MultipleConditions": {
reason: "Should return true when CRD has multiple conditions including Established=True",
args: args{
crd: &apiextensionsv1.CustomResourceDefinition{
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.NamesAccepted,
Status: apiextensionsv1.ConditionTrue,
},
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionTrue,
},
{
Type: apiextensionsv1.Terminating,
Status: apiextensionsv1.ConditionFalse,
},
},
},
},
},
want: want{
established: true,
},
},
}
for name, tc := range cases {
t.Run(name, func(t *testing.T) {
got := isEstablished(tc.args.crd)
if diff := cmp.Diff(tc.want.established, got); diff != "" {
t.Errorf("\n%s\nisEstablished(...): -want, +got:\n%s", tc.reason, diff)
}
})
}
}
// MockGate implements the controller.Gate interface for testing.
type MockGate struct {
TrueCalls []schema.GroupVersionKind
FalseCalls []schema.GroupVersionKind
}
func NewMockGate() *MockGate {
return &MockGate{
TrueCalls: make([]schema.GroupVersionKind, 0),
FalseCalls: make([]schema.GroupVersionKind, 0),
}
}
func (m *MockGate) Set(gvk schema.GroupVersionKind, value bool) bool {
if value {
if m.TrueCalls == nil {
m.TrueCalls = make([]schema.GroupVersionKind, 0)
}
m.TrueCalls = append(m.TrueCalls, gvk)
} else {
if m.FalseCalls == nil {
m.FalseCalls = make([]schema.GroupVersionKind, 0)
}
m.FalseCalls = append(m.FalseCalls, gvk)
}
return true
}
func (m *MockGate) Register(func(), ...schema.GroupVersionKind) {}
func TestReconcile(t *testing.T) {
now := metav1.Now()
type fields struct {
gate *MockGate
}
type args struct {
ctx context.Context
crd *apiextensionsv1.CustomResourceDefinition
}
type want struct {
result ctrl.Result
err error
trueCalls []schema.GroupVersionKind
falseCalls []schema.GroupVersionKind
}
cases := map[string]struct {
reason string
fields fields
args args
want want
}{
"EstablishedCRDCallsGateTrue": {
reason: "Should call gate.True for all GVKs when CRD is established",
fields: fields{
gate: NewMockGate(),
},
args: args{
ctx: context.Background(),
crd: &apiextensionsv1.CustomResourceDefinition{
ObjectMeta: metav1.ObjectMeta{
Name: "testresources.example.com",
},
Spec: apiextensionsv1.CustomResourceDefinitionSpec{
Group: "example.com",
Names: apiextensionsv1.CustomResourceDefinitionNames{
Kind: "TestResource",
},
Versions: []apiextensionsv1.CustomResourceDefinitionVersion{
{Name: "v1alpha1", Served: true},
{Name: "v1", Served: true},
},
},
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionTrue,
},
},
},
},
},
want: want{
result: ctrl.Result{},
err: nil,
trueCalls: []schema.GroupVersionKind{
{Group: "example.com", Version: "v1alpha1", Kind: "TestResource"},
{Group: "example.com", Version: "v1", Kind: "TestResource"},
},
falseCalls: []schema.GroupVersionKind{},
},
},
"NotEstablishedCRDCallsGateFalse": {
reason: "Should call gate.False for all GVKs when CRD is not established",
fields: fields{
gate: NewMockGate(),
},
args: args{
ctx: context.Background(),
crd: &apiextensionsv1.CustomResourceDefinition{
ObjectMeta: metav1.ObjectMeta{
Name: "testresources.example.com",
},
Spec: apiextensionsv1.CustomResourceDefinitionSpec{
Group: "example.com",
Names: apiextensionsv1.CustomResourceDefinitionNames{
Kind: "TestResource",
},
Versions: []apiextensionsv1.CustomResourceDefinitionVersion{
{Name: "v1", Served: true},
},
},
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionFalse,
},
},
},
},
},
want: want{
result: ctrl.Result{},
err: nil,
trueCalls: []schema.GroupVersionKind{},
falseCalls: []schema.GroupVersionKind{
{Group: "example.com", Version: "v1", Kind: "TestResource"},
},
},
},
"DeletingCRDCallsGateFalse": {
reason: "Should call gate.False for all GVKs when CRD is being deleted",
fields: fields{
gate: NewMockGate(),
},
args: args{
ctx: context.Background(),
crd: &apiextensionsv1.CustomResourceDefinition{
ObjectMeta: metav1.ObjectMeta{
Name: "testresources.example.com",
DeletionTimestamp: &now,
},
Spec: apiextensionsv1.CustomResourceDefinitionSpec{
Group: "example.com",
Names: apiextensionsv1.CustomResourceDefinitionNames{
Kind: "TestResource",
},
Versions: []apiextensionsv1.CustomResourceDefinitionVersion{
{Name: "v1", Served: true},
},
},
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionTrue,
},
},
},
},
},
want: want{
result: ctrl.Result{},
err: nil,
trueCalls: []schema.GroupVersionKind{},
falseCalls: []schema.GroupVersionKind{
{Group: "example.com", Version: "v1", Kind: "TestResource"},
},
},
},
"MixedServedVersions": {
reason: "Should only call gate.True for served versions",
fields: fields{
gate: NewMockGate(),
},
args: args{
ctx: context.Background(),
crd: &apiextensionsv1.CustomResourceDefinition{
ObjectMeta: metav1.ObjectMeta{
Name: "testresources.example.com",
},
Spec: apiextensionsv1.CustomResourceDefinitionSpec{
Group: "example.com",
Names: apiextensionsv1.CustomResourceDefinitionNames{
Kind: "TestResource",
},
Versions: []apiextensionsv1.CustomResourceDefinitionVersion{
{Name: "v1alpha1", Served: false},
{Name: "v1", Served: true},
},
},
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionTrue,
},
},
},
},
},
want: want{
result: ctrl.Result{},
err: nil,
trueCalls: []schema.GroupVersionKind{
{Group: "example.com", Version: "v1", Kind: "TestResource"},
},
falseCalls: []schema.GroupVersionKind{},
},
},
"NoVersionsCRD": {
reason: "Should handle CRD with no versions gracefully",
fields: fields{
gate: NewMockGate(),
},
args: args{
ctx: context.Background(),
crd: &apiextensionsv1.CustomResourceDefinition{
ObjectMeta: metav1.ObjectMeta{
Name: "testresources.example.com",
},
Spec: apiextensionsv1.CustomResourceDefinitionSpec{
Group: "example.com",
Names: apiextensionsv1.CustomResourceDefinitionNames{
Kind: "TestResource",
},
Versions: []apiextensionsv1.CustomResourceDefinitionVersion{},
},
Status: apiextensionsv1.CustomResourceDefinitionStatus{
Conditions: []apiextensionsv1.CustomResourceDefinitionCondition{
{
Type: apiextensionsv1.Established,
Status: apiextensionsv1.ConditionTrue,
},
},
},
},
},
want: want{
result: ctrl.Result{},
err: nil,
trueCalls: []schema.GroupVersionKind{},
falseCalls: []schema.GroupVersionKind{},
},
},
}
for name, tc := range cases {
t.Run(name, func(t *testing.T) {
r := &Reconciler{
log: logging.NewNopLogger(),
gate: tc.fields.gate,
}
got, err := r.Reconcile(tc.args.ctx, tc.args.crd)
if diff := cmp.Diff(tc.want.err, err, test.EquateErrors()); diff != "" {
t.Errorf("\n%s\nr.Reconcile(...): -want error, +got error:\n%s", tc.reason, diff)
}
if diff := cmp.Diff(tc.want.result, got); diff != "" {
t.Errorf("\n%s\nr.Reconcile(...): -want result, +got result:\n%s", tc.reason, diff)
}
// Only check gate calls if gate is not nil
if tc.fields.gate != nil {
slices.SortFunc(tc.want.trueCalls, func(a, b schema.GroupVersionKind) int {
return strings.Compare(a.Kind, b.Kind)
})
slices.SortFunc(tc.fields.gate.TrueCalls, func(a, b schema.GroupVersionKind) int {
return strings.Compare(a.Kind, b.Kind)
})
if diff := cmp.Diff(tc.want.trueCalls, tc.fields.gate.TrueCalls); diff != "" {
t.Errorf("\n%s\ngate.True calls: -want, +got:\n%s", tc.reason, diff)
}
slices.SortFunc(tc.want.falseCalls, func(a, b schema.GroupVersionKind) int {
return strings.Compare(a.Kind, b.Kind)
})
slices.SortFunc(tc.fields.gate.FalseCalls, func(a, b schema.GroupVersionKind) int {
return strings.Compare(a.Kind, b.Kind)
})
if diff := cmp.Diff(tc.want.falseCalls, tc.fields.gate.FalseCalls); diff != "" {
t.Errorf("\n%s\ngate.False calls: -want, +got:\n%s", tc.reason, diff)
}
}
})
}
}

View File

@ -0,0 +1,46 @@
/*
Copyright 2025 The Crossplane Authors.
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 customresourcesgate
import (
"errors"
"reflect"
apiextensionsv1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1"
ctrl "sigs.k8s.io/controller-runtime"
"sigs.k8s.io/controller-runtime/pkg/reconcile"
"github.com/crossplane/crossplane-runtime/pkg/controller"
)
// Setup adds a controller that reconciles CustomResourceDefinitions to support delayed start of controllers.
// o.Gate is expected to be something like *gate.Gate[schema.GroupVersionKind].
func Setup(mgr ctrl.Manager, o controller.Options) error {
if o.Gate == nil || reflect.ValueOf(o.Gate).IsNil() {
return errors.New("gate is required")
}
r := &Reconciler{
log: o.Logger,
gate: o.Gate,
}
return ctrl.NewControllerManagedBy(mgr).
For(&apiextensionsv1.CustomResourceDefinition{}).
Named("crd-gate").
Complete(reconcile.AsReconciler[*apiextensionsv1.CustomResourceDefinition](mgr.GetClient(), r))
}