1
0
mirror of https://github.com/rancher/steve.git synced 2025-09-04 00:44:55 +00:00

Add basic watch functionality for SQL cache (#653)

* Remove unused method

* Add basic watch functionality

* Remove TestWatchNamesErrReceive test
This commit is contained in:
Tom Lebreux
2025-06-03 16:07:18 -06:00
committed by GitHub
parent 2672969496
commit e3f207ddc2
7 changed files with 292 additions and 133 deletions

View File

@@ -29,8 +29,12 @@ type Informer struct {
ByOptionsLister
}
type WatchOptions struct {
}
type ByOptionsLister interface {
ListByOptions(ctx context.Context, lo *sqltypes.ListOptions, partitions []partition.Partition, namespace string) (*unstructured.UnstructuredList, int, string, error)
Watch(ctx context.Context, opts WatchOptions, eventsCh chan<- watch.Event) error
}
// this is set to a var so that it can be overridden by test code for mocking purposes

View File

@@ -17,6 +17,7 @@ import (
sqltypes "github.com/rancher/steve/pkg/sqlcache/sqltypes"
gomock "go.uber.org/mock/gomock"
unstructured "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
watch "k8s.io/apimachinery/pkg/watch"
)
// MockByOptionsLister is a mock of ByOptionsLister interface.
@@ -58,3 +59,17 @@ func (mr *MockByOptionsListerMockRecorder) ListByOptions(arg0, arg1, arg2, arg3
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListByOptions", reflect.TypeOf((*MockByOptionsLister)(nil).ListByOptions), arg0, arg1, arg2, arg3)
}
// Watch mocks base method.
func (m *MockByOptionsLister) Watch(arg0 context.Context, arg1 WatchOptions, arg2 chan<- watch.Event) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Watch", arg0, arg1, arg2)
ret0, _ := ret[0].(error)
return ret0
}
// Watch indicates an expected call of Watch.
func (mr *MockByOptionsListerMockRecorder) Watch(arg0, arg1, arg2 any) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Watch", reflect.TypeOf((*MockByOptionsLister)(nil).Watch), arg0, arg1, arg2)
}

View File

@@ -10,11 +10,14 @@ import (
"sort"
"strconv"
"strings"
"sync"
"github.com/rancher/steve/pkg/sqlcache/db/transaction"
"github.com/rancher/steve/pkg/sqlcache/sqltypes"
"github.com/sirupsen/logrus"
"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/watch"
"k8s.io/client-go/tools/cache"
"github.com/rancher/steve/pkg/sqlcache/db"
@@ -28,6 +31,9 @@ type ListOptionIndexer struct {
namespaced bool
indexedFields []string
watchersLock sync.RWMutex
watchers map[*watchKey]chan<- watch.Event
addFieldsQuery string
deleteFieldsByKeyQuery string
deleteFieldsQuery string
@@ -105,13 +111,17 @@ func NewListOptionIndexer(ctx context.Context, fields [][]string, s Store, names
Indexer: i,
namespaced: namespaced,
indexedFields: indexedFields,
watchers: make(map[*watchKey]chan<- watch.Event),
}
l.RegisterAfterAdd(l.addIndexFields)
l.RegisterAfterAdd(l.notifyEventAdded)
l.RegisterAfterUpdate(l.addIndexFields)
l.RegisterAfterUpdate(l.notifyEventModified)
l.RegisterAfterAdd(l.addLabels)
l.RegisterAfterUpdate(l.addLabels)
l.RegisterAfterDelete(l.deleteFieldsByKey)
l.RegisterAfterDelete(l.deleteLabelsByKey)
l.RegisterAfterDelete(l.notifyEventDeleted)
l.RegisterAfterDeleteAll(l.deleteFields)
l.RegisterAfterDeleteAll(l.deleteLabels)
columnDefs := make([]string, len(indexedFields))
@@ -191,8 +201,57 @@ func NewListOptionIndexer(ctx context.Context, fields [][]string, s Store, names
return l, nil
}
func (l *ListOptionIndexer) Watch(ctx context.Context, opts WatchOptions, eventsCh chan<- watch.Event) error {
key := l.addWatcher(eventsCh)
<-ctx.Done()
l.removeWatcher(key)
return nil
}
type watchKey struct {
_ bool // ensure watchKey is NOT zero-sized to get unique pointers
}
func (l *ListOptionIndexer) addWatcher(eventCh chan<- watch.Event) *watchKey {
key := new(watchKey)
l.watchersLock.Lock()
l.watchers[key] = eventCh
l.watchersLock.Unlock()
return key
}
func (l *ListOptionIndexer) removeWatcher(key *watchKey) {
l.watchersLock.Lock()
delete(l.watchers, key)
l.watchersLock.Unlock()
}
/* Core methods */
func (l *ListOptionIndexer) notifyEventAdded(key string, obj any, tx transaction.Client) error {
return l.notifyEvent(watch.Added, obj, tx)
}
func (l *ListOptionIndexer) notifyEventModified(key string, obj any, tx transaction.Client) error {
return l.notifyEvent(watch.Modified, obj, tx)
}
func (l *ListOptionIndexer) notifyEventDeleted(key string, obj any, tx transaction.Client) error {
return l.notifyEvent(watch.Deleted, obj, tx)
}
func (l *ListOptionIndexer) notifyEvent(eventType watch.EventType, obj any, tx transaction.Client) error {
l.watchersLock.RLock()
for _, watcher := range l.watchers {
watcher <- watch.Event{
Type: eventType,
Object: obj.(runtime.Object),
}
}
l.watchersLock.RUnlock()
return nil
}
// addIndexFields saves sortable/filterable fields into tables
func (l *ListOptionIndexer) addIndexFields(key string, obj any, tx transaction.Client) error {
args := []any{key}

View File

@@ -12,6 +12,7 @@ import (
"errors"
"fmt"
"testing"
"time"
"github.com/rancher/steve/pkg/sqlcache/db"
"github.com/rancher/steve/pkg/sqlcache/encryption"
@@ -24,6 +25,7 @@ import (
"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
"k8s.io/apimachinery/pkg/runtime/schema"
"k8s.io/apimachinery/pkg/util/sets"
watch "k8s.io/apimachinery/pkg/watch"
"k8s.io/client-go/tools/cache"
)
@@ -88,9 +90,9 @@ func TestNewListOptionIndexer(t *testing.T) {
store.EXPECT().Prepare(gomock.Any()).Return(stmt).AnyTimes()
// end NewIndexer() logic
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDeleteAll(gomock.Any()).Times(2)
// create field table
@@ -156,9 +158,9 @@ func TestNewListOptionIndexer(t *testing.T) {
store.EXPECT().Prepare(gomock.Any()).Return(stmt).AnyTimes()
// end NewIndexer() logic
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDeleteAll(gomock.Any()).Times(2)
store.EXPECT().WithTransaction(gomock.Any(), true, gomock.Any()).Return(fmt.Errorf("error"))
@@ -188,9 +190,9 @@ func TestNewListOptionIndexer(t *testing.T) {
store.EXPECT().Prepare(gomock.Any()).Return(stmt).AnyTimes()
// end NewIndexer() logic
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDeleteAll(gomock.Any()).Times(2)
txClient.EXPECT().Exec(fmt.Sprintf(createFieldsTableFmt, id, `"metadata.name" TEXT, "metadata.creationTimestamp" TEXT, "metadata.namespace" TEXT, "something" TEXT`)).Return(nil, nil)
@@ -228,9 +230,9 @@ func TestNewListOptionIndexer(t *testing.T) {
store.EXPECT().Prepare(gomock.Any()).Return(stmt).AnyTimes()
// end NewIndexer() logic
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDeleteAll(gomock.Any()).Times(2)
txClient.EXPECT().Exec(fmt.Sprintf(createFieldsTableFmt, id, `"metadata.name" TEXT, "metadata.creationTimestamp" TEXT, "metadata.namespace" TEXT, "something" TEXT`)).Return(nil, nil)
@@ -272,9 +274,9 @@ func TestNewListOptionIndexer(t *testing.T) {
store.EXPECT().Prepare(gomock.Any()).Return(stmt).AnyTimes()
// end NewIndexer() logic
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(2)
store.EXPECT().RegisterAfterAdd(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterUpdate(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDelete(gomock.Any()).Times(3)
store.EXPECT().RegisterAfterDeleteAll(gomock.Any()).Times(2)
txClient.EXPECT().Exec(fmt.Sprintf(createFieldsTableFmt, id, `"metadata.name" TEXT, "metadata.creationTimestamp" TEXT, "metadata.namespace" TEXT, "something" TEXT`)).Return(nil, nil)
@@ -1816,3 +1818,114 @@ func TestGetField(t *testing.T) {
})
}
}
func TestWatchMany(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
loi, err := makeListOptionIndexer(ctx, [][]string{{"metadata", "somefield"}})
assert.NoError(t, err)
startWatcher := func(ctx context.Context) (chan watch.Event, chan error) {
errCh := make(chan error, 1)
eventsCh := make(chan watch.Event, 100)
go func() {
watchErr := loi.Watch(ctx, WatchOptions{}, eventsCh)
errCh <- watchErr
}()
time.Sleep(100 * time.Millisecond)
return eventsCh, errCh
}
waitStopWatcher := func(errCh chan error) error {
select {
case <-time.After(time.Second * 5):
return fmt.Errorf("not finished in time")
case err := <-errCh:
return err
}
}
receiveEvents := func(eventsCh chan watch.Event) []watch.Event {
timer := time.NewTimer(time.Millisecond * 50)
var events []watch.Event
for {
select {
case <-timer.C:
return events
case ev := <-eventsCh:
events = append(events, ev)
}
}
}
watcher1, errCh1 := startWatcher(ctx)
events := receiveEvents(watcher1)
assert.Len(t, events, 0)
foo := &unstructured.Unstructured{
Object: map[string]any{
"metadata": map[string]any{
"name": "foo",
},
},
}
fooUpdated := foo.DeepCopy()
fooUpdated.SetLabels(map[string]string{
"hello": "world",
})
err = loi.Add(foo)
assert.NoError(t, err)
events = receiveEvents(watcher1)
assert.Equal(t, []watch.Event{{Type: watch.Added, Object: foo}}, events)
ctx2, cancel2 := context.WithCancel(context.Background())
watcher2, errCh2 := startWatcher(ctx2)
err = loi.Update(fooUpdated)
assert.NoError(t, err)
events = receiveEvents(watcher1)
assert.Equal(t, []watch.Event{{Type: watch.Modified, Object: fooUpdated}}, events)
events = receiveEvents(watcher2)
assert.Equal(t, []watch.Event{{Type: watch.Modified, Object: fooUpdated}}, events)
watcher3, errCh3 := startWatcher(ctx)
cancel2()
err = waitStopWatcher(errCh2)
assert.NoError(t, err)
err = loi.Delete(fooUpdated)
assert.NoError(t, err)
err = loi.Add(foo)
assert.NoError(t, err)
err = loi.Update(fooUpdated)
assert.NoError(t, err)
events = receiveEvents(watcher3)
assert.Equal(t, []watch.Event{
{Type: watch.Deleted, Object: fooUpdated},
{Type: watch.Added, Object: foo},
{Type: watch.Modified, Object: fooUpdated},
}, events)
// Verify cancelled watcher don't receive anything anymore
events = receiveEvents(watcher2)
assert.Len(t, events, 0)
events = receiveEvents(watcher1)
assert.Equal(t, []watch.Event{
{Type: watch.Deleted, Object: fooUpdated},
{Type: watch.Added, Object: foo},
{Type: watch.Modified, Object: fooUpdated},
}, events)
cancel()
err = waitStopWatcher(errCh1)
assert.NoError(t, err)
err = waitStopWatcher(errCh3)
assert.NoError(t, err)
}