mirror of
https://github.com/hoernschen/dendrite.git
synced 2025-08-01 13:52:46 +00:00
Refactor notifications (#2688)
This PR changes the handling of notifications - removes the `StreamEvent` and `ReadUpdate` stream - listens on the `OutputRoomEvent` stream in the UserAPI to inform the SyncAPI about unread notifications - listens on the `OutputReceiptEvent` stream in the UserAPI to set receipts/update notifications - sets the `read_markers` directly from within the internal UserAPI Co-authored-by: Neil Alexander <neilalexander@users.noreply.github.com>
This commit is contained in:
parent
f18bce93cc
commit
249b32c4f3
32 changed files with 368 additions and 598 deletions
127
userapi/consumers/clientapi.go
Normal file
127
userapi/consumers/clientapi.go
Normal file
|
@ -0,0 +1,127 @@
|
|||
// Copyright 2022 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package consumers
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
"github.com/nats-io/nats.go"
|
||||
log "github.com/sirupsen/logrus"
|
||||
|
||||
"github.com/matrix-org/dendrite/internal/pushgateway"
|
||||
"github.com/matrix-org/dendrite/userapi/storage"
|
||||
|
||||
"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/userapi/producers"
|
||||
"github.com/matrix-org/dendrite/userapi/util"
|
||||
)
|
||||
|
||||
// OutputReceiptEventConsumer consumes events that originated in the clientAPI.
|
||||
type OutputReceiptEventConsumer struct {
|
||||
ctx context.Context
|
||||
jetstream nats.JetStreamContext
|
||||
durable string
|
||||
topic string
|
||||
db storage.Database
|
||||
serverName gomatrixserverlib.ServerName
|
||||
syncProducer *producers.SyncAPI
|
||||
pgClient pushgateway.Client
|
||||
}
|
||||
|
||||
// NewOutputReceiptEventConsumer creates a new OutputReceiptEventConsumer.
|
||||
// Call Start() to begin consuming from the EDU server.
|
||||
func NewOutputReceiptEventConsumer(
|
||||
process *process.ProcessContext,
|
||||
cfg *config.UserAPI,
|
||||
js nats.JetStreamContext,
|
||||
store storage.Database,
|
||||
syncProducer *producers.SyncAPI,
|
||||
pgClient pushgateway.Client,
|
||||
) *OutputReceiptEventConsumer {
|
||||
return &OutputReceiptEventConsumer{
|
||||
ctx: process.Context(),
|
||||
jetstream: js,
|
||||
topic: cfg.Matrix.JetStream.Prefixed(jetstream.OutputReceiptEvent),
|
||||
durable: cfg.Matrix.JetStream.Durable("UserAPIReceiptConsumer"),
|
||||
db: store,
|
||||
serverName: cfg.Matrix.ServerName,
|
||||
syncProducer: syncProducer,
|
||||
pgClient: pgClient,
|
||||
}
|
||||
}
|
||||
|
||||
// Start consuming receipts events.
|
||||
func (s *OutputReceiptEventConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
func (s *OutputReceiptEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
|
||||
userID := msg.Header.Get(jetstream.UserID)
|
||||
roomID := msg.Header.Get(jetstream.RoomID)
|
||||
readPos := msg.Header.Get(jetstream.EventID)
|
||||
evType := msg.Header.Get("type")
|
||||
|
||||
if readPos == "" || evType != "m.read" {
|
||||
return true
|
||||
}
|
||||
|
||||
log := log.WithFields(log.Fields{
|
||||
"room_id": roomID,
|
||||
"user_id": userID,
|
||||
})
|
||||
|
||||
localpart, domain, err := gomatrixserverlib.SplitID('@', userID)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("userapi clientapi consumer: SplitID failure")
|
||||
return true
|
||||
}
|
||||
if domain != s.serverName {
|
||||
return true
|
||||
}
|
||||
|
||||
metadata, err := msg.Metadata()
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
|
||||
updated, err := s.db.SetNotificationsRead(ctx, localpart, roomID, uint64(gomatrixserverlib.AsTimestamp(metadata.Timestamp)), true)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("userapi EDU consumer")
|
||||
return false
|
||||
}
|
||||
|
||||
if err = s.syncProducer.GetAndSendNotificationData(ctx, userID, roomID); err != nil {
|
||||
log.WithError(err).Error("userapi EDU consumer: GetAndSendNotificationData failed")
|
||||
return false
|
||||
}
|
||||
|
||||
if !updated {
|
||||
return true
|
||||
}
|
||||
if err = util.NotifyUserCountsAsync(ctx, s.pgClient, localpart, s.db); err != nil {
|
||||
log.WithError(err).Error("userapi EDU consumer: NotifyUserCounts failed")
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
|
@ -26,7 +26,7 @@ import (
|
|||
"github.com/matrix-org/dendrite/userapi/util"
|
||||
)
|
||||
|
||||
type OutputStreamEventConsumer struct {
|
||||
type OutputRoomEventConsumer struct {
|
||||
ctx context.Context
|
||||
cfg *config.UserAPI
|
||||
rsAPI rsapi.UserRoomserverAPI
|
||||
|
@ -38,7 +38,7 @@ type OutputStreamEventConsumer struct {
|
|||
syncProducer *producers.SyncAPI
|
||||
}
|
||||
|
||||
func NewOutputStreamEventConsumer(
|
||||
func NewOutputRoomEventConsumer(
|
||||
process *process.ProcessContext,
|
||||
cfg *config.UserAPI,
|
||||
js nats.JetStreamContext,
|
||||
|
@ -46,21 +46,21 @@ func NewOutputStreamEventConsumer(
|
|||
pgClient pushgateway.Client,
|
||||
rsAPI rsapi.UserRoomserverAPI,
|
||||
syncProducer *producers.SyncAPI,
|
||||
) *OutputStreamEventConsumer {
|
||||
return &OutputStreamEventConsumer{
|
||||
) *OutputRoomEventConsumer {
|
||||
return &OutputRoomEventConsumer{
|
||||
ctx: process.Context(),
|
||||
cfg: cfg,
|
||||
jetstream: js,
|
||||
db: store,
|
||||
durable: cfg.Matrix.JetStream.Durable("UserAPISyncAPIStreamEventConsumer"),
|
||||
topic: cfg.Matrix.JetStream.Prefixed(jetstream.OutputStreamEvent),
|
||||
durable: cfg.Matrix.JetStream.Durable("UserAPIRoomServerConsumer"),
|
||||
topic: cfg.Matrix.JetStream.Prefixed(jetstream.OutputRoomEvent),
|
||||
pgClient: pgClient,
|
||||
rsAPI: rsAPI,
|
||||
syncProducer: syncProducer,
|
||||
}
|
||||
}
|
||||
|
||||
func (s *OutputStreamEventConsumer) Start() error {
|
||||
func (s *OutputRoomEventConsumer) Start() error {
|
||||
if err := jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
|
@ -70,35 +70,43 @@ func (s *OutputStreamEventConsumer) Start() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *OutputStreamEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
func (s *OutputRoomEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
var output types.StreamedEvent
|
||||
output.Event = &gomatrixserverlib.HeaderedEvent{}
|
||||
var output rsapi.OutputEvent
|
||||
if err := json.Unmarshal(msg.Data, &output); err != nil {
|
||||
log.WithError(err).Errorf("userapi consumer: message parse failure")
|
||||
// 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
|
||||
}
|
||||
if output.Event.Event == nil {
|
||||
if output.Type != rsapi.OutputTypeNewRoomEvent {
|
||||
return true
|
||||
}
|
||||
event := output.NewRoomEvent.Event
|
||||
if event == nil {
|
||||
log.Errorf("userapi consumer: expected event")
|
||||
return true
|
||||
}
|
||||
|
||||
log.WithFields(log.Fields{
|
||||
"event_id": output.Event.EventID(),
|
||||
"event_type": output.Event.Type(),
|
||||
"stream_pos": output.StreamPosition,
|
||||
}).Tracef("Received message from sync API: %#v", output)
|
||||
"event_id": event.EventID(),
|
||||
"event_type": event.Type(),
|
||||
}).Tracef("Received message from roomserver: %#v", output)
|
||||
|
||||
if err := s.processMessage(ctx, output.Event, int64(output.StreamPosition)); err != nil {
|
||||
metadata, err := msg.Metadata()
|
||||
if err != nil {
|
||||
return true
|
||||
}
|
||||
|
||||
if err := s.processMessage(ctx, event, uint64(gomatrixserverlib.AsTimestamp(metadata.Timestamp))); err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"event_id": output.Event.EventID(),
|
||||
"event_id": event.EventID(),
|
||||
}).WithError(err).Errorf("userapi consumer: process room event failure")
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
func (s *OutputStreamEventConsumer) processMessage(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, pos int64) error {
|
||||
func (s *OutputRoomEventConsumer) processMessage(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, streamPos uint64) error {
|
||||
members, roomSize, err := s.localRoomMembers(ctx, event.RoomID())
|
||||
if err != nil {
|
||||
return fmt.Errorf("s.localRoomMembers: %w", err)
|
||||
|
@ -138,10 +146,10 @@ func (s *OutputStreamEventConsumer) processMessage(ctx context.Context, event *g
|
|||
// removing it means we can send all notifications to
|
||||
// e.g. Element's Push gateway in one go.
|
||||
for _, mem := range members {
|
||||
if err := s.notifyLocal(ctx, event, pos, mem, roomSize, roomName); err != nil {
|
||||
if err := s.notifyLocal(ctx, event, mem, roomSize, roomName, streamPos); err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"localpart": mem.Localpart,
|
||||
}).WithError(err).Debugf("Unable to push to local user")
|
||||
}).WithError(err).Error("Unable to push to local user")
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
@ -179,7 +187,7 @@ func newLocalMembership(event *gomatrixserverlib.ClientEvent) (*localMembership,
|
|||
|
||||
// localRoomMembers fetches the current local members of a room, and
|
||||
// the total number of members.
|
||||
func (s *OutputStreamEventConsumer) localRoomMembers(ctx context.Context, roomID string) ([]*localMembership, int, error) {
|
||||
func (s *OutputRoomEventConsumer) localRoomMembers(ctx context.Context, roomID string) ([]*localMembership, int, error) {
|
||||
req := &rsapi.QueryMembershipsForRoomRequest{
|
||||
RoomID: roomID,
|
||||
JoinedOnly: true,
|
||||
|
@ -219,7 +227,7 @@ func (s *OutputStreamEventConsumer) localRoomMembers(ctx context.Context, roomID
|
|||
// looks it up in roomserver. If there is no name,
|
||||
// m.room.canonical_alias is consulted. Returns an empty string if the
|
||||
// room has no name.
|
||||
func (s *OutputStreamEventConsumer) roomName(ctx context.Context, event *gomatrixserverlib.HeaderedEvent) (string, error) {
|
||||
func (s *OutputRoomEventConsumer) roomName(ctx context.Context, event *gomatrixserverlib.HeaderedEvent) (string, error) {
|
||||
if event.Type() == gomatrixserverlib.MRoomName {
|
||||
name, err := unmarshalRoomName(event)
|
||||
if err != nil {
|
||||
|
@ -287,7 +295,7 @@ func unmarshalCanonicalAlias(event *gomatrixserverlib.HeaderedEvent) (string, er
|
|||
}
|
||||
|
||||
// notifyLocal finds the right push actions for a local user, given an event.
|
||||
func (s *OutputStreamEventConsumer) notifyLocal(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, pos int64, mem *localMembership, roomSize int, roomName string) error {
|
||||
func (s *OutputRoomEventConsumer) notifyLocal(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, mem *localMembership, roomSize int, roomName string, streamPos uint64) error {
|
||||
actions, err := s.evaluatePushRules(ctx, event, mem, roomSize)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -302,7 +310,7 @@ func (s *OutputStreamEventConsumer) notifyLocal(ctx context.Context, event *goma
|
|||
"event_id": event.EventID(),
|
||||
"room_id": event.RoomID(),
|
||||
"localpart": mem.Localpart,
|
||||
}).Debugf("Push rule evaluation rejected the event")
|
||||
}).Tracef("Push rule evaluation rejected the event")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -325,7 +333,7 @@ func (s *OutputStreamEventConsumer) notifyLocal(ctx context.Context, event *goma
|
|||
RoomID: event.RoomID(),
|
||||
TS: gomatrixserverlib.AsTimestamp(time.Now()),
|
||||
}
|
||||
if err = s.db.InsertNotification(ctx, mem.Localpart, event.EventID(), pos, tweaks, n); err != nil {
|
||||
if err = s.db.InsertNotification(ctx, mem.Localpart, event.EventID(), streamPos, tweaks, n); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -345,7 +353,7 @@ func (s *OutputStreamEventConsumer) notifyLocal(ctx context.Context, event *goma
|
|||
"localpart": mem.Localpart,
|
||||
"num_urls": len(devicesByURLAndFormat),
|
||||
"num_unread": userNumUnreadNotifs,
|
||||
}).Debugf("Notifying single member")
|
||||
}).Trace("Notifying single member")
|
||||
|
||||
// Push gateways are out of our control, and we cannot risk
|
||||
// looking up the server on a misbehaving push gateway. Each user
|
||||
|
@ -396,7 +404,7 @@ func (s *OutputStreamEventConsumer) notifyLocal(ctx context.Context, event *goma
|
|||
|
||||
// evaluatePushRules fetches and evaluates the push rules of a local
|
||||
// user. Returns actions (including dont_notify).
|
||||
func (s *OutputStreamEventConsumer) evaluatePushRules(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, mem *localMembership, roomSize int) ([]*pushrules.Action, error) {
|
||||
func (s *OutputRoomEventConsumer) evaluatePushRules(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, mem *localMembership, roomSize int) ([]*pushrules.Action, error) {
|
||||
if event.Sender() == mem.UserID {
|
||||
// SPEC: Homeservers MUST NOT notify the Push Gateway for
|
||||
// events that the user has sent themselves.
|
||||
|
@ -447,7 +455,7 @@ func (s *OutputStreamEventConsumer) evaluatePushRules(ctx context.Context, event
|
|||
"room_id": event.RoomID(),
|
||||
"localpart": mem.Localpart,
|
||||
"rule_id": rule.RuleID,
|
||||
}).Tracef("Matched a push rule")
|
||||
}).Trace("Matched a push rule")
|
||||
|
||||
return rule.Actions, nil
|
||||
}
|
||||
|
@ -491,7 +499,7 @@ func (rse *ruleSetEvalContext) HasPowerLevel(userID, levelKey string) (bool, err
|
|||
|
||||
// localPushDevices pushes to the configured devices of a local
|
||||
// user. The map keys are [url][format].
|
||||
func (s *OutputStreamEventConsumer) localPushDevices(ctx context.Context, localpart string, tweaks map[string]interface{}) (map[string]map[string][]*pushgateway.Device, string, error) {
|
||||
func (s *OutputRoomEventConsumer) localPushDevices(ctx context.Context, localpart string, tweaks map[string]interface{}) (map[string]map[string][]*pushgateway.Device, string, error) {
|
||||
pusherDevices, err := util.GetPushDevices(ctx, localpart, tweaks, s.db)
|
||||
if err != nil {
|
||||
return nil, "", err
|
||||
|
@ -515,7 +523,7 @@ func (s *OutputStreamEventConsumer) localPushDevices(ctx context.Context, localp
|
|||
}
|
||||
|
||||
// notifyHTTP performs a notificatation to a Push Gateway.
|
||||
func (s *OutputStreamEventConsumer) notifyHTTP(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, url, format string, devices []*pushgateway.Device, localpart, roomName string, userNumUnreadNotifs int) ([]*pushgateway.Device, error) {
|
||||
func (s *OutputRoomEventConsumer) notifyHTTP(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, url, format string, devices []*pushgateway.Device, localpart, roomName string, userNumUnreadNotifs int) ([]*pushgateway.Device, error) {
|
||||
logger := log.WithFields(log.Fields{
|
||||
"event_id": event.EventID(),
|
||||
"url": url,
|
||||
|
@ -561,13 +569,13 @@ func (s *OutputStreamEventConsumer) notifyHTTP(ctx context.Context, event *gomat
|
|||
}
|
||||
}
|
||||
|
||||
logger.Debugf("Notifying push gateway %s", url)
|
||||
logger.Tracef("Notifying push gateway %s", url)
|
||||
var res pushgateway.NotifyResponse
|
||||
if err := s.pgClient.Notify(ctx, url, &req, &res); err != nil {
|
||||
logger.WithError(err).Errorf("Failed to notify push gateway %s", url)
|
||||
return nil, err
|
||||
}
|
||||
logger.WithField("num_rejected", len(res.Rejected)).Tracef("Push gateway result")
|
||||
logger.WithField("num_rejected", len(res.Rejected)).Trace("Push gateway result")
|
||||
|
||||
if len(res.Rejected) == 0 {
|
||||
return nil, nil
|
||||
|
@ -589,7 +597,7 @@ func (s *OutputStreamEventConsumer) notifyHTTP(ctx context.Context, event *gomat
|
|||
}
|
||||
|
||||
// deleteRejectedPushers deletes the pushers associated with the given devices.
|
||||
func (s *OutputStreamEventConsumer) deleteRejectedPushers(ctx context.Context, devices []*pushgateway.Device, localpart string) {
|
||||
func (s *OutputRoomEventConsumer) deleteRejectedPushers(ctx context.Context, devices []*pushgateway.Device, localpart string) {
|
||||
log.WithFields(log.Fields{
|
||||
"localpart": localpart,
|
||||
"app_id0": devices[0].AppID,
|
|
@ -40,7 +40,7 @@ func Test_evaluatePushRules(t *testing.T) {
|
|||
test.WithAllDatabases(t, func(t *testing.T, dbType test.DBType) {
|
||||
db, close := mustCreateDatabase(t, dbType)
|
||||
defer close()
|
||||
consumer := OutputStreamEventConsumer{db: db}
|
||||
consumer := OutputRoomEventConsumer{db: db}
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
|
@ -1,137 +0,0 @@
|
|||
package consumers
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
|
||||
"github.com/matrix-org/dendrite/internal/pushgateway"
|
||||
"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/types"
|
||||
uapi "github.com/matrix-org/dendrite/userapi/api"
|
||||
"github.com/matrix-org/dendrite/userapi/producers"
|
||||
"github.com/matrix-org/dendrite/userapi/storage"
|
||||
"github.com/matrix-org/dendrite/userapi/util"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
"github.com/nats-io/nats.go"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type OutputReadUpdateConsumer struct {
|
||||
ctx context.Context
|
||||
cfg *config.UserAPI
|
||||
jetstream nats.JetStreamContext
|
||||
durable string
|
||||
db storage.Database
|
||||
pgClient pushgateway.Client
|
||||
ServerName gomatrixserverlib.ServerName
|
||||
topic string
|
||||
userAPI uapi.UserInternalAPI
|
||||
syncProducer *producers.SyncAPI
|
||||
}
|
||||
|
||||
func NewOutputReadUpdateConsumer(
|
||||
process *process.ProcessContext,
|
||||
cfg *config.UserAPI,
|
||||
js nats.JetStreamContext,
|
||||
store storage.Database,
|
||||
pgClient pushgateway.Client,
|
||||
userAPI uapi.UserInternalAPI,
|
||||
syncProducer *producers.SyncAPI,
|
||||
) *OutputReadUpdateConsumer {
|
||||
return &OutputReadUpdateConsumer{
|
||||
ctx: process.Context(),
|
||||
cfg: cfg,
|
||||
jetstream: js,
|
||||
db: store,
|
||||
ServerName: cfg.Matrix.ServerName,
|
||||
durable: cfg.Matrix.JetStream.Durable("UserAPISyncAPIReadUpdateConsumer"),
|
||||
topic: cfg.Matrix.JetStream.Prefixed(jetstream.OutputReadUpdate),
|
||||
pgClient: pgClient,
|
||||
userAPI: userAPI,
|
||||
syncProducer: syncProducer,
|
||||
}
|
||||
}
|
||||
|
||||
func (s *OutputReadUpdateConsumer) Start() error {
|
||||
if err := jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *OutputReadUpdateConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
var read types.ReadUpdate
|
||||
if err := json.Unmarshal(msg.Data, &read); err != nil {
|
||||
log.WithError(err).Error("userapi clientapi consumer: message parse failure")
|
||||
return true
|
||||
}
|
||||
if read.FullyRead == 0 && read.Read == 0 {
|
||||
return true
|
||||
}
|
||||
|
||||
userID := string(msg.Header.Get(jetstream.UserID))
|
||||
roomID := string(msg.Header.Get(jetstream.RoomID))
|
||||
|
||||
localpart, domain, err := gomatrixserverlib.SplitID('@', userID)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("userapi clientapi consumer: SplitID failure")
|
||||
return true
|
||||
}
|
||||
if domain != s.ServerName {
|
||||
log.Error("userapi clientapi consumer: not a local user")
|
||||
return true
|
||||
}
|
||||
|
||||
log := log.WithFields(log.Fields{
|
||||
"room_id": roomID,
|
||||
"user_id": userID,
|
||||
})
|
||||
log.Tracef("Received read update from sync API: %#v", read)
|
||||
|
||||
if read.Read > 0 {
|
||||
updated, err := s.db.SetNotificationsRead(ctx, localpart, roomID, int64(read.Read), true)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("userapi EDU consumer")
|
||||
return false
|
||||
}
|
||||
|
||||
if updated {
|
||||
if err = s.syncProducer.GetAndSendNotificationData(ctx, userID, roomID); err != nil {
|
||||
log.WithError(err).Error("userapi EDU consumer: GetAndSendNotificationData failed")
|
||||
return false
|
||||
}
|
||||
if err = util.NotifyUserCountsAsync(ctx, s.pgClient, localpart, s.db); err != nil {
|
||||
log.WithError(err).Error("userapi EDU consumer: NotifyUserCounts failed")
|
||||
return false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if read.FullyRead > 0 {
|
||||
deleted, err := s.db.DeleteNotificationsUpTo(ctx, localpart, roomID, int64(read.FullyRead))
|
||||
if err != nil {
|
||||
log.WithError(err).Errorf("userapi clientapi consumer: DeleteNotificationsUpTo failed")
|
||||
return false
|
||||
}
|
||||
|
||||
if deleted {
|
||||
if err := util.NotifyUserCountsAsync(ctx, s.pgClient, localpart, s.db); err != nil {
|
||||
log.WithError(err).Error("userapi clientapi consumer: NotifyUserCounts failed")
|
||||
return false
|
||||
}
|
||||
|
||||
if err := s.syncProducer.GetAndSendNotificationData(ctx, userID, read.RoomID); err != nil {
|
||||
log.WithError(err).Errorf("userapi clientapi consumer: GetAndSendNotificationData failed")
|
||||
return false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue