mirror of
https://github.com/hoernschen/dendrite.git
synced 2025-08-01 22:02:46 +00:00
Sync refactor — Part 1 (#1688)
* It's half-alive * Wakeups largely working * Other tweaks, typing works * Fix bugs, add receipt stream * Delete notifier, other tweaks * Dedupe a bit, add a template for the invite stream * Clean up, add templates for other streams * Don't leak channels * Bring forward some more PDU logic, clean up other places * Add some more wakeups * Use addRoomDeltaToResponse * Log tweaks, typing fixed? * Fix timed out syncs * Don't reset next batch position on timeout * Add account data stream/position * End of day * Fix complete sync for receipt, typing * Streams package * Clean up a bit * Complete sync send-to-device * Don't drop errors * More lightweight notifications * Fix typing positions * Don't advance position on remove again unless needed * Device list updates * Advance account data position * Use limit for incremental sync * Limit fixes, amongst other things * Remove some fmt.Println * Tweaks * Re-add notifier * Fix invite position * Fixes * Notify account data without advancing PDU position in notifier * Apply account data position * Get initial position for account data * Fix position update * Fix complete sync positions * Review comments @Kegsay * Room consumer parameters
This commit is contained in:
parent
56a7839aed
commit
b5a8935042
35 changed files with 1452 additions and 1116 deletions
|
@ -17,8 +17,6 @@
|
|||
package sync
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/http"
|
||||
"strings"
|
||||
|
@ -30,13 +28,13 @@ import (
|
|||
roomserverAPI "github.com/matrix-org/dendrite/roomserver/api"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
"github.com/matrix-org/dendrite/syncapi/internal"
|
||||
"github.com/matrix-org/dendrite/syncapi/notifier"
|
||||
"github.com/matrix-org/dendrite/syncapi/storage"
|
||||
"github.com/matrix-org/dendrite/syncapi/streams"
|
||||
"github.com/matrix-org/dendrite/syncapi/types"
|
||||
userapi "github.com/matrix-org/dendrite/userapi/api"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
"github.com/matrix-org/util"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// RequestPool manages HTTP long-poll connections for /sync
|
||||
|
@ -44,19 +42,30 @@ type RequestPool struct {
|
|||
db storage.Database
|
||||
cfg *config.SyncAPI
|
||||
userAPI userapi.UserInternalAPI
|
||||
Notifier *Notifier
|
||||
keyAPI keyapi.KeyInternalAPI
|
||||
rsAPI roomserverAPI.RoomserverInternalAPI
|
||||
lastseen sync.Map
|
||||
streams *streams.Streams
|
||||
Notifier *notifier.Notifier
|
||||
}
|
||||
|
||||
// NewRequestPool makes a new RequestPool
|
||||
func NewRequestPool(
|
||||
db storage.Database, cfg *config.SyncAPI, n *Notifier,
|
||||
db storage.Database, cfg *config.SyncAPI,
|
||||
userAPI userapi.UserInternalAPI, keyAPI keyapi.KeyInternalAPI,
|
||||
rsAPI roomserverAPI.RoomserverInternalAPI,
|
||||
streams *streams.Streams, notifier *notifier.Notifier,
|
||||
) *RequestPool {
|
||||
rp := &RequestPool{db, cfg, userAPI, n, keyAPI, rsAPI, sync.Map{}}
|
||||
rp := &RequestPool{
|
||||
db: db,
|
||||
cfg: cfg,
|
||||
userAPI: userAPI,
|
||||
keyAPI: keyAPI,
|
||||
rsAPI: rsAPI,
|
||||
lastseen: sync.Map{},
|
||||
streams: streams,
|
||||
Notifier: notifier,
|
||||
}
|
||||
go rp.cleanLastSeen()
|
||||
return rp
|
||||
}
|
||||
|
@ -128,8 +137,6 @@ var waitingSyncRequests = prometheus.NewGauge(
|
|||
// called in a dedicated goroutine for this request. This function will block the goroutine
|
||||
// until a response is ready, or it times out.
|
||||
func (rp *RequestPool) OnIncomingSyncRequest(req *http.Request, device *userapi.Device) util.JSONResponse {
|
||||
var syncData *types.Response
|
||||
|
||||
// Extract values from request
|
||||
syncReq, err := newSyncRequest(req, *device, rp.db)
|
||||
if err != nil {
|
||||
|
@ -139,88 +146,108 @@ func (rp *RequestPool) OnIncomingSyncRequest(req *http.Request, device *userapi.
|
|||
}
|
||||
}
|
||||
|
||||
logger := util.GetLogger(req.Context()).WithFields(log.Fields{
|
||||
"user_id": device.UserID,
|
||||
"device_id": device.ID,
|
||||
"since": syncReq.since,
|
||||
"timeout": syncReq.timeout,
|
||||
"limit": syncReq.limit,
|
||||
})
|
||||
|
||||
activeSyncRequests.Inc()
|
||||
defer activeSyncRequests.Dec()
|
||||
|
||||
rp.updateLastSeen(req, device)
|
||||
|
||||
currPos := rp.Notifier.CurrentPosition()
|
||||
|
||||
if rp.shouldReturnImmediately(syncReq) {
|
||||
syncData, err = rp.currentSyncForUser(*syncReq, currPos)
|
||||
if err != nil {
|
||||
logger.WithError(err).Error("rp.currentSyncForUser failed")
|
||||
return jsonerror.InternalServerError()
|
||||
}
|
||||
logger.WithField("next", syncData.NextBatch).Info("Responding immediately")
|
||||
return util.JSONResponse{
|
||||
Code: http.StatusOK,
|
||||
JSON: syncData,
|
||||
}
|
||||
}
|
||||
|
||||
waitingSyncRequests.Inc()
|
||||
defer waitingSyncRequests.Dec()
|
||||
|
||||
// Otherwise, we wait for the notifier to tell us if something *may* have
|
||||
// happened. We loop in case it turns out that nothing did happen.
|
||||
currentPos := rp.Notifier.CurrentPosition()
|
||||
|
||||
timer := time.NewTimer(syncReq.timeout) // case of timeout=0 is handled above
|
||||
defer timer.Stop()
|
||||
if !rp.shouldReturnImmediately(syncReq) {
|
||||
timer := time.NewTimer(syncReq.Timeout) // case of timeout=0 is handled above
|
||||
defer timer.Stop()
|
||||
|
||||
userStreamListener := rp.Notifier.GetListener(*syncReq)
|
||||
defer userStreamListener.Close()
|
||||
userStreamListener := rp.Notifier.GetListener(*syncReq)
|
||||
defer userStreamListener.Close()
|
||||
|
||||
// We need the loop in case userStreamListener wakes up even if there isn't
|
||||
// anything to send down. In this case, we'll jump out of the select but
|
||||
// don't want to send anything back until we get some actual content to
|
||||
// respond with, so we skip the return an go back to waiting for content to
|
||||
// be sent down or the request timing out.
|
||||
var hasTimedOut bool
|
||||
sincePos := syncReq.since
|
||||
for {
|
||||
select {
|
||||
// Wait for notifier to wake us up
|
||||
case <-userStreamListener.GetNotifyChannel(sincePos):
|
||||
currPos = userStreamListener.GetSyncPosition()
|
||||
// Or for timeout to expire
|
||||
case <-timer.C:
|
||||
// We just need to ensure we get out of the select after reaching the
|
||||
// timeout, but there's nothing specific we want to do in this case
|
||||
// apart from that, so we do nothing except stating we're timing out
|
||||
// and need to respond.
|
||||
hasTimedOut = true
|
||||
// Or for the request to be cancelled
|
||||
case <-req.Context().Done():
|
||||
logger.WithError(err).Error("request cancelled")
|
||||
return jsonerror.InternalServerError()
|
||||
}
|
||||
|
||||
// Note that we don't time out during calculation of sync
|
||||
// response. This ensures that we don't waste the hard work
|
||||
// of calculating the sync only to get timed out before we
|
||||
// can respond
|
||||
syncData, err = rp.currentSyncForUser(*syncReq, currPos)
|
||||
if err != nil {
|
||||
logger.WithError(err).Error("rp.currentSyncForUser failed")
|
||||
return jsonerror.InternalServerError()
|
||||
}
|
||||
|
||||
if !syncData.IsEmpty() || hasTimedOut {
|
||||
logger.WithField("next", syncData.NextBatch).WithField("timed_out", hasTimedOut).Info("Responding")
|
||||
giveup := func() util.JSONResponse {
|
||||
syncReq.Response.NextBatch = syncReq.Since
|
||||
return util.JSONResponse{
|
||||
Code: http.StatusOK,
|
||||
JSON: syncData,
|
||||
JSON: syncReq.Response,
|
||||
}
|
||||
}
|
||||
|
||||
select {
|
||||
case <-syncReq.Context.Done(): // Caller gave up
|
||||
return giveup()
|
||||
|
||||
case <-timer.C: // Timeout reached
|
||||
return giveup()
|
||||
|
||||
case <-userStreamListener.GetNotifyChannel(syncReq.Since):
|
||||
syncReq.Log.Debugln("Responding to sync after wake-up")
|
||||
currentPos.ApplyUpdates(userStreamListener.GetSyncPosition())
|
||||
}
|
||||
} else {
|
||||
syncReq.Log.Debugln("Responding to sync immediately")
|
||||
}
|
||||
|
||||
if syncReq.Since.IsEmpty() {
|
||||
// Complete sync
|
||||
syncReq.Response.NextBatch = types.StreamingToken{
|
||||
PDUPosition: rp.streams.PDUStreamProvider.CompleteSync(
|
||||
syncReq.Context, syncReq,
|
||||
),
|
||||
TypingPosition: rp.streams.TypingStreamProvider.CompleteSync(
|
||||
syncReq.Context, syncReq,
|
||||
),
|
||||
ReceiptPosition: rp.streams.ReceiptStreamProvider.CompleteSync(
|
||||
syncReq.Context, syncReq,
|
||||
),
|
||||
InvitePosition: rp.streams.InviteStreamProvider.CompleteSync(
|
||||
syncReq.Context, syncReq,
|
||||
),
|
||||
SendToDevicePosition: rp.streams.SendToDeviceStreamProvider.CompleteSync(
|
||||
syncReq.Context, syncReq,
|
||||
),
|
||||
AccountDataPosition: rp.streams.AccountDataStreamProvider.CompleteSync(
|
||||
syncReq.Context, syncReq,
|
||||
),
|
||||
DeviceListPosition: rp.streams.DeviceListStreamProvider.CompleteSync(
|
||||
syncReq.Context, syncReq,
|
||||
),
|
||||
}
|
||||
} else {
|
||||
// Incremental sync
|
||||
syncReq.Response.NextBatch = types.StreamingToken{
|
||||
PDUPosition: rp.streams.PDUStreamProvider.IncrementalSync(
|
||||
syncReq.Context, syncReq,
|
||||
syncReq.Since.PDUPosition, currentPos.PDUPosition,
|
||||
),
|
||||
TypingPosition: rp.streams.TypingStreamProvider.IncrementalSync(
|
||||
syncReq.Context, syncReq,
|
||||
syncReq.Since.TypingPosition, currentPos.TypingPosition,
|
||||
),
|
||||
ReceiptPosition: rp.streams.ReceiptStreamProvider.IncrementalSync(
|
||||
syncReq.Context, syncReq,
|
||||
syncReq.Since.ReceiptPosition, currentPos.ReceiptPosition,
|
||||
),
|
||||
InvitePosition: rp.streams.InviteStreamProvider.IncrementalSync(
|
||||
syncReq.Context, syncReq,
|
||||
syncReq.Since.InvitePosition, currentPos.InvitePosition,
|
||||
),
|
||||
SendToDevicePosition: rp.streams.SendToDeviceStreamProvider.IncrementalSync(
|
||||
syncReq.Context, syncReq,
|
||||
syncReq.Since.SendToDevicePosition, currentPos.SendToDevicePosition,
|
||||
),
|
||||
AccountDataPosition: rp.streams.AccountDataStreamProvider.IncrementalSync(
|
||||
syncReq.Context, syncReq,
|
||||
syncReq.Since.AccountDataPosition, currentPos.AccountDataPosition,
|
||||
),
|
||||
DeviceListPosition: rp.streams.DeviceListStreamProvider.IncrementalSync(
|
||||
syncReq.Context, syncReq,
|
||||
syncReq.Since.DeviceListPosition, currentPos.DeviceListPosition,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
return util.JSONResponse{
|
||||
Code: http.StatusOK,
|
||||
JSON: syncReq.Response,
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -247,18 +274,18 @@ func (rp *RequestPool) OnIncomingKeyChangeRequest(req *http.Request, device *use
|
|||
JSON: jsonerror.InvalidArgumentValue("bad 'to' value"),
|
||||
}
|
||||
}
|
||||
// work out room joins/leaves
|
||||
res, err := rp.db.IncrementalSync(
|
||||
req.Context(), types.NewResponse(), *device, fromToken, toToken, 10, false,
|
||||
)
|
||||
syncReq, err := newSyncRequest(req, *device, rp.db)
|
||||
if err != nil {
|
||||
util.GetLogger(req.Context()).WithError(err).Error("Failed to IncrementalSync")
|
||||
util.GetLogger(req.Context()).WithError(err).Error("newSyncRequest failed")
|
||||
return jsonerror.InternalServerError()
|
||||
}
|
||||
|
||||
res, err = rp.appendDeviceLists(res, device.UserID, fromToken, toToken)
|
||||
rp.streams.PDUStreamProvider.IncrementalSync(req.Context(), syncReq, fromToken.PDUPosition, toToken.PDUPosition)
|
||||
_, _, err = internal.DeviceListCatchup(
|
||||
req.Context(), rp.keyAPI, rp.rsAPI, syncReq.Device.UserID,
|
||||
syncReq.Response, fromToken.DeviceListPosition, toToken.DeviceListPosition,
|
||||
)
|
||||
if err != nil {
|
||||
util.GetLogger(req.Context()).WithError(err).Error("Failed to appendDeviceLists info")
|
||||
util.GetLogger(req.Context()).WithError(err).Error("Failed to DeviceListCatchup info")
|
||||
return jsonerror.InternalServerError()
|
||||
}
|
||||
return util.JSONResponse{
|
||||
|
@ -267,199 +294,18 @@ func (rp *RequestPool) OnIncomingKeyChangeRequest(req *http.Request, device *use
|
|||
Changed []string `json:"changed"`
|
||||
Left []string `json:"left"`
|
||||
}{
|
||||
Changed: res.DeviceLists.Changed,
|
||||
Left: res.DeviceLists.Left,
|
||||
Changed: syncReq.Response.DeviceLists.Changed,
|
||||
Left: syncReq.Response.DeviceLists.Left,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// nolint:gocyclo
|
||||
func (rp *RequestPool) currentSyncForUser(req syncRequest, latestPos types.StreamingToken) (*types.Response, error) {
|
||||
res := types.NewResponse()
|
||||
|
||||
// See if we have any new tasks to do for the send-to-device messaging.
|
||||
lastPos, events, updates, deletions, err := rp.db.SendToDeviceUpdatesForSync(req.ctx, req.device.UserID, req.device.ID, req.since)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("rp.db.SendToDeviceUpdatesForSync: %w", err)
|
||||
}
|
||||
|
||||
// TODO: handle ignored users
|
||||
if req.since.IsEmpty() {
|
||||
res, err = rp.db.CompleteSync(req.ctx, res, req.device, req.limit)
|
||||
if err != nil {
|
||||
return res, fmt.Errorf("rp.db.CompleteSync: %w", err)
|
||||
}
|
||||
} else {
|
||||
res, err = rp.db.IncrementalSync(req.ctx, res, req.device, req.since, latestPos, req.limit, req.wantFullState)
|
||||
if err != nil {
|
||||
return res, fmt.Errorf("rp.db.IncrementalSync: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
accountDataFilter := gomatrixserverlib.DefaultEventFilter() // TODO: use filter provided in req instead
|
||||
res, err = rp.appendAccountData(res, req.device.UserID, req, latestPos.PDUPosition, &accountDataFilter)
|
||||
if err != nil {
|
||||
return res, fmt.Errorf("rp.appendAccountData: %w", err)
|
||||
}
|
||||
res, err = rp.appendDeviceLists(res, req.device.UserID, req.since, latestPos)
|
||||
if err != nil {
|
||||
return res, fmt.Errorf("rp.appendDeviceLists: %w", err)
|
||||
}
|
||||
err = internal.DeviceOTKCounts(req.ctx, rp.keyAPI, req.device.UserID, req.device.ID, res)
|
||||
if err != nil {
|
||||
return res, fmt.Errorf("internal.DeviceOTKCounts: %w", err)
|
||||
}
|
||||
|
||||
// Before we return the sync response, make sure that we take action on
|
||||
// any send-to-device database updates or deletions that we need to do.
|
||||
// Then add the updates into the sync response.
|
||||
if len(updates) > 0 || len(deletions) > 0 {
|
||||
// Handle the updates and deletions in the database.
|
||||
err = rp.db.CleanSendToDeviceUpdates(context.Background(), updates, deletions, req.since)
|
||||
if err != nil {
|
||||
return res, fmt.Errorf("rp.db.CleanSendToDeviceUpdates: %w", err)
|
||||
}
|
||||
}
|
||||
if len(events) > 0 {
|
||||
// Add the updates into the sync response.
|
||||
for _, event := range events {
|
||||
res.ToDevice.Events = append(res.ToDevice.Events, event.SendToDeviceEvent)
|
||||
}
|
||||
}
|
||||
|
||||
res.NextBatch.SendToDevicePosition = lastPos
|
||||
return res, err
|
||||
}
|
||||
|
||||
func (rp *RequestPool) appendDeviceLists(
|
||||
data *types.Response, userID string, since, to types.StreamingToken,
|
||||
) (*types.Response, error) {
|
||||
_, err := internal.DeviceListCatchup(context.Background(), rp.keyAPI, rp.rsAPI, userID, data, since, to)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("internal.DeviceListCatchup: %w", err)
|
||||
}
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
// nolint:gocyclo
|
||||
func (rp *RequestPool) appendAccountData(
|
||||
data *types.Response, userID string, req syncRequest, currentPos types.StreamPosition,
|
||||
accountDataFilter *gomatrixserverlib.EventFilter,
|
||||
) (*types.Response, error) {
|
||||
// TODO: Account data doesn't have a sync position of its own, meaning that
|
||||
// account data might be sent multiple time to the client if multiple account
|
||||
// data keys were set between two message. This isn't a huge issue since the
|
||||
// duplicate data doesn't represent a huge quantity of data, but an optimisation
|
||||
// here would be making sure each data is sent only once to the client.
|
||||
if req.since.IsEmpty() {
|
||||
// If this is the initial sync, we don't need to check if a data has
|
||||
// already been sent. Instead, we send the whole batch.
|
||||
dataReq := &userapi.QueryAccountDataRequest{
|
||||
UserID: userID,
|
||||
}
|
||||
dataRes := &userapi.QueryAccountDataResponse{}
|
||||
if err := rp.userAPI.QueryAccountData(req.ctx, dataReq, dataRes); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
for datatype, databody := range dataRes.GlobalAccountData {
|
||||
data.AccountData.Events = append(
|
||||
data.AccountData.Events,
|
||||
gomatrixserverlib.ClientEvent{
|
||||
Type: datatype,
|
||||
Content: gomatrixserverlib.RawJSON(databody),
|
||||
},
|
||||
)
|
||||
}
|
||||
for r, j := range data.Rooms.Join {
|
||||
for datatype, databody := range dataRes.RoomAccountData[r] {
|
||||
j.AccountData.Events = append(
|
||||
j.AccountData.Events,
|
||||
gomatrixserverlib.ClientEvent{
|
||||
Type: datatype,
|
||||
Content: gomatrixserverlib.RawJSON(databody),
|
||||
},
|
||||
)
|
||||
data.Rooms.Join[r] = j
|
||||
}
|
||||
}
|
||||
return data, nil
|
||||
}
|
||||
|
||||
r := types.Range{
|
||||
From: req.since.PDUPosition,
|
||||
To: currentPos,
|
||||
}
|
||||
// If both positions are the same, it means that the data was saved after the
|
||||
// latest room event. In that case, we need to decrement the old position as
|
||||
// results are exclusive of Low.
|
||||
if r.Low() == r.High() {
|
||||
r.From--
|
||||
}
|
||||
|
||||
// Sync is not initial, get all account data since the latest sync
|
||||
dataTypes, err := rp.db.GetAccountDataInRange(
|
||||
req.ctx, userID, r, accountDataFilter,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("rp.db.GetAccountDataInRange: %w", err)
|
||||
}
|
||||
|
||||
if len(dataTypes) == 0 {
|
||||
// TODO: this fixes the sytest but is it the right thing to do?
|
||||
dataTypes[""] = []string{"m.push_rules"}
|
||||
}
|
||||
|
||||
// Iterate over the rooms
|
||||
for roomID, dataTypes := range dataTypes {
|
||||
// Request the missing data from the database
|
||||
for _, dataType := range dataTypes {
|
||||
dataReq := userapi.QueryAccountDataRequest{
|
||||
UserID: userID,
|
||||
RoomID: roomID,
|
||||
DataType: dataType,
|
||||
}
|
||||
dataRes := userapi.QueryAccountDataResponse{}
|
||||
err = rp.userAPI.QueryAccountData(req.ctx, &dataReq, &dataRes)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
if roomID == "" {
|
||||
if globalData, ok := dataRes.GlobalAccountData[dataType]; ok {
|
||||
data.AccountData.Events = append(
|
||||
data.AccountData.Events,
|
||||
gomatrixserverlib.ClientEvent{
|
||||
Type: dataType,
|
||||
Content: gomatrixserverlib.RawJSON(globalData),
|
||||
},
|
||||
)
|
||||
}
|
||||
} else {
|
||||
if roomData, ok := dataRes.RoomAccountData[roomID][dataType]; ok {
|
||||
joinData := data.Rooms.Join[roomID]
|
||||
joinData.AccountData.Events = append(
|
||||
joinData.AccountData.Events,
|
||||
gomatrixserverlib.ClientEvent{
|
||||
Type: dataType,
|
||||
Content: gomatrixserverlib.RawJSON(roomData),
|
||||
},
|
||||
)
|
||||
data.Rooms.Join[roomID] = joinData
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return data, nil
|
||||
}
|
||||
|
||||
// shouldReturnImmediately returns whether the /sync request is an initial sync,
|
||||
// or timeout=0, or full_state=true, in any of the cases the request should
|
||||
// return immediately.
|
||||
func (rp *RequestPool) shouldReturnImmediately(syncReq *syncRequest) bool {
|
||||
if syncReq.since.IsEmpty() || syncReq.timeout == 0 || syncReq.wantFullState {
|
||||
func (rp *RequestPool) shouldReturnImmediately(syncReq *types.SyncRequest) bool {
|
||||
if syncReq.Since.IsEmpty() || syncReq.Timeout == 0 || syncReq.WantFullState {
|
||||
return true
|
||||
}
|
||||
waiting, werr := rp.db.SendToDeviceUpdatesWaiting(context.TODO(), syncReq.device.UserID, syncReq.device.ID)
|
||||
return werr == nil && waiting
|
||||
return false
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue