1
0
mirror of https://github.com/mainflux/mainflux.git synced 2025-05-01 13:48:56 +08:00
b1ackd0t 824156fbf0
NOISSUE - Use Nats JetStream As Internal Message Bus (#1893)
* 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>
2023-10-18 17:00:38 +02:00

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)
}