Compare commits

...

3 commits

Author SHA1 Message Date
Till Faelligen da215c2f1d
Avoid race conditions, remove unreachable code 2022-10-28 11:27:30 +02:00
Till Faelligen 2999559982
Async incremental sync 2022-10-28 10:06:38 +02:00
Till Faelligen 9d6c9e4cc3
Calculate initial sync async 2022-10-28 09:44:27 +02:00
5 changed files with 157 additions and 161 deletions

View file

@ -80,6 +80,7 @@ func (p *InviteStreamProvider) IncrementalSync(
if _, ok := req.Response.Rooms.Join[roomID]; ok { if _, ok := req.Response.Rooms.Join[roomID]; ok {
continue continue
} }
lr := types.NewLeaveResponse() lr := types.NewLeaveResponse()
h := sha256.Sum256(append([]byte(roomID), []byte(strconv.FormatInt(int64(to), 10))...)) h := sha256.Sum256(append([]byte(roomID), []byte(strconv.FormatInt(int64(to), 10))...))
lr.Timeline.Events = append(lr.Timeline.Events, gomatrixserverlib.ClientEvent{ lr.Timeline.Events = append(lr.Timeline.Events, gomatrixserverlib.ClientEvent{

View file

@ -2,6 +2,7 @@ package streams
import ( import (
"context" "context"
"fmt"
"github.com/matrix-org/dendrite/internal/caching" "github.com/matrix-org/dendrite/internal/caching"
"github.com/matrix-org/dendrite/internal/sqlutil" "github.com/matrix-org/dendrite/internal/sqlutil"
@ -103,3 +104,53 @@ func (s *Streams) Latest(ctx context.Context) types.StreamingToken {
PresencePosition: s.PresenceStreamProvider.LatestPosition(ctx), PresencePosition: s.PresenceStreamProvider.LatestPosition(ctx),
} }
} }
func ToToken(provider StreamProvider, position types.StreamPosition) types.StreamingToken {
switch t := provider.(type) {
case *PDUStreamProvider:
return types.StreamingToken{PDUPosition: position}
case *TypingStreamProvider:
return types.StreamingToken{TypingPosition: position}
case *ReceiptStreamProvider:
return types.StreamingToken{ReceiptPosition: position}
case *SendToDeviceStreamProvider:
return types.StreamingToken{SendToDevicePosition: position}
case *InviteStreamProvider:
return types.StreamingToken{InvitePosition: position}
case *AccountDataStreamProvider:
return types.StreamingToken{AccountDataPosition: position}
case *DeviceListStreamProvider:
return types.StreamingToken{DeviceListPosition: position}
case *NotificationDataStreamProvider:
return types.StreamingToken{NotificationDataPosition: position}
case *PresenceStreamProvider:
return types.StreamingToken{PresencePosition: position}
default:
panic(fmt.Sprintf("unknown stream provider: %T", t))
}
}
func IncrementalPositions(provider StreamProvider, current, since types.StreamingToken) (types.StreamPosition, types.StreamPosition) {
switch t := provider.(type) {
case *PDUStreamProvider:
return current.PDUPosition, since.PDUPosition
case *TypingStreamProvider:
return current.TypingPosition, since.TypingPosition
case *ReceiptStreamProvider:
return current.ReceiptPosition, since.ReceiptPosition
case *SendToDeviceStreamProvider:
return current.SendToDevicePosition, since.SendToDevicePosition
case *InviteStreamProvider:
return current.InvitePosition, since.InvitePosition
case *AccountDataStreamProvider:
return current.AccountDataPosition, since.AccountDataPosition
case *DeviceListStreamProvider:
return current.DeviceListPosition, since.DeviceListPosition
case *NotificationDataStreamProvider:
return current.NotificationDataPosition, since.NotificationDataPosition
case *PresenceStreamProvider:
return current.PresencePosition, since.PresencePosition
default:
panic(fmt.Sprintf("unknown stream provider: %T", t))
}
}

View file

@ -21,6 +21,7 @@ import (
"math" "math"
"net/http" "net/http"
"strconv" "strconv"
"sync"
"time" "time"
"github.com/matrix-org/gomatrixserverlib" "github.com/matrix-org/gomatrixserverlib"
@ -101,6 +102,7 @@ func newSyncRequest(req *http.Request, device userapi.Device, syncDB storage.Dat
Rooms: make(map[string]string), // Populated by the PDU stream Rooms: make(map[string]string), // Populated by the PDU stream
WantFullState: wantFullState, // WantFullState: wantFullState, //
MembershipChanges: make(map[string]struct{}), // Populated by the PDU stream MembershipChanges: make(map[string]struct{}), // Populated by the PDU stream
SyncMu: &sync.Mutex{},
}, nil }, nil
} }

View file

@ -225,6 +225,12 @@ var waitingSyncRequests = prometheus.NewGauge(
}, },
) )
// streamPosResponse is the response from a goroutine
type streamPosResponse struct {
provider streams.StreamProvider
pos types.StreamPosition
}
// OnIncomingSyncRequest is called when a client makes a /sync request. This function MUST be // 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 // called in a dedicated goroutine for this request. This function will block the goroutine
// until a response is ready, or it times out. // until a response is ready, or it times out.
@ -307,182 +313,116 @@ func (rp *RequestPool) OnIncomingSyncRequest(req *http.Request, device *userapi.
} }
withTransaction := func(from types.StreamPosition, f func(snapshot storage.DatabaseTransaction) types.StreamPosition) types.StreamPosition { withTransaction := func(from types.StreamPosition, f func(snapshot storage.DatabaseTransaction) types.StreamPosition) types.StreamPosition {
if err := req.Context().Err(); err != nil {
return from
}
var succeeded bool var succeeded bool
snapshot, err := rp.db.NewDatabaseSnapshot(req.Context()) snapshot, err := rp.db.NewDatabaseSnapshot(req.Context())
if err != nil { if err != nil {
logrus.WithError(err).Error("Failed to acquire database snapshot for sync request") syncReq.Log.WithError(err).Error("Failed to acquire database snapshot for sync request")
return from return from
} }
defer func() { defer func() {
succeeded = err == nil succeeded = err == nil
sqlutil.EndTransactionWithCheck(snapshot, &succeeded, &err) sqlutil.EndTransactionWithCheck(snapshot, &succeeded, &err)
}() }()
syncReq.SyncMu.Lock()
defer syncReq.SyncMu.Unlock()
return f(snapshot) return f(snapshot)
} }
allStreams := []streams.StreamProvider{
rp.streams.DeviceListStreamProvider,
rp.streams.TypingStreamProvider,
rp.streams.ReceiptStreamProvider,
rp.streams.InviteStreamProvider,
rp.streams.SendToDeviceStreamProvider,
rp.streams.AccountDataStreamProvider,
rp.streams.NotificationDataStreamProvider,
rp.streams.PresenceStreamProvider,
}
if syncReq.Since.IsEmpty() { if syncReq.Since.IsEmpty() {
// Complete sync // Complete sync
syncReq.Response.NextBatch = types.StreamingToken{ // The PDU stream needs to be the very first stream to get the data,
// Get the current DeviceListPosition first, as the currentPosition // as it sets values the other streams need
// might advance while processing other streams, resulting in flakey pduPos := withTransaction(
// tests. 0,
DeviceListPosition: withTransaction( func(txn storage.DatabaseTransaction) types.StreamPosition {
syncReq.Since.DeviceListPosition, return rp.streams.PDUStreamProvider.CompleteSync(
func(txn storage.DatabaseTransaction) types.StreamPosition { syncReq.Context, txn, syncReq,
return rp.streams.DeviceListStreamProvider.CompleteSync( )
syncReq.Context, txn, syncReq, },
) )
}, syncReq.Response.NextBatch.PDUPosition = pduPos
),
PDUPosition: withTransaction( streamPosCh := make(chan streamPosResponse, len(allStreams))
syncReq.Since.PDUPosition, wg := sync.WaitGroup{}
func(txn storage.DatabaseTransaction) types.StreamPosition { wg.Add(len(allStreams))
return rp.streams.PDUStreamProvider.CompleteSync(
syncReq.Context, txn, syncReq, // fan out stream calculations
) for _, s := range allStreams {
}, go func(stream streams.StreamProvider) {
), streamPos := withTransaction(
TypingPosition: withTransaction( 0, // we're doing an initial sync
syncReq.Since.TypingPosition, func(txn storage.DatabaseTransaction) types.StreamPosition {
func(txn storage.DatabaseTransaction) types.StreamPosition { return stream.CompleteSync(
return rp.streams.TypingStreamProvider.CompleteSync( syncReq.Context, txn, syncReq,
syncReq.Context, txn, syncReq, )
) },
}, )
), streamPosCh <- streamPosResponse{provider: stream, pos: streamPos}
ReceiptPosition: withTransaction( wg.Done()
syncReq.Since.ReceiptPosition, }(s)
func(txn storage.DatabaseTransaction) types.StreamPosition { }
return rp.streams.ReceiptStreamProvider.CompleteSync( // Wait for all streams to finish their work
syncReq.Context, txn, syncReq, wg.Wait()
) close(streamPosCh)
}, for resp := range streamPosCh {
), syncReq.Response.NextBatch.ApplyUpdates(streams.ToToken(resp.provider, resp.pos))
InvitePosition: withTransaction(
syncReq.Since.InvitePosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.InviteStreamProvider.CompleteSync(
syncReq.Context, txn, syncReq,
)
},
),
SendToDevicePosition: withTransaction(
syncReq.Since.SendToDevicePosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.SendToDeviceStreamProvider.CompleteSync(
syncReq.Context, txn, syncReq,
)
},
),
AccountDataPosition: withTransaction(
syncReq.Since.AccountDataPosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.AccountDataStreamProvider.CompleteSync(
syncReq.Context, txn, syncReq,
)
},
),
NotificationDataPosition: withTransaction(
syncReq.Since.NotificationDataPosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.NotificationDataStreamProvider.CompleteSync(
syncReq.Context, txn, syncReq,
)
},
),
PresencePosition: withTransaction(
syncReq.Since.PresencePosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.PresenceStreamProvider.CompleteSync(
syncReq.Context, txn, syncReq,
)
},
),
} }
} else { } else {
// Incremental sync // Incremental sync
syncReq.Response.NextBatch = types.StreamingToken{ // The PDU stream needs to be the very first stream to get the data,
PDUPosition: withTransaction( // as it sets values the other streams need
syncReq.Since.PDUPosition, current, since := streams.IncrementalPositions(rp.streams.PDUStreamProvider, rp.Notifier.CurrentPosition(), syncReq.Since)
func(txn storage.DatabaseTransaction) types.StreamPosition { pduPos := withTransaction(
return rp.streams.PDUStreamProvider.IncrementalSync( since,
syncReq.Context, txn, syncReq, func(txn storage.DatabaseTransaction) types.StreamPosition {
syncReq.Since.PDUPosition, rp.Notifier.CurrentPosition().PDUPosition, return rp.streams.PDUStreamProvider.IncrementalSync(
) syncReq.Context, txn, syncReq,
}, since, current,
), )
TypingPosition: withTransaction( },
syncReq.Since.TypingPosition, )
func(txn storage.DatabaseTransaction) types.StreamPosition { syncReq.Response.NextBatch.PDUPosition = pduPos
return rp.streams.TypingStreamProvider.IncrementalSync(
syncReq.Context, txn, syncReq, streamPosCh := make(chan streamPosResponse, len(allStreams))
syncReq.Since.TypingPosition, rp.Notifier.CurrentPosition().TypingPosition, wg := sync.WaitGroup{}
) wg.Add(len(allStreams))
},
), // fan out stream calculations
ReceiptPosition: withTransaction( for _, s := range allStreams {
syncReq.Since.ReceiptPosition, go func(stream streams.StreamProvider) {
func(txn storage.DatabaseTransaction) types.StreamPosition { current, since := streams.IncrementalPositions(stream, rp.Notifier.CurrentPosition(), syncReq.Since)
return rp.streams.ReceiptStreamProvider.IncrementalSync( streamPos := withTransaction(
syncReq.Context, txn, syncReq, since,
syncReq.Since.ReceiptPosition, rp.Notifier.CurrentPosition().ReceiptPosition, func(txn storage.DatabaseTransaction) types.StreamPosition {
) return stream.IncrementalSync(
}, syncReq.Context, txn, syncReq,
), since, current,
InvitePosition: withTransaction( )
syncReq.Since.InvitePosition, },
func(txn storage.DatabaseTransaction) types.StreamPosition { )
return rp.streams.InviteStreamProvider.IncrementalSync( streamPosCh <- streamPosResponse{provider: stream, pos: streamPos}
syncReq.Context, txn, syncReq, wg.Done()
syncReq.Since.InvitePosition, rp.Notifier.CurrentPosition().InvitePosition, }(s)
) }
}, // Wait for all streams to finish their work
), wg.Wait()
SendToDevicePosition: withTransaction( close(streamPosCh)
syncReq.Since.SendToDevicePosition, for resp := range streamPosCh {
func(txn storage.DatabaseTransaction) types.StreamPosition { syncReq.Response.NextBatch.ApplyUpdates(streams.ToToken(resp.provider, resp.pos))
return rp.streams.SendToDeviceStreamProvider.IncrementalSync(
syncReq.Context, txn, syncReq,
syncReq.Since.SendToDevicePosition, rp.Notifier.CurrentPosition().SendToDevicePosition,
)
},
),
AccountDataPosition: withTransaction(
syncReq.Since.AccountDataPosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.AccountDataStreamProvider.IncrementalSync(
syncReq.Context, txn, syncReq,
syncReq.Since.AccountDataPosition, rp.Notifier.CurrentPosition().AccountDataPosition,
)
},
),
NotificationDataPosition: withTransaction(
syncReq.Since.NotificationDataPosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.NotificationDataStreamProvider.IncrementalSync(
syncReq.Context, txn, syncReq,
syncReq.Since.NotificationDataPosition, rp.Notifier.CurrentPosition().NotificationDataPosition,
)
},
),
DeviceListPosition: withTransaction(
syncReq.Since.DeviceListPosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.DeviceListStreamProvider.IncrementalSync(
syncReq.Context, txn, syncReq,
syncReq.Since.DeviceListPosition, rp.Notifier.CurrentPosition().DeviceListPosition,
)
},
),
PresencePosition: withTransaction(
syncReq.Since.PresencePosition,
func(txn storage.DatabaseTransaction) types.StreamPosition {
return rp.streams.PresenceStreamProvider.IncrementalSync(
syncReq.Context, txn, syncReq,
syncReq.Since.PresencePosition, rp.Notifier.CurrentPosition().PresencePosition,
)
},
),
} }
// it's possible for there to be no updates for this user even though since < current pos, // 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 // e.g busy servers with a quiet user. In this scenario, we don't want to return a no-op
@ -544,7 +484,7 @@ func (rp *RequestPool) OnIncomingKeyChangeRequest(req *http.Request, device *use
} }
snapshot, err := rp.db.NewDatabaseSnapshot(req.Context()) snapshot, err := rp.db.NewDatabaseSnapshot(req.Context())
if err != nil { if err != nil {
logrus.WithError(err).Error("Failed to acquire database snapshot for key change") syncReq.Log.WithError(err).Error("Failed to acquire database snapshot for key change")
return jsonerror.InternalServerError() return jsonerror.InternalServerError()
} }
var succeeded bool var succeeded bool
@ -555,7 +495,7 @@ func (rp *RequestPool) OnIncomingKeyChangeRequest(req *http.Request, device *use
syncReq.Response, fromToken.DeviceListPosition, toToken.DeviceListPosition, syncReq.Response, fromToken.DeviceListPosition, toToken.DeviceListPosition,
) )
if err != nil { if err != nil {
util.GetLogger(req.Context()).WithError(err).Error("Failed to DeviceListCatchup info") syncReq.Log.WithError(err).Error("Failed to DeviceListCatchup info")
return jsonerror.InternalServerError() return jsonerror.InternalServerError()
} }
succeeded = true succeeded = true

View file

@ -2,6 +2,7 @@ package types
import ( import (
"context" "context"
"sync"
"time" "time"
"github.com/matrix-org/gomatrixserverlib" "github.com/matrix-org/gomatrixserverlib"
@ -26,6 +27,7 @@ type SyncRequest struct {
MembershipChanges map[string]struct{} MembershipChanges map[string]struct{}
// Updated by the PDU stream. // Updated by the PDU stream.
IgnoredUsers IgnoredUsers IgnoredUsers IgnoredUsers
SyncMu *sync.Mutex
} }
func (r *SyncRequest) IsRoomPresent(roomID string) bool { func (r *SyncRequest) IsRoomPresent(roomID string) bool {