mirror of
https://github.com/minio/minio.git
synced 2025-11-07 12:52:58 -05:00
Improve tracing & notification scalability (#18903)
* Perform JSON encoding on remote machines and only forward byte slices. * Migrate tracing & notification to WebSockets.
This commit is contained in:
@@ -18,11 +18,18 @@
|
||||
package pubsub
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
)
|
||||
|
||||
// GetByteBuffer returns a byte buffer from the pool.
|
||||
var GetByteBuffer = func() []byte {
|
||||
return make([]byte, 0, 4096)
|
||||
}
|
||||
|
||||
// Sub - subscriber entity.
|
||||
type Sub[T Maskable] struct {
|
||||
ch chan T
|
||||
@@ -96,6 +103,62 @@ func (ps *PubSub[T, M]) Subscribe(mask M, subCh chan T, doneCh <-chan struct{},
|
||||
return nil
|
||||
}
|
||||
|
||||
// SubscribeJSON - Adds a subscriber to pubsub system and returns results with JSON encoding.
|
||||
func (ps *PubSub[T, M]) SubscribeJSON(mask M, subCh chan<- []byte, doneCh <-chan struct{}, filter func(entry T) 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()
|
||||
subChT := make(chan T, 10000)
|
||||
sub := &Sub[T]{ch: subChT, types: Mask(mask.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(mask.Mask()))
|
||||
atomic.StoreUint64(&ps.types, uint64(combined))
|
||||
|
||||
go func() {
|
||||
var buf bytes.Buffer
|
||||
enc := json.NewEncoder(&buf)
|
||||
for {
|
||||
select {
|
||||
case <-doneCh:
|
||||
case v, ok := <-subChT:
|
||||
if !ok {
|
||||
break
|
||||
}
|
||||
buf.Reset()
|
||||
err := enc.Encode(v)
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
subCh <- append(GetByteBuffer()[:0], buf.Bytes()...)
|
||||
continue
|
||||
}
|
||||
break
|
||||
}
|
||||
|
||||
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,
|
||||
// The mask is checked against the active subscribed types,
|
||||
// and 0 will be returned if nobody is subscribed for the type(s).
|
||||
|
||||
Reference in New Issue
Block a user