First pass as de-MSC-ifying space summaries

- Opt for referring to it as 'room hierarchies', reflecting the route
- Split msc2945.go across clientapi and roomserverapi
- Still TODO fed api
- Some other TODOs still to do
This commit is contained in:
Sam Wedgwood 2023-07-04 15:06:36 +01:00
parent 5267cc0f54
commit c97ba0b9e6
14 changed files with 856 additions and 25 deletions

View file

@ -0,0 +1,150 @@
// Copyright 2021 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 routing
import (
"net/http"
"strconv"
"sync"
"github.com/google/uuid"
roomserverAPI "github.com/matrix-org/dendrite/roomserver/api"
"github.com/matrix-org/dendrite/roomserver/types"
userapi "github.com/matrix-org/dendrite/userapi/api"
"github.com/matrix-org/gomatrixserverlib/fclient"
"github.com/matrix-org/gomatrixserverlib/spec"
"github.com/matrix-org/util"
)
type RoomHierarchyPaginationCache struct {
cache map[string]roomserverAPI.CachedRoomHierarchyWalker
mu sync.Mutex
}
func NewRoomHierarchyPaginationCache() RoomHierarchyPaginationCache {
return RoomHierarchyPaginationCache{}
}
func (c *RoomHierarchyPaginationCache) Get(token string) roomserverAPI.CachedRoomHierarchyWalker {
c.mu.Lock()
defer c.mu.Unlock()
line := c.cache[token]
return line
}
func (c *RoomHierarchyPaginationCache) AddLine(line roomserverAPI.CachedRoomHierarchyWalker) string {
c.mu.Lock()
defer c.mu.Unlock()
token := uuid.NewString()
c.cache[token] = line
return token
}
func QueryRoomHierarchy(req *http.Request, device *userapi.Device, roomIDStr string, rsAPI roomserverAPI.ClientRoomserverAPI, paginationCache *RoomHierarchyPaginationCache) util.JSONResponse {
parsedRoomID, err := spec.NewRoomID(roomIDStr)
if err != nil {
return util.JSONResponse{
Code: http.StatusNotFound,
JSON: spec.InvalidParam("room is unknown/forbidden"),
}
}
roomID := *parsedRoomID
suggestedOnly := false // Defaults to false (spec-defined)
switch req.URL.Query().Get("suggested_only") {
case "true":
suggestedOnly = true
case "false":
case "": // Empty string is returned when query param is not set
default:
return util.JSONResponse{
Code: http.StatusBadRequest,
JSON: spec.InvalidParam("query parameter 'suggested_only', if set, must be 'true' or 'false'"),
}
}
limit := 1000 // Default to 1000
limitStr := req.URL.Query().Get("limit")
if limitStr != "" {
maybeLimit, err := strconv.Atoi(limitStr)
if err != nil || maybeLimit < 0 {
return util.JSONResponse{
Code: http.StatusBadRequest,
JSON: spec.InvalidParam("query parameter 'limit', if set, must be a positive integer"),
}
}
limit = maybeLimit
if limit > 1000 {
limit = 1000 // Maximum limit of 1000
}
}
maxDepth := -1 // '-1' representing no maximum depth
maxDepthStr := req.URL.Query().Get("max_depth")
if maxDepthStr != "" {
maybeMaxDepth, err := strconv.Atoi(maxDepthStr)
if err != nil || maybeMaxDepth < 0 {
return util.JSONResponse{
Code: http.StatusBadRequest,
JSON: spec.InvalidParam("query parameter 'max_depth', if set, must be a positive integer"),
}
}
maxDepth = maybeMaxDepth
}
from := req.URL.Query().Get("from")
var walker roomserverAPI.RoomHierarchyWalker
if from == "" { // No pagination token provided, so start new hierarchy walker
walker = rsAPI.QueryRoomHierarchy(req.Context(), types.NewDeviceNotServerName(*device), roomID, suggestedOnly, maxDepth)
} else { // Attempt to resume cached walker
cachedWalker := paginationCache.Get(from)
if cachedWalker == nil || !cachedWalker.ValidateParams(suggestedOnly, maxDepth) {
return util.JSONResponse{
Code: http.StatusBadRequest,
JSON: spec.InvalidParam("pagination not found for provided token ('from') with given 'max_depth', 'suggested_only' and room ID"),
}
}
walker = cachedWalker.GetWalker()
}
discoveredRooms, err := walker.NextPage(limit)
if err != nil {
// TODO
}
nextBatch := ""
if !walker.Done() {
cacheLine := walker.GetCached()
nextBatch = paginationCache.AddLine(cacheLine)
}
return util.JSONResponse{
Code: http.StatusOK,
JSON: MSC2946ClientResponse{
Rooms: discoveredRooms,
NextBatch: nextBatch,
},
}
}
type MSC2946ClientResponse struct {
Rooms []fclient.MSC2946Room `json:"rooms"`
NextBatch string `json:"next_batch,omitempty"`
}

View file

@ -288,6 +288,8 @@ func Setup(
// Note that 'apiversion' is chosen because it must not collide with a variable used in any of the routing!
v3mux := publicAPIMux.PathPrefix("/{apiversion:(?:r0|v3)}/").Subrouter()
v1mux := publicAPIMux.PathPrefix("/v1/").Subrouter()
unstableMux := publicAPIMux.PathPrefix("/unstable").Subrouter()
v3mux.Handle("/createRoom",
@ -505,6 +507,18 @@ func Setup(
}, httputil.WithAllowGuests()),
).Methods(http.MethodPut, http.MethodOptions)
// Defined outside of handler to persist between calls
roomHierarchyPaginationCache := new(RoomHierarchyPaginationCache)
v1mux.Handle("/rooms/{roomID}/hierarchy",
httputil.MakeAuthAPI("spaces", userAPI, func(req *http.Request, device *userapi.Device) util.JSONResponse {
vars, err := httputil.URLDecodeMapValues(mux.Vars(req))
if err != nil {
return util.ErrorResponse(err)
}
return QueryRoomHierarchy(req, device, vars["roomID"], rsAPI, roomHierarchyPaginationCache)
}, httputil.WithAllowGuests()),
).Methods(http.MethodGet, http.MethodOptions)
v3mux.Handle("/register", httputil.MakeExternalAPI("register", func(req *http.Request) util.JSONResponse {
if r := rateLimits.Limit(req, nil); r != nil {
return *r

View file

@ -27,7 +27,6 @@ type FederationInternalAPI interface {
QueryServerKeys(ctx context.Context, request *QueryServerKeysRequest, response *QueryServerKeysResponse) error
LookupServerKeys(ctx context.Context, s spec.ServerName, keyRequests map[gomatrixserverlib.PublicKeyLookupRequest]spec.Timestamp) ([]gomatrixserverlib.ServerKeys, error)
MSC2836EventRelationships(ctx context.Context, origin, dst spec.ServerName, r fclient.MSC2836EventRelationshipsRequest, roomVersion gomatrixserverlib.RoomVersion) (res fclient.MSC2836EventRelationshipsResponse, err error)
MSC2946Spaces(ctx context.Context, origin, dst spec.ServerName, roomID string, suggestedOnly bool) (res fclient.MSC2946SpacesResponse, err error)
// Broadcasts an EDU to all servers in rooms we are joined to. Used in the yggdrasil demos.
PerformBroadcastEDU(
@ -75,6 +74,8 @@ type RoomserverFederationAPI interface {
GetEventAuth(ctx context.Context, origin, s spec.ServerName, roomVersion gomatrixserverlib.RoomVersion, roomID, eventID string) (res fclient.RespEventAuth, err error)
GetEvent(ctx context.Context, origin, s spec.ServerName, eventID string) (res gomatrixserverlib.Transaction, err error)
LookupMissingEvents(ctx context.Context, origin, s spec.ServerName, roomID string, missing fclient.MissingEvents, roomVersion gomatrixserverlib.RoomVersion) (res fclient.RespMissingEvents, err error)
MSC2946Spaces(ctx context.Context, origin, dst spec.ServerName, roomID string, suggestedOnly bool) (res fclient.MSC2946SpacesResponse, err error)
}
type P2PFederationAPI interface {

View file

@ -8,6 +8,7 @@ type RoomServerCaches interface {
RoomServerNIDsCache
RoomVersionCache
RoomServerEventsCache
RoomHierarchyCache
EventStateKeyCache
EventTypeCache
}

View file

@ -2,15 +2,16 @@ package caching
import "github.com/matrix-org/gomatrixserverlib/fclient"
type SpaceSummaryRoomsCache interface {
GetSpaceSummary(roomID string) (r fclient.MSC2946SpacesResponse, ok bool)
StoreSpaceSummary(roomID string, r fclient.MSC2946SpacesResponse)
// RoomHierarchy cache caches responses to federated room hierarchy requests (A.K.A. 'space summaries')
type RoomHierarchyCache interface {
GetRoomHierarchy(roomID string) (r fclient.MSC2946SpacesResponse, ok bool)
StoreRoomHierarchy(roomID string, r fclient.MSC2946SpacesResponse)
}
func (c Caches) GetSpaceSummary(roomID string) (r fclient.MSC2946SpacesResponse, ok bool) {
return c.SpaceSummaryRooms.Get(roomID)
func (c Caches) GetRoomHierarchy(roomID string) (r fclient.MSC2946SpacesResponse, ok bool) {
return c.RoomHierarchies.Get(roomID)
}
func (c Caches) StoreSpaceSummary(roomID string, r fclient.MSC2946SpacesResponse) {
c.SpaceSummaryRooms.Set(roomID, r)
func (c Caches) StoreRoomHierarchy(roomID string, r fclient.MSC2946SpacesResponse) {
c.RoomHierarchies.Set(roomID, r)
}

View file

@ -35,7 +35,7 @@ type Caches struct {
RoomServerEventTypes Cache[types.EventTypeNID, string] // eventType NID -> eventType
FederationPDUs Cache[int64, *types.HeaderedEvent] // queue NID -> PDU
FederationEDUs Cache[int64, *gomatrixserverlib.EDU] // queue NID -> EDU
SpaceSummaryRooms Cache[string, fclient.MSC2946SpacesResponse] // room ID -> space response
RoomHierarchies Cache[string, fclient.MSC2946SpacesResponse] // room ID -> space response
LazyLoading Cache[lazyLoadingCacheKey, string] // composite key -> event ID
}

View file

@ -147,7 +147,7 @@ func NewRistrettoCache(maxCost config.DataUnit, maxAge time.Duration, enableProm
MaxAge: lesserOf(time.Hour/2, maxAge),
},
},
SpaceSummaryRooms: &RistrettoCachePartition[string, fclient.MSC2946SpacesResponse]{ // room ID -> space response
RoomHierarchies: &RistrettoCachePartition[string, fclient.MSC2946SpacesResponse]{ // room ID -> space response
cache: cache,
Prefix: spaceSummaryRoomsCache,
Mutable: true,

View file

@ -113,6 +113,10 @@ type QueryEventsAPI interface {
QueryCurrentState(ctx context.Context, req *QueryCurrentStateRequest, res *QueryCurrentStateResponse) error
}
type QueryRoomHierarchyAPI interface {
QueryRoomHierarchy(ctx context.Context, caller types.DeviceOrServerName, roomID spec.RoomID, suggestedOnly bool, maxDepth int) RoomHierarchyWalker
}
// API functions required by the syncapi
type SyncRoomserverAPI interface {
QueryLatestEventsAndStateAPI
@ -187,6 +191,7 @@ type ClientRoomserverAPI interface {
QueryEventsAPI
QuerySenderIDAPI
UserRoomPrivateKeyCreator
QueryRoomHierarchyAPI
QueryMembershipForUser(ctx context.Context, req *QueryMembershipForUserRequest, res *QueryMembershipForUserResponse) error
QueryMembershipsForRoom(ctx context.Context, req *QueryMembershipsForRoomRequest, res *QueryMembershipsForRoomResponse) error
QueryRoomsForUser(ctx context.Context, req *QueryRoomsForUserRequest, res *QueryRoomsForUserResponse) error
@ -236,6 +241,7 @@ type FederationRoomserverAPI interface {
QueryLatestEventsAndStateAPI
QueryBulkStateContentAPI
QuerySenderIDAPI
QueryRoomHierarchyAPI
UserRoomPrivateKeyCreator
AssignRoomNID(ctx context.Context, roomID spec.RoomID, roomVersion gomatrixserverlib.RoomVersion) (roomNID types.RoomNID, err error)
SigningIdentityFor(ctx context.Context, roomID spec.RoomID, senderID spec.UserID) (fclient.SigningIdentity, error)

View file

@ -23,6 +23,7 @@ import (
"strings"
"github.com/matrix-org/gomatrixserverlib"
"github.com/matrix-org/gomatrixserverlib/fclient"
"github.com/matrix-org/gomatrixserverlib/spec"
"github.com/matrix-org/util"
@ -503,3 +504,24 @@ func (mq *MembershipQuerier) CurrentMembership(ctx context.Context, roomID spec.
}
return membership, err
}
type QueryRoomHierarchyRequest struct {
SuggestedOnly bool `json:"suggested_only"`
Limit int `json:"limit"`
MaxDepth int `json:"max_depth"`
From int `json:"json"`
}
type RoomHierarchyWalker interface {
NextPage(limit int) ([]fclient.MSC2946Room, error)
Done() bool
GetCached() CachedRoomHierarchyWalker
}
// Stripped down version of RoomHierarchyWalker suitable for caching (for pagination)
type CachedRoomHierarchyWalker interface {
// Converts this cached walker back into an actual walker, to resume walking from.
GetWalker() RoomHierarchyWalker
// Validates that the given parameters match those stored in the cache
ValidateParams(suggestedOnly bool, maxDepth int) bool
}

View file

@ -91,15 +91,8 @@ func NewRoomserverAPI(
NATSClient: nc,
Durable: dendriteCfg.Global.JetStream.Durable("RoomserverInputConsumer"),
ServerACLs: serverACLs,
Queryer: &query.Queryer{
DB: roomserverDB,
Cache: caches,
IsLocalServerName: dendriteCfg.Global.IsLocalServerName,
ServerACLs: serverACLs,
Cfg: dendriteCfg,
},
enableMetrics: enableMetrics,
// perform-er structs get initialised when we have a federation sender to use
// perform-er structs + queryer struct get initialised when we have a federation sender to use
}
return a
}
@ -111,6 +104,15 @@ func (r *RoomserverInternalAPI) SetFederationAPI(fsAPI fsAPI.RoomserverFederatio
r.fsAPI = fsAPI
r.KeyRing = keyRing
r.Queryer = &query.Queryer{
DB: r.DB,
Cache: r.Cache,
IsLocalServerName: r.Cfg.Global.IsLocalServerName,
ServerACLs: r.ServerACLs,
Cfg: r.Cfg,
FSAPI: fsAPI,
}
r.Inputer = &input.Inputer{
Cfg: &r.Cfg.RoomServer,
ProcessContext: r.ProcessContext,

View file

@ -32,6 +32,7 @@ import (
"github.com/matrix-org/dendrite/syncapi/synctypes"
"github.com/matrix-org/dendrite/clientapi/auth/authtypes"
fsAPI "github.com/matrix-org/dendrite/federationapi/api"
"github.com/matrix-org/dendrite/internal/caching"
"github.com/matrix-org/dendrite/roomserver/acls"
"github.com/matrix-org/dendrite/roomserver/api"
@ -47,6 +48,7 @@ type Queryer struct {
IsLocalServerName func(spec.ServerName) bool
ServerACLs *acls.ServerACLs
Cfg *config.Dendrite
FSAPI fsAPI.RoomserverFederationAPI
}
func (r *Queryer) RestrictedRoomJoinInfo(ctx context.Context, roomID spec.RoomID, senderID spec.SenderID, localServerName spec.ServerName) (*gomatrixserverlib.RestrictedRoomJoinInfo, error) {

View file

@ -0,0 +1,597 @@
package query
import (
"context"
"encoding/json"
"sort"
"strings"
fs "github.com/matrix-org/dendrite/federationapi/api"
"github.com/matrix-org/dendrite/internal/caching"
roomserver "github.com/matrix-org/dendrite/roomserver/api"
"github.com/matrix-org/dendrite/roomserver/types"
userapi "github.com/matrix-org/dendrite/userapi/api"
"github.com/matrix-org/gomatrixserverlib"
"github.com/matrix-org/gomatrixserverlib/fclient"
"github.com/matrix-org/gomatrixserverlib/spec"
"github.com/matrix-org/util"
"github.com/tidwall/gjson"
)
// Query the hierarchy of a room (A.K.A 'space summary')
//
// This function returns a iterator-like struct over the hierarchy of a room.
func (r *Queryer) QueryRoomHierarchy(ctx context.Context, caller types.DeviceOrServerName, roomID spec.RoomID, suggestedOnly bool, maxDepth int) roomserver.RoomHierarchyWalker {
walker := RoomHierarchyWalker{
rootRoomID: roomID.String(),
caller: caller,
thisServer: r.Cfg.Global.ServerName,
rsAPI: r,
fsAPI: r.FSAPI,
ctx: ctx,
roomHierarchyCache: r.Cache,
suggestedOnly: suggestedOnly,
maxDepth: maxDepth,
unvisited: []roomVisit{{
roomID: roomID.String(),
parentRoomID: "",
depth: 0,
}},
}
return &walker
}
type stringSet map[string]struct{}
func (s stringSet) contains(val string) bool {
_, ok := s[val]
return ok
}
func (s stringSet) add(val string) {
s[val] = struct{}{}
}
type RoomHierarchyWalker struct {
rootRoomID string // TODO change to spec.RoomID
caller types.DeviceOrServerName
thisServer spec.ServerName
rsAPI *Queryer
fsAPI fs.RoomserverFederationAPI
ctx context.Context
roomHierarchyCache caching.RoomHierarchyCache
suggestedOnly bool
maxDepth int
processed stringSet
unvisited []roomVisit
done bool
}
const (
ConstCreateEventContentKey = "type"
ConstCreateEventContentValueSpace = "m.space"
ConstSpaceChildEventType = "m.space.child"
ConstSpaceParentEventType = "m.space.parent"
)
func (w *RoomHierarchyWalker) NextPage(limit int) ([]fclient.MSC2946Room, error) {
if authorised, _ := w.authorised(w.rootRoomID, ""); !authorised {
return nil, spec.Forbidden("room is unknown/forbidden")
}
var discoveredRooms []fclient.MSC2946Room
// Depth first -> stack data structure
for len(w.unvisited) > 0 {
if len(discoveredRooms) >= limit {
break
}
// pop the stack
rv := w.unvisited[len(w.unvisited)-1]
w.unvisited = w.unvisited[:len(w.unvisited)-1]
// If this room has already been processed, skip.
// If this room exceeds the specified depth, skip.
if w.processed.contains(rv.roomID) || rv.roomID == "" || (w.maxDepth > 0 && rv.depth > w.maxDepth) {
continue
}
// Mark this room as processed.
w.processed.add(rv.roomID)
// if this room is not a space room, skip.
var roomType string
create := w.stateEvent(rv.roomID, spec.MRoomCreate, "")
if create != nil {
// escape the `.`s so gjson doesn't think it's nested
roomType = gjson.GetBytes(create.Content(), strings.ReplaceAll(ConstCreateEventContentKey, ".", `\.`)).Str
}
// Collect rooms/events to send back (either locally or fetched via federation)
var discoveredChildEvents []fclient.MSC2946StrippedEvent
// If we know about this room and the caller is authorised (joined/world_readable) then pull
// events locally
roomExists := w.roomExists(rv.roomID)
if !roomExists {
// attempt to query this room over federation, as either we've never heard of it before
// or we've left it and hence are not authorised (but info may be exposed regardless)
fedRes := w.federatedRoomInfo(rv.roomID, rv.vias)
if fedRes != nil {
discoveredChildEvents = fedRes.Room.ChildrenState
discoveredRooms = append(discoveredRooms, fedRes.Room)
if len(fedRes.Children) > 0 {
discoveredRooms = append(discoveredRooms, fedRes.Children...)
}
// mark this room as a space room as the federated server responded.
// we need to do this so we add the children of this room to the unvisited stack
// as these children may be rooms we do know about.
roomType = ConstCreateEventContentValueSpace
}
} else if authorised, isJoinedOrInvited := w.authorised(rv.roomID, rv.parentRoomID); authorised {
// Get all `m.space.child` state events for this room
events, err := w.childReferences(rv.roomID)
if err != nil {
util.GetLogger(w.ctx).WithError(err).WithField("room_id", rv.roomID).Error("failed to extract references for room")
continue
}
discoveredChildEvents = events
pubRoom := w.publicRoomsChunk(rv.roomID)
discoveredRooms = append(discoveredRooms, fclient.MSC2946Room{
PublicRoom: *pubRoom,
RoomType: roomType,
ChildrenState: events,
})
// don't walk children if the user is not joined/invited to the space
if !isJoinedOrInvited {
continue
}
} else {
// room exists but user is not authorised
continue
}
// don't walk the children
// if the parent is not a space room
if roomType != ConstCreateEventContentValueSpace {
continue
}
// For each referenced room ID in the child events being returned to the caller
// add the room ID to the queue of unvisited rooms. Loop from the beginning.
// We need to invert the order here because the child events are lo->hi on the timestamp,
// so we need to ensure we pop in the same lo->hi order, which won't be the case if we
// insert the highest timestamp last in a stack.
for i := len(discoveredChildEvents) - 1; i >= 0; i-- {
spaceContent := struct {
Via []string `json:"via"`
}{}
ev := discoveredChildEvents[i]
_ = json.Unmarshal(ev.Content, &spaceContent)
w.unvisited = append(w.unvisited, roomVisit{
roomID: ev.StateKey,
parentRoomID: rv.roomID,
depth: rv.depth + 1,
vias: spaceContent.Via,
})
}
}
if len(w.unvisited) == 0 {
w.done = true
}
return discoveredRooms, nil
}
func (w *RoomHierarchyWalker) Done() bool {
return w.done
}
func (w *RoomHierarchyWalker) GetCached() roomserver.CachedRoomHierarchyWalker {
return CachedRoomHierarchyWalker{
rootRoomID: w.rootRoomID,
caller: w.caller,
thisServer: w.thisServer,
rsAPI: w.rsAPI,
fsAPI: w.fsAPI,
ctx: w.ctx,
cache: w.roomHierarchyCache,
suggestedOnly: w.suggestedOnly,
maxDepth: w.maxDepth,
processed: w.processed,
unvisited: w.unvisited,
done: w.done,
}
}
func (w *RoomHierarchyWalker) stateEvent(roomID, evType, stateKey string) *types.HeaderedEvent {
var queryRes roomserver.QueryCurrentStateResponse
tuple := gomatrixserverlib.StateKeyTuple{
EventType: evType,
StateKey: stateKey,
}
err := w.rsAPI.QueryCurrentState(w.ctx, &roomserver.QueryCurrentStateRequest{
RoomID: roomID,
StateTuples: []gomatrixserverlib.StateKeyTuple{tuple},
}, &queryRes)
if err != nil {
return nil
}
return queryRes.StateEvents[tuple]
}
func (w *RoomHierarchyWalker) roomExists(roomID string) bool {
var queryRes roomserver.QueryServerJoinedToRoomResponse
err := w.rsAPI.QueryServerJoinedToRoom(w.ctx, &roomserver.QueryServerJoinedToRoomRequest{
RoomID: roomID,
ServerName: w.thisServer,
}, &queryRes)
if err != nil {
util.GetLogger(w.ctx).WithError(err).Error("failed to QueryServerJoinedToRoom")
return false
}
// if the room exists but we aren't in the room then we might have stale data so we want to fetch
// it fresh via federation
return queryRes.RoomExists && queryRes.IsInRoom
}
// federatedRoomInfo returns more of the spaces graph from another server. Returns nil if this was
// unsuccessful.
func (w *RoomHierarchyWalker) federatedRoomInfo(roomID string, vias []string) *fclient.MSC2946SpacesResponse {
// only do federated requests for client requests
if w.caller.Device() == nil {
return nil
}
resp, ok := w.roomHierarchyCache.GetRoomHierarchy(roomID)
if ok {
util.GetLogger(w.ctx).Debugf("Returning cached response for %s", roomID)
return &resp
}
util.GetLogger(w.ctx).Debugf("Querying %s via %+v", roomID, vias)
ctx := context.Background()
// query more of the spaces graph using these servers
for _, serverName := range vias {
if serverName == string(w.thisServer) {
continue
}
res, err := w.fsAPI.MSC2946Spaces(ctx, w.thisServer, spec.ServerName(serverName), roomID, w.suggestedOnly)
if err != nil {
util.GetLogger(w.ctx).WithError(err).Warnf("failed to call MSC2946Spaces on server %s", serverName)
continue
}
// ensure nil slices are empty as we send this to the client sometimes
if res.Room.ChildrenState == nil {
res.Room.ChildrenState = []fclient.MSC2946StrippedEvent{}
}
for i := 0; i < len(res.Children); i++ {
child := res.Children[i]
if child.ChildrenState == nil {
child.ChildrenState = []fclient.MSC2946StrippedEvent{}
}
res.Children[i] = child
}
w.roomHierarchyCache.StoreRoomHierarchy(roomID, res)
return &res
}
return nil
}
// references returns all child references pointing to or from this room.
func (w *RoomHierarchyWalker) childReferences(roomID string) ([]fclient.MSC2946StrippedEvent, error) {
createTuple := gomatrixserverlib.StateKeyTuple{
EventType: spec.MRoomCreate,
StateKey: "",
}
var res roomserver.QueryCurrentStateResponse
err := w.rsAPI.QueryCurrentState(context.Background(), &roomserver.QueryCurrentStateRequest{
RoomID: roomID,
AllowWildcards: true,
StateTuples: []gomatrixserverlib.StateKeyTuple{
createTuple, {
EventType: ConstSpaceChildEventType,
StateKey: "*",
},
},
}, &res)
if err != nil {
return nil, err
}
// don't return any child refs if the room is not a space room
if res.StateEvents[createTuple] != nil {
// escape the `.`s so gjson doesn't think it's nested
roomType := gjson.GetBytes(res.StateEvents[createTuple].Content(), strings.ReplaceAll(ConstCreateEventContentKey, ".", `\.`)).Str
if roomType != ConstCreateEventContentValueSpace {
return []fclient.MSC2946StrippedEvent{}, nil
}
}
delete(res.StateEvents, createTuple)
el := make([]fclient.MSC2946StrippedEvent, 0, len(res.StateEvents))
for _, ev := range res.StateEvents {
content := gjson.ParseBytes(ev.Content())
// only return events that have a `via` key as per MSC1772
// else we'll incorrectly walk redacted events (as the link
// is in the state_key)
if content.Get("via").Exists() {
strip := stripped(ev.PDU)
if strip == nil {
continue
}
// if suggested only and this child isn't suggested, skip it.
// if suggested only = false we include everything so don't need to check the content.
if w.suggestedOnly && !content.Get("suggested").Bool() {
continue
}
el = append(el, *strip)
}
}
// sort by origin_server_ts as per MSC2946
sort.Slice(el, func(i, j int) bool {
return el[i].OriginServerTS < el[j].OriginServerTS
})
return el, nil
}
// authorised returns true iff the user is joined this room or the room is world_readable
func (w *RoomHierarchyWalker) authorised(roomID, parentRoomID string) (authed, isJoinedOrInvited bool) {
if clientCaller := w.caller.Device(); clientCaller != nil {
return w.authorisedUser(roomID, clientCaller, parentRoomID)
} else {
return w.authorisedServer(roomID, *w.caller.ServerName()), false
}
}
// authorisedServer returns true iff the server is joined this room or the room is world_readable, public, or knockable
func (w *RoomHierarchyWalker) authorisedServer(roomID string, callerServerName spec.ServerName) bool {
// Check history visibility / join rules first
hisVisTuple := gomatrixserverlib.StateKeyTuple{
EventType: spec.MRoomHistoryVisibility,
StateKey: "",
}
joinRuleTuple := gomatrixserverlib.StateKeyTuple{
EventType: spec.MRoomJoinRules,
StateKey: "",
}
var queryRoomRes roomserver.QueryCurrentStateResponse
err := w.rsAPI.QueryCurrentState(w.ctx, &roomserver.QueryCurrentStateRequest{
RoomID: roomID,
StateTuples: []gomatrixserverlib.StateKeyTuple{
hisVisTuple, joinRuleTuple,
},
}, &queryRoomRes)
if err != nil {
util.GetLogger(w.ctx).WithError(err).Error("failed to QueryCurrentState")
return false
}
hisVisEv := queryRoomRes.StateEvents[hisVisTuple]
if hisVisEv != nil {
hisVis, _ := hisVisEv.HistoryVisibility()
if hisVis == "world_readable" {
return true
}
}
// check if this room is a restricted room and if so, we need to check if the server is joined to an allowed room ID
// in addition to the actual room ID (but always do the actual one first as it's quicker in the common case)
allowJoinedToRoomIDs := []string{roomID}
joinRuleEv := queryRoomRes.StateEvents[joinRuleTuple]
if joinRuleEv != nil {
rule, ruleErr := joinRuleEv.JoinRule()
if ruleErr != nil {
util.GetLogger(w.ctx).WithError(ruleErr).WithField("parent_room_id", roomID).Warn("failed to get join rule")
return false
}
if rule == spec.Public || rule == spec.Knock {
return true
}
if rule == spec.Restricted {
allowJoinedToRoomIDs = append(allowJoinedToRoomIDs, w.restrictedJoinRuleAllowedRooms(joinRuleEv, "m.room_membership")...)
}
}
// check if server is joined to any allowed room
for _, allowedRoomID := range allowJoinedToRoomIDs {
var queryRes fs.QueryJoinedHostServerNamesInRoomResponse
err = w.fsAPI.QueryJoinedHostServerNamesInRoom(w.ctx, &fs.QueryJoinedHostServerNamesInRoomRequest{
RoomID: allowedRoomID,
}, &queryRes)
if err != nil {
util.GetLogger(w.ctx).WithError(err).Error("failed to QueryJoinedHostServerNamesInRoom")
continue
}
for _, srv := range queryRes.ServerNames {
if srv == callerServerName {
return true
}
}
}
return false
}
// authorisedUser returns true iff the user is invited/joined this room or the room is world_readable
// or if the room has a public or knock join rule.
// Failing that, if the room has a restricted join rule and belongs to the space parent listed, it will return true.
func (w *RoomHierarchyWalker) authorisedUser(roomID string, clientCaller *userapi.Device, parentRoomID string) (authed bool, isJoinedOrInvited bool) {
hisVisTuple := gomatrixserverlib.StateKeyTuple{
EventType: spec.MRoomHistoryVisibility,
StateKey: "",
}
joinRuleTuple := gomatrixserverlib.StateKeyTuple{
EventType: spec.MRoomJoinRules,
StateKey: "",
}
roomMemberTuple := gomatrixserverlib.StateKeyTuple{
EventType: spec.MRoomMember,
StateKey: clientCaller.UserID,
}
var queryRes roomserver.QueryCurrentStateResponse
err := w.rsAPI.QueryCurrentState(w.ctx, &roomserver.QueryCurrentStateRequest{
RoomID: roomID,
StateTuples: []gomatrixserverlib.StateKeyTuple{
hisVisTuple, joinRuleTuple, roomMemberTuple,
},
}, &queryRes)
if err != nil {
util.GetLogger(w.ctx).WithError(err).Error("failed to QueryCurrentState")
return false, false
}
memberEv := queryRes.StateEvents[roomMemberTuple]
if memberEv != nil {
membership, _ := memberEv.Membership()
if membership == spec.Join || membership == spec.Invite {
return true, true
}
}
hisVisEv := queryRes.StateEvents[hisVisTuple]
if hisVisEv != nil {
hisVis, _ := hisVisEv.HistoryVisibility()
if hisVis == "world_readable" {
return true, false
}
}
joinRuleEv := queryRes.StateEvents[joinRuleTuple]
if parentRoomID != "" && joinRuleEv != nil {
var allowed bool
rule, ruleErr := joinRuleEv.JoinRule()
if ruleErr != nil {
util.GetLogger(w.ctx).WithError(ruleErr).WithField("parent_room_id", parentRoomID).Warn("failed to get join rule")
} else if rule == spec.Public || rule == spec.Knock {
allowed = true
} else if rule == spec.Restricted {
allowedRoomIDs := w.restrictedJoinRuleAllowedRooms(joinRuleEv, "m.room_membership")
// check parent is in the allowed set
for _, a := range allowedRoomIDs {
if parentRoomID == a {
allowed = true
break
}
}
}
if allowed {
// ensure caller is joined to the parent room
var queryRes2 roomserver.QueryCurrentStateResponse
err = w.rsAPI.QueryCurrentState(w.ctx, &roomserver.QueryCurrentStateRequest{
RoomID: parentRoomID,
StateTuples: []gomatrixserverlib.StateKeyTuple{
roomMemberTuple,
},
}, &queryRes2)
if err != nil {
util.GetLogger(w.ctx).WithError(err).WithField("parent_room_id", parentRoomID).Warn("failed to check user is joined to parent room")
} else {
memberEv = queryRes2.StateEvents[roomMemberTuple]
if memberEv != nil {
membership, _ := memberEv.Membership()
if membership == spec.Join {
return true, false
}
}
}
}
}
return false, false
}
func (w *RoomHierarchyWalker) publicRoomsChunk(roomID string) *fclient.PublicRoom {
pubRooms, err := roomserver.PopulatePublicRooms(w.ctx, []string{roomID}, w.rsAPI)
if err != nil {
util.GetLogger(w.ctx).WithError(err).Error("failed to PopulatePublicRooms")
return nil
}
if len(pubRooms) == 0 {
return nil
}
return &pubRooms[0]
}
type roomVisit struct {
roomID string
parentRoomID string
depth int
vias []string // vias to query this room by
}
func stripped(ev gomatrixserverlib.PDU) *fclient.MSC2946StrippedEvent {
if ev.StateKey() == nil {
return nil
}
return &fclient.MSC2946StrippedEvent{
Type: ev.Type(),
StateKey: *ev.StateKey(),
Content: ev.Content(),
Sender: string(ev.SenderID()),
OriginServerTS: ev.OriginServerTS(),
}
}
func (w *RoomHierarchyWalker) restrictedJoinRuleAllowedRooms(joinRuleEv *types.HeaderedEvent, allowType string) (allows []string) {
rule, _ := joinRuleEv.JoinRule()
if rule != spec.Restricted {
return nil
}
var jrContent gomatrixserverlib.JoinRuleContent
if err := json.Unmarshal(joinRuleEv.Content(), &jrContent); err != nil {
util.GetLogger(w.ctx).Warnf("failed to check join_rule on room %s: %s", joinRuleEv.RoomID(), err)
return nil
}
for _, allow := range jrContent.Allow {
if allow.Type == allowType {
allows = append(allows, allow.RoomID)
}
}
return
}
// Stripped down version of RoomHierarchyWalker suitable for caching (For pagination purposes)
//
// TODO remove more stuff
type CachedRoomHierarchyWalker struct {
rootRoomID string
caller types.DeviceOrServerName
thisServer spec.ServerName
rsAPI *Queryer
fsAPI fs.RoomserverFederationAPI
ctx context.Context
cache caching.RoomHierarchyCache
suggestedOnly bool
maxDepth int
processed stringSet
unvisited []roomVisit
done bool
}
func (c CachedRoomHierarchyWalker) GetWalker() roomserver.RoomHierarchyWalker {
return &RoomHierarchyWalker{
rootRoomID: c.rootRoomID,
caller: c.caller,
thisServer: c.thisServer,
rsAPI: c.rsAPI,
fsAPI: c.fsAPI,
ctx: c.ctx,
roomHierarchyCache: c.cache,
suggestedOnly: c.suggestedOnly,
maxDepth: c.maxDepth,
processed: c.processed,
unvisited: c.unvisited,
done: c.done,
}
}
func (c CachedRoomHierarchyWalker) ValidateParams(suggestedOnly bool, maxDepth int) bool {
return c.suggestedOnly == suggestedOnly && c.maxDepth == maxDepth
}

View file

@ -22,7 +22,9 @@ import (
"strings"
"sync"
userapi "github.com/matrix-org/dendrite/userapi/api"
"github.com/matrix-org/gomatrixserverlib"
"github.com/matrix-org/gomatrixserverlib/spec"
"github.com/matrix-org/util"
"golang.org/x/crypto/blake2b"
)
@ -336,3 +338,36 @@ func (r *RoomInfo) CopyFrom(r2 *RoomInfo) {
}
var ErrorInvalidRoomInfo = fmt.Errorf("room info is invalid")
// Struct to represent a device or a server name.
//
// May be used to designate a caller for functions that can be called
// by a client (device) or by a server (server name).
//
// Exactly 1 of Device() and ServerName() will return a non-nil result.
type DeviceOrServerName struct {
device *userapi.Device
serverName *spec.ServerName
}
func NewDeviceNotServerName(device userapi.Device) DeviceOrServerName {
return DeviceOrServerName{
device: &device,
serverName: nil,
}
}
func NewServerNameNotDevice(serverName spec.ServerName) DeviceOrServerName {
return DeviceOrServerName{
device: nil,
serverName: &serverName,
}
}
func (s *DeviceOrServerName) Device() *userapi.Device {
return s.device
}
func (s *DeviceOrServerName) ServerName() *spec.ServerName {
return s.serverName
}

View file

@ -57,7 +57,7 @@ type MSC2946ClientResponse struct {
// Enable this MSC
func Enable(
cfg *config.Dendrite, routers httputil.Routers, rsAPI roomserver.RoomserverInternalAPI, userAPI userapi.UserInternalAPI,
fsAPI fs.FederationInternalAPI, keyRing gomatrixserverlib.JSONVerifier, cache caching.SpaceSummaryRoomsCache,
fsAPI fs.FederationInternalAPI, keyRing gomatrixserverlib.JSONVerifier, cache caching.RoomHierarchyCache,
) error {
clientAPI := httputil.MakeAuthAPI("spaces", userAPI, spacesHandler(rsAPI, fsAPI, cache, cfg.Global.ServerName), httputil.WithAllowGuests())
routers.Client.Handle("/v1/rooms/{roomID}/hierarchy", clientAPI).Methods(http.MethodGet, http.MethodOptions)
@ -87,7 +87,7 @@ func Enable(
func federatedSpacesHandler(
ctx context.Context, fedReq *fclient.FederationRequest, roomID string,
cache caching.SpaceSummaryRoomsCache,
cache caching.RoomHierarchyCache,
rsAPI roomserver.RoomserverInternalAPI, fsAPI fs.FederationInternalAPI,
thisServer spec.ServerName,
) util.JSONResponse {
@ -122,7 +122,7 @@ func federatedSpacesHandler(
func spacesHandler(
rsAPI roomserver.RoomserverInternalAPI,
fsAPI fs.FederationInternalAPI,
cache caching.SpaceSummaryRoomsCache,
cache caching.RoomHierarchyCache,
thisServer spec.ServerName,
) func(*http.Request, *userapi.Device) util.JSONResponse {
// declared outside the returned handler so it persists between calls
@ -168,7 +168,7 @@ type walker struct {
rsAPI roomserver.RoomserverInternalAPI
fsAPI fs.FederationInternalAPI
ctx context.Context
cache caching.SpaceSummaryRoomsCache
cache caching.RoomHierarchyCache
suggestedOnly bool
limit int
maxDepth int
@ -423,7 +423,7 @@ func (w *walker) federatedRoomInfo(roomID string, vias []string) *fclient.MSC294
if w.caller == nil {
return nil
}
resp, ok := w.cache.GetSpaceSummary(roomID)
resp, ok := w.cache.GetRoomHierarchy(roomID)
if ok {
util.GetLogger(w.ctx).Debugf("Returning cached response for %s", roomID)
return &resp
@ -451,7 +451,7 @@ func (w *walker) federatedRoomInfo(roomID string, vias []string) *fclient.MSC294
}
res.Children[i] = child
}
w.cache.StoreSpaceSummary(roomID, res)
w.cache.StoreRoomHierarchy(roomID, res)
return &res
}