mirror of
https://github.com/matrix-org/dendrite.git
synced 2025-12-29 01:33:10 -06:00
Use key change topic
This commit is contained in:
parent
f2191d6041
commit
64fc48f756
|
|
@ -52,7 +52,7 @@ func NewInternalAPI(
|
||||||
OutputTypingEventTopic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputTypingEvent),
|
OutputTypingEventTopic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputTypingEvent),
|
||||||
OutputSendToDeviceEventTopic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputSendToDeviceEvent),
|
OutputSendToDeviceEventTopic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputSendToDeviceEvent),
|
||||||
OutputReceiptEventTopic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputReceiptEvent),
|
OutputReceiptEventTopic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputReceiptEvent),
|
||||||
OutputCrossSigningKeyUpdateTopic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputCrossSigningKeyUpdate),
|
OutputKeyChangeEventTopic: cfg.Matrix.Kafka.TopicFor(config.TopicOutputKeyChangeEvent),
|
||||||
ServerName: cfg.Matrix.ServerName,
|
ServerName: cfg.Matrix.ServerName,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -24,6 +24,7 @@ import (
|
||||||
"github.com/Shopify/sarama"
|
"github.com/Shopify/sarama"
|
||||||
"github.com/matrix-org/dendrite/eduserver/api"
|
"github.com/matrix-org/dendrite/eduserver/api"
|
||||||
"github.com/matrix-org/dendrite/eduserver/cache"
|
"github.com/matrix-org/dendrite/eduserver/cache"
|
||||||
|
keyapi "github.com/matrix-org/dendrite/keyserver/api"
|
||||||
userapi "github.com/matrix-org/dendrite/userapi/api"
|
userapi "github.com/matrix-org/dendrite/userapi/api"
|
||||||
"github.com/matrix-org/gomatrixserverlib"
|
"github.com/matrix-org/gomatrixserverlib"
|
||||||
"github.com/sirupsen/logrus"
|
"github.com/sirupsen/logrus"
|
||||||
|
|
@ -39,8 +40,8 @@ type EDUServerInputAPI struct {
|
||||||
OutputSendToDeviceEventTopic string
|
OutputSendToDeviceEventTopic string
|
||||||
// The kafka topic to output new receipt events to
|
// The kafka topic to output new receipt events to
|
||||||
OutputReceiptEventTopic string
|
OutputReceiptEventTopic string
|
||||||
// The kafka topic to output new signing key changes to
|
// The kafka topic to output new key change events to
|
||||||
OutputCrossSigningKeyUpdateTopic string
|
OutputKeyChangeEventTopic string
|
||||||
// kafka producer
|
// kafka producer
|
||||||
Producer sarama.SyncProducer
|
Producer sarama.SyncProducer
|
||||||
// Internal user query API
|
// Internal user query API
|
||||||
|
|
@ -85,8 +86,11 @@ func (t *EDUServerInputAPI) InputCrossSigningKeyUpdate(
|
||||||
request *api.InputCrossSigningKeyUpdateRequest,
|
request *api.InputCrossSigningKeyUpdateRequest,
|
||||||
response *api.InputCrossSigningKeyUpdateResponse,
|
response *api.InputCrossSigningKeyUpdateResponse,
|
||||||
) error {
|
) error {
|
||||||
eventJSON, err := json.Marshal(&api.OutputCrossSigningKeyUpdate{
|
eventJSON, err := json.Marshal(&keyapi.DeviceMessage{
|
||||||
|
Type: keyapi.TypeCrossSigningUpdate,
|
||||||
|
OutputCrossSigningKeyUpdate: &api.OutputCrossSigningKeyUpdate{
|
||||||
CrossSigningKeyUpdate: request.CrossSigningKeyUpdate,
|
CrossSigningKeyUpdate: request.CrossSigningKeyUpdate,
|
||||||
|
},
|
||||||
})
|
})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
|
|
@ -94,10 +98,10 @@ func (t *EDUServerInputAPI) InputCrossSigningKeyUpdate(
|
||||||
|
|
||||||
logrus.WithFields(logrus.Fields{
|
logrus.WithFields(logrus.Fields{
|
||||||
"user_id": request.UserID,
|
"user_id": request.UserID,
|
||||||
}).Infof("Producing to topic '%s'", t.OutputCrossSigningKeyUpdateTopic)
|
}).Infof("Producing to topic '%s'", t.OutputKeyChangeEventTopic)
|
||||||
|
|
||||||
m := &sarama.ProducerMessage{
|
m := &sarama.ProducerMessage{
|
||||||
Topic: string(t.OutputCrossSigningKeyUpdateTopic),
|
Topic: string(t.OutputKeyChangeEventTopic),
|
||||||
Key: sarama.StringEncoder(request.UserID),
|
Key: sarama.StringEncoder(request.UserID),
|
||||||
Value: sarama.ByteEncoder(eventJSON),
|
Value: sarama.ByteEncoder(eventJSON),
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,121 +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 consumers
|
|
||||||
|
|
||||||
import (
|
|
||||||
"context"
|
|
||||||
"encoding/json"
|
|
||||||
"fmt"
|
|
||||||
|
|
||||||
"github.com/Shopify/sarama"
|
|
||||||
eduapi "github.com/matrix-org/dendrite/eduserver/api"
|
|
||||||
"github.com/matrix-org/dendrite/federationsender/queue"
|
|
||||||
"github.com/matrix-org/dendrite/federationsender/storage"
|
|
||||||
"github.com/matrix-org/dendrite/internal"
|
|
||||||
roomserverAPI "github.com/matrix-org/dendrite/roomserver/api"
|
|
||||||
"github.com/matrix-org/dendrite/setup/config"
|
|
||||||
"github.com/matrix-org/dendrite/setup/process"
|
|
||||||
"github.com/matrix-org/gomatrixserverlib"
|
|
||||||
"github.com/sirupsen/logrus"
|
|
||||||
log "github.com/sirupsen/logrus"
|
|
||||||
)
|
|
||||||
|
|
||||||
type CrossSigningKeyUpdateConsumer struct {
|
|
||||||
consumer *internal.ContinualConsumer
|
|
||||||
db storage.Database
|
|
||||||
queues *queue.OutgoingQueues
|
|
||||||
serverName gomatrixserverlib.ServerName
|
|
||||||
rsAPI roomserverAPI.RoomserverInternalAPI
|
|
||||||
}
|
|
||||||
|
|
||||||
func NewCrossSigningKeyUpdateConsumer(
|
|
||||||
process *process.ProcessContext,
|
|
||||||
cfg *config.KeyServer,
|
|
||||||
kafkaConsumer sarama.Consumer,
|
|
||||||
queues *queue.OutgoingQueues,
|
|
||||||
store storage.Database,
|
|
||||||
rsAPI roomserverAPI.RoomserverInternalAPI,
|
|
||||||
) *CrossSigningKeyUpdateConsumer {
|
|
||||||
c := &CrossSigningKeyUpdateConsumer{
|
|
||||||
consumer: &internal.ContinualConsumer{
|
|
||||||
Process: process,
|
|
||||||
ComponentName: "federationsender/crosssigning",
|
|
||||||
Topic: string(cfg.Matrix.Kafka.TopicFor(config.TopicOutputCrossSigningKeyUpdate)),
|
|
||||||
Consumer: kafkaConsumer,
|
|
||||||
PartitionStore: store,
|
|
||||||
},
|
|
||||||
queues: queues,
|
|
||||||
db: store,
|
|
||||||
serverName: cfg.Matrix.ServerName,
|
|
||||||
rsAPI: rsAPI,
|
|
||||||
}
|
|
||||||
c.consumer.ProcessMessage = c.onMessage
|
|
||||||
|
|
||||||
return c
|
|
||||||
}
|
|
||||||
|
|
||||||
func (t *CrossSigningKeyUpdateConsumer) Start() error {
|
|
||||||
if err := t.consumer.Start(); err != nil {
|
|
||||||
return fmt.Errorf("t.consumer.Start: %w", err)
|
|
||||||
}
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (t *CrossSigningKeyUpdateConsumer) onMessage(msg *sarama.ConsumerMessage) error {
|
|
||||||
var output eduapi.OutputCrossSigningKeyUpdate
|
|
||||||
if err := json.Unmarshal(msg.Value, &output); err != nil {
|
|
||||||
logrus.WithError(err).Errorf("eduserver output log: message parse failure")
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
_, host, err := gomatrixserverlib.SplitID('@', output.UserID)
|
|
||||||
if err != nil {
|
|
||||||
logrus.WithError(err).Errorf("eduserver output log: user ID parse failure")
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
if host != gomatrixserverlib.ServerName(t.serverName) {
|
|
||||||
// Ignore any messages that didn't originate locally, otherwise we'll
|
|
||||||
// end up parroting information we received from other servers.
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
logger := log.WithField("user_id", output.UserID)
|
|
||||||
|
|
||||||
var queryRes roomserverAPI.QueryRoomsForUserResponse
|
|
||||||
err = t.rsAPI.QueryRoomsForUser(context.Background(), &roomserverAPI.QueryRoomsForUserRequest{
|
|
||||||
UserID: output.UserID,
|
|
||||||
WantMembership: "join",
|
|
||||||
}, &queryRes)
|
|
||||||
if err != nil {
|
|
||||||
logger.WithError(err).Error("failed to calculate joined rooms for user")
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
// send this key change to all servers who share rooms with this user.
|
|
||||||
destinations, err := t.db.GetJoinedHostsForRooms(context.Background(), queryRes.RoomIDs)
|
|
||||||
if err != nil {
|
|
||||||
logger.WithError(err).Error("failed to calculate joined hosts for rooms user is in")
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// Pack the EDU and marshal it
|
|
||||||
edu := &gomatrixserverlib.EDU{
|
|
||||||
Type: eduapi.MSigningKeyUpdate,
|
|
||||||
Origin: string(t.serverName),
|
|
||||||
}
|
|
||||||
if edu.Content, err = json.Marshal(output.CrossSigningKeyUpdate); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
logger.Infof("Sending cross-signing update message to %q", destinations)
|
|
||||||
return t.queues.SendEDU(edu, t.serverName, destinations)
|
|
||||||
}
|
|
||||||
|
|
@ -20,6 +20,7 @@ import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
|
||||||
"github.com/Shopify/sarama"
|
"github.com/Shopify/sarama"
|
||||||
|
eduserverAPI "github.com/matrix-org/dendrite/eduserver/api"
|
||||||
"github.com/matrix-org/dendrite/federationsender/queue"
|
"github.com/matrix-org/dendrite/federationsender/queue"
|
||||||
"github.com/matrix-org/dendrite/federationsender/storage"
|
"github.com/matrix-org/dendrite/federationsender/storage"
|
||||||
"github.com/matrix-org/dendrite/internal"
|
"github.com/matrix-org/dendrite/internal"
|
||||||
|
|
@ -28,6 +29,7 @@ import (
|
||||||
"github.com/matrix-org/dendrite/setup/config"
|
"github.com/matrix-org/dendrite/setup/config"
|
||||||
"github.com/matrix-org/dendrite/setup/process"
|
"github.com/matrix-org/dendrite/setup/process"
|
||||||
"github.com/matrix-org/gomatrixserverlib"
|
"github.com/matrix-org/gomatrixserverlib"
|
||||||
|
"github.com/sirupsen/logrus"
|
||||||
log "github.com/sirupsen/logrus"
|
log "github.com/sirupsen/logrus"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
@ -83,6 +85,17 @@ func (t *KeyChangeConsumer) onMessage(msg *sarama.ConsumerMessage) error {
|
||||||
log.WithError(err).Errorf("failed to read device message from key change topic")
|
log.WithError(err).Errorf("failed to read device message from key change topic")
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
switch m.Type {
|
||||||
|
case api.TypeCrossSigningUpdate:
|
||||||
|
return t.onCrossSigningMessage(m)
|
||||||
|
case api.TypeDeviceKeyUpdate:
|
||||||
|
fallthrough
|
||||||
|
default:
|
||||||
|
return t.onDeviceKeyMessage(m)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *KeyChangeConsumer) onDeviceKeyMessage(m api.DeviceMessage) error {
|
||||||
logger := log.WithField("user_id", m.UserID)
|
logger := log.WithField("user_id", m.UserID)
|
||||||
|
|
||||||
// only send key change events which originated from us
|
// only send key change events which originated from us
|
||||||
|
|
@ -133,6 +146,49 @@ func (t *KeyChangeConsumer) onMessage(msg *sarama.ConsumerMessage) error {
|
||||||
return t.queues.SendEDU(edu, t.serverName, destinations)
|
return t.queues.SendEDU(edu, t.serverName, destinations)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (t *KeyChangeConsumer) onCrossSigningMessage(m api.DeviceMessage) error {
|
||||||
|
output := m.CrossSigningKeyUpdate
|
||||||
|
_, host, err := gomatrixserverlib.SplitID('@', output.UserID)
|
||||||
|
if err != nil {
|
||||||
|
logrus.WithError(err).Errorf("eduserver output log: user ID parse failure")
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
if host != gomatrixserverlib.ServerName(t.serverName) {
|
||||||
|
// Ignore any messages that didn't originate locally, otherwise we'll
|
||||||
|
// end up parroting information we received from other servers.
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
logger := log.WithField("user_id", output.UserID)
|
||||||
|
|
||||||
|
var queryRes roomserverAPI.QueryRoomsForUserResponse
|
||||||
|
err = t.rsAPI.QueryRoomsForUser(context.Background(), &roomserverAPI.QueryRoomsForUserRequest{
|
||||||
|
UserID: output.UserID,
|
||||||
|
WantMembership: "join",
|
||||||
|
}, &queryRes)
|
||||||
|
if err != nil {
|
||||||
|
logger.WithError(err).Error("failed to calculate joined rooms for user")
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
// send this key change to all servers who share rooms with this user.
|
||||||
|
destinations, err := t.db.GetJoinedHostsForRooms(context.Background(), queryRes.RoomIDs)
|
||||||
|
if err != nil {
|
||||||
|
logger.WithError(err).Error("failed to calculate joined hosts for rooms user is in")
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// Pack the EDU and marshal it
|
||||||
|
edu := &gomatrixserverlib.EDU{
|
||||||
|
Type: eduserverAPI.MSigningKeyUpdate,
|
||||||
|
Origin: string(t.serverName),
|
||||||
|
}
|
||||||
|
if edu.Content, err = json.Marshal(output); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
logger.Infof("Sending cross-signing update message to %q", destinations)
|
||||||
|
return t.queues.SendEDU(edu, t.serverName, destinations)
|
||||||
|
}
|
||||||
|
|
||||||
func prevID(streamID int) []int {
|
func prevID(streamID int) []int {
|
||||||
if streamID <= 1 {
|
if streamID <= 1 {
|
||||||
return nil
|
return nil
|
||||||
|
|
|
||||||
|
|
@ -94,12 +94,6 @@ func NewInternalAPI(
|
||||||
if err := keyConsumer.Start(); err != nil {
|
if err := keyConsumer.Start(); err != nil {
|
||||||
logrus.WithError(err).Panic("failed to start key server consumer")
|
logrus.WithError(err).Panic("failed to start key server consumer")
|
||||||
}
|
}
|
||||||
signingKeyConsumer := consumers.NewCrossSigningKeyUpdateConsumer(
|
|
||||||
base.ProcessContext, &base.Cfg.KeyServer, consumer, queues, federationSenderDB, rsAPI,
|
|
||||||
)
|
|
||||||
if err := signingKeyConsumer.Start(); err != nil {
|
|
||||||
logrus.WithError(err).Panic("failed to start signing key consumer")
|
|
||||||
}
|
|
||||||
|
|
||||||
return internal.NewFederationSenderInternalAPI(federationSenderDB, cfg, rsAPI, federation, keyRing, stats, queues)
|
return internal.NewFederationSenderInternalAPI(federationSenderDB, cfg, rsAPI, federation, keyRing, stats, queues)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -20,6 +20,7 @@ import (
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
eduapi "github.com/matrix-org/dendrite/eduserver/api"
|
||||||
"github.com/matrix-org/dendrite/keyserver/types"
|
"github.com/matrix-org/dendrite/keyserver/types"
|
||||||
userapi "github.com/matrix-org/dendrite/userapi/api"
|
userapi "github.com/matrix-org/dendrite/userapi/api"
|
||||||
"github.com/matrix-org/gomatrixserverlib"
|
"github.com/matrix-org/gomatrixserverlib"
|
||||||
|
|
@ -64,6 +65,7 @@ const (
|
||||||
type DeviceMessage struct {
|
type DeviceMessage struct {
|
||||||
Type DeviceMessageType `json:"Type,omitempty"`
|
Type DeviceMessageType `json:"Type,omitempty"`
|
||||||
*DeviceKeys `json:"DeviceKeys,omitempty"`
|
*DeviceKeys `json:"DeviceKeys,omitempty"`
|
||||||
|
*eduapi.OutputCrossSigningKeyUpdate `json:"CrossSigningKeyUpdate,omitempty"`
|
||||||
// A monotonically increasing number which represents device changes for this user.
|
// A monotonically increasing number which represents device changes for this user.
|
||||||
StreamID int
|
StreamID int
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,6 @@ import (
|
||||||
"context"
|
"context"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
|
|
||||||
eduapi "github.com/matrix-org/dendrite/eduserver/api"
|
|
||||||
"github.com/matrix-org/dendrite/internal"
|
"github.com/matrix-org/dendrite/internal"
|
||||||
"github.com/matrix-org/dendrite/keyserver/api"
|
"github.com/matrix-org/dendrite/keyserver/api"
|
||||||
"github.com/matrix-org/dendrite/keyserver/storage"
|
"github.com/matrix-org/dendrite/keyserver/storage"
|
||||||
|
|
@ -33,7 +32,7 @@ func NewOutputCrossSigningKeyUpdateConsumer(
|
||||||
consumer := internal.ContinualConsumer{
|
consumer := internal.ContinualConsumer{
|
||||||
Process: process,
|
Process: process,
|
||||||
ComponentName: "keyserver/crosssigning",
|
ComponentName: "keyserver/crosssigning",
|
||||||
Topic: cfg.Global.Kafka.TopicFor(config.TopicOutputCrossSigningKeyUpdate),
|
Topic: cfg.Global.Kafka.TopicFor(config.TopicOutputKeyChangeEvent),
|
||||||
Consumer: kafkaConsumer,
|
Consumer: kafkaConsumer,
|
||||||
PartitionStore: keyDB,
|
PartitionStore: keyDB,
|
||||||
}
|
}
|
||||||
|
|
@ -52,12 +51,24 @@ func (s *OutputCrossSigningKeyUpdateConsumer) Start() error {
|
||||||
return s.eduServerConsumer.Start()
|
return s.eduServerConsumer.Start()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *OutputCrossSigningKeyUpdateConsumer) onMessage(msg *sarama.ConsumerMessage) error {
|
// onMessage is called in response to a message received on the
|
||||||
var output eduapi.OutputCrossSigningKeyUpdate
|
// key change events topic from the key server.
|
||||||
if err := json.Unmarshal(msg.Value, &output); err != nil {
|
func (t *OutputCrossSigningKeyUpdateConsumer) onMessage(msg *sarama.ConsumerMessage) error {
|
||||||
logrus.WithError(err).Errorf("eduserver output log: message parse failure")
|
var m api.DeviceMessage
|
||||||
|
if err := json.Unmarshal(msg.Value, &m); err != nil {
|
||||||
|
logrus.WithError(err).Errorf("failed to read device message from key change topic")
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
switch m.Type {
|
||||||
|
case api.TypeCrossSigningUpdate:
|
||||||
|
return t.onCrossSigningMessage(m)
|
||||||
|
default:
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *OutputCrossSigningKeyUpdateConsumer) onCrossSigningMessage(m api.DeviceMessage) error {
|
||||||
|
output := m.CrossSigningKeyUpdate
|
||||||
_, host, err := gomatrixserverlib.SplitID('@', output.UserID)
|
_, host, err := gomatrixserverlib.SplitID('@', output.UserID)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logrus.WithError(err).Errorf("eduserver output log: user ID parse failure")
|
logrus.WithError(err).Errorf("eduserver output log: user ID parse failure")
|
||||||
|
|
|
||||||
|
|
@ -235,9 +235,9 @@ func (a *KeyInternalAPI) PerformUploadDeviceKeys(ctx context.Context, req *api.P
|
||||||
if _, ok := toStore[gomatrixserverlib.CrossSigningKeyPurposeSelfSigning]; ok {
|
if _, ok := toStore[gomatrixserverlib.CrossSigningKeyPurposeSelfSigning]; ok {
|
||||||
update.SelfSigningKey = &req.SelfSigningKey
|
update.SelfSigningKey = &req.SelfSigningKey
|
||||||
}
|
}
|
||||||
if err := a.CrossSigningProducer.ProduceSigningKeyUpdate(update); err != nil {
|
if err := a.Producer.ProduceSigningKeyUpdate(update); err != nil {
|
||||||
res.Error = &api.KeyError{
|
res.Error = &api.KeyError{
|
||||||
Err: fmt.Sprintf("a.CrossSigningProducer.ProduceSigningKeyUpdate: %s", err),
|
Err: fmt.Sprintf("a.Producer.ProduceSigningKeyUpdate: %s", err),
|
||||||
}
|
}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -39,8 +39,7 @@ type KeyInternalAPI struct {
|
||||||
ThisServer gomatrixserverlib.ServerName
|
ThisServer gomatrixserverlib.ServerName
|
||||||
FedClient fedsenderapi.FederationClient
|
FedClient fedsenderapi.FederationClient
|
||||||
UserAPI userapi.UserInternalAPI
|
UserAPI userapi.UserInternalAPI
|
||||||
DeviceKeysProducer *producers.KeyChange
|
Producer *producers.KeyChange
|
||||||
CrossSigningProducer *producers.CrossSigningKeyUpdate
|
|
||||||
Updater *DeviceListUpdater
|
Updater *DeviceListUpdater
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -61,7 +60,7 @@ func (a *KeyInternalAPI) InputDeviceListUpdate(
|
||||||
|
|
||||||
func (a *KeyInternalAPI) QueryKeyChanges(ctx context.Context, req *api.QueryKeyChangesRequest, res *api.QueryKeyChangesResponse) {
|
func (a *KeyInternalAPI) QueryKeyChanges(ctx context.Context, req *api.QueryKeyChangesRequest, res *api.QueryKeyChangesResponse) {
|
||||||
if req.Partition < 0 {
|
if req.Partition < 0 {
|
||||||
req.Partition = a.DeviceKeysProducer.DefaultPartition()
|
req.Partition = a.Producer.DefaultPartition()
|
||||||
}
|
}
|
||||||
userIDs, latest, err := a.DB.KeyChanges(ctx, req.Partition, req.Offset, req.ToOffset)
|
userIDs, latest, err := a.DB.KeyChanges(ctx, req.Partition, req.Offset, req.ToOffset)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
|
@ -599,7 +598,7 @@ func (a *KeyInternalAPI) uploadLocalDeviceKeys(ctx context.Context, req *api.Per
|
||||||
}
|
}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
err = emitDeviceKeyChanges(a.DeviceKeysProducer, existingKeys, keysToStore)
|
err = emitDeviceKeyChanges(a.Producer, existingKeys, keysToStore)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
util.GetLogger(ctx).Errorf("Failed to emitDeviceKeyChanges: %s", err)
|
util.GetLogger(ctx).Errorf("Failed to emitDeviceKeyChanges: %s", err)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -51,16 +51,11 @@ func NewInternalAPI(
|
||||||
Producer: producer,
|
Producer: producer,
|
||||||
DB: db,
|
DB: db,
|
||||||
}
|
}
|
||||||
crossSigningKeyUpdateProducer := &producers.CrossSigningKeyUpdate{
|
|
||||||
Topic: string(cfg.Matrix.Kafka.TopicFor(config.TopicOutputCrossSigningKeyUpdate)),
|
|
||||||
Producer: producer,
|
|
||||||
}
|
|
||||||
ap := &internal.KeyInternalAPI{
|
ap := &internal.KeyInternalAPI{
|
||||||
DB: db,
|
DB: db,
|
||||||
ThisServer: cfg.Matrix.ServerName,
|
ThisServer: cfg.Matrix.ServerName,
|
||||||
FedClient: fedClient,
|
FedClient: fedClient,
|
||||||
DeviceKeysProducer: keyChangeProducer,
|
Producer: keyChangeProducer,
|
||||||
CrossSigningProducer: crossSigningKeyUpdateProducer,
|
|
||||||
}
|
}
|
||||||
updater := internal.NewDeviceListUpdater(db, ap, keyChangeProducer, fedClient, 8) // 8 workers TODO: configurable
|
updater := internal.NewDeviceListUpdater(db, ap, keyChangeProducer, fedClient, 8) // 8 workers TODO: configurable
|
||||||
ap.Updater = updater
|
ap.Updater = updater
|
||||||
|
|
|
||||||
|
|
@ -1,58 +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 producers
|
|
||||||
|
|
||||||
import (
|
|
||||||
"encoding/json"
|
|
||||||
|
|
||||||
"github.com/Shopify/sarama"
|
|
||||||
"github.com/matrix-org/dendrite/eduserver/api"
|
|
||||||
"github.com/sirupsen/logrus"
|
|
||||||
)
|
|
||||||
|
|
||||||
type CrossSigningKeyUpdate struct {
|
|
||||||
Topic string
|
|
||||||
Producer sarama.SyncProducer
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *CrossSigningKeyUpdate) DefaultPartition() int32 {
|
|
||||||
return 0
|
|
||||||
}
|
|
||||||
|
|
||||||
func (p *CrossSigningKeyUpdate) ProduceSigningKeyUpdate(key api.CrossSigningKeyUpdate) error {
|
|
||||||
var m sarama.ProducerMessage
|
|
||||||
output := &api.OutputCrossSigningKeyUpdate{
|
|
||||||
CrossSigningKeyUpdate: key,
|
|
||||||
}
|
|
||||||
|
|
||||||
value, err := json.Marshal(output)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
m.Topic = string(p.Topic)
|
|
||||||
m.Key = sarama.StringEncoder(key.UserID)
|
|
||||||
m.Value = sarama.ByteEncoder(value)
|
|
||||||
|
|
||||||
_, _, err = p.Producer.SendMessage(&m)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
logrus.WithFields(logrus.Fields{
|
|
||||||
"user_id": key.UserID,
|
|
||||||
}).Infof("Produced to cross-signing update topic '%s'", p.Topic)
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
@ -19,6 +19,7 @@ import (
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
|
|
||||||
"github.com/Shopify/sarama"
|
"github.com/Shopify/sarama"
|
||||||
|
eduapi "github.com/matrix-org/dendrite/eduserver/api"
|
||||||
"github.com/matrix-org/dendrite/keyserver/api"
|
"github.com/matrix-org/dendrite/keyserver/api"
|
||||||
"github.com/matrix-org/dendrite/keyserver/storage"
|
"github.com/matrix-org/dendrite/keyserver/storage"
|
||||||
"github.com/sirupsen/logrus"
|
"github.com/sirupsen/logrus"
|
||||||
|
|
@ -73,3 +74,32 @@ func (p *KeyChange) ProduceKeyChanges(keys []api.DeviceMessage) error {
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (p *KeyChange) ProduceSigningKeyUpdate(key eduapi.CrossSigningKeyUpdate) error {
|
||||||
|
var m sarama.ProducerMessage
|
||||||
|
output := &api.DeviceMessage{
|
||||||
|
Type: api.TypeCrossSigningUpdate,
|
||||||
|
OutputCrossSigningKeyUpdate: &eduapi.OutputCrossSigningKeyUpdate{
|
||||||
|
CrossSigningKeyUpdate: key,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
value, err := json.Marshal(output)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
m.Topic = string(p.Topic)
|
||||||
|
m.Key = sarama.StringEncoder(key.UserID)
|
||||||
|
m.Value = sarama.ByteEncoder(value)
|
||||||
|
|
||||||
|
_, _, err = p.Producer.SendMessage(&m)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
logrus.WithFields(logrus.Fields{
|
||||||
|
"user_id": key.UserID,
|
||||||
|
}).Infof("Produced to cross-signing update topic '%s'", p.Topic)
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -10,7 +10,6 @@ const (
|
||||||
TopicOutputRoomEvent = "OutputRoomEvent"
|
TopicOutputRoomEvent = "OutputRoomEvent"
|
||||||
TopicOutputClientData = "OutputClientData"
|
TopicOutputClientData = "OutputClientData"
|
||||||
TopicOutputReceiptEvent = "OutputReceiptEvent"
|
TopicOutputReceiptEvent = "OutputReceiptEvent"
|
||||||
TopicOutputCrossSigningKeyUpdate = "OutputCrossSigningKeyUpdate"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type Kafka struct {
|
type Kafka struct {
|
||||||
|
|
|
||||||
|
|
@ -1,140 +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 consumers
|
|
||||||
|
|
||||||
import (
|
|
||||||
"context"
|
|
||||||
"encoding/json"
|
|
||||||
"sync"
|
|
||||||
|
|
||||||
"github.com/Shopify/sarama"
|
|
||||||
"github.com/getsentry/sentry-go"
|
|
||||||
eduserverAPI "github.com/matrix-org/dendrite/eduserver/api"
|
|
||||||
"github.com/matrix-org/dendrite/internal"
|
|
||||||
"github.com/matrix-org/dendrite/keyserver/api"
|
|
||||||
roomserverAPI "github.com/matrix-org/dendrite/roomserver/api"
|
|
||||||
"github.com/matrix-org/dendrite/setup/process"
|
|
||||||
"github.com/matrix-org/dendrite/syncapi/notifier"
|
|
||||||
"github.com/matrix-org/dendrite/syncapi/storage"
|
|
||||||
"github.com/matrix-org/dendrite/syncapi/types"
|
|
||||||
"github.com/matrix-org/gomatrixserverlib"
|
|
||||||
"github.com/sirupsen/logrus"
|
|
||||||
log "github.com/sirupsen/logrus"
|
|
||||||
)
|
|
||||||
|
|
||||||
// OutputCrossSigningKeyUpdateConsumer consumes events that originated in the key server.
|
|
||||||
type OutputCrossSigningKeyUpdateConsumer struct {
|
|
||||||
keyChangeConsumer *internal.ContinualConsumer
|
|
||||||
db storage.Database
|
|
||||||
notifier *notifier.Notifier
|
|
||||||
stream types.PartitionedStreamProvider
|
|
||||||
serverName gomatrixserverlib.ServerName // our server name
|
|
||||||
rsAPI roomserverAPI.RoomserverInternalAPI
|
|
||||||
keyAPI api.KeyInternalAPI
|
|
||||||
partitionToOffset map[int32]int64
|
|
||||||
partitionToOffsetMu sync.Mutex
|
|
||||||
}
|
|
||||||
|
|
||||||
// NewOutputCrossSigningKeyUpdateConsumer creates a new OutputCrossSigningKeyUpdateConsumer.
|
|
||||||
// Call Start() to begin consuming from the key server.
|
|
||||||
func NewOutputCrossSigningKeyUpdateConsumer(
|
|
||||||
process *process.ProcessContext,
|
|
||||||
serverName gomatrixserverlib.ServerName,
|
|
||||||
topic string,
|
|
||||||
kafkaConsumer sarama.Consumer,
|
|
||||||
keyAPI api.KeyInternalAPI,
|
|
||||||
rsAPI roomserverAPI.RoomserverInternalAPI,
|
|
||||||
store storage.Database,
|
|
||||||
notifier *notifier.Notifier,
|
|
||||||
stream types.PartitionedStreamProvider,
|
|
||||||
) *OutputCrossSigningKeyUpdateConsumer {
|
|
||||||
|
|
||||||
consumer := internal.ContinualConsumer{
|
|
||||||
Process: process,
|
|
||||||
ComponentName: "syncapi/crosssigning",
|
|
||||||
Topic: topic,
|
|
||||||
Consumer: kafkaConsumer,
|
|
||||||
PartitionStore: store,
|
|
||||||
}
|
|
||||||
|
|
||||||
s := &OutputCrossSigningKeyUpdateConsumer{
|
|
||||||
keyChangeConsumer: &consumer,
|
|
||||||
db: store,
|
|
||||||
serverName: serverName,
|
|
||||||
keyAPI: keyAPI,
|
|
||||||
rsAPI: rsAPI,
|
|
||||||
partitionToOffset: make(map[int32]int64),
|
|
||||||
partitionToOffsetMu: sync.Mutex{},
|
|
||||||
notifier: notifier,
|
|
||||||
stream: stream,
|
|
||||||
}
|
|
||||||
|
|
||||||
consumer.ProcessMessage = s.onMessage
|
|
||||||
|
|
||||||
return s
|
|
||||||
}
|
|
||||||
|
|
||||||
// Start consuming from the key server
|
|
||||||
func (s *OutputCrossSigningKeyUpdateConsumer) Start() error {
|
|
||||||
offsets, err := s.keyChangeConsumer.StartOffsets()
|
|
||||||
s.partitionToOffsetMu.Lock()
|
|
||||||
for _, o := range offsets {
|
|
||||||
s.partitionToOffset[o.Partition] = o.Offset
|
|
||||||
}
|
|
||||||
s.partitionToOffsetMu.Unlock()
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
func (s *OutputCrossSigningKeyUpdateConsumer) updateOffset(msg *sarama.ConsumerMessage) {
|
|
||||||
s.partitionToOffsetMu.Lock()
|
|
||||||
defer s.partitionToOffsetMu.Unlock()
|
|
||||||
s.partitionToOffset[msg.Partition] = msg.Offset
|
|
||||||
}
|
|
||||||
|
|
||||||
func (s *OutputCrossSigningKeyUpdateConsumer) onMessage(msg *sarama.ConsumerMessage) error {
|
|
||||||
defer s.updateOffset(msg)
|
|
||||||
|
|
||||||
var output eduserverAPI.OutputCrossSigningKeyUpdate
|
|
||||||
if err := json.Unmarshal(msg.Value, &output); err != nil {
|
|
||||||
logrus.WithError(err).Errorf("eduserver output log: message parse failure")
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
logrus.Infof("XXX: Sync API consumed crosssigning key update for %s", output.UserID)
|
|
||||||
|
|
||||||
// work out who we need to notify about the new key
|
|
||||||
var queryRes roomserverAPI.QuerySharedUsersResponse
|
|
||||||
err := s.rsAPI.QuerySharedUsers(context.Background(), &roomserverAPI.QuerySharedUsersRequest{
|
|
||||||
UserID: output.UserID,
|
|
||||||
}, &queryRes)
|
|
||||||
if err != nil {
|
|
||||||
log.WithError(err).Error("syncapi: failed to QuerySharedUsers for key change event from key server")
|
|
||||||
sentry.CaptureException(err)
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
// make sure we get our own key updates too!
|
|
||||||
queryRes.UserIDsToCount[output.UserID] = 1
|
|
||||||
posUpdate := types.LogPosition{
|
|
||||||
Offset: msg.Offset,
|
|
||||||
Partition: msg.Partition,
|
|
||||||
}
|
|
||||||
|
|
||||||
s.stream.Advance(posUpdate)
|
|
||||||
for userID := range queryRes.UserIDsToCount {
|
|
||||||
s.notifier.OnNewKeyChange(types.StreamingToken{DeviceListPosition: posUpdate}, userID, output.UserID)
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
@ -29,6 +29,7 @@ import (
|
||||||
"github.com/matrix-org/dendrite/syncapi/storage"
|
"github.com/matrix-org/dendrite/syncapi/storage"
|
||||||
"github.com/matrix-org/dendrite/syncapi/types"
|
"github.com/matrix-org/dendrite/syncapi/types"
|
||||||
"github.com/matrix-org/gomatrixserverlib"
|
"github.com/matrix-org/gomatrixserverlib"
|
||||||
|
"github.com/sirupsen/logrus"
|
||||||
log "github.com/sirupsen/logrus"
|
log "github.com/sirupsen/logrus"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
@ -104,13 +105,23 @@ func (s *OutputKeyChangeEventConsumer) updateOffset(msg *sarama.ConsumerMessage)
|
||||||
func (s *OutputKeyChangeEventConsumer) onMessage(msg *sarama.ConsumerMessage) error {
|
func (s *OutputKeyChangeEventConsumer) onMessage(msg *sarama.ConsumerMessage) error {
|
||||||
defer s.updateOffset(msg)
|
defer s.updateOffset(msg)
|
||||||
|
|
||||||
var output api.DeviceMessage
|
var m api.DeviceMessage
|
||||||
if err := json.Unmarshal(msg.Value, &output); err != nil {
|
if err := json.Unmarshal(msg.Value, &m); err != nil {
|
||||||
// If the message was invalid, log it and move on to the next message in the stream
|
logrus.WithError(err).Errorf("failed to read device message from key change topic")
|
||||||
log.WithError(err).Error("syncapi: failed to unmarshal key change event from key server")
|
return nil
|
||||||
sentry.CaptureException(err)
|
|
||||||
return err
|
|
||||||
}
|
}
|
||||||
|
switch m.Type {
|
||||||
|
case api.TypeCrossSigningUpdate:
|
||||||
|
return s.onCrossSigningMessage(m, msg.Offset, msg.Partition)
|
||||||
|
case api.TypeDeviceKeyUpdate:
|
||||||
|
fallthrough
|
||||||
|
default:
|
||||||
|
return s.onDeviceKeyMessage(m, msg.Offset, msg.Partition)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *OutputKeyChangeEventConsumer) onDeviceKeyMessage(m api.DeviceMessage, offset int64, partition int32) error {
|
||||||
|
output := m.DeviceKeys
|
||||||
// work out who we need to notify about the new key
|
// work out who we need to notify about the new key
|
||||||
var queryRes roomserverAPI.QuerySharedUsersResponse
|
var queryRes roomserverAPI.QuerySharedUsersResponse
|
||||||
err := s.rsAPI.QuerySharedUsers(context.Background(), &roomserverAPI.QuerySharedUsersRequest{
|
err := s.rsAPI.QuerySharedUsers(context.Background(), &roomserverAPI.QuerySharedUsersRequest{
|
||||||
|
|
@ -124,8 +135,35 @@ func (s *OutputKeyChangeEventConsumer) onMessage(msg *sarama.ConsumerMessage) er
|
||||||
// make sure we get our own key updates too!
|
// make sure we get our own key updates too!
|
||||||
queryRes.UserIDsToCount[output.UserID] = 1
|
queryRes.UserIDsToCount[output.UserID] = 1
|
||||||
posUpdate := types.LogPosition{
|
posUpdate := types.LogPosition{
|
||||||
Offset: msg.Offset,
|
Offset: offset,
|
||||||
Partition: msg.Partition,
|
Partition: partition,
|
||||||
|
}
|
||||||
|
|
||||||
|
s.stream.Advance(posUpdate)
|
||||||
|
for userID := range queryRes.UserIDsToCount {
|
||||||
|
s.notifier.OnNewKeyChange(types.StreamingToken{DeviceListPosition: posUpdate}, userID, output.UserID)
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *OutputKeyChangeEventConsumer) onCrossSigningMessage(m api.DeviceMessage, offset int64, partition int32) error {
|
||||||
|
output := m.CrossSigningKeyUpdate
|
||||||
|
// work out who we need to notify about the new key
|
||||||
|
var queryRes roomserverAPI.QuerySharedUsersResponse
|
||||||
|
err := s.rsAPI.QuerySharedUsers(context.Background(), &roomserverAPI.QuerySharedUsersRequest{
|
||||||
|
UserID: output.UserID,
|
||||||
|
}, &queryRes)
|
||||||
|
if err != nil {
|
||||||
|
log.WithError(err).Error("syncapi: failed to QuerySharedUsers for key change event from key server")
|
||||||
|
sentry.CaptureException(err)
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
// make sure we get our own key updates too!
|
||||||
|
queryRes.UserIDsToCount[output.UserID] = 1
|
||||||
|
posUpdate := types.LogPosition{
|
||||||
|
Offset: offset,
|
||||||
|
Partition: partition,
|
||||||
}
|
}
|
||||||
|
|
||||||
s.stream.Advance(posUpdate)
|
s.stream.Advance(posUpdate)
|
||||||
|
|
|
||||||
|
|
@ -72,14 +72,6 @@ func AddPublicRoutes(
|
||||||
logrus.WithError(err).Panicf("failed to start key change consumer")
|
logrus.WithError(err).Panicf("failed to start key change consumer")
|
||||||
}
|
}
|
||||||
|
|
||||||
crossSigningKeyUpdateConsumer := consumers.NewOutputCrossSigningKeyUpdateConsumer(
|
|
||||||
process, cfg.Matrix.ServerName, string(cfg.Matrix.Kafka.TopicFor(config.TopicOutputCrossSigningKeyUpdate)),
|
|
||||||
consumer, keyAPI, rsAPI, syncDB, notifier, streams.DeviceListStreamProvider,
|
|
||||||
)
|
|
||||||
if err = crossSigningKeyUpdateConsumer.Start(); err != nil {
|
|
||||||
logrus.WithError(err).Panicf("failed to start cross-signing key change consumer")
|
|
||||||
}
|
|
||||||
|
|
||||||
roomConsumer := consumers.NewOutputRoomEventConsumer(
|
roomConsumer := consumers.NewOutputRoomEventConsumer(
|
||||||
process, cfg, consumer, syncDB, notifier, streams.PDUStreamProvider,
|
process, cfg, consumer, syncDB, notifier, streams.PDUStreamProvider,
|
||||||
streams.InviteStreamProvider, rsAPI,
|
streams.InviteStreamProvider, rsAPI,
|
||||||
|
|
|
||||||
Loading…
Reference in a new issue