Add detailed scanner metrics (#15161)

This commit is contained in:
Klaus Post
2022-07-05 14:45:49 -07:00
committed by GitHub
parent df42914da6
commit ac055b09e9
55 changed files with 1735 additions and 1753 deletions

66
internal/pubsub/mask.go Normal file
View File

@@ -0,0 +1,66 @@
package pubsub
import (
"math"
"math/bits"
)
// Mask allows filtering by a bitset mask.
type Mask uint64
const (
// MaskAll is the mask for all entries.
MaskAll Mask = math.MaxUint64
)
// MaskFromMaskable extracts mask from an interface.
func MaskFromMaskable(m Maskable) Mask {
return Mask(m.Mask())
}
// Contains returns whether *all* flags in other is present in t.
func (t Mask) Contains(other Mask) bool {
return t&other == other
}
// Overlaps returns whether *any* flags in t overlaps with other.
func (t Mask) Overlaps(other Mask) bool {
return t&other != 0
}
// SingleType returns whether t has a single type set.
func (t Mask) SingleType() bool {
return bits.OnesCount64(uint64(t)) == 1
}
// FromUint64 will set a mask to the uint64 value.
func (t *Mask) FromUint64(m uint64) {
*t = Mask(m)
}
// Merge will merge other into t.
func (t *Mask) Merge(other Mask) {
*t |= other
}
// MergeMaskable will merge other into t.
func (t *Mask) MergeMaskable(other Maskable) {
*t |= Mask(other.Mask())
}
// SetIf will add other if b is true.
func (t *Mask) SetIf(b bool, other Mask) {
if b {
*t |= other
}
}
// Mask returns the mask as a uint64.
func (t Mask) Mask() uint64 {
return uint64(t)
}
// Maskable implementations must return their mask as a 64 bit uint.
type Maskable interface {
Mask() uint64
}

View File

@@ -25,27 +25,31 @@ import (
// Sub - subscriber entity.
type Sub struct {
ch chan interface{}
filter func(entry interface{}) bool
ch chan Maskable
types Mask
filter func(entry Maskable) bool
}
// PubSub holds publishers and subscribers
type PubSub struct {
subs []*Sub
// atomics, keep at top:
types uint64
numSubscribers int32
maxSubscribers int32
// not atomics:
subs []*Sub
sync.RWMutex
}
// Publish message to the subscribers.
// Note that publish is always nob-blocking send so that we don't block on slow receivers.
// Hence receivers should use buffered channel so as not to miss the published events.
func (ps *PubSub) Publish(item interface{}) {
func (ps *PubSub) Publish(item Maskable) {
ps.RLock()
defer ps.RUnlock()
for _, sub := range ps.subs {
if sub.filter == nil || sub.filter(item) {
if sub.types.Contains(Mask(item.Mask())) && (sub.filter == nil || sub.filter(item)) {
select {
case sub.ch <- item:
default:
@@ -55,38 +59,54 @@ func (ps *PubSub) Publish(item interface{}) {
}
// Subscribe - Adds a subscriber to pubsub system
func (ps *PubSub) Subscribe(subCh chan interface{}, doneCh <-chan struct{}, filter func(entry interface{}) bool) error {
func (ps *PubSub) Subscribe(mask Mask, subCh chan Maskable, doneCh <-chan struct{}, filter func(entry Maskable) bool) error {
totalSubs := atomic.AddInt32(&ps.numSubscribers, 1)
if ps.maxSubscribers > 0 && totalSubs > ps.maxSubscribers {
atomic.AddInt32(&ps.numSubscribers, -1)
return fmt.Errorf("the limit of `%d` subscribers is reached", ps.maxSubscribers)
}
ps.Lock()
defer ps.Unlock()
sub := &Sub{subCh, filter}
sub := &Sub{ch: subCh, types: mask, filter: filter}
ps.subs = append(ps.subs, sub)
// We hold a lock, so we are safe to update
combined := Mask(atomic.LoadUint64(&ps.types))
combined.Merge(mask)
atomic.StoreUint64(&ps.types, uint64(combined))
go func() {
<-doneCh
ps.Lock()
defer ps.Unlock()
var remainTypes Mask
for i, s := range ps.subs {
if s == sub {
ps.subs = append(ps.subs[:i], ps.subs[i+1:]...)
} else {
remainTypes.Merge(s.types)
}
}
atomic.StoreUint64(&ps.types, uint64(remainTypes))
atomic.AddInt32(&ps.numSubscribers, -1)
}()
return nil
}
// NumSubscribers returns the number of current subscribers
func (ps *PubSub) NumSubscribers() int32 {
// NumSubscribers returns the number of current subscribers,
// If t is non-nil, the type is checked against the active subscribed types,
// and 0 will be returned if nobody is subscribed for the type,
// otherwise the *total* number of subscribers is returned.
func (ps *PubSub) NumSubscribers(m Maskable) int32 {
if m != nil {
types := Mask(atomic.LoadUint64(&ps.types))
if !types.Overlaps(Mask(m.Mask())) {
return 0
}
}
return atomic.LoadInt32(&ps.numSubscribers)
}

View File

@@ -25,51 +25,87 @@ import (
func TestSubscribe(t *testing.T) {
ps := New(2)
ch1 := make(chan interface{}, 1)
ch2 := make(chan interface{}, 1)
ch1 := make(chan Maskable, 1)
ch2 := make(chan Maskable, 1)
doneCh := make(chan struct{})
defer close(doneCh)
if err := ps.Subscribe(ch1, doneCh, nil); err != nil {
if err := ps.Subscribe(MaskAll, ch1, doneCh, nil); err != nil {
t.Fatalf("unexpected error: %v", err)
}
if err := ps.Subscribe(ch2, doneCh, nil); err != nil {
if err := ps.Subscribe(MaskAll, ch2, doneCh, nil); err != nil {
t.Fatalf("unexpected error: %v", err)
}
ps.Lock()
defer ps.Unlock()
if len(ps.subs) != 2 || ps.NumSubscribers(nil) != 2 {
t.Fatalf("expected 2 subscribers")
}
}
func TestNumSubscribersMask(t *testing.T) {
ps := New(2)
ch1 := make(chan Maskable, 1)
ch2 := make(chan Maskable, 1)
doneCh := make(chan struct{})
defer close(doneCh)
if err := ps.Subscribe(1, ch1, doneCh, nil); err != nil {
t.Fatalf("unexpected error: %v", err)
}
if err := ps.Subscribe(2, ch2, doneCh, nil); err != nil {
t.Fatalf("unexpected error: %v", err)
}
ps.Lock()
defer ps.Unlock()
if len(ps.subs) != 2 {
t.Fatalf("expected 2 subscribers")
}
if want, got := int32(2), ps.NumSubscribers(Mask(1)); got != want {
t.Fatalf("want %d subscribers, got %d", want, got)
}
if want, got := int32(2), ps.NumSubscribers(Mask(2)); got != want {
t.Fatalf("want %d subscribers, got %d", want, got)
}
if want, got := int32(2), ps.NumSubscribers(Mask(1|2)); got != want {
t.Fatalf("want %d subscribers, got %d", want, got)
}
if want, got := int32(2), ps.NumSubscribers(nil); got != want {
t.Fatalf("want %d subscribers, got %d", want, got)
}
if want, got := int32(0), ps.NumSubscribers(Mask(4)); got != want {
t.Fatalf("want %d subscribers, got %d", want, got)
}
}
func TestSubscribeExceedingLimit(t *testing.T) {
ps := New(2)
ch1 := make(chan interface{}, 1)
ch2 := make(chan interface{}, 1)
ch3 := make(chan interface{}, 1)
ch1 := make(chan Maskable, 1)
ch2 := make(chan Maskable, 1)
ch3 := make(chan Maskable, 1)
doneCh := make(chan struct{})
defer close(doneCh)
if err := ps.Subscribe(ch1, doneCh, nil); err != nil {
if err := ps.Subscribe(MaskAll, ch1, doneCh, nil); err != nil {
t.Fatalf("unexpected error: %v", err)
}
if err := ps.Subscribe(ch2, doneCh, nil); err != nil {
if err := ps.Subscribe(MaskAll, ch2, doneCh, nil); err != nil {
t.Fatalf("unexpected error: %v", err)
}
if err := ps.Subscribe(ch3, doneCh, nil); err == nil {
if err := ps.Subscribe(MaskAll, ch3, doneCh, nil); err == nil {
t.Fatalf("unexpected nil err")
}
}
func TestUnsubscribe(t *testing.T) {
ps := New(2)
ch1 := make(chan interface{}, 1)
ch2 := make(chan interface{}, 1)
ch1 := make(chan Maskable, 1)
ch2 := make(chan Maskable, 1)
doneCh1 := make(chan struct{})
doneCh2 := make(chan struct{})
if err := ps.Subscribe(ch1, doneCh1, nil); err != nil {
if err := ps.Subscribe(MaskAll, ch1, doneCh1, nil); err != nil {
t.Fatalf("unexpected error: %v", err)
}
if err := ps.Subscribe(ch2, doneCh2, nil); err != nil {
if err := ps.Subscribe(MaskAll, ch2, doneCh2, nil); err != nil {
t.Fatalf("unexpected error: %v", err)
}
@@ -84,40 +120,81 @@ func TestUnsubscribe(t *testing.T) {
close(doneCh2)
}
type maskString string
func (m maskString) Mask() uint64 {
return 1
}
func TestPubSub(t *testing.T) {
ps := New(1)
ch1 := make(chan interface{}, 1)
ch1 := make(chan Maskable, 1)
doneCh1 := make(chan struct{})
defer close(doneCh1)
if err := ps.Subscribe(ch1, doneCh1, func(entry interface{}) bool { return true }); err != nil {
if err := ps.Subscribe(MaskAll, ch1, doneCh1, func(entry Maskable) bool { return true }); err != nil {
t.Fatalf("unexpected error: %v", err)
}
val := "hello"
val := maskString("hello")
ps.Publish(val)
msg := <-ch1
if msg != "hello" {
if msg != val {
t.Fatalf(fmt.Sprintf("expected %s , found %s", val, msg))
}
}
func TestMultiPubSub(t *testing.T) {
ps := New(2)
ch1 := make(chan interface{}, 1)
ch2 := make(chan interface{}, 1)
ch1 := make(chan Maskable, 1)
ch2 := make(chan Maskable, 1)
doneCh := make(chan struct{})
defer close(doneCh)
if err := ps.Subscribe(ch1, doneCh, func(entry interface{}) bool { return true }); err != nil {
if err := ps.Subscribe(MaskAll, ch1, doneCh, func(entry Maskable) bool { return true }); err != nil {
t.Fatalf("unexpected error: %v", err)
}
if err := ps.Subscribe(ch2, doneCh, func(entry interface{}) bool { return true }); err != nil {
if err := ps.Subscribe(MaskAll, ch2, doneCh, func(entry Maskable) bool { return true }); err != nil {
t.Fatalf("unexpected error: %v", err)
}
val := "hello"
val := maskString("hello")
ps.Publish(val)
msg1 := <-ch1
msg2 := <-ch2
if msg1 != "hello" && msg2 != "hello" {
if msg1 != val && msg2 != val {
t.Fatalf(fmt.Sprintf("expected both subscribers to have%s , found %s and %s", val, msg1, msg2))
}
}
func TestMultiPubSubMask(t *testing.T) {
ps := New(3)
ch1 := make(chan Maskable, 1)
ch2 := make(chan Maskable, 1)
ch3 := make(chan Maskable, 1)
doneCh := make(chan struct{})
defer close(doneCh)
// Mask matches maskString, should get result
if err := ps.Subscribe(Mask(1), ch1, doneCh, func(entry Maskable) bool { return true }); err != nil {
t.Fatalf("unexpected error: %v", err)
}
// Mask matches maskString, should get result
if err := ps.Subscribe(Mask(1|2), ch2, doneCh, func(entry Maskable) bool { return true }); err != nil {
t.Fatalf("unexpected error: %v", err)
}
// Does NOT overlap maskString
if err := ps.Subscribe(Mask(2), ch3, doneCh, func(entry Maskable) bool { return true }); err != nil {
t.Fatalf("unexpected error: %v", err)
}
val := maskString("hello")
ps.Publish(val)
msg1 := <-ch1
msg2 := <-ch2
if msg1 != val && msg2 != val {
t.Fatalf(fmt.Sprintf("expected both subscribers to have%s , found %s and %s", val, msg1, msg2))
}
select {
case msg := <-ch3:
t.Fatalf(fmt.Sprintf("unexpect msg, f got %s", msg))
default:
}
}