2020-01-03 15:07:05 +01: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.
|
|
|
|
|
2020-01-03 15:07:05 +01:00
|
|
|
package postgres
|
2017-03-29 15:05:43 +02:00
|
|
|
|
|
|
|
import (
|
2017-09-18 17:52:22 +02:00
|
|
|
"context"
|
2017-03-29 15:05:43 +02:00
|
|
|
"database/sql"
|
2019-07-12 16:59:53 +02:00
|
|
|
"encoding/json"
|
2017-06-07 17:35:41 +02:00
|
|
|
"fmt"
|
2019-07-12 16:59:53 +02:00
|
|
|
"strconv"
|
|
|
|
"time"
|
2017-12-06 10:37:18 +01:00
|
|
|
|
2017-12-15 16:42:55 +01:00
|
|
|
"github.com/sirupsen/logrus"
|
|
|
|
|
|
|
|
"github.com/matrix-org/dendrite/clientapi/auth/authtypes"
|
2017-12-06 10:37:18 +01:00
|
|
|
"github.com/matrix-org/dendrite/roomserver/api"
|
2019-06-19 15:05:03 +02:00
|
|
|
|
2017-03-29 15:05:43 +02:00
|
|
|
// Import the postgres database driver.
|
|
|
|
_ "github.com/lib/pq"
|
2017-03-30 16:29:23 +02:00
|
|
|
"github.com/matrix-org/dendrite/common"
|
2017-04-20 18:22:44 +02:00
|
|
|
"github.com/matrix-org/dendrite/syncapi/types"
|
2019-07-12 16:59:53 +02:00
|
|
|
"github.com/matrix-org/dendrite/typingserver/cache"
|
2017-03-30 16:29:23 +02:00
|
|
|
"github.com/matrix-org/gomatrixserverlib"
|
2017-03-29 15:05:43 +02:00
|
|
|
)
|
|
|
|
|
2017-05-17 11:25:59 +02:00
|
|
|
type stateDelta struct {
|
|
|
|
roomID string
|
|
|
|
stateEvents []gomatrixserverlib.Event
|
|
|
|
membership string
|
2019-07-12 16:59:53 +02:00
|
|
|
// The PDU stream position of the latest membership event for this user, if applicable.
|
2017-05-17 17:21:27 +02:00
|
|
|
// Can be 0 if there is no membership event in this delta.
|
2019-07-12 16:59:53 +02:00
|
|
|
membershipPos int64
|
2017-05-17 17:21:27 +02:00
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
// Same as gomatrixserverlib.Event but also has the PDU stream position for this event.
|
2017-05-17 17:21:27 +02:00
|
|
|
type streamEvent struct {
|
|
|
|
gomatrixserverlib.Event
|
2019-07-12 16:59:53 +02:00
|
|
|
streamPosition int64
|
2017-12-06 10:37:18 +01:00
|
|
|
transactionID *api.TransactionID
|
2017-05-17 11:25:59 +02:00
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
// SyncServerDatabase represents a sync server datasource which manages
|
|
|
|
// both the database for PDUs and caches for EDUs.
|
|
|
|
type SyncServerDatasource struct {
|
2017-09-21 16:09:19 +02:00
|
|
|
db *sql.DB
|
|
|
|
common.PartitionOffsetStatements
|
2017-08-02 17:21:35 +02:00
|
|
|
accountData accountDataStatements
|
|
|
|
events outputRoomEventsStatements
|
|
|
|
roomstate currentRoomStateStatements
|
2017-09-20 16:36:41 +02:00
|
|
|
invites inviteEventsStatements
|
2019-07-12 16:59:53 +02:00
|
|
|
typingCache *cache.TypingCache
|
2017-03-29 15:05:43 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
// NewSyncServerDatabase creates a new sync server database
|
2019-07-12 16:59:53 +02:00
|
|
|
func NewSyncServerDatasource(dbDataSourceName string) (*SyncServerDatasource, error) {
|
|
|
|
var d SyncServerDatasource
|
2017-03-29 15:05:43 +02:00
|
|
|
var err error
|
2019-07-12 16:59:53 +02:00
|
|
|
if d.db, err = sql.Open("postgres", dbDataSourceName); err != nil {
|
2017-03-29 15:05:43 +02:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-09-21 16:09:19 +02:00
|
|
|
if err = d.PartitionOffsetStatements.Prepare(d.db, "syncapi"); err != nil {
|
2017-03-29 15:05:43 +02:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-09-20 16:36:41 +02:00
|
|
|
if err = d.accountData.prepare(d.db); err != nil {
|
2017-08-02 17:21:35 +02:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-09-20 16:36:41 +02:00
|
|
|
if err = d.events.prepare(d.db); err != nil {
|
2017-03-30 16:29:23 +02:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-09-20 16:36:41 +02:00
|
|
|
if err := d.roomstate.prepare(d.db); err != nil {
|
2017-04-05 11:30:13 +02:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-09-20 16:36:41 +02:00
|
|
|
if err := d.invites.prepare(d.db); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2019-07-12 16:59:53 +02:00
|
|
|
d.typingCache = cache.NewTypingCache()
|
2017-09-20 16:36:41 +02:00
|
|
|
return &d, nil
|
2017-03-30 16:29:23 +02:00
|
|
|
}
|
|
|
|
|
2017-05-17 16:38:24 +02:00
|
|
|
// AllJoinedUsersInRooms returns a map of room ID to a list of all joined user IDs.
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) AllJoinedUsersInRooms(ctx context.Context) (map[string][]string, error) {
|
2017-09-18 17:52:22 +02:00
|
|
|
return d.roomstate.selectJoinedUsers(ctx)
|
2017-05-17 16:38:24 +02:00
|
|
|
}
|
|
|
|
|
2017-06-07 17:35:41 +02:00
|
|
|
// Events lookups a list of event by their event ID.
|
|
|
|
// Returns a list of events matching the requested IDs found in the database.
|
|
|
|
// If an event is not found in the database then it will be omitted from the list.
|
2017-12-15 16:42:55 +01:00
|
|
|
// Returns an error if there was a problem talking with the database.
|
|
|
|
// Does not include any transaction IDs in the returned events.
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) Events(ctx context.Context, eventIDs []string) ([]gomatrixserverlib.Event, error) {
|
2017-09-18 17:52:22 +02:00
|
|
|
streamEvents, err := d.events.selectEvents(ctx, nil, eventIDs)
|
2017-06-07 17:35:41 +02:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-12-15 16:42:55 +01:00
|
|
|
|
|
|
|
// We don't include a device here as we only include transaction IDs in
|
|
|
|
// incremental syncs.
|
|
|
|
return streamEventsToEvents(nil, streamEvents), nil
|
2017-06-07 17:35:41 +02:00
|
|
|
}
|
|
|
|
|
2017-03-30 16:29:23 +02:00
|
|
|
// WriteEvent into the database. It is not safe to call this function from multiple goroutines, as it would create races
|
2019-07-12 16:59:53 +02:00
|
|
|
// when generating the sync stream position for this event. Returns the sync stream position for the inserted event.
|
2017-04-10 16:12:18 +02:00
|
|
|
// Returns an error if there was a problem inserting this event.
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) WriteEvent(
|
2017-09-18 17:52:22 +02:00
|
|
|
ctx context.Context,
|
|
|
|
ev *gomatrixserverlib.Event,
|
|
|
|
addStateEvents []gomatrixserverlib.Event,
|
|
|
|
addStateEventIDs, removeStateEventIDs []string,
|
2017-12-06 10:37:18 +01:00
|
|
|
transactionID *api.TransactionID,
|
2019-07-12 16:59:53 +02:00
|
|
|
) (pduPosition int64, returnErr error) {
|
2017-07-17 18:20:57 +02:00
|
|
|
returnErr = common.WithTransaction(d.db, func(txn *sql.Tx) error {
|
2017-04-10 16:12:18 +02:00
|
|
|
var err error
|
2017-12-06 10:37:18 +01:00
|
|
|
pos, err := d.events.insertEvent(ctx, txn, ev, addStateEventIDs, removeStateEventIDs, transactionID)
|
2017-04-10 16:12:18 +02:00
|
|
|
if err != nil {
|
2017-04-05 11:30:13 +02:00
|
|
|
return err
|
|
|
|
}
|
2019-07-12 16:59:53 +02:00
|
|
|
pduPosition = pos
|
2017-04-05 11:30:13 +02:00
|
|
|
|
2017-06-07 17:35:41 +02:00
|
|
|
if len(addStateEvents) == 0 && len(removeStateEventIDs) == 0 {
|
2017-04-05 11:30:13 +02:00
|
|
|
// Nothing to do, the event may have just been a message event.
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
return d.updateRoomState(ctx, txn, removeStateEventIDs, addStateEvents, pduPosition)
|
2017-04-05 11:30:13 +02:00
|
|
|
})
|
2017-04-10 16:12:18 +02:00
|
|
|
return
|
2017-03-29 15:05:43 +02:00
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) updateRoomState(
|
2017-09-18 17:52:22 +02:00
|
|
|
ctx context.Context, txn *sql.Tx,
|
|
|
|
removedEventIDs []string,
|
|
|
|
addedEvents []gomatrixserverlib.Event,
|
2019-07-12 16:59:53 +02:00
|
|
|
pduPosition int64,
|
2017-06-07 17:35:41 +02:00
|
|
|
) error {
|
2017-06-05 11:37:04 +02:00
|
|
|
// remove first, then add, as we do not ever delete state, but do replace state which is a remove followed by an add.
|
|
|
|
for _, eventID := range removedEventIDs {
|
2017-09-18 17:52:22 +02:00
|
|
|
if err := d.roomstate.deleteRoomStateByEventID(ctx, txn, eventID); err != nil {
|
2017-06-05 11:37:04 +02:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, event := range addedEvents {
|
|
|
|
if event.StateKey() == nil {
|
|
|
|
// ignore non state events
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
var membership *string
|
|
|
|
if event.Type() == "m.room.member" {
|
2017-07-07 15:11:32 +02:00
|
|
|
value, err := event.Membership()
|
|
|
|
if err != nil {
|
2017-06-05 11:37:04 +02:00
|
|
|
return err
|
|
|
|
}
|
2017-07-07 15:11:32 +02:00
|
|
|
membership = &value
|
2017-06-05 11:37:04 +02:00
|
|
|
}
|
2019-07-12 16:59:53 +02:00
|
|
|
if err := d.roomstate.upsertRoomState(ctx, txn, event, membership, pduPosition); err != nil {
|
2017-06-05 11:37:04 +02:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-07-25 17:10:59 +02:00
|
|
|
// GetStateEvent returns the Matrix state event of a given type for a given room with a given state key
|
|
|
|
// If no event could be found, returns nil
|
|
|
|
// If there was an issue during the retrieval, returns an error
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) GetStateEvent(
|
2017-10-06 12:23:58 +02:00
|
|
|
ctx context.Context, roomID, evType, stateKey string,
|
2017-09-18 17:52:22 +02:00
|
|
|
) (*gomatrixserverlib.Event, error) {
|
2017-10-06 12:23:58 +02:00
|
|
|
return d.roomstate.selectStateEvent(ctx, roomID, evType, stateKey)
|
2017-07-25 17:10:59 +02:00
|
|
|
}
|
|
|
|
|
2017-09-22 12:34:54 +02:00
|
|
|
// GetStateEventsForRoom fetches the state events for a given room.
|
|
|
|
// Returns an empty slice if no state events could be found for this room.
|
|
|
|
// Returns an error if there was an issue with the retrieval.
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) GetStateEventsForRoom(
|
2019-08-07 12:12:09 +02:00
|
|
|
ctx context.Context, roomID string, stateFilterPart *gomatrixserverlib.FilterPart,
|
2017-09-22 12:34:54 +02:00
|
|
|
) (stateEvents []gomatrixserverlib.Event, err error) {
|
|
|
|
err = common.WithTransaction(d.db, func(txn *sql.Tx) error {
|
2019-08-07 12:12:09 +02:00
|
|
|
stateEvents, err = d.roomstate.selectCurrentState(ctx, txn, roomID, stateFilterPart)
|
2017-09-22 12:34:54 +02:00
|
|
|
return err
|
|
|
|
})
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
// SyncPosition returns the latest positions for syncing.
|
|
|
|
func (d *SyncServerDatasource) SyncPosition(ctx context.Context) (types.SyncPosition, error) {
|
|
|
|
return d.syncPositionTx(ctx, nil)
|
2017-09-19 18:15:46 +02:00
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) syncPositionTx(
|
2017-09-19 18:15:46 +02:00
|
|
|
ctx context.Context, txn *sql.Tx,
|
2019-07-12 16:59:53 +02:00
|
|
|
) (sp types.SyncPosition, err error) {
|
|
|
|
|
|
|
|
maxEventID, err := d.events.selectMaxEventID(ctx, txn)
|
2017-09-19 18:15:46 +02:00
|
|
|
if err != nil {
|
2019-07-12 16:59:53 +02:00
|
|
|
return sp, err
|
2017-09-19 18:15:46 +02:00
|
|
|
}
|
|
|
|
maxAccountDataID, err := d.accountData.selectMaxAccountDataID(ctx, txn)
|
2017-04-11 12:52:26 +02:00
|
|
|
if err != nil {
|
2019-07-12 16:59:53 +02:00
|
|
|
return sp, err
|
2017-09-19 18:15:46 +02:00
|
|
|
}
|
2019-07-12 16:59:53 +02:00
|
|
|
if maxAccountDataID > maxEventID {
|
|
|
|
maxEventID = maxAccountDataID
|
2017-04-11 12:52:26 +02:00
|
|
|
}
|
2017-09-20 16:36:41 +02:00
|
|
|
maxInviteID, err := d.invites.selectMaxInviteID(ctx, txn)
|
|
|
|
if err != nil {
|
2019-07-12 16:59:53 +02:00
|
|
|
return sp, err
|
2017-09-20 16:36:41 +02:00
|
|
|
}
|
2019-07-12 16:59:53 +02:00
|
|
|
if maxInviteID > maxEventID {
|
|
|
|
maxEventID = maxInviteID
|
2017-09-20 16:36:41 +02:00
|
|
|
}
|
2019-07-12 16:59:53 +02:00
|
|
|
sp.PDUPosition = maxEventID
|
|
|
|
|
|
|
|
sp.TypingPosition = d.typingCache.GetLatestSyncPosition()
|
|
|
|
|
|
|
|
return
|
2017-04-10 16:12:18 +02:00
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
// addPDUDeltaToResponse adds all PDU deltas to a sync response.
|
|
|
|
// IDs of all rooms the user joined are returned so EDU deltas can be added for them.
|
|
|
|
func (d *SyncServerDatasource) addPDUDeltaToResponse(
|
2017-09-18 17:52:22 +02:00
|
|
|
ctx context.Context,
|
2017-12-15 16:42:55 +01:00
|
|
|
device authtypes.Device,
|
2019-07-12 16:59:53 +02:00
|
|
|
fromPos, toPos int64,
|
2017-09-18 17:52:22 +02:00
|
|
|
numRecentEventsPerRoom int,
|
2019-08-01 06:36:13 +02:00
|
|
|
wantFullState bool,
|
2019-07-12 16:59:53 +02:00
|
|
|
res *types.Response,
|
|
|
|
) ([]string, error) {
|
2017-09-19 17:22:02 +02:00
|
|
|
txn, err := d.db.BeginTx(ctx, &txReadOnlySnapshot)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
var succeeded bool
|
|
|
|
defer common.EndTransaction(txn, &succeeded)
|
|
|
|
|
2019-08-07 12:12:09 +02:00
|
|
|
stateFilterPart := gomatrixserverlib.DefaultFilterPart() // TODO: use filter provided in request
|
|
|
|
|
2017-09-19 17:22:02 +02:00
|
|
|
// Work out which rooms to return in the response. This is done by getting not only the currently
|
2019-07-12 16:59:53 +02:00
|
|
|
// joined rooms, but also which rooms have membership transitions for this user between the 2 PDU stream positions.
|
2017-09-19 17:22:02 +02:00
|
|
|
// This works out what the 'state' key should be for each room as well as which membership block
|
|
|
|
// to put the room into.
|
2019-08-01 06:36:13 +02:00
|
|
|
var deltas []stateDelta
|
|
|
|
var joinedRoomIDs []string
|
|
|
|
if !wantFullState {
|
2019-08-07 12:12:09 +02:00
|
|
|
deltas, joinedRoomIDs, err = d.getStateDeltas(
|
|
|
|
ctx, &device, txn, fromPos, toPos, device.UserID, &stateFilterPart,
|
|
|
|
)
|
2019-08-01 06:36:13 +02:00
|
|
|
} else {
|
2019-08-07 12:12:09 +02:00
|
|
|
deltas, joinedRoomIDs, err = d.getStateDeltasForFullStateSync(
|
|
|
|
ctx, &device, txn, fromPos, toPos, device.UserID, &stateFilterPart,
|
|
|
|
)
|
2019-08-01 06:36:13 +02:00
|
|
|
}
|
2017-09-19 17:22:02 +02:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, delta := range deltas {
|
2017-12-15 16:42:55 +01:00
|
|
|
err = d.addRoomDeltaToResponse(ctx, &device, txn, fromPos, toPos, delta, numRecentEventsPerRoom, res)
|
2017-04-19 17:04:01 +02:00
|
|
|
if err != nil {
|
2017-09-19 17:22:02 +02:00
|
|
|
return nil, err
|
2017-04-19 17:04:01 +02:00
|
|
|
}
|
2017-09-19 17:22:02 +02:00
|
|
|
}
|
2017-05-15 18:41:54 +02:00
|
|
|
|
2017-09-19 17:22:02 +02:00
|
|
|
// TODO: This should be done in getStateDeltas
|
2017-12-15 16:42:55 +01:00
|
|
|
if err = d.addInvitesToResponse(ctx, txn, device.UserID, fromPos, toPos, res); err != nil {
|
2017-09-19 17:22:02 +02:00
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
succeeded = true
|
2019-07-12 16:59:53 +02:00
|
|
|
return joinedRoomIDs, nil
|
2017-04-19 17:04:01 +02:00
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
// addTypingDeltaToResponse adds all typing notifications to a sync response
|
|
|
|
// since the specified position.
|
|
|
|
func (d *SyncServerDatasource) addTypingDeltaToResponse(
|
|
|
|
since int64,
|
|
|
|
joinedRoomIDs []string,
|
|
|
|
res *types.Response,
|
|
|
|
) error {
|
|
|
|
var jr types.JoinResponse
|
|
|
|
var ok bool
|
|
|
|
var err error
|
|
|
|
for _, roomID := range joinedRoomIDs {
|
|
|
|
if typingUsers, updated := d.typingCache.GetTypingUsersIfUpdatedAfter(
|
|
|
|
roomID, since,
|
|
|
|
); updated {
|
|
|
|
ev := gomatrixserverlib.ClientEvent{
|
|
|
|
Type: gomatrixserverlib.MTyping,
|
|
|
|
}
|
|
|
|
ev.Content, err = json.Marshal(map[string]interface{}{
|
|
|
|
"user_ids": typingUsers,
|
|
|
|
})
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
if jr, ok = res.Rooms.Join[roomID]; !ok {
|
|
|
|
jr = *types.NewJoinResponse()
|
|
|
|
}
|
|
|
|
jr.Ephemeral.Events = append(jr.Ephemeral.Events, ev)
|
|
|
|
res.Rooms.Join[roomID] = jr
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// addEDUDeltaToResponse adds updates for EDUs of each type since fromPos if
|
|
|
|
// the positions of that type are not equal in fromPos and toPos.
|
|
|
|
func (d *SyncServerDatasource) addEDUDeltaToResponse(
|
|
|
|
fromPos, toPos types.SyncPosition,
|
|
|
|
joinedRoomIDs []string,
|
|
|
|
res *types.Response,
|
|
|
|
) (err error) {
|
|
|
|
|
|
|
|
if fromPos.TypingPosition != toPos.TypingPosition {
|
|
|
|
err = d.addTypingDeltaToResponse(
|
|
|
|
fromPos.TypingPosition, joinedRoomIDs, res,
|
|
|
|
)
|
|
|
|
}
|
|
|
|
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// IncrementalSync returns all the data needed in order to create an incremental
|
|
|
|
// sync response for the given user. Events returned will include any client
|
|
|
|
// transaction IDs associated with the given device. These transaction IDs come
|
|
|
|
// from when the device sent the event via an API that included a transaction
|
|
|
|
// ID.
|
|
|
|
func (d *SyncServerDatasource) IncrementalSync(
|
|
|
|
ctx context.Context,
|
|
|
|
device authtypes.Device,
|
|
|
|
fromPos, toPos types.SyncPosition,
|
|
|
|
numRecentEventsPerRoom int,
|
2019-08-01 06:36:13 +02:00
|
|
|
wantFullState bool,
|
2017-09-19 17:22:02 +02:00
|
|
|
) (*types.Response, error) {
|
2019-07-12 16:59:53 +02:00
|
|
|
nextBatchPos := fromPos.WithUpdates(toPos)
|
|
|
|
res := types.NewResponse(nextBatchPos)
|
|
|
|
|
|
|
|
var joinedRoomIDs []string
|
|
|
|
var err error
|
2019-08-01 06:36:13 +02:00
|
|
|
if fromPos.PDUPosition != toPos.PDUPosition || wantFullState {
|
2019-07-12 16:59:53 +02:00
|
|
|
joinedRoomIDs, err = d.addPDUDeltaToResponse(
|
2019-08-01 06:36:13 +02:00
|
|
|
ctx, device, fromPos.PDUPosition, toPos.PDUPosition, numRecentEventsPerRoom, wantFullState, res,
|
2019-07-12 16:59:53 +02:00
|
|
|
)
|
|
|
|
} else {
|
|
|
|
joinedRoomIDs, err = d.roomstate.selectRoomIDsWithMembership(
|
2019-08-06 16:07:36 +02:00
|
|
|
ctx, nil, device.UserID, gomatrixserverlib.Join,
|
2019-07-12 16:59:53 +02:00
|
|
|
)
|
|
|
|
}
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
err = d.addEDUDeltaToResponse(
|
|
|
|
fromPos, toPos, joinedRoomIDs, res,
|
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return res, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// getResponseWithPDUsForCompleteSync creates a response and adds all PDUs needed
|
|
|
|
// to it. It returns toPos and joinedRoomIDs for use of adding EDUs.
|
|
|
|
func (d *SyncServerDatasource) getResponseWithPDUsForCompleteSync(
|
|
|
|
ctx context.Context,
|
|
|
|
userID string,
|
|
|
|
numRecentEventsPerRoom int,
|
|
|
|
) (
|
|
|
|
res *types.Response,
|
|
|
|
toPos types.SyncPosition,
|
|
|
|
joinedRoomIDs []string,
|
|
|
|
err error,
|
|
|
|
) {
|
2017-04-13 17:56:46 +02:00
|
|
|
// This needs to be all done in a transaction as we need to do multiple SELECTs, and we need to have
|
2019-07-12 16:59:53 +02:00
|
|
|
// a consistent view of the database throughout. This includes extracting the sync position.
|
2017-05-15 16:18:08 +02:00
|
|
|
// This does have the unfortunate side-effect that all the matrixy logic resides in this function,
|
|
|
|
// but it's better to not hide the fact that this is being done in a transaction.
|
2017-09-19 17:22:02 +02:00
|
|
|
txn, err := d.db.BeginTx(ctx, &txReadOnlySnapshot)
|
|
|
|
if err != nil {
|
2019-07-12 16:59:53 +02:00
|
|
|
return
|
2017-09-19 17:22:02 +02:00
|
|
|
}
|
|
|
|
var succeeded bool
|
|
|
|
defer common.EndTransaction(txn, &succeeded)
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
// Get the current sync position which we will base the sync response on.
|
|
|
|
toPos, err = d.syncPositionTx(ctx, txn)
|
2017-09-19 17:22:02 +02:00
|
|
|
if err != nil {
|
2019-07-12 16:59:53 +02:00
|
|
|
return
|
2017-09-19 17:22:02 +02:00
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
res = types.NewResponse(toPos)
|
|
|
|
|
2017-09-19 17:22:02 +02:00
|
|
|
// Extract room state and recent events for all rooms the user is joined to.
|
2019-08-06 16:07:36 +02:00
|
|
|
joinedRoomIDs, err = d.roomstate.selectRoomIDsWithMembership(ctx, txn, userID, gomatrixserverlib.Join)
|
2017-09-19 17:22:02 +02:00
|
|
|
if err != nil {
|
2019-07-12 16:59:53 +02:00
|
|
|
return
|
2017-09-19 17:22:02 +02:00
|
|
|
}
|
|
|
|
|
2019-08-07 12:12:09 +02:00
|
|
|
stateFilterPart := gomatrixserverlib.DefaultFilterPart() // TODO: use filter provided in request
|
|
|
|
|
2017-09-19 17:22:02 +02:00
|
|
|
// Build up a /sync response. Add joined rooms.
|
2019-07-12 16:59:53 +02:00
|
|
|
for _, roomID := range joinedRoomIDs {
|
2017-09-19 17:22:02 +02:00
|
|
|
var stateEvents []gomatrixserverlib.Event
|
2019-08-07 12:12:09 +02:00
|
|
|
stateEvents, err = d.roomstate.selectCurrentState(ctx, txn, roomID, &stateFilterPart)
|
2017-04-13 17:56:46 +02:00
|
|
|
if err != nil {
|
2019-07-12 16:59:53 +02:00
|
|
|
return
|
2017-04-13 17:56:46 +02:00
|
|
|
}
|
2017-09-19 17:22:02 +02:00
|
|
|
// TODO: When filters are added, we may need to call this multiple times to get enough events.
|
|
|
|
// See: https://github.com/matrix-org/synapse/blob/v0.19.3/synapse/handlers/sync.py#L316
|
|
|
|
var recentStreamEvents []streamEvent
|
|
|
|
recentStreamEvents, err = d.events.selectRecentEvents(
|
2019-07-12 16:59:53 +02:00
|
|
|
ctx, txn, roomID, 0, toPos.PDUPosition, numRecentEventsPerRoom,
|
2017-09-19 17:22:02 +02:00
|
|
|
)
|
2017-04-13 17:56:46 +02:00
|
|
|
if err != nil {
|
2019-07-12 16:59:53 +02:00
|
|
|
return
|
2017-04-13 17:56:46 +02:00
|
|
|
}
|
2017-12-15 16:42:55 +01:00
|
|
|
|
|
|
|
// We don't include a device here as we don't need to send down
|
|
|
|
// transaction IDs for complete syncs
|
|
|
|
recentEvents := streamEventsToEvents(nil, recentStreamEvents)
|
2017-05-15 16:18:08 +02:00
|
|
|
|
2017-09-19 17:22:02 +02:00
|
|
|
stateEvents = removeDuplicates(stateEvents, recentEvents)
|
|
|
|
jr := types.NewJoinResponse()
|
2019-07-12 16:59:53 +02:00
|
|
|
if prevPDUPos := recentStreamEvents[0].streamPosition - 1; prevPDUPos > 0 {
|
|
|
|
// Use the short form of batch token for prev_batch
|
|
|
|
jr.Timeline.PrevBatch = strconv.FormatInt(prevPDUPos, 10)
|
2018-11-08 16:11:11 +01:00
|
|
|
} else {
|
2019-07-12 16:59:53 +02:00
|
|
|
// Use the short form of batch token for prev_batch
|
|
|
|
jr.Timeline.PrevBatch = "1"
|
2018-11-08 16:11:11 +01:00
|
|
|
}
|
2017-09-19 17:22:02 +02:00
|
|
|
jr.Timeline.Events = gomatrixserverlib.ToClientEvents(recentEvents, gomatrixserverlib.FormatSync)
|
|
|
|
jr.Timeline.Limited = true
|
|
|
|
jr.State.Events = gomatrixserverlib.ToClientEvents(stateEvents, gomatrixserverlib.FormatSync)
|
|
|
|
res.Rooms.Join[roomID] = *jr
|
|
|
|
}
|
2017-05-10 18:48:35 +02:00
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
if err = d.addInvitesToResponse(ctx, txn, userID, 0, toPos.PDUPosition, res); err != nil {
|
|
|
|
return
|
2017-09-19 17:22:02 +02:00
|
|
|
}
|
2017-05-15 18:41:54 +02:00
|
|
|
|
2017-09-19 17:22:02 +02:00
|
|
|
succeeded = true
|
2019-07-12 16:59:53 +02:00
|
|
|
return res, toPos, joinedRoomIDs, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// CompleteSync returns a complete /sync API response for the given user.
|
|
|
|
func (d *SyncServerDatasource) CompleteSync(
|
|
|
|
ctx context.Context, userID string, numRecentEventsPerRoom int,
|
|
|
|
) (*types.Response, error) {
|
|
|
|
res, toPos, joinedRoomIDs, err := d.getResponseWithPDUsForCompleteSync(
|
|
|
|
ctx, userID, numRecentEventsPerRoom,
|
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Use a zero value SyncPosition for fromPos so all EDU states are added.
|
|
|
|
err = d.addEDUDeltaToResponse(
|
|
|
|
types.SyncPosition{}, toPos, joinedRoomIDs, res,
|
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return res, nil
|
2017-09-19 17:22:02 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
var txReadOnlySnapshot = sql.TxOptions{
|
|
|
|
// Set the isolation level so that we see a snapshot of the database.
|
|
|
|
// In PostgreSQL repeatable read transactions will see a snapshot taken
|
|
|
|
// at the first query, and since the transaction is read-only it can't
|
|
|
|
// run into any serialisation errors.
|
|
|
|
// https://www.postgresql.org/docs/9.5/static/transaction-iso.html#XACT-REPEATABLE-READ
|
|
|
|
Isolation: sql.LevelRepeatableRead,
|
|
|
|
ReadOnly: true,
|
2017-04-13 17:56:46 +02:00
|
|
|
}
|
|
|
|
|
2017-08-02 17:21:35 +02:00
|
|
|
// GetAccountDataInRange returns all account data for a given user inserted or
|
|
|
|
// updated between two given positions
|
|
|
|
// Returns a map following the format data[roomID] = []dataTypes
|
|
|
|
// If no data is retrieved, returns an empty map
|
|
|
|
// If there was an issue with the retrieval, returns an error
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) GetAccountDataInRange(
|
|
|
|
ctx context.Context, userID string, oldPos, newPos int64,
|
2019-08-08 07:10:42 +02:00
|
|
|
accountDataFilterPart *gomatrixserverlib.FilterPart,
|
2017-08-02 17:21:35 +02:00
|
|
|
) (map[string][]string, error) {
|
2019-08-08 07:10:42 +02:00
|
|
|
return d.accountData.selectAccountDataInRange(ctx, userID, oldPos, newPos, accountDataFilterPart)
|
2017-08-02 17:21:35 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
// UpsertAccountData keeps track of new or updated account data, by saving the type
|
|
|
|
// of the new/updated data, and the user ID and room ID the data is related to (empty)
|
|
|
|
// room ID means the data isn't specific to any room)
|
|
|
|
// If no data with the given type, user ID and room ID exists in the database,
|
|
|
|
// creates a new row, else update the existing one
|
|
|
|
// Returns an error if there was an issue with the upsert
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) UpsertAccountData(
|
2017-09-18 17:52:22 +02:00
|
|
|
ctx context.Context, userID, roomID, dataType string,
|
2019-07-12 16:59:53 +02:00
|
|
|
) (int64, error) {
|
|
|
|
return d.accountData.insertAccountData(ctx, userID, roomID, dataType)
|
2017-08-02 17:21:35 +02:00
|
|
|
}
|
|
|
|
|
2017-09-20 16:36:41 +02:00
|
|
|
// AddInviteEvent stores a new invite event for a user.
|
|
|
|
// If the invite was successfully stored this returns the stream ID it was stored at.
|
|
|
|
// Returns an error if there was a problem communicating with the database.
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) AddInviteEvent(
|
2017-09-20 16:36:41 +02:00
|
|
|
ctx context.Context, inviteEvent gomatrixserverlib.Event,
|
2019-07-12 16:59:53 +02:00
|
|
|
) (int64, error) {
|
|
|
|
return d.invites.insertInviteEvent(ctx, inviteEvent)
|
2017-09-20 16:36:41 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
// RetireInviteEvent removes an old invite event from the database.
|
|
|
|
// Returns an error if there was a problem communicating with the database.
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) RetireInviteEvent(
|
2017-09-20 16:36:41 +02:00
|
|
|
ctx context.Context, inviteEventID string,
|
|
|
|
) error {
|
|
|
|
// TODO: Record that invite has been retired in a stream so that we can
|
|
|
|
// notify the user in an incremental sync.
|
|
|
|
err := d.invites.deleteInviteEvent(ctx, inviteEventID)
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) SetTypingTimeoutCallback(fn cache.TimeoutCallbackFn) {
|
|
|
|
d.typingCache.SetTimeoutCallback(fn)
|
|
|
|
}
|
|
|
|
|
|
|
|
// AddTypingUser adds a typing user to the typing cache.
|
|
|
|
// Returns the newly calculated sync position for typing notifications.
|
|
|
|
func (d *SyncServerDatasource) AddTypingUser(
|
|
|
|
userID, roomID string, expireTime *time.Time,
|
|
|
|
) int64 {
|
|
|
|
return d.typingCache.AddTypingUser(userID, roomID, expireTime)
|
|
|
|
}
|
|
|
|
|
|
|
|
// RemoveTypingUser removes a typing user from the typing cache.
|
|
|
|
// Returns the newly calculated sync position for typing notifications.
|
|
|
|
func (d *SyncServerDatasource) RemoveTypingUser(
|
|
|
|
userID, roomID string,
|
|
|
|
) int64 {
|
|
|
|
return d.typingCache.RemoveUser(userID, roomID)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (d *SyncServerDatasource) addInvitesToResponse(
|
2017-09-20 16:36:41 +02:00
|
|
|
ctx context.Context, txn *sql.Tx,
|
|
|
|
userID string,
|
2019-07-12 16:59:53 +02:00
|
|
|
fromPos, toPos int64,
|
2017-09-20 16:36:41 +02:00
|
|
|
res *types.Response,
|
|
|
|
) error {
|
|
|
|
invites, err := d.invites.selectInviteEventsInRange(
|
|
|
|
ctx, txn, userID, int64(fromPos), int64(toPos),
|
|
|
|
)
|
2017-05-15 18:41:54 +02:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2017-09-20 16:36:41 +02:00
|
|
|
for roomID, inviteEvent := range invites {
|
2017-05-15 18:41:54 +02:00
|
|
|
ir := types.NewInviteResponse()
|
2017-09-20 16:36:41 +02:00
|
|
|
ir.InviteState.Events = gomatrixserverlib.ToClientEvents(
|
|
|
|
[]gomatrixserverlib.Event{inviteEvent}, gomatrixserverlib.FormatSync,
|
|
|
|
)
|
|
|
|
// TODO: add the invite state from the invite event.
|
2017-05-15 18:41:54 +02:00
|
|
|
res.Rooms.Invite[roomID] = *ir
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-10-17 18:13:41 +02:00
|
|
|
// addRoomDeltaToResponse adds a room state delta to a sync response
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) addRoomDeltaToResponse(
|
2017-12-15 16:42:55 +01:00
|
|
|
ctx context.Context,
|
|
|
|
device *authtypes.Device,
|
|
|
|
txn *sql.Tx,
|
2019-07-12 16:59:53 +02:00
|
|
|
fromPos, toPos int64,
|
2017-10-17 18:13:41 +02:00
|
|
|
delta stateDelta,
|
|
|
|
numRecentEventsPerRoom int,
|
|
|
|
res *types.Response,
|
|
|
|
) error {
|
|
|
|
endPos := toPos
|
2019-08-06 16:07:36 +02:00
|
|
|
if delta.membershipPos > 0 && delta.membership == gomatrixserverlib.Leave {
|
2017-10-17 18:13:41 +02:00
|
|
|
// make sure we don't leak recent events after the leave event.
|
|
|
|
// TODO: History visibility makes this somewhat complex to handle correctly. For example:
|
|
|
|
// TODO: This doesn't work for join -> leave in a single /sync request (see events prior to join).
|
|
|
|
// TODO: This will fail on join -> leave -> sensitive msg -> join -> leave
|
|
|
|
// in a single /sync request
|
|
|
|
// This is all "okay" assuming history_visibility == "shared" which it is by default.
|
|
|
|
endPos = delta.membershipPos
|
|
|
|
}
|
|
|
|
recentStreamEvents, err := d.events.selectRecentEvents(
|
|
|
|
ctx, txn, delta.roomID, fromPos, endPos, numRecentEventsPerRoom,
|
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2017-12-15 16:42:55 +01:00
|
|
|
recentEvents := streamEventsToEvents(device, recentStreamEvents)
|
2017-10-17 18:13:41 +02:00
|
|
|
delta.stateEvents = removeDuplicates(delta.stateEvents, recentEvents) // roll back
|
|
|
|
|
2019-08-01 06:36:13 +02:00
|
|
|
var prevPDUPos int64
|
|
|
|
|
|
|
|
if len(recentEvents) == 0 {
|
|
|
|
if len(delta.stateEvents) == 0 {
|
|
|
|
// Don't bother appending empty room entries
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// If full_state=true and since is already up to date, then we'll have
|
|
|
|
// state events but no recent events.
|
|
|
|
prevPDUPos = toPos - 1
|
|
|
|
} else {
|
|
|
|
prevPDUPos = recentStreamEvents[0].streamPosition - 1
|
|
|
|
}
|
|
|
|
|
|
|
|
if prevPDUPos <= 0 {
|
|
|
|
prevPDUPos = 1
|
2017-10-18 11:36:45 +02:00
|
|
|
}
|
|
|
|
|
2017-10-17 18:13:41 +02:00
|
|
|
switch delta.membership {
|
2019-08-06 16:07:36 +02:00
|
|
|
case gomatrixserverlib.Join:
|
2017-10-17 18:13:41 +02:00
|
|
|
jr := types.NewJoinResponse()
|
2019-08-01 06:36:13 +02:00
|
|
|
// Use the short form of batch token for prev_batch
|
|
|
|
jr.Timeline.PrevBatch = strconv.FormatInt(prevPDUPos, 10)
|
2017-10-17 18:13:41 +02:00
|
|
|
jr.Timeline.Events = gomatrixserverlib.ToClientEvents(recentEvents, gomatrixserverlib.FormatSync)
|
|
|
|
jr.Timeline.Limited = false // TODO: if len(events) >= numRecents + 1 and then set limited:true
|
|
|
|
jr.State.Events = gomatrixserverlib.ToClientEvents(delta.stateEvents, gomatrixserverlib.FormatSync)
|
|
|
|
res.Rooms.Join[delta.roomID] = *jr
|
2019-08-06 16:07:36 +02:00
|
|
|
case gomatrixserverlib.Leave:
|
2017-10-17 18:13:41 +02:00
|
|
|
fallthrough // transitions to leave are the same as ban
|
2019-08-06 16:07:36 +02:00
|
|
|
case gomatrixserverlib.Ban:
|
2017-10-17 18:13:41 +02:00
|
|
|
// TODO: recentEvents may contain events that this user is not allowed to see because they are
|
|
|
|
// no longer in the room.
|
|
|
|
lr := types.NewLeaveResponse()
|
2019-08-01 06:36:13 +02:00
|
|
|
// Use the short form of batch token for prev_batch
|
|
|
|
lr.Timeline.PrevBatch = strconv.FormatInt(prevPDUPos, 10)
|
2017-10-17 18:13:41 +02:00
|
|
|
lr.Timeline.Events = gomatrixserverlib.ToClientEvents(recentEvents, gomatrixserverlib.FormatSync)
|
|
|
|
lr.Timeline.Limited = false // TODO: if len(events) >= numRecents + 1 and then set limited:true
|
|
|
|
lr.State.Events = gomatrixserverlib.ToClientEvents(delta.stateEvents, gomatrixserverlib.FormatSync)
|
|
|
|
res.Rooms.Leave[delta.roomID] = *lr
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-06-05 11:37:04 +02:00
|
|
|
// fetchStateEvents converts the set of event IDs into a set of events. It will fetch any which are missing from the database.
|
|
|
|
// Returns a map of room ID to list of events.
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) fetchStateEvents(
|
2017-09-18 17:52:22 +02:00
|
|
|
ctx context.Context, txn *sql.Tx,
|
|
|
|
roomIDToEventIDSet map[string]map[string]bool,
|
|
|
|
eventIDToEvent map[string]streamEvent,
|
|
|
|
) (map[string][]streamEvent, error) {
|
2017-06-05 11:37:04 +02:00
|
|
|
stateBetween := make(map[string][]streamEvent)
|
|
|
|
missingEvents := make(map[string][]string)
|
|
|
|
for roomID, ids := range roomIDToEventIDSet {
|
|
|
|
events := stateBetween[roomID]
|
|
|
|
for id, need := range ids {
|
|
|
|
if !need {
|
|
|
|
continue // deleted state
|
|
|
|
}
|
|
|
|
e, ok := eventIDToEvent[id]
|
|
|
|
if ok {
|
|
|
|
events = append(events, e)
|
|
|
|
} else {
|
|
|
|
m := missingEvents[roomID]
|
|
|
|
m = append(m, id)
|
|
|
|
missingEvents[roomID] = m
|
|
|
|
}
|
|
|
|
}
|
|
|
|
stateBetween[roomID] = events
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(missingEvents) > 0 {
|
|
|
|
// This happens when add_state_ids has an event ID which is not in the provided range.
|
|
|
|
// We need to explicitly fetch them.
|
|
|
|
allMissingEventIDs := []string{}
|
|
|
|
for _, missingEvIDs := range missingEvents {
|
|
|
|
allMissingEventIDs = append(allMissingEventIDs, missingEvIDs...)
|
|
|
|
}
|
2017-09-18 17:52:22 +02:00
|
|
|
evs, err := d.fetchMissingStateEvents(ctx, txn, allMissingEventIDs)
|
2017-06-05 11:37:04 +02:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
// we know we got them all otherwise an error would've been returned, so just loop the events
|
|
|
|
for _, ev := range evs {
|
|
|
|
roomID := ev.RoomID()
|
|
|
|
stateBetween[roomID] = append(stateBetween[roomID], ev)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return stateBetween, nil
|
|
|
|
}
|
|
|
|
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) fetchMissingStateEvents(
|
2017-09-18 17:52:22 +02:00
|
|
|
ctx context.Context, txn *sql.Tx, eventIDs []string,
|
|
|
|
) ([]streamEvent, error) {
|
2017-06-07 17:35:41 +02:00
|
|
|
// Fetch from the events table first so we pick up the stream ID for the
|
|
|
|
// event.
|
2017-09-18 17:52:22 +02:00
|
|
|
events, err := d.events.selectEvents(ctx, txn, eventIDs)
|
2017-06-07 17:35:41 +02:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
have := map[string]bool{}
|
|
|
|
for _, event := range events {
|
|
|
|
have[event.EventID()] = true
|
|
|
|
}
|
|
|
|
var missing []string
|
|
|
|
for _, eventID := range eventIDs {
|
|
|
|
if !have[eventID] {
|
|
|
|
missing = append(missing, eventID)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if len(missing) == 0 {
|
|
|
|
return events, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// If they are missing from the events table then they should be state
|
|
|
|
// events that we received from outside the main event stream.
|
|
|
|
// These should be in the room state table.
|
2017-09-18 17:52:22 +02:00
|
|
|
stateEvents, err := d.roomstate.selectEventsWithEventIDs(ctx, txn, missing)
|
2017-06-07 17:35:41 +02:00
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
if len(stateEvents) != len(missing) {
|
|
|
|
return nil, fmt.Errorf("failed to map all event IDs to events: (got %d, wanted %d)", len(stateEvents), len(missing))
|
|
|
|
}
|
2017-09-20 15:15:38 +02:00
|
|
|
events = append(events, stateEvents...)
|
2017-06-07 17:35:41 +02:00
|
|
|
return events, nil
|
|
|
|
}
|
|
|
|
|
2019-07-31 15:20:11 +02:00
|
|
|
// getStateDeltas returns the state deltas between fromPos and toPos,
|
|
|
|
// exclusive of oldPos, inclusive of newPos, for the rooms in which
|
|
|
|
// the user has new membership events.
|
|
|
|
// A list of joined room IDs is also returned in case the caller needs it.
|
2019-07-12 16:59:53 +02:00
|
|
|
func (d *SyncServerDatasource) getStateDeltas(
|
2017-12-15 16:42:55 +01:00
|
|
|
ctx context.Context, device *authtypes.Device, txn *sql.Tx,
|
2019-07-12 16:59:53 +02:00
|
|
|
fromPos, toPos int64, userID string,
|
2019-08-07 12:12:09 +02:00
|
|
|
stateFilterPart *gomatrixserverlib.FilterPart,
|
2019-07-31 15:20:11 +02:00
|
|
|
) ([]stateDelta, []string, error) {
|
2017-05-17 11:25:59 +02:00
|
|
|
// Implement membership change algorithm: https://github.com/matrix-org/synapse/blob/v0.19.3/synapse/handlers/sync.py#L821
|
|
|
|
// - Get membership list changes for this user in this sync response
|
|
|
|
// - For each room which has membership list changes:
|
|
|
|
// * Check if the room is 'newly joined' (insufficient to just check for a join event because we allow dupe joins TODO).
|
|
|
|
// If it is, then we need to send the full room state down (and 'limited' is always true).
|
|
|
|
// * Check if user is still CURRENTLY invited to the room. If so, add room to 'invited' block.
|
|
|
|
// * Check if the user is CURRENTLY (TODO) left/banned. If so, add room to 'archived' block.
|
|
|
|
// - Get all CURRENTLY joined rooms, and add them to 'joined' block.
|
|
|
|
var deltas []stateDelta
|
|
|
|
|
|
|
|
// get all the state events ever between these two positions
|
2019-08-07 12:12:09 +02:00
|
|
|
stateNeeded, eventMap, err := d.events.selectStateInRange(ctx, txn, fromPos, toPos, stateFilterPart)
|
2017-05-17 11:25:59 +02:00
|
|
|
if err != nil {
|
2019-07-31 15:20:11 +02:00
|
|
|
return nil, nil, err
|
2017-05-17 11:25:59 +02:00
|
|
|
}
|
2017-09-18 17:52:22 +02:00
|
|
|
state, err := d.fetchStateEvents(ctx, txn, stateNeeded, eventMap)
|
2017-06-05 11:37:04 +02:00
|
|
|
if err != nil {
|
2019-07-31 15:20:11 +02:00
|
|
|
return nil, nil, err
|
2017-06-05 11:37:04 +02:00
|
|
|
}
|
|
|
|
|
2017-05-17 17:21:27 +02:00
|
|
|
for roomID, stateStreamEvents := range state {
|
|
|
|
for _, ev := range stateStreamEvents {
|
2017-05-17 11:25:59 +02:00
|
|
|
// TODO: Currently this will incorrectly add rooms which were ALREADY joined but they sent another no-op join event.
|
|
|
|
// We should be checking if the user was already joined at fromPos and not proceed if so. As a result of this,
|
|
|
|
// dupe join events will result in the entire room state coming down to the client again. This is added in
|
|
|
|
// the 'state' part of the response though, so is transparent modulo bandwidth concerns as it is not added to
|
|
|
|
// the timeline.
|
2017-05-17 17:21:27 +02:00
|
|
|
if membership := getMembershipFromEvent(&ev.Event, userID); membership != "" {
|
2019-08-06 16:07:36 +02:00
|
|
|
if membership == gomatrixserverlib.Join {
|
2017-05-17 11:25:59 +02:00
|
|
|
// send full room state down instead of a delta
|
2019-08-01 06:36:13 +02:00
|
|
|
var s []streamEvent
|
2019-08-07 12:12:09 +02:00
|
|
|
s, err = d.currentStateStreamEventsForRoom(ctx, txn, roomID, stateFilterPart)
|
2017-05-17 11:25:59 +02:00
|
|
|
if err != nil {
|
2019-07-31 15:20:11 +02:00
|
|
|
return nil, nil, err
|
2017-05-17 11:25:59 +02:00
|
|
|
}
|
2017-05-17 17:21:27 +02:00
|
|
|
state[roomID] = s
|
2017-05-17 11:25:59 +02:00
|
|
|
continue // we'll add this room in when we do joined rooms
|
|
|
|
}
|
|
|
|
|
|
|
|
deltas = append(deltas, stateDelta{
|
2017-05-17 17:21:27 +02:00
|
|
|
membership: membership,
|
|
|
|
membershipPos: ev.streamPosition,
|
2017-12-15 16:42:55 +01:00
|
|
|
stateEvents: streamEventsToEvents(device, stateStreamEvents),
|
2017-05-17 17:21:27 +02:00
|
|
|
roomID: roomID,
|
2017-05-17 11:25:59 +02:00
|
|
|
})
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Add in currently joined rooms
|
2019-08-06 16:07:36 +02:00
|
|
|
joinedRoomIDs, err := d.roomstate.selectRoomIDsWithMembership(ctx, txn, userID, gomatrixserverlib.Join)
|
2017-05-17 11:25:59 +02:00
|
|
|
if err != nil {
|
2019-07-31 15:20:11 +02:00
|
|
|
return nil, nil, err
|
2017-05-17 11:25:59 +02:00
|
|
|
}
|
|
|
|
for _, joinedRoomID := range joinedRoomIDs {
|
|
|
|
deltas = append(deltas, stateDelta{
|
2019-08-06 16:07:36 +02:00
|
|
|
membership: gomatrixserverlib.Join,
|
2017-12-15 16:42:55 +01:00
|
|
|
stateEvents: streamEventsToEvents(device, state[joinedRoomID]),
|
2017-05-17 11:25:59 +02:00
|
|
|
roomID: joinedRoomID,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2019-07-31 15:20:11 +02:00
|
|
|
return deltas, joinedRoomIDs, nil
|
2017-05-17 11:25:59 +02:00
|
|
|
}
|
|
|
|
|
2019-08-01 06:36:13 +02:00
|
|
|
// getStateDeltasForFullStateSync is a variant of getStateDeltas used for /sync
|
|
|
|
// requests with full_state=true.
|
|
|
|
// Fetches full state for all joined rooms and uses selectStateInRange to get
|
|
|
|
// updates for other rooms.
|
|
|
|
func (d *SyncServerDatasource) getStateDeltasForFullStateSync(
|
|
|
|
ctx context.Context, device *authtypes.Device, txn *sql.Tx,
|
|
|
|
fromPos, toPos int64, userID string,
|
2019-08-07 12:12:09 +02:00
|
|
|
stateFilterPart *gomatrixserverlib.FilterPart,
|
2019-08-01 06:36:13 +02:00
|
|
|
) ([]stateDelta, []string, error) {
|
2019-08-06 16:07:36 +02:00
|
|
|
joinedRoomIDs, err := d.roomstate.selectRoomIDsWithMembership(ctx, txn, userID, gomatrixserverlib.Join)
|
2019-08-01 06:36:13 +02:00
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Use a reasonable initial capacity
|
|
|
|
deltas := make([]stateDelta, 0, len(joinedRoomIDs))
|
|
|
|
|
|
|
|
// Add full states for all joined rooms
|
|
|
|
for _, joinedRoomID := range joinedRoomIDs {
|
2019-08-07 12:12:09 +02:00
|
|
|
s, stateErr := d.currentStateStreamEventsForRoom(ctx, txn, joinedRoomID, stateFilterPart)
|
2019-08-01 06:36:13 +02:00
|
|
|
if stateErr != nil {
|
|
|
|
return nil, nil, stateErr
|
|
|
|
}
|
|
|
|
deltas = append(deltas, stateDelta{
|
2019-08-06 16:07:36 +02:00
|
|
|
membership: gomatrixserverlib.Join,
|
2019-08-01 06:36:13 +02:00
|
|
|
stateEvents: streamEventsToEvents(device, s),
|
|
|
|
roomID: joinedRoomID,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
// Get all the state events ever between these two positions
|
2019-08-07 12:12:09 +02:00
|
|
|
stateNeeded, eventMap, err := d.events.selectStateInRange(ctx, txn, fromPos, toPos, stateFilterPart)
|
2019-08-01 06:36:13 +02:00
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
state, err := d.fetchStateEvents(ctx, txn, stateNeeded, eventMap)
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
for roomID, stateStreamEvents := range state {
|
|
|
|
for _, ev := range stateStreamEvents {
|
|
|
|
if membership := getMembershipFromEvent(&ev.Event, userID); membership != "" {
|
2019-08-06 16:07:36 +02:00
|
|
|
if membership != gomatrixserverlib.Join { // We've already added full state for all joined rooms above.
|
2019-08-01 06:36:13 +02:00
|
|
|
deltas = append(deltas, stateDelta{
|
|
|
|
membership: membership,
|
|
|
|
membershipPos: ev.streamPosition,
|
|
|
|
stateEvents: streamEventsToEvents(device, stateStreamEvents),
|
|
|
|
roomID: roomID,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return deltas, joinedRoomIDs, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (d *SyncServerDatasource) currentStateStreamEventsForRoom(
|
|
|
|
ctx context.Context, txn *sql.Tx, roomID string,
|
2019-08-07 12:12:09 +02:00
|
|
|
stateFilterPart *gomatrixserverlib.FilterPart,
|
2019-08-01 06:36:13 +02:00
|
|
|
) ([]streamEvent, error) {
|
2019-08-07 12:12:09 +02:00
|
|
|
allState, err := d.roomstate.selectCurrentState(ctx, txn, roomID, stateFilterPart)
|
2019-08-01 06:36:13 +02:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
s := make([]streamEvent, len(allState))
|
|
|
|
for i := 0; i < len(s); i++ {
|
|
|
|
s[i] = streamEvent{Event: allState[i], streamPosition: 0}
|
|
|
|
}
|
|
|
|
return s, nil
|
|
|
|
}
|
|
|
|
|
2017-12-15 16:42:55 +01:00
|
|
|
// streamEventsToEvents converts streamEvent to Event. If device is non-nil and
|
|
|
|
// matches the streamevent.transactionID device then the transaction ID gets
|
|
|
|
// added to the unsigned section of the output event.
|
|
|
|
func streamEventsToEvents(device *authtypes.Device, in []streamEvent) []gomatrixserverlib.Event {
|
2017-05-17 17:21:27 +02:00
|
|
|
out := make([]gomatrixserverlib.Event, len(in))
|
|
|
|
for i := 0; i < len(in); i++ {
|
|
|
|
out[i] = in[i].Event
|
2017-12-15 16:42:55 +01:00
|
|
|
if device != nil && in[i].transactionID != nil {
|
2019-08-23 18:55:40 +02:00
|
|
|
if device.UserID == in[i].Sender() && device.SessionID == in[i].transactionID.SessionID {
|
2017-12-15 16:42:55 +01:00
|
|
|
err := out[i].SetUnsignedField(
|
|
|
|
"transaction_id", in[i].transactionID.TransactionID,
|
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
logrus.WithFields(logrus.Fields{
|
|
|
|
"event_id": out[i].EventID(),
|
|
|
|
}).WithError(err).Warnf("Failed to add transaction ID to event")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2017-05-17 17:21:27 +02:00
|
|
|
}
|
|
|
|
return out
|
|
|
|
}
|
|
|
|
|
2017-05-11 16:51:35 +02:00
|
|
|
// There may be some overlap where events in stateEvents are already in recentEvents, so filter
|
|
|
|
// them out so we don't include them twice in the /sync response. They should be in recentEvents
|
|
|
|
// only, so clients get to the correct state once they have rolled forward.
|
|
|
|
func removeDuplicates(stateEvents, recentEvents []gomatrixserverlib.Event) []gomatrixserverlib.Event {
|
|
|
|
for _, recentEv := range recentEvents {
|
|
|
|
if recentEv.StateKey() == nil {
|
|
|
|
continue // not a state event
|
|
|
|
}
|
|
|
|
// TODO: This is a linear scan over all the current state events in this room. This will
|
|
|
|
// be slow for big rooms. We should instead sort the state events by event ID (ORDER BY)
|
|
|
|
// then do a binary search to find matching events, similar to what roomserver does.
|
|
|
|
for j := 0; j < len(stateEvents); j++ {
|
|
|
|
if stateEvents[j].EventID() == recentEv.EventID() {
|
|
|
|
// overwrite the element to remove with the last element then pop the last element.
|
|
|
|
// This is orders of magnitude faster than re-slicing, but doesn't preserve ordering
|
|
|
|
// (we don't care about the order of stateEvents)
|
|
|
|
stateEvents[j] = stateEvents[len(stateEvents)-1]
|
|
|
|
stateEvents = stateEvents[:len(stateEvents)-1]
|
|
|
|
break // there shouldn't be multiple events with the same event ID
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return stateEvents
|
|
|
|
}
|
|
|
|
|
2017-05-17 11:25:59 +02:00
|
|
|
// getMembershipFromEvent returns the value of content.membership iff the event is a state event
|
|
|
|
// with type 'm.room.member' and state_key of userID. Otherwise, an empty string is returned.
|
|
|
|
func getMembershipFromEvent(ev *gomatrixserverlib.Event, userID string) string {
|
|
|
|
if ev.Type() == "m.room.member" && ev.StateKeyEquals(userID) {
|
2017-07-07 15:11:32 +02:00
|
|
|
membership, err := ev.Membership()
|
|
|
|
if err != nil {
|
2017-05-17 11:25:59 +02:00
|
|
|
return ""
|
|
|
|
}
|
2017-07-07 15:11:32 +02:00
|
|
|
return membership
|
2017-05-17 11:25:59 +02:00
|
|
|
}
|
|
|
|
return ""
|
|
|
|
}
|