Add NATS JetStream support (#1866)

* Add NATS JetStream support
Update shopify/sarama

* Fix addresses

* Don't change Addresses in Defaults

* Update saramajetstream

* Add missing error check

Keep typing events for at least one minute

* Use all configured NATS addresses

* Update saramajetstream

* Try setting up with NATS

* Make sure NATS uses own persistent directory (TODO: make this configurable)

* Update go.mod/go.sum

* Jetstream package

* Various other refactoring

* Build fixes

* Config tweaks, make random jetstream storage path for CI

* Disable interest policies

* Try to sane default on jetstream base path

* Try to use in-memory for CI

* Restore storage/retention

* Update nats.go dependency

* Adapt changes to config

* Remove unneeded TopicFor

* Dep update

* Revert "Remove unneeded TopicFor"

This reverts commit f5a4e4a339.

* Revert changes made to streams

* Fix build problems

* Update nats-server

* Update go.mod/go.sum

* Roomserver input API queuing using NATS

* Fix topic naming

* Prometheus metrics

* More refactoring to remove saramajetstream

* Add missing topic

* Don't try to populate map that doesn't exist

* Roomserver output topic

* Update go.mod/go.sum

* Message acknowledgements

* Ack tweaks

* Try to resume transaction re-sends

* Try to resume transaction re-sends

* Update to matrix-org/gomatrixserverlib@91dadfb

* Remove internal.PartitionStorer from components that don't consume keychanges

* Try to reduce re-allocations a bit in resolveConflictsV2

* Tweak delivery options on RS input

* Publish send-to-device messages into correct JetStream subject

* Async and sync roomserver input

* Update dendrite-config.yaml

* Remove roomserver tests for now (they need rewriting)

* Remove roomserver test again (was merged back in)

* Update documentation

* Docker updates

* More Docker updates

* Update Docker readme again

* Fix lint issues

* Send final event in `processEvent` synchronously (since this might stop Sytest from being so upset)

* Don't report event rejection errors via `/send`, since apparently this is upsetting tests that don't expect that

* Go 1.16 instead of Go 1.13 for upgrade tests and Complement

* Revert "Don't report event rejection errors via `/send`, since apparently this is upsetting tests that don't expect that"

This reverts commit 368675283f.

* Don't report any errors on `/send` to see what fun that creates

* Fix panics on closed channel sends

* Enforce state key matches sender

* Do the same for leave

* Various tweaks to make tests happier

Squashed commit of the following:

commit 13f9028e7a
Author: Neil Alexander <neilalexander@users.noreply.github.com>
Date:   Tue Jan 4 15:47:14 2022 +0000

    Do the same for leave

commit e6be7f05c3
Author: Neil Alexander <neilalexander@users.noreply.github.com>
Date:   Tue Jan 4 15:33:42 2022 +0000

    Enforce state key matches sender

commit 85ede6d64b
Author: Neil Alexander <neilalexander@users.noreply.github.com>
Date:   Tue Jan 4 14:07:04 2022 +0000

    Fix panics on closed channel sends

commit 9755494a98
Author: Neil Alexander <neilalexander@users.noreply.github.com>
Date:   Tue Jan 4 13:38:22 2022 +0000

    Don't report any errors on `/send` to see what fun that creates

commit 3bb4f87b5d
Author: Neil Alexander <neilalexander@users.noreply.github.com>
Date:   Tue Jan 4 13:00:26 2022 +0000

    Revert "Don't report event rejection errors via `/send`, since apparently this is upsetting tests that don't expect that"

    This reverts commit 368675283f.

commit fe2673ed7b
Author: Neil Alexander <neilalexander@users.noreply.github.com>
Date:   Tue Jan 4 12:09:34 2022 +0000

    Go 1.16 instead of Go 1.13 for upgrade tests and Complement

commit 368675283f
Author: Neil Alexander <neilalexander@users.noreply.github.com>
Date:   Tue Jan 4 11:51:45 2022 +0000

    Don't report event rejection errors via `/send`, since apparently this is upsetting tests that don't expect that

commit b028dfc085
Author: Neil Alexander <neilalexander@users.noreply.github.com>
Date:   Tue Jan 4 10:29:08 2022 +0000

    Send final event in `processEvent` synchronously (since this might stop Sytest from being so upset)

* Merge in NATS Server v2.6.6 and nats.go v1.13 into the in-process connection fork

* Add `jetstream.WithJetStreamMessage` to make ack/nak-ing less messy, use process context in consumers

* Fix consumer component name in  federation API

* Add comment explaining where streams are defined

* Tweaks to roomserver input with comments

* Finish that sentence that I apparently forgot to finish in INSTALL.md

* Bump version number of config to 2

* Add comments around asynchronous sends to roomserver in processEventWithMissingState

* More useful error message when the config version does not match

* Set version in generate-config

* Fix version in config.Defaults

Co-authored-by: Neil Alexander <neilalexander@users.noreply.github.com>
This commit is contained in:
S7evinK 2022-01-05 18:44:49 +01:00 committed by GitHub
parent a47b12dc7d
commit 161f145176
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
75 changed files with 1317 additions and 1696 deletions

View file

@ -18,90 +18,88 @@ import (
"context"
"encoding/json"
"github.com/Shopify/sarama"
"github.com/getsentry/sentry-go"
"github.com/matrix-org/dendrite/internal"
"github.com/matrix-org/dendrite/internal/eventutil"
"github.com/matrix-org/dendrite/setup/config"
"github.com/matrix-org/dendrite/setup/jetstream"
"github.com/matrix-org/dendrite/setup/process"
"github.com/matrix-org/dendrite/syncapi/notifier"
"github.com/matrix-org/dendrite/syncapi/storage"
"github.com/matrix-org/dendrite/syncapi/types"
"github.com/nats-io/nats.go"
log "github.com/sirupsen/logrus"
)
// OutputClientDataConsumer consumes events that originated in the client API server.
type OutputClientDataConsumer struct {
clientAPIConsumer *internal.ContinualConsumer
db storage.Database
stream types.StreamProvider
notifier *notifier.Notifier
ctx context.Context
jetstream nats.JetStreamContext
topic string
db storage.Database
stream types.StreamProvider
notifier *notifier.Notifier
}
// NewOutputClientDataConsumer creates a new OutputClientData consumer. Call Start() to begin consuming from room servers.
func NewOutputClientDataConsumer(
process *process.ProcessContext,
cfg *config.SyncAPI,
kafkaConsumer sarama.Consumer,
js nats.JetStreamContext,
store storage.Database,
notifier *notifier.Notifier,
stream types.StreamProvider,
) *OutputClientDataConsumer {
consumer := internal.ContinualConsumer{
Process: process,
ComponentName: "syncapi/clientapi",
Topic: string(cfg.Matrix.Kafka.TopicFor(config.TopicOutputClientData)),
Consumer: kafkaConsumer,
PartitionStore: store,
return &OutputClientDataConsumer{
ctx: process.Context(),
jetstream: js,
topic: cfg.Matrix.JetStream.TopicFor(jetstream.OutputClientData),
db: store,
notifier: notifier,
stream: stream,
}
s := &OutputClientDataConsumer{
clientAPIConsumer: &consumer,
db: store,
notifier: notifier,
stream: stream,
}
consumer.ProcessMessage = s.onMessage
return s
}
// Start consuming from room servers
func (s *OutputClientDataConsumer) Start() error {
return s.clientAPIConsumer.Start()
_, err := s.jetstream.Subscribe(s.topic, s.onMessage)
return err
}
// onMessage is called when the sync server receives a new event from the client API server output log.
// It is not safe for this function to be called from multiple goroutines, or else the
// sync stream position may race and be incorrectly calculated.
func (s *OutputClientDataConsumer) onMessage(msg *sarama.ConsumerMessage) error {
// Parse out the event JSON
var output eventutil.AccountData
if err := json.Unmarshal(msg.Value, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("client API server output log: message parse failure")
sentry.CaptureException(err)
return nil
}
func (s *OutputClientDataConsumer) onMessage(msg *nats.Msg) {
jetstream.WithJetStreamMessage(msg, func(msg *nats.Msg) bool {
// Parse out the event JSON
userID := msg.Header.Get(jetstream.UserID)
var output eventutil.AccountData
if err := json.Unmarshal(msg.Data, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("client API server output log: message parse failure")
sentry.CaptureException(err)
return true
}
log.WithFields(log.Fields{
"type": output.Type,
"room_id": output.RoomID,
}).Info("received data from client API server")
streamPos, err := s.db.UpsertAccountData(
context.TODO(), string(msg.Key), output.RoomID, output.Type,
)
if err != nil {
sentry.CaptureException(err)
log.WithFields(log.Fields{
"type": output.Type,
"room_id": output.RoomID,
log.ErrorKey: err,
}).Panicf("could not save account data")
}
"type": output.Type,
"room_id": output.RoomID,
}).Info("received data from client API server")
s.stream.Advance(streamPos)
s.notifier.OnNewAccountData(string(msg.Key), types.StreamingToken{AccountDataPosition: streamPos})
streamPos, err := s.db.UpsertAccountData(
s.ctx, userID, output.RoomID, output.Type,
)
if err != nil {
sentry.CaptureException(err)
log.WithFields(log.Fields{
"type": output.Type,
"room_id": output.RoomID,
log.ErrorKey: err,
}).Panicf("could not save account data")
}
return nil
s.stream.Advance(streamPos)
s.notifier.OnNewAccountData(userID, types.StreamingToken{AccountDataPosition: streamPos})
return true
})
}

View file

@ -18,24 +18,26 @@ import (
"context"
"encoding/json"
"github.com/Shopify/sarama"
"github.com/getsentry/sentry-go"
"github.com/matrix-org/dendrite/eduserver/api"
"github.com/matrix-org/dendrite/internal"
"github.com/matrix-org/dendrite/setup/config"
"github.com/matrix-org/dendrite/setup/jetstream"
"github.com/matrix-org/dendrite/setup/process"
"github.com/matrix-org/dendrite/syncapi/notifier"
"github.com/matrix-org/dendrite/syncapi/storage"
"github.com/matrix-org/dendrite/syncapi/types"
"github.com/nats-io/nats.go"
log "github.com/sirupsen/logrus"
)
// OutputReceiptEventConsumer consumes events that originated in the EDU server.
type OutputReceiptEventConsumer struct {
receiptConsumer *internal.ContinualConsumer
db storage.Database
stream types.StreamProvider
notifier *notifier.Notifier
ctx context.Context
jetstream nats.JetStreamContext
topic string
db storage.Database
stream types.StreamProvider
notifier *notifier.Notifier
}
// NewOutputReceiptEventConsumer creates a new OutputReceiptEventConsumer.
@ -43,61 +45,53 @@ type OutputReceiptEventConsumer struct {
func NewOutputReceiptEventConsumer(
process *process.ProcessContext,
cfg *config.SyncAPI,
kafkaConsumer sarama.Consumer,
js nats.JetStreamContext,
store storage.Database,
notifier *notifier.Notifier,
stream types.StreamProvider,
) *OutputReceiptEventConsumer {
consumer := internal.ContinualConsumer{
Process: process,
ComponentName: "syncapi/eduserver/receipt",
Topic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputReceiptEvent),
Consumer: kafkaConsumer,
PartitionStore: store,
return &OutputReceiptEventConsumer{
ctx: process.Context(),
jetstream: js,
topic: cfg.Matrix.JetStream.TopicFor(jetstream.OutputReceiptEvent),
db: store,
notifier: notifier,
stream: stream,
}
s := &OutputReceiptEventConsumer{
receiptConsumer: &consumer,
db: store,
notifier: notifier,
stream: stream,
}
consumer.ProcessMessage = s.onMessage
return s
}
// Start consuming from EDU api
func (s *OutputReceiptEventConsumer) Start() error {
return s.receiptConsumer.Start()
_, err := s.jetstream.Subscribe(s.topic, s.onMessage)
return err
}
func (s *OutputReceiptEventConsumer) onMessage(msg *sarama.ConsumerMessage) error {
var output api.OutputReceiptEvent
if err := json.Unmarshal(msg.Value, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("EDU server output log: message parse failure")
sentry.CaptureException(err)
return nil
}
func (s *OutputReceiptEventConsumer) onMessage(msg *nats.Msg) {
jetstream.WithJetStreamMessage(msg, func(msg *nats.Msg) bool {
var output api.OutputReceiptEvent
if err := json.Unmarshal(msg.Data, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("EDU server output log: message parse failure")
sentry.CaptureException(err)
return true
}
streamPos, err := s.db.StoreReceipt(
context.TODO(),
output.RoomID,
output.Type,
output.UserID,
output.EventID,
output.Timestamp,
)
if err != nil {
sentry.CaptureException(err)
return err
}
streamPos, err := s.db.StoreReceipt(
s.ctx,
output.RoomID,
output.Type,
output.UserID,
output.EventID,
output.Timestamp,
)
if err != nil {
sentry.CaptureException(err)
return true
}
s.stream.Advance(streamPos)
s.notifier.OnNewReceipt(output.RoomID, types.StreamingToken{ReceiptPosition: streamPos})
s.stream.Advance(streamPos)
s.notifier.OnNewReceipt(output.RoomID, types.StreamingToken{ReceiptPosition: streamPos})
return nil
return true
})
}

View file

@ -18,27 +18,29 @@ import (
"context"
"encoding/json"
"github.com/Shopify/sarama"
"github.com/getsentry/sentry-go"
"github.com/matrix-org/dendrite/eduserver/api"
"github.com/matrix-org/dendrite/internal"
"github.com/matrix-org/dendrite/setup/config"
"github.com/matrix-org/dendrite/setup/jetstream"
"github.com/matrix-org/dendrite/setup/process"
"github.com/matrix-org/dendrite/syncapi/notifier"
"github.com/matrix-org/dendrite/syncapi/storage"
"github.com/matrix-org/dendrite/syncapi/types"
"github.com/matrix-org/gomatrixserverlib"
"github.com/matrix-org/util"
"github.com/nats-io/nats.go"
log "github.com/sirupsen/logrus"
)
// OutputSendToDeviceEventConsumer consumes events that originated in the EDU server.
type OutputSendToDeviceEventConsumer struct {
sendToDeviceConsumer *internal.ContinualConsumer
db storage.Database
serverName gomatrixserverlib.ServerName // our server name
stream types.StreamProvider
notifier *notifier.Notifier
ctx context.Context
jetstream nats.JetStreamContext
topic string
db storage.Database
serverName gomatrixserverlib.ServerName // our server name
stream types.StreamProvider
notifier *notifier.Notifier
}
// NewOutputSendToDeviceEventConsumer creates a new OutputSendToDeviceEventConsumer.
@ -46,78 +48,70 @@ type OutputSendToDeviceEventConsumer struct {
func NewOutputSendToDeviceEventConsumer(
process *process.ProcessContext,
cfg *config.SyncAPI,
kafkaConsumer sarama.Consumer,
js nats.JetStreamContext,
store storage.Database,
notifier *notifier.Notifier,
stream types.StreamProvider,
) *OutputSendToDeviceEventConsumer {
consumer := internal.ContinualConsumer{
Process: process,
ComponentName: "syncapi/eduserver/sendtodevice",
Topic: string(cfg.Matrix.Kafka.TopicFor(config.TopicOutputSendToDeviceEvent)),
Consumer: kafkaConsumer,
PartitionStore: store,
return &OutputSendToDeviceEventConsumer{
ctx: process.Context(),
jetstream: js,
topic: cfg.Matrix.JetStream.TopicFor(jetstream.OutputSendToDeviceEvent),
db: store,
serverName: cfg.Matrix.ServerName,
notifier: notifier,
stream: stream,
}
s := &OutputSendToDeviceEventConsumer{
sendToDeviceConsumer: &consumer,
db: store,
serverName: cfg.Matrix.ServerName,
notifier: notifier,
stream: stream,
}
consumer.ProcessMessage = s.onMessage
return s
}
// Start consuming from EDU api
func (s *OutputSendToDeviceEventConsumer) Start() error {
return s.sendToDeviceConsumer.Start()
_, err := s.jetstream.Subscribe(s.topic, s.onMessage)
return err
}
func (s *OutputSendToDeviceEventConsumer) onMessage(msg *sarama.ConsumerMessage) error {
var output api.OutputSendToDeviceEvent
if err := json.Unmarshal(msg.Value, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("EDU server output log: message parse failure")
sentry.CaptureException(err)
return err
}
func (s *OutputSendToDeviceEventConsumer) onMessage(msg *nats.Msg) {
jetstream.WithJetStreamMessage(msg, func(msg *nats.Msg) bool {
var output api.OutputSendToDeviceEvent
if err := json.Unmarshal(msg.Data, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("EDU server output log: message parse failure")
sentry.CaptureException(err)
return true
}
_, domain, err := gomatrixserverlib.SplitID('@', output.UserID)
if err != nil {
sentry.CaptureException(err)
return err
}
if domain != s.serverName {
return nil
}
_, domain, err := gomatrixserverlib.SplitID('@', output.UserID)
if err != nil {
sentry.CaptureException(err)
return true
}
if domain != s.serverName {
return true
}
util.GetLogger(context.TODO()).WithFields(log.Fields{
"sender": output.Sender,
"user_id": output.UserID,
"device_id": output.DeviceID,
"event_type": output.Type,
}).Info("sync API received send-to-device event from EDU server")
util.GetLogger(context.TODO()).WithFields(log.Fields{
"sender": output.Sender,
"user_id": output.UserID,
"device_id": output.DeviceID,
"event_type": output.Type,
}).Info("sync API received send-to-device event from EDU server")
streamPos, err := s.db.StoreNewSendForDeviceMessage(
context.TODO(), output.UserID, output.DeviceID, output.SendToDeviceEvent,
)
if err != nil {
sentry.CaptureException(err)
log.WithError(err).Errorf("failed to store send-to-device message")
return err
}
streamPos, err := s.db.StoreNewSendForDeviceMessage(
s.ctx, output.UserID, output.DeviceID, output.SendToDeviceEvent,
)
if err != nil {
sentry.CaptureException(err)
log.WithError(err).Errorf("failed to store send-to-device message")
return false
}
s.stream.Advance(streamPos)
s.notifier.OnNewSendToDevice(
output.UserID,
[]string{output.DeviceID},
types.StreamingToken{SendToDevicePosition: streamPos},
)
s.stream.Advance(streamPos)
s.notifier.OnNewSendToDevice(
output.UserID,
[]string{output.DeviceID},
types.StreamingToken{SendToDevicePosition: streamPos},
)
return nil
return true
})
}

View file

@ -15,27 +15,30 @@
package consumers
import (
"context"
"encoding/json"
"github.com/Shopify/sarama"
"github.com/getsentry/sentry-go"
"github.com/matrix-org/dendrite/eduserver/api"
"github.com/matrix-org/dendrite/eduserver/cache"
"github.com/matrix-org/dendrite/internal"
"github.com/matrix-org/dendrite/setup/config"
"github.com/matrix-org/dendrite/setup/jetstream"
"github.com/matrix-org/dendrite/setup/process"
"github.com/matrix-org/dendrite/syncapi/notifier"
"github.com/matrix-org/dendrite/syncapi/storage"
"github.com/matrix-org/dendrite/syncapi/types"
"github.com/nats-io/nats.go"
log "github.com/sirupsen/logrus"
)
// OutputTypingEventConsumer consumes events that originated in the EDU server.
type OutputTypingEventConsumer struct {
typingConsumer *internal.ContinualConsumer
eduCache *cache.EDUCache
stream types.StreamProvider
notifier *notifier.Notifier
ctx context.Context
jetstream nats.JetStreamContext
topic string
eduCache *cache.EDUCache
stream types.StreamProvider
notifier *notifier.Notifier
}
// NewOutputTypingEventConsumer creates a new OutputTypingEventConsumer.
@ -43,72 +46,59 @@ type OutputTypingEventConsumer struct {
func NewOutputTypingEventConsumer(
process *process.ProcessContext,
cfg *config.SyncAPI,
kafkaConsumer sarama.Consumer,
js nats.JetStreamContext,
store storage.Database,
eduCache *cache.EDUCache,
notifier *notifier.Notifier,
stream types.StreamProvider,
) *OutputTypingEventConsumer {
consumer := internal.ContinualConsumer{
Process: process,
ComponentName: "syncapi/eduserver/typing",
Topic: string(cfg.Matrix.Kafka.TopicFor(config.TopicOutputTypingEvent)),
Consumer: kafkaConsumer,
PartitionStore: store,
return &OutputTypingEventConsumer{
ctx: process.Context(),
jetstream: js,
topic: cfg.Matrix.JetStream.TopicFor(jetstream.OutputTypingEvent),
eduCache: eduCache,
notifier: notifier,
stream: stream,
}
s := &OutputTypingEventConsumer{
typingConsumer: &consumer,
eduCache: eduCache,
notifier: notifier,
stream: stream,
}
consumer.ProcessMessage = s.onMessage
return s
}
// Start consuming from EDU api
func (s *OutputTypingEventConsumer) Start() error {
s.eduCache.SetTimeoutCallback(func(userID, roomID string, latestSyncPosition int64) {
pos := types.StreamPosition(latestSyncPosition)
s.stream.Advance(pos)
s.notifier.OnNewTyping(roomID, types.StreamingToken{TypingPosition: pos})
_, err := s.jetstream.Subscribe(s.topic, s.onMessage)
return err
}
func (s *OutputTypingEventConsumer) onMessage(msg *nats.Msg) {
jetstream.WithJetStreamMessage(msg, func(msg *nats.Msg) bool {
var output api.OutputTypingEvent
if err := json.Unmarshal(msg.Data, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("EDU server output log: message parse failure")
sentry.CaptureException(err)
return true
}
log.WithFields(log.Fields{
"room_id": output.Event.RoomID,
"user_id": output.Event.UserID,
"typing": output.Event.Typing,
}).Debug("received data from EDU server")
var typingPos types.StreamPosition
typingEvent := output.Event
if typingEvent.Typing {
typingPos = types.StreamPosition(
s.eduCache.AddTypingUser(typingEvent.UserID, typingEvent.RoomID, output.ExpireTime),
)
} else {
typingPos = types.StreamPosition(
s.eduCache.RemoveUser(typingEvent.UserID, typingEvent.RoomID),
)
}
s.stream.Advance(typingPos)
s.notifier.OnNewTyping(output.Event.RoomID, types.StreamingToken{TypingPosition: typingPos})
return true
})
return s.typingConsumer.Start()
}
func (s *OutputTypingEventConsumer) onMessage(msg *sarama.ConsumerMessage) error {
var output api.OutputTypingEvent
if err := json.Unmarshal(msg.Value, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("EDU server output log: message parse failure")
sentry.CaptureException(err)
return nil
}
log.WithFields(log.Fields{
"room_id": output.Event.RoomID,
"user_id": output.Event.UserID,
"typing": output.Event.Typing,
}).Debug("received data from EDU server")
var typingPos types.StreamPosition
typingEvent := output.Event
if typingEvent.Typing {
typingPos = types.StreamPosition(
s.eduCache.AddTypingUser(typingEvent.UserID, typingEvent.RoomID, output.ExpireTime),
)
} else {
typingPos = types.StreamPosition(
s.eduCache.RemoveUser(typingEvent.UserID, typingEvent.RoomID),
)
}
s.stream.Advance(typingPos)
s.notifier.OnNewTyping(output.Event.RoomID, types.StreamingToken{TypingPosition: typingPos})
return nil
}

View file

@ -34,6 +34,7 @@ import (
// OutputKeyChangeEventConsumer consumes events that originated in the key server.
type OutputKeyChangeEventConsumer struct {
ctx context.Context
keyChangeConsumer *internal.ContinualConsumer
db storage.Database
notifier *notifier.Notifier
@ -68,6 +69,7 @@ func NewOutputKeyChangeEventConsumer(
}
s := &OutputKeyChangeEventConsumer{
ctx: process.Context(),
keyChangeConsumer: &consumer,
db: store,
serverName: serverName,
@ -125,10 +127,13 @@ func (s *OutputKeyChangeEventConsumer) onMessage(msg *sarama.ConsumerMessage) er
}
func (s *OutputKeyChangeEventConsumer) onDeviceKeyMessage(m api.DeviceMessage, offset int64, partition int32) error {
if m.DeviceKeys == nil {
return nil
}
output := m.DeviceKeys
// work out who we need to notify about the new key
var queryRes roomserverAPI.QuerySharedUsersResponse
err := s.rsAPI.QuerySharedUsers(context.Background(), &roomserverAPI.QuerySharedUsersRequest{
err := s.rsAPI.QuerySharedUsers(s.ctx, &roomserverAPI.QuerySharedUsersRequest{
UserID: output.UserID,
}, &queryRes)
if err != nil {
@ -155,7 +160,7 @@ func (s *OutputKeyChangeEventConsumer) onCrossSigningMessage(m api.DeviceMessage
output := m.CrossSigningKeyUpdate
// work out who we need to notify about the new key
var queryRes roomserverAPI.QuerySharedUsersResponse
err := s.rsAPI.QuerySharedUsers(context.Background(), &roomserverAPI.QuerySharedUsersRequest{
err := s.rsAPI.QuerySharedUsers(s.ctx, &roomserverAPI.QuerySharedUsersRequest{
UserID: output.UserID,
}, &queryRes)
if err != nil {

View file

@ -19,24 +19,26 @@ import (
"encoding/json"
"fmt"
"github.com/Shopify/sarama"
"github.com/getsentry/sentry-go"
"github.com/matrix-org/dendrite/internal"
"github.com/matrix-org/dendrite/roomserver/api"
"github.com/matrix-org/dendrite/setup/config"
"github.com/matrix-org/dendrite/setup/jetstream"
"github.com/matrix-org/dendrite/setup/process"
"github.com/matrix-org/dendrite/syncapi/notifier"
"github.com/matrix-org/dendrite/syncapi/storage"
"github.com/matrix-org/dendrite/syncapi/types"
"github.com/matrix-org/gomatrixserverlib"
"github.com/nats-io/nats.go"
log "github.com/sirupsen/logrus"
)
// OutputRoomEventConsumer consumes events that originated in the room server.
type OutputRoomEventConsumer struct {
ctx context.Context
cfg *config.SyncAPI
rsAPI api.RoomserverInternalAPI
rsConsumer *internal.ContinualConsumer
jetstream nats.JetStreamContext
topic string
db storage.Database
pduStream types.StreamProvider
inviteStream types.StreamProvider
@ -47,83 +49,83 @@ type OutputRoomEventConsumer struct {
func NewOutputRoomEventConsumer(
process *process.ProcessContext,
cfg *config.SyncAPI,
kafkaConsumer sarama.Consumer,
js nats.JetStreamContext,
store storage.Database,
notifier *notifier.Notifier,
pduStream types.StreamProvider,
inviteStream types.StreamProvider,
rsAPI api.RoomserverInternalAPI,
) *OutputRoomEventConsumer {
consumer := internal.ContinualConsumer{
Process: process,
ComponentName: "syncapi/roomserver",
Topic: string(cfg.Matrix.Kafka.TopicFor(config.TopicOutputRoomEvent)),
Consumer: kafkaConsumer,
PartitionStore: store,
}
s := &OutputRoomEventConsumer{
return &OutputRoomEventConsumer{
ctx: process.Context(),
cfg: cfg,
rsConsumer: &consumer,
jetstream: js,
topic: cfg.Matrix.JetStream.TopicFor(jetstream.OutputRoomEvent),
db: store,
notifier: notifier,
pduStream: pduStream,
inviteStream: inviteStream,
rsAPI: rsAPI,
}
consumer.ProcessMessage = s.onMessage
return s
}
// Start consuming from room servers
func (s *OutputRoomEventConsumer) Start() error {
return s.rsConsumer.Start()
_, err := s.jetstream.Subscribe(s.topic, s.onMessage)
return err
}
// onMessage is called when the sync server receives a new event from the room server output log.
// It is not safe for this function to be called from multiple goroutines, or else the
// sync stream position may race and be incorrectly calculated.
func (s *OutputRoomEventConsumer) onMessage(msg *sarama.ConsumerMessage) error {
// Parse out the event JSON
var output api.OutputEvent
if err := json.Unmarshal(msg.Value, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("roomserver output log: message parse failure")
return nil
}
switch output.Type {
case api.OutputTypeNewRoomEvent:
// Ignore redaction events. We will add them to the database when they are
// validated (when we receive OutputTypeRedactedEvent)
event := output.NewRoomEvent.Event
if event.Type() == gomatrixserverlib.MRoomRedaction && event.StateKey() == nil {
// in the special case where the event redacts itself, just pass the message through because
// we will never see the other part of the pair
if event.Redacts() != event.EventID() {
return nil
}
func (s *OutputRoomEventConsumer) onMessage(msg *nats.Msg) {
jetstream.WithJetStreamMessage(msg, func(msg *nats.Msg) bool {
// Parse out the event JSON
var err error
var output api.OutputEvent
if err = json.Unmarshal(msg.Data, &output); err != nil {
// If the message was invalid, log it and move on to the next message in the stream
log.WithError(err).Errorf("roomserver output log: message parse failure")
return true
}
return s.onNewRoomEvent(context.TODO(), *output.NewRoomEvent)
case api.OutputTypeOldRoomEvent:
return s.onOldRoomEvent(context.TODO(), *output.OldRoomEvent)
case api.OutputTypeNewInviteEvent:
return s.onNewInviteEvent(context.TODO(), *output.NewInviteEvent)
case api.OutputTypeRetireInviteEvent:
return s.onRetireInviteEvent(context.TODO(), *output.RetireInviteEvent)
case api.OutputTypeNewPeek:
return s.onNewPeek(context.TODO(), *output.NewPeek)
case api.OutputTypeRetirePeek:
return s.onRetirePeek(context.TODO(), *output.RetirePeek)
case api.OutputTypeRedactedEvent:
return s.onRedactEvent(context.TODO(), *output.RedactedEvent)
default:
log.WithField("type", output.Type).Debug(
"roomserver output log: ignoring unknown output type",
)
return nil
}
switch output.Type {
case api.OutputTypeNewRoomEvent:
// Ignore redaction events. We will add them to the database when they are
// validated (when we receive OutputTypeRedactedEvent)
event := output.NewRoomEvent.Event
if event.Type() == gomatrixserverlib.MRoomRedaction && event.StateKey() == nil {
// in the special case where the event redacts itself, just pass the message through because
// we will never see the other part of the pair
if event.Redacts() != event.EventID() {
return true
}
}
err = s.onNewRoomEvent(s.ctx, *output.NewRoomEvent)
case api.OutputTypeOldRoomEvent:
err = s.onOldRoomEvent(s.ctx, *output.OldRoomEvent)
case api.OutputTypeNewInviteEvent:
s.onNewInviteEvent(s.ctx, *output.NewInviteEvent)
case api.OutputTypeRetireInviteEvent:
s.onRetireInviteEvent(s.ctx, *output.RetireInviteEvent)
case api.OutputTypeNewPeek:
s.onNewPeek(s.ctx, *output.NewPeek)
case api.OutputTypeRetirePeek:
s.onRetirePeek(s.ctx, *output.RetirePeek)
case api.OutputTypeRedactedEvent:
err = s.onRedactEvent(s.ctx, *output.RedactedEvent)
default:
log.WithField("type", output.Type).Debug(
"roomserver output log: ignoring unknown output type",
)
}
if err != nil {
log.WithError(err).Error("roomserver output log: failed to process event")
return false
}
return true
})
}
func (s *OutputRoomEventConsumer) onRedactEvent(
@ -275,12 +277,12 @@ func (s *OutputRoomEventConsumer) notifyJoinedPeeks(ctx context.Context, ev *gom
func (s *OutputRoomEventConsumer) onNewInviteEvent(
ctx context.Context, msg api.OutputNewInviteEvent,
) error {
) {
if msg.Event.StateKey() == nil {
log.WithFields(log.Fields{
"event": string(msg.Event.JSON()),
}).Panicf("roomserver output log: invite has no state key")
return nil
return
}
pduPos, err := s.db.AddInviteEvent(ctx, msg.Event)
if err != nil {
@ -292,18 +294,16 @@ func (s *OutputRoomEventConsumer) onNewInviteEvent(
"pdupos": pduPos,
log.ErrorKey: err,
}).Panicf("roomserver output log: write invite failure")
return nil
return
}
s.inviteStream.Advance(pduPos)
s.notifier.OnNewInvite(types.StreamingToken{InvitePosition: pduPos}, *msg.Event.StateKey())
return nil
}
func (s *OutputRoomEventConsumer) onRetireInviteEvent(
ctx context.Context, msg api.OutputRetireInviteEvent,
) error {
) {
pduPos, err := s.db.RetireInviteEvent(ctx, msg.EventID)
if err != nil {
sentry.CaptureException(err)
@ -312,19 +312,17 @@ func (s *OutputRoomEventConsumer) onRetireInviteEvent(
"event_id": msg.EventID,
log.ErrorKey: err,
}).Panicf("roomserver output log: remove invite failure")
return nil
return
}
// Notify any active sync requests that the invite has been retired.
s.inviteStream.Advance(pduPos)
s.notifier.OnNewInvite(types.StreamingToken{InvitePosition: pduPos}, msg.TargetUserID)
return nil
}
func (s *OutputRoomEventConsumer) onNewPeek(
ctx context.Context, msg api.OutputNewPeek,
) error {
) {
sp, err := s.db.AddPeek(ctx, msg.RoomID, msg.UserID, msg.DeviceID)
if err != nil {
sentry.CaptureException(err)
@ -332,7 +330,7 @@ func (s *OutputRoomEventConsumer) onNewPeek(
log.WithFields(log.Fields{
log.ErrorKey: err,
}).Panicf("roomserver output log: write peek failure")
return nil
return
}
// tell the notifier about the new peek so it knows to wake up new devices
@ -340,20 +338,18 @@ func (s *OutputRoomEventConsumer) onNewPeek(
// index as PDUs, but we should fix this
s.pduStream.Advance(sp)
s.notifier.OnNewPeek(msg.RoomID, msg.UserID, msg.DeviceID, types.StreamingToken{PDUPosition: sp})
return nil
}
func (s *OutputRoomEventConsumer) onRetirePeek(
ctx context.Context, msg api.OutputRetirePeek,
) error {
) {
sp, err := s.db.DeletePeek(ctx, msg.RoomID, msg.UserID, msg.DeviceID)
if err != nil {
// panic rather than continue with an inconsistent database
log.WithFields(log.Fields{
log.ErrorKey: err,
}).Panicf("roomserver output log: write peek failure")
return nil
return
}
// tell the notifier about the new peek so it knows to wake up new devices
@ -361,8 +357,6 @@ func (s *OutputRoomEventConsumer) onRetirePeek(
// index as PDUs, but we should fix this
s.pduStream.Advance(sp)
s.notifier.OnRetirePeek(msg.RoomID, msg.UserID, msg.DeviceID, types.StreamingToken{PDUPosition: sp})
return nil
}
func (s *OutputRoomEventConsumer) updateStateEvent(event *gomatrixserverlib.HeaderedEvent) (*gomatrixserverlib.HeaderedEvent, error) {