1
0
mirror of https://github.com/rancher/steve.git synced 2025-09-22 11:58:18 +00:00

Stop single caches instead of all of them (#812)

* Revert OnSchemas change work

* Track schema changes

* Only stop a single GVK informer factory

* Add tests

* Rename crd to crdClient

* Rename s to sqlStore

* Don't wait for synced caches if request is canceled

* Move schematracker to pkg/sqlcache/schematracker
This commit is contained in:
Tom Lebreux
2025-09-10 17:04:25 -04:00
committed by GitHub
parent 45a3b07816
commit 13d5ad3ccb
10 changed files with 543 additions and 246 deletions

View File

@@ -28,14 +28,12 @@ const EncryptAllEnvVar = "CATTLE_ENCRYPT_CACHE_ALL"
// CacheFactory builds Informer instances and keeps a cache of instances it created
type CacheFactory struct {
wg wait.Group
dbClient db.Client
// ctx determines when informers need to stop
ctx context.Context
cancel context.CancelFunc
mutex sync.RWMutex
encryptAll bool
gcInterval time.Duration
@@ -49,13 +47,31 @@ type CacheFactory struct {
type guardedInformer struct {
informer *informer.Informer
mutex *sync.Mutex
// informerMutex ensures informer is only set by one goroutine even if
// multiple concurrent calls to CacheFor are made
informerMutex *sync.Mutex
// stopMutex ensures no CacheFor call can be made for a given GVK when
// a Stop call is ongoing.
//
// CacheFactory.informersMutex is not enough because part of the code
// might still have an old cache from a previous CacheFor call.
stopMutex *sync.RWMutex
ctx context.Context
cancel context.CancelFunc
wg wait.Group
}
type newInformer func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespace bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error)
type Cache struct {
informer.ByOptionsLister
gvk schema.GroupVersionKind
}
func (c *Cache) GVK() schema.GroupVersionKind {
return c.gvk
}
var defaultEncryptedResourceTypes = map[schema.GroupVersionKind]struct{}{
@@ -90,8 +106,6 @@ func NewCacheFactory(opts CacheFactoryOptions) (*CacheFactory, error) {
}
ctx, cancel := context.WithCancel(context.Background())
return &CacheFactory{
wg: wait.Group{},
ctx: ctx,
cancel: cancel,
@@ -111,17 +125,6 @@ func NewCacheFactory(opts CacheFactoryOptions) (*CacheFactory, error) {
//
// Don't forget to call DoneWithCache with the given informer once done with it.
func (f *CacheFactory) CacheFor(ctx context.Context, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, client dynamic.ResourceInterface, gvk schema.GroupVersionKind, namespaced bool, watchable bool) (*Cache, error) {
// First of all block Reset() until we are done
f.mutex.RLock()
cache, err := f.cacheForLocked(ctx, fields, externalUpdateInfo, selfUpdateInfo, transform, client, gvk, namespaced, watchable)
if err != nil {
f.mutex.RUnlock()
return nil, err
}
return cache, nil
}
func (f *CacheFactory) cacheForLocked(ctx context.Context, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, client dynamic.ResourceInterface, gvk schema.GroupVersionKind, namespaced bool, watchable bool) (*Cache, error) {
// Second, check if the informer and its accompanying informer-specific mutex exist already in the informers cache
// If not, start by creating such informer-specific mutex. That is used later to ensure no two goroutines create
// informers for the same GVK at the same type
@@ -130,17 +133,32 @@ func (f *CacheFactory) cacheForLocked(ctx context.Context, fields [][]string, ex
// that blocks CacheFor for other GVKs, hence not deferring unlock here
gi, ok := f.informers[gvk]
if !ok {
giCtx, giCancel := context.WithCancel(f.ctx)
gi = &guardedInformer{
informer: nil,
mutex: &sync.Mutex{},
informer: nil,
informerMutex: &sync.Mutex{},
stopMutex: &sync.RWMutex{},
ctx: giCtx,
cancel: giCancel,
}
f.informers[gvk] = gi
}
f.informersMutex.Unlock()
// Prevent Stop() to be called for that GVK
gi.stopMutex.RLock()
gvkCache, err := f.cacheForLocked(ctx, gi, fields, externalUpdateInfo, selfUpdateInfo, transform, client, gvk, namespaced, watchable)
if err != nil {
gi.stopMutex.RUnlock()
return nil, err
}
return gvkCache, nil
}
func (f *CacheFactory) cacheForLocked(ctx context.Context, gi *guardedInformer, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, client dynamic.ResourceInterface, gvk schema.GroupVersionKind, namespaced bool, watchable bool) (*Cache, error) {
// At this point an informer-specific mutex (gi.mutex) is guaranteed to exist. Lock it
gi.mutex.Lock()
defer gi.mutex.Unlock()
gi.informerMutex.Lock()
// Then: if the informer really was not created yet (first time here or previous times have errored out)
// actually create the informer
@@ -155,8 +173,9 @@ func (f *CacheFactory) cacheForLocked(ctx context.Context, fields [][]string, ex
shouldEncrypt := f.encryptAll || encryptResourceAlways
// In non-test code this invokes pkg/sqlcache/informer/informer.go: NewInformer()
// search for "func NewInformer(ctx"
i, err := f.newInformer(f.ctx, client, fields, externalUpdateInfo, selfUpdateInfo, transform, gvk, f.dbClient, shouldEncrypt, namespaced, watchable, f.gcInterval, f.gcKeepCount)
i, err := f.newInformer(gi.ctx, client, fields, externalUpdateInfo, selfUpdateInfo, transform, gvk, f.dbClient, shouldEncrypt, namespaced, watchable, f.gcInterval, f.gcKeepCount)
if err != nil {
gi.informerMutex.Unlock()
return nil, err
}
@@ -168,69 +187,92 @@ func (f *CacheFactory) cacheForLocked(ctx context.Context, fields [][]string, ex
cache.DefaultWatchErrorHandler(ctx, r, err)
})
if err != nil {
gi.informerMutex.Unlock()
return nil, err
}
f.wg.StartWithChannel(f.ctx.Done(), i.Run)
gi.wg.StartWithChannel(gi.ctx.Done(), i.Run)
gi.informer = i
}
gi.informerMutex.Unlock()
if !cache.WaitForCacheSync(f.ctx.Done(), gi.informer.HasSynced) {
// We don't want to get stuck in WaitForCachesSync if the request from
// the client has been canceled.
waitCh := make(chan struct{}, 1)
go func() {
select {
case <-ctx.Done():
close(waitCh)
case <-gi.ctx.Done():
close(waitCh)
}
}()
if !cache.WaitForCacheSync(waitCh, gi.informer.HasSynced) {
return nil, fmt.Errorf("failed to sync SQLite Informer cache for GVK %v", gvk)
}
// At this point the informer is ready, return it
return &Cache{ByOptionsLister: gi.informer}, nil
return &Cache{ByOptionsLister: gi.informer, gvk: gvk}, nil
}
// DoneWithCache must be called for every CacheFor call.
// DoneWithCache must be called for every successful CacheFor call. The Cache should
// no longer be used after DoneWithCache is called.
//
// This ensures that there aren't any inflight list requests while we are resetting the database.
//
// TODO: Use the *Cache once we go per-GVK
func (f *CacheFactory) DoneWithCache(_ *Cache) {
f.mutex.RUnlock()
func (f *CacheFactory) DoneWithCache(cache *Cache) {
if cache == nil {
return
}
f.informersMutex.Lock()
defer f.informersMutex.Unlock()
// Note: the informers cache is protected by informersMutex, which we don't want to hold for very long because
// that blocks CacheFor for other GVKs, hence not deferring unlock here
gi, ok := f.informers[cache.gvk]
if !ok {
return
}
gi.stopMutex.RUnlock()
}
// Stop cancels ctx which stops any running informers, assigns a new ctx, resets the GVK-informer cache, and resets
// the database connection which wipes any current sqlite database at the default location.
func (f *CacheFactory) Stop() error {
func (f *CacheFactory) Stop(gvk schema.GroupVersionKind) error {
if f.dbClient == nil {
// nothing to reset
return nil
}
// We must stop informers here to unblock those stuck in WaitForCacheSync
// which is blocking DoneWithCache call.
//
// This is fine without a lock as long as multiple Stop() call aren't made
// concurrently (which they currently aren't)
f.cancel()
// Prevent more CacheFor calls
f.mutex.Lock()
defer f.mutex.Unlock()
// Wait for all informers to have exited
f.wg.Wait()
f.ctx, f.cancel = context.WithCancel(context.Background())
// and get rid of all references to those informers and their mutexes
f.informersMutex.Lock()
defer f.informersMutex.Unlock()
for gvk, informer := range f.informers {
// DropAll needs its own context because the context from the CacheFactory
// is canceled
err := informer.informer.DropAll(context.Background())
if err != nil {
return fmt.Errorf("dropall %q: %w", gvk, err)
}
gi, ok := f.informers[gvk]
if !ok {
return nil
}
delete(f.informers, gvk)
f.informers = make(map[schema.GroupVersionKind]*guardedInformer)
// We must stop informers here to unblock those stuck in WaitForCacheSync
// which is blocking DoneWithCache call.
gi.cancel()
// Prevent other CacheFor calls for that GVK
gi.stopMutex.Lock()
defer gi.stopMutex.Unlock()
// Wait for all informers to have exited
gi.wg.Wait()
// DropAll needs its own context because the context from the informer
// is canceled
err := gi.informer.DropAll(context.Background())
if err != nil {
return fmt.Errorf("dropall %q: %w", gvk, err)
}
return nil
}

View File

@@ -79,6 +79,7 @@ func TestCacheFor(t *testing.T) {
}
expectedC := &Cache{
ByOptionsLister: i,
gvk: expectedGVK,
}
testNewInformer := func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespaced bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error) {
assert.Equal(t, client, dynamicClient)
@@ -100,7 +101,7 @@ func TestCacheFor(t *testing.T) {
go func() {
// this function ensures that ctx is open for the duration of this test but if part of a longer process it will be closed eventually
time.Sleep(5 * time.Second)
f.Stop()
f.Stop(expectedGVK)
}()
c, err := f.CacheFor(context.Background(), fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)
assert.Nil(t, err)
@@ -148,13 +149,63 @@ func TestCacheFor(t *testing.T) {
go func() {
time.Sleep(1 * time.Second)
f.Stop()
f.Stop(expectedGVK)
}()
var err error
_, err = f.CacheFor(context.Background(), fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)
_, err := f.CacheFor(context.Background(), fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)
assert.NotNil(t, err)
time.Sleep(2 * time.Second)
}})
tests = append(tests, testCase{description: "CacheFor() with no errors returned, HasSync returning false, request is canceled", test: func(t *testing.T) {
dbClient := NewMockClient(gomock.NewController(t))
dynamicClient := NewMockResourceInterface(gomock.NewController(t))
fields := [][]string{{"something"}}
expectedGVK := schema.GroupVersionKind{}
bloi := NewMockByOptionsLister(gomock.NewController(t))
bloi.EXPECT().RunGC(gomock.Any()).AnyTimes()
bloi.EXPECT().DropAll(gomock.Any()).AnyTimes()
sii := NewMockSharedIndexInformer(gomock.NewController(t))
sii.EXPECT().HasSynced().Return(false).AnyTimes()
sii.EXPECT().Run(gomock.Any())
sii.EXPECT().SetWatchErrorHandler(gomock.Any())
expectedI := &informer.Informer{
// need to set this so Run function is not nil
SharedIndexInformer: sii,
ByOptionsLister: bloi,
}
testNewInformer := func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespaced bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error) {
assert.Equal(t, client, dynamicClient)
assert.Equal(t, fields, fields)
assert.Equal(t, expectedGVK, gvk)
assert.Equal(t, db, dbClient)
assert.Equal(t, false, shouldEncrypt)
assert.Equal(t, 0, gcKeepCount)
assert.Nil(t, externalUpdateInfo)
return expectedI, nil
}
f := &CacheFactory{
dbClient: dbClient,
newInformer: testNewInformer,
informers: map[schema.GroupVersionKind]*guardedInformer{},
}
f.ctx, f.cancel = context.WithCancel(context.Background())
errCh := make(chan error, 1)
go func() {
ctx, cancel := context.WithTimeout(context.Background(), time.Duration(1)*time.Second)
defer cancel()
_, err := f.CacheFor(ctx, fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)
errCh <- err
}()
select {
case err := <-errCh:
assert.NotNil(t, err)
case <-time.After(2 * time.Second):
assert.Fail(t, "CacheFor never exited")
}
time.Sleep(2 * time.Second)
}})
tests = append(tests, testCase{description: "CacheFor() with no errors returned, HasSync returning true, and ctx is canceled, should not call Run() more than once and not return an error", test: func(t *testing.T) {
dbClient := NewMockClient(gomock.NewController(t))
dynamicClient := NewMockResourceInterface(gomock.NewController(t))
@@ -176,6 +227,7 @@ func TestCacheFor(t *testing.T) {
}
expectedC := &Cache{
ByOptionsLister: i,
gvk: expectedGVK,
}
testNewInformer := func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespaced bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error) {
assert.Equal(t, client, dynamicClient)
@@ -193,7 +245,7 @@ func TestCacheFor(t *testing.T) {
informers: map[schema.GroupVersionKind]*guardedInformer{},
}
f.ctx, f.cancel = context.WithCancel(context.Background())
f.Stop()
f.Stop(expectedGVK)
c, err := f.CacheFor(context.Background(), fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)
assert.Nil(t, err)
@@ -219,6 +271,7 @@ func TestCacheFor(t *testing.T) {
}
expectedC := &Cache{
ByOptionsLister: i,
gvk: expectedGVK,
}
testNewInformer := func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespaced bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error) {
assert.Equal(t, client, dynamicClient)
@@ -240,7 +293,7 @@ func TestCacheFor(t *testing.T) {
go func() {
time.Sleep(10 * time.Second)
f.Stop()
f.Stop(expectedGVK)
}()
c, err := f.CacheFor(context.Background(), fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)
assert.Nil(t, err)
@@ -271,6 +324,7 @@ func TestCacheFor(t *testing.T) {
}
expectedC := &Cache{
ByOptionsLister: i,
gvk: expectedGVK,
}
testNewInformer := func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespaced bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error) {
assert.Equal(t, client, dynamicClient)
@@ -292,7 +346,7 @@ func TestCacheFor(t *testing.T) {
go func() {
time.Sleep(10 * time.Second)
f.Stop()
f.Stop(expectedGVK)
}()
c, err := f.CacheFor(context.Background(), fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)
assert.Nil(t, err)
@@ -322,6 +376,7 @@ func TestCacheFor(t *testing.T) {
}
expectedC := &Cache{
ByOptionsLister: i,
gvk: expectedGVK,
}
testNewInformer := func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespaced bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error) {
assert.Equal(t, client, dynamicClient)
@@ -343,7 +398,7 @@ func TestCacheFor(t *testing.T) {
go func() {
time.Sleep(10 * time.Second)
f.Stop()
f.Stop(expectedGVK)
}()
c, err := f.CacheFor(context.Background(), fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)
assert.Nil(t, err)
@@ -373,6 +428,7 @@ func TestCacheFor(t *testing.T) {
}
expectedC := &Cache{
ByOptionsLister: i,
gvk: expectedGVK,
}
testNewInformer := func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespaced bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error) {
// we can't test func == func, so instead we check if the output was as expected
@@ -402,7 +458,7 @@ func TestCacheFor(t *testing.T) {
go func() {
// this function ensures that ctx is not canceled for the duration of this test but if part of a longer process it will be closed eventually
time.Sleep(5 * time.Second)
f.Stop()
f.Stop(expectedGVK)
}()
var c *Cache
var err error
@@ -430,6 +486,7 @@ func TestCacheFor(t *testing.T) {
}
expectedC := &Cache{
ByOptionsLister: i,
gvk: expectedGVK,
}
testNewInformer := func(ctx context.Context, client dynamic.ResourceInterface, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, selfUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, gvk schema.GroupVersionKind, db db.Client, shouldEncrypt bool, namespaced bool, watchable bool, gcInterval time.Duration, gcKeepCount int) (*informer.Informer, error) {
assert.Equal(t, client, dynamicClient)
@@ -454,7 +511,7 @@ func TestCacheFor(t *testing.T) {
go func() {
time.Sleep(10 * time.Second)
f.Stop()
f.Stop(expectedGVK)
}()
// CacheFor(ctx context.Context, fields [][]string, externalUpdateInfo *sqltypes.ExternalGVKUpdates, transform cache.TransformFunc, client dynamic.ResourceInterface, gvk schema.GroupVersionKind, namespaced bool, watchable bool)
c, err := f.CacheFor(context.Background(), fields, nil, nil, nil, dynamicClient, expectedGVK, false, true)

View File

@@ -97,7 +97,7 @@ func (i *IntegrationSuite) TestSQLCacheFilters() {
require.NoError(err)
defer func() {
cacheFactory.DoneWithCache(cache)
cacheFactory.Stop()
cacheFactory.Stop(cache.GVK())
}()
// doesn't match the filter for somekey == somevalue

View File

@@ -0,0 +1,78 @@
package schematracker
import (
"errors"
"slices"
"github.com/rancher/steve/pkg/attributes"
"github.com/rancher/steve/pkg/resources/common"
"github.com/rancher/steve/pkg/schema"
k8sschema "k8s.io/apimachinery/pkg/runtime/schema"
)
type Resetter interface {
Reset(k8sschema.GroupVersionKind) error
}
type SchemaTracker struct {
knownSchemas map[k8sschema.GroupVersionKind][]common.ColumnDefinition
resetter Resetter
}
func NewSchemaTracker(resetter Resetter) *SchemaTracker {
return &SchemaTracker{
knownSchemas: make(map[k8sschema.GroupVersionKind][]common.ColumnDefinition),
resetter: resetter,
}
}
func (s *SchemaTracker) OnSchemas(schemas *schema.Collection) error {
knownSchemas := make(map[k8sschema.GroupVersionKind][]common.ColumnDefinition)
needsReset := make(map[k8sschema.GroupVersionKind]struct{})
deletedSchemas := make(map[k8sschema.GroupVersionKind]struct{})
for gvk := range s.knownSchemas {
deletedSchemas[gvk] = struct{}{}
}
for _, id := range schemas.IDs() {
theSchema := schemas.Schema(id)
if theSchema == nil {
continue
}
gvk := attributes.GVK(theSchema)
cols := common.GetColumnDefinitions(theSchema)
knownSchemas[gvk] = cols
oldCols, exists := s.knownSchemas[gvk]
if exists {
if !slices.Equal(cols, oldCols) {
needsReset[gvk] = struct{}{}
}
} else {
needsReset[gvk] = struct{}{}
}
// Schema is still there so it hasn't been deleted
delete(deletedSchemas, gvk)
}
// All deleted schemas must be resetted as well
for gvk := range deletedSchemas {
needsReset[gvk] = struct{}{}
}
// Reset known schemas
var retErr error
for gvk := range needsReset {
err := s.resetter.Reset(gvk)
retErr = errors.Join(retErr, err)
}
s.knownSchemas = knownSchemas
return retErr
}

View File

@@ -0,0 +1,231 @@
package schematracker
import (
"context"
"testing"
"github.com/rancher/apiserver/pkg/types"
"github.com/rancher/steve/pkg/attributes"
"github.com/rancher/steve/pkg/resources/common"
"github.com/rancher/steve/pkg/schema"
"github.com/rancher/wrangler/v3/pkg/schemas"
"github.com/stretchr/testify/assert"
k8sschema "k8s.io/apimachinery/pkg/runtime/schema"
)
type testResetter struct {
Resets map[k8sschema.GroupVersionKind]struct{}
}
func (r *testResetter) Reset(gvk k8sschema.GroupVersionKind) error {
if r.Resets == nil {
r.Resets = make(map[k8sschema.GroupVersionKind]struct{})
}
r.Resets[gvk] = struct{}{}
return nil
}
func TestSchemaTracker(t *testing.T) {
pods := &types.APISchema{
Schema: &schemas.Schema{ID: "pods"},
}
attributes.SetGVK(pods, k8sschema.GroupVersionKind{
Version: "v1",
Kind: "Pod",
})
attributes.SetGVR(pods, k8sschema.GroupVersionResource{
Version: "v1",
Resource: "pods",
})
configmaps := &types.APISchema{
Schema: &schemas.Schema{ID: "configmaps"},
}
attributes.SetGVK(configmaps, k8sschema.GroupVersionKind{
Version: "v1",
Kind: "ConfigMap",
})
attributes.SetGVR(configmaps, k8sschema.GroupVersionResource{
Version: "v1",
Resource: "configmaps",
})
foo := &types.APISchema{
Schema: &schemas.Schema{ID: "test.io.foos"},
}
attributes.SetGVK(foo, k8sschema.GroupVersionKind{
Group: "test.io",
Version: "v1",
Kind: "Foo",
})
attributes.SetGVR(foo, k8sschema.GroupVersionResource{
Group: "test.io",
Version: "v1",
Resource: "foos",
})
foos1 := &types.APISchema{
Schema: &schemas.Schema{ID: "test.io.foos"},
}
attributes.SetGVK(foos1, k8sschema.GroupVersionKind{
Group: "test.io",
Version: "v1",
Kind: "Foo",
})
attributes.SetGVR(foos1, k8sschema.GroupVersionResource{
Group: "test.io",
Version: "v1",
Resource: "foos",
})
attributes.SetColumns(foos1, []common.ColumnDefinition{
{Field: "field1"}, {Field: "field2"},
})
foos2 := &types.APISchema{
Schema: &schemas.Schema{ID: "test.io.foos"},
}
attributes.SetGVK(foos2, k8sschema.GroupVersionKind{
Group: "test.io",
Version: "v1",
Kind: "Foo",
})
attributes.SetGVR(foos2, k8sschema.GroupVersionResource{
Group: "test.io",
Version: "v1",
Resource: "foos",
})
attributes.SetColumns(foos2, []common.ColumnDefinition{
{Field: "field1"}, {Field: "field2"}, {Field: "field3"},
})
bars := &types.APISchema{
Schema: &schemas.Schema{ID: "test.io.bars"},
}
attributes.SetGVK(bars, k8sschema.GroupVersionKind{
Group: "test.io",
Version: "v1",
Kind: "Bar",
})
attributes.SetGVR(bars, k8sschema.GroupVersionResource{
Group: "test.io",
Version: "v1",
Resource: "bars",
})
tests := []struct {
name string
initialSchemas map[string]*types.APISchema
refreshedSchemas map[string]*types.APISchema
expectedResets map[k8sschema.GroupVersionKind]struct{}
}{
{
name: "no change",
initialSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
},
refreshedSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
},
},
{
name: "single schema added",
initialSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
},
refreshedSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
"pods": pods,
},
expectedResets: map[k8sschema.GroupVersionKind]struct{}{
attributes.GVK(pods): {},
},
},
{
name: "multiple schemas added",
initialSchemas: map[string]*types.APISchema{},
refreshedSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
"pods": pods,
},
expectedResets: map[k8sschema.GroupVersionKind]struct{}{
attributes.GVK(configmaps): {},
attributes.GVK(pods): {},
},
},
{
name: "single schema removed",
initialSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
"pods": pods,
},
refreshedSchemas: map[string]*types.APISchema{
"pods": pods,
},
expectedResets: map[k8sschema.GroupVersionKind]struct{}{
attributes.GVK(configmaps): {},
},
},
{
name: "multiple schemas removed",
initialSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
"pods": pods,
},
refreshedSchemas: map[string]*types.APISchema{},
expectedResets: map[k8sschema.GroupVersionKind]struct{}{
attributes.GVK(configmaps): {},
attributes.GVK(pods): {},
},
},
{
name: "field changed",
initialSchemas: map[string]*types.APISchema{
"test.io.foos": foos1,
},
refreshedSchemas: map[string]*types.APISchema{
"test.io.foos": foos2,
},
expectedResets: map[k8sschema.GroupVersionKind]struct{}{
attributes.GVK(foos2): {},
},
},
{
name: "added deleted and changed",
initialSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
"pods": pods,
"test.io.foos": foos1,
},
refreshedSchemas: map[string]*types.APISchema{
"configmaps": configmaps,
"test.io.bars": bars,
"test.io.foos": foos2,
},
expectedResets: map[k8sschema.GroupVersionKind]struct{}{
attributes.GVK(foos2): {},
attributes.GVK(pods): {},
attributes.GVK(bars): {},
},
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
resetter := &testResetter{}
tracker := NewSchemaTracker(resetter)
collection := schema.NewCollection(context.TODO(), types.EmptyAPISchemas(), nil)
collection.Reset(test.initialSchemas)
err := tracker.OnSchemas(collection)
assert.NoError(t, err)
// Reset because we don't care about the initial list of resets
resetter.Resets = nil
collection.Reset(test.refreshedSchemas)
err = tracker.OnSchemas(collection)
assert.NoError(t, err)
assert.Equal(t, test.expectedResets, resetter.Resets)
})
}
}