client-go/tools/cache: add APIs with context parameter

The context is used for cancellation and to support contextual logging.

In most cases, alternative *WithContext APIs get added, except for
NewIntegerResourceVersionMutationCache where code searches indicate that the
API is not used downstream.

An API break around SharedInformer couldn't be avoided because the
alternative (keeping the interface unchanged and adding a second one with
the new method) would have been worse. controller-runtime needs to be updated
because it implements that interface in a test package. Downstream consumers of
controller-runtime will work unless they use those test package.

Converting Kubernetes to use the other new alternatives will follow. In the
meantime, usage of the new alternatives cannot be enforced via logcheck
yet (see https://github.com/kubernetes/kubernetes/issues/126379 for the
process).

Passing context through and checking it for cancellation is tricky for event
handlers. A better approach is to map the context cancellation to the normal
removal of an event handler via a helper goroutine. Thanks to the new
HandleErrorWithLogr and HandleCrashWithLogr, remembering the logger is
sufficient for handling problems at runtime.

Kubernetes-commit: 4638ba971661497b147906b8977ae206c9dd6e44
This commit is contained in:
Patrick Ohly
2024-07-26 15:26:00 +02:00
committed by Kubernetes Publisher
parent b836a27b07
commit 5d289bc44c
16 changed files with 800 additions and 312 deletions

View File

@@ -45,6 +45,7 @@ import (
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/apimachinery/pkg/watch"
"k8s.io/klog/v2/ktesting"
"k8s.io/utils/clock"
testingclock "k8s.io/utils/clock/testing"
)
@@ -64,6 +65,7 @@ func (t *testLW) Watch(options metav1.ListOptions) (watch.Interface, error) {
}
func TestCloseWatchChannelOnError(t *testing.T) {
_, ctx := ktesting.NewTestContext(t)
r := NewReflector(&testLW{}, &v1.Pod{}, NewStore(MetaNamespaceKeyFunc), 0)
pod := &v1.Pod{ObjectMeta: metav1.ObjectMeta{Name: "bar"}}
fw := watch.NewFake()
@@ -75,7 +77,7 @@ func TestCloseWatchChannelOnError(t *testing.T) {
return &v1.PodList{ListMeta: metav1.ListMeta{ResourceVersion: "1"}}, nil
},
}
go r.ListAndWatch(wait.NeverStop)
go func() { assert.NoError(t, r.ListAndWatchWithContext(ctx)) }()
fw.Error(pod)
select {
case _, ok := <-fw.ResultChan():
@@ -89,7 +91,8 @@ func TestCloseWatchChannelOnError(t *testing.T) {
}
func TestRunUntil(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
store := NewStore(MetaNamespaceKeyFunc)
r := NewReflector(&testLW{}, &v1.Pod{}, store, 0)
fw := watch.NewFake()
@@ -104,13 +107,13 @@ func TestRunUntil(t *testing.T) {
doneCh := make(chan struct{})
go func() {
defer close(doneCh)
r.Run(stopCh)
r.RunWithContext(ctx)
}()
// Synchronously add a dummy pod into the watch channel so we
// know the RunUntil go routine is in the watch handler.
fw.Add(&v1.Pod{ObjectMeta: metav1.ObjectMeta{Name: "bar"}})
close(stopCh)
cancel(errors.New("done"))
resultCh := fw.ResultChan()
for {
select {
@@ -149,8 +152,9 @@ func TestReflectorResyncChan(t *testing.T) {
// TestReflectorWatchStoppedBefore ensures that neither List nor Watch are
// called if the stop channel is closed before Reflector.watch is called.
func TestReflectorWatchStoppedBefore(t *testing.T) {
stopCh := make(chan struct{})
close(stopCh)
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
cancel(errors.New("don't run"))
lw := &ListWatch{
ListFunc: func(_ metav1.ListOptions) (runtime.Object, error) {
@@ -165,14 +169,15 @@ func TestReflectorWatchStoppedBefore(t *testing.T) {
}
target := NewReflector(lw, &v1.Pod{}, nil, 0)
err := target.watch(nil, stopCh, nil)
err := target.watch(ctx, nil, nil)
require.NoError(t, err)
}
// TestReflectorWatchStoppedAfter ensures that neither the watcher is stopped if
// the stop channel is closed after Reflector.watch has started watching.
func TestReflectorWatchStoppedAfter(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
var watchers []*watch.FakeWatcher
@@ -185,7 +190,7 @@ func TestReflectorWatchStoppedAfter(t *testing.T) {
// Simulate the stop channel being closed after watching has started
go func() {
time.Sleep(10 * time.Millisecond)
close(stopCh)
cancel(errors.New("10ms timeout reached"))
}()
// Use a fake watcher that never sends events
w := watch.NewFake()
@@ -195,7 +200,7 @@ func TestReflectorWatchStoppedAfter(t *testing.T) {
}
target := NewReflector(lw, &v1.Pod{}, nil, 0)
err := target.watch(nil, stopCh, nil)
err := target.watch(ctx, nil, nil)
require.NoError(t, err)
require.Len(t, watchers, 1)
require.True(t, watchers[0].IsStopped())
@@ -219,9 +224,10 @@ func BenchmarkReflectorResyncChanMany(b *testing.B) {
func TestReflectorHandleWatchStoppedBefore(t *testing.T) {
s := NewStore(MetaNamespaceKeyFunc)
g := NewReflector(&testLW{}, &v1.Pod{}, s, 0)
stopCh := make(chan struct{})
// Simulate the watch channel being closed before the watchHandler is called
close(stopCh)
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
// Simulate the context being canceled before the watchHandler is called
cancel(errors.New("don't run"))
var calls []string
resultCh := make(chan watch.Event)
fw := watch.MockWatcher{
@@ -234,7 +240,7 @@ func TestReflectorHandleWatchStoppedBefore(t *testing.T) {
return resultCh
},
}
err := handleWatch(time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc, stopCh)
err := handleWatch(ctx, time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc)
if err == nil {
t.Errorf("unexpected non-error")
}
@@ -251,7 +257,8 @@ func TestReflectorHandleWatchStoppedAfter(t *testing.T) {
s := NewStore(MetaNamespaceKeyFunc)
g := NewReflector(&testLW{}, &v1.Pod{}, s, 0)
var calls []string
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
resultCh := make(chan watch.Event)
fw := watch.MockWatcher{
StopFunc: func() {
@@ -265,12 +272,12 @@ func TestReflectorHandleWatchStoppedAfter(t *testing.T) {
// caller, after watching has started.
go func() {
time.Sleep(10 * time.Millisecond)
close(stopCh)
cancel(errors.New("10ms timeout reached"))
}()
return resultCh
},
}
err := handleWatch(time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc, stopCh)
err := handleWatch(ctx, time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc)
if err == nil {
t.Errorf("unexpected non-error")
}
@@ -285,6 +292,7 @@ func TestReflectorHandleWatchStoppedAfter(t *testing.T) {
func TestReflectorHandleWatchResultChanClosedBefore(t *testing.T) {
s := NewStore(MetaNamespaceKeyFunc)
g := NewReflector(&testLW{}, &v1.Pod{}, s, 0)
_, ctx := ktesting.NewTestContext(t)
var calls []string
resultCh := make(chan watch.Event)
fw := watch.MockWatcher{
@@ -298,7 +306,7 @@ func TestReflectorHandleWatchResultChanClosedBefore(t *testing.T) {
}
// Simulate the result channel being closed by the producer before handleWatch is called.
close(resultCh)
err := handleWatch(time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc, wait.NeverStop)
err := handleWatch(ctx, time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc)
if err == nil {
t.Errorf("unexpected non-error")
}
@@ -313,6 +321,7 @@ func TestReflectorHandleWatchResultChanClosedBefore(t *testing.T) {
func TestReflectorHandleWatchResultChanClosedAfter(t *testing.T) {
s := NewStore(MetaNamespaceKeyFunc)
g := NewReflector(&testLW{}, &v1.Pod{}, s, 0)
_, ctx := ktesting.NewTestContext(t)
var calls []string
resultCh := make(chan watch.Event)
fw := watch.MockWatcher{
@@ -331,7 +340,7 @@ func TestReflectorHandleWatchResultChanClosedAfter(t *testing.T) {
return resultCh
},
}
err := handleWatch(time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc, wait.NeverStop)
err := handleWatch(ctx, time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc)
if err == nil {
t.Errorf("unexpected non-error")
}
@@ -348,11 +357,12 @@ func TestReflectorWatchHandler(t *testing.T) {
// watching after all the events have been consumed. This avoids race
// conditions which can happen if the producer calls Stop(), instead of the
// consumer.
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
setLastSyncResourceVersion := func(rv string) {
g.setLastSyncResourceVersion(rv)
if rv == "32" {
close(stopCh)
cancel(errors.New("LastSyncResourceVersion is 32"))
}
}
fw := watch.NewFake()
@@ -365,7 +375,7 @@ func TestReflectorWatchHandler(t *testing.T) {
fw.Add(&v1.Pod{ObjectMeta: metav1.ObjectMeta{Name: "baz", ResourceVersion: "32"}})
fw.Stop()
}()
err := handleWatch(time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, setLastSyncResourceVersion, g.clock, nevererrc, stopCh)
err := handleWatch(ctx, time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, setLastSyncResourceVersion, g.clock, nevererrc)
// TODO(karlkfi): Fix FakeWatcher to avoid race condition between watcher.Stop() & close(stopCh)
if err != nil && !errors.Is(err, errorStopRequested) {
t.Errorf("unexpected error %v", err)
@@ -408,15 +418,19 @@ func TestReflectorStopWatch(t *testing.T) {
s := NewStore(MetaNamespaceKeyFunc)
g := NewReflector(&testLW{}, &v1.Pod{}, s, 0)
fw := watch.NewFake()
stopWatch := make(chan struct{})
close(stopWatch)
err := handleWatch(time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc, stopWatch)
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
cancel(errors.New("don't run"))
err := handleWatch(ctx, time.Now(), fw, s, g.expectedType, g.expectedGVK, g.name, g.typeDescription, g.setLastSyncResourceVersion, g.clock, nevererrc)
if err != errorStopRequested {
t.Errorf("expected stop error, got %q", err)
}
}
func TestReflectorListAndWatch(t *testing.T) {
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancel(ctx)
defer cancel()
createdFakes := make(chan *watch.FakeWatcher)
// The ListFunc says that it's at revision 1. Therefore, we expect our WatchFunc
@@ -442,7 +456,7 @@ func TestReflectorListAndWatch(t *testing.T) {
}
s := NewFIFO(MetaNamespaceKeyFunc)
r := NewReflector(lw, &v1.Pod{}, s, 0)
go r.ListAndWatch(wait.NeverStop)
go func() { assert.NoError(t, r.ListAndWatchWithContext(ctx)) }()
ids := []string{"foo", "bar", "baz", "qux", "zoo"}
var fw *watch.FakeWatcher
@@ -539,7 +553,8 @@ func TestReflectorListAndWatchWithErrors(t *testing.T) {
}
}
watchRet, watchErr := item.events, item.watchErr
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
lw := &testLW{
WatchFunc: func(options metav1.ListOptions) (watch.Interface, error) {
if watchErr != nil {
@@ -557,7 +572,7 @@ func TestReflectorListAndWatchWithErrors(t *testing.T) {
// result channel, and wait for the consumer to stop the
// watcher, to avoid race conditions.
// TODO: Fix the FakeWatcher to separate watcher.Stop from close(resultCh)
close(stopCh)
cancel(errors.New("done"))
}()
return fw, nil
},
@@ -566,7 +581,7 @@ func TestReflectorListAndWatchWithErrors(t *testing.T) {
},
}
r := NewReflector(lw, &v1.Pod{}, s, 0)
err := r.ListAndWatch(stopCh)
err := r.ListAndWatchWithContext(ctx)
if item.listErr != nil && !errors.Is(err, item.listErr) {
t.Errorf("unexpected ListAndWatch error: %v", err)
}
@@ -593,7 +608,8 @@ func TestReflectorListAndWatchInitConnBackoff(t *testing.T) {
for _, test := range table {
t.Run(fmt.Sprintf("%d connection failures takes at least %d ms", test.numConnFails, 1<<test.numConnFails),
func(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
connFails := test.numConnFails
fakeClock := testingclock.NewFakeClock(time.Unix(0, 0))
bm := wait.NewExponentialBackoffManager(time.Millisecond, maxBackoff, 100*time.Millisecond, 2.0, 1.0, fakeClock)
@@ -624,7 +640,7 @@ func TestReflectorListAndWatchInitConnBackoff(t *testing.T) {
connFails--
return nil, syscall.ECONNREFUSED
}
close(stopCh)
cancel(errors.New("done"))
return watch.NewFake(), nil
},
ListFunc: func(options metav1.ListOptions) (runtime.Object, error) {
@@ -637,10 +653,10 @@ func TestReflectorListAndWatchInitConnBackoff(t *testing.T) {
store: NewFIFO(MetaNamespaceKeyFunc),
backoffManager: bm,
clock: fakeClock,
watchErrorHandler: WatchErrorHandler(DefaultWatchErrorHandler),
watchErrorHandler: WatchErrorHandlerWithContext(DefaultWatchErrorHandler),
}
start := fakeClock.Now()
err := r.ListAndWatch(stopCh)
err := r.ListAndWatchWithContext(ctx)
elapsed := fakeClock.Since(start)
if err != nil {
t.Errorf("unexpected error %v", err)
@@ -666,6 +682,7 @@ func (f *fakeBackoff) Backoff() clock.Timer {
}
func TestBackoffOnTooManyRequests(t *testing.T) {
_, ctx := ktesting.NewTestContext(t)
err := apierrors.NewTooManyRequests("too many requests", 1)
clock := &clock.RealClock{}
bm := &fakeBackoff{clock: clock}
@@ -697,11 +714,11 @@ func TestBackoffOnTooManyRequests(t *testing.T) {
store: NewFIFO(MetaNamespaceKeyFunc),
backoffManager: bm,
clock: clock,
watchErrorHandler: WatchErrorHandler(DefaultWatchErrorHandler),
watchErrorHandler: WatchErrorHandlerWithContext(DefaultWatchErrorHandler),
}
stopCh := make(chan struct{})
if err := r.ListAndWatch(stopCh); err != nil {
if err := r.ListAndWatchWithContext(ctx); err != nil {
t.Fatal(err)
}
close(stopCh)
@@ -738,14 +755,15 @@ func TestNoRelistOnTooManyRequests(t *testing.T) {
store: NewFIFO(MetaNamespaceKeyFunc),
backoffManager: bm,
clock: clock,
watchErrorHandler: WatchErrorHandler(DefaultWatchErrorHandler),
watchErrorHandler: WatchErrorHandlerWithContext(DefaultWatchErrorHandler),
}
stopCh := make(chan struct{})
if err := r.ListAndWatch(stopCh); err != nil {
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
if err := r.ListAndWatchWithContext(ctx); err != nil {
t.Fatal(err)
}
close(stopCh)
cancel(errors.New("done"))
if listCalls != 1 {
t.Errorf("unexpected list calls: %d", listCalls)
}
@@ -812,14 +830,15 @@ func TestRetryInternalError(t *testing.T) {
store: NewFIFO(MetaNamespaceKeyFunc),
backoffManager: bm,
clock: fakeClock,
watchErrorHandler: WatchErrorHandler(DefaultWatchErrorHandler),
watchErrorHandler: WatchErrorHandlerWithContext(DefaultWatchErrorHandler),
}
r.MaxInternalErrorRetryDuration = tc.maxInternalDuration
stopCh := make(chan struct{})
r.ListAndWatch(stopCh)
close(stopCh)
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
require.NoError(t, r.ListAndWatchWithContext(ctx))
cancel(errors.New("done"))
if counter-1 != tc.wantRetries {
t.Errorf("%v unexpected number of retries: %d", tc, counter-1)
@@ -829,7 +848,7 @@ func TestRetryInternalError(t *testing.T) {
func TestReflectorResync(t *testing.T) {
iteration := 0
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
rerr := errors.New("expected resync reached")
s := &FakeCustomStore{
ResyncFunc: func() error {
@@ -852,7 +871,7 @@ func TestReflectorResync(t *testing.T) {
}
resyncPeriod := 1 * time.Millisecond
r := NewReflector(lw, &v1.Pod{}, s, resyncPeriod)
if err := r.ListAndWatch(stopCh); err != nil {
if err := r.ListAndWatchWithContext(ctx); err != nil {
// error from Resync is not propaged up to here.
t.Errorf("expected error %v", err)
}
@@ -862,13 +881,14 @@ func TestReflectorResync(t *testing.T) {
}
func TestReflectorWatchListPageSize(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
s := NewStore(MetaNamespaceKeyFunc)
lw := &testLW{
WatchFunc: func(options metav1.ListOptions) (watch.Interface, error) {
// Stop once the reflector begins watching since we're only interested in the list.
close(stopCh)
cancel(errors.New("done"))
fw := watch.NewFake()
return fw, nil
},
@@ -898,7 +918,7 @@ func TestReflectorWatchListPageSize(t *testing.T) {
r.setLastSyncResourceVersion("10")
// Set the reflector to paginate the list request in 4 item chunks.
r.WatchListPageSize = 4
r.ListAndWatch(stopCh)
require.NoError(t, r.ListAndWatchWithContext(ctx))
results := s.List()
if len(results) != 10 {
@@ -907,13 +927,14 @@ func TestReflectorWatchListPageSize(t *testing.T) {
}
func TestReflectorNotPaginatingNotConsistentReads(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
s := NewStore(MetaNamespaceKeyFunc)
lw := &testLW{
WatchFunc: func(options metav1.ListOptions) (watch.Interface, error) {
// Stop once the reflector begins watching since we're only interested in the list.
close(stopCh)
cancel(errors.New("done"))
fw := watch.NewFake()
return fw, nil
},
@@ -933,7 +954,7 @@ func TestReflectorNotPaginatingNotConsistentReads(t *testing.T) {
}
r := NewReflector(lw, &v1.Pod{}, s, 0)
r.setLastSyncResourceVersion("10")
r.ListAndWatch(stopCh)
require.NoError(t, r.ListAndWatchWithContext(ctx))
results := s.List()
if len(results) != 10 {
@@ -942,13 +963,14 @@ func TestReflectorNotPaginatingNotConsistentReads(t *testing.T) {
}
func TestReflectorPaginatingNonConsistentReadsIfWatchCacheDisabled(t *testing.T) {
var stopCh chan struct{}
_, ctx := ktesting.NewTestContext(t)
var cancel func(error)
s := NewStore(MetaNamespaceKeyFunc)
lw := &testLW{
WatchFunc: func(options metav1.ListOptions) (watch.Interface, error) {
// Stop once the reflector begins watching since we're only interested in the list.
close(stopCh)
cancel(errors.New("done"))
fw := watch.NewFake()
return fw, nil
},
@@ -977,16 +999,17 @@ func TestReflectorPaginatingNonConsistentReadsIfWatchCacheDisabled(t *testing.T)
r := NewReflector(lw, &v1.Pod{}, s, 0)
// Initial list should initialize paginatedResult in the reflector.
stopCh = make(chan struct{})
r.ListAndWatch(stopCh)
var cancelCtx context.Context
cancelCtx, cancel = context.WithCancelCause(ctx)
require.NoError(t, r.ListAndWatchWithContext(cancelCtx))
if results := s.List(); len(results) != 10 {
t.Errorf("Expected 10 results, got %d", len(results))
}
// Since initial list for ResourceVersion="0" was paginated, the subsequent
// ones should also be paginated.
stopCh = make(chan struct{})
r.ListAndWatch(stopCh)
cancelCtx, cancel = context.WithCancelCause(ctx)
require.NoError(t, r.ListAndWatchWithContext(cancelCtx))
if results := s.List(); len(results) != 10 {
t.Errorf("Expected 10 results, got %d", len(results))
}
@@ -996,14 +1019,15 @@ func TestReflectorPaginatingNonConsistentReadsIfWatchCacheDisabled(t *testing.T)
// it in relist requests to prevent the reflector from traveling back in time if the relist is to a api-server or
// etcd that is partitioned and serving older data than the reflector has already processed.
func TestReflectorResyncWithResourceVersion(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
cancelCtx, cancel := context.WithCancelCause(ctx)
s := NewStore(MetaNamespaceKeyFunc)
listCallRVs := []string{}
lw := &testLW{
WatchFunc: func(options metav1.ListOptions) (watch.Interface, error) {
// Stop once the reflector begins watching since we're only interested in the list.
close(stopCh)
cancel(errors.New("done"))
fw := watch.NewFake()
return fw, nil
},
@@ -1027,7 +1051,7 @@ func TestReflectorResyncWithResourceVersion(t *testing.T) {
r := NewReflector(lw, &v1.Pod{}, s, 0)
// Initial list should use RV=0
r.ListAndWatch(stopCh)
require.NoError(t, r.ListAndWatchWithContext(cancelCtx))
results := s.List()
if len(results) != 4 {
@@ -1035,8 +1059,8 @@ func TestReflectorResyncWithResourceVersion(t *testing.T) {
}
// relist should use lastSyncResourceVersions (RV=10)
stopCh = make(chan struct{})
r.ListAndWatch(stopCh)
cancelCtx, cancel = context.WithCancelCause(ctx)
require.NoError(t, r.ListAndWatchWithContext(cancelCtx))
results = s.List()
if len(results) != 8 {
@@ -1055,14 +1079,16 @@ func TestReflectorResyncWithResourceVersion(t *testing.T) {
// (In kubernetes 1.17, or when the watch cache is enabled, the List will instead return the list that is no older than
// the requested ResourceVersion).
func TestReflectorExpiredExactResourceVersion(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
var cancelCtx context.Context
var cancel func(error)
s := NewStore(MetaNamespaceKeyFunc)
listCallRVs := []string{}
lw := &testLW{
WatchFunc: func(options metav1.ListOptions) (watch.Interface, error) {
// Stop once the reflector begins watching since we're only interested in the list.
close(stopCh)
cancel(errors.New("done"))
fw := watch.NewFake()
return fw, nil
},
@@ -1089,7 +1115,8 @@ func TestReflectorExpiredExactResourceVersion(t *testing.T) {
r := NewReflector(lw, &v1.Pod{}, s, 0)
// Initial list should use RV=0
r.ListAndWatch(stopCh)
cancelCtx, cancel = context.WithCancelCause(ctx)
require.NoError(t, r.ListAndWatchWithContext(cancelCtx))
results := s.List()
if len(results) != 4 {
@@ -1097,8 +1124,8 @@ func TestReflectorExpiredExactResourceVersion(t *testing.T) {
}
// relist should use lastSyncResourceVersions (RV=10) and since RV=10 is expired, it should retry with RV="".
stopCh = make(chan struct{})
r.ListAndWatch(stopCh)
cancelCtx, cancel = context.WithCancelCause(ctx)
require.NoError(t, r.ListAndWatchWithContext(cancelCtx))
results = s.List()
if len(results) != 8 {
@@ -1112,14 +1139,16 @@ func TestReflectorExpiredExactResourceVersion(t *testing.T) {
}
func TestReflectorFullListIfExpired(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
var cancelCtx context.Context
var cancel func(error)
s := NewStore(MetaNamespaceKeyFunc)
listCallRVs := []string{}
lw := &testLW{
WatchFunc: func(options metav1.ListOptions) (watch.Interface, error) {
// Stop once the reflector begins watching since we're only interested in the list.
close(stopCh)
cancel(errors.New("done"))
fw := watch.NewFake()
return fw, nil
},
@@ -1156,7 +1185,8 @@ func TestReflectorFullListIfExpired(t *testing.T) {
r.WatchListPageSize = 4
// Initial list should use RV=0
if err := r.ListAndWatch(stopCh); err != nil {
cancelCtx, cancel = context.WithCancelCause(ctx)
if err := r.ListAndWatchWithContext(cancelCtx); err != nil {
t.Fatal(err)
}
@@ -1166,8 +1196,8 @@ func TestReflectorFullListIfExpired(t *testing.T) {
}
// relist should use lastSyncResourceVersions (RV=10) and since second page of that expired, it should full list with RV=10
stopCh = make(chan struct{})
if err := r.ListAndWatch(stopCh); err != nil {
cancelCtx, cancel = context.WithCancelCause(ctx)
if err := r.ListAndWatchWithContext(cancelCtx); err != nil {
t.Fatal(err)
}
@@ -1183,7 +1213,9 @@ func TestReflectorFullListIfExpired(t *testing.T) {
}
func TestReflectorFullListIfTooLarge(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
var cancelCtx context.Context
var cancel func(error)
s := NewStore(MetaNamespaceKeyFunc)
listCallRVs := []string{}
version := 30
@@ -1191,7 +1223,7 @@ func TestReflectorFullListIfTooLarge(t *testing.T) {
lw := &testLW{
WatchFunc: func(options metav1.ListOptions) (watch.Interface, error) {
// Stop once the reflector begins watching since we're only interested in the list.
close(stopCh)
cancel(errors.New("done"))
fw := watch.NewFake()
return fw, nil
},
@@ -1229,7 +1261,8 @@ func TestReflectorFullListIfTooLarge(t *testing.T) {
r := NewReflector(lw, &v1.Pod{}, s, 0)
// Initial list should use RV=0
if err := r.ListAndWatch(stopCh); err != nil {
cancelCtx, cancel = context.WithCancelCause(ctx)
if err := r.ListAndWatchWithContext(cancelCtx); err != nil {
t.Fatal(err)
}
@@ -1241,8 +1274,8 @@ func TestReflectorFullListIfTooLarge(t *testing.T) {
// done we simply try to relist from now to avoid continuous errors on relists.
for i := 1; i <= 3; i++ {
// relist twice to cover the two variants of TooLargeResourceVersion api errors
stopCh = make(chan struct{})
if err := r.ListAndWatch(stopCh); err != nil {
cancelCtx, cancel = context.WithCancelCause(ctx)
if err := r.ListAndWatchWithContext(cancelCtx); err != nil {
t.Fatal(err)
}
}
@@ -1356,7 +1389,8 @@ func TestWatchTimeout(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
s := NewStore(MetaNamespaceKeyFunc)
var gotTimeoutSeconds int64
@@ -1370,7 +1404,7 @@ func TestWatchTimeout(t *testing.T) {
}
// Stop once the reflector begins watching since we're only interested in the list.
close(stopCh)
cancel(errors.New("done"))
return watch.NewFake(), nil
},
}
@@ -1379,7 +1413,7 @@ func TestWatchTimeout(t *testing.T) {
MinWatchTimeout: tc.minWatchTimeout,
}
r := NewReflectorWithOptions(lw, &v1.Pod{}, s, opts)
if err := r.ListAndWatch(stopCh); err != nil {
if err := r.ListAndWatchWithContext(ctx); err != nil {
t.Fatal(err)
}
@@ -1412,7 +1446,8 @@ func newStoreWithRV() *storeWithRV {
func TestReflectorResourceVersionUpdate(t *testing.T) {
s := newStoreWithRV()
stopCh := make(chan struct{})
_, ctx := ktesting.NewTestContext(t)
ctx, cancel := context.WithCancelCause(ctx)
fw := watch.NewFake()
lw := &testLW{
@@ -1434,11 +1469,11 @@ func TestReflectorResourceVersionUpdate(t *testing.T) {
fw.Action(watch.Modified, makePod("20"))
fw.Action(watch.Bookmark, makePod("30"))
fw.Action(watch.Deleted, makePod("40"))
close(stopCh)
cancel(errors.New("done"))
}()
// Initial list should use RV=0
if err := r.ListAndWatch(stopCh); err != nil {
if err := r.ListAndWatchWithContext(ctx); err != nil {
t.Fatal(err)
}
@@ -1680,6 +1715,7 @@ func (t *TestPagingPodsLW) Watch(options metav1.ListOptions) (watch.Interface, e
}
func TestReflectorListExtract(t *testing.T) {
_, ctx := ktesting.NewTestContext(t)
store := NewStore(func(obj interface{}) (string, error) {
pod, ok := obj.(*v1.Pod)
if !ok {
@@ -1693,8 +1729,7 @@ func TestReflectorListExtract(t *testing.T) {
reflector.WatchListPageSize = fakeItemsNum
// execute list to fill store
stopCh := make(chan struct{})
if err := reflector.list(stopCh); err != nil {
if err := reflector.list(ctx); err != nil {
t.Fatal(err)
}
@@ -2074,7 +2109,7 @@ func BenchmarkReflectorList(b *testing.B) {
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := reflector.list(ctx.Done())
err := reflector.list(ctx)
if err != nil {
b.Fatalf("reflect list: %v", err)
}