mirror of
https://github.com/matrix-org/dendrite.git
synced 2025-12-07 23:13:11 -06:00
Merge pull request #34 from globekeeper/release/upstream-0.9.6
Release/upstream 0.9.6
This commit is contained in:
commit
8e6ae48f4a
20
CHANGES.md
20
CHANGES.md
|
|
@ -1,5 +1,25 @@
|
|||
# Changelog
|
||||
|
||||
## Dendrite 0.9.6 (2022-09-01)
|
||||
|
||||
### Features
|
||||
|
||||
* The appservice API has been refactored for improved performance and stability
|
||||
* The appservice database has been deprecated, as the roomserver output stream is now used as the data source instead
|
||||
* The `generate-config` tool has been updated to support additional scenarios, i.e. for CI configuration generation and generating both monolith and polylith skeleton config files
|
||||
|
||||
### Fixes
|
||||
|
||||
* The username length check has been fixed on new account creation
|
||||
* The length of the `type`, `sender`, `state_key` and `room_id` fields in events are now verified by number of codepoints rather than bytes, fixing the "Cat Overflow" bug
|
||||
* UTF-16 surrogate handling in the canonical JSON implementation has been fixed
|
||||
* A race condition when starting the keyserver has been fixed
|
||||
* A race condition when configuring HTTP servers and routing at startup has been fixed
|
||||
* A bug where the incorrect limit was used for lazy-loading memberships has been fixed
|
||||
* The number of push notifications will now be sent to the push gateway
|
||||
* A missing index causing slow performance on the sync API send-to-device table has been added (contributed by [PiotrKozimor](https://github.com/PiotrKozimor))
|
||||
* Event auth will now correctly check for the existence of the `"creator"` field in create events
|
||||
|
||||
## Dendrite 0.9.5 (2022-08-25)
|
||||
|
||||
### Fixes
|
||||
|
|
|
|||
|
|
@ -1,10 +0,0 @@
|
|||
# Application Service
|
||||
|
||||
This component interfaces with external [Application
|
||||
Services](https://matrix.org/docs/spec/application_service/unstable.html).
|
||||
This includes any HTTP endpoints that application services call, as well as talking
|
||||
to any HTTP endpoints that application services provide themselves.
|
||||
|
||||
## Consumers
|
||||
|
||||
This component consumes and filters events from the Roomserver Kafka stream, passing on any necessary events to subscribing application services.
|
||||
|
|
@ -18,7 +18,6 @@ import (
|
|||
"context"
|
||||
"crypto/tls"
|
||||
"net/http"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
|
|
@ -28,9 +27,6 @@ import (
|
|||
"github.com/matrix-org/dendrite/appservice/consumers"
|
||||
"github.com/matrix-org/dendrite/appservice/inthttp"
|
||||
"github.com/matrix-org/dendrite/appservice/query"
|
||||
"github.com/matrix-org/dendrite/appservice/storage"
|
||||
"github.com/matrix-org/dendrite/appservice/types"
|
||||
"github.com/matrix-org/dendrite/appservice/workers"
|
||||
roomserverAPI "github.com/matrix-org/dendrite/roomserver/api"
|
||||
"github.com/matrix-org/dendrite/setup/base"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
|
|
@ -59,57 +55,40 @@ func NewInternalAPI(
|
|||
Proxy: http.ProxyFromEnvironment,
|
||||
},
|
||||
}
|
||||
js, _ := base.NATS.Prepare(base.ProcessContext, &base.Cfg.Global.JetStream)
|
||||
// Create appserivce query API with an HTTP client that will be used for all
|
||||
// outbound and inbound requests (inbound only for the internal API)
|
||||
appserviceQueryAPI := &query.AppServiceQueryAPI{
|
||||
HTTPClient: client,
|
||||
Cfg: &base.Cfg.AppServiceAPI,
|
||||
}
|
||||
|
||||
// Create a connection to the appservice postgres DB
|
||||
appserviceDB, err := storage.NewDatabase(base, &base.Cfg.AppServiceAPI.Database)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Panicf("failed to connect to appservice db")
|
||||
if len(base.Cfg.Derived.ApplicationServices) == 0 {
|
||||
return appserviceQueryAPI
|
||||
}
|
||||
|
||||
// Wrap application services in a type that relates the application service and
|
||||
// a sync.Cond object that can be used to notify workers when there are new
|
||||
// events to be sent out.
|
||||
workerStates := make([]*types.ApplicationServiceWorkerState, len(base.Cfg.Derived.ApplicationServices))
|
||||
for i, appservice := range base.Cfg.Derived.ApplicationServices {
|
||||
m := sync.Mutex{}
|
||||
ws := types.ApplicationServiceWorkerState{
|
||||
AppService: appservice,
|
||||
Cond: sync.NewCond(&m),
|
||||
}
|
||||
workerStates[i] = &ws
|
||||
|
||||
for _, appservice := range base.Cfg.Derived.ApplicationServices {
|
||||
// Create bot account for this AS if it doesn't already exist
|
||||
if err = generateAppServiceAccount(userAPI, appservice); err != nil {
|
||||
if err := generateAppServiceAccount(userAPI, appservice); err != nil {
|
||||
logrus.WithFields(logrus.Fields{
|
||||
"appservice": appservice.ID,
|
||||
}).WithError(err).Panicf("failed to generate bot account for appservice")
|
||||
}
|
||||
}
|
||||
|
||||
// Create appserivce query API with an HTTP client that will be used for all
|
||||
// outbound and inbound requests (inbound only for the internal API)
|
||||
appserviceQueryAPI := &query.AppServiceQueryAPI{
|
||||
HTTPClient: client,
|
||||
Cfg: base.Cfg,
|
||||
}
|
||||
|
||||
// Only consume if we actually have ASes to track, else we'll just chew cycles needlessly.
|
||||
// We can't add ASes at runtime so this is safe to do.
|
||||
if len(workerStates) > 0 {
|
||||
js, _ := base.NATS.Prepare(base.ProcessContext, &base.Cfg.Global.JetStream)
|
||||
consumer := consumers.NewOutputRoomEventConsumer(
|
||||
base.ProcessContext, base.Cfg, js, appserviceDB,
|
||||
rsAPI, workerStates,
|
||||
base.ProcessContext, &base.Cfg.AppServiceAPI,
|
||||
client, js, rsAPI,
|
||||
)
|
||||
if err := consumer.Start(); err != nil {
|
||||
logrus.WithError(err).Panicf("failed to start appservice roomserver consumer")
|
||||
}
|
||||
}
|
||||
|
||||
// Create application service transaction workers
|
||||
if err := workers.SetupTransactionWorkers(client, appserviceDB, workerStates); err != nil {
|
||||
logrus.WithError(err).Panicf("failed to start app service transaction workers")
|
||||
}
|
||||
return appserviceQueryAPI
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -15,17 +15,22 @@
|
|||
package consumers
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"math"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"time"
|
||||
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
"github.com/nats-io/nats.go"
|
||||
|
||||
"github.com/matrix-org/dendrite/appservice/storage"
|
||||
"github.com/matrix-org/dendrite/appservice/types"
|
||||
"github.com/matrix-org/dendrite/roomserver/api"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
"github.com/matrix-org/dendrite/setup/jetstream"
|
||||
"github.com/matrix-org/dendrite/setup/process"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
"github.com/nats-io/nats.go"
|
||||
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
|
@ -33,65 +38,83 @@ import (
|
|||
// OutputRoomEventConsumer consumes events that originated in the room server.
|
||||
type OutputRoomEventConsumer struct {
|
||||
ctx context.Context
|
||||
cfg *config.AppServiceAPI
|
||||
client *http.Client
|
||||
jetstream nats.JetStreamContext
|
||||
durable string
|
||||
topic string
|
||||
asDB storage.Database
|
||||
rsAPI api.AppserviceRoomserverAPI
|
||||
serverName string
|
||||
workerStates []*types.ApplicationServiceWorkerState
|
||||
}
|
||||
|
||||
type appserviceState struct {
|
||||
*config.ApplicationService
|
||||
backoff int
|
||||
}
|
||||
|
||||
// NewOutputRoomEventConsumer creates a new OutputRoomEventConsumer. Call
|
||||
// Start() to begin consuming from room servers.
|
||||
func NewOutputRoomEventConsumer(
|
||||
process *process.ProcessContext,
|
||||
cfg *config.Dendrite,
|
||||
cfg *config.AppServiceAPI,
|
||||
client *http.Client,
|
||||
js nats.JetStreamContext,
|
||||
appserviceDB storage.Database,
|
||||
rsAPI api.AppserviceRoomserverAPI,
|
||||
workerStates []*types.ApplicationServiceWorkerState,
|
||||
) *OutputRoomEventConsumer {
|
||||
return &OutputRoomEventConsumer{
|
||||
ctx: process.Context(),
|
||||
cfg: cfg,
|
||||
client: client,
|
||||
jetstream: js,
|
||||
durable: cfg.Global.JetStream.Durable("AppserviceRoomserverConsumer"),
|
||||
topic: cfg.Global.JetStream.Prefixed(jetstream.OutputRoomEvent),
|
||||
asDB: appserviceDB,
|
||||
topic: cfg.Matrix.JetStream.Prefixed(jetstream.OutputRoomEvent),
|
||||
rsAPI: rsAPI,
|
||||
serverName: string(cfg.Global.ServerName),
|
||||
workerStates: workerStates,
|
||||
}
|
||||
}
|
||||
|
||||
// Start consuming from room servers
|
||||
func (s *OutputRoomEventConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
for _, as := range s.cfg.Derived.ApplicationServices {
|
||||
appsvc := as
|
||||
state := &appserviceState{
|
||||
ApplicationService: &appsvc,
|
||||
}
|
||||
token := jetstream.Tokenise(as.ID)
|
||||
if err := jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic,
|
||||
s.cfg.Matrix.JetStream.Durable("Appservice_"+token),
|
||||
50, // maximum number of events to send in a single transaction
|
||||
func(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
return s.onMessage(ctx, state, msgs)
|
||||
},
|
||||
nats.DeliverNew(), nats.ManualAck(),
|
||||
); err != nil {
|
||||
return fmt.Errorf("failed to create %q consumer: %w", token, err)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// onMessage is called when the appservice component receives a new event from
|
||||
// the room server output log.
|
||||
func (s *OutputRoomEventConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputRoomEventConsumer) onMessage(
|
||||
ctx context.Context, state *appserviceState, msgs []*nats.Msg,
|
||||
) bool {
|
||||
log.WithField("appservice", state.ID).Tracef("Appservice worker received %d message(s) from roomserver", len(msgs))
|
||||
events := make([]*gomatrixserverlib.HeaderedEvent, 0, len(msgs))
|
||||
for _, msg := range msgs {
|
||||
// Parse out the event JSON
|
||||
var output api.OutputEvent
|
||||
if err := json.Unmarshal(msg.Data, &output); err != nil {
|
||||
// If the message was invalid, log it and move on to the next message in the stream
|
||||
log.WithError(err).Errorf("roomserver output log: message parse failure")
|
||||
return true
|
||||
log.WithField("appservice", state.ID).WithError(err).Errorf("Appservice failed to parse message, ignoring")
|
||||
continue
|
||||
}
|
||||
switch output.Type {
|
||||
case api.OutputTypeNewRoomEvent:
|
||||
if output.NewRoomEvent == nil || !s.appserviceIsInterestedInEvent(ctx, output.NewRoomEvent.Event, state.ApplicationService) {
|
||||
continue
|
||||
}
|
||||
|
||||
log.WithFields(log.Fields{
|
||||
"type": output.Type,
|
||||
}).Debug("Got a message in OutputRoomEventConsumer")
|
||||
|
||||
events := []*gomatrixserverlib.HeaderedEvent{}
|
||||
if output.Type == api.OutputTypeNewRoomEvent && output.NewRoomEvent != nil {
|
||||
newEventID := output.NewRoomEvent.Event.EventID()
|
||||
events = append(events, output.NewRoomEvent.Event)
|
||||
if len(output.NewRoomEvent.AddsStateEventIDs) > 0 {
|
||||
newEventID := output.NewRoomEvent.Event.EventID()
|
||||
eventsReq := &api.QueryEventsByIDRequest{
|
||||
EventIDs: make([]string, 0, len(output.NewRoomEvent.AddsStateEventIDs)),
|
||||
}
|
||||
|
|
@ -103,105 +126,103 @@ func (s *OutputRoomEventConsumer) onMessage(ctx context.Context, msg *nats.Msg)
|
|||
}
|
||||
if len(eventsReq.EventIDs) > 0 {
|
||||
if err := s.rsAPI.QueryEventsByID(s.ctx, eventsReq, eventsRes); err != nil {
|
||||
log.WithError(err).Errorf("s.rsAPI.QueryEventsByID failed")
|
||||
return false
|
||||
}
|
||||
events = append(events, eventsRes.Events...)
|
||||
}
|
||||
}
|
||||
} else if output.Type == api.OutputTypeNewInviteEvent && output.NewInviteEvent != nil {
|
||||
|
||||
case api.OutputTypeNewInviteEvent:
|
||||
if output.NewInviteEvent == nil {
|
||||
continue
|
||||
}
|
||||
events = append(events, output.NewInviteEvent.Event)
|
||||
} else {
|
||||
log.WithFields(log.Fields{
|
||||
"type": output.Type,
|
||||
}).Debug("appservice OutputRoomEventConsumer ignoring event", string(msg.Data))
|
||||
|
||||
default:
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
// If there are no events selected for sending then we should
|
||||
// ack the messages so that we don't get sent them again in the
|
||||
// future.
|
||||
if len(events) == 0 {
|
||||
return true
|
||||
}
|
||||
|
||||
// Send event to any relevant application services
|
||||
if err := s.filterRoomserverEvents(context.TODO(), events); err != nil {
|
||||
log.WithError(err).Errorf("roomserver output log: filter error")
|
||||
return true
|
||||
// Send event to any relevant application services. If we hit
|
||||
// an error here, return false, so that we negatively ack.
|
||||
log.WithField("appservice", state.ID).Debugf("Appservice worker sending %d events(s) from roomserver", len(events))
|
||||
return s.sendEvents(ctx, state, events) == nil
|
||||
}
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// filterRoomserverEvents takes in events and decides whether any of them need
|
||||
// to be passed on to an external application service. It does this by checking
|
||||
// each namespace of each registered application service, and if there is a
|
||||
// match, adds the event to the queue for events to be sent to a particular
|
||||
// application service.
|
||||
func (s *OutputRoomEventConsumer) filterRoomserverEvents(
|
||||
ctx context.Context,
|
||||
// sendEvents passes events to the appservice by using the transactions
|
||||
// endpoint. It will block for the backoff period if necessary.
|
||||
func (s *OutputRoomEventConsumer) sendEvents(
|
||||
ctx context.Context, state *appserviceState,
|
||||
events []*gomatrixserverlib.HeaderedEvent,
|
||||
) error {
|
||||
for _, ws := range s.workerStates {
|
||||
for _, event := range events {
|
||||
// Check if this event is interesting to this application service
|
||||
if s.appserviceIsInterestedInEvent(ctx, event, ws.AppService) {
|
||||
// Queue this event to be sent off to the application service
|
||||
if id, err := s.asDB.StoreEvent(ctx, ws.AppService.ID, event); err != nil {
|
||||
log.WithError(err).Warnf("failed to insert incoming event into appservices database. id: %d", id)
|
||||
// Create the transaction body.
|
||||
transaction, err := json.Marshal(
|
||||
gomatrixserverlib.ApplicationServiceTransaction{
|
||||
Events: gomatrixserverlib.HeaderedToClientEvents(events, gomatrixserverlib.FormatAll),
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
} else {
|
||||
// Tell our worker to send out new messages by updating remaining message
|
||||
// count and waking them up with a broadcast
|
||||
ws.NotifyNewEvents(id)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: We should probably be more intelligent and pick something not
|
||||
// in the control of the event. A NATS timestamp header or something maybe.
|
||||
txnID := events[0].Event.OriginServerTS()
|
||||
|
||||
// Send the transaction to the appservice.
|
||||
// https://matrix.org/docs/spec/application_service/r0.1.2#put-matrix-app-v1-transactions-txnid
|
||||
address := fmt.Sprintf("%s/transactions/%d?access_token=%s", state.URL, txnID, url.QueryEscape(state.HSToken))
|
||||
req, err := http.NewRequestWithContext(ctx, "PUT", address, bytes.NewBuffer(transaction))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
req.Header.Set("Content-Type", "application/json")
|
||||
resp, err := s.client.Do(req)
|
||||
if err != nil {
|
||||
return state.backoffAndPause(err)
|
||||
}
|
||||
|
||||
// If the response was fine then we can clear any backoffs in place and
|
||||
// report that everything was OK. Otherwise, back off for a while.
|
||||
switch resp.StatusCode {
|
||||
case http.StatusOK:
|
||||
state.backoff = 0
|
||||
default:
|
||||
return state.backoffAndPause(fmt.Errorf("received HTTP status code %d from appservice", resp.StatusCode))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// appserviceJoinedAtEvent returns a boolean depending on whether a given
|
||||
// appservice has membership at the time a given event was created.
|
||||
func (s *OutputRoomEventConsumer) appserviceJoinedAtEvent(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, appservice config.ApplicationService) bool {
|
||||
// TODO: This is only checking the current room state, not the state at
|
||||
// the event in question. Pretty sure this is what Synapse does too, but
|
||||
// until we have a lighter way of checking the state before the event that
|
||||
// doesn't involve state res, then this is probably OK.
|
||||
membershipReq := &api.QueryMembershipsForRoomRequest{
|
||||
RoomID: event.RoomID(),
|
||||
JoinedOnly: true,
|
||||
// backoff pauses the calling goroutine for a 2^some backoff exponent seconds
|
||||
func (s *appserviceState) backoffAndPause(err error) error {
|
||||
if s.backoff < 6 {
|
||||
s.backoff++
|
||||
}
|
||||
membershipRes := &api.QueryMembershipsForRoomResponse{}
|
||||
|
||||
// XXX: This could potentially race if the state for the event is not known yet
|
||||
// e.g. the event came over federation but we do not have the full state persisted.
|
||||
if err := s.rsAPI.QueryMembershipsForRoom(ctx, membershipReq, membershipRes); err == nil {
|
||||
for _, ev := range membershipRes.JoinEvents {
|
||||
var membership gomatrixserverlib.MemberContent
|
||||
if err = json.Unmarshal(ev.Content, &membership); err != nil || ev.StateKey == nil {
|
||||
continue
|
||||
}
|
||||
if appservice.IsInterestedInUserID(*ev.StateKey) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
} else {
|
||||
log.WithFields(log.Fields{
|
||||
"room_id": event.RoomID(),
|
||||
}).WithError(err).Errorf("Unable to get membership for room")
|
||||
}
|
||||
return false
|
||||
duration := time.Second * time.Duration(math.Pow(2, float64(s.backoff)))
|
||||
log.WithField("appservice", s.ID).WithError(err).Errorf("Unable to send transaction to appservice, backing off for %s", duration.String())
|
||||
time.Sleep(duration)
|
||||
return err
|
||||
}
|
||||
|
||||
// appserviceIsInterestedInEvent returns a boolean depending on whether a given
|
||||
// event falls within one of a given application service's namespaces.
|
||||
//
|
||||
// TODO: This should be cached, see https://github.com/matrix-org/dendrite/issues/1682
|
||||
func (s *OutputRoomEventConsumer) appserviceIsInterestedInEvent(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, appservice config.ApplicationService) bool {
|
||||
// No reason to queue events if they'll never be sent to the application
|
||||
// service
|
||||
if appservice.URL == "" {
|
||||
func (s *OutputRoomEventConsumer) appserviceIsInterestedInEvent(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, appservice *config.ApplicationService) bool {
|
||||
switch {
|
||||
case appservice.URL == "":
|
||||
return false
|
||||
}
|
||||
|
||||
// Check Room ID and Sender of the event
|
||||
if appservice.IsInterestedInUserID(event.Sender()) ||
|
||||
appservice.IsInterestedInRoomID(event.RoomID()) {
|
||||
case appservice.IsInterestedInUserID(event.Sender()):
|
||||
return true
|
||||
case appservice.IsInterestedInRoomID(event.RoomID()):
|
||||
return true
|
||||
}
|
||||
|
||||
|
|
@ -222,6 +243,7 @@ func (s *OutputRoomEventConsumer) appserviceIsInterestedInEvent(ctx context.Cont
|
|||
}
|
||||
} else {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": appservice.ID,
|
||||
"room_id": event.RoomID(),
|
||||
}).WithError(err).Errorf("Unable to get aliases for room")
|
||||
}
|
||||
|
|
@ -229,3 +251,44 @@ func (s *OutputRoomEventConsumer) appserviceIsInterestedInEvent(ctx context.Cont
|
|||
// Check if any of the members in the room match the appservice
|
||||
return s.appserviceJoinedAtEvent(ctx, event, appservice)
|
||||
}
|
||||
|
||||
// appserviceJoinedAtEvent returns a boolean depending on whether a given
|
||||
// appservice has membership at the time a given event was created.
|
||||
func (s *OutputRoomEventConsumer) appserviceJoinedAtEvent(ctx context.Context, event *gomatrixserverlib.HeaderedEvent, appservice *config.ApplicationService) bool {
|
||||
// TODO: This is only checking the current room state, not the state at
|
||||
// the event in question. Pretty sure this is what Synapse does too, but
|
||||
// until we have a lighter way of checking the state before the event that
|
||||
// doesn't involve state res, then this is probably OK.
|
||||
membershipReq := &api.QueryMembershipsForRoomRequest{
|
||||
RoomID: event.RoomID(),
|
||||
JoinedOnly: true,
|
||||
}
|
||||
membershipRes := &api.QueryMembershipsForRoomResponse{}
|
||||
|
||||
// XXX: This could potentially race if the state for the event is not known yet
|
||||
// e.g. the event came over federation but we do not have the full state persisted.
|
||||
if err := s.rsAPI.QueryMembershipsForRoom(ctx, membershipReq, membershipRes); err == nil {
|
||||
for _, ev := range membershipRes.JoinEvents {
|
||||
switch {
|
||||
case ev.StateKey == nil:
|
||||
continue
|
||||
case ev.Type != gomatrixserverlib.MRoomMember:
|
||||
continue
|
||||
}
|
||||
var membership gomatrixserverlib.MemberContent
|
||||
err = json.Unmarshal(ev.Content, &membership)
|
||||
switch {
|
||||
case err != nil:
|
||||
continue
|
||||
case membership.Membership == gomatrixserverlib.Join:
|
||||
return true
|
||||
}
|
||||
}
|
||||
} else {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": appservice.ID,
|
||||
"room_id": event.RoomID(),
|
||||
}).WithError(err).Errorf("Unable to get membership for room")
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ const userIDExistsPath = "/users/"
|
|||
// AppServiceQueryAPI is an implementation of api.AppServiceQueryAPI
|
||||
type AppServiceQueryAPI struct {
|
||||
HTTPClient *http.Client
|
||||
Cfg *config.Dendrite
|
||||
Cfg *config.AppServiceAPI
|
||||
}
|
||||
|
||||
// RoomAliasExists performs a request to '/room/{roomAlias}' on all known
|
||||
|
|
|
|||
|
|
@ -1,30 +0,0 @@
|
|||
// Copyright 2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
)
|
||||
|
||||
type Database interface {
|
||||
StoreEvent(ctx context.Context, appServiceID string, event *gomatrixserverlib.HeaderedEvent) (int, error)
|
||||
GetEventsWithAppServiceID(ctx context.Context, appServiceID string, limit int) (int, int, []gomatrixserverlib.HeaderedEvent, error)
|
||||
GetLatestId(ctx context.Context, appServiceID string) (int, error)
|
||||
UpdateTxnIDForEvents(ctx context.Context, appserviceID string, maxID, txnID int) error
|
||||
RemoveEventsBeforeAndIncludingID(ctx context.Context, appserviceID string, eventTableID int) error
|
||||
GetLatestTxnID(ctx context.Context) (int, error)
|
||||
}
|
||||
|
|
@ -1,254 +0,0 @@
|
|||
// Copyright 2018 New Vector Ltd
|
||||
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package postgres
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"encoding/json"
|
||||
"time"
|
||||
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
const appserviceEventsSchema = `
|
||||
-- Stores events to be sent to application services
|
||||
CREATE TABLE IF NOT EXISTS appservice_events (
|
||||
-- An auto-incrementing id unique to each event in the table
|
||||
id BIGSERIAL NOT NULL PRIMARY KEY,
|
||||
-- The ID of the application service the event will be sent to
|
||||
as_id TEXT NOT NULL,
|
||||
-- JSON representation of the event
|
||||
headered_event_json TEXT NOT NULL,
|
||||
-- The ID of the transaction that this event is a part of
|
||||
txn_id BIGINT NOT NULL
|
||||
);
|
||||
|
||||
CREATE INDEX IF NOT EXISTS appservice_events_as_id ON appservice_events(as_id);
|
||||
`
|
||||
|
||||
const selectEventsByApplicationServiceIDSQL = "" +
|
||||
"SELECT id, headered_event_json, txn_id " +
|
||||
"FROM appservice_events WHERE as_id = $1 ORDER BY txn_id DESC, id ASC"
|
||||
|
||||
const getLatestIdSQL = "" +
|
||||
"SELECT id FROM appservice_events WHERE as_id = $1 ORDER BY id DESC LIMIT 1"
|
||||
|
||||
const insertEventSQL = "" +
|
||||
"INSERT INTO appservice_events(as_id, headered_event_json, txn_id) " +
|
||||
"VALUES ($1, $2, $3)" +
|
||||
"RETURNING id"
|
||||
|
||||
const updateTxnIDForEventsSQL = "" +
|
||||
"UPDATE appservice_events SET txn_id = $1 WHERE as_id = $2 AND id <= $3"
|
||||
|
||||
const deleteEventsBeforeAndIncludingIDSQL = "" +
|
||||
"DELETE FROM appservice_events WHERE as_id = $1 AND id <= $2"
|
||||
|
||||
const (
|
||||
// A transaction ID number that no transaction should ever have. Used for
|
||||
// checking again the default value.
|
||||
invalidTxnID = -2
|
||||
)
|
||||
|
||||
type eventsStatements struct {
|
||||
selectEventsByApplicationServiceIDStmt *sql.Stmt
|
||||
getLatestIdStmt *sql.Stmt
|
||||
insertEventStmt *sql.Stmt
|
||||
updateTxnIDForEventsStmt *sql.Stmt
|
||||
deleteEventsBeforeAndIncludingIDStmt *sql.Stmt
|
||||
}
|
||||
|
||||
func (s *eventsStatements) prepare(db *sql.DB) (err error) {
|
||||
_, err = db.Exec(appserviceEventsSchema)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if s.selectEventsByApplicationServiceIDStmt, err = db.Prepare(selectEventsByApplicationServiceIDSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.getLatestIdStmt, err = db.Prepare(getLatestIdSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.insertEventStmt, err = db.Prepare(insertEventSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.updateTxnIDForEventsStmt, err = db.Prepare(updateTxnIDForEventsSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.deleteEventsBeforeAndIncludingIDStmt, err = db.Prepare(deleteEventsBeforeAndIncludingIDSQL); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// selectEventsByApplicationServiceID takes in an application service ID and
|
||||
// returns a slice of events that need to be sent to that application service,
|
||||
// as well as an int later used to remove these same events from the database
|
||||
// once successfully sent to an application service.
|
||||
func (s *eventsStatements) selectEventsByApplicationServiceID(
|
||||
ctx context.Context,
|
||||
applicationServiceID string,
|
||||
limit int,
|
||||
) (
|
||||
txnID, maxID int,
|
||||
events []gomatrixserverlib.HeaderedEvent,
|
||||
err error,
|
||||
) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": applicationServiceID,
|
||||
}).WithError(err).Fatalf("appservice unable to select new events to send")
|
||||
}
|
||||
}()
|
||||
// Retrieve events from the database. Unsuccessfully sent events first
|
||||
eventRows, err := s.selectEventsByApplicationServiceIDStmt.QueryContext(ctx, applicationServiceID)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
defer checkNamedErr(eventRows.Close, &err)
|
||||
events, maxID, txnID, err = retrieveEvents(eventRows, limit)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// checkNamedErr calls fn and overwrite err if it was nil and fn returned non-nil
|
||||
func checkNamedErr(fn func() error, err *error) {
|
||||
if e := fn(); e != nil && *err == nil {
|
||||
*err = e
|
||||
}
|
||||
}
|
||||
|
||||
func retrieveEvents(eventRows *sql.Rows, limit int) (events []gomatrixserverlib.HeaderedEvent, maxID, txnID int, err error) {
|
||||
// Get current time for use in calculating event age
|
||||
nowMilli := time.Now().UnixNano() / int64(time.Millisecond)
|
||||
|
||||
// Iterate through each row and store event contents
|
||||
// If txn_id changes dramatically, we've switched from collecting old events to
|
||||
// new ones. Send back those events first.
|
||||
lastTxnID := invalidTxnID
|
||||
for eventsProcessed := 0; eventRows.Next(); {
|
||||
var event gomatrixserverlib.HeaderedEvent
|
||||
var eventJSON []byte
|
||||
var id int
|
||||
err = eventRows.Scan(
|
||||
&id,
|
||||
&eventJSON,
|
||||
&txnID,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
|
||||
// Unmarshal eventJSON
|
||||
if err = json.Unmarshal(eventJSON, &event); err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
|
||||
// If txnID has changed on this event from the previous event, then we've
|
||||
// reached the end of a transaction's events. Return only those events.
|
||||
if lastTxnID > invalidTxnID && lastTxnID != txnID {
|
||||
return events, maxID, lastTxnID, nil
|
||||
}
|
||||
lastTxnID = txnID
|
||||
|
||||
// Limit events that aren't part of an old transaction
|
||||
if txnID == -1 {
|
||||
// Return if we've hit the limit
|
||||
if eventsProcessed++; eventsProcessed > limit {
|
||||
return events, maxID, lastTxnID, nil
|
||||
}
|
||||
}
|
||||
|
||||
if id > maxID {
|
||||
maxID = id
|
||||
}
|
||||
|
||||
// Portion of the event that is unsigned due to rapid change
|
||||
// TODO: Consider removing age as not many app services use it
|
||||
if err = event.SetUnsignedField("age", nowMilli-int64(event.OriginServerTS())); err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
|
||||
events = append(events, event)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (s *eventsStatements) getLatestId(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
) (int, error) {
|
||||
var count int
|
||||
err := s.getLatestIdStmt.QueryRowContext(ctx, appServiceID).Scan(&count)
|
||||
if err != nil && err != sql.ErrNoRows {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
return count, nil
|
||||
}
|
||||
|
||||
// insertEvent inserts an event mapped to its corresponding application service
|
||||
// IDs into the db.
|
||||
func (s *eventsStatements) insertEvent(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
event *gomatrixserverlib.HeaderedEvent,
|
||||
) (id int, err error) {
|
||||
// Convert event to JSON before inserting
|
||||
var eventJSON []byte
|
||||
eventJSON, err = json.Marshal(event)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
err = s.insertEventStmt.QueryRowContext(
|
||||
ctx,
|
||||
appServiceID,
|
||||
eventJSON,
|
||||
-1, // No transaction ID yet
|
||||
).Scan(&id)
|
||||
return
|
||||
}
|
||||
|
||||
// updateTxnIDForEvents sets the transactionID for a collection of events. Done
|
||||
// before sending them to an AppService. Referenced before sending to make sure
|
||||
// we aren't constructing multiple transactions with the same events.
|
||||
func (s *eventsStatements) updateTxnIDForEvents(
|
||||
ctx context.Context,
|
||||
appserviceID string,
|
||||
maxID, txnID int,
|
||||
) (err error) {
|
||||
_, err = s.updateTxnIDForEventsStmt.ExecContext(ctx, txnID, appserviceID, maxID)
|
||||
return
|
||||
}
|
||||
|
||||
// deleteEventsBeforeAndIncludingID removes events matching given IDs from the database.
|
||||
func (s *eventsStatements) deleteEventsBeforeAndIncludingID(
|
||||
ctx context.Context,
|
||||
appserviceID string,
|
||||
eventTableID int,
|
||||
) (err error) {
|
||||
_, err = s.deleteEventsBeforeAndIncludingIDStmt.ExecContext(ctx, appserviceID, eventTableID)
|
||||
return
|
||||
}
|
||||
|
|
@ -1,118 +0,0 @@
|
|||
// Copyright 2018 New Vector Ltd
|
||||
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package postgres
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
|
||||
// Import postgres database driver
|
||||
_ "github.com/lib/pq"
|
||||
"github.com/matrix-org/dendrite/internal/sqlutil"
|
||||
"github.com/matrix-org/dendrite/setup/base"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
)
|
||||
|
||||
// Database stores events intended to be later sent to application services
|
||||
type Database struct {
|
||||
events eventsStatements
|
||||
txnID txnStatements
|
||||
db *sql.DB
|
||||
writer sqlutil.Writer
|
||||
}
|
||||
|
||||
// NewDatabase opens a new database
|
||||
func NewDatabase(base *base.BaseDendrite, dbProperties *config.DatabaseOptions) (*Database, error) {
|
||||
var result Database
|
||||
var err error
|
||||
if result.db, result.writer, err = base.DatabaseConnection(dbProperties, sqlutil.NewDummyWriter()); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err = result.prepare(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &result, nil
|
||||
}
|
||||
|
||||
func (d *Database) prepare() error {
|
||||
if err := d.events.prepare(d.db); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return d.txnID.prepare(d.db)
|
||||
}
|
||||
|
||||
// StoreEvent takes in a gomatrixserverlib.HeaderedEvent and stores it in the database
|
||||
// for a transaction worker to pull and later send to an application service.
|
||||
func (d *Database) StoreEvent(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
event *gomatrixserverlib.HeaderedEvent,
|
||||
) (int, error) {
|
||||
return d.events.insertEvent(ctx, appServiceID, event)
|
||||
}
|
||||
|
||||
// GetEventsWithAppServiceID returns a slice of events and their IDs intended to
|
||||
// be sent to an application service given its ID.
|
||||
func (d *Database) GetEventsWithAppServiceID(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
limit int,
|
||||
) (int, int, []gomatrixserverlib.HeaderedEvent, error) {
|
||||
return d.events.selectEventsByApplicationServiceID(ctx, appServiceID, limit)
|
||||
}
|
||||
|
||||
// GetLatestId returns the latest incremental id associated with appservice.
|
||||
func (d *Database) GetLatestId(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
) (int, error) {
|
||||
id, err := d.events.getLatestId(ctx, appServiceID)
|
||||
if err == sql.ErrNoRows {
|
||||
return 0, nil
|
||||
}
|
||||
return id, err
|
||||
}
|
||||
|
||||
// UpdateTxnIDForEvents takes in an application service ID and a
|
||||
// and stores them in the DB, unless the pair already exists, in
|
||||
// which case it updates them.
|
||||
func (d *Database) UpdateTxnIDForEvents(
|
||||
ctx context.Context,
|
||||
appserviceID string,
|
||||
maxID, txnID int,
|
||||
) error {
|
||||
return d.events.updateTxnIDForEvents(ctx, appserviceID, maxID, txnID)
|
||||
}
|
||||
|
||||
// RemoveEventsBeforeAndIncludingID removes all events from the database that
|
||||
// are less than or equal to a given maximum ID. IDs here are implemented as a
|
||||
// serial, thus this should always delete events in chronological order.
|
||||
func (d *Database) RemoveEventsBeforeAndIncludingID(
|
||||
ctx context.Context,
|
||||
appserviceID string,
|
||||
eventTableID int,
|
||||
) error {
|
||||
return d.events.deleteEventsBeforeAndIncludingID(ctx, appserviceID, eventTableID)
|
||||
}
|
||||
|
||||
// GetLatestTxnID returns the latest available transaction id
|
||||
func (d *Database) GetLatestTxnID(
|
||||
ctx context.Context,
|
||||
) (int, error) {
|
||||
return d.txnID.selectTxnID(ctx)
|
||||
}
|
||||
|
|
@ -1,53 +0,0 @@
|
|||
// Copyright 2018 New Vector Ltd
|
||||
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package postgres
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
)
|
||||
|
||||
const txnIDSchema = `
|
||||
-- Keeps a count of the current transaction ID
|
||||
CREATE SEQUENCE IF NOT EXISTS txn_id_counter START 1;
|
||||
`
|
||||
|
||||
const selectTxnIDSQL = "SELECT nextval('txn_id_counter')"
|
||||
|
||||
type txnStatements struct {
|
||||
selectTxnIDStmt *sql.Stmt
|
||||
}
|
||||
|
||||
func (s *txnStatements) prepare(db *sql.DB) (err error) {
|
||||
_, err = db.Exec(txnIDSchema)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if s.selectTxnIDStmt, err = db.Prepare(selectTxnIDSQL); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// selectTxnID selects the latest ascending transaction ID
|
||||
func (s *txnStatements) selectTxnID(
|
||||
ctx context.Context,
|
||||
) (txnID int, err error) {
|
||||
err = s.selectTxnIDStmt.QueryRowContext(ctx).Scan(&txnID)
|
||||
return
|
||||
}
|
||||
|
|
@ -1,265 +0,0 @@
|
|||
// Copyright 2018 New Vector Ltd
|
||||
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package sqlite3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
"encoding/json"
|
||||
"time"
|
||||
|
||||
"github.com/matrix-org/dendrite/internal/sqlutil"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
const appserviceEventsSchema = `
|
||||
-- Stores events to be sent to application services
|
||||
CREATE TABLE IF NOT EXISTS appservice_events (
|
||||
-- An auto-incrementing id unique to each event in the table
|
||||
id INTEGER PRIMARY KEY AUTOINCREMENT,
|
||||
-- The ID of the application service the event will be sent to
|
||||
as_id TEXT NOT NULL,
|
||||
-- JSON representation of the event
|
||||
headered_event_json TEXT NOT NULL,
|
||||
-- The ID of the transaction that this event is a part of
|
||||
txn_id INTEGER NOT NULL
|
||||
);
|
||||
|
||||
CREATE INDEX IF NOT EXISTS appservice_events_as_id ON appservice_events(as_id);
|
||||
`
|
||||
|
||||
const selectEventsByApplicationServiceIDSQL = "" +
|
||||
"SELECT id, headered_event_json, txn_id " +
|
||||
"FROM appservice_events WHERE as_id = $1 ORDER BY txn_id DESC, id ASC"
|
||||
|
||||
const getLatestIdSQL = "" +
|
||||
"SELECT id FROM appservice_events WHERE as_id = $1 ORDER BY id DESC LIMIT 1"
|
||||
|
||||
const insertEventSQL = "" +
|
||||
"INSERT INTO appservice_events(as_id, headered_event_json, txn_id) " +
|
||||
"VALUES ($1, $2, $3)" +
|
||||
"RETURNING id"
|
||||
|
||||
const updateTxnIDForEventsSQL = "" +
|
||||
"UPDATE appservice_events SET txn_id = $1 WHERE as_id = $2 AND id <= $3"
|
||||
|
||||
const deleteEventsBeforeAndIncludingIDSQL = "" +
|
||||
"DELETE FROM appservice_events WHERE as_id = $1 AND id <= $2"
|
||||
|
||||
const (
|
||||
// A transaction ID number that no transaction should ever have. Used for
|
||||
// checking again the default value.
|
||||
invalidTxnID = -2
|
||||
)
|
||||
|
||||
type eventsStatements struct {
|
||||
db *sql.DB
|
||||
writer sqlutil.Writer
|
||||
selectEventsByApplicationServiceIDStmt *sql.Stmt
|
||||
getLatestIdStmt *sql.Stmt
|
||||
insertEventStmt *sql.Stmt
|
||||
updateTxnIDForEventsStmt *sql.Stmt
|
||||
deleteEventsBeforeAndIncludingIDStmt *sql.Stmt
|
||||
}
|
||||
|
||||
func (s *eventsStatements) prepare(db *sql.DB, writer sqlutil.Writer) (err error) {
|
||||
s.db = db
|
||||
s.writer = writer
|
||||
_, err = db.Exec(appserviceEventsSchema)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if s.selectEventsByApplicationServiceIDStmt, err = db.Prepare(selectEventsByApplicationServiceIDSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.getLatestIdStmt, err = db.Prepare(getLatestIdSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.insertEventStmt, err = db.Prepare(insertEventSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.updateTxnIDForEventsStmt, err = db.Prepare(updateTxnIDForEventsSQL); err != nil {
|
||||
return
|
||||
}
|
||||
if s.deleteEventsBeforeAndIncludingIDStmt, err = db.Prepare(deleteEventsBeforeAndIncludingIDSQL); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// selectEventsByApplicationServiceID takes in an application service ID and
|
||||
// returns a slice of events that need to be sent to that application service,
|
||||
// as well as an int later used to remove these same events from the database
|
||||
// once successfully sent to an application service.
|
||||
func (s *eventsStatements) selectEventsByApplicationServiceID(
|
||||
ctx context.Context,
|
||||
applicationServiceID string,
|
||||
limit int,
|
||||
) (
|
||||
txnID, maxID int,
|
||||
events []gomatrixserverlib.HeaderedEvent,
|
||||
err error,
|
||||
) {
|
||||
defer func() {
|
||||
if err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": applicationServiceID,
|
||||
}).WithError(err).Fatalf("appservice unable to select new events to send")
|
||||
}
|
||||
}()
|
||||
// Retrieve events from the database. Unsuccessfully sent events first
|
||||
eventRows, err := s.selectEventsByApplicationServiceIDStmt.QueryContext(ctx, applicationServiceID)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
defer checkNamedErr(eventRows.Close, &err)
|
||||
events, maxID, txnID, err = retrieveEvents(eventRows, limit)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// checkNamedErr calls fn and overwrite err if it was nil and fn returned non-nil
|
||||
func checkNamedErr(fn func() error, err *error) {
|
||||
if e := fn(); e != nil && *err == nil {
|
||||
*err = e
|
||||
}
|
||||
}
|
||||
|
||||
func retrieveEvents(eventRows *sql.Rows, limit int) (events []gomatrixserverlib.HeaderedEvent, maxID, txnID int, err error) {
|
||||
// Get current time for use in calculating event age
|
||||
nowMilli := time.Now().UnixNano() / int64(time.Millisecond)
|
||||
|
||||
// Iterate through each row and store event contents
|
||||
// If txn_id changes dramatically, we've switched from collecting old events to
|
||||
// new ones. Send back those events first.
|
||||
lastTxnID := invalidTxnID
|
||||
for eventsProcessed := 0; eventRows.Next(); {
|
||||
var event gomatrixserverlib.HeaderedEvent
|
||||
var eventJSON []byte
|
||||
var id int
|
||||
err = eventRows.Scan(
|
||||
&id,
|
||||
&eventJSON,
|
||||
&txnID,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
|
||||
// Unmarshal eventJSON
|
||||
if err = json.Unmarshal(eventJSON, &event); err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
|
||||
// If txnID has changed on this event from the previous event, then we've
|
||||
// reached the end of a transaction's events. Return only those events.
|
||||
if lastTxnID > invalidTxnID && lastTxnID != txnID {
|
||||
return events, maxID, lastTxnID, nil
|
||||
}
|
||||
lastTxnID = txnID
|
||||
|
||||
// Limit events that aren't part of an old transaction
|
||||
if txnID == -1 {
|
||||
// Return if we've hit the limit
|
||||
if eventsProcessed++; eventsProcessed > limit {
|
||||
return events, maxID, lastTxnID, nil
|
||||
}
|
||||
}
|
||||
|
||||
if id > maxID {
|
||||
maxID = id
|
||||
}
|
||||
|
||||
// Portion of the event that is unsigned due to rapid change
|
||||
// TODO: Consider removing age as not many app services use it
|
||||
if err = event.SetUnsignedField("age", nowMilli-int64(event.OriginServerTS())); err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
|
||||
events = append(events, event)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (s *eventsStatements) getLatestId(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
) (int, error) {
|
||||
var count int
|
||||
err := s.getLatestIdStmt.QueryRowContext(ctx, appServiceID).Scan(&count)
|
||||
if err != nil && err != sql.ErrNoRows {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
return count, nil
|
||||
}
|
||||
|
||||
// insertEvent inserts an event mapped to its corresponding application service
|
||||
// IDs into the db.
|
||||
func (s *eventsStatements) insertEvent(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
event *gomatrixserverlib.HeaderedEvent,
|
||||
) (id int, err error) {
|
||||
// Convert event to JSON before inserting
|
||||
eventJSON, err := json.Marshal(event)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
err = s.writer.Do(s.db, nil, func(txn *sql.Tx) error {
|
||||
err = s.insertEventStmt.QueryRowContext(
|
||||
ctx,
|
||||
appServiceID,
|
||||
eventJSON,
|
||||
-1, // No transaction ID yet
|
||||
).Scan(&id)
|
||||
return err
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
// updateTxnIDForEvents sets the transactionID for a collection of events. Done
|
||||
// before sending them to an AppService. Referenced before sending to make sure
|
||||
// we aren't constructing multiple transactions with the same events.
|
||||
func (s *eventsStatements) updateTxnIDForEvents(
|
||||
ctx context.Context,
|
||||
appserviceID string,
|
||||
maxID, txnID int,
|
||||
) (err error) {
|
||||
return s.writer.Do(s.db, nil, func(txn *sql.Tx) error {
|
||||
_, err := s.updateTxnIDForEventsStmt.ExecContext(ctx, txnID, appserviceID, maxID)
|
||||
return err
|
||||
})
|
||||
}
|
||||
|
||||
// deleteEventsBeforeAndIncludingID removes events matching given IDs from the database.
|
||||
func (s *eventsStatements) deleteEventsBeforeAndIncludingID(
|
||||
ctx context.Context,
|
||||
appserviceID string,
|
||||
eventTableID int,
|
||||
) (err error) {
|
||||
return s.writer.Do(s.db, nil, func(txn *sql.Tx) error {
|
||||
_, err := s.deleteEventsBeforeAndIncludingIDStmt.ExecContext(ctx, appserviceID, eventTableID)
|
||||
return err
|
||||
})
|
||||
}
|
||||
|
|
@ -1,117 +0,0 @@
|
|||
// Copyright 2018 New Vector Ltd
|
||||
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package sqlite3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
|
||||
// Import SQLite database driver
|
||||
"github.com/matrix-org/dendrite/internal/sqlutil"
|
||||
"github.com/matrix-org/dendrite/setup/base"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
)
|
||||
|
||||
// Database stores events intended to be later sent to application services
|
||||
type Database struct {
|
||||
events eventsStatements
|
||||
txnID txnStatements
|
||||
db *sql.DB
|
||||
writer sqlutil.Writer
|
||||
}
|
||||
|
||||
// NewDatabase opens a new database
|
||||
func NewDatabase(base *base.BaseDendrite, dbProperties *config.DatabaseOptions) (*Database, error) {
|
||||
var result Database
|
||||
var err error
|
||||
if result.db, result.writer, err = base.DatabaseConnection(dbProperties, sqlutil.NewExclusiveWriter()); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err = result.prepare(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &result, nil
|
||||
}
|
||||
|
||||
func (d *Database) prepare() error {
|
||||
if err := d.events.prepare(d.db, d.writer); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return d.txnID.prepare(d.db, d.writer)
|
||||
}
|
||||
|
||||
// StoreEvent takes in a gomatrixserverlib.HeaderedEvent and stores it in the database
|
||||
// for a transaction worker to pull and later send to an application service.
|
||||
func (d *Database) StoreEvent(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
event *gomatrixserverlib.HeaderedEvent,
|
||||
) (int, error) {
|
||||
return d.events.insertEvent(ctx, appServiceID, event)
|
||||
}
|
||||
|
||||
// GetEventsWithAppServiceID returns a slice of events and their IDs intended to
|
||||
// be sent to an application service given its ID.
|
||||
func (d *Database) GetEventsWithAppServiceID(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
limit int,
|
||||
) (int, int, []gomatrixserverlib.HeaderedEvent, error) {
|
||||
return d.events.selectEventsByApplicationServiceID(ctx, appServiceID, limit)
|
||||
}
|
||||
|
||||
// GetLatestId returns the latest incremental id associated with appservice.
|
||||
func (d *Database) GetLatestId(
|
||||
ctx context.Context,
|
||||
appServiceID string,
|
||||
) (int, error) {
|
||||
id, err := d.events.getLatestId(ctx, appServiceID)
|
||||
if err == sql.ErrNoRows {
|
||||
return 0, nil
|
||||
}
|
||||
return id, err
|
||||
}
|
||||
|
||||
// UpdateTxnIDForEvents takes in an application service ID and a
|
||||
// and stores them in the DB, unless the pair already exists, in
|
||||
// which case it updates them.
|
||||
func (d *Database) UpdateTxnIDForEvents(
|
||||
ctx context.Context,
|
||||
appserviceID string,
|
||||
maxID, txnID int,
|
||||
) error {
|
||||
return d.events.updateTxnIDForEvents(ctx, appserviceID, maxID, txnID)
|
||||
}
|
||||
|
||||
// RemoveEventsBeforeAndIncludingID removes all events from the database that
|
||||
// are less than or equal to a given maximum ID. IDs here are implemented as a
|
||||
// serial, thus this should always delete events in chronological order.
|
||||
func (d *Database) RemoveEventsBeforeAndIncludingID(
|
||||
ctx context.Context,
|
||||
appserviceID string,
|
||||
eventTableID int,
|
||||
) error {
|
||||
return d.events.deleteEventsBeforeAndIncludingID(ctx, appserviceID, eventTableID)
|
||||
}
|
||||
|
||||
// GetLatestTxnID returns the latest available transaction id
|
||||
func (d *Database) GetLatestTxnID(
|
||||
ctx context.Context,
|
||||
) (int, error) {
|
||||
return d.txnID.selectTxnID(ctx)
|
||||
}
|
||||
|
|
@ -1,82 +0,0 @@
|
|||
// Copyright 2018 New Vector Ltd
|
||||
// Copyright 2019-2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package sqlite3
|
||||
|
||||
import (
|
||||
"context"
|
||||
"database/sql"
|
||||
|
||||
"github.com/matrix-org/dendrite/internal/sqlutil"
|
||||
)
|
||||
|
||||
const txnIDSchema = `
|
||||
-- Keeps a count of the current transaction ID
|
||||
CREATE TABLE IF NOT EXISTS appservice_counters (
|
||||
name TEXT PRIMARY KEY NOT NULL,
|
||||
last_id INTEGER DEFAULT 1
|
||||
);
|
||||
INSERT OR IGNORE INTO appservice_counters (name, last_id) VALUES('txn_id', 1);
|
||||
`
|
||||
|
||||
const selectTxnIDSQL = `
|
||||
SELECT last_id FROM appservice_counters WHERE name='txn_id'
|
||||
`
|
||||
|
||||
const updateTxnIDSQL = `
|
||||
UPDATE appservice_counters SET last_id=last_id+1 WHERE name='txn_id'
|
||||
`
|
||||
|
||||
type txnStatements struct {
|
||||
db *sql.DB
|
||||
writer sqlutil.Writer
|
||||
selectTxnIDStmt *sql.Stmt
|
||||
updateTxnIDStmt *sql.Stmt
|
||||
}
|
||||
|
||||
func (s *txnStatements) prepare(db *sql.DB, writer sqlutil.Writer) (err error) {
|
||||
s.db = db
|
||||
s.writer = writer
|
||||
_, err = db.Exec(txnIDSchema)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if s.selectTxnIDStmt, err = db.Prepare(selectTxnIDSQL); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if s.updateTxnIDStmt, err = db.Prepare(updateTxnIDSQL); err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// selectTxnID selects the latest ascending transaction ID
|
||||
func (s *txnStatements) selectTxnID(
|
||||
ctx context.Context,
|
||||
) (txnID int, err error) {
|
||||
err = s.writer.Do(s.db, nil, func(txn *sql.Tx) error {
|
||||
err := s.selectTxnIDStmt.QueryRowContext(ctx).Scan(&txnID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
_, err = s.updateTxnIDStmt.ExecContext(ctx)
|
||||
return err
|
||||
})
|
||||
return
|
||||
}
|
||||
|
|
@ -1,40 +0,0 @@
|
|||
// Copyright 2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
//go:build !wasm
|
||||
// +build !wasm
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/matrix-org/dendrite/appservice/storage/postgres"
|
||||
"github.com/matrix-org/dendrite/appservice/storage/sqlite3"
|
||||
"github.com/matrix-org/dendrite/setup/base"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
)
|
||||
|
||||
// NewDatabase opens a new Postgres or Sqlite database (based on dataSourceName scheme)
|
||||
// and sets DB connection parameters
|
||||
func NewDatabase(base *base.BaseDendrite, dbProperties *config.DatabaseOptions) (Database, error) {
|
||||
switch {
|
||||
case dbProperties.ConnectionString.IsSQLite():
|
||||
return sqlite3.NewDatabase(base, dbProperties)
|
||||
case dbProperties.ConnectionString.IsPostgres():
|
||||
return postgres.NewDatabase(base, dbProperties)
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected database type")
|
||||
}
|
||||
}
|
||||
|
|
@ -1,34 +0,0 @@
|
|||
// Copyright 2020 The Matrix.org Foundation C.I.C.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/matrix-org/dendrite/appservice/storage/sqlite3"
|
||||
"github.com/matrix-org/dendrite/setup/base"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
)
|
||||
|
||||
func NewDatabase(base *base.BaseDendrite, dbProperties *config.DatabaseOptions) (Database, error) {
|
||||
switch {
|
||||
case dbProperties.ConnectionString.IsSQLite():
|
||||
return sqlite3.NewDatabase(base, dbProperties)
|
||||
case dbProperties.ConnectionString.IsPostgres():
|
||||
return nil, fmt.Errorf("can't use Postgres implementation")
|
||||
default:
|
||||
return nil, fmt.Errorf("unexpected database type")
|
||||
}
|
||||
}
|
||||
|
|
@ -1,56 +0,0 @@
|
|||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package types
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
)
|
||||
|
||||
const (
|
||||
// AppServiceDeviceID is the AS dummy device ID
|
||||
AppServiceDeviceID = "AS_Device"
|
||||
)
|
||||
|
||||
// ApplicationServiceWorkerState is a type that couples an application service,
|
||||
// a lockable condition as well as some other state variables, allowing the
|
||||
// roomserver to notify appservice workers when there are events ready to send
|
||||
// externally to application services.
|
||||
type ApplicationServiceWorkerState struct {
|
||||
AppService config.ApplicationService
|
||||
Cond *sync.Cond
|
||||
// Lastest incremental ID from appservice_events table that is ready to be sent to application service
|
||||
latestId int
|
||||
// Backoff exponent (2^x secs). Max 6, aka 64s.
|
||||
Backoff int
|
||||
}
|
||||
|
||||
// NotifyNewEvents wakes up all waiting goroutines, notifying that events remain
|
||||
// in the event queue for this application service worker.
|
||||
func (a *ApplicationServiceWorkerState) NotifyNewEvents(id int) {
|
||||
a.Cond.L.Lock()
|
||||
a.latestId = id
|
||||
a.Cond.Broadcast()
|
||||
a.Cond.L.Unlock()
|
||||
}
|
||||
|
||||
// WaitForNewEvents causes the calling goroutine to wait on the worker state's
|
||||
// condition for a broadcast or similar wakeup, if there are no events ready.
|
||||
func (a *ApplicationServiceWorkerState) WaitForNewEvents(id int) {
|
||||
a.Cond.L.Lock()
|
||||
if a.latestId <= id {
|
||||
a.Cond.Wait()
|
||||
}
|
||||
a.Cond.L.Unlock()
|
||||
}
|
||||
|
|
@ -1,231 +0,0 @@
|
|||
// Copyright 2018 Vector Creations Ltd
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package workers
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"math"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"time"
|
||||
|
||||
"github.com/matrix-org/dendrite/appservice/storage"
|
||||
"github.com/matrix-org/dendrite/appservice/types"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
var (
|
||||
// Maximum size of events sent in each transaction.
|
||||
transactionBatchSize = 50
|
||||
)
|
||||
|
||||
// SetupTransactionWorkers spawns a separate goroutine for each application
|
||||
// service. Each of these "workers" handle taking all events intended for their
|
||||
// app service, batch them up into a single transaction (up to a max transaction
|
||||
// size), then send that off to the AS's /transactions/{txnID} endpoint. It also
|
||||
// handles exponentially backing off in case the AS isn't currently available.
|
||||
func SetupTransactionWorkers(
|
||||
client *http.Client,
|
||||
appserviceDB storage.Database,
|
||||
workerStates []*types.ApplicationServiceWorkerState,
|
||||
) error {
|
||||
// Create a worker that handles transmitting events to a single homeserver
|
||||
for _, workerState := range workerStates {
|
||||
// Don't create a worker if this AS doesn't want to receive events
|
||||
if workerState.AppService.URL != "" {
|
||||
go worker(client, appserviceDB, workerState)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// worker is a goroutine that sends any queued events to the application service
|
||||
// it is given.
|
||||
func worker(client *http.Client, db storage.Database, ws *types.ApplicationServiceWorkerState) {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": ws.AppService.ID,
|
||||
}).Info("Starting application service")
|
||||
ctx := context.Background()
|
||||
|
||||
// Initial check for any leftover events to send from last time
|
||||
latestId, err := db.GetLatestId(ctx, ws.AppService.ID)
|
||||
if err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": ws.AppService.ID,
|
||||
}).WithError(err).Fatal("appservice worker unable to read queued events from DB")
|
||||
return
|
||||
}
|
||||
ws.NotifyNewEvents(latestId)
|
||||
id := 0
|
||||
// Loop forever and keep waiting for more events to send
|
||||
for {
|
||||
// Wait for more events if we've sent all the events in the database
|
||||
ws.WaitForNewEvents(id)
|
||||
|
||||
// Batch events up into a transaction
|
||||
transactionJSON, txnID, maxEventID, err := createTransaction(ctx, db, ws.AppService.ID)
|
||||
if err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": ws.AppService.ID,
|
||||
}).WithError(err).Fatal("appservice worker unable to create transaction")
|
||||
|
||||
return
|
||||
}
|
||||
// Transactions have a maximum event size (or new events may arrive while
|
||||
// transaction is processed by Application Service), so there may still be
|
||||
// some events left over to send. We will keep sending if id < ws.latestID.
|
||||
id = maxEventID
|
||||
|
||||
// Send the events off to the application service
|
||||
// Backoff if the application service does not respond
|
||||
err = send(client, ws.AppService, txnID, transactionJSON)
|
||||
if err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": ws.AppService.ID,
|
||||
}).WithError(err).Error("unable to send event")
|
||||
// Backoff
|
||||
backoff(ws, err)
|
||||
continue
|
||||
}
|
||||
|
||||
// We sent successfully, hooray!
|
||||
ws.Backoff = 0
|
||||
|
||||
// Remove sent events from the DB
|
||||
err = db.RemoveEventsBeforeAndIncludingID(ctx, ws.AppService.ID, maxEventID)
|
||||
if err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": ws.AppService.ID,
|
||||
}).WithError(err).Fatal("unable to remove appservice events from the database")
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// backoff pauses the calling goroutine for a 2^some backoff exponent seconds
|
||||
func backoff(ws *types.ApplicationServiceWorkerState, err error) {
|
||||
// Calculate how long to backoff for
|
||||
backoffDuration := time.Duration(math.Pow(2, float64(ws.Backoff)))
|
||||
backoffSeconds := time.Second * backoffDuration
|
||||
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": ws.AppService.ID,
|
||||
}).WithError(err).Warnf("unable to send transactions successfully, backing off for %ds",
|
||||
backoffDuration)
|
||||
|
||||
ws.Backoff++
|
||||
if ws.Backoff > 6 {
|
||||
ws.Backoff = 6
|
||||
}
|
||||
|
||||
// Backoff
|
||||
time.Sleep(backoffSeconds)
|
||||
}
|
||||
|
||||
// createTransaction takes in a slice of AS events, stores them in an AS
|
||||
// transaction, and JSON-encodes the results.
|
||||
func createTransaction(
|
||||
ctx context.Context,
|
||||
db storage.Database,
|
||||
appserviceID string,
|
||||
) (
|
||||
transactionJSON []byte,
|
||||
txnID, maxID int,
|
||||
err error,
|
||||
) {
|
||||
// Retrieve the latest events from the DB (will return old events if they weren't successfully sent)
|
||||
txnID, maxID, events, err := db.GetEventsWithAppServiceID(ctx, appserviceID, transactionBatchSize)
|
||||
if err != nil {
|
||||
log.WithFields(log.Fields{
|
||||
"appservice": appserviceID,
|
||||
}).WithError(err).Fatalf("appservice worker unable to read queued events from DB")
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// Check if these events do not already have a transaction ID
|
||||
if txnID == -1 {
|
||||
// If not, grab next available ID from the DB
|
||||
txnID, err = db.GetLatestTxnID(ctx)
|
||||
if err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
|
||||
// Mark new events with current transactionID
|
||||
if err = db.UpdateTxnIDForEvents(ctx, appserviceID, maxID, txnID); err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
}
|
||||
|
||||
var ev []*gomatrixserverlib.HeaderedEvent
|
||||
for i := range events {
|
||||
ev = append(ev, &events[i])
|
||||
}
|
||||
|
||||
// Create a transaction and store the events inside
|
||||
transaction := gomatrixserverlib.ApplicationServiceTransaction{
|
||||
Events: gomatrixserverlib.HeaderedToClientEvents(ev, gomatrixserverlib.FormatAll),
|
||||
}
|
||||
|
||||
transactionJSON, err = json.Marshal(transaction)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// send sends events to an application service. Returns an error if an OK was not
|
||||
// received back from the application service or the request timed out.
|
||||
func send(
|
||||
client *http.Client,
|
||||
appservice config.ApplicationService,
|
||||
txnID int,
|
||||
transaction []byte,
|
||||
) (err error) {
|
||||
// PUT a transaction to our AS
|
||||
// https://matrix.org/docs/spec/application_service/r0.1.2#put-matrix-app-v1-transactions-txnid
|
||||
address := fmt.Sprintf("%s/transactions/%d?access_token=%s", appservice.URL, txnID, url.QueryEscape(appservice.HSToken))
|
||||
req, err := http.NewRequest("PUT", address, bytes.NewBuffer(transaction))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
req.Header.Set("Content-Type", "application/json")
|
||||
resp, err := client.Do(req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer checkNamedErr(resp.Body.Close, &err)
|
||||
|
||||
// Check the AS received the events correctly
|
||||
if resp.StatusCode != http.StatusOK {
|
||||
// TODO: Handle non-200 error codes from application services
|
||||
return fmt.Errorf("non-OK status code %d returned from AS", resp.StatusCode)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// checkNamedErr calls fn and overwrite err if it was nil and fn returned non-nil
|
||||
func checkNamedErr(fn func() error, err *error) {
|
||||
if e := fn(); e != nil && *err == nil {
|
||||
*err = e
|
||||
}
|
||||
}
|
||||
|
|
@ -242,7 +242,10 @@ func (m *DendriteMonolith) Start() {
|
|||
|
||||
prefix := hex.EncodeToString(pk)
|
||||
cfg := &config.Dendrite{}
|
||||
cfg.Defaults(true)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.ServerName = gomatrixserverlib.ServerName(hex.EncodeToString(pk))
|
||||
cfg.Global.PrivateKey = sk
|
||||
cfg.Global.KeyID = gomatrixserverlib.KeyID(signing.KeyID)
|
||||
|
|
@ -254,7 +257,6 @@ func (m *DendriteMonolith) Start() {
|
|||
cfg.RoomServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s/%s-roomserver.db", m.StorageDirectory, prefix))
|
||||
cfg.KeyServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s/%s-keyserver.db", m.StorageDirectory, prefix))
|
||||
cfg.FederationAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s/%s-federationsender.db", m.StorageDirectory, prefix))
|
||||
cfg.AppServiceAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s/%s-appservice.db", m.StorageDirectory, prefix))
|
||||
cfg.MediaAPI.BasePath = config.Path(fmt.Sprintf("%s/media", m.CacheDirectory))
|
||||
cfg.MediaAPI.AbsBasePath = config.Path(fmt.Sprintf("%s/media", m.CacheDirectory))
|
||||
cfg.MSCs.MSCs = []string{"msc2836", "msc2946"}
|
||||
|
|
|
|||
|
|
@ -82,7 +82,10 @@ func (m *DendriteMonolith) Start() {
|
|||
m.YggdrasilNode = ygg
|
||||
|
||||
cfg := &config.Dendrite{}
|
||||
cfg.Defaults(true)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.ServerName = gomatrixserverlib.ServerName(ygg.DerivedServerName())
|
||||
cfg.Global.PrivateKey = ygg.PrivateKey()
|
||||
cfg.Global.KeyID = gomatrixserverlib.KeyID(signing.KeyID)
|
||||
|
|
@ -94,7 +97,6 @@ func (m *DendriteMonolith) Start() {
|
|||
cfg.RoomServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s/dendrite-p2p-roomserver.db", m.StorageDirectory))
|
||||
cfg.KeyServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s/dendrite-p2p-keyserver.db", m.StorageDirectory))
|
||||
cfg.FederationAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s/dendrite-p2p-federationsender.db", m.StorageDirectory))
|
||||
cfg.AppServiceAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s/dendrite-p2p-appservice.db", m.StorageDirectory))
|
||||
cfg.MediaAPI.BasePath = config.Path(fmt.Sprintf("%s/tmp", m.StorageDirectory))
|
||||
cfg.MediaAPI.AbsBasePath = config.Path(fmt.Sprintf("%s/tmp", m.StorageDirectory))
|
||||
cfg.ClientAPI.RegistrationDisabled = false
|
||||
|
|
|
|||
|
|
@ -46,9 +46,8 @@ EXPOSE 8008 8448
|
|||
# At runtime, generate TLS cert based on the CA now mounted at /ca
|
||||
# At runtime, replace the SERVER_NAME with what we are told
|
||||
CMD /build/run_postgres.sh && ./generate-keys --keysize 1024 --server $SERVER_NAME --tls-cert server.crt --tls-key server.key --tls-authority-cert /complement/ca/ca.crt --tls-authority-key /complement/ca/ca.key && \
|
||||
./generate-config -server $SERVER_NAME --ci > dendrite.yaml && \
|
||||
# Replace the connection string with a single postgres DB, using user/db = 'postgres' and no password, bump max_conns
|
||||
sed -i "s%connection_string:.*$%connection_string: postgresql://postgres@localhost/postgres?sslmode=disable%g" dendrite.yaml && \
|
||||
sed -i 's/max_open_conns:.*$/max_open_conns: 100/g' dendrite.yaml && \
|
||||
./generate-config -server $SERVER_NAME --ci --db postgresql://postgres@localhost/postgres?sslmode=disable > dendrite.yaml && \
|
||||
# Bump max_open_conns up here in the global database config
|
||||
sed -i 's/max_open_conns:.*$/max_open_conns: 1990/g' dendrite.yaml && \
|
||||
cp /complement/ca/ca.crt /usr/local/share/ca-certificates/ && update-ca-certificates && \
|
||||
exec ./dendrite-monolith-server --really-enable-open-registration --tls-cert server.crt --tls-key server.key --config dendrite.yaml -api=${API:-0}
|
||||
|
|
@ -278,19 +278,19 @@ type recaptchaResponse struct {
|
|||
}
|
||||
|
||||
// validateUsername returns an error response if the username is invalid
|
||||
func validateUsername(username string) *util.JSONResponse {
|
||||
func validateUsername(localpart string, domain gomatrixserverlib.ServerName) *util.JSONResponse {
|
||||
// https://github.com/matrix-org/synapse/blob/v0.20.0/synapse/rest/client/v2_alpha/register.py#L161
|
||||
if len(username) > maxUsernameLength {
|
||||
if id := fmt.Sprintf("@%s:%s", localpart, domain); len(id) > maxUsernameLength {
|
||||
return &util.JSONResponse{
|
||||
Code: http.StatusBadRequest,
|
||||
JSON: jsonerror.BadJSON(fmt.Sprintf("'username' >%d characters", maxUsernameLength)),
|
||||
JSON: jsonerror.BadJSON(fmt.Sprintf("%q exceeds the maximum length of %d characters", id, maxUsernameLength)),
|
||||
}
|
||||
} else if !validUsernameRegex.MatchString(username) {
|
||||
} else if !validUsernameRegex.MatchString(localpart) {
|
||||
return &util.JSONResponse{
|
||||
Code: http.StatusBadRequest,
|
||||
JSON: jsonerror.InvalidUsername("Username can only contain characters a-z, 0-9, or '_-./='"),
|
||||
}
|
||||
} else if username[0] == '_' { // Regex checks its not a zero length string
|
||||
} else if localpart[0] == '_' { // Regex checks its not a zero length string
|
||||
return &util.JSONResponse{
|
||||
Code: http.StatusBadRequest,
|
||||
JSON: jsonerror.InvalidUsername("Username cannot start with a '_'"),
|
||||
|
|
@ -300,13 +300,13 @@ func validateUsername(username string) *util.JSONResponse {
|
|||
}
|
||||
|
||||
// validateApplicationServiceUsername returns an error response if the username is invalid for an application service
|
||||
func validateApplicationServiceUsername(username string) *util.JSONResponse {
|
||||
if len(username) > maxUsernameLength {
|
||||
func validateApplicationServiceUsername(localpart string, domain gomatrixserverlib.ServerName) *util.JSONResponse {
|
||||
if id := fmt.Sprintf("@%s:%s", localpart, domain); len(id) > maxUsernameLength {
|
||||
return &util.JSONResponse{
|
||||
Code: http.StatusBadRequest,
|
||||
JSON: jsonerror.BadJSON(fmt.Sprintf("'username' >%d characters", maxUsernameLength)),
|
||||
JSON: jsonerror.BadJSON(fmt.Sprintf("%q exceeds the maximum length of %d characters", id, maxUsernameLength)),
|
||||
}
|
||||
} else if !validUsernameRegex.MatchString(username) {
|
||||
} else if !validUsernameRegex.MatchString(localpart) {
|
||||
return &util.JSONResponse{
|
||||
Code: http.StatusBadRequest,
|
||||
JSON: jsonerror.InvalidUsername("Username can only contain characters a-z, 0-9, or '_-./='"),
|
||||
|
|
@ -525,7 +525,7 @@ func validateApplicationService(
|
|||
}
|
||||
|
||||
// Check username application service is trying to register is valid
|
||||
if err := validateApplicationServiceUsername(username); err != nil {
|
||||
if err := validateApplicationServiceUsername(username, cfg.Matrix.ServerName); err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
|
|
@ -606,7 +606,7 @@ func Register(
|
|||
case r.Type == authtypes.LoginTypeApplicationService && accessTokenErr == nil:
|
||||
// Spec-compliant case (the access_token is specified and the login type
|
||||
// is correctly set, so it's an appservice registration)
|
||||
if resErr := validateApplicationServiceUsername(r.Username); resErr != nil {
|
||||
if resErr := validateApplicationServiceUsername(r.Username, cfg.Matrix.ServerName); resErr != nil {
|
||||
return *resErr
|
||||
}
|
||||
case accessTokenErr == nil:
|
||||
|
|
@ -619,7 +619,7 @@ func Register(
|
|||
default:
|
||||
// Spec-compliant case (neither the access_token nor the login type are
|
||||
// specified, so it's a normal user registration)
|
||||
if resErr := validateUsername(r.Username); resErr != nil {
|
||||
if resErr := validateUsername(r.Username, cfg.Matrix.ServerName); resErr != nil {
|
||||
return *resErr
|
||||
}
|
||||
}
|
||||
|
|
@ -1061,7 +1061,7 @@ func RegisterAvailable(
|
|||
// Squash username to all lowercase letters
|
||||
username = strings.ToLower(username)
|
||||
|
||||
if err := validateUsername(username); err != nil {
|
||||
if err := validateUsername(username, cfg.Matrix.ServerName); err != nil {
|
||||
return *err
|
||||
}
|
||||
|
||||
|
|
@ -1102,7 +1102,7 @@ func RegisterAvailable(
|
|||
}
|
||||
}
|
||||
|
||||
func handleSharedSecretRegistration(userAPI userapi.ClientUserAPI, sr *SharedSecretRegistration, req *http.Request) util.JSONResponse {
|
||||
func handleSharedSecretRegistration(cfg *config.ClientAPI, userAPI userapi.ClientUserAPI, sr *SharedSecretRegistration, req *http.Request) util.JSONResponse {
|
||||
ssrr, err := NewSharedSecretRegistrationRequest(req.Body)
|
||||
if err != nil {
|
||||
return util.JSONResponse{
|
||||
|
|
@ -1123,7 +1123,7 @@ func handleSharedSecretRegistration(userAPI userapi.ClientUserAPI, sr *SharedSec
|
|||
// downcase capitals
|
||||
ssrr.User = strings.ToLower(ssrr.User)
|
||||
|
||||
if resErr := validateUsername(ssrr.User); resErr != nil {
|
||||
if resErr := validateUsername(ssrr.User, cfg.Matrix.ServerName); resErr != nil {
|
||||
return *resErr
|
||||
}
|
||||
if resErr := validatePassword(ssrr.Password); resErr != nil {
|
||||
|
|
|
|||
|
|
@ -181,7 +181,10 @@ func TestValidationOfApplicationServices(t *testing.T) {
|
|||
|
||||
// Set up a config
|
||||
fakeConfig := &config.Dendrite{}
|
||||
fakeConfig.Defaults(true)
|
||||
fakeConfig.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
fakeConfig.Global.ServerName = "localhost"
|
||||
fakeConfig.ClientAPI.Derived.ApplicationServices = []config.ApplicationService{fakeApplicationService}
|
||||
|
||||
|
|
|
|||
|
|
@ -135,7 +135,7 @@ func Setup(
|
|||
}
|
||||
}
|
||||
if req.Method == http.MethodPost {
|
||||
return handleSharedSecretRegistration(userAPI, sr, req)
|
||||
return handleSharedSecretRegistration(cfg, userAPI, sr, req)
|
||||
}
|
||||
return util.JSONResponse{
|
||||
Code: http.StatusMethodNotAllowed,
|
||||
|
|
|
|||
|
|
@ -24,6 +24,7 @@ import (
|
|||
"net"
|
||||
"net/http"
|
||||
"os"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
|
|
@ -42,6 +43,7 @@ import (
|
|||
"github.com/matrix-org/dendrite/setup"
|
||||
"github.com/matrix-org/dendrite/setup/base"
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
"github.com/matrix-org/dendrite/test"
|
||||
"github.com/matrix-org/dendrite/userapi"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
|
||||
|
|
@ -70,23 +72,76 @@ func main() {
|
|||
var pk ed25519.PublicKey
|
||||
var sk ed25519.PrivateKey
|
||||
|
||||
keyfile := *instanceName + ".key"
|
||||
// iterate through the cli args and check if the config flag was set
|
||||
configFlagSet := false
|
||||
for _, arg := range os.Args {
|
||||
if arg == "--config" || arg == "-config" {
|
||||
configFlagSet = true
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
cfg := &config.Dendrite{}
|
||||
|
||||
// use custom config if config flag is set
|
||||
if configFlagSet {
|
||||
cfg = setup.ParseFlags(true)
|
||||
sk = cfg.Global.PrivateKey
|
||||
} else {
|
||||
keyfile := *instanceName + ".pem"
|
||||
if _, err := os.Stat(keyfile); os.IsNotExist(err) {
|
||||
if pk, sk, err = ed25519.GenerateKey(nil); err != nil {
|
||||
panic(err)
|
||||
oldkeyfile := *instanceName + ".key"
|
||||
if _, err = os.Stat(oldkeyfile); os.IsNotExist(err) {
|
||||
if err = test.NewMatrixKey(keyfile); err != nil {
|
||||
panic("failed to generate a new PEM key: " + err.Error())
|
||||
}
|
||||
if err = os.WriteFile(keyfile, sk, 0644); err != nil {
|
||||
panic(err)
|
||||
if _, sk, err = config.LoadMatrixKey(keyfile, os.ReadFile); err != nil {
|
||||
panic("failed to load PEM key: " + err.Error())
|
||||
}
|
||||
} else if err == nil {
|
||||
if sk, err = os.ReadFile(keyfile); err != nil {
|
||||
panic(err)
|
||||
} else {
|
||||
if sk, err = os.ReadFile(oldkeyfile); err != nil {
|
||||
panic("failed to read the old private key: " + err.Error())
|
||||
}
|
||||
if len(sk) != ed25519.PrivateKeySize {
|
||||
panic("the private key is not long enough")
|
||||
}
|
||||
pk = sk.Public().(ed25519.PublicKey)
|
||||
if err := test.SaveMatrixKey(keyfile, sk); err != nil {
|
||||
panic("failed to convert the private key to PEM format: " + err.Error())
|
||||
}
|
||||
}
|
||||
} else {
|
||||
var err error
|
||||
if _, sk, err = config.LoadMatrixKey(keyfile, os.ReadFile); err != nil {
|
||||
panic("failed to load PEM key: " + err.Error())
|
||||
}
|
||||
}
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.PrivateKey = sk
|
||||
cfg.Global.JetStream.StoragePath = config.Path(fmt.Sprintf("%s/", *instanceName))
|
||||
cfg.UserAPI.AccountDatabase.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-account.db", *instanceName))
|
||||
cfg.MediaAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-mediaapi.db", *instanceName))
|
||||
cfg.SyncAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-syncapi.db", *instanceName))
|
||||
cfg.RoomServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-roomserver.db", *instanceName))
|
||||
cfg.KeyServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-keyserver.db", *instanceName))
|
||||
cfg.FederationAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-federationapi.db", *instanceName))
|
||||
cfg.MSCs.MSCs = []string{"msc2836", "msc2946"}
|
||||
cfg.MSCs.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-mscs.db", *instanceName))
|
||||
cfg.ClientAPI.RegistrationDisabled = false
|
||||
cfg.ClientAPI.OpenRegistrationWithoutVerificationEnabled = true
|
||||
if err := cfg.Derive(); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
pk = sk.Public().(ed25519.PublicKey)
|
||||
cfg.Global.ServerName = gomatrixserverlib.ServerName(hex.EncodeToString(pk))
|
||||
cfg.Global.KeyID = gomatrixserverlib.KeyID(signing.KeyID)
|
||||
|
||||
base := base.NewBaseDendrite(cfg, "Monolith")
|
||||
defer base.Close() // nolint: errcheck
|
||||
|
||||
pRouter := pineconeRouter.NewRouter(logrus.WithField("pinecone", "router"), sk, false)
|
||||
pQUIC := pineconeSessions.NewSessions(logrus.WithField("pinecone", "sessions"), pRouter, []string{"matrix"})
|
||||
|
|
@ -94,7 +149,9 @@ func main() {
|
|||
pManager := pineconeConnections.NewConnectionManager(pRouter, nil)
|
||||
pMulticast.Start()
|
||||
if instancePeer != nil && *instancePeer != "" {
|
||||
pManager.AddPeer(*instancePeer)
|
||||
for _, peer := range strings.Split(*instancePeer, ",") {
|
||||
pManager.AddPeer(strings.Trim(peer, " \t\r\n"))
|
||||
}
|
||||
}
|
||||
|
||||
go func() {
|
||||
|
|
@ -125,29 +182,6 @@ func main() {
|
|||
}
|
||||
}()
|
||||
|
||||
cfg := &config.Dendrite{}
|
||||
cfg.Defaults(true)
|
||||
cfg.Global.ServerName = gomatrixserverlib.ServerName(hex.EncodeToString(pk))
|
||||
cfg.Global.PrivateKey = sk
|
||||
cfg.Global.KeyID = gomatrixserverlib.KeyID(signing.KeyID)
|
||||
cfg.Global.JetStream.StoragePath = config.Path(fmt.Sprintf("%s/", *instanceName))
|
||||
cfg.UserAPI.AccountDatabase.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-account.db", *instanceName))
|
||||
cfg.MediaAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-mediaapi.db", *instanceName))
|
||||
cfg.SyncAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-syncapi.db", *instanceName))
|
||||
cfg.RoomServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-roomserver.db", *instanceName))
|
||||
cfg.KeyServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-keyserver.db", *instanceName))
|
||||
cfg.FederationAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-federationapi.db", *instanceName))
|
||||
cfg.AppServiceAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-appservice.db", *instanceName))
|
||||
cfg.MSCs.MSCs = []string{"msc2836", "msc2946"}
|
||||
cfg.ClientAPI.RegistrationDisabled = false
|
||||
cfg.ClientAPI.OpenRegistrationWithoutVerificationEnabled = true
|
||||
if err := cfg.Derive(); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
base := base.NewBaseDendrite(cfg, "Monolith")
|
||||
defer base.Close() // nolint: errcheck
|
||||
|
||||
federation := conn.CreateFederationClient(base, pQUIC)
|
||||
|
||||
serverKeyAPI := &signing.YggdrasilKeys{}
|
||||
|
|
|
|||
|
|
@ -78,7 +78,10 @@ func main() {
|
|||
if configFlagSet {
|
||||
cfg = setup.ParseFlags(true)
|
||||
} else {
|
||||
cfg.Defaults(true)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.JetStream.StoragePath = config.Path(fmt.Sprintf("%s/", *instanceName))
|
||||
cfg.UserAPI.AccountDatabase.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-account.db", *instanceName))
|
||||
cfg.MediaAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-mediaapi.db", *instanceName))
|
||||
|
|
@ -86,7 +89,6 @@ func main() {
|
|||
cfg.RoomServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-roomserver.db", *instanceName))
|
||||
cfg.KeyServer.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-keyserver.db", *instanceName))
|
||||
cfg.FederationAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-federationapi.db", *instanceName))
|
||||
cfg.AppServiceAPI.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-appservice.db", *instanceName))
|
||||
cfg.MSCs.MSCs = []string{"msc2836"}
|
||||
cfg.MSCs.Database.ConnectionString = config.DataSource(fmt.Sprintf("file:%s-mscs.db", *instanceName))
|
||||
cfg.ClientAPI.RegistrationDisabled = false
|
||||
|
|
|
|||
|
|
@ -3,6 +3,7 @@ package main
|
|||
import (
|
||||
"flag"
|
||||
"fmt"
|
||||
"path/filepath"
|
||||
|
||||
"github.com/matrix-org/dendrite/setup/config"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
|
|
@ -11,85 +12,69 @@ import (
|
|||
)
|
||||
|
||||
func main() {
|
||||
defaultsForCI := flag.Bool("ci", false, "sane defaults for CI testing")
|
||||
defaultsForCI := flag.Bool("ci", false, "Populate the configuration with sane defaults for use in CI")
|
||||
serverName := flag.String("server", "", "The domain name of the server if not 'localhost'")
|
||||
dbURI := flag.String("db", "", "The DB URI to use for all components if not SQLite files")
|
||||
dbURI := flag.String("db", "", "The DB URI to use for all components (PostgreSQL only)")
|
||||
dirPath := flag.String("dir", "./", "The folder to use for paths (like SQLite databases, media storage)")
|
||||
normalise := flag.String("normalise", "", "Normalise an existing configuration file by adding new/missing options and defaults")
|
||||
polylith := flag.Bool("polylith", false, "Generate a config that makes sense for polylith deployments")
|
||||
flag.Parse()
|
||||
|
||||
cfg := &config.Dendrite{
|
||||
var cfg *config.Dendrite
|
||||
if *normalise == "" {
|
||||
cfg = &config.Dendrite{
|
||||
Version: config.Version,
|
||||
}
|
||||
cfg.Defaults(true)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: !*polylith,
|
||||
})
|
||||
if *serverName != "" {
|
||||
cfg.Global.ServerName = gomatrixserverlib.ServerName(*serverName)
|
||||
}
|
||||
if *dbURI != "" {
|
||||
cfg.AppServiceAPI.Database.ConnectionString = config.DataSource(*dbURI)
|
||||
cfg.FederationAPI.Database.ConnectionString = config.DataSource(*dbURI)
|
||||
cfg.KeyServer.Database.ConnectionString = config.DataSource(*dbURI)
|
||||
cfg.MSCs.Database.ConnectionString = config.DataSource(*dbURI)
|
||||
cfg.MediaAPI.Database.ConnectionString = config.DataSource(*dbURI)
|
||||
cfg.RoomServer.Database.ConnectionString = config.DataSource(*dbURI)
|
||||
cfg.SyncAPI.Database.ConnectionString = config.DataSource(*dbURI)
|
||||
cfg.UserAPI.AccountDatabase.ConnectionString = config.DataSource(*dbURI)
|
||||
uri := config.DataSource(*dbURI)
|
||||
if *polylith || uri.IsSQLite() || uri == "" {
|
||||
for name, db := range map[string]*config.DatabaseOptions{
|
||||
"federationapi": &cfg.FederationAPI.Database,
|
||||
"keyserver": &cfg.KeyServer.Database,
|
||||
"mscs": &cfg.MSCs.Database,
|
||||
"mediaapi": &cfg.MediaAPI.Database,
|
||||
"roomserver": &cfg.RoomServer.Database,
|
||||
"syncapi": &cfg.SyncAPI.Database,
|
||||
"userapi": &cfg.UserAPI.AccountDatabase,
|
||||
} {
|
||||
if uri == "" {
|
||||
path := filepath.Join(*dirPath, fmt.Sprintf("dendrite_%s.db", name))
|
||||
db.ConnectionString = config.DataSource(fmt.Sprintf("file:%s", path))
|
||||
} else {
|
||||
db.ConnectionString = uri
|
||||
}
|
||||
cfg.Global.TrustedIDServers = []string{
|
||||
"matrix.org",
|
||||
"vector.im",
|
||||
}
|
||||
} else {
|
||||
cfg.Global.DatabaseOptions.ConnectionString = uri
|
||||
}
|
||||
cfg.Logging = []config.LogrusHook{
|
||||
{
|
||||
Type: "file",
|
||||
Level: "info",
|
||||
Params: map[string]interface{}{
|
||||
"path": "/var/log/dendrite",
|
||||
"path": filepath.Join(*dirPath, "log"),
|
||||
},
|
||||
},
|
||||
}
|
||||
cfg.FederationAPI.KeyPerspectives = config.KeyPerspectives{
|
||||
{
|
||||
ServerName: "matrix.org",
|
||||
Keys: []config.KeyPerspectiveTrustKey{
|
||||
{
|
||||
KeyID: "ed25519:auto",
|
||||
PublicKey: "Noi6WqcDj0QmPxCNQqgezwTlBKrfqehY1u2FyWP9uYw",
|
||||
},
|
||||
{
|
||||
KeyID: "ed25519:a_RXGa",
|
||||
PublicKey: "l8Hft5qXKn1vfHrg3p4+W8gELQVo8N13JkluMfmn2sQ",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
cfg.MediaAPI.ThumbnailSizes = []config.ThumbnailSize{
|
||||
{
|
||||
Width: 32,
|
||||
Height: 32,
|
||||
ResizeMethod: "crop",
|
||||
},
|
||||
{
|
||||
Width: 96,
|
||||
Height: 96,
|
||||
ResizeMethod: "crop",
|
||||
},
|
||||
{
|
||||
Width: 640,
|
||||
Height: 480,
|
||||
ResizeMethod: "scale",
|
||||
},
|
||||
}
|
||||
|
||||
if *defaultsForCI {
|
||||
cfg.AppServiceAPI.DisableTLSValidation = true
|
||||
cfg.ClientAPI.RateLimiting.Enabled = false
|
||||
cfg.FederationAPI.DisableTLSValidation = false
|
||||
// don't hit matrix.org when running tests!!!
|
||||
cfg.FederationAPI.KeyPerspectives = config.KeyPerspectives{}
|
||||
cfg.MediaAPI.BasePath = config.Path(filepath.Join(*dirPath, "media"))
|
||||
cfg.MSCs.MSCs = []string{"msc2836", "msc2946", "msc2444", "msc2753"}
|
||||
cfg.Logging[0].Level = "trace"
|
||||
cfg.Logging[0].Type = "std"
|
||||
cfg.UserAPI.BCryptCost = bcrypt.MinCost
|
||||
cfg.Global.JetStream.InMemory = true
|
||||
cfg.Global.JetStream.StoragePath = config.Path(*dirPath)
|
||||
cfg.ClientAPI.RegistrationDisabled = false
|
||||
cfg.ClientAPI.OpenRegistrationWithoutVerificationEnabled = true
|
||||
cfg.ClientAPI.RegistrationSharedSecret = "complement"
|
||||
|
|
@ -98,6 +83,12 @@ func main() {
|
|||
EnableOutbound: true,
|
||||
}
|
||||
}
|
||||
} else {
|
||||
var err error
|
||||
if cfg, err = config.Load(*normalise, !*polylith); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
j, err := yaml.Marshal(cfg)
|
||||
if err != nil {
|
||||
|
|
|
|||
|
|
@ -132,13 +132,6 @@ app_service_api:
|
|||
listen: http://[::]:7777 # The listen address for incoming API requests
|
||||
connect: http://app_service_api:7777 # The connect address for other components to use
|
||||
|
||||
# Database configuration for this component.
|
||||
database:
|
||||
connection_string: postgresql://username:password@hostname/dendrite_appservice?sslmode=disable
|
||||
max_open_conns: 10
|
||||
max_idle_conns: 2
|
||||
conn_max_lifetime: -1
|
||||
|
||||
# Disable the validation of TLS certificates of appservices. This is
|
||||
# not recommended in production since it may allow appservice traffic
|
||||
# to be sent to an insecure endpoint.
|
||||
|
|
|
|||
|
|
@ -67,14 +67,15 @@ func NewKeyChangeConsumer(
|
|||
// Start consuming from key servers
|
||||
func (t *KeyChangeConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
t.ctx, t.jetstream, t.topic, t.durable, t.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
t.ctx, t.jetstream, t.topic, t.durable, 1,
|
||||
t.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
// onMessage is called in response to a message received on the
|
||||
// key change events topic from the key server.
|
||||
func (t *KeyChangeConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (t *KeyChangeConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
var m api.DeviceMessage
|
||||
if err := json.Unmarshal(msg.Data, &m); err != nil {
|
||||
logrus.WithError(err).Errorf("failed to read device message from key change topic")
|
||||
|
|
|
|||
|
|
@ -69,14 +69,15 @@ func (t *OutputPresenceConsumer) Start() error {
|
|||
return nil
|
||||
}
|
||||
return jetstream.JetStreamConsumer(
|
||||
t.ctx, t.jetstream, t.topic, t.durable, t.onMessage,
|
||||
t.ctx, t.jetstream, t.topic, t.durable, 1, t.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(), nats.HeadersOnly(),
|
||||
)
|
||||
}
|
||||
|
||||
// onMessage is called in response to a message received on the presence
|
||||
// events topic from the client api.
|
||||
func (t *OutputPresenceConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (t *OutputPresenceConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
// only send presence events which originated from us
|
||||
userID := msg.Header.Get(jetstream.UserID)
|
||||
_, serverName, err := gomatrixserverlib.SplitID('@', userID)
|
||||
|
|
|
|||
|
|
@ -65,14 +65,15 @@ func NewOutputReceiptConsumer(
|
|||
// Start consuming from the clientapi
|
||||
func (t *OutputReceiptConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
t.ctx, t.jetstream, t.topic, t.durable, t.onMessage,
|
||||
t.ctx, t.jetstream, t.topic, t.durable, 1, t.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(), nats.HeadersOnly(),
|
||||
)
|
||||
}
|
||||
|
||||
// onMessage is called in response to a message received on the receipt
|
||||
// events topic from the client api.
|
||||
func (t *OutputReceiptConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (t *OutputReceiptConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
receipt := syncTypes.OutputReceiptEvent{
|
||||
UserID: msg.Header.Get(jetstream.UserID),
|
||||
RoomID: msg.Header.Get(jetstream.RoomID),
|
||||
|
|
|
|||
|
|
@ -68,8 +68,8 @@ func NewOutputRoomEventConsumer(
|
|||
// Start consuming from room servers
|
||||
func (s *OutputRoomEventConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -77,7 +77,8 @@ func (s *OutputRoomEventConsumer) Start() error {
|
|||
// It is unsafe to call this with messages for the same room in multiple gorountines
|
||||
// because updates it will likely fail with a types.EventIDMismatchError when it
|
||||
// realises that it cannot update the room state using the deltas.
|
||||
func (s *OutputRoomEventConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputRoomEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
// Parse out the event JSON
|
||||
var output api.OutputEvent
|
||||
if err := json.Unmarshal(msg.Data, &output); err != nil {
|
||||
|
|
|
|||
|
|
@ -63,14 +63,15 @@ func NewOutputSendToDeviceConsumer(
|
|||
// Start consuming from the client api
|
||||
func (t *OutputSendToDeviceConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
t.ctx, t.jetstream, t.topic, t.durable, t.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
t.ctx, t.jetstream, t.topic, t.durable, 1,
|
||||
t.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
// onMessage is called in response to a message received on the
|
||||
// send-to-device events topic from the client api.
|
||||
func (t *OutputSendToDeviceConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (t *OutputSendToDeviceConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
// only send send-to-device events which originated from us
|
||||
sender := msg.Header.Get("sender")
|
||||
_, originServerName, err := gomatrixserverlib.SplitID('@', sender)
|
||||
|
|
|
|||
|
|
@ -62,14 +62,15 @@ func NewOutputTypingConsumer(
|
|||
// Start consuming from the clientapi
|
||||
func (t *OutputTypingConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
t.ctx, t.jetstream, t.topic, t.durable, t.onMessage,
|
||||
t.ctx, t.jetstream, t.topic, t.durable, 1, t.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(), nats.HeadersOnly(),
|
||||
)
|
||||
}
|
||||
|
||||
// onMessage is called in response to a message received on the typing
|
||||
// events topic from the client api.
|
||||
func (t *OutputTypingConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (t *OutputTypingConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
// Extract the typing event from msg.
|
||||
roomID := msg.Header.Get(jetstream.RoomID)
|
||||
userID := msg.Header.Get(jetstream.UserID)
|
||||
|
|
|
|||
|
|
@ -75,7 +75,10 @@ func TestMain(m *testing.M) {
|
|||
// Draw up just enough Dendrite config for the server key
|
||||
// API to work.
|
||||
cfg := &config.Dendrite{}
|
||||
cfg.Defaults(true)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.ServerName = gomatrixserverlib.ServerName(s.name)
|
||||
cfg.Global.PrivateKey = testPriv
|
||||
cfg.Global.JetStream.InMemory = true
|
||||
|
|
|
|||
|
|
@ -263,7 +263,10 @@ func testFederationAPIJoinThenKeyUpdate(t *testing.T, dbType test.DBType) {
|
|||
func TestRoomsV3URLEscapeDoNot404(t *testing.T) {
|
||||
_, privKey, _ := ed25519.GenerateKey(nil)
|
||||
cfg := &config.Dendrite{}
|
||||
cfg.Defaults(true)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.KeyID = gomatrixserverlib.KeyID("ed25519:auto")
|
||||
cfg.Global.ServerName = gomatrixserverlib.ServerName("localhost")
|
||||
cfg.Global.PrivateKey = privKey
|
||||
|
|
|
|||
2
go.mod
2
go.mod
|
|
@ -22,7 +22,7 @@ require (
|
|||
github.com/matrix-org/dugong v0.0.0-20210921133753-66e6b1c67e2e
|
||||
github.com/matrix-org/go-sqlite3-js v0.0.0-20220419092513-28aa791a1c91
|
||||
github.com/matrix-org/gomatrix v0.0.0-20210324163249-be2af5ef2e16
|
||||
github.com/matrix-org/gomatrixserverlib v0.0.0-20220824082345-662dca17bf94
|
||||
github.com/matrix-org/gomatrixserverlib v0.0.0-20220830164018-c71e518537a2
|
||||
github.com/matrix-org/pinecone v0.0.0-20220803093810-b7a830c08fb9
|
||||
github.com/matrix-org/util v0.0.0-20200807132607-55161520e1d4
|
||||
github.com/matryer/is v1.4.0
|
||||
|
|
|
|||
4
go.sum
4
go.sum
|
|
@ -343,8 +343,8 @@ github.com/matrix-org/go-sqlite3-js v0.0.0-20220419092513-28aa791a1c91/go.mod h1
|
|||
github.com/matrix-org/gomatrix v0.0.0-20190528120928-7df988a63f26/go.mod h1:3fxX6gUjWyI/2Bt7J1OLhpCzOfO/bB3AiX0cJtEKud0=
|
||||
github.com/matrix-org/gomatrix v0.0.0-20210324163249-be2af5ef2e16 h1:ZtO5uywdd5dLDCud4r0r55eP4j9FuUNpl60Gmntcop4=
|
||||
github.com/matrix-org/gomatrix v0.0.0-20210324163249-be2af5ef2e16/go.mod h1:/gBX06Kw0exX1HrwmoBibFA98yBk/jxKpGVeyQbff+s=
|
||||
github.com/matrix-org/gomatrixserverlib v0.0.0-20220824082345-662dca17bf94 h1:zoTv/qxg7C/O995JBPvp+Z8KMR69HhB+M+P22A8Hmm0=
|
||||
github.com/matrix-org/gomatrixserverlib v0.0.0-20220824082345-662dca17bf94/go.mod h1:jX38yp3SSLJNftBg3PXU1ayd0PCLIiDHQ4xAc9DIixk=
|
||||
github.com/matrix-org/gomatrixserverlib v0.0.0-20220830164018-c71e518537a2 h1:esbNn9hg//tAStA6TogatAJAursw23A+yfVRQsdiv70=
|
||||
github.com/matrix-org/gomatrixserverlib v0.0.0-20220830164018-c71e518537a2/go.mod h1:jX38yp3SSLJNftBg3PXU1ayd0PCLIiDHQ4xAc9DIixk=
|
||||
github.com/matrix-org/pinecone v0.0.0-20220803093810-b7a830c08fb9 h1:ed8yvWhTLk7+sNeK/eOZRTvESFTOHDRevoRoyeqPtvY=
|
||||
github.com/matrix-org/pinecone v0.0.0-20220803093810-b7a830c08fb9/go.mod h1:P4MqPf+u83OPulPJ+XTbSDbbWrdFYNY4LZ/B1PIduFE=
|
||||
github.com/matrix-org/util v0.0.0-20190711121626-527ce5ddefc7/go.mod h1:vVQlW/emklohkZnOPwD3LrZUBqdfsbiyO3p1lNV8F6U=
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ var build string
|
|||
const (
|
||||
VersionMajor = 0
|
||||
VersionMinor = 9
|
||||
VersionPatch = 5
|
||||
VersionPatch = 6
|
||||
VersionTag = "" // example: "rc1"
|
||||
)
|
||||
|
||||
|
|
|
|||
|
|
@ -55,14 +55,15 @@ func NewDeviceListUpdateConsumer(
|
|||
// Start consuming from key servers
|
||||
func (t *DeviceListUpdateConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
t.ctx, t.jetstream, t.topic, t.durable, t.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
t.ctx, t.jetstream, t.topic, t.durable, 1,
|
||||
t.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
// onMessage is called in response to a message received on the
|
||||
// key change events topic from the key server.
|
||||
func (t *DeviceListUpdateConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (t *DeviceListUpdateConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
var m gomatrixserverlib.DeviceListUpdateEvent
|
||||
if err := json.Unmarshal(msg.Data, &m); err != nil {
|
||||
logrus.WithError(err).Errorf("Failed to read from device list update input topic")
|
||||
|
|
|
|||
|
|
@ -60,7 +60,7 @@ func NewInternalAPI(
|
|||
updater := internal.NewDeviceListUpdater(db, ap, keyChangeProducer, fedClient, 8) // 8 workers TODO: configurable
|
||||
ap.Updater = updater
|
||||
go func() {
|
||||
if err = updater.Start(); err != nil {
|
||||
if err := updater.Start(); err != nil {
|
||||
logrus.WithError(err).Panicf("failed to start device list updater")
|
||||
}
|
||||
}()
|
||||
|
|
@ -68,7 +68,7 @@ func NewInternalAPI(
|
|||
dlConsumer := consumers.NewDeviceListUpdateConsumer(
|
||||
base.ProcessContext, cfg, js, updater,
|
||||
)
|
||||
if err = dlConsumer.Start(); err != nil {
|
||||
if err := dlConsumer.Start(); err != nil {
|
||||
logrus.WithError(err).Panic("failed to start device list consumer")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -25,21 +25,23 @@ import (
|
|||
_ "net/http/pprof"
|
||||
"os"
|
||||
"os/signal"
|
||||
"sync"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/getsentry/sentry-go"
|
||||
sentryhttp "github.com/getsentry/sentry-go/http"
|
||||
"github.com/matrix-org/dendrite/internal/caching"
|
||||
"github.com/matrix-org/dendrite/internal/httputil"
|
||||
"github.com/matrix-org/dendrite/internal/pushgateway"
|
||||
"github.com/matrix-org/dendrite/internal/sqlutil"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
"github.com/prometheus/client_golang/prometheus/promhttp"
|
||||
"go.uber.org/atomic"
|
||||
"golang.org/x/net/http2"
|
||||
"golang.org/x/net/http2/h2c"
|
||||
|
||||
"github.com/matrix-org/dendrite/internal/caching"
|
||||
"github.com/matrix-org/dendrite/internal/httputil"
|
||||
"github.com/matrix-org/dendrite/internal/pushgateway"
|
||||
"github.com/matrix-org/dendrite/internal/sqlutil"
|
||||
|
||||
"github.com/matrix-org/dendrite/internal"
|
||||
"github.com/matrix-org/dendrite/setup/jetstream"
|
||||
"github.com/matrix-org/dendrite/setup/process"
|
||||
|
|
@ -47,6 +49,8 @@ import (
|
|||
"github.com/gorilla/mux"
|
||||
"github.com/kardianos/minwinsvc"
|
||||
|
||||
"github.com/sirupsen/logrus"
|
||||
|
||||
appserviceAPI "github.com/matrix-org/dendrite/appservice/api"
|
||||
asinthttp "github.com/matrix-org/dendrite/appservice/inthttp"
|
||||
federationAPI "github.com/matrix-org/dendrite/federationapi/api"
|
||||
|
|
@ -58,7 +62,6 @@ import (
|
|||
"github.com/matrix-org/dendrite/setup/config"
|
||||
userapi "github.com/matrix-org/dendrite/userapi/api"
|
||||
userapiinthttp "github.com/matrix-org/dendrite/userapi/inthttp"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// BaseDendrite is a base for creating new instances of dendrite. It parses
|
||||
|
|
@ -87,6 +90,7 @@ type BaseDendrite struct {
|
|||
Database *sql.DB
|
||||
DatabaseWriter sqlutil.Writer
|
||||
EnableMetrics bool
|
||||
startupLock sync.Mutex
|
||||
}
|
||||
|
||||
const NoListener = ""
|
||||
|
|
@ -393,6 +397,9 @@ func (b *BaseDendrite) SetupAndServeHTTP(
|
|||
internalHTTPAddr, externalHTTPAddr config.HTTPAddress,
|
||||
certFile, keyFile *string,
|
||||
) {
|
||||
// Manually unlocked right before actually serving requests,
|
||||
// as we don't return from this method (defer doesn't work).
|
||||
b.startupLock.Lock()
|
||||
internalAddr, _ := internalHTTPAddr.Address()
|
||||
externalAddr, _ := externalHTTPAddr.Address()
|
||||
|
||||
|
|
@ -471,6 +478,7 @@ func (b *BaseDendrite) SetupAndServeHTTP(
|
|||
externalRouter.PathPrefix(httputil.PublicMediaPathPrefix).Handler(b.PublicMediaAPIMux)
|
||||
externalRouter.PathPrefix(httputil.PublicWellKnownPrefix).Handler(b.PublicWellKnownAPIMux)
|
||||
|
||||
b.startupLock.Unlock()
|
||||
if internalAddr != NoListener && internalAddr != externalAddr {
|
||||
go func() {
|
||||
var internalShutdown atomic.Bool // RegisterOnShutdown can be called more than once
|
||||
|
|
|
|||
|
|
@ -212,7 +212,10 @@ func loadConfig(
|
|||
monolithic bool,
|
||||
) (*Dendrite, error) {
|
||||
var c Dendrite
|
||||
c.Defaults(false)
|
||||
c.Defaults(DefaultOpts{
|
||||
Generate: false,
|
||||
Monolithic: monolithic,
|
||||
})
|
||||
c.IsMonolith = monolithic
|
||||
|
||||
var err error
|
||||
|
|
@ -225,12 +228,7 @@ func loadConfig(
|
|||
}
|
||||
|
||||
privateKeyPath := absPath(basePath, c.Global.PrivateKeyPath)
|
||||
privateKeyData, err := readFile(privateKeyPath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if c.Global.KeyID, c.Global.PrivateKey, err = readKeyPEM(privateKeyPath, privateKeyData, true); err != nil {
|
||||
if c.Global.KeyID, c.Global.PrivateKey, err = LoadMatrixKey(privateKeyPath, readFile); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
|
@ -275,6 +273,14 @@ func loadConfig(
|
|||
return &c, nil
|
||||
}
|
||||
|
||||
func LoadMatrixKey(privateKeyPath string, readFile func(string) ([]byte, error)) (gomatrixserverlib.KeyID, ed25519.PrivateKey, error) {
|
||||
privateKeyData, err := readFile(privateKeyPath)
|
||||
if err != nil {
|
||||
return "", nil, err
|
||||
}
|
||||
return readKeyPEM(privateKeyPath, privateKeyData, true)
|
||||
}
|
||||
|
||||
// Derive generates data that is derived from various values provided in
|
||||
// the config file.
|
||||
func (config *Dendrite) Derive() error {
|
||||
|
|
@ -305,21 +311,25 @@ func (config *Dendrite) Derive() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
type DefaultOpts struct {
|
||||
Generate bool
|
||||
Monolithic bool
|
||||
}
|
||||
|
||||
// SetDefaults sets default config values if they are not explicitly set.
|
||||
func (c *Dendrite) Defaults(generate bool) {
|
||||
func (c *Dendrite) Defaults(opts DefaultOpts) {
|
||||
c.Version = Version
|
||||
|
||||
c.Global.Defaults(generate)
|
||||
c.ClientAPI.Defaults(generate)
|
||||
c.FederationAPI.Defaults(generate)
|
||||
c.KeyServer.Defaults(generate)
|
||||
c.MediaAPI.Defaults(generate)
|
||||
c.RoomServer.Defaults(generate)
|
||||
c.SyncAPI.Defaults(generate)
|
||||
c.UserAPI.Defaults(generate)
|
||||
c.AppServiceAPI.Defaults(generate)
|
||||
c.MSCs.Defaults(generate)
|
||||
|
||||
c.Global.Defaults(opts)
|
||||
c.ClientAPI.Defaults(opts)
|
||||
c.FederationAPI.Defaults(opts)
|
||||
c.KeyServer.Defaults(opts)
|
||||
c.MediaAPI.Defaults(opts)
|
||||
c.RoomServer.Defaults(opts)
|
||||
c.SyncAPI.Defaults(opts)
|
||||
c.UserAPI.Defaults(opts)
|
||||
c.AppServiceAPI.Defaults(opts)
|
||||
c.MSCs.Defaults(opts)
|
||||
c.Wiring()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -28,9 +28,7 @@ type AppServiceAPI struct {
|
|||
Matrix *Global `yaml:"-"`
|
||||
Derived *Derived `yaml:"-"` // TODO: Nuke Derived from orbit
|
||||
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api"`
|
||||
|
||||
Database DatabaseOptions `yaml:"database"`
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api,omitempty"`
|
||||
|
||||
// DisableTLSValidation disables the validation of X.509 TLS certs
|
||||
// on appservice endpoints. This is not recommended in production!
|
||||
|
|
@ -39,19 +37,14 @@ type AppServiceAPI struct {
|
|||
ConfigFiles []string `yaml:"config_files"`
|
||||
}
|
||||
|
||||
func (c *AppServiceAPI) Defaults(generate bool) {
|
||||
func (c *AppServiceAPI) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.InternalAPI.Listen = "http://localhost:7777"
|
||||
c.InternalAPI.Connect = "http://localhost:7777"
|
||||
c.Database.Defaults(5)
|
||||
if generate {
|
||||
c.Database.ConnectionString = "file:appservice.db"
|
||||
}
|
||||
}
|
||||
|
||||
func (c *AppServiceAPI) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "app_service_api.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
if isMonolith { // polylith required configs below
|
||||
return
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,8 +12,8 @@ type ClientAPI struct {
|
|||
Matrix *Global `yaml:"-"`
|
||||
Derived *Derived `yaml:"-"` // TODO: Nuke Derived from orbit
|
||||
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api"`
|
||||
ExternalAPI ExternalAPIOptions `yaml:"external_api"`
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api,omitempty"`
|
||||
ExternalAPI ExternalAPIOptions `yaml:"external_api,omitempty"`
|
||||
|
||||
// If set disables new users from registering (except via shared
|
||||
// secrets)
|
||||
|
|
@ -52,7 +52,7 @@ type ClientAPI struct {
|
|||
RateLimiting RateLimiting `yaml:"rate_limiting"`
|
||||
RtFailedLogin ratelimit.RtFailedLoginConfig `yaml:"rate_limiting_failed_login"`
|
||||
|
||||
MSCs *MSCs `yaml:"mscs"`
|
||||
MSCs *MSCs `yaml:"-"`
|
||||
|
||||
ThreePidDelegate string `yaml:"three_pid_delegate"`
|
||||
|
||||
|
|
@ -68,10 +68,12 @@ type JwtConfig struct {
|
|||
Audiences []string `yaml:"audiences"`
|
||||
}
|
||||
|
||||
func (c *ClientAPI) Defaults(generate bool) {
|
||||
func (c *ClientAPI) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.InternalAPI.Listen = "http://localhost:7771"
|
||||
c.InternalAPI.Connect = "http://localhost:7771"
|
||||
c.ExternalAPI.Listen = "http://[::]:8071"
|
||||
}
|
||||
c.RegistrationSharedSecret = ""
|
||||
c.RecaptchaPublicKey = ""
|
||||
c.RecaptchaPrivateKey = ""
|
||||
|
|
|
|||
|
|
@ -5,12 +5,12 @@ import "github.com/matrix-org/gomatrixserverlib"
|
|||
type FederationAPI struct {
|
||||
Matrix *Global `yaml:"-"`
|
||||
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api"`
|
||||
ExternalAPI ExternalAPIOptions `yaml:"external_api"`
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api,omitempty"`
|
||||
ExternalAPI ExternalAPIOptions `yaml:"external_api,omitempty"`
|
||||
|
||||
// The database stores information used by the federation destination queues to
|
||||
// send transactions to remote servers.
|
||||
Database DatabaseOptions `yaml:"database"`
|
||||
Database DatabaseOptions `yaml:"database,omitempty"`
|
||||
|
||||
// Federation failure threshold. How many consecutive failures that we should
|
||||
// tolerate when sending federation requests to a specific server. The backoff
|
||||
|
|
@ -30,25 +30,44 @@ type FederationAPI struct {
|
|||
PreferDirectFetch bool `yaml:"prefer_direct_fetch"`
|
||||
}
|
||||
|
||||
func (c *FederationAPI) Defaults(generate bool) {
|
||||
func (c *FederationAPI) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.InternalAPI.Listen = "http://localhost:7772"
|
||||
c.InternalAPI.Connect = "http://localhost:7772"
|
||||
c.ExternalAPI.Listen = "http://[::]:8072"
|
||||
c.Database.Defaults(10)
|
||||
}
|
||||
c.FederationMaxRetries = 16
|
||||
c.DisableTLSValidation = false
|
||||
c.Database.Defaults(10)
|
||||
if generate {
|
||||
if opts.Generate {
|
||||
c.KeyPerspectives = KeyPerspectives{
|
||||
{
|
||||
ServerName: "matrix.org",
|
||||
Keys: []KeyPerspectiveTrustKey{
|
||||
{
|
||||
KeyID: "ed25519:auto",
|
||||
PublicKey: "Noi6WqcDj0QmPxCNQqgezwTlBKrfqehY1u2FyWP9uYw",
|
||||
},
|
||||
{
|
||||
KeyID: "ed25519:a_RXGa",
|
||||
PublicKey: "l8Hft5qXKn1vfHrg3p4+W8gELQVo8N13JkluMfmn2sQ",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
if !opts.Monolithic {
|
||||
c.Database.ConnectionString = "file:federationapi.db"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *FederationAPI) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "federation_api.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
if isMonolith { // polylith required configs below
|
||||
return
|
||||
}
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "federation_api.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
checkURL(configErrs, "federation_api.external_api.listen", string(c.ExternalAPI.Listen))
|
||||
checkURL(configErrs, "federation_api.internal_api.listen", string(c.InternalAPI.Listen))
|
||||
checkURL(configErrs, "federation_api.internal_api.connect", string(c.InternalAPI.Connect))
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ type Global struct {
|
|||
// connections will be used instead. This way we don't have to manage connection
|
||||
// counts on a per-component basis, but can instead do it for the entire monolith.
|
||||
// In a polylith deployment, this will be ignored.
|
||||
DatabaseOptions DatabaseOptions `yaml:"database"`
|
||||
DatabaseOptions DatabaseOptions `yaml:"database,omitempty"`
|
||||
|
||||
// The server name to delegate server-server communications to, with optional port
|
||||
WellKnownServerName string `yaml:"well_known_server_name"`
|
||||
|
|
@ -83,22 +83,28 @@ type Global struct {
|
|||
Cache Cache `yaml:"cache"`
|
||||
}
|
||||
|
||||
func (c *Global) Defaults(generate bool) {
|
||||
if generate {
|
||||
func (c *Global) Defaults(opts DefaultOpts) {
|
||||
if opts.Generate {
|
||||
c.ServerName = "localhost"
|
||||
c.PrivateKeyPath = "matrix_key.pem"
|
||||
_, c.PrivateKey, _ = ed25519.GenerateKey(rand.New(rand.NewSource(0)))
|
||||
c.KeyID = "ed25519:auto"
|
||||
c.TrustedIDServers = []string{
|
||||
"matrix.org",
|
||||
"vector.im",
|
||||
}
|
||||
}
|
||||
c.KeyValidityPeriod = time.Hour * 24 * 7
|
||||
|
||||
c.JetStream.Defaults(generate)
|
||||
c.Metrics.Defaults(generate)
|
||||
if opts.Monolithic {
|
||||
c.DatabaseOptions.Defaults(90)
|
||||
}
|
||||
c.JetStream.Defaults(opts)
|
||||
c.Metrics.Defaults(opts)
|
||||
c.DNSCache.Defaults()
|
||||
c.Sentry.Defaults()
|
||||
c.ServerNotices.Defaults(generate)
|
||||
c.ServerNotices.Defaults(opts)
|
||||
c.ReportStats.Defaults()
|
||||
c.Cache.Defaults(generate)
|
||||
c.Cache.Defaults()
|
||||
}
|
||||
|
||||
func (c *Global) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
|
|
@ -142,9 +148,9 @@ type Metrics struct {
|
|||
} `yaml:"basic_auth"`
|
||||
}
|
||||
|
||||
func (c *Metrics) Defaults(generate bool) {
|
||||
func (c *Metrics) Defaults(opts DefaultOpts) {
|
||||
c.Enabled = false
|
||||
if generate {
|
||||
if opts.Generate {
|
||||
c.BasicAuth.Username = "metrics"
|
||||
c.BasicAuth.Password = "metrics"
|
||||
}
|
||||
|
|
@ -166,8 +172,8 @@ type ServerNotices struct {
|
|||
RoomName string `yaml:"room_name"`
|
||||
}
|
||||
|
||||
func (c *ServerNotices) Defaults(generate bool) {
|
||||
if generate {
|
||||
func (c *ServerNotices) Defaults(opts DefaultOpts) {
|
||||
if opts.Generate {
|
||||
c.Enabled = true
|
||||
c.LocalPart = "_server"
|
||||
c.DisplayName = "Server Alert"
|
||||
|
|
@ -183,7 +189,7 @@ type Cache struct {
|
|||
MaxAge time.Duration `yaml:"max_age"`
|
||||
}
|
||||
|
||||
func (c *Cache) Defaults(generate bool) {
|
||||
func (c *Cache) Defaults() {
|
||||
c.EstimatedMaxSize = 1024 * 1024 * 1024 // 1GB
|
||||
c.MaxAge = time.Hour
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,10 +31,10 @@ func (c *JetStream) Durable(name string) string {
|
|||
return c.Prefixed(name)
|
||||
}
|
||||
|
||||
func (c *JetStream) Defaults(generate bool) {
|
||||
func (c *JetStream) Defaults(opts DefaultOpts) {
|
||||
c.Addresses = []string{}
|
||||
c.TopicPrefix = "Dendrite"
|
||||
if generate {
|
||||
if opts.Generate {
|
||||
c.StoragePath = Path("./")
|
||||
c.NoLog = true
|
||||
c.DisableTLSValidation = true
|
||||
|
|
|
|||
|
|
@ -3,27 +3,31 @@ package config
|
|||
type KeyServer struct {
|
||||
Matrix *Global `yaml:"-"`
|
||||
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api"`
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api,omitempty"`
|
||||
|
||||
Database DatabaseOptions `yaml:"database"`
|
||||
Database DatabaseOptions `yaml:"database,omitempty"`
|
||||
}
|
||||
|
||||
func (c *KeyServer) Defaults(generate bool) {
|
||||
func (c *KeyServer) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.InternalAPI.Listen = "http://localhost:7779"
|
||||
c.InternalAPI.Connect = "http://localhost:7779"
|
||||
c.Database.Defaults(10)
|
||||
if generate {
|
||||
}
|
||||
if opts.Generate {
|
||||
if !opts.Monolithic {
|
||||
c.Database.ConnectionString = "file:keyserver.db"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *KeyServer) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "key_server.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
if isMonolith { // polylith required configs below
|
||||
return
|
||||
}
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "key_server.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
checkURL(configErrs, "key_server.internal_api.listen", string(c.InternalAPI.Listen))
|
||||
checkURL(configErrs, "key_server.internal_api.connect", string(c.InternalAPI.Connect))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,12 +7,12 @@ import (
|
|||
type MediaAPI struct {
|
||||
Matrix *Global `yaml:"-"`
|
||||
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api"`
|
||||
ExternalAPI ExternalAPIOptions `yaml:"external_api"`
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api,omitempty"`
|
||||
ExternalAPI ExternalAPIOptions `yaml:"external_api,omitempty"`
|
||||
|
||||
// The MediaAPI database stores information about files uploaded and downloaded
|
||||
// by local users. It is only accessed by the MediaAPI.
|
||||
Database DatabaseOptions `yaml:"database"`
|
||||
Database DatabaseOptions `yaml:"database,omitempty"`
|
||||
|
||||
// The base path to where the media files will be stored. May be relative or absolute.
|
||||
BasePath Path `yaml:"base_path"`
|
||||
|
|
@ -38,23 +38,41 @@ type MediaAPI struct {
|
|||
// DefaultMaxFileSizeBytes defines the default file size allowed in transfers
|
||||
var DefaultMaxFileSizeBytes = FileSizeBytes(10485760)
|
||||
|
||||
func (c *MediaAPI) Defaults(generate bool) {
|
||||
func (c *MediaAPI) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.InternalAPI.Listen = "http://localhost:7774"
|
||||
c.InternalAPI.Connect = "http://localhost:7774"
|
||||
c.ExternalAPI.Listen = "http://[::]:8074"
|
||||
c.Database.Defaults(5)
|
||||
}
|
||||
c.MaxFileSizeBytes = DefaultMaxFileSizeBytes
|
||||
c.MaxThumbnailGenerators = 10
|
||||
c.Database.Defaults(5)
|
||||
if generate {
|
||||
if opts.Generate {
|
||||
c.ThumbnailSizes = []ThumbnailSize{
|
||||
{
|
||||
Width: 32,
|
||||
Height: 32,
|
||||
ResizeMethod: "crop",
|
||||
},
|
||||
{
|
||||
Width: 96,
|
||||
Height: 96,
|
||||
ResizeMethod: "crop",
|
||||
},
|
||||
{
|
||||
Width: 640,
|
||||
Height: 480,
|
||||
ResizeMethod: "scale",
|
||||
},
|
||||
}
|
||||
if !opts.Monolithic {
|
||||
c.Database.ConnectionString = "file:mediaapi.db"
|
||||
}
|
||||
c.BasePath = "./media_store"
|
||||
}
|
||||
}
|
||||
|
||||
func (c *MediaAPI) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "media_api.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
checkNotEmpty(configErrs, "media_api.base_path", string(c.BasePath))
|
||||
checkPositive(configErrs, "media_api.max_file_size_bytes", int64(c.MaxFileSizeBytes))
|
||||
checkPositive(configErrs, "media_api.max_thumbnail_generators", int64(c.MaxThumbnailGenerators))
|
||||
|
|
@ -66,6 +84,9 @@ func (c *MediaAPI) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
|||
if isMonolith { // polylith required configs below
|
||||
return
|
||||
}
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "media_api.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
checkURL(configErrs, "media_api.internal_api.listen", string(c.InternalAPI.Listen))
|
||||
checkURL(configErrs, "media_api.internal_api.connect", string(c.InternalAPI.Connect))
|
||||
checkURL(configErrs, "media_api.external_api.listen", string(c.ExternalAPI.Listen))
|
||||
|
|
|
|||
|
|
@ -10,15 +10,19 @@ type MSCs struct {
|
|||
// 'msc2946': Spaces Summary - https://github.com/matrix-org/matrix-doc/pull/2946
|
||||
MSCs []string `yaml:"mscs"`
|
||||
|
||||
Database DatabaseOptions `yaml:"database"`
|
||||
Database DatabaseOptions `yaml:"database,omitempty"`
|
||||
}
|
||||
|
||||
func (c *MSCs) Defaults(generate bool) {
|
||||
func (c *MSCs) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.Database.Defaults(5)
|
||||
if generate {
|
||||
}
|
||||
if opts.Generate {
|
||||
if !opts.Monolithic {
|
||||
c.Database.ConnectionString = "file:mscs.db"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Enabled returns true if the given msc is enabled. Should in the form 'msc12345'.
|
||||
func (c *MSCs) Enabled(msc string) bool {
|
||||
|
|
@ -31,6 +35,9 @@ func (c *MSCs) Enabled(msc string) bool {
|
|||
}
|
||||
|
||||
func (c *MSCs) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
if isMonolith { // polylith required configs below
|
||||
return
|
||||
}
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "mscs.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,27 +3,31 @@ package config
|
|||
type RoomServer struct {
|
||||
Matrix *Global `yaml:"-"`
|
||||
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api"`
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api,omitempty"`
|
||||
|
||||
Database DatabaseOptions `yaml:"database"`
|
||||
Database DatabaseOptions `yaml:"database,omitempty"`
|
||||
}
|
||||
|
||||
func (c *RoomServer) Defaults(generate bool) {
|
||||
func (c *RoomServer) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.InternalAPI.Listen = "http://localhost:7770"
|
||||
c.InternalAPI.Connect = "http://localhost:7770"
|
||||
c.Database.Defaults(10)
|
||||
if generate {
|
||||
c.Database.Defaults(20)
|
||||
}
|
||||
if opts.Generate {
|
||||
if !opts.Monolithic {
|
||||
c.Database.ConnectionString = "file:roomserver.db"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *RoomServer) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "room_server.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
if isMonolith { // polylith required configs below
|
||||
return
|
||||
}
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "room_server.database.connection_string", string(c.Database.ConnectionString))
|
||||
}
|
||||
checkURL(configErrs, "room_server.internal_api.listen", string(c.InternalAPI.Listen))
|
||||
checkURL(configErrs, "room_server.internal_ap.connect", string(c.InternalAPI.Connect))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,31 +3,35 @@ package config
|
|||
type SyncAPI struct {
|
||||
Matrix *Global `yaml:"-"`
|
||||
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api"`
|
||||
ExternalAPI ExternalAPIOptions `yaml:"external_api"`
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api,omitempty"`
|
||||
ExternalAPI ExternalAPIOptions `yaml:"external_api,omitempty"`
|
||||
|
||||
Database DatabaseOptions `yaml:"database"`
|
||||
Database DatabaseOptions `yaml:"database,omitempty"`
|
||||
|
||||
RealIPHeader string `yaml:"real_ip_header"`
|
||||
}
|
||||
|
||||
func (c *SyncAPI) Defaults(generate bool) {
|
||||
func (c *SyncAPI) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.InternalAPI.Listen = "http://localhost:7773"
|
||||
c.InternalAPI.Connect = "http://localhost:7773"
|
||||
c.ExternalAPI.Listen = "http://localhost:8073"
|
||||
c.Database.Defaults(10)
|
||||
if generate {
|
||||
c.Database.Defaults(20)
|
||||
}
|
||||
if opts.Generate {
|
||||
if !opts.Monolithic {
|
||||
c.Database.ConnectionString = "file:syncapi.db"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *SyncAPI) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "sync_api.database", string(c.Database.ConnectionString))
|
||||
}
|
||||
if isMonolith { // polylith required configs below
|
||||
return
|
||||
}
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "sync_api.database", string(c.Database.ConnectionString))
|
||||
}
|
||||
checkURL(configErrs, "sync_api.internal_api.listen", string(c.InternalAPI.Listen))
|
||||
checkURL(configErrs, "sync_api.internal_api.connect", string(c.InternalAPI.Connect))
|
||||
checkURL(configErrs, "sync_api.external_api.listen", string(c.ExternalAPI.Listen))
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ import "golang.org/x/crypto/bcrypt"
|
|||
type UserAPI struct {
|
||||
Matrix *Global `yaml:"-"`
|
||||
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api"`
|
||||
InternalAPI InternalAPIOptions `yaml:"internal_api,omitempty"`
|
||||
|
||||
// The cost when hashing passwords.
|
||||
BCryptCost int `yaml:"bcrypt_cost"`
|
||||
|
|
@ -18,30 +18,34 @@ type UserAPI struct {
|
|||
|
||||
// The Account database stores the login details and account information
|
||||
// for local users. It is accessed by the UserAPI.
|
||||
AccountDatabase DatabaseOptions `yaml:"account_database"`
|
||||
AccountDatabase DatabaseOptions `yaml:"account_database,omitempty"`
|
||||
}
|
||||
|
||||
const DefaultOpenIDTokenLifetimeMS = 3600000 // 60 minutes
|
||||
|
||||
func (c *UserAPI) Defaults(generate bool) {
|
||||
func (c *UserAPI) Defaults(opts DefaultOpts) {
|
||||
if !opts.Monolithic {
|
||||
c.InternalAPI.Listen = "http://localhost:7781"
|
||||
c.InternalAPI.Connect = "http://localhost:7781"
|
||||
c.AccountDatabase.Defaults(10)
|
||||
}
|
||||
c.BCryptCost = bcrypt.DefaultCost
|
||||
c.OpenIDTokenLifetimeMS = DefaultOpenIDTokenLifetimeMS
|
||||
c.AccountDatabase.Defaults(10)
|
||||
if generate {
|
||||
if opts.Generate {
|
||||
if !opts.Monolithic {
|
||||
c.AccountDatabase.ConnectionString = "file:userapi_accounts.db"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *UserAPI) Verify(configErrs *ConfigErrors, isMonolith bool) {
|
||||
checkPositive(configErrs, "user_api.openid_token_lifetime_ms", c.OpenIDTokenLifetimeMS)
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "user_api.account_database.connection_string", string(c.AccountDatabase.ConnectionString))
|
||||
}
|
||||
if isMonolith { // polylith required configs below
|
||||
return
|
||||
}
|
||||
if c.Matrix.DatabaseOptions.ConnectionString == "" {
|
||||
checkNotEmpty(configErrs, "user_api.account_database.connection_string", string(c.AccountDatabase.ConnectionString))
|
||||
}
|
||||
checkURL(configErrs, "user_api.internal_api.listen", string(c.InternalAPI.Listen))
|
||||
checkURL(configErrs, "user_api.internal_api.connect", string(c.InternalAPI.Connect))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,9 +9,16 @@ import (
|
|||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// JetStreamConsumer starts a durable consumer on the given subject with the
|
||||
// given durable name. The function will be called when one or more messages
|
||||
// is available, up to the maximum batch size specified. If the batch is set to
|
||||
// 1 then messages will be delivered one at a time. If the function is called,
|
||||
// the messages array is guaranteed to be at least 1 in size. Any provided NATS
|
||||
// options will be passed through to the pull subscriber creation. The consumer
|
||||
// will continue to run until the context expires, at which point it will stop.
|
||||
func JetStreamConsumer(
|
||||
ctx context.Context, js nats.JetStreamContext, subj, durable string,
|
||||
f func(ctx context.Context, msg *nats.Msg) bool,
|
||||
ctx context.Context, js nats.JetStreamContext, subj, durable string, batch int,
|
||||
f func(ctx context.Context, msgs []*nats.Msg) bool,
|
||||
opts ...nats.SubOpt,
|
||||
) error {
|
||||
defer func() {
|
||||
|
|
@ -50,7 +57,7 @@ func JetStreamConsumer(
|
|||
// enforce its own deadline (roughly 5 seconds by default). Therefore
|
||||
// it is our responsibility to check whether our context expired or
|
||||
// not when a context error is returned. Footguns. Footguns everywhere.
|
||||
msgs, err := sub.Fetch(1, nats.Context(ctx))
|
||||
msgs, err := sub.Fetch(batch, nats.Context(ctx))
|
||||
if err != nil {
|
||||
if err == context.Canceled || err == context.DeadlineExceeded {
|
||||
// Work out whether it was the JetStream context that expired
|
||||
|
|
@ -74,24 +81,29 @@ func JetStreamConsumer(
|
|||
if len(msgs) < 1 {
|
||||
continue
|
||||
}
|
||||
msg := msgs[0]
|
||||
for _, msg := range msgs {
|
||||
if err = msg.InProgress(nats.Context(ctx)); err != nil {
|
||||
logrus.WithContext(ctx).WithField("subject", subj).Warn(fmt.Errorf("msg.InProgress: %w", err))
|
||||
sentry.CaptureException(err)
|
||||
continue
|
||||
}
|
||||
if f(ctx, msg) {
|
||||
}
|
||||
if f(ctx, msgs) {
|
||||
for _, msg := range msgs {
|
||||
if err = msg.AckSync(nats.Context(ctx)); err != nil {
|
||||
logrus.WithContext(ctx).WithField("subject", subj).Warn(fmt.Errorf("msg.AckSync: %w", err))
|
||||
sentry.CaptureException(err)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for _, msg := range msgs {
|
||||
if err = msg.Nak(nats.Context(ctx)); err != nil {
|
||||
logrus.WithContext(ctx).WithField("subject", subj).Warn(fmt.Errorf("msg.Nak: %w", err))
|
||||
sentry.CaptureException(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
return nil
|
||||
}
|
||||
|
|
|
|||
|
|
@ -183,6 +183,7 @@ func setupNATS(process *process.ProcessContext, cfg *config.JetStream, nc *natsc
|
|||
OutputReceiptEvent: {"SyncAPIEDUServerReceiptConsumer", "FederationAPIEDUServerConsumer"},
|
||||
OutputSendToDeviceEvent: {"SyncAPIEDUServerSendToDeviceConsumer", "FederationAPIEDUServerConsumer"},
|
||||
OutputTypingEvent: {"SyncAPIEDUServerTypingConsumer", "FederationAPIEDUServerConsumer"},
|
||||
OutputRoomEvent: {"AppserviceRoomserverConsumer"},
|
||||
} {
|
||||
streamName := cfg.Matrix.JetStream.Prefixed(stream)
|
||||
for _, consumer := range consumers {
|
||||
|
|
|
|||
|
|
@ -547,7 +547,10 @@ func (r *testRoomserverAPI) QueryMembershipForUser(ctx context.Context, req *roo
|
|||
func injectEvents(t *testing.T, userAPI userapi.UserInternalAPI, rsAPI roomserver.RoomserverInternalAPI, events []*gomatrixserverlib.HeaderedEvent) *mux.Router {
|
||||
t.Helper()
|
||||
cfg := &config.Dendrite{}
|
||||
cfg.Defaults(true)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.ServerName = "localhost"
|
||||
cfg.MSCs.Database.ConnectionString = "file:msc2836_test.db"
|
||||
cfg.MSCs.MSCs = []string{"msc2836"}
|
||||
|
|
|
|||
|
|
@ -75,15 +75,16 @@ func NewOutputClientDataConsumer(
|
|||
// Start consuming from room servers
|
||||
func (s *OutputClientDataConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
// onMessage is called when the sync server receives a new event from the client API server output log.
|
||||
// It is not safe for this function to be called from multiple goroutines, or else the
|
||||
// sync stream position may race and be incorrectly calculated.
|
||||
func (s *OutputClientDataConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputClientDataConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
// Parse out the event JSON
|
||||
userID := msg.Header.Get(jetstream.UserID)
|
||||
var output eventutil.AccountData
|
||||
|
|
|
|||
|
|
@ -75,12 +75,13 @@ func NewOutputKeyChangeEventConsumer(
|
|||
// Start consuming from the key server
|
||||
func (s *OutputKeyChangeEventConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
func (s *OutputKeyChangeEventConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputKeyChangeEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
var m api.DeviceMessage
|
||||
if err := json.Unmarshal(msg.Data, &m); err != nil {
|
||||
logrus.WithError(err).Errorf("failed to read device message from key change topic")
|
||||
|
|
|
|||
|
|
@ -128,12 +128,13 @@ func (s *PresenceConsumer) Start() error {
|
|||
return nil
|
||||
}
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.presenceTopic, s.durable, s.onMessage,
|
||||
s.ctx, s.jetstream, s.presenceTopic, s.durable, 1, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(), nats.HeadersOnly(),
|
||||
)
|
||||
}
|
||||
|
||||
func (s *PresenceConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *PresenceConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
userID := msg.Header.Get(jetstream.UserID)
|
||||
presence := msg.Header.Get("presence")
|
||||
timestamp := msg.Header.Get("last_active_ts")
|
||||
|
|
|
|||
|
|
@ -74,12 +74,13 @@ func NewOutputReceiptEventConsumer(
|
|||
// Start consuming receipts events.
|
||||
func (s *OutputReceiptEventConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
func (s *OutputReceiptEventConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputReceiptEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
output := types.OutputReceiptEvent{
|
||||
UserID: msg.Header.Get(jetstream.UserID),
|
||||
RoomID: msg.Header.Get(jetstream.RoomID),
|
||||
|
|
|
|||
|
|
@ -79,15 +79,16 @@ func NewOutputRoomEventConsumer(
|
|||
// Start consuming from room servers
|
||||
func (s *OutputRoomEventConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
// onMessage is called when the sync server receives a new event from the room server output log.
|
||||
// It is not safe for this function to be called from multiple goroutines, or else the
|
||||
// sync stream position may race and be incorrectly calculated.
|
||||
func (s *OutputRoomEventConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputRoomEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
// Parse out the event JSON
|
||||
var err error
|
||||
var output api.OutputEvent
|
||||
|
|
|
|||
|
|
@ -68,12 +68,13 @@ func NewOutputSendToDeviceEventConsumer(
|
|||
// Start consuming send-to-device events.
|
||||
func (s *OutputSendToDeviceEventConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
func (s *OutputSendToDeviceEventConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputSendToDeviceEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
userID := msg.Header.Get(jetstream.UserID)
|
||||
_, domain, err := gomatrixserverlib.SplitID('@', userID)
|
||||
if err != nil {
|
||||
|
|
|
|||
|
|
@ -64,12 +64,13 @@ func NewOutputTypingEventConsumer(
|
|||
// Start consuming typing events.
|
||||
func (s *OutputTypingEventConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
func (s *OutputTypingEventConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputTypingEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
roomID := msg.Header.Get(jetstream.RoomID)
|
||||
userID := msg.Header.Get(jetstream.UserID)
|
||||
typing, err := strconv.ParseBool(msg.Header.Get("typing"))
|
||||
|
|
|
|||
|
|
@ -67,8 +67,8 @@ func NewOutputNotificationDataConsumer(
|
|||
// Start starts consumption.
|
||||
func (s *OutputNotificationDataConsumer) Start() error {
|
||||
return jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
)
|
||||
}
|
||||
|
||||
|
|
@ -76,7 +76,8 @@ func (s *OutputNotificationDataConsumer) Start() error {
|
|||
// the push server. It is not safe for this function to be called from
|
||||
// multiple goroutines, or else the sync stream position may race and
|
||||
// be incorrectly calculated.
|
||||
func (s *OutputNotificationDataConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputNotificationDataConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
userID := string(msg.Header.Get(jetstream.UserID))
|
||||
|
||||
// Parse out the event JSON
|
||||
|
|
|
|||
|
|
@ -39,6 +39,8 @@ CREATE TABLE IF NOT EXISTS syncapi_send_to_device (
|
|||
-- The event content JSON.
|
||||
content TEXT NOT NULL
|
||||
);
|
||||
|
||||
CREATE INDEX IF NOT EXISTS syncapi_send_to_device_user_id_device_id_idx ON syncapi_send_to_device(user_id, device_id);
|
||||
`
|
||||
|
||||
const insertSendToDeviceMessageSQL = `
|
||||
|
|
|
|||
|
|
@ -303,7 +303,7 @@ func (p *PDUStreamProvider) addRoomDeltaToResponse(
|
|||
|
||||
if stateFilter.LazyLoadMembers {
|
||||
delta.StateEvents, err = p.lazyLoadMembers(
|
||||
ctx, delta.RoomID, true, limited, stateFilter.IncludeRedundantMembers,
|
||||
ctx, delta.RoomID, true, limited, stateFilter,
|
||||
device, recentEvents, delta.StateEvents,
|
||||
)
|
||||
if err != nil && err != sql.ErrNoRows {
|
||||
|
|
@ -532,7 +532,7 @@ func (p *PDUStreamProvider) getJoinResponseForCompleteSync(
|
|||
return nil, err
|
||||
}
|
||||
stateEvents, err = p.lazyLoadMembers(ctx, roomID,
|
||||
false, limited, stateFilter.IncludeRedundantMembers,
|
||||
false, limited, stateFilter,
|
||||
device, recentEvents, stateEvents,
|
||||
)
|
||||
if err != nil && err != sql.ErrNoRows {
|
||||
|
|
@ -551,7 +551,7 @@ func (p *PDUStreamProvider) getJoinResponseForCompleteSync(
|
|||
|
||||
func (p *PDUStreamProvider) lazyLoadMembers(
|
||||
ctx context.Context, roomID string,
|
||||
incremental, limited, includeRedundant bool,
|
||||
incremental, limited bool, stateFilter *gomatrixserverlib.StateFilter,
|
||||
device *userapi.Device,
|
||||
timelineEvents, stateEvents []*gomatrixserverlib.HeaderedEvent,
|
||||
) ([]*gomatrixserverlib.HeaderedEvent, error) {
|
||||
|
|
@ -581,7 +581,7 @@ func (p *PDUStreamProvider) lazyLoadMembers(
|
|||
stateKey := *event.StateKey()
|
||||
if _, ok := timelineUsers[stateKey]; ok || isGappedIncremental {
|
||||
newStateEvents = append(newStateEvents, event)
|
||||
if !includeRedundant {
|
||||
if !stateFilter.IncludeRedundantMembers {
|
||||
p.lazyLoadCache.StoreLazyLoadedUser(device, roomID, stateKey, event.EventID())
|
||||
}
|
||||
delete(timelineUsers, stateKey)
|
||||
|
|
@ -596,6 +596,7 @@ func (p *PDUStreamProvider) lazyLoadMembers(
|
|||
}
|
||||
// Query missing membership events
|
||||
filter := gomatrixserverlib.DefaultStateFilter()
|
||||
filter.Limit = stateFilter.Limit
|
||||
filter.Senders = &wantUsers
|
||||
filter.Types = &[]string{gomatrixserverlib.MRoomMember}
|
||||
memberships, err := p.DB.GetStateEventsForRoom(ctx, roomID, &filter)
|
||||
|
|
|
|||
|
|
@ -15,6 +15,7 @@
|
|||
package test
|
||||
|
||||
import (
|
||||
"crypto/ed25519"
|
||||
"crypto/rand"
|
||||
"crypto/rsa"
|
||||
"crypto/x509"
|
||||
|
|
@ -44,6 +45,10 @@ func NewMatrixKey(matrixKeyPath string) (err error) {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return SaveMatrixKey(matrixKeyPath, data[3:])
|
||||
}
|
||||
|
||||
func SaveMatrixKey(matrixKeyPath string, data ed25519.PrivateKey) error {
|
||||
keyOut, err := os.OpenFile(matrixKeyPath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600)
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
@ -62,7 +67,7 @@ func NewMatrixKey(matrixKeyPath string) (err error) {
|
|||
Headers: map[string]string{
|
||||
"Key-ID": fmt.Sprintf("ed25519:%s", keyID[:6]),
|
||||
},
|
||||
Bytes: data[3:],
|
||||
Bytes: data,
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
|
|
|||
|
|
@ -30,12 +30,21 @@ import (
|
|||
|
||||
func CreateBaseDendrite(t *testing.T, dbType test.DBType) (*base.BaseDendrite, func()) {
|
||||
var cfg config.Dendrite
|
||||
cfg.Defaults(false)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: false,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.JetStream.InMemory = true
|
||||
switch dbType {
|
||||
case test.DBTypePostgres:
|
||||
cfg.Global.Defaults(true) // autogen a signing key
|
||||
cfg.MediaAPI.Defaults(true) // autogen a media path
|
||||
cfg.Global.Defaults(config.DefaultOpts{ // autogen a signing key
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.MediaAPI.Defaults(config.DefaultOpts{ // autogen a media path
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
cfg.Global.ServerName = "test"
|
||||
// use a distinct prefix else concurrent postgres/sqlite runs will clash since NATS will use
|
||||
// the file system event with InMemory=true :(
|
||||
|
|
@ -49,7 +58,10 @@ func CreateBaseDendrite(t *testing.T, dbType test.DBType) (*base.BaseDendrite, f
|
|||
}
|
||||
return base.NewBaseDendrite(&cfg, "Test", base.DisableMetrics), close
|
||||
case test.DBTypeSQLite:
|
||||
cfg.Defaults(true) // sets a sqlite db per component
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: false, // because we need a database per component
|
||||
})
|
||||
cfg.Global.ServerName = "test"
|
||||
// use a distinct prefix else concurrent postgres/sqlite runs will clash since NATS will use
|
||||
// the file system event with InMemory=true :(
|
||||
|
|
@ -57,7 +69,6 @@ func CreateBaseDendrite(t *testing.T, dbType test.DBType) (*base.BaseDendrite, f
|
|||
return base.NewBaseDendrite(&cfg, "Test", base.DisableMetrics), func() {
|
||||
// cleanup db files. This risks getting out of sync as we add more database strings :(
|
||||
dbFiles := []config.DataSource{
|
||||
cfg.AppServiceAPI.Database.ConnectionString,
|
||||
cfg.FederationAPI.Database.ConnectionString,
|
||||
cfg.KeyServer.Database.ConnectionString,
|
||||
cfg.MSCs.Database.ConnectionString,
|
||||
|
|
@ -83,7 +94,10 @@ func CreateBaseDendrite(t *testing.T, dbType test.DBType) (*base.BaseDendrite, f
|
|||
func Base(cfg *config.Dendrite) (*base.BaseDendrite, nats.JetStreamContext, *nats.Conn) {
|
||||
if cfg == nil {
|
||||
cfg = &config.Dendrite{}
|
||||
cfg.Defaults(true)
|
||||
cfg.Defaults(config.DefaultOpts{
|
||||
Generate: true,
|
||||
Monolithic: true,
|
||||
})
|
||||
}
|
||||
cfg.Global.JetStream.InMemory = true
|
||||
base := base.NewBaseDendrite(cfg, "Tests")
|
||||
|
|
|
|||
|
|
@ -56,15 +56,16 @@ func NewOutputReadUpdateConsumer(
|
|||
|
||||
func (s *OutputReadUpdateConsumer) Start() error {
|
||||
if err := jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *OutputReadUpdateConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputReadUpdateConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
var read types.ReadUpdate
|
||||
if err := json.Unmarshal(msg.Data, &read); err != nil {
|
||||
log.WithError(err).Error("userapi clientapi consumer: message parse failure")
|
||||
|
|
|
|||
|
|
@ -7,6 +7,10 @@ import (
|
|||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
"github.com/nats-io/nats.go"
|
||||
log "github.com/sirupsen/logrus"
|
||||
|
||||
"github.com/matrix-org/dendrite/internal/eventutil"
|
||||
"github.com/matrix-org/dendrite/internal/pushgateway"
|
||||
"github.com/matrix-org/dendrite/internal/pushrules"
|
||||
|
|
@ -20,9 +24,6 @@ import (
|
|||
"github.com/matrix-org/dendrite/userapi/storage"
|
||||
"github.com/matrix-org/dendrite/userapi/storage/tables"
|
||||
"github.com/matrix-org/dendrite/userapi/util"
|
||||
"github.com/matrix-org/gomatrixserverlib"
|
||||
"github.com/nats-io/nats.go"
|
||||
log "github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
type OutputStreamEventConsumer struct {
|
||||
|
|
@ -64,15 +65,16 @@ func NewOutputStreamEventConsumer(
|
|||
|
||||
func (s *OutputStreamEventConsumer) Start() error {
|
||||
if err := jetstream.JetStreamConsumer(
|
||||
s.ctx, s.jetstream, s.topic, s.durable, s.onMessage,
|
||||
nats.DeliverAll(), nats.ManualAck(),
|
||||
s.ctx, s.jetstream, s.topic, s.durable, 1,
|
||||
s.onMessage, nats.DeliverAll(), nats.ManualAck(),
|
||||
); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *OutputStreamEventConsumer) onMessage(ctx context.Context, msg *nats.Msg) bool {
|
||||
func (s *OutputStreamEventConsumer) onMessage(ctx context.Context, msgs []*nats.Msg) bool {
|
||||
msg := msgs[0] // Guaranteed to exist if onMessage is called
|
||||
var output types.StreamedEvent
|
||||
output.Event = &gomatrixserverlib.HeaderedEvent{}
|
||||
if err := json.Unmarshal(msg.Data, &output); err != nil {
|
||||
|
|
|
|||
|
|
@ -28,7 +28,6 @@ import (
|
|||
"github.com/sirupsen/logrus"
|
||||
"golang.org/x/crypto/bcrypt"
|
||||
|
||||
"github.com/matrix-org/dendrite/appservice/types"
|
||||
"github.com/matrix-org/dendrite/clientapi/userutil"
|
||||
"github.com/matrix-org/dendrite/internal/eventutil"
|
||||
"github.com/matrix-org/dendrite/internal/pushrules"
|
||||
|
|
@ -454,7 +453,7 @@ func (a *UserInternalAPI) queryAppServiceToken(ctx context.Context, token, appSe
|
|||
// Create a dummy device for AS user
|
||||
dev := api.Device{
|
||||
// Use AS dummy device ID
|
||||
ID: types.AppServiceDeviceID,
|
||||
ID: "AS_Device",
|
||||
// AS dummy device has AS's token.
|
||||
AccessToken: token,
|
||||
AppserviceID: appService.ID,
|
||||
|
|
|
|||
Loading…
Reference in a new issue