2017-04-21 00:40:52 +02:00
|
|
|
// Copyright 2017 Vector Creations Ltd
|
Send-to-device support (#1072)
* Groundwork for send-to-device messaging
* Update sample config
* Add unstable routing for now
* Send to device consumer in sync API
* Start the send-to-device consumer
* fix indentation in dendrite-config.yaml
* Create send-to-device database tables, other tweaks
* Add some logic for send-to-device messages, add them into sync stream
* Handle incoming send-to-device messages, count them with EDU stream pos
* Undo changes to test
* pq.Array
* Fix sync
* Logging
* Fix a couple of transaction things, fix client API
* Add send-to-device test, hopefully fix bugs
* Comments
* Refactor a bit
* Fix schema
* Fix queries
* Debug logging
* Fix storing and retrieving of send-to-device messages
* Try to avoid database locks
* Update sync position
* Use latest sync position
* Jiggle about sync a bit
* Fix tests
* Break out the retrieval from the update/delete behaviour
* Comments
* nolint on getResponseWithPDUsForCompleteSync
* Try to line up sync tokens again
* Implement wildcard
* Add all send-to-device tests to whitelist, what could possibly go wrong?
* Only care about wildcard when targeted locally
* Deduplicate transactions
* Handle tokens properly, return immediately if waiting send-to-device messages
* Fix sync
* Update sytest-whitelist
* Fix copyright notice (need to do more of this)
* Comments, copyrights
* Return errors from Do, fix dendritejs
* Review comments
* Comments
* Constructor for TransactionWriter
* defletions
* Update gomatrixserverlib, sytest-blacklist
2020-06-01 18:50:19 +02:00
|
|
|
// Copyright 2017-2018 New Vector Ltd
|
|
|
|
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
2017-04-21 00:40:52 +02:00
|
|
|
//
|
|
|
|
// 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.
|
|
|
|
|
2017-04-07 15:32:42 +02:00
|
|
|
package sync
|
|
|
|
|
|
|
|
import (
|
2022-04-06 13:11:19 +02:00
|
|
|
"context"
|
|
|
|
"database/sql"
|
2020-11-20 12:29:02 +01:00
|
|
|
"net"
|
2017-04-07 15:32:42 +02:00
|
|
|
"net/http"
|
2020-11-20 12:29:02 +01:00
|
|
|
"strings"
|
|
|
|
"sync"
|
2017-04-07 15:32:42 +02:00
|
|
|
"time"
|
|
|
|
|
2022-08-09 10:40:46 +02:00
|
|
|
"github.com/matrix-org/gomatrixserverlib"
|
|
|
|
"github.com/matrix-org/util"
|
|
|
|
"github.com/prometheus/client_golang/prometheus"
|
|
|
|
"github.com/sirupsen/logrus"
|
|
|
|
|
2017-04-10 16:12:18 +02:00
|
|
|
"github.com/matrix-org/dendrite/clientapi/jsonerror"
|
2020-07-30 12:15:46 +02:00
|
|
|
keyapi "github.com/matrix-org/dendrite/keyserver/api"
|
2020-09-04 15:25:01 +02:00
|
|
|
roomserverAPI "github.com/matrix-org/dendrite/roomserver/api"
|
2020-12-02 18:41:00 +01:00
|
|
|
"github.com/matrix-org/dendrite/setup/config"
|
2020-07-30 12:15:46 +02:00
|
|
|
"github.com/matrix-org/dendrite/syncapi/internal"
|
2021-01-08 17:59:06 +01:00
|
|
|
"github.com/matrix-org/dendrite/syncapi/notifier"
|
2017-04-20 18:22:44 +02:00
|
|
|
"github.com/matrix-org/dendrite/syncapi/storage"
|
2021-01-08 17:59:06 +01:00
|
|
|
"github.com/matrix-org/dendrite/syncapi/streams"
|
2017-04-20 18:22:44 +02:00
|
|
|
"github.com/matrix-org/dendrite/syncapi/types"
|
2020-06-16 15:10:55 +02:00
|
|
|
userapi "github.com/matrix-org/dendrite/userapi/api"
|
2017-04-07 15:32:42 +02:00
|
|
|
)
|
|
|
|
|
|
|
|
// RequestPool manages HTTP long-poll connections for /sync
|
|
|
|
type RequestPool struct {
|
2020-06-16 18:05:38 +02:00
|
|
|
db storage.Database
|
2020-11-20 12:29:02 +01:00
|
|
|
cfg *config.SyncAPI
|
2022-05-05 10:56:03 +02:00
|
|
|
userAPI userapi.SyncUserAPI
|
|
|
|
keyAPI keyapi.SyncKeyAPI
|
|
|
|
rsAPI roomserverAPI.SyncRoomserverAPI
|
2022-04-06 13:11:19 +02:00
|
|
|
lastseen *sync.Map
|
|
|
|
presence *sync.Map
|
2021-01-08 17:59:06 +01:00
|
|
|
streams *streams.Streams
|
|
|
|
Notifier *notifier.Notifier
|
2022-04-06 13:11:19 +02:00
|
|
|
producer PresencePublisher
|
2022-05-17 16:53:08 +02:00
|
|
|
consumer PresenceConsumer
|
2022-04-06 13:11:19 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
type PresencePublisher interface {
|
|
|
|
SendPresence(userID string, presence types.Presence, statusMsg *string) error
|
2017-04-10 16:12:18 +02:00
|
|
|
}
|
|
|
|
|
2022-05-17 16:53:08 +02:00
|
|
|
type PresenceConsumer interface {
|
2022-07-07 16:29:25 +02:00
|
|
|
EmitPresence(ctx context.Context, userID string, presence types.Presence, statusMsg *string, ts gomatrixserverlib.Timestamp, fromSync bool)
|
2022-05-17 16:53:08 +02:00
|
|
|
}
|
|
|
|
|
2017-04-10 16:12:18 +02:00
|
|
|
// NewRequestPool makes a new RequestPool
|
2020-07-30 12:15:46 +02:00
|
|
|
func NewRequestPool(
|
2021-01-08 17:59:06 +01:00
|
|
|
db storage.Database, cfg *config.SyncAPI,
|
2022-05-05 10:56:03 +02:00
|
|
|
userAPI userapi.SyncUserAPI, keyAPI keyapi.SyncKeyAPI,
|
|
|
|
rsAPI roomserverAPI.SyncRoomserverAPI,
|
2021-01-08 17:59:06 +01:00
|
|
|
streams *streams.Streams, notifier *notifier.Notifier,
|
2022-05-17 16:53:08 +02:00
|
|
|
producer PresencePublisher, consumer PresenceConsumer, enableMetrics bool,
|
2020-07-30 12:15:46 +02:00
|
|
|
) *RequestPool {
|
2022-05-09 18:23:02 +02:00
|
|
|
if enableMetrics {
|
|
|
|
prometheus.MustRegister(
|
|
|
|
activeSyncRequests, waitingSyncRequests,
|
|
|
|
)
|
|
|
|
}
|
2021-01-08 17:59:06 +01:00
|
|
|
rp := &RequestPool{
|
|
|
|
db: db,
|
|
|
|
cfg: cfg,
|
|
|
|
userAPI: userAPI,
|
|
|
|
keyAPI: keyAPI,
|
|
|
|
rsAPI: rsAPI,
|
2022-04-06 13:11:19 +02:00
|
|
|
lastseen: &sync.Map{},
|
|
|
|
presence: &sync.Map{},
|
2021-01-08 17:59:06 +01:00
|
|
|
streams: streams,
|
|
|
|
Notifier: notifier,
|
2022-04-06 13:11:19 +02:00
|
|
|
producer: producer,
|
2022-05-17 16:53:08 +02:00
|
|
|
consumer: consumer,
|
2021-01-08 17:59:06 +01:00
|
|
|
}
|
2020-11-20 12:29:02 +01:00
|
|
|
go rp.cleanLastSeen()
|
2022-04-06 13:11:19 +02:00
|
|
|
go rp.cleanPresence(db, time.Minute*5)
|
2020-11-20 12:29:02 +01:00
|
|
|
return rp
|
|
|
|
}
|
|
|
|
|
|
|
|
func (rp *RequestPool) cleanLastSeen() {
|
|
|
|
for {
|
|
|
|
rp.lastseen.Range(func(key interface{}, _ interface{}) bool {
|
|
|
|
rp.lastseen.Delete(key)
|
|
|
|
return true
|
|
|
|
})
|
|
|
|
time.Sleep(time.Minute)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-04-06 13:11:19 +02:00
|
|
|
func (rp *RequestPool) cleanPresence(db storage.Presence, cleanupTime time.Duration) {
|
|
|
|
if !rp.cfg.Matrix.Presence.EnableOutbound {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
for {
|
|
|
|
rp.presence.Range(func(key interface{}, v interface{}) bool {
|
|
|
|
p := v.(types.PresenceInternal)
|
|
|
|
if time.Since(p.LastActiveTS.Time()) > cleanupTime {
|
|
|
|
rp.updatePresence(db, types.PresenceUnavailable.String(), p.UserID)
|
|
|
|
rp.presence.Delete(key)
|
|
|
|
}
|
|
|
|
return true
|
|
|
|
})
|
|
|
|
time.Sleep(cleanupTime)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// updatePresence sends presence updates to the SyncAPI and FederationAPI
|
|
|
|
func (rp *RequestPool) updatePresence(db storage.Presence, presence string, userID string) {
|
|
|
|
if !rp.cfg.Matrix.Presence.EnableOutbound {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if presence == "" {
|
|
|
|
presence = types.PresenceOnline.String()
|
|
|
|
}
|
|
|
|
|
|
|
|
presenceID, ok := types.PresenceFromString(presence)
|
|
|
|
if !ok { // this should almost never happen
|
|
|
|
return
|
|
|
|
}
|
2022-04-27 12:25:07 +02:00
|
|
|
|
2022-04-06 13:11:19 +02:00
|
|
|
newPresence := types.PresenceInternal{
|
|
|
|
Presence: presenceID,
|
|
|
|
UserID: userID,
|
|
|
|
LastActiveTS: gomatrixserverlib.AsTimestamp(time.Now()),
|
|
|
|
}
|
2022-04-27 12:25:07 +02:00
|
|
|
|
|
|
|
// ensure we also send the current status_msg to federated servers and not nil
|
|
|
|
dbPresence, err := db.GetPresence(context.Background(), userID)
|
|
|
|
if err != nil && err != sql.ErrNoRows {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if dbPresence != nil {
|
|
|
|
newPresence.ClientFields = dbPresence.ClientFields
|
|
|
|
}
|
|
|
|
newPresence.ClientFields.Presence = presenceID.String()
|
|
|
|
|
2022-04-06 13:11:19 +02:00
|
|
|
defer rp.presence.Store(userID, newPresence)
|
|
|
|
// avoid spamming presence updates when syncing
|
|
|
|
existingPresence, ok := rp.presence.LoadOrStore(userID, newPresence)
|
|
|
|
if ok {
|
|
|
|
p := existingPresence.(types.PresenceInternal)
|
|
|
|
if p.ClientFields.Presence == newPresence.ClientFields.Presence {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-04-27 12:25:07 +02:00
|
|
|
if err := rp.producer.SendPresence(userID, presenceID, newPresence.ClientFields.StatusMsg); err != nil {
|
2022-04-06 13:11:19 +02:00
|
|
|
logrus.WithError(err).Error("Unable to publish presence message from sync")
|
|
|
|
return
|
|
|
|
}
|
2022-05-17 16:53:08 +02:00
|
|
|
|
|
|
|
// now synchronously update our view of the world. It's critical we do this before calculating
|
|
|
|
// the /sync response else we may not return presence: online immediately.
|
|
|
|
rp.consumer.EmitPresence(
|
|
|
|
context.Background(), userID, presenceID, newPresence.ClientFields.StatusMsg,
|
2022-07-07 16:29:25 +02:00
|
|
|
gomatrixserverlib.AsTimestamp(time.Now()), true,
|
2022-05-17 16:53:08 +02:00
|
|
|
)
|
2022-04-06 13:11:19 +02:00
|
|
|
}
|
|
|
|
|
2020-11-20 12:29:02 +01:00
|
|
|
func (rp *RequestPool) updateLastSeen(req *http.Request, device *userapi.Device) {
|
|
|
|
if _, ok := rp.lastseen.LoadOrStore(device.UserID+device.ID, struct{}{}); ok {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
remoteAddr := req.RemoteAddr
|
|
|
|
if rp.cfg.RealIPHeader != "" {
|
|
|
|
if header := req.Header.Get(rp.cfg.RealIPHeader); header != "" {
|
|
|
|
// TODO: Maybe this isn't great but it will satisfy both X-Real-IP
|
|
|
|
// and X-Forwarded-For (which can be a list where the real client
|
|
|
|
// address is the first listed address). Make more intelligent?
|
|
|
|
addresses := strings.Split(header, ",")
|
|
|
|
if ip := net.ParseIP(addresses[0]); ip != nil {
|
|
|
|
remoteAddr = addresses[0]
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
lsreq := &userapi.PerformLastSeenUpdateRequest{
|
|
|
|
UserID: device.UserID,
|
|
|
|
DeviceID: device.ID,
|
|
|
|
RemoteAddr: remoteAddr,
|
2022-05-04 19:04:28 +02:00
|
|
|
UserAgent: req.UserAgent(),
|
2020-11-20 12:29:02 +01:00
|
|
|
}
|
|
|
|
lsres := &userapi.PerformLastSeenUpdateResponse{}
|
|
|
|
go rp.userAPI.PerformLastSeenUpdate(req.Context(), lsreq, lsres) // nolint:errcheck
|
|
|
|
|
|
|
|
rp.lastseen.Store(device.UserID+device.ID, time.Now())
|
2017-04-07 15:32:42 +02:00
|
|
|
}
|
|
|
|
|
2020-12-16 16:02:39 +01:00
|
|
|
var activeSyncRequests = prometheus.NewGauge(
|
|
|
|
prometheus.GaugeOpts{
|
|
|
|
Namespace: "dendrite",
|
|
|
|
Subsystem: "syncapi",
|
|
|
|
Name: "active_sync_requests",
|
|
|
|
Help: "The number of sync requests that are active right now",
|
|
|
|
},
|
|
|
|
)
|
|
|
|
|
|
|
|
var waitingSyncRequests = prometheus.NewGauge(
|
|
|
|
prometheus.GaugeOpts{
|
|
|
|
Namespace: "dendrite",
|
|
|
|
Subsystem: "syncapi",
|
|
|
|
Name: "waiting_sync_requests",
|
|
|
|
Help: "The number of sync requests that are waiting to be woken by a notifier",
|
|
|
|
},
|
|
|
|
)
|
|
|
|
|
2017-04-07 15:32:42 +02:00
|
|
|
// OnIncomingSyncRequest is called when a client makes a /sync request. This function MUST be
|
|
|
|
// called in a dedicated goroutine for this request. This function will block the goroutine
|
|
|
|
// until a response is ready, or it times out.
|
2020-06-16 15:10:55 +02:00
|
|
|
func (rp *RequestPool) OnIncomingSyncRequest(req *http.Request, device *userapi.Device) util.JSONResponse {
|
2017-04-07 15:32:42 +02:00
|
|
|
// Extract values from request
|
2020-06-26 16:34:41 +02:00
|
|
|
syncReq, err := newSyncRequest(req, *device, rp.db)
|
2017-04-10 16:12:18 +02:00
|
|
|
if err != nil {
|
2022-01-20 16:26:45 +01:00
|
|
|
if err == types.ErrMalformedSyncToken {
|
|
|
|
return util.JSONResponse{
|
|
|
|
Code: http.StatusBadRequest,
|
|
|
|
JSON: jsonerror.InvalidArgumentValue(err.Error()),
|
|
|
|
}
|
|
|
|
}
|
2017-04-10 16:12:18 +02:00
|
|
|
return util.JSONResponse{
|
2018-03-13 16:55:45 +01:00
|
|
|
Code: http.StatusBadRequest,
|
2017-04-10 16:12:18 +02:00
|
|
|
JSON: jsonerror.Unknown(err.Error()),
|
|
|
|
}
|
|
|
|
}
|
Send-to-device support (#1072)
* Groundwork for send-to-device messaging
* Update sample config
* Add unstable routing for now
* Send to device consumer in sync API
* Start the send-to-device consumer
* fix indentation in dendrite-config.yaml
* Create send-to-device database tables, other tweaks
* Add some logic for send-to-device messages, add them into sync stream
* Handle incoming send-to-device messages, count them with EDU stream pos
* Undo changes to test
* pq.Array
* Fix sync
* Logging
* Fix a couple of transaction things, fix client API
* Add send-to-device test, hopefully fix bugs
* Comments
* Refactor a bit
* Fix schema
* Fix queries
* Debug logging
* Fix storing and retrieving of send-to-device messages
* Try to avoid database locks
* Update sync position
* Use latest sync position
* Jiggle about sync a bit
* Fix tests
* Break out the retrieval from the update/delete behaviour
* Comments
* nolint on getResponseWithPDUsForCompleteSync
* Try to line up sync tokens again
* Implement wildcard
* Add all send-to-device tests to whitelist, what could possibly go wrong?
* Only care about wildcard when targeted locally
* Deduplicate transactions
* Handle tokens properly, return immediately if waiting send-to-device messages
* Fix sync
* Update sytest-whitelist
* Fix copyright notice (need to do more of this)
* Comments, copyrights
* Return errors from Do, fix dendritejs
* Review comments
* Comments
* Constructor for TransactionWriter
* defletions
* Update gomatrixserverlib, sytest-blacklist
2020-06-01 18:50:19 +02:00
|
|
|
|
2020-12-16 16:02:39 +01:00
|
|
|
activeSyncRequests.Inc()
|
|
|
|
defer activeSyncRequests.Dec()
|
|
|
|
|
2020-11-20 12:29:02 +01:00
|
|
|
rp.updateLastSeen(req, device)
|
2022-04-06 13:11:19 +02:00
|
|
|
rp.updatePresence(rp.db, req.FormValue("set_presence"), device.UserID)
|
2020-11-20 12:29:02 +01:00
|
|
|
|
2020-12-16 16:02:39 +01:00
|
|
|
waitingSyncRequests.Inc()
|
|
|
|
defer waitingSyncRequests.Dec()
|
|
|
|
|
2022-08-09 10:40:46 +02:00
|
|
|
// Clean up old send-to-device messages from before this stream position.
|
|
|
|
// This is needed to avoid sending the same message multiple times
|
|
|
|
if err = rp.db.CleanSendToDeviceUpdates(syncReq.Context, syncReq.Device.UserID, syncReq.Device.ID, syncReq.Since.SendToDevicePosition); err != nil {
|
|
|
|
syncReq.Log.WithError(err).Error("p.DB.CleanSendToDeviceUpdates failed")
|
|
|
|
}
|
|
|
|
|
2022-05-19 10:00:56 +02:00
|
|
|
// loop until we get some data
|
|
|
|
for {
|
|
|
|
startTime := time.Now()
|
|
|
|
currentPos := rp.Notifier.CurrentPosition()
|
|
|
|
|
|
|
|
// if the since token matches the current positions, wait via the notifier
|
|
|
|
if !rp.shouldReturnImmediately(syncReq, currentPos) {
|
|
|
|
timer := time.NewTimer(syncReq.Timeout) // case of timeout=0 is handled above
|
|
|
|
defer timer.Stop()
|
|
|
|
|
|
|
|
userStreamListener := rp.Notifier.GetListener(*syncReq)
|
|
|
|
defer userStreamListener.Close()
|
|
|
|
|
|
|
|
giveup := func() util.JSONResponse {
|
|
|
|
syncReq.Log.Debugln("Responding to sync since client gave up or timeout was reached")
|
|
|
|
syncReq.Response.NextBatch = syncReq.Since
|
|
|
|
// We should always try to include OTKs in sync responses, otherwise clients might upload keys
|
|
|
|
// even if that's not required. See also:
|
|
|
|
// https://github.com/matrix-org/synapse/blob/29f06704b8871a44926f7c99e73cf4a978fb8e81/synapse/rest/client/sync.py#L276-L281
|
|
|
|
// Only try to get OTKs if the context isn't already done.
|
|
|
|
if syncReq.Context.Err() == nil {
|
|
|
|
err = internal.DeviceOTKCounts(syncReq.Context, rp.keyAPI, syncReq.Device.UserID, syncReq.Device.ID, syncReq.Response)
|
|
|
|
if err != nil && err != context.Canceled {
|
|
|
|
syncReq.Log.WithError(err).Warn("failed to get OTK counts")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return util.JSONResponse{
|
|
|
|
Code: http.StatusOK,
|
|
|
|
JSON: syncReq.Response,
|
2022-05-13 10:24:26 +02:00
|
|
|
}
|
2022-05-11 18:15:18 +02:00
|
|
|
}
|
2017-10-16 14:34:08 +02:00
|
|
|
|
2022-05-19 10:00:56 +02:00
|
|
|
select {
|
|
|
|
case <-syncReq.Context.Done(): // Caller gave up
|
|
|
|
return giveup()
|
2021-01-08 17:59:06 +01:00
|
|
|
|
2022-05-19 10:00:56 +02:00
|
|
|
case <-timer.C: // Timeout reached
|
|
|
|
return giveup()
|
2021-01-08 17:59:06 +01:00
|
|
|
|
2022-05-19 10:00:56 +02:00
|
|
|
case <-userStreamListener.GetNotifyChannel(syncReq.Since):
|
|
|
|
currentPos.ApplyUpdates(userStreamListener.GetSyncPosition())
|
2022-08-25 14:42:47 +02:00
|
|
|
syncReq.Log.WithField("currentPos", currentPos).Debugln("Responding to sync after wake-up")
|
2022-05-19 10:00:56 +02:00
|
|
|
}
|
|
|
|
} else {
|
|
|
|
syncReq.Log.WithField("currentPos", currentPos).Debugln("Responding to sync immediately")
|
2017-10-16 14:34:08 +02:00
|
|
|
}
|
2018-12-18 18:56:08 +01:00
|
|
|
|
2022-05-19 10:00:56 +02:00
|
|
|
if syncReq.Since.IsEmpty() {
|
|
|
|
// Complete sync
|
|
|
|
syncReq.Response.NextBatch = types.StreamingToken{
|
2022-09-22 17:49:35 +02:00
|
|
|
// Get the current DeviceListPosition first, as the currentPosition
|
|
|
|
// might advance while processing other streams, resulting in flakey
|
|
|
|
// tests.
|
|
|
|
DeviceListPosition: rp.streams.DeviceListStreamProvider.CompleteSync(
|
|
|
|
syncReq.Context, syncReq,
|
|
|
|
),
|
2022-05-19 10:00:56 +02:00
|
|
|
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,
|
|
|
|
),
|
|
|
|
NotificationDataPosition: rp.streams.NotificationDataStreamProvider.CompleteSync(
|
|
|
|
syncReq.Context, syncReq,
|
|
|
|
),
|
|
|
|
PresencePosition: rp.streams.PresenceStreamProvider.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,
|
|
|
|
),
|
|
|
|
NotificationDataPosition: rp.streams.NotificationDataStreamProvider.IncrementalSync(
|
|
|
|
syncReq.Context, syncReq,
|
|
|
|
syncReq.Since.NotificationDataPosition, currentPos.NotificationDataPosition,
|
|
|
|
),
|
|
|
|
DeviceListPosition: rp.streams.DeviceListStreamProvider.IncrementalSync(
|
|
|
|
syncReq.Context, syncReq,
|
|
|
|
syncReq.Since.DeviceListPosition, currentPos.DeviceListPosition,
|
|
|
|
),
|
|
|
|
PresencePosition: rp.streams.PresenceStreamProvider.IncrementalSync(
|
|
|
|
syncReq.Context, syncReq,
|
|
|
|
syncReq.Since.PresencePosition, currentPos.PresencePosition,
|
|
|
|
),
|
|
|
|
}
|
|
|
|
// it's possible for there to be no updates for this user even though since < current pos,
|
|
|
|
// e.g busy servers with a quiet user. In this scenario, we don't want to return a no-op
|
|
|
|
// response immediately, so let's try this again but pretend they bumped their since token.
|
|
|
|
// If the incremental sync was processed very quickly then we expect the next loop to block
|
|
|
|
// with a notifier, but if things are slow it's entirely possible that currentPos is no
|
|
|
|
// longer the current position so we will hit this code path again. We need to do this and
|
|
|
|
// not return a no-op response because:
|
|
|
|
// - It's an inefficient use of bandwidth.
|
|
|
|
// - Some sytests which test 'waking up' sync rely on some sync requests to block, which
|
|
|
|
// they weren't always doing, resulting in flakey tests.
|
|
|
|
if !syncReq.Response.HasUpdates() {
|
|
|
|
syncReq.Since = currentPos
|
|
|
|
// do not loop again if the ?timeout= is 0 as that means "return immediately"
|
|
|
|
if syncReq.Timeout > 0 {
|
|
|
|
syncReq.Timeout = syncReq.Timeout - time.Since(startTime)
|
|
|
|
if syncReq.Timeout < 0 {
|
|
|
|
syncReq.Timeout = 0
|
|
|
|
}
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
2017-04-10 16:12:18 +02:00
|
|
|
}
|
2021-01-08 17:59:06 +01:00
|
|
|
|
2022-05-19 10:00:56 +02:00
|
|
|
return util.JSONResponse{
|
|
|
|
Code: http.StatusOK,
|
|
|
|
JSON: syncReq.Response,
|
|
|
|
}
|
2021-01-08 17:59:06 +01:00
|
|
|
}
|
2017-04-07 15:32:42 +02:00
|
|
|
}
|
|
|
|
|
2020-07-30 15:52:21 +02:00
|
|
|
func (rp *RequestPool) OnIncomingKeyChangeRequest(req *http.Request, device *userapi.Device) util.JSONResponse {
|
|
|
|
from := req.URL.Query().Get("from")
|
|
|
|
to := req.URL.Query().Get("to")
|
|
|
|
if from == "" || to == "" {
|
|
|
|
return util.JSONResponse{
|
|
|
|
Code: 400,
|
|
|
|
JSON: jsonerror.InvalidArgumentValue("missing ?from= or ?to="),
|
|
|
|
}
|
|
|
|
}
|
|
|
|
fromToken, err := types.NewStreamTokenFromString(from)
|
|
|
|
if err != nil {
|
|
|
|
return util.JSONResponse{
|
|
|
|
Code: 400,
|
|
|
|
JSON: jsonerror.InvalidArgumentValue("bad 'from' value"),
|
|
|
|
}
|
|
|
|
}
|
|
|
|
toToken, err := types.NewStreamTokenFromString(to)
|
|
|
|
if err != nil {
|
|
|
|
return util.JSONResponse{
|
|
|
|
Code: 400,
|
|
|
|
JSON: jsonerror.InvalidArgumentValue("bad 'to' value"),
|
|
|
|
}
|
|
|
|
}
|
2021-01-08 17:59:06 +01:00
|
|
|
syncReq, err := newSyncRequest(req, *device, rp.db)
|
2020-07-30 15:52:21 +02:00
|
|
|
if err != nil {
|
2021-01-08 17:59:06 +01:00
|
|
|
util.GetLogger(req.Context()).WithError(err).Error("newSyncRequest failed")
|
2020-07-30 15:52:21 +02:00
|
|
|
return jsonerror.InternalServerError()
|
|
|
|
}
|
2021-01-08 17:59:06 +01:00
|
|
|
rp.streams.PDUStreamProvider.IncrementalSync(req.Context(), syncReq, fromToken.PDUPosition, toToken.PDUPosition)
|
|
|
|
_, _, err = internal.DeviceListCatchup(
|
2022-07-15 17:25:26 +02:00
|
|
|
req.Context(), rp.db, rp.keyAPI, rp.rsAPI, syncReq.Device.UserID,
|
2021-01-08 17:59:06 +01:00
|
|
|
syncReq.Response, fromToken.DeviceListPosition, toToken.DeviceListPosition,
|
|
|
|
)
|
2020-07-30 15:52:21 +02:00
|
|
|
if err != nil {
|
2021-01-08 17:59:06 +01:00
|
|
|
util.GetLogger(req.Context()).WithError(err).Error("Failed to DeviceListCatchup info")
|
2020-07-30 15:52:21 +02:00
|
|
|
return jsonerror.InternalServerError()
|
|
|
|
}
|
|
|
|
return util.JSONResponse{
|
|
|
|
Code: 200,
|
|
|
|
JSON: struct {
|
|
|
|
Changed []string `json:"changed"`
|
|
|
|
Left []string `json:"left"`
|
|
|
|
}{
|
2021-01-08 17:59:06 +01:00
|
|
|
Changed: syncReq.Response.DeviceLists.Changed,
|
|
|
|
Left: syncReq.Response.DeviceLists.Left,
|
2020-07-30 15:52:21 +02:00
|
|
|
},
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-08-01 06:36:13 +02:00
|
|
|
// 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.
|
2021-04-26 17:33:31 +02:00
|
|
|
func (rp *RequestPool) shouldReturnImmediately(syncReq *types.SyncRequest, currentPos types.StreamingToken) bool {
|
|
|
|
if currentPos.IsAfter(syncReq.Since) || syncReq.Timeout == 0 || syncReq.WantFullState {
|
Send-to-device support (#1072)
* Groundwork for send-to-device messaging
* Update sample config
* Add unstable routing for now
* Send to device consumer in sync API
* Start the send-to-device consumer
* fix indentation in dendrite-config.yaml
* Create send-to-device database tables, other tweaks
* Add some logic for send-to-device messages, add them into sync stream
* Handle incoming send-to-device messages, count them with EDU stream pos
* Undo changes to test
* pq.Array
* Fix sync
* Logging
* Fix a couple of transaction things, fix client API
* Add send-to-device test, hopefully fix bugs
* Comments
* Refactor a bit
* Fix schema
* Fix queries
* Debug logging
* Fix storing and retrieving of send-to-device messages
* Try to avoid database locks
* Update sync position
* Use latest sync position
* Jiggle about sync a bit
* Fix tests
* Break out the retrieval from the update/delete behaviour
* Comments
* nolint on getResponseWithPDUsForCompleteSync
* Try to line up sync tokens again
* Implement wildcard
* Add all send-to-device tests to whitelist, what could possibly go wrong?
* Only care about wildcard when targeted locally
* Deduplicate transactions
* Handle tokens properly, return immediately if waiting send-to-device messages
* Fix sync
* Update sytest-whitelist
* Fix copyright notice (need to do more of this)
* Comments, copyrights
* Return errors from Do, fix dendritejs
* Review comments
* Comments
* Constructor for TransactionWriter
* defletions
* Update gomatrixserverlib, sytest-blacklist
2020-06-01 18:50:19 +02:00
|
|
|
return true
|
|
|
|
}
|
2021-01-08 17:59:06 +01:00
|
|
|
return false
|
2019-08-01 06:36:13 +02:00
|
|
|
}
|