mirror of
https://github.com/mainflux/mainflux.git
synced 2025-05-01 13:48:56 +08:00

* Replace Nats with Nats Jestream For PubSub Signed-off-by: rodneyosodo <blackd0t@protonmail.com> * Add Stream Description Signed-off-by: rodneyosodo <blackd0t@protonmail.com> * Subscribe using wildcard Signed-off-by: rodneyosodo <blackd0t@protonmail.com> * Add consumers description Signed-off-by: rodneyosodo <blackd0t@protonmail.com> * Remove unused queue variable Signed-off-by: rodneyosodo <blackd0t@protonmail.com> * Add extra configs to stream Signed-off-by: Rodney Osodo <socials@rodneyosodo.com> * Use inline error handling Signed-off-by: Rodney Osodo <socials@rodneyosodo.com> * Fix connection leak in NATS publisher The publisher struct in pkg/messaging/nats/publisher.go was modified to include a new `conn` field of type `*broker.Conn`. This change was made to fix a connection leak issue in the NATS publisher. The `NewPublisher` function was updated to assign the `conn` parameter to the new `conn` field in the publisher struct. Additionally, the `Close` method in the publisher struct was modified to close the `conn` connection. This commit fixes the connection leak issue in the NATS publisher and ensures that connections are properly closed. Signed-off-by: Rodney Osodo <socials@rodneyosodo.com> * feat(messaging): Add support for durable consumers This commit adds support for durable consumers to NATS JS in the messaging package. To support this functionality, the `strings` package has been imported in the `pubsub.go` file to check the topic. Signed-off-by: Rodney Osodo <28790446+rodneyosodo@users.noreply.github.com> * feat : remove internal logic to keep subscribers Signed-off-by: Rodney Osodo <28790446+rodneyosodo@users.noreply.github.com> * feat(messaging): add function to format consumer name This commit adds a new function to the `pubsub` package in the `messaging` module. The function is called `formatConsumerName` and is used to generate a consumer name for NATS messaging. The function takes two parameters, `topic` and `id`, and returns a formatted consumer name. The consumer name is generated by concatenating the `topic` and `id` parameters, with some restrictions on the characters that can be used. This function will be useful for creating durable subscriptions in NATS messaging. Signed-off-by: Rodney Osodo <28790446+rodneyosodo@users.noreply.github.com> --------- Signed-off-by: rodneyosodo <blackd0t@protonmail.com> Signed-off-by: Rodney Osodo <socials@rodneyosodo.com> Signed-off-by: Rodney Osodo <28790446+rodneyosodo@users.noreply.github.com>
161 lines
4.0 KiB
Go
161 lines
4.0 KiB
Go
// Copyright (c) Mainflux
|
|
// SPDX-License-Identifier: Apache-2.0
|
|
|
|
package nats
|
|
|
|
import (
|
|
"context"
|
|
"errors"
|
|
"fmt"
|
|
"strings"
|
|
"time"
|
|
|
|
mflog "github.com/mainflux/mainflux/logger"
|
|
"github.com/mainflux/mainflux/pkg/messaging"
|
|
broker "github.com/nats-io/nats.go"
|
|
"github.com/nats-io/nats.go/jetstream"
|
|
"google.golang.org/protobuf/proto"
|
|
)
|
|
|
|
const chansPrefix = "channels"
|
|
|
|
// Publisher and Subscriber errors.
|
|
var (
|
|
ErrNotSubscribed = errors.New("not subscribed")
|
|
ErrEmptyTopic = errors.New("empty topic")
|
|
ErrEmptyID = errors.New("empty id")
|
|
|
|
jsStreamConfig = jetstream.StreamConfig{
|
|
Name: "channels",
|
|
Description: "Mainflux stream for sending and receiving messages in between Mainflux channels",
|
|
Subjects: []string{"channels.>"},
|
|
Retention: jetstream.LimitsPolicy,
|
|
MaxMsgsPerSubject: 1e6,
|
|
MaxAge: time.Hour * 24,
|
|
MaxMsgSize: 1024 * 1024,
|
|
Discard: jetstream.DiscardOld,
|
|
Storage: jetstream.FileStorage,
|
|
}
|
|
)
|
|
|
|
var _ messaging.PubSub = (*pubsub)(nil)
|
|
|
|
type pubsub struct {
|
|
publisher
|
|
logger mflog.Logger
|
|
stream jetstream.Stream
|
|
}
|
|
|
|
// NewPubSub returns NATS message publisher/subscriber.
|
|
// Parameter queue specifies the queue for the Subscribe method.
|
|
// If queue is specified (is not an empty string), Subscribe method
|
|
// will execute NATS QueueSubscribe which is conceptually different
|
|
// from ordinary subscribe. For more information, please take a look
|
|
// here: https://docs.nats.io/developing-with-nats/receiving/queues.
|
|
// If the queue is empty, Subscribe will be used.
|
|
func NewPubSub(ctx context.Context, url string, logger mflog.Logger) (messaging.PubSub, error) {
|
|
conn, err := broker.Connect(url, broker.MaxReconnects(maxReconnects))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
js, err := jetstream.New(conn)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
stream, err := js.CreateStream(ctx, jsStreamConfig)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
ret := &pubsub{
|
|
publisher: publisher{
|
|
js: js,
|
|
conn: conn,
|
|
},
|
|
stream: stream,
|
|
logger: logger,
|
|
}
|
|
|
|
return ret, nil
|
|
}
|
|
|
|
func (ps *pubsub) Subscribe(ctx context.Context, id, topic string, handler messaging.MessageHandler) error {
|
|
if id == "" {
|
|
return ErrEmptyID
|
|
}
|
|
if topic == "" {
|
|
return ErrEmptyTopic
|
|
}
|
|
|
|
nh := ps.natsHandler(handler)
|
|
|
|
consumerConfig := jetstream.ConsumerConfig{
|
|
Name: formatConsumerName(topic, id),
|
|
Durable: formatConsumerName(topic, id),
|
|
Description: fmt.Sprintf("Mainflux consumer of id %s for topic %s", id, topic),
|
|
DeliverPolicy: jetstream.DeliverNewPolicy,
|
|
FilterSubject: topic,
|
|
}
|
|
|
|
consumer, err := ps.stream.CreateOrUpdateConsumer(ctx, consumerConfig)
|
|
if err != nil {
|
|
return fmt.Errorf("failed to create consumer: %w", err)
|
|
}
|
|
|
|
if _, err = consumer.Consume(nh); err != nil {
|
|
return fmt.Errorf("failed to consume: %w", err)
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (ps *pubsub) Unsubscribe(ctx context.Context, id, topic string) error {
|
|
if id == "" {
|
|
return ErrEmptyID
|
|
}
|
|
if topic == "" {
|
|
return ErrEmptyTopic
|
|
}
|
|
|
|
err := ps.stream.DeleteConsumer(ctx, formatConsumerName(topic, id))
|
|
switch {
|
|
case errors.Is(err, jetstream.ErrConsumerNotFound):
|
|
return ErrNotSubscribed
|
|
default:
|
|
return err
|
|
}
|
|
}
|
|
|
|
func (ps *pubsub) natsHandler(h messaging.MessageHandler) func(m jetstream.Msg) {
|
|
return func(m jetstream.Msg) {
|
|
var msg messaging.Message
|
|
if err := proto.Unmarshal(m.Data(), &msg); err != nil {
|
|
ps.logger.Warn(fmt.Sprintf("Failed to unmarshal received message: %s", err))
|
|
|
|
return
|
|
}
|
|
|
|
if err := h.Handle(&msg); err != nil {
|
|
ps.logger.Warn(fmt.Sprintf("Failed to handle Mainflux message: %s", err))
|
|
}
|
|
if err := m.Ack(); err != nil {
|
|
ps.logger.Warn(fmt.Sprintf("Failed to ack message: %s", err))
|
|
}
|
|
}
|
|
}
|
|
|
|
func formatConsumerName(topic, id string) string {
|
|
// A durable name cannot contain whitespace, ., *, >, path separators (forward or backwards slash), and non-printable characters.
|
|
chars := []string{
|
|
" ", "_",
|
|
".", "_",
|
|
"*", "_",
|
|
">", "_",
|
|
"/", "_",
|
|
"\\", "_",
|
|
}
|
|
topic = strings.NewReplacer(chars...).Replace(topic)
|
|
|
|
return fmt.Sprintf("%s-%s", topic, id)
|
|
}
|