Enable propagration of HasSynced

* Add tracker types and tests
* Modify ResourceEventHandler interface's OnAdd member
* Add additional ResourceEventHandlerDetailedFuncs struct
* Fix SharedInformer to let users track HasSynced for their handlers
* Fix in-tree controllers which weren't computing HasSynced correctly
* Deprecate the cache.Pop function

Kubernetes-commit: 8100efc7b3122ad119ee8fa4bbbedef3b90f2e0d
This commit is contained in:
Daniel Smith
2022-11-18 00:12:50 +00:00
committed by Kubernetes Publisher
parent e7e7d01afd
commit 5d70a118df
11 changed files with 524 additions and 52 deletions

116
tools/cache/synctrack/synctrack.go vendored Normal file
View File

@@ -0,0 +1,116 @@
/*
Copyright 2022 The Kubernetes 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 synctrack contains utilities for helping controllers track whether
// they are "synced" or not, that is, whether they have processed all items
// from the informer's initial list.
package synctrack
import (
"sync"
"sync/atomic"
"k8s.io/apimachinery/pkg/util/sets"
)
// AsyncTracker helps propagate HasSynced in the face of multiple worker threads.
type AsyncTracker[T comparable] struct {
UpstreamHasSynced func() bool
lock sync.Mutex
waiting sets.Set[T]
}
// Start should be called prior to processing each key which is part of the
// initial list.
func (t *AsyncTracker[T]) Start(key T) {
t.lock.Lock()
defer t.lock.Unlock()
if t.waiting == nil {
t.waiting = sets.New[T](key)
} else {
t.waiting.Insert(key)
}
}
// Finished should be called when finished processing a key which was part of
// the initial list. Since keys are tracked individually, nothing bad happens
// if you call Finished without a corresponding call to Start. This makes it
// easier to use this in combination with e.g. queues which don't make it easy
// to plumb through the isInInitialList boolean.
func (t *AsyncTracker[T]) Finished(key T) {
t.lock.Lock()
defer t.lock.Unlock()
if t.waiting != nil {
t.waiting.Delete(key)
}
}
// HasSynced returns true if the source is synced and every key present in the
// initial list has been processed. This relies on the source not considering
// itself synced until *after* it has delivered the notification for the last
// key, and that notification handler must have called Start.
func (t *AsyncTracker[T]) HasSynced() bool {
// Call UpstreamHasSynced first: it might take a lock, which might take
// a significant amount of time, and we can't hold our lock while
// waiting on that or a user is likely to get a deadlock.
if !t.UpstreamHasSynced() {
return false
}
t.lock.Lock()
defer t.lock.Unlock()
return t.waiting.Len() == 0
}
// SingleFileTracker helps propagate HasSynced when events are processed in
// order (i.e. via a queue).
type SingleFileTracker struct {
UpstreamHasSynced func() bool
count int64
}
// Start should be called prior to processing each key which is part of the
// initial list.
func (t *SingleFileTracker) Start() {
atomic.AddInt64(&t.count, 1)
}
// Finished should be called when finished processing a key which was part of
// the initial list. You must never call Finished() before (or without) its
// corresponding Start(), that is a logic error that could cause HasSynced to
// return a wrong value. To help you notice this should it happen, Finished()
// will panic if the internal counter goes negative.
func (t *SingleFileTracker) Finished() {
result := atomic.AddInt64(&t.count, -1)
if result < 0 {
panic("synctrack: negative counter; this logic error means HasSynced may return incorrect value")
}
}
// HasSynced returns true if the source is synced and every key present in the
// initial list has been processed. This relies on the source not considering
// itself synced until *after* it has delivered the notification for the last
// key, and that notification handler must have called Start.
func (t *SingleFileTracker) HasSynced() bool {
// Call UpstreamHasSynced first: it might take a lock, which might take
// a significant amount of time, and we don't want to then act on a
// stale count value.
if !t.UpstreamHasSynced() {
return false
}
return atomic.LoadInt64(&t.count) <= 0
}

239
tools/cache/synctrack/synctrack_test.go vendored Normal file
View File

@@ -0,0 +1,239 @@
/*
Copyright 2022 The Kubernetes 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 synctrack
import (
"strings"
"sync"
"time"
"testing"
)
func testSingleFileFuncs(upstreamHasSynced func() bool) (start func(), finished func(), hasSynced func() bool) {
tracker := SingleFileTracker{
UpstreamHasSynced: upstreamHasSynced,
}
return tracker.Start, tracker.Finished, tracker.HasSynced
}
func testAsyncFuncs(upstreamHasSynced func() bool) (start func(), finished func(), hasSynced func() bool) {
tracker := AsyncTracker[string]{
UpstreamHasSynced: upstreamHasSynced,
}
return func() { tracker.Start("key") }, func() { tracker.Finished("key") }, tracker.HasSynced
}
func TestBasicLogic(t *testing.T) {
table := []struct {
name string
construct func(func() bool) (func(), func(), func() bool)
}{
{"SingleFile", testSingleFileFuncs},
{"Async", testAsyncFuncs},
}
for _, entry := range table {
t.Run(entry.name, func(t *testing.T) {
table := []struct {
synced bool
start bool
finish bool
expectSynced bool
}{
{false, true, true, false},
{true, true, false, false},
{false, true, false, false},
{true, true, true, true},
}
for _, tt := range table {
Start, Finished, HasSynced := entry.construct(func() bool { return tt.synced })
if tt.start {
Start()
}
if tt.finish {
Finished()
}
got := HasSynced()
if e, a := tt.expectSynced, got; e != a {
t.Errorf("for %#v got %v (wanted %v)", tt, a, e)
}
}
})
}
}
func TestAsyncLocking(t *testing.T) {
aft := AsyncTracker[int]{UpstreamHasSynced: func() bool { return true }}
var wg sync.WaitGroup
for _, i := range []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10} {
wg.Add(1)
go func(i int) {
aft.Start(i)
go func() {
aft.Finished(i)
wg.Done()
}()
}(i)
}
wg.Wait()
if !aft.HasSynced() {
t.Errorf("async tracker must have made a threading error?")
}
}
func TestSingleFileCounting(t *testing.T) {
sft := SingleFileTracker{UpstreamHasSynced: func() bool { return true }}
for i := 0; i < 100; i++ {
sft.Start()
}
if sft.HasSynced() {
t.Fatal("Unexpectedly synced?")
}
for i := 0; i < 99; i++ {
sft.Finished()
}
if sft.HasSynced() {
t.Fatal("Unexpectedly synced?")
}
sft.Finished()
if !sft.HasSynced() {
t.Fatal("Unexpectedly not synced?")
}
// Calling an extra time will panic.
func() {
defer func() {
x := recover()
if x == nil {
t.Error("no panic?")
return
}
msg, ok := x.(string)
if !ok {
t.Errorf("unexpected panic value: %v", x)
return
}
if !strings.Contains(msg, "negative counter") {
t.Errorf("unexpected panic message: %v", msg)
return
}
}()
sft.Finished()
}()
// Negative counter still means it is synced
if !sft.HasSynced() {
t.Fatal("Unexpectedly not synced?")
}
}
func TestSingleFile(t *testing.T) {
table := []struct {
synced bool
starts int
stops int
expectSynced bool
}{
{false, 1, 1, false},
{true, 1, 0, false},
{false, 1, 0, false},
{true, 1, 1, true},
}
for _, tt := range table {
sft := SingleFileTracker{UpstreamHasSynced: func() bool { return tt.synced }}
for i := 0; i < tt.starts; i++ {
sft.Start()
}
for i := 0; i < tt.stops; i++ {
sft.Finished()
}
got := sft.HasSynced()
if e, a := tt.expectSynced, got; e != a {
t.Errorf("for %#v got %v (wanted %v)", tt, a, e)
}
}
}
func TestNoStaleValue(t *testing.T) {
table := []struct {
name string
construct func(func() bool) (func(), func(), func() bool)
}{
{"SingleFile", testSingleFileFuncs},
{"Async", testAsyncFuncs},
}
for _, entry := range table {
t.Run(entry.name, func(t *testing.T) {
var lock sync.Mutex
upstreamHasSynced := func() bool {
lock.Lock()
defer lock.Unlock()
return true
}
Start, Finished, HasSynced := entry.construct(upstreamHasSynced)
// Ordinarily the corresponding lock would be held and you wouldn't be
// able to call this function at this point.
if !HasSynced() {
t.Fatal("Unexpectedly not synced??")
}
Start()
if HasSynced() {
t.Fatal("Unexpectedly synced??")
}
Finished()
if !HasSynced() {
t.Fatal("Unexpectedly not synced??")
}
// Now we will prove that if the lock is held, you can't get a false
// HasSynced return.
lock.Lock()
// This goroutine calls HasSynced
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
if HasSynced() {
t.Error("Unexpectedly synced??")
}
}()
// This goroutine increments + unlocks. The sleep is to bias the
// runtime such that the other goroutine usually wins (it needs to work
// in both orderings, this one is more likely to be buggy).
go func() {
time.Sleep(time.Millisecond)
Start()
lock.Unlock()
}()
wg.Wait()
})
}
}