mirror of
https://github.com/minio/minio.git
synced 2025-11-07 12:52:58 -05:00
Add detailed scanner metrics (#15161)
This commit is contained in:
66
internal/pubsub/mask.go
Normal file
66
internal/pubsub/mask.go
Normal 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
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
|
||||
@@ -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:
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user