WIP: broker batch processing

Signed-off-by: Vasiliy Tolstov <v.tolstov@unistack.org>
This commit is contained in:
Василий Толстов 2021-07-22 22:53:44 +03:00
parent e4673bcc50
commit d357fb1e0d
5 changed files with 461 additions and 26 deletions

View File

@ -8,31 +8,79 @@ import (
"github.com/unistack-org/micro/v3/metadata" "github.com/unistack-org/micro/v3/metadata"
) )
// DefaultBroker default broker // DefaultBroker default memory broker
var DefaultBroker Broker = NewBroker() var DefaultBroker Broker = NewBroker()
var (
// ErrNotConnected returns when broker used but not connected yet
ErrNotConnected = errors.New("broker not connected")
// ErrDisconnected returns when broker disconnected
ErrDisconnected = errors.New("broker disconnected")
)
// Broker is an interface used for asynchronous messaging. // Broker is an interface used for asynchronous messaging.
type Broker interface { type Broker interface {
// Name returns broker instance name
Name() string Name() string
Init(...Option) error // Init initilize broker
Init(opts ...Option) error
// Options returns broker options
Options() Options Options() Options
// Address return configured address
Address() string Address() string
Connect(context.Context) error // Connect connects to broker
Disconnect(context.Context) error Connect(ctx context.Context) error
Publish(context.Context, string, *Message, ...PublishOption) error // Disconnect disconnect from broker
Subscribe(context.Context, string, Handler, ...SubscribeOption) (Subscriber, error) Disconnect(ctx context.Context) error
// Publish message to broker topic
Publish(ctx context.Context, topic string, msg *Message, opts ...PublishOption) error
// BatchPublish messages to broker with multiple topics
BatchPublish(ctx context.Context, msgs []*Message, opts ...PublishOption) error
// Subscribe subscribes to topic message via handler
Subscribe(ctx context.Context, topic string, h Handler, opts ...SubscribeOption) (Subscriber, error)
// BatchSubscribe subscribes to topic messages via handler
BatchSubscribe(ctx context.Context, topic string, h BatchHandler, opts ...SubscribeOption) (Subscriber, error)
// String type of broker
String() string String() string
} }
// Handler is used to process messages via a subscription of a topic. // Handler is used to process messages via a subscription of a topic.
type Handler func(Event) error type Handler func(Event) error
// Events contains multiple events
type Events []Event
func (evs Events) Ack() error {
var err error
for _, ev := range evs {
if err = ev.Ack(); err != nil {
return err
}
}
return nil
}
func (evs Events) SetError(err error) {
for _, ev := range evs {
ev.SetError(err)
}
}
// BatchHandler is used to process messages in batches via a subscription of a topic.
type BatchHandler func(Events) error
// Event is given to a subscription handler for processing // Event is given to a subscription handler for processing
type Event interface { type Event interface {
// Topic returns event topic
Topic() string Topic() string
// Message returns broker message
Message() *Message Message() *Message
// Ack acknowledge message
Ack() error Ack() error
// Error returns message error (like decoding errors or some other)
Error() error Error() error
// SetError set event processing error
SetError(err error)
} }
// RawMessage is a raw encoded JSON value. // RawMessage is a raw encoded JSON value.
@ -58,13 +106,18 @@ func (m *RawMessage) UnmarshalJSON(data []byte) error {
// Message is used to transfer data // Message is used to transfer data
type Message struct { type Message struct {
Header metadata.Metadata // contains message metadata // Header contains message metadata
Body RawMessage // contains message body Header metadata.Metadata
// Body contains message body
Body RawMessage
} }
// Subscriber is a convenience return type for the Subscribe method // Subscriber is a convenience return type for the Subscribe method
type Subscriber interface { type Subscriber interface {
// Options returns subscriber options
Options() SubscribeOptions Options() SubscribeOptions
// Topic returns topic for subscription
Topic() string Topic() string
Unsubscribe(context.Context) error // Unsubscribe from topic
Unsubscribe(ctx context.Context) error
} }

View File

@ -2,7 +2,6 @@ package broker
import ( import (
"context" "context"
"errors"
"sync" "sync"
"github.com/google/uuid" "github.com/google/uuid"
@ -13,9 +12,10 @@ import (
) )
type memoryBroker struct { type memoryBroker struct {
Subscribers map[string][]*memorySubscriber subscribers map[string][]*memorySubscriber
addr string batchsubscribers map[string][]*memoryBatchSubscriber
opts Options addr string
opts Options
sync.RWMutex sync.RWMutex
connected bool connected bool
} }
@ -36,6 +36,15 @@ type memorySubscriber struct {
opts SubscribeOptions opts SubscribeOptions
} }
type memoryBatchSubscriber struct {
ctx context.Context
exit chan bool
handler BatchHandler
id string
topic string
opts SubscribeOptions
}
func (m *memoryBroker) Options() Options { func (m *memoryBroker) Options() Options {
return m.opts return m.opts
} }
@ -77,7 +86,6 @@ func (m *memoryBroker) Disconnect(ctx context.Context) error {
} }
m.connected = false m.connected = false
return nil return nil
} }
@ -88,14 +96,127 @@ func (m *memoryBroker) Init(opts ...Option) error {
return nil return nil
} }
func (m *memoryBroker) BatchPublish(ctx context.Context, msgs []*Message, opts ...PublishOption) error {
m.RLock()
if !m.connected {
m.RUnlock()
return ErrNotConnected
}
m.RUnlock()
type msgWrapper struct {
topic string
body interface{}
}
vs := make([]msgWrapper, 0, len(msgs))
if m.opts.Codec == nil {
m.RLock()
for _, msg := range msgs {
topic, _ := msg.Header.Get("Micro-Topic")
vs = append(vs, msgWrapper{topic: topic, body: m})
}
m.RUnlock()
} else {
m.RLock()
for _, msg := range msgs {
topic, _ := msg.Header.Get("Micro-Topic")
buf, err := m.opts.Codec.Marshal(msg)
if err != nil {
m.RUnlock()
return err
}
vs = append(vs, msgWrapper{topic: topic, body: buf})
}
m.RUnlock()
}
if len(m.batchsubscribers) > 0 {
eh := m.opts.BatchErrorHandler
msgTopicMap := make(map[string]Events)
for _, v := range vs {
p := &memoryEvent{
topic: v.topic,
message: v.body,
opts: m.opts,
}
msgTopicMap[p.topic] = append(msgTopicMap[p.topic], p)
}
for t, ms := range msgTopicMap {
m.RLock()
subs, ok := m.batchsubscribers[t]
m.RUnlock()
if !ok {
continue
}
for _, sub := range subs {
if err := sub.handler(ms); err != nil {
ms.SetError(err)
if sub.opts.BatchErrorHandler != nil {
eh = sub.opts.BatchErrorHandler
}
if eh != nil {
eh(ms)
} else if m.opts.Logger.V(logger.ErrorLevel) {
m.opts.Logger.Error(m.opts.Context, err.Error())
}
} else if sub.opts.AutoAck {
if err := ms.Ack(); err != nil {
m.opts.Logger.Errorf(m.opts.Context, "ack failed: %v", err)
}
}
}
}
}
eh := m.opts.ErrorHandler
for _, v := range vs {
p := &memoryEvent{
topic: v.topic,
message: v.body,
opts: m.opts,
}
m.RLock()
subs, ok := m.subscribers[p.topic]
m.RUnlock()
if !ok {
continue
}
for _, sub := range subs {
if err := sub.handler(p); err != nil {
p.SetError(err)
if sub.opts.ErrorHandler != nil {
eh = sub.opts.ErrorHandler
}
if eh != nil {
eh(p)
} else if m.opts.Logger.V(logger.ErrorLevel) {
m.opts.Logger.Error(m.opts.Context, err.Error())
}
} else if sub.opts.AutoAck {
if err := p.Ack(); err != nil {
m.opts.Logger.Errorf(m.opts.Context, "ack failed: %v", err)
}
}
}
}
return nil
}
func (m *memoryBroker) Publish(ctx context.Context, topic string, msg *Message, opts ...PublishOption) error { func (m *memoryBroker) Publish(ctx context.Context, topic string, msg *Message, opts ...PublishOption) error {
m.RLock() m.RLock()
if !m.connected { if !m.connected {
m.RUnlock() m.RUnlock()
return errors.New("not connected") return ErrNotConnected
} }
subs, ok := m.Subscribers[topic] subs, ok := m.subscribers[topic]
m.RUnlock() m.RUnlock()
if !ok { if !ok {
return nil return nil
@ -138,11 +259,58 @@ func (m *memoryBroker) Publish(ctx context.Context, topic string, msg *Message,
return nil return nil
} }
func (m *memoryBroker) BatchSubscribe(ctx context.Context, topic string, handler BatchHandler, opts ...SubscribeOption) (Subscriber, error) {
m.RLock()
if !m.connected {
m.RUnlock()
return nil, ErrNotConnected
}
m.RUnlock()
options := NewSubscribeOptions(opts...)
id, err := uuid.NewRandom()
if err != nil {
return nil, err
}
sub := &memoryBatchSubscriber{
exit: make(chan bool, 1),
id: id.String(),
topic: topic,
handler: handler,
opts: options,
ctx: ctx,
}
m.Lock()
m.batchsubscribers[topic] = append(m.batchsubscribers[topic], sub)
m.Unlock()
go func() {
<-sub.exit
m.Lock()
var newSubscribers []*memoryBatchSubscriber
for _, sb := range m.batchsubscribers[topic] {
if sb.id == sub.id {
continue
}
newSubscribers = append(newSubscribers, sb)
}
m.batchsubscribers[topic] = newSubscribers
m.Unlock()
}()
return sub, nil
return nil, nil
}
func (m *memoryBroker) Subscribe(ctx context.Context, topic string, handler Handler, opts ...SubscribeOption) (Subscriber, error) { func (m *memoryBroker) Subscribe(ctx context.Context, topic string, handler Handler, opts ...SubscribeOption) (Subscriber, error) {
m.RLock() m.RLock()
if !m.connected { if !m.connected {
m.RUnlock() m.RUnlock()
return nil, errors.New("not connected") return nil, ErrNotConnected
} }
m.RUnlock() m.RUnlock()
@ -163,20 +331,20 @@ func (m *memoryBroker) Subscribe(ctx context.Context, topic string, handler Hand
} }
m.Lock() m.Lock()
m.Subscribers[topic] = append(m.Subscribers[topic], sub) m.subscribers[topic] = append(m.subscribers[topic], sub)
m.Unlock() m.Unlock()
go func() { go func() {
<-sub.exit <-sub.exit
m.Lock() m.Lock()
var newSubscribers []*memorySubscriber var newSubscribers []*memorySubscriber
for _, sb := range m.Subscribers[topic] { for _, sb := range m.subscribers[topic] {
if sb.id == sub.id { if sb.id == sub.id {
continue continue
} }
newSubscribers = append(newSubscribers, sb) newSubscribers = append(newSubscribers, sb)
} }
m.Subscribers[topic] = newSubscribers m.subscribers[topic] = newSubscribers
m.Unlock() m.Unlock()
}() }()
@ -221,6 +389,23 @@ func (m *memoryEvent) Error() error {
return m.err return m.err
} }
func (m *memoryEvent) SetError(err error) {
m.err = err
}
func (m *memoryBatchSubscriber) Options() SubscribeOptions {
return m.opts
}
func (m *memoryBatchSubscriber) Topic() string {
return m.topic
}
func (m *memoryBatchSubscriber) Unsubscribe(ctx context.Context) error {
m.exit <- true
return nil
}
func (m *memorySubscriber) Options() SubscribeOptions { func (m *memorySubscriber) Options() SubscribeOptions {
return m.opts return m.opts
} }
@ -237,7 +422,8 @@ func (m *memorySubscriber) Unsubscribe(ctx context.Context) error {
// NewBroker return new memory broker // NewBroker return new memory broker
func NewBroker(opts ...Option) Broker { func NewBroker(opts ...Option) Broker {
return &memoryBroker{ return &memoryBroker{
opts: NewOptions(opts...), opts: NewOptions(opts...),
Subscribers: make(map[string][]*memorySubscriber), subscribers: make(map[string][]*memorySubscriber),
batchsubscribers: make(map[string][]*memoryBatchSubscriber),
} }
} }

View File

@ -6,6 +6,51 @@ import (
"testing" "testing"
) )
func TestMemoryBatchBroker(t *testing.T) {
b := NewBroker()
ctx := context.Background()
if err := b.Connect(ctx); err != nil {
t.Fatalf("Unexpected connect error %v", err)
}
topic := "test"
count := 10
fn := func(evts Events) error {
return evts.Ack()
}
sub, err := b.BatchSubscribe(ctx, topic, fn)
if err != nil {
t.Fatalf("Unexpected error subscribing %v", err)
}
msgs := make([]*Message, 0, 0)
for i := 0; i < count; i++ {
message := &Message{
Header: map[string]string{
"Micro-Topic": topic,
"foo": "bar",
"id": fmt.Sprintf("%d", i),
},
Body: []byte(`"hello world"`),
}
msgs = append(msgs, message)
}
if err := b.BatchPublish(ctx, msgs); err != nil {
t.Fatalf("Unexpected error publishing %v", err)
}
if err := sub.Unsubscribe(ctx); err != nil {
t.Fatalf("Unexpected error unsubscribing from %s: %v", topic, err)
}
if err := b.Disconnect(ctx); err != nil {
t.Fatalf("Unexpected connect error %v", err)
}
}
func TestMemoryBroker(t *testing.T) { func TestMemoryBroker(t *testing.T) {
b := NewBroker() b := NewBroker()
ctx := context.Background() ctx := context.Background()
@ -26,20 +71,27 @@ func TestMemoryBroker(t *testing.T) {
t.Fatalf("Unexpected error subscribing %v", err) t.Fatalf("Unexpected error subscribing %v", err)
} }
msgs := make([]*Message, 0, 0)
for i := 0; i < count; i++ { for i := 0; i < count; i++ {
message := &Message{ message := &Message{
Header: map[string]string{ Header: map[string]string{
"foo": "bar", "Micro-Topic": topic,
"id": fmt.Sprintf("%d", i), "foo": "bar",
"id": fmt.Sprintf("%d", i),
}, },
Body: []byte(`"hello world"`), Body: []byte(`"hello world"`),
} }
msgs = append(msgs, message)
if err := b.Publish(ctx, topic, message); err != nil { if err := b.Publish(ctx, topic, message); err != nil {
t.Fatalf("Unexpected error publishing %d", i) t.Fatalf("Unexpected error publishing %d err: %v", i, err)
} }
} }
if err := b.BatchPublish(ctx, msgs); err != nil {
t.Fatalf("Unexpected error publishing %v", err)
}
if err := sub.Unsubscribe(ctx); err != nil { if err := sub.Unsubscribe(ctx); err != nil {
t.Fatalf("Unexpected error unsubscribing from %s: %v", topic, err) t.Fatalf("Unexpected error unsubscribing from %s: %v", topic, err)
} }

View File

@ -29,6 +29,8 @@ type Options struct {
TLSConfig *tls.Config TLSConfig *tls.Config
// ErrorHandler used when broker can't unmarshal incoming message // ErrorHandler used when broker can't unmarshal incoming message
ErrorHandler Handler ErrorHandler Handler
// BatchErrorHandler used when broker can't unmashal incoming messages
BatchErrorHandler BatchHandler
// Name holds the broker name // Name holds the broker name
Name string Name string
// Addrs holds the broker address // Addrs holds the broker address
@ -85,6 +87,8 @@ type SubscribeOptions struct {
Context context.Context Context context.Context
// ErrorHandler used when broker can't unmarshal incoming message // ErrorHandler used when broker can't unmarshal incoming message
ErrorHandler Handler ErrorHandler Handler
// BatchErrorHandler used when broker can't unmashal incoming messages
BatchErrorHandler BatchHandler
// Group holds consumer group // Group holds consumer group
Group string Group string
// AutoAck flag specifies auto ack of incoming message when no error happens // AutoAck flag specifies auto ack of incoming message when no error happens
@ -175,6 +179,14 @@ func ErrorHandler(h Handler) Option {
} }
} }
// BatchErrorHandler will catch all broker errors that cant be handled
// in normal way, for example Codec errors
func BatchErrorHandler(h BatchHandler) Option {
return func(o *Options) {
o.BatchErrorHandler = h
}
}
// SubscribeErrorHandler will catch all broker errors that cant be handled // SubscribeErrorHandler will catch all broker errors that cant be handled
// in normal way, for example Codec errors // in normal way, for example Codec errors
func SubscribeErrorHandler(h Handler) SubscribeOption { func SubscribeErrorHandler(h Handler) SubscribeOption {
@ -183,6 +195,14 @@ func SubscribeErrorHandler(h Handler) SubscribeOption {
} }
} }
// SubscribeBatchErrorHandler will catch all broker errors that cant be handled
// in normal way, for example Codec errors
func SubscribeBatchErrorHandler(h BatchHandler) SubscribeOption {
return func(o *SubscribeOptions) {
o.BatchErrorHandler = h
}
}
// Queue sets the subscribers queue // Queue sets the subscribers queue
// Deprecated // Deprecated
func Queue(name string) SubscribeOption { func Queue(name string) SubscribeOption {

View File

@ -24,9 +24,18 @@ type tunSubscriber struct {
opts broker.SubscribeOptions opts broker.SubscribeOptions
} }
type tunBatchSubscriber struct {
listener tunnel.Listener
handler broker.BatchHandler
closed chan bool
topic string
opts broker.SubscribeOptions
}
type tunEvent struct { type tunEvent struct {
message *broker.Message message *broker.Message
topic string topic string
err error
} }
// used to access tunnel from options context // used to access tunnel from options context
@ -62,6 +71,36 @@ func (t *tunBroker) Disconnect(ctx context.Context) error {
return t.tunnel.Close(ctx) return t.tunnel.Close(ctx)
} }
func (t *tunBroker) BatchPublish(ctx context.Context, msgs []*broker.Message, opts ...broker.PublishOption) error {
// TODO: this is probably inefficient, we might want to just maintain an open connection
// it may be easier to add broadcast to the tunnel
topicMap := make(map[string]tunnel.Session)
var err error
for _, msg := range msgs {
topic, _ := msg.Header.Get("Micro-Topic")
c, ok := topicMap[topic]
if !ok {
c, err := t.tunnel.Dial(ctx, topic, tunnel.DialMode(tunnel.Multicast))
if err != nil {
return err
}
defer c.Close()
topicMap[topic] = c
}
if err = c.Send(&transport.Message{
Header: msg.Header,
Body: msg.Body,
}); err != nil {
// msg.SetError(err)
return err
}
}
return nil
}
func (t *tunBroker) Publish(ctx context.Context, topic string, m *broker.Message, opts ...broker.PublishOption) error { func (t *tunBroker) Publish(ctx context.Context, topic string, m *broker.Message, opts ...broker.PublishOption) error {
// TODO: this is probably inefficient, we might want to just maintain an open connection // TODO: this is probably inefficient, we might want to just maintain an open connection
// it may be easier to add broadcast to the tunnel // it may be easier to add broadcast to the tunnel
@ -77,6 +116,26 @@ func (t *tunBroker) Publish(ctx context.Context, topic string, m *broker.Message
}) })
} }
func (t *tunBroker) BatchSubscribe(ctx context.Context, topic string, h broker.BatchHandler, opts ...broker.SubscribeOption) (broker.Subscriber, error) {
l, err := t.tunnel.Listen(ctx, topic, tunnel.ListenMode(tunnel.Multicast))
if err != nil {
return nil, err
}
tunSub := &tunBatchSubscriber{
topic: topic,
handler: h,
opts: broker.NewSubscribeOptions(opts...),
closed: make(chan bool),
listener: l,
}
// start processing
go tunSub.run()
return tunSub, nil
}
func (t *tunBroker) Subscribe(ctx context.Context, topic string, h broker.Handler, opts ...broker.SubscribeOption) (broker.Subscriber, error) { func (t *tunBroker) Subscribe(ctx context.Context, topic string, h broker.Handler, opts ...broker.SubscribeOption) (broker.Subscriber, error) {
l, err := t.tunnel.Listen(ctx, topic, tunnel.ListenMode(tunnel.Multicast)) l, err := t.tunnel.Listen(ctx, topic, tunnel.ListenMode(tunnel.Multicast))
if err != nil { if err != nil {
@ -101,6 +160,49 @@ func (t *tunBroker) String() string {
return "tunnel" return "tunnel"
} }
func (t *tunBatchSubscriber) run() {
for {
// accept a new connection
c, err := t.listener.Accept()
if err != nil {
select {
case <-t.closed:
return
default:
continue
}
}
// receive message
m := new(transport.Message)
if err := c.Recv(m); err != nil {
if logger.V(logger.ErrorLevel) {
logger.Error(t.opts.Context, err.Error())
}
if err = c.Close(); err != nil {
if logger.V(logger.ErrorLevel) {
logger.Error(t.opts.Context, err.Error())
}
}
continue
}
// close the connection
c.Close()
evts := broker.Events{&tunEvent{
topic: t.topic,
message: &broker.Message{
Header: m.Header,
Body: m.Body,
},
}}
// handle the message
go t.handler(evts)
}
}
func (t *tunSubscriber) run() { func (t *tunSubscriber) run() {
for { for {
// accept a new connection // accept a new connection
@ -142,6 +244,24 @@ func (t *tunSubscriber) run() {
} }
} }
func (t *tunBatchSubscriber) Options() broker.SubscribeOptions {
return t.opts
}
func (t *tunBatchSubscriber) Topic() string {
return t.topic
}
func (t *tunBatchSubscriber) Unsubscribe(ctx context.Context) error {
select {
case <-t.closed:
return nil
default:
close(t.closed)
return t.listener.Close()
}
}
func (t *tunSubscriber) Options() broker.SubscribeOptions { func (t *tunSubscriber) Options() broker.SubscribeOptions {
return t.opts return t.opts
} }
@ -173,7 +293,11 @@ func (t *tunEvent) Ack() error {
} }
func (t *tunEvent) Error() error { func (t *tunEvent) Error() error {
return nil return t.err
}
func (t *tunEvent) SetError(err error) {
t.err = err
} }
// NewBroker returns new tunnel broker // NewBroker returns new tunnel broker