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

@@ -95,7 +95,7 @@ type Reflector struct {
// lastSyncResourceVersionMutex guards read/write access to lastSyncResourceVersion
lastSyncResourceVersionMutex sync.RWMutex
// Called whenever the ListAndWatch drops the connection with an error.
watchErrorHandler WatchErrorHandler
watchErrorHandler WatchErrorHandlerWithContext
// WatchListPageSize is the requested chunk size of initial and resync watch lists.
// If unset, for consistent reads (RV="") or reads that opt-into arbitrarily old data
// (RV="0") it will default to pager.PageSize, for the rest (RV != "" && RV != "0")
@@ -150,20 +150,32 @@ type ResourceVersionUpdater interface {
// should be offloaded.
type WatchErrorHandler func(r *Reflector, err error)
// DefaultWatchErrorHandler is the default implementation of WatchErrorHandler
func DefaultWatchErrorHandler(r *Reflector, err error) {
// The WatchErrorHandler is called whenever ListAndWatch drops the
// connection with an error. After calling this handler, the informer
// will backoff and retry.
//
// The default implementation looks at the error type and tries to log
// the error message at an appropriate level.
//
// Implementations of this handler may display the error message in other
// ways. Implementations should return quickly - any expensive processing
// should be offloaded.
type WatchErrorHandlerWithContext func(ctx context.Context, r *Reflector, err error)
// DefaultWatchErrorHandler is the default implementation of WatchErrorHandlerWithContext.
func DefaultWatchErrorHandler(ctx context.Context, r *Reflector, err error) {
switch {
case isExpiredError(err):
// Don't set LastSyncResourceVersionUnavailable - LIST call with ResourceVersion=RV already
// has a semantic that it returns data at least as fresh as provided RV.
// So first try to LIST with setting RV to resource version of last observed object.
klog.V(4).Infof("%s: watch of %v closed with: %v", r.name, r.typeDescription, err)
klog.FromContext(ctx).V(4).Info("Watch closed", "reflector", r.name, "type", r.typeDescription, "err", err)
case err == io.EOF:
// watch closed normally
case err == io.ErrUnexpectedEOF:
klog.V(1).Infof("%s: Watch for %v closed with unexpected EOF: %v", r.name, r.typeDescription, err)
klog.FromContext(ctx).V(1).Info("Watch closed with unexpected EOF", "reflector", r.name, "type", r.typeDescription, "err", err)
default:
utilruntime.HandleError(fmt.Errorf("%s: Failed to watch %v: %v", r.name, r.typeDescription, err))
utilruntime.HandleErrorWithContext(ctx, err, "Failed to watch", "reflector", r.name, "type", r.typeDescription)
}
}
@@ -243,7 +255,7 @@ func NewReflectorWithOptions(lw ListerWatcher, expectedType interface{}, store S
// 0.22 QPS. If we don't backoff for 2min, assume API server is healthy and we reset the backoff.
backoffManager: wait.NewExponentialBackoffManager(800*time.Millisecond, 30*time.Second, 2*time.Minute, 2.0, 1.0, reflectorClock),
clock: reflectorClock,
watchErrorHandler: WatchErrorHandler(DefaultWatchErrorHandler),
watchErrorHandler: WatchErrorHandlerWithContext(DefaultWatchErrorHandler),
expectedType: reflect.TypeOf(expectedType),
}
@@ -309,14 +321,24 @@ var internalPackages = []string{"client-go/tools/cache/"}
// Run repeatedly uses the reflector's ListAndWatch to fetch all the
// objects and subsequent deltas.
// Run will exit when stopCh is closed.
//
// Contextual logging: RunWithContext should be used instead of Run in code which supports contextual logging.
func (r *Reflector) Run(stopCh <-chan struct{}) {
klog.V(3).Infof("Starting reflector %s (%s) from %s", r.typeDescription, r.resyncPeriod, r.name)
r.RunWithContext(wait.ContextForChannel(stopCh))
}
// RunWithContext repeatedly uses the reflector's ListAndWatch to fetch all the
// objects and subsequent deltas.
// Run will exit when the context is canceled.
func (r *Reflector) RunWithContext(ctx context.Context) {
logger := klog.FromContext(ctx)
logger.V(3).Info("Starting reflector", "type", r.typeDescription, "resyncPeriod", r.resyncPeriod, "reflector", r.name)
wait.BackoffUntil(func() {
if err := r.ListAndWatch(stopCh); err != nil {
r.watchErrorHandler(r, err)
if err := r.ListAndWatchWithContext(ctx); err != nil {
r.watchErrorHandler(ctx, r, err)
}
}, r.backoffManager, true, stopCh)
klog.V(3).Infof("Stopping reflector %s (%s) from %s", r.typeDescription, r.resyncPeriod, r.name)
}, r.backoffManager, true, ctx.Done())
logger.V(3).Info("Stopping reflector", "type", r.typeDescription, "resyncPeriod", r.resyncPeriod, "reflector", r.name)
}
var (
@@ -345,21 +367,31 @@ func (r *Reflector) resyncChan() (<-chan time.Time, func() bool) {
// ListAndWatch first lists all items and get the resource version at the moment of call,
// and then use the resource version to watch.
// It returns error if ListAndWatch didn't even try to initialize watch.
//
// Contextual logging: ListAndWatchWithContext should be used instead of ListAndWatch in code which supports contextual logging.
func (r *Reflector) ListAndWatch(stopCh <-chan struct{}) error {
klog.V(3).Infof("Listing and watching %v from %s", r.typeDescription, r.name)
return r.ListAndWatchWithContext(wait.ContextForChannel(stopCh))
}
// ListAndWatchWithContext first lists all items and get the resource version at the moment of call,
// and then use the resource version to watch.
// It returns error if ListAndWatchWithContext didn't even try to initialize watch.
func (r *Reflector) ListAndWatchWithContext(ctx context.Context) error {
logger := klog.FromContext(ctx)
logger.V(3).Info("Listing and watching", "type", r.typeDescription, "reflector", r.name)
var err error
var w watch.Interface
useWatchList := ptr.Deref(r.UseWatchList, false)
fallbackToList := !useWatchList
if useWatchList {
w, err = r.watchList(stopCh)
w, err = r.watchList(ctx)
if w == nil && err == nil {
// stopCh was closed
return nil
}
if err != nil {
klog.Warningf("The watchlist request ended with an error, falling back to the standard LIST/WATCH semantics because making progress is better than deadlocking, err = %v", err)
logger.Error(err, "The watchlist request ended with an error, falling back to the standard LIST/WATCH semantics because making progress is better than deadlocking")
fallbackToList = true
// ensure that we won't accidentally pass some garbage down the watch.
w = nil
@@ -367,20 +399,21 @@ func (r *Reflector) ListAndWatch(stopCh <-chan struct{}) error {
}
if fallbackToList {
err = r.list(stopCh)
err = r.list(ctx)
if err != nil {
return err
}
}
klog.V(2).Infof("Caches populated for %v from %s", r.typeDescription, r.name)
return r.watchWithResync(w, stopCh)
logger.V(2).Info("Caches populated", "type", r.typeDescription, "reflector", r.name)
return r.watchWithResync(ctx, w)
}
// startResync periodically calls r.store.Resync() method.
// Note that this method is blocking and should be
// called in a separate goroutine.
func (r *Reflector) startResync(stopCh <-chan struct{}, cancelCh <-chan struct{}, resyncerrc chan error) {
func (r *Reflector) startResync(ctx context.Context, resyncerrc chan error) {
logger := klog.FromContext(ctx)
resyncCh, cleanup := r.resyncChan()
defer func() {
cleanup() // Call the last one written into cleanup
@@ -388,13 +421,11 @@ func (r *Reflector) startResync(stopCh <-chan struct{}, cancelCh <-chan struct{}
for {
select {
case <-resyncCh:
case <-stopCh:
return
case <-cancelCh:
case <-ctx.Done():
return
}
if r.ShouldResync == nil || r.ShouldResync() {
klog.V(4).Infof("%s: forcing resync", r.name)
logger.V(4).Info("Forcing resync", "reflector", r.name)
if err := r.store.Resync(); err != nil {
resyncerrc <- err
return
@@ -406,16 +437,27 @@ func (r *Reflector) startResync(stopCh <-chan struct{}, cancelCh <-chan struct{}
}
// watchWithResync runs watch with startResync in the background.
func (r *Reflector) watchWithResync(w watch.Interface, stopCh <-chan struct{}) error {
func (r *Reflector) watchWithResync(ctx context.Context, w watch.Interface) error {
resyncerrc := make(chan error, 1)
cancelCh := make(chan struct{})
defer close(cancelCh)
go r.startResync(stopCh, cancelCh, resyncerrc)
return r.watch(w, stopCh, resyncerrc)
cancelCtx, cancel := context.WithCancel(ctx)
// Waiting for completion of the goroutine is relevant for race detector.
// Without this, there is a race between "this function returns + code
// waiting for it" and "goroutine does something".
var wg wait.Group
defer func() {
cancel()
wg.Wait()
}()
wg.Start(func() {
r.startResync(cancelCtx, resyncerrc)
})
return r.watch(ctx, w, resyncerrc)
}
// watch simply starts a watch request with the server.
func (r *Reflector) watch(w watch.Interface, stopCh <-chan struct{}, resyncerrc chan error) error {
func (r *Reflector) watch(ctx context.Context, w watch.Interface, resyncerrc chan error) error {
stopCh := ctx.Done()
logger := klog.FromContext(ctx)
var err error
retry := NewRetryWithDeadline(r.MaxInternalErrorRetryDuration, time.Minute, apierrors.IsInternalError, r.clock)
@@ -451,7 +493,7 @@ func (r *Reflector) watch(w watch.Interface, stopCh <-chan struct{}, resyncerrc
w, err = r.listerWatcher.Watch(options)
if err != nil {
if canRetry := isWatchErrorRetriable(err); canRetry {
klog.V(4).Infof("%s: watch of %v returned %v - backing off", r.name, r.typeDescription, err)
logger.V(4).Info("Watch failed - backing off", "reflector", r.name, "type", r.typeDescription, "err", err)
select {
case <-stopCh:
return nil
@@ -463,8 +505,8 @@ func (r *Reflector) watch(w watch.Interface, stopCh <-chan struct{}, resyncerrc
}
}
err = handleWatch(start, w, r.store, r.expectedType, r.expectedGVK, r.name, r.typeDescription, r.setLastSyncResourceVersion,
r.clock, resyncerrc, stopCh)
err = handleWatch(ctx, start, w, r.store, r.expectedType, r.expectedGVK, r.name, r.typeDescription, r.setLastSyncResourceVersion,
r.clock, resyncerrc)
// Ensure that watch will not be reused across iterations.
w.Stop()
w = nil
@@ -476,9 +518,9 @@ func (r *Reflector) watch(w watch.Interface, stopCh <-chan struct{}, resyncerrc
// Don't set LastSyncResourceVersionUnavailable - LIST call with ResourceVersion=RV already
// has a semantic that it returns data at least as fresh as provided RV.
// So first try to LIST with setting RV to resource version of last observed object.
klog.V(4).Infof("%s: watch of %v closed with: %v", r.name, r.typeDescription, err)
logger.V(4).Info("Watch closed", "reflector", r.name, "type", r.typeDescription, "err", err)
case apierrors.IsTooManyRequests(err):
klog.V(2).Infof("%s: watch of %v returned 429 - backing off", r.name, r.typeDescription)
logger.V(2).Info("Watch returned 429 - backing off", "reflector", r.name, "type", r.typeDescription)
select {
case <-stopCh:
return nil
@@ -486,10 +528,10 @@ func (r *Reflector) watch(w watch.Interface, stopCh <-chan struct{}, resyncerrc
continue
}
case apierrors.IsInternalError(err) && retry.ShouldRetry():
klog.V(2).Infof("%s: retrying watch of %v internal error: %v", r.name, r.typeDescription, err)
logger.V(2).Info("Retrying watch after internal error", "reflector", r.name, "type", r.typeDescription, "err", err)
continue
default:
klog.Warningf("%s: watch of %v ended with: %v", r.name, r.typeDescription, err)
logger.Info("Warning: watch ended with error", "reflector", r.name, "type", r.typeDescription, "err", err)
}
}
return nil
@@ -499,7 +541,7 @@ func (r *Reflector) watch(w watch.Interface, stopCh <-chan struct{}, resyncerrc
// list simply lists all items and records a resource version obtained from the server at the moment of the call.
// the resource version can be used for further progress notification (aka. watch).
func (r *Reflector) list(stopCh <-chan struct{}) error {
func (r *Reflector) list(ctx context.Context) error {
var resourceVersion string
options := metav1.ListOptions{ResourceVersion: r.relistResourceVersion()}
@@ -558,7 +600,7 @@ func (r *Reflector) list(stopCh <-chan struct{}) error {
close(listCh)
}()
select {
case <-stopCh:
case <-ctx.Done():
return nil
case r := <-panicCh:
panic(r)
@@ -566,7 +608,6 @@ func (r *Reflector) list(stopCh <-chan struct{}) error {
}
initTrace.Step("Objects listed", trace.Field{Key: "error", Value: err})
if err != nil {
klog.Warningf("%s: failed to list %v: %v", r.name, r.typeDescription, err)
return fmt.Errorf("failed to list %v: %w", r.typeDescription, err)
}
@@ -624,7 +665,9 @@ func (r *Reflector) list(stopCh <-chan struct{}) error {
// After receiving a "Bookmark" event the reflector is considered to be synchronized.
// It replaces its internal store with the collected items and
// reuses the current watch requests for getting further events.
func (r *Reflector) watchList(stopCh <-chan struct{}) (watch.Interface, error) {
func (r *Reflector) watchList(ctx context.Context) (watch.Interface, error) {
stopCh := ctx.Done()
logger := klog.FromContext(ctx)
var w watch.Interface
var err error
var temporaryStore Store
@@ -634,7 +677,7 @@ func (r *Reflector) watchList(stopCh <-chan struct{}) (watch.Interface, error) {
// could be unified with the r.watch method
isErrorRetriableWithSideEffectsFn := func(err error) bool {
if canRetry := isWatchErrorRetriable(err); canRetry {
klog.V(2).Infof("%s: watch-list of %v returned %v - backing off", r.name, r.typeDescription, err)
logger.V(2).Info("watch-list failed - backing off", "reflector", r.name, "type", r.typeDescription, "err", err)
<-r.backoffManager.Backoff().C()
return true
}
@@ -681,9 +724,9 @@ func (r *Reflector) watchList(stopCh <-chan struct{}) (watch.Interface, error) {
}
return nil, err
}
watchListBookmarkReceived, err := handleListWatch(start, w, temporaryStore, r.expectedType, r.expectedGVK, r.name, r.typeDescription,
watchListBookmarkReceived, err := handleListWatch(ctx, start, w, temporaryStore, r.expectedType, r.expectedGVK, r.name, r.typeDescription,
func(rv string) { resourceVersion = rv },
r.clock, make(chan error), stopCh)
r.clock, make(chan error))
if err != nil {
w.Stop() // stop and retry with clean state
if errors.Is(err, errorStopRequested) {
@@ -706,7 +749,7 @@ func (r *Reflector) watchList(stopCh <-chan struct{}) (watch.Interface, error) {
// we utilize the temporaryStore to ensure independence from the current store implementation.
// as of today, the store is implemented as a queue and will be drained by the higher-level
// component as soon as it finishes replacing the content.
checkWatchListDataConsistencyIfRequested(wait.ContextForChannel(stopCh), r.name, resourceVersion, wrapListFuncWithContext(r.listerWatcher.List), temporaryStore.List)
checkWatchListDataConsistencyIfRequested(ctx, r.name, resourceVersion, wrapListFuncWithContext(r.listerWatcher.List), temporaryStore.List)
if err := r.store.Replace(temporaryStore.List(), resourceVersion); err != nil {
return nil, fmt.Errorf("unable to sync watch-list result: %w", err)
@@ -731,6 +774,7 @@ func (r *Reflector) syncWith(items []runtime.Object, resourceVersion string) err
// retry. If successful, the watcher will be left open after receiving the
// initial set of objects, to allow watching for future events.
func handleListWatch(
ctx context.Context,
start time.Time,
w watch.Interface,
store Store,
@@ -741,17 +785,17 @@ func handleListWatch(
setLastSyncResourceVersion func(string),
clock clock.Clock,
errCh chan error,
stopCh <-chan struct{},
) (bool, error) {
exitOnWatchListBookmarkReceived := true
return handleAnyWatch(start, w, store, expectedType, expectedGVK, name, expectedTypeName,
setLastSyncResourceVersion, exitOnWatchListBookmarkReceived, clock, errCh, stopCh)
return handleAnyWatch(ctx, start, w, store, expectedType, expectedGVK, name, expectedTypeName,
setLastSyncResourceVersion, exitOnWatchListBookmarkReceived, clock, errCh)
}
// handleListWatch consumes events from w, updates the Store, and records the
// last seen ResourceVersion, to allow continuing from that ResourceVersion on
// retry. The watcher will always be stopped on exit.
func handleWatch(
ctx context.Context,
start time.Time,
w watch.Interface,
store Store,
@@ -762,11 +806,10 @@ func handleWatch(
setLastSyncResourceVersion func(string),
clock clock.Clock,
errCh chan error,
stopCh <-chan struct{},
) error {
exitOnWatchListBookmarkReceived := false
_, err := handleAnyWatch(start, w, store, expectedType, expectedGVK, name, expectedTypeName,
setLastSyncResourceVersion, exitOnWatchListBookmarkReceived, clock, errCh, stopCh)
_, err := handleAnyWatch(ctx, start, w, store, expectedType, expectedGVK, name, expectedTypeName,
setLastSyncResourceVersion, exitOnWatchListBookmarkReceived, clock, errCh)
return err
}
@@ -779,7 +822,9 @@ func handleWatch(
// The watcher will always be stopped, unless exitOnWatchListBookmarkReceived is
// true and watchListBookmarkReceived is true. This allows the same watch stream
// to be re-used by the caller to continue watching for new events.
func handleAnyWatch(start time.Time,
func handleAnyWatch(
ctx context.Context,
start time.Time,
w watch.Interface,
store Store,
expectedType reflect.Type,
@@ -790,17 +835,17 @@ func handleAnyWatch(start time.Time,
exitOnWatchListBookmarkReceived bool,
clock clock.Clock,
errCh chan error,
stopCh <-chan struct{},
) (bool, error) {
watchListBookmarkReceived := false
eventCount := 0
initialEventsEndBookmarkWarningTicker := newInitialEventsEndBookmarkTicker(name, clock, start, exitOnWatchListBookmarkReceived)
logger := klog.FromContext(ctx)
initialEventsEndBookmarkWarningTicker := newInitialEventsEndBookmarkTicker(logger, name, clock, start, exitOnWatchListBookmarkReceived)
defer initialEventsEndBookmarkWarningTicker.Stop()
loop:
for {
select {
case <-stopCh:
case <-ctx.Done():
return watchListBookmarkReceived, errorStopRequested
case err := <-errCh:
return watchListBookmarkReceived, err
@@ -813,19 +858,19 @@ loop:
}
if expectedType != nil {
if e, a := expectedType, reflect.TypeOf(event.Object); e != a {
utilruntime.HandleError(fmt.Errorf("%s: expected type %v, but watch event object had type %v", name, e, a))
utilruntime.HandleErrorWithContext(ctx, nil, "Unexpected watch event object type", "reflector", name, "expectedType", e, "actualType", a)
continue
}
}
if expectedGVK != nil {
if e, a := *expectedGVK, event.Object.GetObjectKind().GroupVersionKind(); e != a {
utilruntime.HandleError(fmt.Errorf("%s: expected gvk %v, but watch event object had gvk %v", name, e, a))
utilruntime.HandleErrorWithContext(ctx, nil, "Unexpected watch event object gvk", "reflector", name, "expectedGVK", e, "actualGVK", a)
continue
}
}
meta, err := meta.Accessor(event.Object)
if err != nil {
utilruntime.HandleError(fmt.Errorf("%s: unable to understand watch event %#v", name, event))
utilruntime.HandleErrorWithContext(ctx, err, "Unable to understand watch event", "reflector", name, "event", event)
continue
}
resourceVersion := meta.GetResourceVersion()
@@ -833,12 +878,12 @@ loop:
case watch.Added:
err := store.Add(event.Object)
if err != nil {
utilruntime.HandleError(fmt.Errorf("%s: unable to add watch event object (%#v) to store: %v", name, event.Object, err))
utilruntime.HandleErrorWithContext(ctx, err, "Unable to add watch event object to store", "reflector", name, "object", event.Object)
}
case watch.Modified:
err := store.Update(event.Object)
if err != nil {
utilruntime.HandleError(fmt.Errorf("%s: unable to update watch event object (%#v) to store: %v", name, event.Object, err))
utilruntime.HandleErrorWithContext(ctx, err, "Unable to update watch event object to store", "reflector", name, "object", event.Object)
}
case watch.Deleted:
// TODO: Will any consumers need access to the "last known
@@ -846,7 +891,7 @@ loop:
// to change this.
err := store.Delete(event.Object)
if err != nil {
utilruntime.HandleError(fmt.Errorf("%s: unable to delete watch event object (%#v) from store: %v", name, event.Object, err))
utilruntime.HandleErrorWithContext(ctx, err, "Unable to delete watch event object from store", "reflector", name, "object", event.Object)
}
case watch.Bookmark:
// A `Bookmark` means watch has synced here, just update the resourceVersion
@@ -854,7 +899,7 @@ loop:
watchListBookmarkReceived = true
}
default:
utilruntime.HandleError(fmt.Errorf("%s: unable to understand watch event %#v", name, event))
utilruntime.HandleErrorWithContext(ctx, err, "Unknown watch event", "reflector", name, "event", event)
}
setLastSyncResourceVersion(resourceVersion)
if rvu, ok := store.(ResourceVersionUpdater); ok {
@@ -863,7 +908,7 @@ loop:
eventCount++
if exitOnWatchListBookmarkReceived && watchListBookmarkReceived {
watchDuration := clock.Since(start)
klog.V(4).Infof("exiting %v Watch because received the bookmark that marks the end of initial events stream, total %v items received in %v", name, eventCount, watchDuration)
klog.FromContext(ctx).V(4).Info("Exiting watch because received the bookmark that marks the end of initial events stream", "reflector", name, "totalItems", eventCount, "duration", watchDuration)
return watchListBookmarkReceived, nil
}
initialEventsEndBookmarkWarningTicker.observeLastEventTimeStamp(clock.Now())
@@ -876,7 +921,7 @@ loop:
if watchDuration < 1*time.Second && eventCount == 0 {
return watchListBookmarkReceived, fmt.Errorf("very short watch: %s: Unexpected watch close - watch lasted less than a second and no items received", name)
}
klog.V(4).Infof("%s: Watch close - %v total %v items received", name, expectedTypeName, eventCount)
klog.FromContext(ctx).V(4).Info("Watch close", "reflector", name, "type", expectedTypeName, "totalItems", eventCount)
return watchListBookmarkReceived, nil
}
@@ -1004,8 +1049,9 @@ func wrapListFuncWithContext(listFn ListFunc) func(ctx context.Context, options
// The methods exposed by this type are not thread-safe.
type initialEventsEndBookmarkTicker struct {
clock.Ticker
clock clock.Clock
name string
clock clock.Clock
name string
logger klog.Logger
watchStart time.Time
tickInterval time.Duration
@@ -1019,15 +1065,15 @@ type initialEventsEndBookmarkTicker struct {
// Note that the caller controls whether to call t.C() and t.Stop().
//
// In practice, the reflector exits the watchHandler as soon as the bookmark event is received and calls the t.C() method.
func newInitialEventsEndBookmarkTicker(name string, c clock.Clock, watchStart time.Time, exitOnWatchListBookmarkReceived bool) *initialEventsEndBookmarkTicker {
return newInitialEventsEndBookmarkTickerInternal(name, c, watchStart, 10*time.Second, exitOnWatchListBookmarkReceived)
func newInitialEventsEndBookmarkTicker(logger klog.Logger, name string, c clock.Clock, watchStart time.Time, exitOnWatchListBookmarkReceived bool) *initialEventsEndBookmarkTicker {
return newInitialEventsEndBookmarkTickerInternal(logger, name, c, watchStart, 10*time.Second, exitOnWatchListBookmarkReceived)
}
func newInitialEventsEndBookmarkTickerInternal(name string, c clock.Clock, watchStart time.Time, tickInterval time.Duration, exitOnWatchListBookmarkReceived bool) *initialEventsEndBookmarkTicker {
func newInitialEventsEndBookmarkTickerInternal(logger klog.Logger, name string, c clock.Clock, watchStart time.Time, tickInterval time.Duration, exitOnWatchListBookmarkReceived bool) *initialEventsEndBookmarkTicker {
clockWithTicker, ok := c.(clock.WithTicker)
if !ok || !exitOnWatchListBookmarkReceived {
if exitOnWatchListBookmarkReceived {
klog.Warningf("clock does not support WithTicker interface but exitOnInitialEventsEndBookmark was requested")
logger.Info("Warning: clock does not support WithTicker interface but exitOnInitialEventsEndBookmark was requested")
}
return &initialEventsEndBookmarkTicker{
Ticker: &noopTicker{},
@@ -1038,6 +1084,7 @@ func newInitialEventsEndBookmarkTickerInternal(name string, c clock.Clock, watch
Ticker: clockWithTicker.NewTicker(tickInterval),
clock: c,
name: name,
logger: logger,
watchStart: watchStart,
tickInterval: tickInterval,
}
@@ -1049,7 +1096,7 @@ func (t *initialEventsEndBookmarkTicker) observeLastEventTimeStamp(lastEventObse
func (t *initialEventsEndBookmarkTicker) warnIfExpired() {
if err := t.produceWarningIfExpired(); err != nil {
klog.Warning(err)
t.logger.Info("Warning: event bookmark expired", "err", err)
}
}