From aa6e5c2ecbe58dec1c38c33949fc70c88f39c242 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Mon, 3 Feb 2025 18:29:15 +0100 Subject: [PATCH 01/65] Add locking to more safely delete state groups: Part 1 (#18107) Currently we don't really have anything that stops us from deleting state groups when an in-flight event references it. This is a fairly rare race currently, but we want to be able to more aggressively delete state groups so it is important to address this to ensure that the database remains valid. This implements the locking, but doesn't actually use it. See the class docstring of the new data store for an explanation for how this works. --------- Co-authored-by: Devon Hudson --- changelog.d/18107.bugfix | 1 + synapse/handlers/federation_event.py | 18 +- synapse/state/__init__.py | 61 ++- synapse/storage/controllers/persist_events.py | 32 +- synapse/storage/databases/__init__.py | 10 +- synapse/storage/databases/state/deletion.py | 446 ++++++++++++++++++ synapse/storage/databases/state/store.py | 39 +- synapse/storage/schema/__init__.py | 5 +- .../delta/89/01_state_groups_deletion.sql | 39 ++ tests/handlers/test_federation_event.py | 9 +- tests/rest/client/test_rooms.py | 4 +- tests/storage/test_state_deletion.py | 411 ++++++++++++++++ tests/test_state.py | 18 +- 13 files changed, 1047 insertions(+), 46 deletions(-) create mode 100644 changelog.d/18107.bugfix create mode 100644 synapse/storage/databases/state/deletion.py create mode 100644 synapse/storage/schema/state/delta/89/01_state_groups_deletion.sql create mode 100644 tests/storage/test_state_deletion.py diff --git a/changelog.d/18107.bugfix b/changelog.d/18107.bugfix new file mode 100644 index 0000000000..4d0c19fab9 --- /dev/null +++ b/changelog.d/18107.bugfix @@ -0,0 +1 @@ +Fix rare edge case where state groups could be deleted while we are persisting new events that reference them. diff --git a/synapse/handlers/federation_event.py b/synapse/handlers/federation_event.py index 1b535ea2cb..1e738f484f 100644 --- a/synapse/handlers/federation_event.py +++ b/synapse/handlers/federation_event.py @@ -151,6 +151,8 @@ class FederationEventHandler: def __init__(self, hs: "HomeServer"): self._clock = hs.get_clock() self._store = hs.get_datastores().main + self._state_store = hs.get_datastores().state + self._state_deletion_store = hs.get_datastores().state_deletion self._storage_controllers = hs.get_storage_controllers() self._state_storage_controller = self._storage_controllers.state @@ -580,7 +582,9 @@ class FederationEventHandler: room_version.identifier, state_maps_to_resolve, event_map=None, - state_res_store=StateResolutionStore(self._store), + state_res_store=StateResolutionStore( + self._store, self._state_deletion_store + ), ) ) else: @@ -1179,7 +1183,9 @@ class FederationEventHandler: room_version, state_maps, event_map={event_id: event}, - state_res_store=StateResolutionStore(self._store), + state_res_store=StateResolutionStore( + self._store, self._state_deletion_store + ), ) except Exception as e: @@ -1874,7 +1880,9 @@ class FederationEventHandler: room_version, [local_state_id_map, claimed_auth_events_id_map], event_map=None, - state_res_store=StateResolutionStore(self._store), + state_res_store=StateResolutionStore( + self._store, self._state_deletion_store + ), ) ) else: @@ -2014,7 +2022,9 @@ class FederationEventHandler: room_version, state_sets, event_map=None, - state_res_store=StateResolutionStore(self._store), + state_res_store=StateResolutionStore( + self._store, self._state_deletion_store + ), ) ) else: diff --git a/synapse/state/__init__.py b/synapse/state/__init__.py index 72b291889b..5b746f2037 100644 --- a/synapse/state/__init__.py +++ b/synapse/state/__init__.py @@ -59,11 +59,13 @@ from synapse.types.state import StateFilter from synapse.util.async_helpers import Linearizer from synapse.util.caches.expiringcache import ExpiringCache from synapse.util.metrics import Measure, measure_func +from synapse.util.stringutils import shortstr if TYPE_CHECKING: from synapse.server import HomeServer from synapse.storage.controllers import StateStorageController from synapse.storage.databases.main import DataStore + from synapse.storage.databases.state.deletion import StateDeletionDataStore logger = logging.getLogger(__name__) metrics_logger = logging.getLogger("synapse.state.metrics") @@ -194,6 +196,8 @@ class StateHandler: self._storage_controllers = hs.get_storage_controllers() self._events_shard_config = hs.config.worker.events_shard_config self._instance_name = hs.get_instance_name() + self._state_store = hs.get_datastores().state + self._state_deletion_store = hs.get_datastores().state_deletion self._update_current_state_client = ( ReplicationUpdateCurrentStateRestServlet.make_client(hs) @@ -475,7 +479,10 @@ class StateHandler: @trace @measure_func() async def resolve_state_groups_for_events( - self, room_id: str, event_ids: StrCollection, await_full_state: bool = True + self, + room_id: str, + event_ids: StrCollection, + await_full_state: bool = True, ) -> _StateCacheEntry: """Given a list of event_ids this method fetches the state at each event, resolves conflicts between them and returns them. @@ -511,6 +518,17 @@ class StateHandler: ) = await self._state_storage_controller.get_state_group_delta( state_group_id ) + + if prev_group: + # Ensure that we still have the prev group, and ensure we don't + # delete it while we're persisting the event. + missing_state_group = await self._state_deletion_store.check_state_groups_and_bump_deletion( + {prev_group} + ) + if missing_state_group: + prev_group = None + delta_ids = None + return _StateCacheEntry( state=None, state_group=state_group_id, @@ -531,7 +549,9 @@ class StateHandler: room_version, state_to_resolve, None, - state_res_store=StateResolutionStore(self.store), + state_res_store=StateResolutionStore( + self.store, self._state_deletion_store + ), ) return result @@ -663,7 +683,25 @@ class StateResolutionHandler: async with self.resolve_linearizer.queue(group_names): cache = self._state_cache.get(group_names, None) if cache: - return cache + # Check that the returned cache entry doesn't point to deleted + # state groups. + state_groups_to_check = set() + if cache.state_group is not None: + state_groups_to_check.add(cache.state_group) + + if cache.prev_group is not None: + state_groups_to_check.add(cache.prev_group) + + missing_state_groups = await state_res_store.state_deletion_store.check_state_groups_and_bump_deletion( + state_groups_to_check + ) + + if not missing_state_groups: + return cache + else: + # There are missing state groups, so let's remove the stale + # entry and continue as if it was a cache miss. + self._state_cache.pop(group_names, None) logger.info( "Resolving state for %s with groups %s", @@ -671,6 +709,16 @@ class StateResolutionHandler: list(group_names), ) + # We double check that none of the state groups have been deleted. + # They shouldn't be as all these state groups should be referenced. + missing_state_groups = await state_res_store.state_deletion_store.check_state_groups_and_bump_deletion( + group_names + ) + if missing_state_groups: + raise Exception( + f"State groups have been deleted: {shortstr(missing_state_groups)}" + ) + state_groups_histogram.observe(len(state_groups_ids)) new_state = await self.resolve_events_with_store( @@ -884,7 +932,8 @@ class StateResolutionStore: in well defined way. """ - store: "DataStore" + main_store: "DataStore" + state_deletion_store: "StateDeletionDataStore" def get_events( self, event_ids: StrCollection, allow_rejected: bool = False @@ -899,7 +948,7 @@ class StateResolutionStore: An awaitable which resolves to a dict from event_id to event. """ - return self.store.get_events( + return self.main_store.get_events( event_ids, redact_behaviour=EventRedactBehaviour.as_is, get_prev_content=False, @@ -920,4 +969,4 @@ class StateResolutionStore: An awaitable that resolves to a set of event IDs. """ - return self.store.get_auth_chain_difference(room_id, state_sets) + return self.main_store.get_auth_chain_difference(room_id, state_sets) diff --git a/synapse/storage/controllers/persist_events.py b/synapse/storage/controllers/persist_events.py index 879ee9039e..7963905479 100644 --- a/synapse/storage/controllers/persist_events.py +++ b/synapse/storage/controllers/persist_events.py @@ -332,6 +332,7 @@ class EventsPersistenceStorageController: # store for now. self.main_store = stores.main self.state_store = stores.state + self._state_deletion_store = stores.state_deletion assert stores.persist_events self.persist_events_store = stores.persist_events @@ -549,7 +550,9 @@ class EventsPersistenceStorageController: room_version, state_maps_by_state_group, event_map=None, - state_res_store=StateResolutionStore(self.main_store), + state_res_store=StateResolutionStore( + self.main_store, self._state_deletion_store + ), ) return await res.get_state(self._state_controller, StateFilter.all()) @@ -635,15 +638,20 @@ class EventsPersistenceStorageController: room_id, [e for e, _ in chunk] ) - await self.persist_events_store._persist_events_and_state_updates( - room_id, - chunk, - state_delta_for_room=state_delta_for_room, - new_forward_extremities=new_forward_extremities, - use_negative_stream_ordering=backfilled, - inhibit_local_membership_updates=backfilled, - new_event_links=new_event_links, - ) + # Stop the state groups from being deleted while we're persisting + # them. + async with self._state_deletion_store.persisting_state_group_references( + events_and_contexts + ): + await self.persist_events_store._persist_events_and_state_updates( + room_id, + chunk, + state_delta_for_room=state_delta_for_room, + new_forward_extremities=new_forward_extremities, + use_negative_stream_ordering=backfilled, + inhibit_local_membership_updates=backfilled, + new_event_links=new_event_links, + ) return replaced_events @@ -965,7 +973,9 @@ class EventsPersistenceStorageController: room_version, state_groups, events_map, - state_res_store=StateResolutionStore(self.main_store), + state_res_store=StateResolutionStore( + self.main_store, self._state_deletion_store + ), ) state_resolutions_during_persistence.inc() diff --git a/synapse/storage/databases/__init__.py b/synapse/storage/databases/__init__.py index dd9fc01fb0..81886ff765 100644 --- a/synapse/storage/databases/__init__.py +++ b/synapse/storage/databases/__init__.py @@ -26,6 +26,7 @@ from synapse.storage._base import SQLBaseStore from synapse.storage.database import DatabasePool, make_conn from synapse.storage.databases.main.events import PersistEventsStore from synapse.storage.databases.state import StateGroupDataStore +from synapse.storage.databases.state.deletion import StateDeletionDataStore from synapse.storage.engines import create_engine from synapse.storage.prepare_database import prepare_database @@ -49,12 +50,14 @@ class Databases(Generic[DataStoreT]): main state persist_events + state_deletion """ databases: List[DatabasePool] main: "DataStore" # FIXME: https://github.com/matrix-org/synapse/issues/11165: actually an instance of `main_store_class` state: StateGroupDataStore persist_events: Optional[PersistEventsStore] + state_deletion: StateDeletionDataStore def __init__(self, main_store_class: Type[DataStoreT], hs: "HomeServer"): # Note we pass in the main store class here as workers use a different main @@ -63,6 +66,7 @@ class Databases(Generic[DataStoreT]): self.databases = [] main: Optional[DataStoreT] = None state: Optional[StateGroupDataStore] = None + state_deletion: Optional[StateDeletionDataStore] = None persist_events: Optional[PersistEventsStore] = None for database_config in hs.config.database.databases: @@ -114,7 +118,8 @@ class Databases(Generic[DataStoreT]): if state: raise Exception("'state' data store already configured") - state = StateGroupDataStore(database, db_conn, hs) + state_deletion = StateDeletionDataStore(database, db_conn, hs) + state = StateGroupDataStore(database, db_conn, hs, state_deletion) db_conn.commit() @@ -135,7 +140,7 @@ class Databases(Generic[DataStoreT]): if not main: raise Exception("No 'main' database configured") - if not state: + if not state or not state_deletion: raise Exception("No 'state' database configured") # We use local variables here to ensure that the databases do not have @@ -143,3 +148,4 @@ class Databases(Generic[DataStoreT]): self.main = main # type: ignore[assignment] self.state = state self.persist_events = persist_events + self.state_deletion = state_deletion diff --git a/synapse/storage/databases/state/deletion.py b/synapse/storage/databases/state/deletion.py new file mode 100644 index 0000000000..07dbbc8e75 --- /dev/null +++ b/synapse/storage/databases/state/deletion.py @@ -0,0 +1,446 @@ +# +# This file is licensed under the Affero General Public License (AGPL) version 3. +# +# Copyright (C) 2025 New Vector, Ltd +# +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as +# published by the Free Software Foundation, either version 3 of the +# License, or (at your option) any later version. +# +# See the GNU Affero General Public License for more details: +# . +# + + +import contextlib +from typing import ( + TYPE_CHECKING, + AbstractSet, + AsyncIterator, + Collection, + Mapping, + Set, + Tuple, +) + +from synapse.events import EventBase +from synapse.events.snapshot import EventContext +from synapse.storage.database import ( + DatabasePool, + LoggingDatabaseConnection, + LoggingTransaction, + make_in_list_sql_clause, +) +from synapse.storage.engines import PostgresEngine +from synapse.util.stringutils import shortstr + +if TYPE_CHECKING: + from synapse.server import HomeServer + + +class StateDeletionDataStore: + """Manages deletion of state groups in a safe manner. + + Deleting state groups is challenging as before we actually delete them we + need to ensure that there are no in-flight events that refer to the state + groups that we want to delete. + + To handle this, we take two approaches. First, before we persist any event + we ensure that the state group still exists and mark in the + `state_groups_persisting` table that the state group is about to be used. + (Note that we have to have the extra table here as state groups and events + can be in different databases, and thus we can't check for the existence of + state groups in the persist event transaction). Once the event has been + persisted, we can remove the row from `state_groups_persisting`. So long as + we check that table before deleting state groups, we can ensure that we + never persist events that reference deleted state groups, maintaining + database integrity. + + However, we want to avoid throwing exceptions so deep in the process of + persisting events. So instead of deleting state groups immediately, we mark + them as pending/proposed for deletion and wait for a certain amount of time + before performing the deletion. When we come to handle new events that + reference state groups, we check if they are pending deletion and bump the + time for when they'll be deleted (to give a chance for the event to be + persisted, or not). + + When deleting, we need to check that state groups remain unreferenced. There + is a race here where we a) fetch state groups that are ready for deletion, + b) check they're unreferenced, c) the state group becomes referenced but + then gets marked as pending deletion again, d) during the deletion + transaction we recheck `state_groups_pending_deletion` table again and see + that it exists and so continue with the deletion. To prevent this from + happening we add a `sequence_number` column to + `state_groups_pending_deletion`, and during deletion we ensure that for a + state group we're about to delete that the sequence number doesn't change + between steps (a) and (d). So long as we always bump the sequence number + whenever an event may become used the race can never happen. + """ + + # How long to wait before we delete state groups. This should be long enough + # for any in-flight events to be persisted. If events take longer to persist + # and any of the state groups they reference have been deleted, then the + # event will fail to persist (as well as any event in the same batch). + DELAY_BEFORE_DELETION_MS = 10 * 60 * 1000 + + def __init__( + self, + database: DatabasePool, + db_conn: LoggingDatabaseConnection, + hs: "HomeServer", + ): + self._clock = hs.get_clock() + self.db_pool = database + self._instance_name = hs.get_instance_name() + + # TODO: Clear from `state_groups_persisting` any holdovers from previous + # running instance. + + async def check_state_groups_and_bump_deletion( + self, state_groups: AbstractSet[int] + ) -> Collection[int]: + """Checks to make sure that the state groups haven't been deleted, and + if they're pending deletion we delay it (allowing time for any event + that will use them to finish persisting). + + Returns: + The state groups that are missing, if any. + """ + + return await self.db_pool.runInteraction( + "check_state_groups_and_bump_deletion", + self._check_state_groups_and_bump_deletion_txn, + state_groups, + ) + + def _check_state_groups_and_bump_deletion_txn( + self, txn: LoggingTransaction, state_groups: AbstractSet[int] + ) -> Collection[int]: + existing_state_groups = self._get_existing_groups_with_lock(txn, state_groups) + + self._bump_deletion_txn(txn, existing_state_groups) + + missing_state_groups = state_groups - existing_state_groups + if missing_state_groups: + return missing_state_groups + + return () + + def _bump_deletion_txn( + self, txn: LoggingTransaction, state_groups: Collection[int] + ) -> None: + """Update any pending deletions of the state group that they may now be + referenced.""" + + if not state_groups: + return + + now = self._clock.time_msec() + if isinstance(self.db_pool.engine, PostgresEngine): + clause, args = make_in_list_sql_clause( + self.db_pool.engine, "state_group", state_groups + ) + sql = f""" + UPDATE state_groups_pending_deletion + SET sequence_number = DEFAULT, insertion_ts = ? + WHERE {clause} + """ + args.insert(0, now) + txn.execute(sql, args) + else: + rows = self.db_pool.simple_select_many_txn( + txn, + table="state_groups_pending_deletion", + column="state_group", + iterable=state_groups, + keyvalues={}, + retcols=("state_group",), + ) + if not rows: + return + + state_groups_to_update = [state_group for (state_group,) in rows] + + self.db_pool.simple_delete_many_txn( + txn, + table="state_groups_pending_deletion", + column="state_group", + values=state_groups_to_update, + keyvalues={}, + ) + self.db_pool.simple_insert_many_txn( + txn, + table="state_groups_pending_deletion", + keys=("state_group", "insertion_ts"), + values=[(state_group, now) for state_group in state_groups_to_update], + ) + + def _get_existing_groups_with_lock( + self, txn: LoggingTransaction, state_groups: Collection[int] + ) -> AbstractSet[int]: + """Return which of the given state groups are in the database, and locks + those rows with `KEY SHARE` to ensure they don't get concurrently + deleted.""" + clause, args = make_in_list_sql_clause(self.db_pool.engine, "id", state_groups) + + sql = f""" + SELECT id FROM state_groups + WHERE {clause} + """ + if isinstance(self.db_pool.engine, PostgresEngine): + # On postgres we add a row level lock to the rows to ensure that we + # conflict with any concurrent DELETEs. `FOR KEY SHARE` lock will + # not conflict with other read + sql += """ + FOR KEY SHARE + """ + + txn.execute(sql, args) + return {state_group for (state_group,) in txn} + + @contextlib.asynccontextmanager + async def persisting_state_group_references( + self, event_and_contexts: Collection[Tuple[EventBase, EventContext]] + ) -> AsyncIterator[None]: + """Wraps the persistence of the given events and contexts, ensuring that + any state groups referenced still exist and that they don't get deleted + during this.""" + + referenced_state_groups: Set[int] = set() + for event, ctx in event_and_contexts: + if ctx.rejected or event.internal_metadata.is_outlier(): + continue + + assert ctx.state_group is not None + + referenced_state_groups.add(ctx.state_group) + + if ctx.state_group_before_event: + referenced_state_groups.add(ctx.state_group_before_event) + + if not referenced_state_groups: + # We don't reference any state groups, so nothing to do + yield + return + + await self.db_pool.runInteraction( + "mark_state_groups_as_persisting", + self._mark_state_groups_as_persisting_txn, + referenced_state_groups, + ) + + error = True + try: + yield None + error = False + finally: + await self.db_pool.runInteraction( + "finish_persisting", + self._finish_persisting_txn, + referenced_state_groups, + error=error, + ) + + def _mark_state_groups_as_persisting_txn( + self, txn: LoggingTransaction, state_groups: Set[int] + ) -> None: + """Marks the given state groups as being persisted.""" + + existing_state_groups = self._get_existing_groups_with_lock(txn, state_groups) + missing_state_groups = state_groups - existing_state_groups + if missing_state_groups: + raise Exception( + f"state groups have been deleted: {shortstr(missing_state_groups)}" + ) + + self.db_pool.simple_insert_many_txn( + txn, + table="state_groups_persisting", + keys=("state_group", "instance_name"), + values=[(state_group, self._instance_name) for state_group in state_groups], + ) + + def _finish_persisting_txn( + self, txn: LoggingTransaction, state_groups: Collection[int], error: bool + ) -> None: + """Mark the state groups as having finished persistence. + + If `error` is true then we assume the state groups were not persisted, + and so we do not clear them from the pending deletion table. + """ + self.db_pool.simple_delete_many_txn( + txn, + table="state_groups_persisting", + column="state_group", + values=state_groups, + keyvalues={"instance_name": self._instance_name}, + ) + + if error: + # The state groups may or may not have been persisted, so we need to + # bump the deletion to ensure we recheck if they have become + # referenced. + self._bump_deletion_txn(txn, state_groups) + return + + self.db_pool.simple_delete_many_batch_txn( + txn, + table="state_groups_pending_deletion", + keys=("state_group",), + values=[(state_group,) for state_group in state_groups], + ) + + async def mark_state_groups_as_pending_deletion( + self, state_groups: Collection[int] + ) -> None: + """Mark the given state groups as pending deletion""" + + now = self._clock.time_msec() + + await self.db_pool.simple_upsert_many( + table="state_groups_pending_deletion", + key_names=("state_group",), + key_values=[(state_group,) for state_group in state_groups], + value_names=("insertion_ts",), + value_values=[(now,) for _ in state_groups], + desc="mark_state_groups_as_pending_deletion", + ) + + async def get_pending_deletions( + self, state_groups: Collection[int] + ) -> Mapping[int, int]: + """Get which state groups are pending deletion. + + Returns: + a mapping from state groups that are pending deletion to their + sequence number + """ + + rows = await self.db_pool.simple_select_many_batch( + table="state_groups_pending_deletion", + column="state_group", + iterable=state_groups, + retcols=("state_group", "sequence_number"), + keyvalues={}, + desc="get_pending_deletions", + ) + + return dict(rows) + + def get_state_groups_ready_for_potential_deletion_txn( + self, + txn: LoggingTransaction, + state_groups_to_sequence_numbers: Mapping[int, int], + ) -> Collection[int]: + """Given a set of state groups, return which state groups can + potentially be deleted. + + The state groups must have been checked to see if they remain + unreferenced before calling this function. + + Note: This must be called within the same transaction that the state + groups are deleted. + + Args: + state_groups_to_sequence_numbers: The state groups, and the sequence + numbers from before the state groups were checked to see if they + were unreferenced. + + Returns: + The subset of state groups that can safely be deleted + + """ + + if not state_groups_to_sequence_numbers: + return state_groups_to_sequence_numbers + + if isinstance(self.db_pool.engine, PostgresEngine): + # On postgres we want to lock the rows FOR UPDATE as early as + # possible to help conflicts. + clause, args = make_in_list_sql_clause( + self.db_pool.engine, "id", state_groups_to_sequence_numbers + ) + sql = f""" + SELECT id FROM state_groups + WHERE {clause} + FOR UPDATE + """ + txn.execute(sql, args) + + # Check the deletion status in the DB of the given state groups + clause, args = make_in_list_sql_clause( + self.db_pool.engine, + column="state_group", + iterable=state_groups_to_sequence_numbers, + ) + + sql = f""" + SELECT state_group, insertion_ts, sequence_number FROM ( + SELECT state_group, insertion_ts, sequence_number FROM state_groups_pending_deletion + UNION + SELECT state_group, null, null FROM state_groups_persisting + ) AS s + WHERE {clause} + """ + + txn.execute(sql, args) + + # The above query will return potentially two rows per state group (one + # for each table), so we track which state groups have enough time + # elapsed and which are not ready to be persisted. + ready_to_be_deleted = set() + not_ready_to_be_deleted = set() + + now = self._clock.time_msec() + for state_group, insertion_ts, sequence_number in txn: + if insertion_ts is None: + # A null insertion_ts means that we are currently persisting + # events that reference the state group, so we don't delete + # them. + not_ready_to_be_deleted.add(state_group) + continue + + # We know this can't be None if insertion_ts is not None + assert sequence_number is not None + + # Check if the sequence number has changed, if it has then it + # indicates that the state group may have become referenced since we + # checked. + if state_groups_to_sequence_numbers[state_group] != sequence_number: + not_ready_to_be_deleted.add(state_group) + continue + + if now - insertion_ts < self.DELAY_BEFORE_DELETION_MS: + # Not enough time has elapsed to allow us to delete. + not_ready_to_be_deleted.add(state_group) + continue + + ready_to_be_deleted.add(state_group) + + can_be_deleted = ready_to_be_deleted - not_ready_to_be_deleted + if not_ready_to_be_deleted: + # If there are any state groups that aren't ready to be deleted, + # then we also need to remove any state groups that are referenced + # by them. + clause, args = make_in_list_sql_clause( + self.db_pool.engine, + column="state_group", + iterable=state_groups_to_sequence_numbers, + ) + sql = f""" + WITH RECURSIVE ancestors(state_group) AS ( + SELECT DISTINCT prev_state_group + FROM state_group_edges WHERE {clause} + UNION + SELECT prev_state_group + FROM state_group_edges + INNER JOIN ancestors USING (state_group) + ) + SELECT state_group FROM ancestors + """ + txn.execute(sql, args) + + can_be_deleted.difference_update(state_group for (state_group,) in txn) + + return can_be_deleted diff --git a/synapse/storage/databases/state/store.py b/synapse/storage/databases/state/store.py index 9944f90015..7e986e0601 100644 --- a/synapse/storage/databases/state/store.py +++ b/synapse/storage/databases/state/store.py @@ -36,7 +36,10 @@ import attr from synapse.api.constants import EventTypes from synapse.events import EventBase -from synapse.events.snapshot import UnpersistedEventContext, UnpersistedEventContextBase +from synapse.events.snapshot import ( + UnpersistedEventContext, + UnpersistedEventContextBase, +) from synapse.logging.opentracing import tag_args, trace from synapse.storage._base import SQLBaseStore from synapse.storage.database import ( @@ -55,6 +58,7 @@ from synapse.util.cancellation import cancellable if TYPE_CHECKING: from synapse.server import HomeServer + from synapse.storage.databases.state.deletion import StateDeletionDataStore logger = logging.getLogger(__name__) @@ -83,8 +87,10 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): database: DatabasePool, db_conn: LoggingDatabaseConnection, hs: "HomeServer", + state_deletion_store: "StateDeletionDataStore", ): super().__init__(database, db_conn, hs) + self._state_deletion_store = state_deletion_store # Originally the state store used a single DictionaryCache to cache the # event IDs for the state types in a given state group to avoid hammering @@ -467,14 +473,15 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): Returns: A list of state groups """ - is_in_db = self.db_pool.simple_select_one_onecol_txn( - txn, - table="state_groups", - keyvalues={"id": prev_group}, - retcol="id", - allow_none=True, + + # We need to check that the prev group isn't about to be deleted + is_missing = ( + self._state_deletion_store._check_state_groups_and_bump_deletion_txn( + txn, + {prev_group}, + ) ) - if not is_in_db: + if is_missing: raise Exception( "Trying to persist state with unpersisted prev_group: %r" % (prev_group,) @@ -546,6 +553,7 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): for key, state_id in context.state_delta_due_to_event.items() ], ) + return events_and_context return await self.db_pool.runInteraction( @@ -601,14 +609,15 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): The state group if successfully created, or None if the state needs to be persisted as a full state. """ - is_in_db = self.db_pool.simple_select_one_onecol_txn( - txn, - table="state_groups", - keyvalues={"id": prev_group}, - retcol="id", - allow_none=True, + + # We need to check that the prev group isn't about to be deleted + is_missing = ( + self._state_deletion_store._check_state_groups_and_bump_deletion_txn( + txn, + {prev_group}, + ) ) - if not is_in_db: + if is_missing: raise Exception( "Trying to persist state with unpersisted prev_group: %r" % (prev_group,) diff --git a/synapse/storage/schema/__init__.py b/synapse/storage/schema/__init__.py index 934e1ccced..49e648a92f 100644 --- a/synapse/storage/schema/__init__.py +++ b/synapse/storage/schema/__init__.py @@ -19,7 +19,7 @@ # # -SCHEMA_VERSION = 88 # remember to update the list below when updating +SCHEMA_VERSION = 89 # remember to update the list below when updating """Represents the expectations made by the codebase about the database schema This should be incremented whenever the codebase changes its requirements on the @@ -155,6 +155,9 @@ Changes in SCHEMA_VERSION = 88 be posted in response to a resettable timeout or an on-demand action. - Add background update to fix data integrity issue in the `sliding_sync_membership_snapshots` -> `forgotten` column + +Changes in SCHEMA_VERSION = 89 + - Add `state_groups_pending_deletion` and `state_groups_persisting` tables. """ diff --git a/synapse/storage/schema/state/delta/89/01_state_groups_deletion.sql b/synapse/storage/schema/state/delta/89/01_state_groups_deletion.sql new file mode 100644 index 0000000000..d4cb27a3a2 --- /dev/null +++ b/synapse/storage/schema/state/delta/89/01_state_groups_deletion.sql @@ -0,0 +1,39 @@ +-- +-- This file is licensed under the Affero General Public License (AGPL) version 3. +-- +-- Copyright (C) 2025 New Vector, Ltd +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- See the GNU Affero General Public License for more details: +-- . + +-- See the `StateDeletionDataStore` for details of these tables. + +-- We add state groups to this table when we want to later delete them. The +-- `insertion_ts` column indicates when the state group was proposed for +-- deletion (rather than when it should be deleted). +CREATE TABLE IF NOT EXISTS state_groups_pending_deletion ( + sequence_number $%AUTO_INCREMENT_PRIMARY_KEY%$, + state_group BIGINT NOT NULL, + insertion_ts BIGINT NOT NULL +); + +CREATE UNIQUE INDEX state_groups_pending_deletion_state_group ON state_groups_pending_deletion(state_group); +CREATE INDEX state_groups_pending_deletion_insertion_ts ON state_groups_pending_deletion(insertion_ts); + + +-- Holds the state groups the worker is currently persisting. +-- +-- The `sequence_number` column of the `state_groups_pending_deletion` table +-- *must* be updated whenever a state group may have become referenced. +CREATE TABLE IF NOT EXISTS state_groups_persisting ( + state_group BIGINT NOT NULL, + instance_name TEXT NOT NULL, + PRIMARY KEY (state_group, instance_name) +); + +CREATE INDEX state_groups_persisting_instance_name ON state_groups_persisting(instance_name); diff --git a/tests/handlers/test_federation_event.py b/tests/handlers/test_federation_event.py index 61b0efb87e..51eca56c3b 100644 --- a/tests/handlers/test_federation_event.py +++ b/tests/handlers/test_federation_event.py @@ -807,6 +807,7 @@ class FederationEventHandlerTests(unittest.FederatingHomeserverTestCase): OTHER_USER = f"@user:{self.OTHER_SERVER_NAME}" main_store = self.hs.get_datastores().main + state_deletion_store = self.hs.get_datastores().state_deletion # Create the room. kermit_user_id = self.register_user("kermit", "test") @@ -958,7 +959,9 @@ class FederationEventHandlerTests(unittest.FederatingHomeserverTestCase): bert_member_event.event_id: bert_member_event, rejected_kick_event.event_id: rejected_kick_event, }, - state_res_store=StateResolutionStore(main_store), + state_res_store=StateResolutionStore( + main_store, state_deletion_store + ), ) ), [bert_member_event.event_id, rejected_kick_event.event_id], @@ -1003,7 +1006,9 @@ class FederationEventHandlerTests(unittest.FederatingHomeserverTestCase): rejected_power_levels_event.event_id, ], event_map={}, - state_res_store=StateResolutionStore(main_store), + state_res_store=StateResolutionStore( + main_store, state_deletion_store + ), full_conflicted_set=set(), ) ), diff --git a/tests/rest/client/test_rooms.py b/tests/rest/client/test_rooms.py index 833bd6fff8..3bb539bf87 100644 --- a/tests/rest/client/test_rooms.py +++ b/tests/rest/client/test_rooms.py @@ -742,7 +742,7 @@ class RoomsCreateTestCase(RoomBase): self.assertEqual(HTTPStatus.OK, channel.code, channel.result) self.assertTrue("room_id" in channel.json_body) assert channel.resource_usage is not None - self.assertEqual(34, channel.resource_usage.db_txn_count) + self.assertEqual(36, channel.resource_usage.db_txn_count) def test_post_room_initial_state(self) -> None: # POST with initial_state config key, expect new room id @@ -755,7 +755,7 @@ class RoomsCreateTestCase(RoomBase): self.assertEqual(HTTPStatus.OK, channel.code, channel.result) self.assertTrue("room_id" in channel.json_body) assert channel.resource_usage is not None - self.assertEqual(36, channel.resource_usage.db_txn_count) + self.assertEqual(38, channel.resource_usage.db_txn_count) def test_post_room_visibility_key(self) -> None: # POST with visibility config key, expect new room id diff --git a/tests/storage/test_state_deletion.py b/tests/storage/test_state_deletion.py new file mode 100644 index 0000000000..19b290b554 --- /dev/null +++ b/tests/storage/test_state_deletion.py @@ -0,0 +1,411 @@ +# +# This file is licensed under the Affero General Public License (AGPL) version 3. +# +# Copyright (C) 2025 New Vector, Ltd +# +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as +# published by the Free Software Foundation, either version 3 of the +# License, or (at your option) any later version. +# +# See the GNU Affero General Public License for more details: +# . +# + + +import logging + +from twisted.test.proto_helpers import MemoryReactor + +from synapse.rest import admin +from synapse.rest.client import login, room +from synapse.server import HomeServer +from synapse.util import Clock + +from tests.test_utils.event_injection import create_event +from tests.unittest import HomeserverTestCase + +logger = logging.getLogger(__name__) + + +class StateDeletionStoreTestCase(HomeserverTestCase): + """Tests for the StateDeletionStore.""" + + servlets = [ + admin.register_servlets, + room.register_servlets, + login.register_servlets, + ] + + def prepare(self, reactor: MemoryReactor, clock: Clock, hs: HomeServer) -> None: + self.store = hs.get_datastores().main + self.state_store = hs.get_datastores().state + self.state_deletion_store = hs.get_datastores().state_deletion + + self.user_id = self.register_user("test", "password") + tok = self.login("test", "password") + self.room_id = self.helper.create_room_as(self.user_id, tok=tok) + + def check_if_can_be_deleted(self, state_group: int) -> bool: + """Check if the state group is pending deletion.""" + + state_group_to_sequence_number = self.get_success( + self.state_deletion_store.get_pending_deletions([state_group]) + ) + + can_be_deleted = self.get_success( + self.state_deletion_store.db_pool.runInteraction( + "test_existing_pending_deletion_is_cleared", + self.state_deletion_store.get_state_groups_ready_for_potential_deletion_txn, + state_group_to_sequence_number, + ) + ) + + return state_group in can_be_deleted + + def test_no_deletion(self) -> None: + """Test that calling persisting_state_group_references is fine if + nothing is pending deletion""" + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + sender=self.user_id, + ) + ) + + ctx_mgr = self.state_deletion_store.persisting_state_group_references( + [(event, context)] + ) + + self.get_success(ctx_mgr.__aenter__()) + self.get_success(ctx_mgr.__aexit__(None, None, None)) + + def test_no_deletion_error(self) -> None: + """Test that calling persisting_state_group_references is fine if + nothing is pending deletion, but an error occurs.""" + + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + sender=self.user_id, + ) + ) + + ctx_mgr = self.state_deletion_store.persisting_state_group_references( + [(event, context)] + ) + + self.get_success(ctx_mgr.__aenter__()) + self.get_success(ctx_mgr.__aexit__(Exception, Exception("test"), None)) + + def test_existing_pending_deletion_is_cleared(self) -> None: + """Test that the pending deletion flag gets cleared when the state group + gets persisted.""" + + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + state_key="", + sender=self.user_id, + ) + ) + assert context.state_group is not None + + # Mark a state group that we're referencing as pending deletion. + self.get_success( + self.state_deletion_store.mark_state_groups_as_pending_deletion( + [context.state_group] + ) + ) + + ctx_mgr = self.state_deletion_store.persisting_state_group_references( + [(event, context)] + ) + + self.get_success(ctx_mgr.__aenter__()) + self.get_success(ctx_mgr.__aexit__(None, None, None)) + + # The pending deletion flag should be cleared + pending_deletion = self.get_success( + self.state_deletion_store.db_pool.simple_select_one_onecol( + table="state_groups_pending_deletion", + keyvalues={"state_group": context.state_group}, + retcol="1", + allow_none=True, + desc="test_existing_pending_deletion_is_cleared", + ) + ) + self.assertIsNone(pending_deletion) + + def test_pending_deletion_is_cleared_during_persist(self) -> None: + """Test that the pending deletion flag is cleared when a state group + gets marked for deletion during persistence""" + + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + state_key="", + sender=self.user_id, + ) + ) + assert context.state_group is not None + + ctx_mgr = self.state_deletion_store.persisting_state_group_references( + [(event, context)] + ) + self.get_success(ctx_mgr.__aenter__()) + + # Mark the state group that we're referencing as pending deletion, + # *after* we have started persisting. + self.get_success( + self.state_deletion_store.mark_state_groups_as_pending_deletion( + [context.state_group] + ) + ) + + self.get_success(ctx_mgr.__aexit__(None, None, None)) + + # The pending deletion flag should be cleared + pending_deletion = self.get_success( + self.state_deletion_store.db_pool.simple_select_one_onecol( + table="state_groups_pending_deletion", + keyvalues={"state_group": context.state_group}, + retcol="1", + allow_none=True, + desc="test_existing_pending_deletion_is_cleared", + ) + ) + self.assertIsNone(pending_deletion) + + def test_deletion_check(self) -> None: + """Test that the `get_state_groups_that_can_be_purged_txn` check is + correct during different points of the lifecycle of persisting an + event.""" + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + state_key="", + sender=self.user_id, + ) + ) + assert context.state_group is not None + + self.get_success( + self.state_deletion_store.mark_state_groups_as_pending_deletion( + [context.state_group] + ) + ) + + # We shouldn't be able to delete the state group as not enough time as passed + can_be_deleted = self.check_if_can_be_deleted(context.state_group) + self.assertFalse(can_be_deleted) + + # After enough time we can delete the state group + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + can_be_deleted = self.check_if_can_be_deleted(context.state_group) + self.assertTrue(can_be_deleted) + + ctx_mgr = self.state_deletion_store.persisting_state_group_references( + [(event, context)] + ) + self.get_success(ctx_mgr.__aenter__()) + + # But once we start persisting we can't delete the state group + can_be_deleted = self.check_if_can_be_deleted(context.state_group) + self.assertFalse(can_be_deleted) + + self.get_success(ctx_mgr.__aexit__(None, None, None)) + + # The pending deletion flag should remain cleared after persistence has + # finished. + can_be_deleted = self.check_if_can_be_deleted(context.state_group) + self.assertFalse(can_be_deleted) + + def test_deletion_error_during_persistence(self) -> None: + """Test that state groups remain marked as pending deletion if persisting + the event fails.""" + + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + state_key="", + sender=self.user_id, + ) + ) + assert context.state_group is not None + + # Mark a state group that we're referencing as pending deletion. + self.get_success( + self.state_deletion_store.mark_state_groups_as_pending_deletion( + [context.state_group] + ) + ) + + ctx_mgr = self.state_deletion_store.persisting_state_group_references( + [(event, context)] + ) + + self.get_success(ctx_mgr.__aenter__()) + self.get_success(ctx_mgr.__aexit__(Exception, Exception("test"), None)) + + # We should be able to delete the state group after a certain amount of + # time + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + can_be_deleted = self.check_if_can_be_deleted(context.state_group) + self.assertTrue(can_be_deleted) + + def test_race_between_check_and_insert(self) -> None: + """Check that we correctly handle the race where we go to delete a + state group, check that it is unreferenced, and then it becomes + referenced just before we delete it.""" + + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + state_key="", + sender=self.user_id, + ) + ) + assert context.state_group is not None + + # Mark a state group that we're referencing as pending deletion. + self.get_success( + self.state_deletion_store.mark_state_groups_as_pending_deletion( + [context.state_group] + ) + ) + + # Advance time enough so we can delete the state group + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + + # Check that we'd be able to delete this state group. + state_group_to_sequence_number = self.get_success( + self.state_deletion_store.get_pending_deletions([context.state_group]) + ) + + can_be_deleted = self.get_success( + self.state_deletion_store.db_pool.runInteraction( + "test_existing_pending_deletion_is_cleared", + self.state_deletion_store.get_state_groups_ready_for_potential_deletion_txn, + state_group_to_sequence_number, + ) + ) + self.assertCountEqual(can_be_deleted, [context.state_group]) + + # ... in the real world we'd check that the state group isn't referenced here ... + + # Now we persist the event to reference the state group, *after* we + # check that the state group wasn't referenced + ctx_mgr = self.state_deletion_store.persisting_state_group_references( + [(event, context)] + ) + + self.get_success(ctx_mgr.__aenter__()) + self.get_success(ctx_mgr.__aexit__(Exception, Exception("test"), None)) + + # We simulate a pause (required to hit the race) + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + + # We should no longer be able to delete the state group, without having + # to recheck if its referenced. + can_be_deleted = self.get_success( + self.state_deletion_store.db_pool.runInteraction( + "test_existing_pending_deletion_is_cleared", + self.state_deletion_store.get_state_groups_ready_for_potential_deletion_txn, + state_group_to_sequence_number, + ) + ) + self.assertCountEqual(can_be_deleted, []) + + def test_remove_ancestors_from_can_delete(self) -> None: + """Test that if a state group is not ready to be deleted, we also don't + delete anything that is refernced by it""" + + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + state_key="", + sender=self.user_id, + ) + ) + assert context.state_group is not None + + # Create a new state group that refernces the one from the event + new_state_group = self.get_success( + self.state_store.store_state_group( + event.event_id, + event.room_id, + prev_group=context.state_group, + delta_ids={}, + current_state_ids=None, + ) + ) + + # Mark them both as pending deletion + self.get_success( + self.state_deletion_store.mark_state_groups_as_pending_deletion( + [context.state_group, new_state_group] + ) + ) + + # Advance time enough so we can delete the state group so they're both + # ready for deletion. + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + + # We can now delete both state groups + self.assertTrue(self.check_if_can_be_deleted(context.state_group)) + self.assertTrue(self.check_if_can_be_deleted(new_state_group)) + + # Use the new_state_group to bump its deletion time + self.get_success( + self.state_store.store_state_group( + event.event_id, + event.room_id, + prev_group=new_state_group, + delta_ids={}, + current_state_ids=None, + ) + ) + + # We should now not be able to delete either of the state groups. + state_group_to_sequence_number = self.get_success( + self.state_deletion_store.get_pending_deletions( + [context.state_group, new_state_group] + ) + ) + + # We shouldn't be able to delete the state group as not enough time has passed + can_be_deleted = self.get_success( + self.state_deletion_store.db_pool.runInteraction( + "test_existing_pending_deletion_is_cleared", + self.state_deletion_store.get_state_groups_ready_for_potential_deletion_txn, + state_group_to_sequence_number, + ) + ) + self.assertCountEqual(can_be_deleted, []) diff --git a/tests/test_state.py b/tests/test_state.py index 311a590693..dce56fe78a 100644 --- a/tests/test_state.py +++ b/tests/test_state.py @@ -31,7 +31,7 @@ from typing import ( Tuple, cast, ) -from unittest.mock import Mock +from unittest.mock import AsyncMock, Mock from twisted.internet import defer @@ -221,7 +221,16 @@ class Graph: class StateTestCase(unittest.TestCase): def setUp(self) -> None: self.dummy_store = _DummyStore() - storage_controllers = Mock(main=self.dummy_store, state=self.dummy_store) + + # Add a dummy epoch store that always retruns that we have all the + # necessary state groups. + dummy_deletion_store = AsyncMock() + dummy_deletion_store.check_state_groups_and_bump_deletion.return_value = [] + + storage_controllers = Mock( + main=self.dummy_store, + state=self.dummy_store, + ) hs = Mock( spec_set=[ "config", @@ -241,7 +250,10 @@ class StateTestCase(unittest.TestCase): ) clock = cast(Clock, MockClock()) hs.config = default_config("tesths", True) - hs.get_datastores.return_value = Mock(main=self.dummy_store) + hs.get_datastores.return_value = Mock( + main=self.dummy_store, + state_deletion=dummy_deletion_store, + ) hs.get_state_handler.return_value = None hs.get_clock.return_value = clock hs.get_macaroon_generator.return_value = MacaroonGenerator( From 27dbb1b4290b9de64e24a11f892777378810b595 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Mon, 3 Feb 2025 18:58:55 +0100 Subject: [PATCH 02/65] Add locking to more safely delete state groups: Part 2 (#18130) This actually makes it so that deleting state groups goes via the new mechanism. c.f. #18107 --- changelog.d/18130.bugfix | 1 + synapse/storage/controllers/purge_events.py | 80 +++++++++++++++++++-- synapse/storage/databases/state/deletion.py | 65 +++++++++++++++++ synapse/storage/databases/state/store.py | 28 ++++++-- tests/rest/client/utils.py | 2 +- tests/storage/test_purge.py | 67 +++++++++++++++++ tests/storage/test_state_deletion.py | 68 +++++++++++++++++- 7 files changed, 297 insertions(+), 14 deletions(-) create mode 100644 changelog.d/18130.bugfix diff --git a/changelog.d/18130.bugfix b/changelog.d/18130.bugfix new file mode 100644 index 0000000000..4d0c19fab9 --- /dev/null +++ b/changelog.d/18130.bugfix @@ -0,0 +1 @@ +Fix rare edge case where state groups could be deleted while we are persisting new events that reference them. diff --git a/synapse/storage/controllers/purge_events.py b/synapse/storage/controllers/purge_events.py index 15c04ffef8..2d6f80f770 100644 --- a/synapse/storage/controllers/purge_events.py +++ b/synapse/storage/controllers/purge_events.py @@ -21,9 +21,10 @@ import itertools import logging -from typing import TYPE_CHECKING, Set +from typing import TYPE_CHECKING, Collection, Mapping, Set from synapse.logging.context import nested_logging_context +from synapse.metrics.background_process_metrics import wrap_as_background_process from synapse.storage.databases import Databases if TYPE_CHECKING: @@ -38,6 +39,11 @@ class PurgeEventsStorageController: def __init__(self, hs: "HomeServer", stores: Databases): self.stores = stores + if hs.config.worker.run_background_tasks: + self._delete_state_loop_call = hs.get_clock().looping_call( + self._delete_state_groups_loop, 60 * 1000 + ) + async def purge_room(self, room_id: str) -> None: """Deletes all record of a room""" @@ -68,11 +74,15 @@ class PurgeEventsStorageController: logger.info("[purge] finding state groups that can be deleted") sg_to_delete = await self._find_unreferenced_groups(state_groups) - await self.stores.state.purge_unreferenced_state_groups( - room_id, sg_to_delete + # Mark these state groups as pending deletion, they will actually + # get deleted automatically later. + await self.stores.state_deletion.mark_state_groups_as_pending_deletion( + sg_to_delete ) - async def _find_unreferenced_groups(self, state_groups: Set[int]) -> Set[int]: + async def _find_unreferenced_groups( + self, state_groups: Collection[int] + ) -> Set[int]: """Used when purging history to figure out which state groups can be deleted. @@ -121,3 +131,65 @@ class PurgeEventsStorageController: to_delete = state_groups_seen - referenced_groups return to_delete + + @wrap_as_background_process("_delete_state_groups_loop") + async def _delete_state_groups_loop(self) -> None: + """Background task that deletes any state groups that may be pending + deletion.""" + + while True: + next_to_delete = await self.stores.state_deletion.get_next_state_group_collection_to_delete() + if next_to_delete is None: + break + + (room_id, groups_to_sequences) = next_to_delete + made_progress = await self._delete_state_groups( + room_id, groups_to_sequences + ) + + # If no progress was made in deleting the state groups, then we + # break to allow a pause before trying again next time we get + # called. + if not made_progress: + break + + async def _delete_state_groups( + self, room_id: str, groups_to_sequences: Mapping[int, int] + ) -> bool: + """Tries to delete the given state groups. + + Returns: + Whether we made progress in deleting the state groups (or marking + them as referenced). + """ + + # We double check if any of the state groups have become referenced. + # This shouldn't happen, as any usages should cause the state group to + # be removed as pending deletion. + referenced_state_groups = await self.stores.main.get_referenced_state_groups( + groups_to_sequences + ) + + if referenced_state_groups: + # We mark any state groups that have become referenced as being + # used. + await self.stores.state_deletion.mark_state_groups_as_used( + referenced_state_groups + ) + + # Update list of state groups to remove referenced ones + groups_to_sequences = { + state_group: sequence_number + for state_group, sequence_number in groups_to_sequences.items() + if state_group not in referenced_state_groups + } + + if not groups_to_sequences: + # We made progress here as long as we marked some state groups as + # now referenced. + return len(referenced_state_groups) > 0 + + return await self.stores.state.purge_unreferenced_state_groups( + room_id, + groups_to_sequences, + ) diff --git a/synapse/storage/databases/state/deletion.py b/synapse/storage/databases/state/deletion.py index 07dbbc8e75..4853e5aa2f 100644 --- a/synapse/storage/databases/state/deletion.py +++ b/synapse/storage/databases/state/deletion.py @@ -20,6 +20,7 @@ from typing import ( AsyncIterator, Collection, Mapping, + Optional, Set, Tuple, ) @@ -307,6 +308,17 @@ class StateDeletionDataStore: desc="mark_state_groups_as_pending_deletion", ) + async def mark_state_groups_as_used(self, state_groups: Collection[int]) -> None: + """Mark the given state groups as now being referenced""" + + await self.db_pool.simple_delete_many( + table="state_groups_pending_deletion", + column="state_group", + iterable=state_groups, + keyvalues={}, + desc="mark_state_groups_as_used", + ) + async def get_pending_deletions( self, state_groups: Collection[int] ) -> Mapping[int, int]: @@ -444,3 +456,56 @@ class StateDeletionDataStore: can_be_deleted.difference_update(state_group for (state_group,) in txn) return can_be_deleted + + async def get_next_state_group_collection_to_delete( + self, + ) -> Optional[Tuple[str, Mapping[int, int]]]: + """Get the next set of state groups to try and delete + + Returns: + 2-tuple of room_id and mapping of state groups to sequence number. + """ + return await self.db_pool.runInteraction( + "get_next_state_group_collection_to_delete", + self._get_next_state_group_collection_to_delete_txn, + ) + + def _get_next_state_group_collection_to_delete_txn( + self, + txn: LoggingTransaction, + ) -> Optional[Tuple[str, Mapping[int, int]]]: + """Implementation of `get_next_state_group_collection_to_delete`""" + + # We want to return chunks of state groups that were marked for deletion + # at the same time (this isn't necessary, just more efficient). We do + # this by looking for the oldest insertion_ts, and then pulling out all + # rows that have the same insertion_ts (and room ID). + now = self._clock.time_msec() + + sql = """ + SELECT room_id, insertion_ts + FROM state_groups_pending_deletion AS sd + INNER JOIN state_groups AS sg ON (id = sd.state_group) + LEFT JOIN state_groups_persisting AS sp USING (state_group) + WHERE insertion_ts < ? AND sp.state_group IS NULL + ORDER BY insertion_ts + LIMIT 1 + """ + txn.execute(sql, (now - self.DELAY_BEFORE_DELETION_MS,)) + row = txn.fetchone() + if not row: + return None + + (room_id, insertion_ts) = row + + sql = """ + SELECT state_group, sequence_number + FROM state_groups_pending_deletion AS sd + INNER JOIN state_groups AS sg ON (id = sd.state_group) + LEFT JOIN state_groups_persisting AS sp USING (state_group) + WHERE room_id = ? AND insertion_ts = ? AND sp.state_group IS NULL + ORDER BY insertion_ts + """ + txn.execute(sql, (room_id, insertion_ts)) + + return room_id, dict(txn) diff --git a/synapse/storage/databases/state/store.py b/synapse/storage/databases/state/store.py index 7e986e0601..0f47642ae5 100644 --- a/synapse/storage/databases/state/store.py +++ b/synapse/storage/databases/state/store.py @@ -22,10 +22,10 @@ import logging from typing import ( TYPE_CHECKING, - Collection, Dict, Iterable, List, + Mapping, Optional, Set, Tuple, @@ -735,8 +735,10 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): ) async def purge_unreferenced_state_groups( - self, room_id: str, state_groups_to_delete: Collection[int] - ) -> None: + self, + room_id: str, + state_groups_to_sequence_numbers: Mapping[int, int], + ) -> bool: """Deletes no longer referenced state groups and de-deltas any state groups that reference them. @@ -744,21 +746,31 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): room_id: The room the state groups belong to (must all be in the same room). state_groups_to_delete: Set of all state groups to delete. + + Returns: + Whether any state groups were actually deleted. """ - await self.db_pool.runInteraction( + return await self.db_pool.runInteraction( "purge_unreferenced_state_groups", self._purge_unreferenced_state_groups, room_id, - state_groups_to_delete, + state_groups_to_sequence_numbers, ) def _purge_unreferenced_state_groups( self, txn: LoggingTransaction, room_id: str, - state_groups_to_delete: Collection[int], - ) -> None: + state_groups_to_sequence_numbers: Mapping[int, int], + ) -> bool: + state_groups_to_delete = self._state_deletion_store.get_state_groups_ready_for_potential_deletion_txn( + txn, state_groups_to_sequence_numbers + ) + + if not state_groups_to_delete: + return False + logger.info( "[purge] found %i state groups to delete", len(state_groups_to_delete) ) @@ -821,6 +833,8 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): [(sg,) for sg in state_groups_to_delete], ) + return True + @trace @tag_args async def get_previous_state_groups( diff --git a/tests/rest/client/utils.py b/tests/rest/client/utils.py index dbd6049f9f..e766630afb 100644 --- a/tests/rest/client/utils.py +++ b/tests/rest/client/utils.py @@ -548,7 +548,7 @@ class RestHelper: room_id: str, event_type: str, body: Dict[str, Any], - tok: Optional[str], + tok: Optional[str] = None, expect_code: int = HTTPStatus.OK, state_key: str = "", ) -> JsonDict: diff --git a/tests/storage/test_purge.py b/tests/storage/test_purge.py index 080d5640a5..efd8d25bd1 100644 --- a/tests/storage/test_purge.py +++ b/tests/storage/test_purge.py @@ -23,6 +23,7 @@ from twisted.test.proto_helpers import MemoryReactor from synapse.api.errors import NotFoundError, SynapseError from synapse.rest.client import room from synapse.server import HomeServer +from synapse.types.state import StateFilter from synapse.util import Clock from tests.unittest import HomeserverTestCase @@ -40,6 +41,8 @@ class PurgeTests(HomeserverTestCase): self.room_id = self.helper.create_room_as(self.user_id) self.store = hs.get_datastores().main + self.state_store = hs.get_datastores().state + self.state_deletion_store = hs.get_datastores().state_deletion self._storage_controllers = self.hs.get_storage_controllers() def test_purge_history(self) -> None: @@ -128,3 +131,67 @@ class PurgeTests(HomeserverTestCase): self.store._invalidate_local_get_event_cache(create_event.event_id) self.get_failure(self.store.get_event(create_event.event_id), NotFoundError) self.get_failure(self.store.get_event(first["event_id"]), NotFoundError) + + def test_purge_history_deletes_state_groups(self) -> None: + """Test that unreferenced state groups get cleaned up after purge""" + + # Send four state changes to the room. + first = self.helper.send_state( + self.room_id, event_type="m.foo", body={"test": 1} + ) + second = self.helper.send_state( + self.room_id, event_type="m.foo", body={"test": 2} + ) + third = self.helper.send_state( + self.room_id, event_type="m.foo", body={"test": 3} + ) + last = self.helper.send_state( + self.room_id, event_type="m.foo", body={"test": 4} + ) + + # Get references to the state groups + event_to_groups = self.get_success( + self.store._get_state_group_for_events( + [ + first["event_id"], + second["event_id"], + third["event_id"], + last["event_id"], + ] + ) + ) + + # Get the topological token + token = self.get_success( + self.store.get_topological_token_for_event(last["event_id"]) + ) + token_str = self.get_success(token.to_string(self.hs.get_datastores().main)) + + # Purge everything before this topological token + self.get_success( + self._storage_controllers.purge_events.purge_history( + self.room_id, token_str, True + ) + ) + + # Advance so that the background jobs to delete the state groups runs + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + + # We expect all the state groups associated with events above, except + # the last one, should return no state. + state_groups = self.get_success( + self.state_store._get_state_groups_from_groups( + list(event_to_groups.values()), StateFilter.all() + ) + ) + first_state = state_groups[event_to_groups[first["event_id"]]] + second_state = state_groups[event_to_groups[second["event_id"]]] + third_state = state_groups[event_to_groups[third["event_id"]]] + last_state = state_groups[event_to_groups[last["event_id"]]] + + self.assertEqual(first_state, {}) + self.assertEqual(second_state, {}) + self.assertEqual(third_state, {}) + self.assertNotEqual(last_state, {}) diff --git a/tests/storage/test_state_deletion.py b/tests/storage/test_state_deletion.py index 19b290b554..a4d318ae20 100644 --- a/tests/storage/test_state_deletion.py +++ b/tests/storage/test_state_deletion.py @@ -41,6 +41,11 @@ class StateDeletionStoreTestCase(HomeserverTestCase): self.store = hs.get_datastores().main self.state_store = hs.get_datastores().state self.state_deletion_store = hs.get_datastores().state_deletion + self.purge_events = hs.get_storage_controllers().purge_events + + # We want to disable the automatic deletion of state groups in the + # background, so we can do controlled tests. + self.purge_events._delete_state_loop_call.stop() self.user_id = self.register_user("test", "password") tok = self.login("test", "password") @@ -341,7 +346,7 @@ class StateDeletionStoreTestCase(HomeserverTestCase): def test_remove_ancestors_from_can_delete(self) -> None: """Test that if a state group is not ready to be deleted, we also don't - delete anything that is refernced by it""" + delete anything that is referenced by it""" event, context = self.get_success( create_event( @@ -354,7 +359,7 @@ class StateDeletionStoreTestCase(HomeserverTestCase): ) assert context.state_group is not None - # Create a new state group that refernces the one from the event + # Create a new state group that references the one from the event new_state_group = self.get_success( self.state_store.store_state_group( event.event_id, @@ -409,3 +414,62 @@ class StateDeletionStoreTestCase(HomeserverTestCase): ) ) self.assertCountEqual(can_be_deleted, []) + + def test_newly_referenced_state_group_gets_removed_from_pending(self) -> None: + """Check that if a state group marked for deletion becomes referenced + (without being removed from pending deletion table), it gets removed + from pending deletion table.""" + + event, context = self.get_success( + create_event( + self.hs, + room_id=self.room_id, + type="m.test", + state_key="", + sender=self.user_id, + ) + ) + assert context.state_group is not None + + # Mark a state group that we're referencing as pending deletion. + self.get_success( + self.state_deletion_store.mark_state_groups_as_pending_deletion( + [context.state_group] + ) + ) + + # Advance time enough so we can delete the state group so they're both + # ready for deletion. + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + + # Manually insert into the table to mimic the state group getting used. + self.get_success( + self.store.db_pool.simple_insert( + table="event_to_state_groups", + values={"state_group": context.state_group, "event_id": event.event_id}, + desc="test_newly_referenced_state_group_gets_removed_from_pending", + ) + ) + + # Manually run the background task to delete pending state groups. + self.get_success(self.purge_events._delete_state_groups_loop()) + + # The pending deletion flag should be cleared... + pending_deletion = self.get_success( + self.state_deletion_store.db_pool.simple_select_one_onecol( + table="state_groups_pending_deletion", + keyvalues={"state_group": context.state_group}, + retcol="1", + allow_none=True, + desc="test_newly_referenced_state_group_gets_removed_from_pending", + ) + ) + self.assertIsNone(pending_deletion) + + # .. but the state should not have been deleted. + state = self.get_success( + self.state_store._get_state_for_groups([context.state_group]) + ) + self.assertGreater(len(state[context.state_group]), 0) From c46d452c7cdf7289ae7e8677b44a88c747761dce Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Mon, 3 Feb 2025 20:04:19 +0100 Subject: [PATCH 03/65] Fix bug where purging history could lead to increase in disk space usage (#18131) When purging history, we try and delete any state groups that become unreferenced (i.e. there are no longer any events that directly reference them). When we delete a state group that is referenced by another state group, we "de-delta" that state group so that it no longer refers to the state group that is deleted. There are two bugs with this approach that we fix here: 1. There is a common pattern where we end up storing two state groups when persisting a state event: the state before and after the new state event, where the latter is stored as a delta to the former. When deleting state groups we only deleted the "new" state and left (and potentially de-deltaed) the old state. This was due to a bug/typo when trying to find referenced state groups. 2. There are times where we store unreferenced state groups in the DB, during the purging of history these would not get rechecked and instead always de-deltaed. Instead, we should check for this case and delete any unreferenced state groups rather than de-deltaing them. The effect of the above bugs is that when purging history we'd end up with lots of unreferenced state groups that had been de-deltaed (i.e. stored as the full state). This can lead to dramatic increases in storage space used. --- changelog.d/18131.bugfix | 1 + synapse/storage/controllers/purge_events.py | 10 +++ synapse/storage/databases/state/store.py | 31 ++++++++- tests/storage/test_purge.py | 75 +++++++++++++++++++++ 4 files changed, 116 insertions(+), 1 deletion(-) create mode 100644 changelog.d/18131.bugfix diff --git a/changelog.d/18131.bugfix b/changelog.d/18131.bugfix new file mode 100644 index 0000000000..4d0c19fab9 --- /dev/null +++ b/changelog.d/18131.bugfix @@ -0,0 +1 @@ +Fix rare edge case where state groups could be deleted while we are persisting new events that reference them. diff --git a/synapse/storage/controllers/purge_events.py b/synapse/storage/controllers/purge_events.py index 2d6f80f770..47cec8c469 100644 --- a/synapse/storage/controllers/purge_events.py +++ b/synapse/storage/controllers/purge_events.py @@ -128,6 +128,16 @@ class PurgeEventsStorageController: next_to_search |= prevs state_groups_seen |= prevs + # We also check to see if anything referencing the state groups are + # also unreferenced. This helps ensure that we delete unreferenced + # state groups, if we don't then we will de-delta them when we + # delete the other state groups leading to increased DB usage. + next_edges = await self.stores.state.get_next_state_groups(current_search) + nexts = set(next_edges.keys()) + nexts -= state_groups_seen + next_to_search |= nexts + state_groups_seen |= nexts + to_delete = state_groups_seen - referenced_groups return to_delete diff --git a/synapse/storage/databases/state/store.py b/synapse/storage/databases/state/store.py index 0f47642ae5..8c7980e719 100644 --- a/synapse/storage/databases/state/store.py +++ b/synapse/storage/databases/state/store.py @@ -853,7 +853,7 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): List[Tuple[int, int]], await self.db_pool.simple_select_many_batch( table="state_group_edges", - column="prev_state_group", + column="state_group", iterable=state_groups, keyvalues={}, retcols=("state_group", "prev_state_group"), @@ -863,6 +863,35 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): return dict(rows) + @trace + @tag_args + async def get_next_state_groups( + self, state_groups: Iterable[int] + ) -> Dict[int, int]: + """Fetch the groups that have the given state groups as their previous + state groups. + + Args: + state_groups + + Returns: + A mapping from state group to previous state group. + """ + + rows = cast( + List[Tuple[int, int]], + await self.db_pool.simple_select_many_batch( + table="state_group_edges", + column="prev_state_group", + iterable=state_groups, + keyvalues={}, + retcols=("state_group", "prev_state_group"), + desc="get_next_state_groups", + ), + ) + + return dict(rows) + async def purge_room_state(self, room_id: str) -> None: return await self.db_pool.runInteraction( "purge_room_state", diff --git a/tests/storage/test_purge.py b/tests/storage/test_purge.py index efd8d25bd1..5d6a8518c0 100644 --- a/tests/storage/test_purge.py +++ b/tests/storage/test_purge.py @@ -195,3 +195,78 @@ class PurgeTests(HomeserverTestCase): self.assertEqual(second_state, {}) self.assertEqual(third_state, {}) self.assertNotEqual(last_state, {}) + + def test_purge_unreferenced_state_group(self) -> None: + """Test that purging a room also gets rid of unreferenced state groups + it encounters during the purge. + + This is important, as otherwise these unreferenced state groups get + "de-deltaed" during the purge process, consuming lots of disk space. + """ + + self.helper.send(self.room_id, body="test1") + state1 = self.helper.send_state( + self.room_id, "org.matrix.test", body={"number": 2} + ) + state2 = self.helper.send_state( + self.room_id, "org.matrix.test", body={"number": 3} + ) + self.helper.send(self.room_id, body="test4") + last = self.helper.send(self.room_id, body="test5") + + # Create an unreferenced state group that has a prev group of one of the + # to-be-purged events. + prev_group = self.get_success( + self.store._get_state_group_for_event(state1["event_id"]) + ) + unreferenced_state_group = self.get_success( + self.state_store.store_state_group( + event_id=last["event_id"], + room_id=self.room_id, + prev_group=prev_group, + delta_ids={("org.matrix.test", ""): state2["event_id"]}, + current_state_ids=None, + ) + ) + + # Get the topological token + token = self.get_success( + self.store.get_topological_token_for_event(last["event_id"]) + ) + token_str = self.get_success(token.to_string(self.hs.get_datastores().main)) + + # Purge everything before this topological token + self.get_success( + self._storage_controllers.purge_events.purge_history( + self.room_id, token_str, True + ) + ) + + # Advance so that the background jobs to delete the state groups runs + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + + # We expect that the unreferenced state group has been deleted. + row = self.get_success( + self.state_store.db_pool.simple_select_one_onecol( + table="state_groups", + keyvalues={"id": unreferenced_state_group}, + retcol="id", + allow_none=True, + desc="test_purge_unreferenced_state_group", + ) + ) + self.assertIsNone(row) + + # We expect there to now only be one state group for the room, which is + # the state group of the last event (as the only outlier). + state_groups = self.get_success( + self.state_store.db_pool.simple_select_onecol( + table="state_groups", + keyvalues={"room_id": self.room_id}, + retcol="id", + desc="test_purge_unreferenced_state_group", + ) + ) + self.assertEqual(len(state_groups), 1) From 37e893499f54872903c365aacc86a9d3eb6b7619 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Tue, 4 Feb 2025 11:53:27 +0000 Subject: [PATCH 04/65] 1.124.0rc1 --- CHANGES.md | 27 +++++++++++++++++++++++++++ changelog.d/18000.bugfix | 1 - changelog.d/18073.bugfix | 1 - changelog.d/18075.bugfix | 1 - changelog.d/18089.bugfix | 2 -- changelog.d/18107.bugfix | 1 - changelog.d/18109.misc | 1 - changelog.d/18112.bugfix | 1 - changelog.d/18119.bugfix | 1 - changelog.d/18130.bugfix | 1 - changelog.d/18131.bugfix | 1 - debian/changelog | 6 ++++++ pyproject.toml | 2 +- 13 files changed, 34 insertions(+), 12 deletions(-) delete mode 100644 changelog.d/18000.bugfix delete mode 100644 changelog.d/18073.bugfix delete mode 100644 changelog.d/18075.bugfix delete mode 100644 changelog.d/18089.bugfix delete mode 100644 changelog.d/18107.bugfix delete mode 100644 changelog.d/18109.misc delete mode 100644 changelog.d/18112.bugfix delete mode 100644 changelog.d/18119.bugfix delete mode 100644 changelog.d/18130.bugfix delete mode 100644 changelog.d/18131.bugfix diff --git a/CHANGES.md b/CHANGES.md index cc6426751d..051ab130b2 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,30 @@ +# Synapse 1.124.0rc1 (2025-02-04) + +### Bugfixes + +- Add rate limit `rc_presence.per_user`. This prevents load from excessive presence updates sent by clients via sync api. Also rate limit `/_matrix/client/v3/presence` as per the spec. Contributed by @rda0. ([\#18000](https://github.com/element-hq/synapse/issues/18000)) +- Deactivated users will no longer automatically accept an invite when `auto_accept_invites` is enabled. ([\#18073](https://github.com/element-hq/synapse/issues/18073)) +- Fix join being denied after being invited over federation. Also fixes other out-of-band membership transitions. ([\#18075](https://github.com/element-hq/synapse/issues/18075)) +- Updates contributed `docker-compose.yml` file to PostgreSQL v15, as v12 is no longer supported by Synapse. + Contributed by @maxkratz. ([\#18089](https://github.com/element-hq/synapse/issues/18089)) +- Fix rare edge case where state groups could be deleted while we are persisting new events that reference them. ([\#18107](https://github.com/element-hq/synapse/issues/18107), [\#18130](https://github.com/element-hq/synapse/issues/18130), [\#18131](https://github.com/element-hq/synapse/issues/18131)) +- Raise an error if someone is using an incorrect suffix in a config duration string. ([\#18112](https://github.com/element-hq/synapse/issues/18112)) +- Fix a bug where the [Delete Room Admin API](https://element-hq.github.io/synapse/latest/admin_api/rooms.html#version-2-new-version) would fail if the `block` parameter was set to `true` and a worker other than the main process was configured to handle background tasks. ([\#18119](https://github.com/element-hq/synapse/issues/18119)) + +### Internal Changes + +- Increase the length of the generated `nonce` parameter when perfoming OIDC logins to comply with the TI-Messenger spec. ([\#18109](https://github.com/element-hq/synapse/issues/18109)) + + + +### Updates to locked dependencies + +* Bump dawidd6/action-download-artifact from 7 to 8. ([\#18108](https://github.com/element-hq/synapse/issues/18108)) +* Bump log from 0.4.22 to 0.4.25. ([\#18098](https://github.com/element-hq/synapse/issues/18098)) +* Bump python-multipart from 0.0.18 to 0.0.20. ([\#18096](https://github.com/element-hq/synapse/issues/18096)) +* Bump serde_json from 1.0.135 to 1.0.137. ([\#18099](https://github.com/element-hq/synapse/issues/18099)) +* Bump types-bleach from 6.1.0.20240331 to 6.2.0.20241123. ([\#18082](https://github.com/element-hq/synapse/issues/18082)) + # Synapse 1.123.0 (2025-01-28) No significant changes since 1.123.0rc1. diff --git a/changelog.d/18000.bugfix b/changelog.d/18000.bugfix deleted file mode 100644 index a8f1545bf5..0000000000 --- a/changelog.d/18000.bugfix +++ /dev/null @@ -1 +0,0 @@ -Add rate limit `rc_presence.per_user`. This prevents load from excessive presence updates sent by clients via sync api. Also rate limit `/_matrix/client/v3/presence` as per the spec. Contributed by @rda0. diff --git a/changelog.d/18073.bugfix b/changelog.d/18073.bugfix deleted file mode 100644 index eeb56a7a61..0000000000 --- a/changelog.d/18073.bugfix +++ /dev/null @@ -1 +0,0 @@ -Deactivated users will no longer automatically accept an invite when `auto_accept_invites` is enabled. \ No newline at end of file diff --git a/changelog.d/18075.bugfix b/changelog.d/18075.bugfix deleted file mode 100644 index 95b486bed1..0000000000 --- a/changelog.d/18075.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix join being denied after being invited over federation. Also fixes other out-of-band membership transitions. diff --git a/changelog.d/18089.bugfix b/changelog.d/18089.bugfix deleted file mode 100644 index 607fab7112..0000000000 --- a/changelog.d/18089.bugfix +++ /dev/null @@ -1,2 +0,0 @@ -Updates contributed `docker-compose.yml` file to PostgreSQL v15, as v12 is no longer supported by Synapse. -Contributed by @maxkratz. \ No newline at end of file diff --git a/changelog.d/18107.bugfix b/changelog.d/18107.bugfix deleted file mode 100644 index 4d0c19fab9..0000000000 --- a/changelog.d/18107.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix rare edge case where state groups could be deleted while we are persisting new events that reference them. diff --git a/changelog.d/18109.misc b/changelog.d/18109.misc deleted file mode 100644 index c310e76f78..0000000000 --- a/changelog.d/18109.misc +++ /dev/null @@ -1 +0,0 @@ -Increase the length of the generated `nonce` parameter when perfoming OIDC logins to comply with the TI-Messenger spec. \ No newline at end of file diff --git a/changelog.d/18112.bugfix b/changelog.d/18112.bugfix deleted file mode 100644 index 61c94280d8..0000000000 --- a/changelog.d/18112.bugfix +++ /dev/null @@ -1 +0,0 @@ -Raise an error if someone is using an incorrect suffix in a config duration string. diff --git a/changelog.d/18119.bugfix b/changelog.d/18119.bugfix deleted file mode 100644 index c8ac53f9d4..0000000000 --- a/changelog.d/18119.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix a bug where the [Delete Room Admin API](https://element-hq.github.io/synapse/latest/admin_api/rooms.html#version-2-new-version) would fail if the `block` parameter was set to `true` and a worker other than the main process was configured to handle background tasks. \ No newline at end of file diff --git a/changelog.d/18130.bugfix b/changelog.d/18130.bugfix deleted file mode 100644 index 4d0c19fab9..0000000000 --- a/changelog.d/18130.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix rare edge case where state groups could be deleted while we are persisting new events that reference them. diff --git a/changelog.d/18131.bugfix b/changelog.d/18131.bugfix deleted file mode 100644 index 4d0c19fab9..0000000000 --- a/changelog.d/18131.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix rare edge case where state groups could be deleted while we are persisting new events that reference them. diff --git a/debian/changelog b/debian/changelog index a470dff676..0b6d50daaa 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +matrix-synapse-py3 (1.124.0~rc1) stable; urgency=medium + + * New Synapse release 1.124.0rc1. + + -- Synapse Packaging team Tue, 04 Feb 2025 11:53:05 +0000 + matrix-synapse-py3 (1.123.0) stable; urgency=medium * New Synapse release 1.123.0. diff --git a/pyproject.toml b/pyproject.toml index 1cd874716e..8febdc4c23 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -97,7 +97,7 @@ module-name = "synapse.synapse_rust" [tool.poetry] name = "matrix-synapse" -version = "1.123.0" +version = "1.124.0rc1" description = "Homeserver for the Matrix decentralised comms protocol" authors = ["Matrix.org Team and Contributors "] license = "AGPL-3.0-or-later" From e41174cae3a74a2e6f9cb1386eb6e0b5510d6fce Mon Sep 17 00:00:00 2001 From: V02460 Date: Tue, 4 Feb 2025 19:45:33 +0100 Subject: [PATCH 05/65] Add MSC3861 config options admin_token_path and client_secret_path (#18004) Another PR on my quest to a `*_path` variant for every secret. Adds two config options `admin_token_path` and `client_secret_path` to the experimental config under `experimental_features.msc3861`. Also includes tests. I tried to be a good citizen here by following `attrs` conventions and not rewriting the corresponding non-path variants in the class, but instead adding methods to retrieve the value. Reading secrets from files has the security advantage of separating the secrets from the config. It also simplifies secrets management in Kubernetes. Also useful to NixOS users. --- changelog.d/18004.feature | 1 + synapse/api/auth/msc3861_delegated.py | 14 ++-- synapse/config/experimental.py | 103 ++++++++++++++++++++++-- tests/config/test_load.py | 10 +++ tests/handlers/test_oauth_delegation.py | 2 +- 5 files changed, 116 insertions(+), 14 deletions(-) create mode 100644 changelog.d/18004.feature diff --git a/changelog.d/18004.feature b/changelog.d/18004.feature new file mode 100644 index 0000000000..8cacd1a0ef --- /dev/null +++ b/changelog.d/18004.feature @@ -0,0 +1 @@ +Add experimental config options `admin_token_path` and `client_secret_path` for MSC 3861. \ No newline at end of file diff --git a/synapse/api/auth/msc3861_delegated.py b/synapse/api/auth/msc3861_delegated.py index 802ea51d18..f825b5c95e 100644 --- a/synapse/api/auth/msc3861_delegated.py +++ b/synapse/api/auth/msc3861_delegated.py @@ -19,7 +19,7 @@ # # import logging -from typing import TYPE_CHECKING, Any, Dict, List, Optional +from typing import TYPE_CHECKING, Any, Callable, Dict, List, Optional from urllib.parse import urlencode from authlib.oauth2 import ClientAuth @@ -119,7 +119,7 @@ class MSC3861DelegatedAuth(BaseAuth): self._clock = hs.get_clock() self._http_client = hs.get_proxied_http_client() self._hostname = hs.hostname - self._admin_token = self._config.admin_token + self._admin_token: Callable[[], Optional[str]] = self._config.admin_token self._issuer_metadata = RetryOnExceptionCachedCall[OpenIDProviderMetadata]( self._load_metadata @@ -133,9 +133,10 @@ class MSC3861DelegatedAuth(BaseAuth): ) else: # Else use the client secret - assert self._config.client_secret, "No client_secret provided" + client_secret = self._config.client_secret() + assert client_secret, "No client_secret provided" self._client_auth = ClientAuth( - self._config.client_id, self._config.client_secret, auth_method + self._config.client_id, client_secret, auth_method ) async def _load_metadata(self) -> OpenIDProviderMetadata: @@ -283,7 +284,7 @@ class MSC3861DelegatedAuth(BaseAuth): requester = await self.get_user_by_access_token(access_token, allow_expired) # Do not record requests from MAS using the virtual `__oidc_admin` user. - if access_token != self._admin_token: + if access_token != self._admin_token(): await self._record_request(request, requester) if not allow_guest and requester.is_guest: @@ -324,7 +325,8 @@ class MSC3861DelegatedAuth(BaseAuth): token: str, allow_expired: bool = False, ) -> Requester: - if self._admin_token is not None and token == self._admin_token: + admin_token = self._admin_token() + if admin_token is not None and token == admin_token: # XXX: This is a temporary solution so that the admin API can be called by # the OIDC provider. This will be removed once we have OIDC client # credentials grant support in matrix-authentication-service. diff --git a/synapse/config/experimental.py b/synapse/config/experimental.py index 94a25c7ee8..3beaeb8869 100644 --- a/synapse/config/experimental.py +++ b/synapse/config/experimental.py @@ -20,14 +20,15 @@ # import enum -from typing import TYPE_CHECKING, Any, Optional +from functools import cache +from typing import TYPE_CHECKING, Any, Iterable, Optional import attr import attr.validators from synapse.api.room_versions import KNOWN_ROOM_VERSIONS, RoomVersions from synapse.config import ConfigError -from synapse.config._base import Config, RootConfig +from synapse.config._base import Config, RootConfig, read_file from synapse.types import JsonDict # Determine whether authlib is installed. @@ -43,6 +44,12 @@ if TYPE_CHECKING: from authlib.jose.rfc7517 import JsonWebKey +@cache +def read_secret_from_file_once(file_path: Any, config_path: Iterable[str]) -> str: + """Returns the memoized secret read from file.""" + return read_file(file_path, config_path).strip() + + class ClientAuthMethod(enum.Enum): """List of supported client auth methods.""" @@ -63,6 +70,40 @@ def _parse_jwks(jwks: Optional[JsonDict]) -> Optional["JsonWebKey"]: return JsonWebKey.import_key(jwks) +def _check_client_secret( + instance: "MSC3861", _attribute: attr.Attribute, _value: Optional[str] +) -> None: + if instance._client_secret and instance._client_secret_path: + raise ConfigError( + ( + "You have configured both " + "`experimental_features.msc3861.client_secret` and " + "`experimental_features.msc3861.client_secret_path`. " + "These are mutually incompatible." + ), + ("experimental", "msc3861", "client_secret"), + ) + # Check client secret can be retrieved + instance.client_secret() + + +def _check_admin_token( + instance: "MSC3861", _attribute: attr.Attribute, _value: Optional[str] +) -> None: + if instance._admin_token and instance._admin_token_path: + raise ConfigError( + ( + "You have configured both " + "`experimental_features.msc3861.admin_token` and " + "`experimental_features.msc3861.admin_token_path`. " + "These are mutually incompatible." + ), + ("experimental", "msc3861", "admin_token"), + ) + # Check client secret can be retrieved + instance.admin_token() + + @attr.s(slots=True, frozen=True) class MSC3861: """Configuration for MSC3861: Matrix architecture change to delegate authentication via OIDC""" @@ -97,15 +138,30 @@ class MSC3861: ) """The auth method used when calling the introspection endpoint.""" - client_secret: Optional[str] = attr.ib( + _client_secret: Optional[str] = attr.ib( default=None, - validator=attr.validators.optional(attr.validators.instance_of(str)), + validator=[ + attr.validators.optional(attr.validators.instance_of(str)), + _check_client_secret, + ], ) """ The client secret to use when calling the introspection endpoint, when using any of the client_secret_* client auth methods. """ + _client_secret_path: Optional[str] = attr.ib( + default=None, + validator=[ + attr.validators.optional(attr.validators.instance_of(str)), + _check_client_secret, + ], + ) + """ + Alternative to `client_secret`: allows the secret to be specified in an + external file. + """ + jwk: Optional["JsonWebKey"] = attr.ib(default=None, converter=_parse_jwks) """ The JWKS to use when calling the introspection endpoint, @@ -133,7 +189,7 @@ class MSC3861: ClientAuthMethod.CLIENT_SECRET_BASIC, ClientAuthMethod.CLIENT_SECRET_JWT, ) - and self.client_secret is None + and self.client_secret() is None ): raise ConfigError( f"A client secret must be provided when using the {value} client auth method", @@ -152,15 +208,48 @@ class MSC3861: ) """The URL of the My Account page on the OIDC Provider as per MSC2965.""" - admin_token: Optional[str] = attr.ib( + _admin_token: Optional[str] = attr.ib( default=None, - validator=attr.validators.optional(attr.validators.instance_of(str)), + validator=[ + attr.validators.optional(attr.validators.instance_of(str)), + _check_admin_token, + ], ) """ A token that should be considered as an admin token. This is used by the OIDC provider, to make admin calls to Synapse. """ + _admin_token_path: Optional[str] = attr.ib( + default=None, + validator=[ + attr.validators.optional(attr.validators.instance_of(str)), + _check_admin_token, + ], + ) + """ + Alternative to `admin_token`: allows the secret to be specified in an + external file. + """ + + def client_secret(self) -> Optional[str]: + """Returns the secret given via `client_secret` or `client_secret_path`.""" + if self._client_secret_path: + return read_secret_from_file_once( + self._client_secret_path, + ("experimental_features", "msc3861", "client_secret_path"), + ) + return self._client_secret + + def admin_token(self) -> Optional[str]: + """Returns the admin token given via `admin_token` or `admin_token_path`.""" + if self._admin_token_path: + return read_secret_from_file_once( + self._admin_token_path, + ("experimental_features", "msc3861", "admin_token_path"), + ) + return self._admin_token + def check_config_conflicts(self, root: RootConfig) -> None: """Checks for any configuration conflicts with other parts of Synapse. diff --git a/tests/config/test_load.py b/tests/config/test_load.py index f8f7b72e40..220ca23aa7 100644 --- a/tests/config/test_load.py +++ b/tests/config/test_load.py @@ -132,6 +132,8 @@ class ConfigLoadingFileTestCase(ConfigFileTestCase): "turn_shared_secret_path: /does/not/exist", "registration_shared_secret_path: /does/not/exist", "macaroon_secret_key_path: /does/not/exist", + "experimental_features:\n msc3861:\n client_secret_path: /does/not/exist", + "experimental_features:\n msc3861:\n admin_token_path: /does/not/exist", *["redis:\n enabled: true\n password_path: /does/not/exist"] * (hiredis is not None), ] @@ -157,6 +159,14 @@ class ConfigLoadingFileTestCase(ConfigFileTestCase): "macaroon_secret_key_path: {}", lambda c: c.key.macaroon_secret_key, ), + ( + "experimental_features:\n msc3861:\n client_secret_path: {}", + lambda c: c.experimental.msc3861.client_secret().encode("utf-8"), + ), + ( + "experimental_features:\n msc3861:\n admin_token_path: {}", + lambda c: c.experimental.msc3861.admin_token().encode("utf-8"), + ), *[ ( "redis:\n enabled: true\n password_path: {}", diff --git a/tests/handlers/test_oauth_delegation.py b/tests/handlers/test_oauth_delegation.py index 5f73469daa..ba2f8ff510 100644 --- a/tests/handlers/test_oauth_delegation.py +++ b/tests/handlers/test_oauth_delegation.py @@ -795,7 +795,7 @@ class MSC3861OAuthDelegation(HomeserverTestCase): req = SynapseRequest(channel, self.site) # type: ignore[arg-type] req.client.host = MAS_IPV4_ADDR req.requestHeaders.addRawHeader( - "Authorization", f"Bearer {self.auth._admin_token}" + "Authorization", f"Bearer {self.auth._admin_token()}" ) req.requestHeaders.addRawHeader("User-Agent", MAS_USER_AGENT) req.content = BytesIO(b"") From b8a333004a9d7591ecf2c477c50e06f122f40565 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Wed, 5 Feb 2025 13:07:49 +0100 Subject: [PATCH 06/65] Fix legacy modules `check_username_for_spam` (#18135) Broke in #17916, as the signature inspection incorrectly looks at the wrapper function. We fix this by setting the signature on the wrapper function to that of the wrapped function via `@functools.wraps`. --- changelog.d/18135.bugfix | 1 + synapse/module_api/callbacks/spamchecker_callbacks.py | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 changelog.d/18135.bugfix diff --git a/changelog.d/18135.bugfix b/changelog.d/18135.bugfix new file mode 100644 index 0000000000..998914a811 --- /dev/null +++ b/changelog.d/18135.bugfix @@ -0,0 +1 @@ +Fix user directory search when using a legacy module with a `check_username_for_spam` callback. Broke in v1.122.0. diff --git a/synapse/module_api/callbacks/spamchecker_callbacks.py b/synapse/module_api/callbacks/spamchecker_callbacks.py index a2f328cafe..9bc572422b 100644 --- a/synapse/module_api/callbacks/spamchecker_callbacks.py +++ b/synapse/module_api/callbacks/spamchecker_callbacks.py @@ -19,6 +19,7 @@ # # +import functools import inspect import logging from typing import ( @@ -297,6 +298,7 @@ def load_legacy_spam_checkers(hs: "synapse.server.HomeServer") -> None: "Bad signature for callback check_registration_for_spam", ) + @functools.wraps(wrapped_func) def run(*args: Any, **kwargs: Any) -> Awaitable: # Assertion required because mypy can't prove we won't change `f` # back to `None`. See From 5b03265cfb30219eb09cba42125683381ea6a49d Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Wed, 5 Feb 2025 13:30:13 +0100 Subject: [PATCH 07/65] Fix 'Fix lint' GHA (#18136) c.f. #18121 --------- Co-authored-by: Quentin Gliech --- .github/workflows/fix_lint.yaml | 2 +- changelog.d/18136.misc | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) create mode 100644 changelog.d/18136.misc diff --git a/.github/workflows/fix_lint.yaml b/.github/workflows/fix_lint.yaml index 909b0a847f..d6aed83774 100644 --- a/.github/workflows/fix_lint.yaml +++ b/.github/workflows/fix_lint.yaml @@ -21,7 +21,7 @@ jobs: # We use nightly so that `fmt` correctly groups together imports, and # clippy correctly fixes up the benchmarks. toolchain: nightly-2022-12-01 - components: rustfmt + components: clippy, rustfmt - uses: Swatinem/rust-cache@v2 - name: Setup Poetry diff --git a/changelog.d/18136.misc b/changelog.d/18136.misc new file mode 100644 index 0000000000..ed88c38acf --- /dev/null +++ b/changelog.d/18136.misc @@ -0,0 +1 @@ +Fix the 'Fix linting' GitHub Actions workflow. From 6fe41d2b479c25cd8cf34c12a07235a6098a11e4 Mon Sep 17 00:00:00 2001 From: Matthew Hodgson Date: Wed, 5 Feb 2025 13:40:10 +0000 Subject: [PATCH 08/65] make dual licensing explicit (#18134) Update readme & LICENSE files to make it explicit that you can buy a commercial license as an AGPL alternative from Element. --- LICENSE => LICENSE-AGPL-3.0 | 0 LICENSE-COMMERCIAL | 6 ++++++ README.rst | 27 +++++++++++++++++++++------ changelog.d/18134.misc | 1 + 4 files changed, 28 insertions(+), 6 deletions(-) rename LICENSE => LICENSE-AGPL-3.0 (100%) create mode 100644 LICENSE-COMMERCIAL create mode 100644 changelog.d/18134.misc diff --git a/LICENSE b/LICENSE-AGPL-3.0 similarity index 100% rename from LICENSE rename to LICENSE-AGPL-3.0 diff --git a/LICENSE-COMMERCIAL b/LICENSE-COMMERCIAL new file mode 100644 index 0000000000..173e03e0c0 --- /dev/null +++ b/LICENSE-COMMERCIAL @@ -0,0 +1,6 @@ +Licensees holding a valid commercial license with Element may use this +software in accordance with the terms contained in a written agreement +between you and Element. + +To purchase a commercial license please contact our sales team at +licensing@element.io diff --git a/README.rst b/README.rst index 2fe4a7e43f..77f861e788 100644 --- a/README.rst +++ b/README.rst @@ -10,14 +10,15 @@ implementation, written and maintained by `Element `_. `Matrix `__ is the open standard for secure and interoperable real time communications. You can directly run and manage the source code in this repository, available under an AGPL -license. There is no support provided from Element unless you have a -subscription. +license (or alternatively under a commercial license from Element). +There is no support provided by Element unless you have a +subscription from Element. -Subscription alternative -======================== +Subscription +============ -Alternatively, for those that need an enterprise-ready solution, Element -Server Suite (ESS) is `available as a subscription `_. +For those that need an enterprise-ready solution, Element +Server Suite (ESS) is `available via subscription `_. ESS builds on Synapse to offer a complete Matrix-based backend including the full `Admin Console product `_, giving admins the power to easily manage an organization-wide @@ -249,6 +250,20 @@ Developers might be particularly interested in: Alongside all that, join our developer community on Matrix: `#synapse-dev:matrix.org `_, featuring real humans! +Copyright and Licensing +======================= + +Copyright 2014-2017 OpenMarket Ltd +Copyright 2017 Vector Creations Ltd +Copyright 2017-2025 New Vector Ltd + +This software is dual-licensed by New Vector Ltd (Element). It can be used either: + +(1) for free under the terms of the GNU Affero General Public License (as published by the Free Software Foundation, either version 3 of the License, or (at your option) any later version); OR + +(2) under the terms of a paid-for Element Commercial License agreement between you and Element (the terms of which may vary depending on what you and Element have agreed to). +Unless required by applicable law or agreed to in writing, software distributed under the Licenses is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the Licenses for the specific language governing permissions and limitations under the Licenses. + .. |support| image:: https://img.shields.io/badge/matrix-community%20support-success :alt: (get community support in #synapse:matrix.org) diff --git a/changelog.d/18134.misc b/changelog.d/18134.misc new file mode 100644 index 0000000000..be90691516 --- /dev/null +++ b/changelog.d/18134.misc @@ -0,0 +1 @@ +Make it explicit that you can buy an AGPL-alternative commercial license from Element. From 3391da348fc759786952637abd6a58360b081bb6 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Wed, 5 Feb 2025 17:26:07 +0100 Subject: [PATCH 09/65] Fix bug where persisting some events fails after unclean shutdown. (#18137) Introduced in #18107 `UniqueViolation: duplicate key value violates unique constraint "state_groups_persisting_pkey"` --- changelog.d/18137.bugfix | 1 + synapse/storage/databases/state/deletion.py | 14 ++++++++++++-- 2 files changed, 13 insertions(+), 2 deletions(-) create mode 100644 changelog.d/18137.bugfix diff --git a/changelog.d/18137.bugfix b/changelog.d/18137.bugfix new file mode 100644 index 0000000000..d7b9d0426f --- /dev/null +++ b/changelog.d/18137.bugfix @@ -0,0 +1 @@ +Fix regression where persisting events in some rooms could fail after a previous unclean shutdown. Introduced in v1.124.0rc1. diff --git a/synapse/storage/databases/state/deletion.py b/synapse/storage/databases/state/deletion.py index 4853e5aa2f..d0949261f2 100644 --- a/synapse/storage/databases/state/deletion.py +++ b/synapse/storage/databases/state/deletion.py @@ -95,8 +95,18 @@ class StateDeletionDataStore: self.db_pool = database self._instance_name = hs.get_instance_name() - # TODO: Clear from `state_groups_persisting` any holdovers from previous - # running instance. + with db_conn.cursor(txn_name="_clear_existing_persising") as txn: + self._clear_existing_persising(txn) + + def _clear_existing_persising(self, txn: LoggingTransaction) -> None: + """On startup we clear any entries in `state_groups_persisting` that + match our instance name, in case of a previous unclean shutdown""" + + self.db_pool.simple_delete_txn( + txn, + table="state_groups_persisting", + keyvalues={"instance_name": self._instance_name}, + ) async def check_state_groups_and_bump_deletion( self, state_groups: AbstractSet[int] From 553e9882bf8e00a59ce7cb505c7eca730f769305 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Wed, 5 Feb 2025 16:35:55 +0000 Subject: [PATCH 10/65] 1.124.0rc2 --- CHANGES.md | 9 +++++++++ changelog.d/18137.bugfix | 1 - debian/changelog | 6 ++++++ pyproject.toml | 2 +- 4 files changed, 16 insertions(+), 2 deletions(-) delete mode 100644 changelog.d/18137.bugfix diff --git a/CHANGES.md b/CHANGES.md index 051ab130b2..3c92bedda3 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,12 @@ +# Synapse 1.124.0rc2 (2025-02-05) + +### Bugfixes + +- Fix regression where persisting events in some rooms could fail after a previous unclean shutdown. Introduced in v1.124.0rc1. ([\#18137](https://github.com/element-hq/synapse/issues/18137)) + + + + # Synapse 1.124.0rc1 (2025-02-04) ### Bugfixes diff --git a/changelog.d/18137.bugfix b/changelog.d/18137.bugfix deleted file mode 100644 index d7b9d0426f..0000000000 --- a/changelog.d/18137.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix regression where persisting events in some rooms could fail after a previous unclean shutdown. Introduced in v1.124.0rc1. diff --git a/debian/changelog b/debian/changelog index 0b6d50daaa..8c84eecb20 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +matrix-synapse-py3 (1.124.0~rc2) stable; urgency=medium + + * New Synapse release 1.124.0rc2. + + -- Synapse Packaging team Wed, 05 Feb 2025 16:35:53 +0000 + matrix-synapse-py3 (1.124.0~rc1) stable; urgency=medium * New Synapse release 1.124.0rc1. diff --git a/pyproject.toml b/pyproject.toml index 8febdc4c23..3f577a1973 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -97,7 +97,7 @@ module-name = "synapse.synapse_rust" [tool.poetry] name = "matrix-synapse" -version = "1.124.0rc1" +version = "1.124.0rc2" description = "Homeserver for the Matrix decentralised comms protocol" authors = ["Matrix.org Team and Contributors "] license = "AGPL-3.0-or-later" From dcf7b392762067e5ee264df629b2ea65b3473609 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Fri, 7 Feb 2025 11:18:32 +0100 Subject: [PATCH 11/65] Fix performance of `check_state_groups_and_bump_deletion` (#18141) Regressed as part of https://github.com/element-hq/synapse/pull/18107 This does two things: 1. Only check if the state groups have been deleted when calculating the event context (as that's when we will insert them). This avoids lots of checks for read operations. 2. Don't lock the `state_groups` rows when doing the check. This adds overhead, and it doesn't prevent any races. --- changelog.d/18141.bugfix | 1 + synapse/state/__init__.py | 32 ++++++++++++++------- synapse/storage/databases/state/deletion.py | 26 +++++++++++++---- tests/rest/client/test_rooms.py | 4 +-- 4 files changed, 46 insertions(+), 17 deletions(-) create mode 100644 changelog.d/18141.bugfix diff --git a/changelog.d/18141.bugfix b/changelog.d/18141.bugfix new file mode 100644 index 0000000000..7826a519fa --- /dev/null +++ b/changelog.d/18141.bugfix @@ -0,0 +1 @@ +Fix regression in performance of sending events due to superfluous reads and locks. Introduced in v1.124.0rc1. diff --git a/synapse/state/__init__.py b/synapse/state/__init__.py index 5b746f2037..9e48e09270 100644 --- a/synapse/state/__init__.py +++ b/synapse/state/__init__.py @@ -359,6 +359,28 @@ class StateHandler: await_full_state=False, ) + # Ensure we still have the state groups we're relying on, and bump + # their usage time to avoid them being deleted from under us. + if entry.state_group: + missing_state_group = await self._state_deletion_store.check_state_groups_and_bump_deletion( + {entry.state_group} + ) + if missing_state_group: + raise Exception(f"Missing state group: {entry.state_group}") + elif entry.prev_group: + # We only rely on the prev group when persisting the event if we + # don't have an `entry.state_group`. + missing_state_group = await self._state_deletion_store.check_state_groups_and_bump_deletion( + {entry.prev_group} + ) + + if missing_state_group: + # If we're missing the prev group then we can just clear the + # entries, and rely on `entry._state` (which must exist if + # `entry.state_group` is None) + entry.prev_group = None + entry.delta_ids = None + state_group_before_event_prev_group = entry.prev_group deltas_to_state_group_before_event = entry.delta_ids state_ids_before_event = None @@ -519,16 +541,6 @@ class StateHandler: state_group_id ) - if prev_group: - # Ensure that we still have the prev group, and ensure we don't - # delete it while we're persisting the event. - missing_state_group = await self._state_deletion_store.check_state_groups_and_bump_deletion( - {prev_group} - ) - if missing_state_group: - prev_group = None - delta_ids = None - return _StateCacheEntry( state=None, state_group=state_group_id, diff --git a/synapse/storage/databases/state/deletion.py b/synapse/storage/databases/state/deletion.py index d0949261f2..d4b1c20a45 100644 --- a/synapse/storage/databases/state/deletion.py +++ b/synapse/storage/databases/state/deletion.py @@ -123,12 +123,28 @@ class StateDeletionDataStore: "check_state_groups_and_bump_deletion", self._check_state_groups_and_bump_deletion_txn, state_groups, + # We don't need to lock if we're just doing a quick check, as the + # lock doesn't prevent any races here. + lock=False, ) def _check_state_groups_and_bump_deletion_txn( - self, txn: LoggingTransaction, state_groups: AbstractSet[int] + self, txn: LoggingTransaction, state_groups: AbstractSet[int], lock: bool = True ) -> Collection[int]: - existing_state_groups = self._get_existing_groups_with_lock(txn, state_groups) + """Checks to make sure that the state groups haven't been deleted, and + if they're pending deletion we delay it (allowing time for any event + that will use them to finish persisting). + + The `lock` flag sets if we should lock the `state_group` rows we're + checking, which we should do when storing new groups. + + Returns: + The state groups that are missing, if any. + """ + + existing_state_groups = self._get_existing_groups_with_lock( + txn, state_groups, lock=lock + ) self._bump_deletion_txn(txn, existing_state_groups) @@ -188,18 +204,18 @@ class StateDeletionDataStore: ) def _get_existing_groups_with_lock( - self, txn: LoggingTransaction, state_groups: Collection[int] + self, txn: LoggingTransaction, state_groups: Collection[int], lock: bool = True ) -> AbstractSet[int]: """Return which of the given state groups are in the database, and locks those rows with `KEY SHARE` to ensure they don't get concurrently - deleted.""" + deleted (if `lock` is true).""" clause, args = make_in_list_sql_clause(self.db_pool.engine, "id", state_groups) sql = f""" SELECT id FROM state_groups WHERE {clause} """ - if isinstance(self.db_pool.engine, PostgresEngine): + if lock and isinstance(self.db_pool.engine, PostgresEngine): # On postgres we add a row level lock to the rows to ensure that we # conflict with any concurrent DELETEs. `FOR KEY SHARE` lock will # not conflict with other read diff --git a/tests/rest/client/test_rooms.py b/tests/rest/client/test_rooms.py index 3bb539bf87..5473a8e769 100644 --- a/tests/rest/client/test_rooms.py +++ b/tests/rest/client/test_rooms.py @@ -742,7 +742,7 @@ class RoomsCreateTestCase(RoomBase): self.assertEqual(HTTPStatus.OK, channel.code, channel.result) self.assertTrue("room_id" in channel.json_body) assert channel.resource_usage is not None - self.assertEqual(36, channel.resource_usage.db_txn_count) + self.assertEqual(35, channel.resource_usage.db_txn_count) def test_post_room_initial_state(self) -> None: # POST with initial_state config key, expect new room id @@ -755,7 +755,7 @@ class RoomsCreateTestCase(RoomBase): self.assertEqual(HTTPStatus.OK, channel.code, channel.result) self.assertTrue("room_id" in channel.json_body) assert channel.resource_usage is not None - self.assertEqual(38, channel.resource_usage.db_txn_count) + self.assertEqual(37, channel.resource_usage.db_txn_count) def test_post_room_visibility_key(self) -> None: # POST with visibility config key, expect new room id From 0076197c97114b38906a972dcf1eb291fbd16f0f Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Fri, 7 Feb 2025 13:42:57 +0000 Subject: [PATCH 12/65] 1.124.0rc3 --- CHANGES.md | 9 +++++++++ changelog.d/18141.bugfix | 1 - debian/changelog | 6 ++++++ pyproject.toml | 2 +- 4 files changed, 16 insertions(+), 2 deletions(-) delete mode 100644 changelog.d/18141.bugfix diff --git a/CHANGES.md b/CHANGES.md index 3c92bedda3..d310241766 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,12 @@ +# Synapse 1.124.0rc3 (2025-02-07) + +### Bugfixes + +- Fix regression in performance of sending events due to superfluous reads and locks. Introduced in v1.124.0rc1. ([\#18141](https://github.com/element-hq/synapse/issues/18141)) + + + + # Synapse 1.124.0rc2 (2025-02-05) ### Bugfixes diff --git a/changelog.d/18141.bugfix b/changelog.d/18141.bugfix deleted file mode 100644 index 7826a519fa..0000000000 --- a/changelog.d/18141.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix regression in performance of sending events due to superfluous reads and locks. Introduced in v1.124.0rc1. diff --git a/debian/changelog b/debian/changelog index 8c84eecb20..886b871b72 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +matrix-synapse-py3 (1.124.0~rc3) stable; urgency=medium + + * New Synapse release 1.124.0rc3. + + -- Synapse Packaging team Fri, 07 Feb 2025 13:42:55 +0000 + matrix-synapse-py3 (1.124.0~rc2) stable; urgency=medium * New Synapse release 1.124.0rc2. diff --git a/pyproject.toml b/pyproject.toml index 3f577a1973..6b9f4e0dda 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -97,7 +97,7 @@ module-name = "synapse.synapse_rust" [tool.poetry] name = "matrix-synapse" -version = "1.124.0rc2" +version = "1.124.0rc3" description = "Homeserver for the Matrix decentralised comms protocol" authors = ["Matrix.org Team and Contributors "] license = "AGPL-3.0-or-later" From deb09b38362be0c567da166bd697ef8abff422b2 Mon Sep 17 00:00:00 2001 From: villepeh <100730729+villepeh@users.noreply.github.com> Date: Mon, 10 Feb 2025 10:17:26 +0200 Subject: [PATCH 13/65] Add Oracle Linux installation instructions (#17436) ### Pull Request Checklist * [X] Pull request is based on the develop branch * [X] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [X] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) I forgot the guide applies to Oracle Linux as well. In fact, I ran a small homeserver on OEL a few months back. I did minimal installations on Rocky and OEL on VirtualBox and noticed Codeready/Powertools repos aren't required, so I removed those commands from the guide. I switched `RHEL`-references to `EL`. #17423 was merged before I remembered about OEL but a new PR shouldn't hurt :) --------- Co-authored-by: Quentin Gliech --- changelog.d/17436.doc | 1 + docs/setup/installation.md | 17 +++-------------- 2 files changed, 4 insertions(+), 14 deletions(-) create mode 100644 changelog.d/17436.doc diff --git a/changelog.d/17436.doc b/changelog.d/17436.doc new file mode 100644 index 0000000000..7846d3a8bf --- /dev/null +++ b/changelog.d/17436.doc @@ -0,0 +1 @@ +Add Oracle Linux 8 and 9 installation instructions. diff --git a/docs/setup/installation.md b/docs/setup/installation.md index d0081a88d9..0853496ab7 100644 --- a/docs/setup/installation.md +++ b/docs/setup/installation.md @@ -310,29 +310,18 @@ sudo dnf install libtiff-devel libjpeg-devel libzip-devel freetype-devel \ sudo dnf group install "Development Tools" ``` -##### Red Hat Enterprise Linux / Rocky Linux +##### Red Hat Enterprise Linux / Rocky Linux / Oracle Linux -*Note: The term "RHEL" below refers to both Red Hat Enterprise Linux and Rocky Linux. The distributions are 1:1 binary compatible.* +*Note: The term "RHEL" below refers to Red Hat Enterprise Linux, Oracle Linux and Rocky Linux. The distributions are 1:1 binary compatible.* It's recommended to use the latest Python versions. -RHEL 8 in particular ships with Python 3.6 by default which is EOL and therefore no longer supported by Synapse. RHEL 9 ship with Python 3.9 which is still supported by the Python core team as of this writing. However, newer Python versions provide significant performance improvements and they're available in official distributions' repositories. Therefore it's recommended to use them. +RHEL 8 in particular ships with Python 3.6 by default which is EOL and therefore no longer supported by Synapse. RHEL 9 ships with Python 3.9 which is still supported by the Python core team as of this writing. However, newer Python versions provide significant performance improvements and they're available in official distributions' repositories. Therefore it's recommended to use them. Python 3.11 and 3.12 are available for both RHEL 8 and 9. These commands should be run as root user. -RHEL 8 -```bash -# Enable PowerTools repository -dnf config-manager --set-enabled powertools -``` -RHEL 9 -```bash -# Enable CodeReady Linux Builder repository -crb enable -``` - Install new version of Python. You only need one of these: ```bash # Python 3.11 From 4c84c9c4ad0d3a1e30b5384604e49f8569c0e015 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Mon, 10 Feb 2025 16:27:46 +0100 Subject: [PATCH 14/65] Don't log exceptions for obviously incorrect stream tokens (#18139) We log incorrect ones as we want to catch bugs where Synapse returns bad tokens. However, sometimes clients just send tokens that are e.g. empty. --------- Co-authored-by: Eric Eastwood --- changelog.d/18139.misc | 1 + synapse/types/__init__.py | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 changelog.d/18139.misc diff --git a/changelog.d/18139.misc b/changelog.d/18139.misc new file mode 100644 index 0000000000..f753033b8e --- /dev/null +++ b/changelog.d/18139.misc @@ -0,0 +1 @@ +Do not log at the exception-level when clients provide empty `since` token to `/sync` API. diff --git a/synapse/types/__init__.py b/synapse/types/__init__.py index 26783c5622..3d15c04faa 100644 --- a/synapse/types/__init__.py +++ b/synapse/types/__init__.py @@ -664,6 +664,11 @@ class RoomStreamToken(AbstractMultiWriterStreamToken): @classmethod async def parse(cls, store: "PurgeEventsStore", string: str) -> "RoomStreamToken": + # Check that it looks like a Synapse token first. We do this so that + # we don't log at the exception-level for obviously incorrect tokens. + if not string or string[0] not in ("s", "t", "m"): + raise SynapseError(400, f"Invalid room stream token {string:!r}") + try: if string[0] == "s": return cls(topological=None, stream=int(string[1:])) From 8f07ef5c93baed5b1259ed9ce3ed1087b7a2d168 Mon Sep 17 00:00:00 2001 From: meise Date: Mon, 10 Feb 2025 16:36:21 +0100 Subject: [PATCH 15/65] feat: Allow multiple values for SSO attribute_requirements via comma separation (#17949) In the current `attribute_requirements` implementation it is only possible to allow exact matching attribute values. Multiple allowed values for one attribute are not possible as described in #13238. ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --------- Co-authored-by: Sebastian Neuser Co-authored-by: Quentin Gliech --- changelog.d/17949.feature | 1 + .../configuration/config_documentation.md | 9 ++-- synapse/config/sso.py | 20 ++++++-- synapse/handlers/sso.py | 6 ++- tests/handlers/test_oidc.py | 32 +++++++++++++ tests/handlers/test_saml.py | 46 +++++++++++++++++++ 6 files changed, 105 insertions(+), 9 deletions(-) create mode 100644 changelog.d/17949.feature diff --git a/changelog.d/17949.feature b/changelog.d/17949.feature new file mode 100644 index 0000000000..ee9bb51e03 --- /dev/null +++ b/changelog.d/17949.feature @@ -0,0 +1 @@ +Add functionality to be able to use multiple values in SSO feature `attribute_requirements`. diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index a1e671ab8e..a5f23149ec 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -3337,8 +3337,9 @@ This setting has the following sub-options: The default is 'uid'. * `attribute_requirements`: It is possible to configure Synapse to only allow logins if SAML attributes match particular values. The requirements can be listed under - `attribute_requirements` as shown in the example. All of the listed attributes must - match for the login to be permitted. + `attribute_requirements` as shown in the example. All of the listed attributes must + match for the login to be permitted. Values can be specified in a `one_of` list to allow + multiple values for an attribute. * `idp_entityid`: If the metadata XML contains multiple IdP entities then the `idp_entityid` option must be set to the entity to redirect users to. Most deployments only have a single IdP entity and so should omit this option. @@ -3419,7 +3420,9 @@ saml2_config: - attribute: userGroup value: "staff" - attribute: department - value: "sales" + one_of: + - "sales" + - "admins" idp_entityid: 'https://our_idp/entityid' ``` diff --git a/synapse/config/sso.py b/synapse/config/sso.py index d7a2187e7d..97b85e47ea 100644 --- a/synapse/config/sso.py +++ b/synapse/config/sso.py @@ -19,7 +19,7 @@ # # import logging -from typing import Any, Dict, Optional +from typing import Any, Dict, List, Optional import attr @@ -43,13 +43,23 @@ class SsoAttributeRequirement: """Object describing a single requirement for SSO attributes.""" attribute: str - # If a value is not given, than the attribute must simply exist. - value: Optional[str] + # If neither value nor one_of is given, the attribute must simply exist. This is + # only true for CAS configs which use a different JSON schema than the one below. + value: Optional[str] = None + one_of: Optional[List[str]] = None JSON_SCHEMA = { "type": "object", - "properties": {"attribute": {"type": "string"}, "value": {"type": "string"}}, - "required": ["attribute", "value"], + "properties": { + "attribute": {"type": "string"}, + "value": {"type": "string"}, + "one_of": {"type": "array", "items": {"type": "string"}}, + }, + "required": ["attribute"], + "oneOf": [ + {"required": ["value"]}, + {"required": ["one_of"]}, + ], } diff --git a/synapse/handlers/sso.py b/synapse/handlers/sso.py index cee2eefbb3..531ed57110 100644 --- a/synapse/handlers/sso.py +++ b/synapse/handlers/sso.py @@ -1277,12 +1277,16 @@ def _check_attribute_requirement( return False # If the requirement is None, the attribute existing is enough. - if req.value is None: + if req.value is None and req.one_of is None: return True values = attributes[req.attribute] if req.value in values: return True + if req.one_of: + for value in req.one_of: + if value in values: + return True logger.info( "SSO attribute %s did not match required value '%s' (was '%s')", diff --git a/tests/handlers/test_oidc.py b/tests/handlers/test_oidc.py index a81501979d..1b43ee43c6 100644 --- a/tests/handlers/test_oidc.py +++ b/tests/handlers/test_oidc.py @@ -1267,6 +1267,38 @@ class OidcHandlerTestCase(HomeserverTestCase): auth_provider_session_id=None, ) + @override_config( + { + "oidc_config": { + **DEFAULT_CONFIG, + "attribute_requirements": [ + {"attribute": "test", "one_of": ["foo", "bar"]} + ], + } + } + ) + def test_attribute_requirements_one_of(self) -> None: + """Test that auth succeeds if userinfo attribute has multiple values and CONTAINS required value""" + # userinfo with "test": ["bar"] attribute should succeed. + userinfo = { + "sub": "tester", + "username": "tester", + "test": ["bar"], + } + request, _ = self.start_authorization(userinfo) + self.get_success(self.handler.handle_oidc_callback(request)) + + # check that the auth handler got called as expected + self.complete_sso_login.assert_called_once_with( + "@tester:test", + self.provider.idp_id, + request, + ANY, + None, + new_user=True, + auth_provider_session_id=None, + ) + @override_config( { "oidc_config": { diff --git a/tests/handlers/test_saml.py b/tests/handlers/test_saml.py index 6ab8fda6e7..1aca354826 100644 --- a/tests/handlers/test_saml.py +++ b/tests/handlers/test_saml.py @@ -363,6 +363,52 @@ class SamlHandlerTestCase(HomeserverTestCase): auth_provider_session_id=None, ) + @override_config( + { + "saml2_config": { + "attribute_requirements": [ + {"attribute": "userGroup", "one_of": ["staff", "admin"]}, + ], + }, + } + ) + def test_attribute_requirements_one_of(self) -> None: + """The required attributes can be comma-separated.""" + + # stub out the auth handler + auth_handler = self.hs.get_auth_handler() + auth_handler.complete_sso_login = AsyncMock() # type: ignore[method-assign] + + # The response doesn't have the proper department. + saml_response = FakeAuthnResponse( + {"uid": "test_user", "username": "test_user", "userGroup": ["nogroup"]} + ) + request = _mock_request() + self.get_success( + self.handler._handle_authn_response(request, saml_response, "redirect_uri") + ) + auth_handler.complete_sso_login.assert_not_called() + + # Add the proper attributes and it should succeed. + saml_response = FakeAuthnResponse( + {"uid": "test_user", "username": "test_user", "userGroup": ["admin"]} + ) + request.reset_mock() + self.get_success( + self.handler._handle_authn_response(request, saml_response, "redirect_uri") + ) + + # check that the auth handler got called as expected + auth_handler.complete_sso_login.assert_called_once_with( + "@test_user:test", + "saml", + request, + "redirect_uri", + None, + new_user=True, + auth_provider_session_id=None, + ) + def _mock_request() -> Mock: """Returns a mock which will stand in as a SynapseRequest""" From e4074749d296ca703c857dedf00c6d0543a0679b Mon Sep 17 00:00:00 2001 From: Andrew Ferrazzutti Date: Mon, 10 Feb 2025 10:37:05 -0500 Subject: [PATCH 16/65] Overload "allow_none" on DB pool static method (#17616) ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --------- Co-authored-by: Quentin Gliech --- changelog.d/17616.misc | 1 + synapse/storage/database.py | 24 +++++++++++++++---- .../storage/databases/main/e2e_room_keys.py | 23 ++++++++---------- .../storage/databases/main/registration.py | 15 ++++++------ synapse/storage/databases/main/stream.py | 15 ++++++------ 5 files changed, 45 insertions(+), 33 deletions(-) create mode 100644 changelog.d/17616.misc diff --git a/changelog.d/17616.misc b/changelog.d/17616.misc new file mode 100644 index 0000000000..8250832dcd --- /dev/null +++ b/changelog.d/17616.misc @@ -0,0 +1 @@ +Overload DatabasePool.simple_select_one_txn to return non-None when the allow_none parameter is False. diff --git a/synapse/storage/database.py b/synapse/storage/database.py index cb4a5857be..8272e39340 100644 --- a/synapse/storage/database.py +++ b/synapse/storage/database.py @@ -2159,10 +2159,26 @@ class DatabasePool: if rowcount > 1: raise StoreError(500, "More than one row matched (%s)" % (table,)) - # Ideally we could use the overload decorator here to specify that the - # return type is only optional if allow_none is True, but this does not work - # when you call a static method from an instance. - # See https://github.com/python/mypy/issues/7781 + @overload + @staticmethod + def simple_select_one_txn( + txn: LoggingTransaction, + table: str, + keyvalues: Dict[str, Any], + retcols: Collection[str], + allow_none: Literal[False] = False, + ) -> Tuple[Any, ...]: ... + + @overload + @staticmethod + def simple_select_one_txn( + txn: LoggingTransaction, + table: str, + keyvalues: Dict[str, Any], + retcols: Collection[str], + allow_none: Literal[True] = True, + ) -> Optional[Tuple[Any, ...]]: ... + @staticmethod def simple_select_one_txn( txn: LoggingTransaction, diff --git a/synapse/storage/databases/main/e2e_room_keys.py b/synapse/storage/databases/main/e2e_room_keys.py index c2c93e12d9..a618a2de69 100644 --- a/synapse/storage/databases/main/e2e_room_keys.py +++ b/synapse/storage/databases/main/e2e_room_keys.py @@ -510,19 +510,16 @@ class EndToEndRoomKeyStore(EndToEndRoomKeyBackgroundStore): # it isn't there. raise StoreError(404, "No backup with that version exists") - row = cast( - Tuple[int, str, str, Optional[int]], - self.db_pool.simple_select_one_txn( - txn, - table="e2e_room_keys_versions", - keyvalues={ - "user_id": user_id, - "version": this_version, - "deleted": 0, - }, - retcols=("version", "algorithm", "auth_data", "etag"), - allow_none=False, - ), + row = self.db_pool.simple_select_one_txn( + txn, + table="e2e_room_keys_versions", + keyvalues={ + "user_id": user_id, + "version": this_version, + "deleted": 0, + }, + retcols=("version", "algorithm", "auth_data", "etag"), + allow_none=False, ) return { "auth_data": db_to_json(row[2]), diff --git a/synapse/storage/databases/main/registration.py b/synapse/storage/databases/main/registration.py index d7cbe33411..8380930c70 100644 --- a/synapse/storage/databases/main/registration.py +++ b/synapse/storage/databases/main/registration.py @@ -1510,15 +1510,14 @@ class RegistrationWorkerStore(CacheInvalidationWorkerStore): # Override type because the return type is only optional if # allow_none is True, and we don't want mypy throwing errors # about None not being indexable. - pending, completed = cast( - Tuple[int, int], - self.db_pool.simple_select_one_txn( - txn, - "registration_tokens", - keyvalues={"token": token}, - retcols=["pending", "completed"], - ), + row = self.db_pool.simple_select_one_txn( + txn, + "registration_tokens", + keyvalues={"token": token}, + retcols=("pending", "completed"), ) + pending = int(row[0]) + completed = int(row[1]) # Decrement pending and increment completed self.db_pool.simple_update_one_txn( diff --git a/synapse/storage/databases/main/stream.py b/synapse/storage/databases/main/stream.py index b4258a4436..40b0bff164 100644 --- a/synapse/storage/databases/main/stream.py +++ b/synapse/storage/databases/main/stream.py @@ -1837,15 +1837,14 @@ class StreamWorkerStore(EventsWorkerStore, SQLBaseStore): dict """ - stream_ordering, topological_ordering = cast( - Tuple[int, int], - self.db_pool.simple_select_one_txn( - txn, - "events", - keyvalues={"event_id": event_id, "room_id": room_id}, - retcols=["stream_ordering", "topological_ordering"], - ), + row = self.db_pool.simple_select_one_txn( + txn, + "events", + keyvalues={"event_id": event_id, "room_id": room_id}, + retcols=("stream_ordering", "topological_ordering"), ) + stream_ordering = int(row[0]) + topological_ordering = int(row[1]) # Paginating backwards includes the event at the token, but paginating # forward doesn't. From 068e22b4b7c5d38064cf04353de36fbc1ead6ba9 Mon Sep 17 00:00:00 2001 From: V02460 Date: Mon, 10 Feb 2025 17:53:24 +0100 Subject: [PATCH 17/65] Cleanup Python 3.8 leftovers (#17967) Some small cleanups after Python3.8 became EOL. - Move some type imports from `typing_extensions` to `typing` - Remove the `abi3-py38` feature from pyo3 ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --------- Co-authored-by: Quentin Gliech --- changelog.d/17967.misc | 1 + rust/Cargo.toml | 2 +- synapse/_scripts/synapse_port_db.py | 2 +- synapse/api/auth/__init__.py | 4 +--- synapse/event_auth.py | 2 +- synapse/events/__init__.py | 2 +- synapse/federation/sender/__init__.py | 2 +- synapse/federation/transport/server/__init__.py | 4 +--- synapse/federation/transport/server/federation.py | 3 +-- synapse/handlers/directory.py | 4 +--- synapse/handlers/e2e_room_keys.py | 4 +--- synapse/handlers/oidc.py | 2 +- synapse/handlers/register.py | 3 +-- synapse/handlers/sso.py | 2 +- synapse/http/client.py | 2 +- synapse/http/matrixfederationclient.py | 2 +- synapse/http/server.py | 2 +- synapse/http/servlet.py | 3 +-- synapse/logging/context.py | 3 ++- synapse/logging/filter.py | 3 +-- synapse/metrics/jemalloc.py | 3 +-- .../module_api/callbacks/spamchecker_callbacks.py | 4 +--- synapse/push/push_types.py | 4 +--- synapse/rest/client/account.py | 3 +-- synapse/rest/client/directory.py | 4 +--- synapse/rest/client/login.py | 3 +-- synapse/state/v2.py | 4 ++-- synapse/storage/database.py | 3 ++- synapse/storage/databases/main/client_ips.py | 13 +++++++++++-- synapse/storage/databases/main/devices.py | 2 +- synapse/storage/databases/main/e2e_room_keys.py | 15 ++++++++++++--- synapse/storage/databases/main/end_to_end_keys.py | 2 +- synapse/storage/databases/main/events.py | 2 +- synapse/storage/databases/main/events_worker.py | 2 +- synapse/storage/databases/main/stream.py | 3 ++- synapse/storage/databases/main/user_directory.py | 3 +-- synapse/storage/types.py | 3 +-- synapse/types/__init__.py | 3 ++- synapse/util/async_helpers.py | 3 ++- synapse/util/caches/dictionary_cache.py | 13 +++++++++++-- synapse/util/caches/expiringcache.py | 3 +-- synapse/util/caches/lrucache.py | 3 +-- synapse/util/iterutils.py | 3 +-- synapse/util/macaroons.py | 3 +-- synapse/util/metrics.py | 13 +++++++++++-- tests/media/test_media_storage.py | 3 +-- tests/metrics/test_metrics.py | 3 +-- tests/rest/client/test_login.py | 2 +- tests/rest/client/test_media.py | 3 +-- tests/rest/client/test_models.py | 3 +-- tests/rest/client/test_rooms.py | 3 +-- tests/rest/client/utils.py | 2 +- tests/unittest.py | 3 ++- tests/util/test_linearizer.py | 4 +--- tests/utils.py | 3 ++- 55 files changed, 104 insertions(+), 94 deletions(-) create mode 100644 changelog.d/17967.misc diff --git a/changelog.d/17967.misc b/changelog.d/17967.misc new file mode 100644 index 0000000000..60e61a6f7d --- /dev/null +++ b/changelog.d/17967.misc @@ -0,0 +1 @@ +Python 3.8 EOL: compile native extensions with the 3.9 ABI and use typing hints from the standard library. \ No newline at end of file diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 7eebeb3b55..a8a24ee1c7 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -34,7 +34,7 @@ pyo3 = { version = "0.23.2", features = [ "macros", "anyhow", "abi3", - "abi3-py38", + "abi3-py39", ] } pyo3-log = "0.12.0" pythonize = "0.23.0" diff --git a/synapse/_scripts/synapse_port_db.py b/synapse/_scripts/synapse_port_db.py index d8f6f8ebdc..3f67a739a0 100755 --- a/synapse/_scripts/synapse_port_db.py +++ b/synapse/_scripts/synapse_port_db.py @@ -42,12 +42,12 @@ from typing import ( Set, Tuple, Type, + TypedDict, TypeVar, cast, ) import yaml -from typing_extensions import TypedDict from twisted.internet import defer, reactor as reactor_ diff --git a/synapse/api/auth/__init__.py b/synapse/api/auth/__init__.py index d5241afe73..1b801d3ad3 100644 --- a/synapse/api/auth/__init__.py +++ b/synapse/api/auth/__init__.py @@ -18,9 +18,7 @@ # [This file includes modifications made by New Vector Limited] # # -from typing import TYPE_CHECKING, Optional, Tuple - -from typing_extensions import Protocol +from typing import TYPE_CHECKING, Optional, Protocol, Tuple from twisted.web.server import Request diff --git a/synapse/event_auth.py b/synapse/event_auth.py index 3fe344ac93..5ecf493f98 100644 --- a/synapse/event_auth.py +++ b/synapse/event_auth.py @@ -32,6 +32,7 @@ from typing import ( Mapping, MutableMapping, Optional, + Protocol, Set, Tuple, Union, @@ -41,7 +42,6 @@ from typing import ( from canonicaljson import encode_canonical_json from signedjson.key import decode_verify_key_bytes from signedjson.sign import SignatureVerifyException, verify_signed_json -from typing_extensions import Protocol from unpaddedbase64 import decode_base64 from synapse.api.constants import ( diff --git a/synapse/events/__init__.py b/synapse/events/__init__.py index 8e9d27138c..13d41592b3 100644 --- a/synapse/events/__init__.py +++ b/synapse/events/__init__.py @@ -30,6 +30,7 @@ from typing import ( Generic, Iterable, List, + Literal, Optional, Tuple, Type, @@ -39,7 +40,6 @@ from typing import ( ) import attr -from typing_extensions import Literal from unpaddedbase64 import encode_base64 from synapse.api.constants import EventTypes, RelationTypes diff --git a/synapse/federation/sender/__init__.py b/synapse/federation/sender/__init__.py index 17cddf18a3..b95b3c629d 100644 --- a/synapse/federation/sender/__init__.py +++ b/synapse/federation/sender/__init__.py @@ -139,13 +139,13 @@ from typing import ( Hashable, Iterable, List, + Literal, Optional, Tuple, ) import attr from prometheus_client import Counter -from typing_extensions import Literal from twisted.internet import defer diff --git a/synapse/federation/transport/server/__init__.py b/synapse/federation/transport/server/__init__.py index 43102567db..174d02ab6b 100644 --- a/synapse/federation/transport/server/__init__.py +++ b/synapse/federation/transport/server/__init__.py @@ -20,9 +20,7 @@ # # import logging -from typing import TYPE_CHECKING, Dict, Iterable, List, Optional, Tuple, Type - -from typing_extensions import Literal +from typing import TYPE_CHECKING, Dict, Iterable, List, Literal, Optional, Tuple, Type from synapse.api.errors import FederationDeniedError, SynapseError from synapse.federation.transport.server._base import ( diff --git a/synapse/federation/transport/server/federation.py b/synapse/federation/transport/server/federation.py index 093ba30d31..eb96ff27f9 100644 --- a/synapse/federation/transport/server/federation.py +++ b/synapse/federation/transport/server/federation.py @@ -24,6 +24,7 @@ from typing import ( TYPE_CHECKING, Dict, List, + Literal, Mapping, Optional, Sequence, @@ -32,8 +33,6 @@ from typing import ( Union, ) -from typing_extensions import Literal - from synapse.api.constants import Direction, EduTypes from synapse.api.errors import Codes, SynapseError from synapse.api.room_versions import RoomVersions diff --git a/synapse/handlers/directory.py b/synapse/handlers/directory.py index 62ce16794f..48c7d411d5 100644 --- a/synapse/handlers/directory.py +++ b/synapse/handlers/directory.py @@ -21,9 +21,7 @@ import logging import string -from typing import TYPE_CHECKING, Iterable, List, Optional, Sequence - -from typing_extensions import Literal +from typing import TYPE_CHECKING, Iterable, List, Literal, Optional, Sequence from synapse.api.constants import MAX_ALIAS_LENGTH, EventTypes from synapse.api.errors import ( diff --git a/synapse/handlers/e2e_room_keys.py b/synapse/handlers/e2e_room_keys.py index f397911f28..623fd33f13 100644 --- a/synapse/handlers/e2e_room_keys.py +++ b/synapse/handlers/e2e_room_keys.py @@ -20,9 +20,7 @@ # import logging -from typing import TYPE_CHECKING, Dict, Optional, cast - -from typing_extensions import Literal +from typing import TYPE_CHECKING, Dict, Literal, Optional, cast from synapse.api.errors import ( Codes, diff --git a/synapse/handlers/oidc.py b/synapse/handlers/oidc.py index 701e828081..c9109c9e79 100644 --- a/synapse/handlers/oidc.py +++ b/synapse/handlers/oidc.py @@ -31,6 +31,7 @@ from typing import ( List, Optional, Type, + TypedDict, TypeVar, Union, ) @@ -52,7 +53,6 @@ from pymacaroons.exceptions import ( MacaroonInitException, MacaroonInvalidSignatureException, ) -from typing_extensions import TypedDict from twisted.web.client import readBody from twisted.web.http_headers import Headers diff --git a/synapse/handlers/register.py b/synapse/handlers/register.py index c49db83ce7..ecfea175c7 100644 --- a/synapse/handlers/register.py +++ b/synapse/handlers/register.py @@ -23,10 +23,9 @@ """Contains functions for registering clients.""" import logging -from typing import TYPE_CHECKING, Iterable, List, Optional, Tuple +from typing import TYPE_CHECKING, Iterable, List, Optional, Tuple, TypedDict from prometheus_client import Counter -from typing_extensions import TypedDict from synapse import types from synapse.api.constants import ( diff --git a/synapse/handlers/sso.py b/synapse/handlers/sso.py index 531ed57110..9c0d665461 100644 --- a/synapse/handlers/sso.py +++ b/synapse/handlers/sso.py @@ -33,12 +33,12 @@ from typing import ( Mapping, NoReturn, Optional, + Protocol, Set, ) from urllib.parse import urlencode import attr -from typing_extensions import Protocol from twisted.web.iweb import IRequest from twisted.web.server import Request diff --git a/synapse/http/client.py b/synapse/http/client.py index 559b1febf0..84a510fb42 100644 --- a/synapse/http/client.py +++ b/synapse/http/client.py @@ -31,6 +31,7 @@ from typing import ( List, Mapping, Optional, + Protocol, Tuple, Union, ) @@ -40,7 +41,6 @@ import treq from canonicaljson import encode_canonical_json from netaddr import AddrFormatError, IPAddress, IPSet from prometheus_client import Counter -from typing_extensions import Protocol from zope.interface import implementer from OpenSSL import SSL diff --git a/synapse/http/matrixfederationclient.py b/synapse/http/matrixfederationclient.py index e658c68e23..f6d2536957 100644 --- a/synapse/http/matrixfederationclient.py +++ b/synapse/http/matrixfederationclient.py @@ -34,6 +34,7 @@ from typing import ( Dict, Generic, List, + Literal, Optional, TextIO, Tuple, @@ -48,7 +49,6 @@ import treq from canonicaljson import encode_canonical_json from prometheus_client import Counter from signedjson.sign import sign_json -from typing_extensions import Literal from twisted.internet import defer from twisted.internet.error import DNSLookupError diff --git a/synapse/http/server.py b/synapse/http/server.py index 792961a147..8bebafc18e 100644 --- a/synapse/http/server.py +++ b/synapse/http/server.py @@ -39,6 +39,7 @@ from typing import ( List, Optional, Pattern, + Protocol, Tuple, Union, ) @@ -46,7 +47,6 @@ from typing import ( import attr import jinja2 from canonicaljson import encode_canonical_json -from typing_extensions import Protocol from zope.interface import implementer from twisted.internet import defer, interfaces diff --git a/synapse/http/servlet.py b/synapse/http/servlet.py index 0330f1c878..ed6ab08336 100644 --- a/synapse/http/servlet.py +++ b/synapse/http/servlet.py @@ -28,6 +28,7 @@ from http import HTTPStatus from typing import ( TYPE_CHECKING, List, + Literal, Mapping, Optional, Sequence, @@ -37,8 +38,6 @@ from typing import ( overload, ) -from typing_extensions import Literal - from twisted.web.server import Request from synapse._pydantic_compat import ( diff --git a/synapse/logging/context.py b/synapse/logging/context.py index 8a2dfeba13..3ef97f23c9 100644 --- a/synapse/logging/context.py +++ b/synapse/logging/context.py @@ -40,6 +40,7 @@ from typing import ( Any, Awaitable, Callable, + Literal, Optional, Tuple, Type, @@ -49,7 +50,7 @@ from typing import ( ) import attr -from typing_extensions import Literal, ParamSpec +from typing_extensions import ParamSpec from twisted.internet import defer, threads from twisted.python.threadpool import ThreadPool diff --git a/synapse/logging/filter.py b/synapse/logging/filter.py index 11c27c63f2..16de488dbc 100644 --- a/synapse/logging/filter.py +++ b/synapse/logging/filter.py @@ -19,8 +19,7 @@ # # import logging - -from typing_extensions import Literal +from typing import Literal class MetadataFilter(logging.Filter): diff --git a/synapse/metrics/jemalloc.py b/synapse/metrics/jemalloc.py index bd25985686..321ff58083 100644 --- a/synapse/metrics/jemalloc.py +++ b/synapse/metrics/jemalloc.py @@ -23,11 +23,10 @@ import ctypes import logging import os import re -from typing import Iterable, Optional, overload +from typing import Iterable, Literal, Optional, overload import attr from prometheus_client import REGISTRY, Metric -from typing_extensions import Literal from synapse.metrics import GaugeMetricFamily from synapse.metrics._types import Collector diff --git a/synapse/module_api/callbacks/spamchecker_callbacks.py b/synapse/module_api/callbacks/spamchecker_callbacks.py index 9bc572422b..a86b46ba54 100644 --- a/synapse/module_api/callbacks/spamchecker_callbacks.py +++ b/synapse/module_api/callbacks/spamchecker_callbacks.py @@ -29,15 +29,13 @@ from typing import ( Callable, Collection, List, + Literal, Optional, Tuple, Union, cast, ) -# `Literal` appears with Python 3.8. -from typing_extensions import Literal - import synapse from synapse.api.errors import Codes from synapse.logging.opentracing import trace diff --git a/synapse/push/push_types.py b/synapse/push/push_types.py index 201ec97219..57fa926a46 100644 --- a/synapse/push/push_types.py +++ b/synapse/push/push_types.py @@ -18,9 +18,7 @@ # [This file includes modifications made by New Vector Limited] # # -from typing import List, Optional - -from typing_extensions import TypedDict +from typing import List, Optional, TypedDict class EmailReason(TypedDict, total=False): diff --git a/synapse/rest/client/account.py b/synapse/rest/client/account.py index 32fa7b4ec4..59dbad3582 100644 --- a/synapse/rest/client/account.py +++ b/synapse/rest/client/account.py @@ -21,11 +21,10 @@ # import logging import random -from typing import TYPE_CHECKING, List, Optional, Tuple +from typing import TYPE_CHECKING, List, Literal, Optional, Tuple from urllib.parse import urlparse import attr -from typing_extensions import Literal from twisted.web.server import Request diff --git a/synapse/rest/client/directory.py b/synapse/rest/client/directory.py index 98ba5c4c2a..479f489623 100644 --- a/synapse/rest/client/directory.py +++ b/synapse/rest/client/directory.py @@ -20,9 +20,7 @@ # import logging -from typing import TYPE_CHECKING, List, Optional, Tuple - -from typing_extensions import Literal +from typing import TYPE_CHECKING, List, Literal, Optional, Tuple from twisted.web.server import Request diff --git a/synapse/rest/client/login.py b/synapse/rest/client/login.py index 3271b02d40..72b219447b 100644 --- a/synapse/rest/client/login.py +++ b/synapse/rest/client/login.py @@ -30,11 +30,10 @@ from typing import ( List, Optional, Tuple, + TypedDict, Union, ) -from typing_extensions import TypedDict - from synapse.api.constants import ApprovalNoticeMedium from synapse.api.errors import ( Codes, diff --git a/synapse/state/v2.py b/synapse/state/v2.py index da926ad146..d0c0a9fc96 100644 --- a/synapse/state/v2.py +++ b/synapse/state/v2.py @@ -29,15 +29,15 @@ from typing import ( Generator, Iterable, List, + Literal, Optional, + Protocol, Sequence, Set, Tuple, overload, ) -from typing_extensions import Literal, Protocol - from synapse import event_auth from synapse.api.constants import EventTypes from synapse.api.errors import AuthError diff --git a/synapse/storage/database.py b/synapse/storage/database.py index 8272e39340..a4941e58f6 100644 --- a/synapse/storage/database.py +++ b/synapse/storage/database.py @@ -35,6 +35,7 @@ from typing import ( Iterable, Iterator, List, + Literal, Mapping, Optional, Sequence, @@ -47,7 +48,7 @@ from typing import ( import attr from prometheus_client import Counter, Histogram -from typing_extensions import Concatenate, Literal, ParamSpec +from typing_extensions import Concatenate, ParamSpec from twisted.enterprise import adbapi from twisted.internet.interfaces import IReactorCore diff --git a/synapse/storage/databases/main/client_ips.py b/synapse/storage/databases/main/client_ips.py index bf6cfcbfd9..e8c322ab5c 100644 --- a/synapse/storage/databases/main/client_ips.py +++ b/synapse/storage/databases/main/client_ips.py @@ -20,10 +20,19 @@ # import logging -from typing import TYPE_CHECKING, Dict, List, Mapping, Optional, Tuple, Union, cast +from typing import ( + TYPE_CHECKING, + Dict, + List, + Mapping, + Optional, + Tuple, + TypedDict, + Union, + cast, +) import attr -from typing_extensions import TypedDict from synapse.metrics.background_process_metrics import wrap_as_background_process from synapse.storage._base import SQLBaseStore diff --git a/synapse/storage/databases/main/devices.py b/synapse/storage/databases/main/devices.py index 8088943253..0b6d1f2b05 100644 --- a/synapse/storage/databases/main/devices.py +++ b/synapse/storage/databases/main/devices.py @@ -27,6 +27,7 @@ from typing import ( Dict, Iterable, List, + Literal, Mapping, Optional, Set, @@ -35,7 +36,6 @@ from typing import ( ) from canonicaljson import encode_canonical_json -from typing_extensions import Literal from synapse.api.constants import EduTypes from synapse.api.errors import Codes, StoreError diff --git a/synapse/storage/databases/main/e2e_room_keys.py b/synapse/storage/databases/main/e2e_room_keys.py index a618a2de69..904ae5cb58 100644 --- a/synapse/storage/databases/main/e2e_room_keys.py +++ b/synapse/storage/databases/main/e2e_room_keys.py @@ -19,9 +19,18 @@ # # -from typing import TYPE_CHECKING, Dict, Iterable, List, Mapping, Optional, Tuple, cast - -from typing_extensions import Literal, TypedDict +from typing import ( + TYPE_CHECKING, + Dict, + Iterable, + List, + Literal, + Mapping, + Optional, + Tuple, + TypedDict, + cast, +) from synapse.api.errors import StoreError from synapse.logging.opentracing import log_kv, trace diff --git a/synapse/storage/databases/main/end_to_end_keys.py b/synapse/storage/databases/main/end_to_end_keys.py index 3bb8fccb5e..b4c7069958 100644 --- a/synapse/storage/databases/main/end_to_end_keys.py +++ b/synapse/storage/databases/main/end_to_end_keys.py @@ -27,6 +27,7 @@ from typing import ( Dict, Iterable, List, + Literal, Mapping, Optional, Sequence, @@ -39,7 +40,6 @@ from typing import ( import attr from canonicaljson import encode_canonical_json -from typing_extensions import Literal from synapse.api.constants import DeviceKeyAlgorithms from synapse.appservice import ( diff --git a/synapse/storage/databases/main/events.py b/synapse/storage/databases/main/events.py index a23aaf5096..26fbc1a483 100644 --- a/synapse/storage/databases/main/events.py +++ b/synapse/storage/databases/main/events.py @@ -35,12 +35,12 @@ from typing import ( Sequence, Set, Tuple, + TypedDict, cast, ) import attr from prometheus_client import Counter -from typing_extensions import TypedDict import synapse.metrics from synapse.api.constants import ( diff --git a/synapse/storage/databases/main/events_worker.py b/synapse/storage/databases/main/events_worker.py index 222df8757a..82b2ad4408 100644 --- a/synapse/storage/databases/main/events_worker.py +++ b/synapse/storage/databases/main/events_worker.py @@ -30,6 +30,7 @@ from typing import ( Dict, Iterable, List, + Literal, Mapping, MutableMapping, Optional, @@ -41,7 +42,6 @@ from typing import ( import attr from prometheus_client import Gauge -from typing_extensions import Literal from twisted.internet import defer diff --git a/synapse/storage/databases/main/stream.py b/synapse/storage/databases/main/stream.py index 40b0bff164..00e5208674 100644 --- a/synapse/storage/databases/main/stream.py +++ b/synapse/storage/databases/main/stream.py @@ -50,6 +50,7 @@ from typing import ( Dict, Iterable, List, + Literal, Mapping, Optional, Protocol, @@ -61,7 +62,7 @@ from typing import ( import attr from immutabledict import immutabledict -from typing_extensions import Literal, assert_never +from typing_extensions import assert_never from twisted.internet import defer diff --git a/synapse/storage/databases/main/user_directory.py b/synapse/storage/databases/main/user_directory.py index 51cffb0986..a51182de55 100644 --- a/synapse/storage/databases/main/user_directory.py +++ b/synapse/storage/databases/main/user_directory.py @@ -31,6 +31,7 @@ from typing import ( Sequence, Set, Tuple, + TypedDict, cast, ) @@ -44,8 +45,6 @@ try: except ModuleNotFoundError: USE_ICU = False -from typing_extensions import TypedDict - from synapse.api.errors import StoreError from synapse.util.stringutils import non_null_str_or_none diff --git a/synapse/storage/types.py b/synapse/storage/types.py index 74f60cc590..4329d88c9a 100644 --- a/synapse/storage/types.py +++ b/synapse/storage/types.py @@ -26,14 +26,13 @@ from typing import ( List, Mapping, Optional, + Protocol, Sequence, Tuple, Type, Union, ) -from typing_extensions import Protocol - """ Some very basic protocol definitions for the DB-API2 classes specified in PEP-249 """ diff --git a/synapse/types/__init__.py b/synapse/types/__init__.py index 3d15c04faa..e9cdd19868 100644 --- a/synapse/types/__init__.py +++ b/synapse/types/__init__.py @@ -40,6 +40,7 @@ from typing import ( Set, Tuple, Type, + TypedDict, TypeVar, Union, overload, @@ -49,7 +50,7 @@ import attr from immutabledict import immutabledict from signedjson.key import decode_verify_key_bytes from signedjson.types import VerifyKey -from typing_extensions import Self, TypedDict +from typing_extensions import Self from unpaddedbase64 import decode_base64 from zope.interface import Interface diff --git a/synapse/util/async_helpers.py b/synapse/util/async_helpers.py index e1eb8a4863..e596e1ed20 100644 --- a/synapse/util/async_helpers.py +++ b/synapse/util/async_helpers.py @@ -41,6 +41,7 @@ from typing import ( Hashable, Iterable, List, + Literal, Optional, Set, Tuple, @@ -51,7 +52,7 @@ from typing import ( ) import attr -from typing_extensions import Concatenate, Literal, ParamSpec, Unpack +from typing_extensions import Concatenate, ParamSpec, Unpack from twisted.internet import defer from twisted.internet.defer import CancelledError diff --git a/synapse/util/caches/dictionary_cache.py b/synapse/util/caches/dictionary_cache.py index 1e6696332f..14bd3ba3b0 100644 --- a/synapse/util/caches/dictionary_cache.py +++ b/synapse/util/caches/dictionary_cache.py @@ -21,10 +21,19 @@ import enum import logging import threading -from typing import Dict, Generic, Iterable, Optional, Set, Tuple, TypeVar, Union +from typing import ( + Dict, + Generic, + Iterable, + Literal, + Optional, + Set, + Tuple, + TypeVar, + Union, +) import attr -from typing_extensions import Literal from synapse.util.caches.lrucache import LruCache from synapse.util.caches.treecache import TreeCache diff --git a/synapse/util/caches/expiringcache.py b/synapse/util/caches/expiringcache.py index 8017c031ee..3198fdd2ed 100644 --- a/synapse/util/caches/expiringcache.py +++ b/synapse/util/caches/expiringcache.py @@ -21,10 +21,9 @@ import logging from collections import OrderedDict -from typing import Any, Generic, Iterable, Optional, TypeVar, Union, overload +from typing import Any, Generic, Iterable, Literal, Optional, TypeVar, Union, overload import attr -from typing_extensions import Literal from twisted.internet import defer diff --git a/synapse/util/caches/lrucache.py b/synapse/util/caches/lrucache.py index 481a1a621e..2e5efa3a52 100644 --- a/synapse/util/caches/lrucache.py +++ b/synapse/util/caches/lrucache.py @@ -34,6 +34,7 @@ from typing import ( Generic, Iterable, List, + Literal, Optional, Set, Tuple, @@ -44,8 +45,6 @@ from typing import ( overload, ) -from typing_extensions import Literal - from twisted.internet import reactor from twisted.internet.interfaces import IReactorTime diff --git a/synapse/util/iterutils.py b/synapse/util/iterutils.py index b73f690b88..ff6adeb716 100644 --- a/synapse/util/iterutils.py +++ b/synapse/util/iterutils.py @@ -30,14 +30,13 @@ from typing import ( Iterator, List, Mapping, + Protocol, Set, Sized, Tuple, TypeVar, ) -from typing_extensions import Protocol - T = TypeVar("T") S = TypeVar("S", bound="_SelfSlice") diff --git a/synapse/util/macaroons.py b/synapse/util/macaroons.py index 84ae226207..6fa15543ec 100644 --- a/synapse/util/macaroons.py +++ b/synapse/util/macaroons.py @@ -22,12 +22,11 @@ """Utilities for manipulating macaroons""" -from typing import Callable, Optional +from typing import Callable, Literal, Optional import attr import pymacaroons from pymacaroons.exceptions import MacaroonVerificationFailedException -from typing_extensions import Literal from synapse.util import Clock, stringutils diff --git a/synapse/util/metrics.py b/synapse/util/metrics.py index 020618598c..6a389f7a7e 100644 --- a/synapse/util/metrics.py +++ b/synapse/util/metrics.py @@ -22,10 +22,19 @@ import logging from functools import wraps from types import TracebackType -from typing import Awaitable, Callable, Dict, Generator, Optional, Type, TypeVar +from typing import ( + Awaitable, + Callable, + Dict, + Generator, + Optional, + Protocol, + Type, + TypeVar, +) from prometheus_client import CollectorRegistry, Counter, Metric -from typing_extensions import Concatenate, ParamSpec, Protocol +from typing_extensions import Concatenate, ParamSpec from synapse.logging.context import ( ContextResourceUsage, diff --git a/tests/media/test_media_storage.py b/tests/media/test_media_storage.py index f4fbc0544a..c2e0e592d7 100644 --- a/tests/media/test_media_storage.py +++ b/tests/media/test_media_storage.py @@ -23,14 +23,13 @@ import shutil import tempfile from binascii import unhexlify from io import BytesIO -from typing import Any, BinaryIO, ClassVar, Dict, List, Optional, Tuple, Union +from typing import Any, BinaryIO, ClassVar, Dict, List, Literal, Optional, Tuple, Union from unittest.mock import MagicMock, Mock, patch from urllib import parse import attr from parameterized import parameterized, parameterized_class from PIL import Image as Image -from typing_extensions import Literal from twisted.internet import defer from twisted.internet.defer import Deferred diff --git a/tests/metrics/test_metrics.py b/tests/metrics/test_metrics.py index 80f24814e8..2e7004df3a 100644 --- a/tests/metrics/test_metrics.py +++ b/tests/metrics/test_metrics.py @@ -19,12 +19,11 @@ # # from importlib import metadata -from typing import Dict, Tuple +from typing import Dict, Protocol, Tuple from unittest.mock import patch from pkg_resources import parse_version from prometheus_client.core import Sample -from typing_extensions import Protocol from synapse.app._base import _set_prometheus_client_use_created_metrics from synapse.metrics import REGISTRY, InFlightGauge, generate_latest diff --git a/tests/rest/client/test_login.py b/tests/rest/client/test_login.py index 1451fd7c29..d7148917d0 100644 --- a/tests/rest/client/test_login.py +++ b/tests/rest/client/test_login.py @@ -27,6 +27,7 @@ from typing import ( Collection, Dict, List, + Literal, Optional, Tuple, Union, @@ -35,7 +36,6 @@ from unittest.mock import Mock from urllib.parse import urlencode import pymacaroons -from typing_extensions import Literal from twisted.test.proto_helpers import MemoryReactor from twisted.web.resource import Resource diff --git a/tests/rest/client/test_media.py b/tests/rest/client/test_media.py index 4060525efe..7d7dbd342b 100644 --- a/tests/rest/client/test_media.py +++ b/tests/rest/client/test_media.py @@ -24,14 +24,13 @@ import json import os import re import shutil -from typing import Any, BinaryIO, Dict, List, Optional, Sequence, Tuple, Type +from typing import Any, BinaryIO, ClassVar, Dict, List, Optional, Sequence, Tuple, Type from unittest.mock import MagicMock, Mock, patch from urllib import parse from urllib.parse import quote, urlencode from parameterized import parameterized, parameterized_class from PIL import Image as Image -from typing_extensions import ClassVar from twisted.internet import defer from twisted.internet._resolver import HostResolution diff --git a/tests/rest/client/test_models.py b/tests/rest/client/test_models.py index f14585ccac..75479e6235 100644 --- a/tests/rest/client/test_models.py +++ b/tests/rest/client/test_models.py @@ -19,8 +19,7 @@ # # import unittest as stdlib_unittest - -from typing_extensions import Literal +from typing import Literal from synapse._pydantic_compat import BaseModel, ValidationError from synapse.types.rest.client import EmailRequestTokenBody diff --git a/tests/rest/client/test_rooms.py b/tests/rest/client/test_rooms.py index 5473a8e769..604b585150 100644 --- a/tests/rest/client/test_rooms.py +++ b/tests/rest/client/test_rooms.py @@ -25,12 +25,11 @@ import json from http import HTTPStatus -from typing import Any, Dict, Iterable, List, Optional, Tuple, Union +from typing import Any, Dict, Iterable, List, Literal, Optional, Tuple, Union from unittest.mock import AsyncMock, Mock, call, patch from urllib import parse as urlparse from parameterized import param, parameterized -from typing_extensions import Literal from twisted.test.proto_helpers import MemoryReactor diff --git a/tests/rest/client/utils.py b/tests/rest/client/utils.py index e766630afb..53f1782d59 100644 --- a/tests/rest/client/utils.py +++ b/tests/rest/client/utils.py @@ -31,6 +31,7 @@ from typing import ( AnyStr, Dict, Iterable, + Literal, Mapping, MutableMapping, Optional, @@ -40,7 +41,6 @@ from typing import ( from urllib.parse import urlencode import attr -from typing_extensions import Literal from twisted.test.proto_helpers import MemoryReactorClock from twisted.web.server import Site diff --git a/tests/unittest.py b/tests/unittest.py index 6a32861a3e..24077d79d6 100644 --- a/tests/unittest.py +++ b/tests/unittest.py @@ -40,6 +40,7 @@ from typing import ( Mapping, NoReturn, Optional, + Protocol, Tuple, Type, TypeVar, @@ -50,7 +51,7 @@ from unittest.mock import Mock, patch import canonicaljson import signedjson.key import unpaddedbase64 -from typing_extensions import Concatenate, ParamSpec, Protocol +from typing_extensions import Concatenate, ParamSpec from twisted.internet.defer import Deferred, ensureDeferred from twisted.python.failure import Failure diff --git a/tests/util/test_linearizer.py b/tests/util/test_linearizer.py index 7cbb1007da..7510657b85 100644 --- a/tests/util/test_linearizer.py +++ b/tests/util/test_linearizer.py @@ -19,9 +19,7 @@ # # -from typing import Hashable, Tuple - -from typing_extensions import Protocol +from typing import Hashable, Protocol, Tuple from twisted.internet import defer, reactor from twisted.internet.base import ReactorBase diff --git a/tests/utils.py b/tests/utils.py index d4aebc3069..0006bd7a8d 100644 --- a/tests/utils.py +++ b/tests/utils.py @@ -28,6 +28,7 @@ from typing import ( Callable, Dict, List, + Literal, Optional, Tuple, Type, @@ -37,7 +38,7 @@ from typing import ( ) import attr -from typing_extensions import Literal, ParamSpec +from typing_extensions import ParamSpec from synapse.api.constants import EventTypes from synapse.api.room_versions import RoomVersions From 703f2e8c43883516e618013c528cd92b42a722ae Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 11 Feb 2025 00:07:43 +0700 Subject: [PATCH 18/65] Bump types-pyyaml from 6.0.12.20240917 to 6.0.12.20241230 (#18097) --- poetry.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/poetry.lock b/poetry.lock index c9c567d6e3..40f1be243b 100644 --- a/poetry.lock +++ b/poetry.lock @@ -2829,13 +2829,13 @@ types-cffi = "*" [[package]] name = "types-pyyaml" -version = "6.0.12.20240917" +version = "6.0.12.20241230" description = "Typing stubs for PyYAML" optional = false python-versions = ">=3.8" files = [ - {file = "types-PyYAML-6.0.12.20240917.tar.gz", hash = "sha256:d1405a86f9576682234ef83bcb4e6fff7c9305c8b1fbad5e0bcd4f7dbdc9c587"}, - {file = "types_PyYAML-6.0.12.20240917-py3-none-any.whl", hash = "sha256:392b267f1c0fe6022952462bf5d6523f31e37f6cea49b14cee7ad634b6301570"}, + {file = "types_PyYAML-6.0.12.20241230-py3-none-any.whl", hash = "sha256:fa4d32565219b68e6dee5f67534c722e53c00d1cfc09c435ef04d7353e1e96e6"}, + {file = "types_pyyaml-6.0.12.20241230.tar.gz", hash = "sha256:7f07622dbd34bb9c8b264fe860a17e0efcad00d50b5f27e93984909d9363498c"}, ] [[package]] From c1815bf5a10629efc0f7f2b1c073e14d5b0e8f57 Mon Sep 17 00:00:00 2001 From: Andrew Morgan <1342360+anoadragon453@users.noreply.github.com> Date: Tue, 11 Feb 2025 14:39:12 +0700 Subject: [PATCH 19/65] Add `rc_presence` ratelimiting config to demo/start.sh (#18145) Missed in https://github.com/element-hq/synapse/pull/18000 --- changelog.d/18145.bugfix | 1 + demo/start.sh | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 changelog.d/18145.bugfix diff --git a/changelog.d/18145.bugfix b/changelog.d/18145.bugfix new file mode 100644 index 0000000000..ad9d353834 --- /dev/null +++ b/changelog.d/18145.bugfix @@ -0,0 +1 @@ +Add rate limit `rc_presence.per_user`. This prevents load from excessive presence updates sent by clients via sync api. Also rate limit `/_matrix/client/v3/presence` as per the spec. Contributed by @rda0. \ No newline at end of file diff --git a/demo/start.sh b/demo/start.sh index 06ec6f985f..7636c41f1f 100755 --- a/demo/start.sh +++ b/demo/start.sh @@ -138,6 +138,10 @@ for port in 8080 8081 8082; do per_user: per_second: 1000 burst_count: 1000 + rc_presence: + per_user: + per_second: 1000 + burst_count: 1000 RC ) echo "${ratelimiting}" >> "$port.config" From c1b7c6b12e7b9b2cf586a3210145f70a5e30ed67 Mon Sep 17 00:00:00 2001 From: Till Faelligen <2353100+S7evinK@users.noreply.github.com> Date: Tue, 11 Feb 2025 11:56:50 +0100 Subject: [PATCH 20/65] 1.124.0 --- CHANGES.md | 7 +++++++ debian/changelog | 6 ++++++ pyproject.toml | 2 +- 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index d310241766..58b410decb 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,10 @@ +# Synapse 1.124.0 (2025-02-11) + +No significant changes since 1.124.0rc3. + + + + # Synapse 1.124.0rc3 (2025-02-07) ### Bugfixes diff --git a/debian/changelog b/debian/changelog index 886b871b72..3e20885c86 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +matrix-synapse-py3 (1.124.0) stable; urgency=medium + + * New Synapse release 1.124.0. + + -- Synapse Packaging team Tue, 11 Feb 2025 11:55:22 +0100 + matrix-synapse-py3 (1.124.0~rc3) stable; urgency=medium * New Synapse release 1.124.0rc3. diff --git a/pyproject.toml b/pyproject.toml index 6b9f4e0dda..2d982eae70 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -97,7 +97,7 @@ module-name = "synapse.synapse_rust" [tool.poetry] name = "matrix-synapse" -version = "1.124.0rc3" +version = "1.124.0" description = "Homeserver for the Matrix decentralised comms protocol" authors = ["Matrix.org Team and Contributors "] license = "AGPL-3.0-or-later" From 26331cbbd5ec23ad37783a462e218ce289e56fc8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Feb 2025 13:24:38 +0000 Subject: [PATCH 21/65] Bump serde_json from 1.0.137 to 1.0.138 (#18129) --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 82ed544226..195b34920f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -449,9 +449,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.137" +version = "1.0.138" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "930cfb6e6abf99298aaad7d29abbef7a9999a9a8806a40088f55f0dcec03146b" +checksum = "d434192e7da787e94a6ea7e9670b26a036d0ca41e0b7efb2676dd32bae872949" dependencies = [ "itoa", "memchr", From 30418653fd5fed04e32f169701de075028a59529 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Feb 2025 13:24:47 +0000 Subject: [PATCH 22/65] Bump gitpython from 3.1.43 to 3.1.44 (#18128) --- poetry.lock | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/poetry.lock b/poetry.lock index 40f1be243b..8cdbe0f8bf 100644 --- a/poetry.lock +++ b/poetry.lock @@ -472,20 +472,20 @@ smmap = ">=3.0.1,<6" [[package]] name = "gitpython" -version = "3.1.43" +version = "3.1.44" description = "GitPython is a Python library used to interact with Git repositories" optional = false python-versions = ">=3.7" files = [ - {file = "GitPython-3.1.43-py3-none-any.whl", hash = "sha256:eec7ec56b92aad751f9912a73404bc02ba212a23adb2c7098ee668417051a1ff"}, - {file = "GitPython-3.1.43.tar.gz", hash = "sha256:35f314a9f878467f5453cc1fee295c3e18e52f1b99f10f6cf5b1682e968a9e7c"}, + {file = "GitPython-3.1.44-py3-none-any.whl", hash = "sha256:9e0e10cda9bed1ee64bc9a6de50e7e38a9c9943241cd7f585f6df3ed28011110"}, + {file = "gitpython-3.1.44.tar.gz", hash = "sha256:c87e30b26253bf5418b01b0660f818967f3c503193838337fe5e573331249269"}, ] [package.dependencies] gitdb = ">=4.0.1,<5" [package.extras] -doc = ["sphinx (==4.3.2)", "sphinx-autodoc-typehints", "sphinx-rtd-theme", "sphinxcontrib-applehelp (>=1.0.2,<=1.0.4)", "sphinxcontrib-devhelp (==1.0.2)", "sphinxcontrib-htmlhelp (>=2.0.0,<=2.0.1)", "sphinxcontrib-qthelp (==1.0.3)", "sphinxcontrib-serializinghtml (==1.1.5)"] +doc = ["sphinx (>=7.1.2,<7.2)", "sphinx-autodoc-typehints", "sphinx_rtd_theme"] test = ["coverage[toml]", "ddt (>=1.1.1,!=1.4.3)", "mock", "mypy", "pre-commit", "pytest (>=7.3.1)", "pytest-cov", "pytest-instafail", "pytest-mock", "pytest-sugar", "typing-extensions"] [[package]] From 5a833ebbc883bed18e66b7fcb46138c95f38e16e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Feb 2025 13:25:21 +0000 Subject: [PATCH 23/65] Bump bcrypt from 4.2.0 to 4.2.1 (#18127) --- poetry.lock | 54 ++++++++++++++++++++++++++--------------------------- 1 file changed, 26 insertions(+), 28 deletions(-) diff --git a/poetry.lock b/poetry.lock index 8cdbe0f8bf..7dee7d098f 100644 --- a/poetry.lock +++ b/poetry.lock @@ -64,38 +64,36 @@ visualize = ["Twisted (>=16.1.1)", "graphviz (>0.5.1)"] [[package]] name = "bcrypt" -version = "4.2.0" +version = "4.2.1" description = "Modern password hashing for your software and your servers" optional = false python-versions = ">=3.7" files = [ - {file = "bcrypt-4.2.0-cp37-abi3-macosx_10_12_universal2.whl", hash = "sha256:096a15d26ed6ce37a14c1ac1e48119660f21b24cba457f160a4b830f3fe6b5cb"}, - {file = "bcrypt-4.2.0-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c02d944ca89d9b1922ceb8a46460dd17df1ba37ab66feac4870f6862a1533c00"}, - {file = "bcrypt-4.2.0-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1d84cf6d877918620b687b8fd1bf7781d11e8a0998f576c7aa939776b512b98d"}, - {file = "bcrypt-4.2.0-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:1bb429fedbe0249465cdd85a58e8376f31bb315e484f16e68ca4c786dcc04291"}, - {file = "bcrypt-4.2.0-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:655ea221910bcac76ea08aaa76df427ef8625f92e55a8ee44fbf7753dbabb328"}, - {file = "bcrypt-4.2.0-cp37-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:1ee38e858bf5d0287c39b7a1fc59eec64bbf880c7d504d3a06a96c16e14058e7"}, - {file = "bcrypt-4.2.0-cp37-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:0da52759f7f30e83f1e30a888d9163a81353ef224d82dc58eb5bb52efcabc399"}, - {file = "bcrypt-4.2.0-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:3698393a1b1f1fd5714524193849d0c6d524d33523acca37cd28f02899285060"}, - {file = "bcrypt-4.2.0-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:762a2c5fb35f89606a9fde5e51392dad0cd1ab7ae64149a8b935fe8d79dd5ed7"}, - {file = "bcrypt-4.2.0-cp37-abi3-win32.whl", hash = "sha256:5a1e8aa9b28ae28020a3ac4b053117fb51c57a010b9f969603ed885f23841458"}, - {file = "bcrypt-4.2.0-cp37-abi3-win_amd64.whl", hash = "sha256:8f6ede91359e5df88d1f5c1ef47428a4420136f3ce97763e31b86dd8280fbdf5"}, - {file = "bcrypt-4.2.0-cp39-abi3-macosx_10_12_universal2.whl", hash = "sha256:c52aac18ea1f4a4f65963ea4f9530c306b56ccd0c6f8c8da0c06976e34a6e841"}, - {file = "bcrypt-4.2.0-cp39-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3bbbfb2734f0e4f37c5136130405332640a1e46e6b23e000eeff2ba8d005da68"}, - {file = "bcrypt-4.2.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3413bd60460f76097ee2e0a493ccebe4a7601918219c02f503984f0a7ee0aebe"}, - {file = "bcrypt-4.2.0-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:8d7bb9c42801035e61c109c345a28ed7e84426ae4865511eb82e913df18f58c2"}, - {file = "bcrypt-4.2.0-cp39-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:3d3a6d28cb2305b43feac298774b997e372e56c7c7afd90a12b3dc49b189151c"}, - {file = "bcrypt-4.2.0-cp39-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:9c1c4ad86351339c5f320ca372dfba6cb6beb25e8efc659bedd918d921956bae"}, - {file = "bcrypt-4.2.0-cp39-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:27fe0f57bb5573104b5a6de5e4153c60814c711b29364c10a75a54bb6d7ff48d"}, - {file = "bcrypt-4.2.0-cp39-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:8ac68872c82f1add6a20bd489870c71b00ebacd2e9134a8aa3f98a0052ab4b0e"}, - {file = "bcrypt-4.2.0-cp39-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:cb2a8ec2bc07d3553ccebf0746bbf3d19426d1c6d1adbd4fa48925f66af7b9e8"}, - {file = "bcrypt-4.2.0-cp39-abi3-win32.whl", hash = "sha256:77800b7147c9dc905db1cba26abe31e504d8247ac73580b4aa179f98e6608f34"}, - {file = "bcrypt-4.2.0-cp39-abi3-win_amd64.whl", hash = "sha256:61ed14326ee023917ecd093ee6ef422a72f3aec6f07e21ea5f10622b735538a9"}, - {file = "bcrypt-4.2.0-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:39e1d30c7233cfc54f5c3f2c825156fe044efdd3e0b9d309512cc514a263ec2a"}, - {file = "bcrypt-4.2.0-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:f4f4acf526fcd1c34e7ce851147deedd4e26e6402369304220250598b26448db"}, - {file = "bcrypt-4.2.0-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:1ff39b78a52cf03fdf902635e4c81e544714861ba3f0efc56558979dd4f09170"}, - {file = "bcrypt-4.2.0-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:373db9abe198e8e2c70d12b479464e0d5092cc122b20ec504097b5f2297ed184"}, - {file = "bcrypt-4.2.0.tar.gz", hash = "sha256:cf69eaf5185fd58f268f805b505ce31f9b9fc2d64b376642164e9244540c1221"}, + {file = "bcrypt-4.2.1-cp37-abi3-macosx_10_12_universal2.whl", hash = "sha256:1340411a0894b7d3ef562fb233e4b6ed58add185228650942bdc885362f32c17"}, + {file = "bcrypt-4.2.1-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b1ee315739bc8387aa36ff127afc99120ee452924e0df517a8f3e4c0187a0f5f"}, + {file = "bcrypt-4.2.1-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8dbd0747208912b1e4ce730c6725cb56c07ac734b3629b60d4398f082ea718ad"}, + {file = "bcrypt-4.2.1-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:aaa2e285be097050dba798d537b6efd9b698aa88eef52ec98d23dcd6d7cf6fea"}, + {file = "bcrypt-4.2.1-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:76d3e352b32f4eeb34703370e370997065d28a561e4a18afe4fef07249cb4396"}, + {file = "bcrypt-4.2.1-cp37-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:b7703ede632dc945ed1172d6f24e9f30f27b1b1a067f32f68bf169c5f08d0425"}, + {file = "bcrypt-4.2.1-cp37-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:89df2aea2c43be1e1fa066df5f86c8ce822ab70a30e4c210968669565c0f4685"}, + {file = "bcrypt-4.2.1-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:04e56e3fe8308a88b77e0afd20bec516f74aecf391cdd6e374f15cbed32783d6"}, + {file = "bcrypt-4.2.1-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:cfdf3d7530c790432046c40cda41dfee8c83e29482e6a604f8930b9930e94139"}, + {file = "bcrypt-4.2.1-cp37-abi3-win32.whl", hash = "sha256:adadd36274510a01f33e6dc08f5824b97c9580583bd4487c564fc4617b328005"}, + {file = "bcrypt-4.2.1-cp37-abi3-win_amd64.whl", hash = "sha256:8c458cd103e6c5d1d85cf600e546a639f234964d0228909d8f8dbeebff82d526"}, + {file = "bcrypt-4.2.1-cp39-abi3-macosx_10_12_universal2.whl", hash = "sha256:8ad2f4528cbf0febe80e5a3a57d7a74e6635e41af1ea5675282a33d769fba413"}, + {file = "bcrypt-4.2.1-cp39-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:909faa1027900f2252a9ca5dfebd25fc0ef1417943824783d1c8418dd7d6df4a"}, + {file = "bcrypt-4.2.1-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cde78d385d5e93ece5479a0a87f73cd6fa26b171c786a884f955e165032b262c"}, + {file = "bcrypt-4.2.1-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:533e7f3bcf2f07caee7ad98124fab7499cb3333ba2274f7a36cf1daee7409d99"}, + {file = "bcrypt-4.2.1-cp39-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:687cf30e6681eeda39548a93ce9bfbb300e48b4d445a43db4298d2474d2a1e54"}, + {file = "bcrypt-4.2.1-cp39-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:041fa0155c9004eb98a232d54da05c0b41d4b8e66b6fc3cb71b4b3f6144ba837"}, + {file = "bcrypt-4.2.1-cp39-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:f85b1ffa09240c89aa2e1ae9f3b1c687104f7b2b9d2098da4e923f1b7082d331"}, + {file = "bcrypt-4.2.1-cp39-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:c6f5fa3775966cca251848d4d5393ab016b3afed251163c1436fefdec3b02c84"}, + {file = "bcrypt-4.2.1-cp39-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:807261df60a8b1ccd13e6599c779014a362ae4e795f5c59747f60208daddd96d"}, + {file = "bcrypt-4.2.1-cp39-abi3-win32.whl", hash = "sha256:b588af02b89d9fad33e5f98f7838bf590d6d692df7153647724a7f20c186f6bf"}, + {file = "bcrypt-4.2.1-cp39-abi3-win_amd64.whl", hash = "sha256:e84e0e6f8e40a242b11bce56c313edc2be121cec3e0ec2d76fce01f6af33c07c"}, + {file = "bcrypt-4.2.1-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:76132c176a6d9953cdc83c296aeaed65e1a708485fd55abf163e0d9f8f16ce0e"}, + {file = "bcrypt-4.2.1-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:e158009a54c4c8bc91d5e0da80920d048f918c61a581f0a63e4e93bb556d362f"}, + {file = "bcrypt-4.2.1.tar.gz", hash = "sha256:6765386e3ab87f569b276988742039baab087b2cdb01e809d74e74503c2faafe"}, ] [package.extras] From c2748392340b25deee42d640e1c399c7170f6664 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Feb 2025 13:29:23 +0000 Subject: [PATCH 24/65] Bump bytes from 1.9.0 to 1.10.0 (#18149) --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 195b34920f..c1055f5674 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -61,9 +61,9 @@ checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "bytes" -version = "1.9.0" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "325918d6fe32f23b19878fe4b34794ae41fc19ddbe53b10571a4874d44ffd39b" +checksum = "f61dac84819c6588b558454b194026eb1f09c293b9036ae9b159e74e73ab6cf9" [[package]] name = "cfg-if" From fe3f462b791dc29671f8b154d9c8aaf41d724bb9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Feb 2025 13:30:36 +0000 Subject: [PATCH 25/65] Bump sigstore/cosign-installer from 3.7.0 to 3.8.0 (#18147) --- .github/workflows/docker.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/docker.yml b/.github/workflows/docker.yml index 82cacdfeb3..b439cb0915 100644 --- a/.github/workflows/docker.yml +++ b/.github/workflows/docker.yml @@ -30,7 +30,7 @@ jobs: run: docker buildx inspect - name: Install Cosign - uses: sigstore/cosign-installer@v3.7.0 + uses: sigstore/cosign-installer@v3.8.0 - name: Checkout repository uses: actions/checkout@v4 From aaffc3566e3303f7b43be262532a09f187c11346 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Feb 2025 13:33:14 +0000 Subject: [PATCH 26/65] Bump ulid from 1.1.4 to 1.2.0 (#18148) --- Cargo.lock | 130 +++++++++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 117 insertions(+), 13 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c1055f5674..8831f7e6fd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -35,6 +35,12 @@ version = "0.21.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9d297deb1925b89f2ccc13d7635fa0714f12c87adce1c75356b39ca9b7178567" +[[package]] +name = "bitflags" +version = "2.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f68f53c83ab957f72c32642f3868eec03eb974d1fb82e453128456482613d36" + [[package]] name = "blake2" version = "0.10.6" @@ -119,13 +125,14 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.15" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" +checksum = "43a49c392881ce6d5c3b8cb70f98717b7c07aabbdff06687b9030dbfbe2725f8" dependencies = [ "cfg-if", "libc", "wasi", + "windows-targets", ] [[package]] @@ -364,20 +371,20 @@ dependencies = [ [[package]] name = "rand" -version = "0.8.5" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +checksum = "3779b94aeb87e8bd4e834cee3650289ee9e0d5677f976ecdb6d219e5f4f6cd94" dependencies = [ - "libc", "rand_chacha", "rand_core", + "zerocopy", ] [[package]] name = "rand_chacha" -version = "0.3.1" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" dependencies = [ "ppv-lite86", "rand_core", @@ -385,11 +392,12 @@ dependencies = [ [[package]] name = "rand_core" -version = "0.6.4" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +checksum = "b08f3c9802962f7e1b25113931d94f43ed9725bebc59db9d0c3e9a23b67e15ff" dependencies = [ "getrandom", + "zerocopy", ] [[package]] @@ -536,9 +544,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "ulid" -version = "1.1.4" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f294bff79170ed1c5633812aff1e565c35d993a36e757f9bc0accf5eec4e6045" +checksum = "ab82fc73182c29b02e2926a6df32f2241dbadb5cfc111fd595515b3598f46bb3" dependencies = [ "rand", "web-time", @@ -564,9 +572,12 @@ checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" [[package]] name = "wasi" -version = "0.11.0+wasi-snapshot-preview1" +version = "0.13.3+wasi-0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" +checksum = "26816d2e1a4a36a2940b96c5296ce403917633dff8f3440e9b236ed6f6bacad2" +dependencies = [ + "wit-bindgen-rt", +] [[package]] name = "wasm-bindgen" @@ -631,3 +642,96 @@ dependencies = [ "js-sys", "wasm-bindgen", ] + +[[package]] +name = "windows-targets" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" +dependencies = [ + "windows_aarch64_gnullvm", + "windows_aarch64_msvc", + "windows_i686_gnu", + "windows_i686_gnullvm", + "windows_i686_msvc", + "windows_x86_64_gnu", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" + +[[package]] +name = "windows_i686_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" + +[[package]] +name = "windows_i686_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" + +[[package]] +name = "wit-bindgen-rt" +version = "0.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3268f3d866458b787f390cf61f4bbb563b922d091359f9608842999eaee3943c" +dependencies = [ + "bitflags", +] + +[[package]] +name = "zerocopy" +version = "0.8.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa91407dacce3a68c56de03abe2760159582b846c6a4acd2f456618087f12713" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.8.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06718a168365cad3d5ff0bb133aad346959a2074bd4a85c121255a11304a8626" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] From 816054b0122fa88cde71ca03aee7f97f382af89b Mon Sep 17 00:00:00 2001 From: qashlan Date: Wed, 12 Feb 2025 16:06:21 +0200 Subject: [PATCH 27/65] Fix internal server error when updating 3pid address with invalid email (#18125) When updating 3pid for a user email from admin api and sending invalid email the server throws 500 internal server error. changed to 400 Bad request and returned the error message Signed-off-by: qashlan Signed-off-by: Ahmed Qashlan --- changelog.d/18125.bugfix | 1 + synapse/handlers/auth.py | 10 ++++++++-- 2 files changed, 9 insertions(+), 2 deletions(-) create mode 100644 changelog.d/18125.bugfix diff --git a/changelog.d/18125.bugfix b/changelog.d/18125.bugfix new file mode 100644 index 0000000000..0e6aa704ff --- /dev/null +++ b/changelog.d/18125.bugfix @@ -0,0 +1 @@ +Fix a bug when updating a user 3pid with invalid returns 500 server error change to 400 with a message. \ No newline at end of file diff --git a/synapse/handlers/auth.py b/synapse/handlers/auth.py index 1f4264ad7e..e96922c08d 100644 --- a/synapse/handlers/auth.py +++ b/synapse/handlers/auth.py @@ -1579,7 +1579,10 @@ class AuthHandler: # for the presence of an email address during password reset was # case sensitive). if medium == "email": - address = canonicalise_email(address) + try: + address = canonicalise_email(address) + except ValueError as e: + raise SynapseError(400, str(e)) await self.store.user_add_threepid( user_id, medium, address, validated_at, self.hs.get_clock().time_msec() @@ -1610,7 +1613,10 @@ class AuthHandler: """ # 'Canonicalise' email addresses as per above if medium == "email": - address = canonicalise_email(address) + try: + address = canonicalise_email(address) + except ValueError as e: + raise SynapseError(400, str(e)) await self.store.user_delete_threepid(user_id, medium, address) From 74aa47828d1013e7c13bee0ec9fd5b1901f20f76 Mon Sep 17 00:00:00 2001 From: Devon Hudson Date: Wed, 12 Feb 2025 14:37:56 +0000 Subject: [PATCH 28/65] Add log message when worker lock timeouts get large (#18124) This is to help track down a possible, but very rare, worker deadlock that was seen on matrix.org. In theory, you could work back from an instance of these new logs to the approximate time when the lock was obtained and focus the diagnostic efforts there. ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --- changelog.d/18124.misc | 1 + synapse/handlers/worker_lock.py | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 changelog.d/18124.misc diff --git a/changelog.d/18124.misc b/changelog.d/18124.misc new file mode 100644 index 0000000000..8ac6a73a20 --- /dev/null +++ b/changelog.d/18124.misc @@ -0,0 +1 @@ +Add log message when worker lock timeouts get large. diff --git a/synapse/handlers/worker_lock.py b/synapse/handlers/worker_lock.py index db998f6701..e58a416026 100644 --- a/synapse/handlers/worker_lock.py +++ b/synapse/handlers/worker_lock.py @@ -19,6 +19,7 @@ # # +import logging import random from types import TracebackType from typing import ( @@ -269,6 +270,10 @@ class WaitingLock: def _get_next_retry_interval(self) -> float: next = self._retry_interval self._retry_interval = max(5, next * 2) + if self._retry_interval > 5 * 2 ^ 7: # ~10 minutes + logging.warning( + f"Lock timeout is getting excessive: {self._retry_interval}s. There may be a deadlock." + ) return next * random.uniform(0.9, 1.1) @@ -344,4 +349,8 @@ class WaitingMultiLock: def _get_next_retry_interval(self) -> float: next = self._retry_interval self._retry_interval = max(5, next * 2) + if self._retry_interval > 5 * 2 ^ 7: # ~10 minutes + logging.warning( + f"Lock timeout is getting excessive: {self._retry_interval}s. There may be a deadlock." + ) return next * random.uniform(0.9, 1.1) From 0b1830b121510c04adf856f252bcc2839c35d738 Mon Sep 17 00:00:00 2001 From: Eric Eastwood Date: Wed, 12 Feb 2025 11:16:06 -0600 Subject: [PATCH 29/65] Document missing server config options (#18122) I was looking into the `USE_FROZEN_DICTS` option during the review of https://github.com/element-hq/synapse/pull/18103#discussion_r1935876168 and noticed that there are several other server config options that aren't in the docs. --- changelog.d/18122.doc | 1 + .../configuration/config_documentation.md | 58 +++++++++++++++++++ 2 files changed, 59 insertions(+) create mode 100644 changelog.d/18122.doc diff --git a/changelog.d/18122.doc b/changelog.d/18122.doc new file mode 100644 index 0000000000..b7bb7107e2 --- /dev/null +++ b/changelog.d/18122.doc @@ -0,0 +1 @@ +Document missing server config options (`daemonize`, `print_pidfile`, `user_agent_suffix`, `use_frozen_dicts`, `manhole`). diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index a5f23149ec..fdca48caf0 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -162,6 +162,53 @@ Example configuration: pid_file: DATADIR/homeserver.pid ``` --- +### `daemonize` + +Specifies whether Synapse should be started as a daemon process. If Synapse is being +managed by [systemd](../../systemd-with-workers/), this option must be omitted or set to +`false`. + +This can also be set by the `--daemonize` (`-D`) argument when starting Synapse. + +See `worker_daemonize` for more information on daemonizing workers. + +Example configuration: +```yaml +daemonize: true +``` +--- +### `print_pidfile` + +Print the path to the pidfile just before daemonizing. Defaults to false. + +This can also be set by the `--print-pidfile` argument when starting Synapse. + +Example configuration: +```yaml +print_pidfile: true +``` +--- +### `user_agent_suffix` + +A suffix that is appended to the Synapse user-agent (ex. `Synapse/v1.123.0`). Defaults +to None + +Example configuration: +```yaml +user_agent_suffix: " (I'm a teapot; Linux x86_64)" +``` +--- +### `use_frozen_dicts` + +Determines whether we should freeze the internal dict object in `FrozenEvent`. Freezing +prevents bugs where we accidentally share e.g. signature dicts. However, freezing a +dict is expensive. Defaults to false. + +Example configuration: +```yaml +use_frozen_dicts: true +``` +--- ### `web_client_location` The absolute URL to the web client which `/` will redirect to. Defaults to none. @@ -595,6 +642,17 @@ listeners: - names: [client, federation] ``` +--- +### `manhole` + +Turn on the Twisted telnet manhole service on the given port. Defaults to none. + +This can also be set by the `--manhole` argument when starting Synapse. + +Example configuration: +```yaml +manhole: 1234 +``` --- ### `manhole_settings` From 74a70190ab3bc17aa7f31d00daa98444f7577b61 Mon Sep 17 00:00:00 2001 From: Andrew Morgan <1342360+anoadragon453@users.noreply.github.com> Date: Fri, 14 Feb 2025 17:23:40 +0700 Subject: [PATCH 30/65] Update `rc_presence` config docs with int burst_count (#18159) --- changelog.d/18159.doc | 1 + docs/usage/configuration/config_documentation.md | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) create mode 100644 changelog.d/18159.doc diff --git a/changelog.d/18159.doc b/changelog.d/18159.doc new file mode 100644 index 0000000000..031f8a149f --- /dev/null +++ b/changelog.d/18159.doc @@ -0,0 +1 @@ +Make `burst_count` field an integer in `rc_presence` config documentation example. \ No newline at end of file diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index fdca48caf0..92ab6438c1 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -1944,7 +1944,7 @@ Example configuration: rc_presence: per_user: per_second: 0.05 - burst_count: 0.5 + burst_count: 1 ``` --- ### `federation_rr_transactions_per_room_per_second` From 22bb3c50d1c10e6687ea4346fb848b7c160441d0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 17 Feb 2025 14:08:11 +0000 Subject: [PATCH 31/65] Bump twine from 6.0.1 to 6.1.0 (#18170) --- poetry.lock | 43 ++++++++++++++++++++++++------------------- 1 file changed, 24 insertions(+), 19 deletions(-) diff --git a/poetry.lock b/poetry.lock index 7dee7d098f..3f9f940fd1 100644 --- a/poetry.lock +++ b/poetry.lock @@ -603,6 +603,25 @@ files = [ [package.dependencies] idna = ">=2.5" +[[package]] +name = "id" +version = "1.5.0" +description = "A tool for generating OIDC identities" +optional = false +python-versions = ">=3.8" +files = [ + {file = "id-1.5.0-py3-none-any.whl", hash = "sha256:f1434e1cef91f2cbb8a4ec64663d5a23b9ed43ef44c4c957d02583d61714c658"}, + {file = "id-1.5.0.tar.gz", hash = "sha256:292cb8a49eacbbdbce97244f47a97b4c62540169c976552e497fd57df0734c1d"}, +] + +[package.dependencies] +requests = "*" + +[package.extras] +dev = ["build", "bump (>=1.3.2)", "id[lint,test]"] +lint = ["bandit", "interrogate", "mypy", "ruff (<0.8.2)", "types-requests"] +test = ["coverage[toml]", "pretend", "pytest", "pytest-cov"] + [[package]] name = "idna" version = "3.10" @@ -1544,20 +1563,6 @@ tests = ["check-manifest", "coverage (>=7.4.2)", "defusedxml", "markdown2", "ole typing = ["typing-extensions"] xmp = ["defusedxml"] -[[package]] -name = "pkginfo" -version = "1.9.6" -description = "Query metadata from sdists / bdists / installed packages." -optional = false -python-versions = ">=3.6" -files = [ - {file = "pkginfo-1.9.6-py3-none-any.whl", hash = "sha256:4b7a555a6d5a22169fcc9cf7bfd78d296b0361adad412a346c1226849af5e546"}, - {file = "pkginfo-1.9.6.tar.gz", hash = "sha256:8fd5896e8718a4372f0ea9cc9d96f6417c9b986e23a4d116dda26b62cc29d046"}, -] - -[package.extras] -testing = ["pytest", "pytest-cov"] - [[package]] name = "prometheus-client" version = "0.21.0" @@ -2625,20 +2630,20 @@ docs = ["sphinx (<7.0.0)"] [[package]] name = "twine" -version = "6.0.1" +version = "6.1.0" description = "Collection of utilities for publishing packages on PyPI" optional = false python-versions = ">=3.8" files = [ - {file = "twine-6.0.1-py3-none-any.whl", hash = "sha256:9c6025b203b51521d53e200f4a08b116dee7500a38591668c6a6033117bdc218"}, - {file = "twine-6.0.1.tar.gz", hash = "sha256:36158b09df5406e1c9c1fb8edb24fc2be387709443e7376689b938531582ee27"}, + {file = "twine-6.1.0-py3-none-any.whl", hash = "sha256:a47f973caf122930bf0fbbf17f80b83bc1602c9ce393c7845f289a3001dc5384"}, + {file = "twine-6.1.0.tar.gz", hash = "sha256:be324f6272eff91d07ee93f251edf232fc647935dd585ac003539b42404a8dbd"}, ] [package.dependencies] +id = "*" importlib-metadata = {version = ">=3.6", markers = "python_version < \"3.10\""} keyring = {version = ">=15.1", markers = "platform_machine != \"ppc64le\" and platform_machine != \"s390x\""} -packaging = "*" -pkginfo = ">=1.8.1" +packaging = ">=24.0" readme-renderer = ">=35.0" requests = ">=2.20" requests-toolbelt = ">=0.8.0,<0.9.0 || >0.9.0" From 8ae9d9e8c547293889b9198b62f7c5bf87693f0d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 17 Feb 2025 14:08:21 +0000 Subject: [PATCH 32/65] Bump service-identity from 24.1.0 to 24.2.0 (#18171) --- poetry.lock | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/poetry.lock b/poetry.lock index 3f9f940fd1..dfd9a838ce 100644 --- a/poetry.lock +++ b/poetry.lock @@ -2376,13 +2376,13 @@ tornado = ["tornado (>=6)"] [[package]] name = "service-identity" -version = "24.1.0" +version = "24.2.0" description = "Service identity verification for pyOpenSSL & cryptography." optional = false python-versions = ">=3.8" files = [ - {file = "service_identity-24.1.0-py3-none-any.whl", hash = "sha256:a28caf8130c8a5c1c7a6f5293faaf239bbfb7751e4862436920ee6f2616f568a"}, - {file = "service_identity-24.1.0.tar.gz", hash = "sha256:6829c9d62fb832c2e1c435629b0a8c476e1929881f28bee4d20bc24161009221"}, + {file = "service_identity-24.2.0-py3-none-any.whl", hash = "sha256:6b047fbd8a84fd0bb0d55ebce4031e400562b9196e1e0d3e0fe2b8a59f6d4a85"}, + {file = "service_identity-24.2.0.tar.gz", hash = "sha256:b8683ba13f0d39c6cd5d625d2c5f65421d6d707b013b375c355751557cbe8e09"}, ] [package.dependencies] @@ -2392,7 +2392,7 @@ pyasn1 = "*" pyasn1-modules = "*" [package.extras] -dev = ["pyopenssl", "service-identity[idna,mypy,tests]"] +dev = ["coverage[toml] (>=5.0.2)", "idna", "mypy", "pyopenssl", "pytest", "types-pyopenssl"] docs = ["furo", "myst-parser", "pyopenssl", "sphinx", "sphinx-notfound-page"] idna = ["idna"] mypy = ["idna", "mypy", "types-pyopenssl"] From 3e34f5ccc706e443d70791e46748520f77dc65a1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 17 Feb 2025 14:08:51 +0000 Subject: [PATCH 33/65] Bump hiredis from 3.0.0 to 3.1.0 (#18169) --- poetry.lock | 205 ++++++++++++++++++++++++++++------------------------ 1 file changed, 110 insertions(+), 95 deletions(-) diff --git a/poetry.lock b/poetry.lock index dfd9a838ce..0d388eff6a 100644 --- a/poetry.lock +++ b/poetry.lock @@ -488,105 +488,120 @@ test = ["coverage[toml]", "ddt (>=1.1.1,!=1.4.3)", "mock", "mypy", "pre-commit", [[package]] name = "hiredis" -version = "3.0.0" +version = "3.1.0" description = "Python wrapper for hiredis" optional = true python-versions = ">=3.8" files = [ - {file = "hiredis-3.0.0-cp310-cp310-macosx_10_15_universal2.whl", hash = "sha256:4b182791c41c5eb1d9ed736f0ff81694b06937ca14b0d4dadde5dadba7ff6dae"}, - {file = "hiredis-3.0.0-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:13c275b483a052dd645eb2cb60d6380f1f5215e4c22d6207e17b86be6dd87ffa"}, - {file = "hiredis-3.0.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:c1018cc7f12824506f165027eabb302735b49e63af73eb4d5450c66c88f47026"}, - {file = "hiredis-3.0.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:83a29cc7b21b746cb6a480189e49f49b2072812c445e66a9e38d2004d496b81c"}, - {file = "hiredis-3.0.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e241fab6332e8fb5f14af00a4a9c6aefa22f19a336c069b7ddbf28ef8341e8d6"}, - {file = "hiredis-3.0.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1fb8de899f0145d6c4d5d4bd0ee88a78eb980a7ffabd51e9889251b8f58f1785"}, - {file = "hiredis-3.0.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b23291951959141173eec10f8573538e9349fa27f47a0c34323d1970bf891ee5"}, - {file = "hiredis-3.0.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e421ac9e4b5efc11705a0d5149e641d4defdc07077f748667f359e60dc904420"}, - {file = "hiredis-3.0.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:77c8006c12154c37691b24ff293c077300c22944018c3ff70094a33e10c1d795"}, - {file = "hiredis-3.0.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:41afc0d3c18b59eb50970479a9c0e5544fb4b95e3a79cf2fbaece6ddefb926fe"}, - {file = "hiredis-3.0.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:04ccae6dcd9647eae6025425ab64edb4d79fde8b9e6e115ebfabc6830170e3b2"}, - {file = "hiredis-3.0.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:fe91d62b0594db5ea7d23fc2192182b1a7b6973f628a9b8b2e0a42a2be721ac6"}, - {file = "hiredis-3.0.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:99516d99316062824a24d145d694f5b0d030c80da693ea6f8c4ecf71a251d8bb"}, - {file = "hiredis-3.0.0-cp310-cp310-win32.whl", hash = "sha256:562eaf820de045eb487afaa37e6293fe7eceb5b25e158b5a1974b7e40bf04543"}, - {file = "hiredis-3.0.0-cp310-cp310-win_amd64.whl", hash = "sha256:a1c81c89ed765198da27412aa21478f30d54ef69bf5e4480089d9c3f77b8f882"}, - {file = "hiredis-3.0.0-cp311-cp311-macosx_10_15_universal2.whl", hash = "sha256:4664dedcd5933364756d7251a7ea86d60246ccf73a2e00912872dacbfcef8978"}, - {file = "hiredis-3.0.0-cp311-cp311-macosx_10_15_x86_64.whl", hash = "sha256:47de0bbccf4c8a9f99d82d225f7672b9dd690d8fd872007b933ef51a302c9fa6"}, - {file = "hiredis-3.0.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e43679eca508ba8240d016d8cca9d27342d70184773c15bea78a23c87a1922f1"}, - {file = "hiredis-3.0.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:13c345e7278c210317e77e1934b27b61394fee0dec2e8bd47e71570900f75823"}, - {file = "hiredis-3.0.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:00018f22f38530768b73ea86c11f47e8d4df65facd4e562bd78773bd1baef35e"}, - {file = "hiredis-3.0.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4ea3a86405baa8eb0d3639ced6926ad03e07113de54cb00fd7510cb0db76a89d"}, - {file = "hiredis-3.0.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c073848d2b1d5561f3903879ccf4e1a70c9b1e7566c7bdcc98d082fa3e7f0a1d"}, - {file = "hiredis-3.0.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5a8dffb5f5b3415a4669d25de48b617fd9d44b0bccfc4c2ab24b06406ecc9ecb"}, - {file = "hiredis-3.0.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:22c17c96143c2a62dfd61b13803bc5de2ac526b8768d2141c018b965d0333b66"}, - {file = "hiredis-3.0.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:c3ece960008dab66c6b8bb3a1350764677ee7c74ccd6270aaf1b1caf9ccebb46"}, - {file = "hiredis-3.0.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:f75999ae00a920f7dce6ecae76fa5e8674a3110e5a75f12c7a2c75ae1af53396"}, - {file = "hiredis-3.0.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:e069967cbd5e1900aafc4b5943888f6d34937fc59bf8918a1a546cb729b4b1e4"}, - {file = "hiredis-3.0.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:0aacc0a78e1d94d843a6d191f224a35893e6bdfeb77a4a89264155015c65f126"}, - {file = "hiredis-3.0.0-cp311-cp311-win32.whl", hash = "sha256:719c32147ba29528cb451f037bf837dcdda4ff3ddb6cdb12c4216b0973174718"}, - {file = "hiredis-3.0.0-cp311-cp311-win_amd64.whl", hash = "sha256:bdc144d56333c52c853c31b4e2e52cfbdb22d3da4374c00f5f3d67c42158970f"}, - {file = "hiredis-3.0.0-cp312-cp312-macosx_10_15_universal2.whl", hash = "sha256:484025d2eb8f6348f7876fc5a2ee742f568915039fcb31b478fd5c242bb0fe3a"}, - {file = "hiredis-3.0.0-cp312-cp312-macosx_10_15_x86_64.whl", hash = "sha256:fcdb552ffd97151dab8e7bc3ab556dfa1512556b48a367db94b5c20253a35ee1"}, - {file = "hiredis-3.0.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:0bb6f9fd92f147ba11d338ef5c68af4fd2908739c09e51f186e1d90958c68cc1"}, - {file = "hiredis-3.0.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fa86bf9a0ed339ec9e8a9a9d0ae4dccd8671625c83f9f9f2640729b15e07fbfd"}, - {file = "hiredis-3.0.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e194a0d5df9456995d8f510eab9f529213e7326af6b94770abf8f8b7952ddcaa"}, - {file = "hiredis-3.0.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c8a1df39d74ec507d79c7a82c8063eee60bf80537cdeee652f576059b9cdd15c"}, - {file = "hiredis-3.0.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f91456507427ba36fd81b2ca11053a8e112c775325acc74e993201ea912d63e9"}, - {file = "hiredis-3.0.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9862db92ef67a8a02e0d5370f07d380e14577ecb281b79720e0d7a89aedb9ee5"}, - {file = "hiredis-3.0.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:d10fcd9e0eeab835f492832b2a6edb5940e2f1230155f33006a8dfd3bd2c94e4"}, - {file = "hiredis-3.0.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:48727d7d405d03977d01885f317328dc21d639096308de126c2c4e9950cbd3c9"}, - {file = "hiredis-3.0.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:8e0bb6102ebe2efecf8a3292c6660a0e6fac98176af6de67f020bea1c2343717"}, - {file = "hiredis-3.0.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:df274e3abb4df40f4c7274dd3e587dfbb25691826c948bc98d5fead019dfb001"}, - {file = "hiredis-3.0.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:034925b5fb514f7b11aac38cd55b3fd7e9d3af23bd6497f3f20aa5b8ba58e232"}, - {file = "hiredis-3.0.0-cp312-cp312-win32.whl", hash = "sha256:120f2dda469b28d12ccff7c2230225162e174657b49cf4cd119db525414ae281"}, - {file = "hiredis-3.0.0-cp312-cp312-win_amd64.whl", hash = "sha256:e584fe5f4e6681d8762982be055f1534e0170f6308a7a90f58d737bab12ff6a8"}, - {file = "hiredis-3.0.0-cp38-cp38-macosx_10_15_universal2.whl", hash = "sha256:122171ff47d96ed8dd4bba6c0e41d8afaba3e8194949f7720431a62aa29d8895"}, - {file = "hiredis-3.0.0-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:ba9fc605ac558f0de67463fb588722878641e6fa1dabcda979e8e69ff581d0bd"}, - {file = "hiredis-3.0.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:a631e2990b8be23178f655cae8ac6c7422af478c420dd54e25f2e26c29e766f1"}, - {file = "hiredis-3.0.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:63482db3fadebadc1d01ad33afa6045ebe2ea528eb77ccaabd33ee7d9c2bad48"}, - {file = "hiredis-3.0.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1f669212c390eebfbe03c4e20181f5970b82c5d0a0ad1df1785f7ffbe7d61150"}, - {file = "hiredis-3.0.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a6a49ef161739f8018c69b371528bdb47d7342edfdee9ddc75a4d8caddf45a6e"}, - {file = "hiredis-3.0.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:98a152052b8878e5e43a2e3a14075218adafc759547c98668a21e9485882696c"}, - {file = "hiredis-3.0.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:50a196af0ce657fcde9bf8a0bbe1032e22c64d8fcec2bc926a35e7ff68b3a166"}, - {file = "hiredis-3.0.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:f2f312eef8aafc2255e3585dcf94d5da116c43ef837db91db9ecdc1bc930072d"}, - {file = "hiredis-3.0.0-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:6ca41fa40fa019cde42c21add74aadd775e71458051a15a352eabeb12eb4d084"}, - {file = "hiredis-3.0.0-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:6eecb343c70629f5af55a8b3e53264e44fa04e155ef7989de13668a0cb102a90"}, - {file = "hiredis-3.0.0-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:c3fdad75e7837a475900a1d3a5cc09aa024293c3b0605155da2d42f41bc0e482"}, - {file = "hiredis-3.0.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:8854969e7480e8d61ed7549eb232d95082a743e94138d98d7222ba4e9f7ecacd"}, - {file = "hiredis-3.0.0-cp38-cp38-win32.whl", hash = "sha256:f114a6c86edbf17554672b050cce72abf489fe58d583c7921904d5f1c9691605"}, - {file = "hiredis-3.0.0-cp38-cp38-win_amd64.whl", hash = "sha256:7d99b91e42217d7b4b63354b15b41ce960e27d216783e04c4a350224d55842a4"}, - {file = "hiredis-3.0.0-cp39-cp39-macosx_10_15_universal2.whl", hash = "sha256:4c6efcbb5687cf8d2aedcc2c3ed4ac6feae90b8547427d417111194873b66b06"}, - {file = "hiredis-3.0.0-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:5b5cff42a522a0d81c2ae7eae5e56d0ee7365e0c4ad50c4de467d8957aff4414"}, - {file = "hiredis-3.0.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:82f794d564f4bc76b80c50b03267fe5d6589e93f08e66b7a2f674faa2fa76ebc"}, - {file = "hiredis-3.0.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d7a4c1791d7aa7e192f60fe028ae409f18ccdd540f8b1e6aeb0df7816c77e4a4"}, - {file = "hiredis-3.0.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a2537b2cd98192323fce4244c8edbf11f3cac548a9d633dbbb12b48702f379f4"}, - {file = "hiredis-3.0.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8fed69bbaa307040c62195a269f82fc3edf46b510a17abb6b30a15d7dab548df"}, - {file = "hiredis-3.0.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:869f6d5537d243080f44253491bb30aa1ec3c21754003b3bddeadedeb65842b0"}, - {file = "hiredis-3.0.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d435ae89073d7cd51e6b6bf78369c412216261c9c01662e7008ff00978153729"}, - {file = "hiredis-3.0.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:204b79b30a0e6be0dc2301a4d385bb61472809f09c49f400497f1cdd5a165c66"}, - {file = "hiredis-3.0.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:3ea635101b739c12effd189cc19b2671c268abb03013fd1f6321ca29df3ca625"}, - {file = "hiredis-3.0.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:f359175197fd833c8dd7a8c288f1516be45415bb5c939862ab60c2918e1e1943"}, - {file = "hiredis-3.0.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:ac6d929cb33dd12ad3424b75725975f0a54b5b12dbff95f2a2d660c510aa106d"}, - {file = "hiredis-3.0.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:100431e04d25a522ef2c3b94f294c4219c4de3bfc7d557b6253296145a144c11"}, - {file = "hiredis-3.0.0-cp39-cp39-win32.whl", hash = "sha256:e1a9c14ae9573d172dc050a6f63a644457df5d01ec4d35a6a0f097f812930f83"}, - {file = "hiredis-3.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:54a6dd7b478e6eb01ce15b3bb5bf771e108c6c148315bf194eb2ab776a3cac4d"}, - {file = "hiredis-3.0.0-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:50da7a9edf371441dfcc56288d790985ee9840d982750580710a9789b8f4a290"}, - {file = "hiredis-3.0.0-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:9b285ef6bf1581310b0d5e8f6ce64f790a1c40e89c660e1320b35f7515433672"}, - {file = "hiredis-3.0.0-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0dcfa684966f25b335072115de2f920228a3c2caf79d4bfa2b30f6e4f674a948"}, - {file = "hiredis-3.0.0-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a41be8af1fd78ca97bc948d789a09b730d1e7587d07ca53af05758f31f4b985d"}, - {file = "hiredis-3.0.0-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:038756db735e417ab36ee6fd7725ce412385ed2bd0767e8179a4755ea11b804f"}, - {file = "hiredis-3.0.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:fcecbd39bd42cef905c0b51c9689c39d0cc8b88b1671e7f40d4fb213423aef3a"}, - {file = "hiredis-3.0.0-pp38-pypy38_pp73-macosx_10_15_x86_64.whl", hash = "sha256:a131377493a59fb0f5eaeb2afd49c6540cafcfba5b0b3752bed707be9e7c4eaf"}, - {file = "hiredis-3.0.0-pp38-pypy38_pp73-macosx_11_0_arm64.whl", hash = "sha256:3d22c53f0ec5c18ecb3d92aa9420563b1c5d657d53f01356114978107b00b860"}, - {file = "hiredis-3.0.0-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c8a91e9520fbc65a799943e5c970ffbcd67905744d8becf2e75f9f0a5e8414f0"}, - {file = "hiredis-3.0.0-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3dc8043959b50141df58ab4f398e8ae84c6f9e673a2c9407be65fc789138f4a6"}, - {file = "hiredis-3.0.0-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:51b99cfac514173d7b8abdfe10338193e8a0eccdfe1870b646009d2fb7cbe4b5"}, - {file = "hiredis-3.0.0-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:fa1fcad89d8a41d8dc10b1e54951ec1e161deabd84ed5a2c95c3c7213bdb3514"}, - {file = "hiredis-3.0.0-pp39-pypy39_pp73-macosx_10_15_x86_64.whl", hash = "sha256:898636a06d9bf575d2c594129085ad6b713414038276a4bfc5db7646b8a5be78"}, - {file = "hiredis-3.0.0-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:466f836dbcf86de3f9692097a7a01533dc9926986022c6617dc364a402b265c5"}, - {file = "hiredis-3.0.0-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:23142a8af92a13fc1e3f2ca1d940df3dcf2af1d176be41fe8d89e30a837a0b60"}, - {file = "hiredis-3.0.0-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:793c80a3d6b0b0e8196a2d5de37a08330125668c8012922685e17aa9108c33ac"}, - {file = "hiredis-3.0.0-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:467d28112c7faa29b7db743f40803d927c8591e9da02b6ce3d5fadc170a542a2"}, - {file = "hiredis-3.0.0-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:dc384874a719c767b50a30750f937af18842ee5e288afba95a5a3ed703b1515a"}, - {file = "hiredis-3.0.0.tar.gz", hash = "sha256:fed8581ae26345dea1f1e0d1a96e05041a727a45e7d8d459164583e23c6ac441"}, + {file = "hiredis-3.1.0-cp310-cp310-macosx_10_15_universal2.whl", hash = "sha256:2892db9db21f0cf7cc298d09f85d3e1f6dc4c4c24463ab67f79bc7a006d51867"}, + {file = "hiredis-3.1.0-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:93cfa6cc25ee2ceb0be81dc61eca9995160b9e16bdb7cca4a00607d57e998918"}, + {file = "hiredis-3.1.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2af62070aa9433802cae7be7364d5e82f76462c6a2ae34e53008b637aaa9a156"}, + {file = "hiredis-3.1.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:072c162260ebb1d892683107da22d0d5da7a1414739eae4e185cac22fe89627f"}, + {file = "hiredis-3.1.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c6b232c43e89755ba332c2745ddab059c0bc1a0f01448a3a14d506f8448b1ce6"}, + {file = "hiredis-3.1.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:eb5316c9a65c4dde80796aa245b76011bab64eb84461a77b0a61c1bf2970bcc9"}, + {file = "hiredis-3.1.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e812a4e656bbd1c1c15c844b28259c49e26bb384837e44e8d2aa55412c91d2f7"}, + {file = "hiredis-3.1.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:93a6c9230e5a5565847130c0e1005c8d3aa5ca681feb0ed542c4651323d32feb"}, + {file = "hiredis-3.1.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:a5f65e89ce50a94d9490d5442a649c6116f53f216c8c14eb37cf9637956482b2"}, + {file = "hiredis-3.1.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:9b2d6e33601c67c074c367fdccdd6033e642284e7a56adc130f18f724c378ca8"}, + {file = "hiredis-3.1.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:bad3b1e0c83849910f28c95953417106f539277035a4b515d1425f93947bc28f"}, + {file = "hiredis-3.1.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:9646de31f5994e6218311dcf216e971703dbf804c510fd3f84ddb9813c495824"}, + {file = "hiredis-3.1.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:59a9230f3aa38a33d09d8171400de202f575d7a38869e5ce2947829bca6fe359"}, + {file = "hiredis-3.1.0-cp310-cp310-win32.whl", hash = "sha256:0322d70f3328b97da14b6e98b18f0090a12ed8a8bf7ae20932e2eb9d1bb0aa2c"}, + {file = "hiredis-3.1.0-cp310-cp310-win_amd64.whl", hash = "sha256:802474c18e878b3f9905e160a8b7df87d57885758083eda76c5978265acb41aa"}, + {file = "hiredis-3.1.0-cp311-cp311-macosx_10_15_universal2.whl", hash = "sha256:c339ff4b4739b2a40da463763dd566129762f72926bca611ad9a457a9fe64abd"}, + {file = "hiredis-3.1.0-cp311-cp311-macosx_10_15_x86_64.whl", hash = "sha256:0ffa2552f704a45954627697a378fc2f559004e53055b82f00daf30bd4305330"}, + {file = "hiredis-3.1.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9acf7f0e7106f631cd618eb60ec9bbd6e43045addd5310f66ba1177209567e59"}, + {file = "hiredis-3.1.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ea4f5ecf9dbea93c827486f59c606684c3496ea71c7ba9a8131932780696e61a"}, + {file = "hiredis-3.1.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:39efab176fca3d5111075f6ba56cd864f18db46d858289d39360c5672e0e5c3e"}, + {file = "hiredis-3.1.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1110eae007f30e70a058d743e369c24430327cd01fd97d99519d6794a58dd587"}, + {file = "hiredis-3.1.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9b390f63191bcccbb6044d4c118acdf4fa55f38e5658ac4cfd5a33a6f0c07659"}, + {file = "hiredis-3.1.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:72a98ccc7b8ec9ce0100ecf59f45f05d2023606e8e3676b07a316d1c1c364072"}, + {file = "hiredis-3.1.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:7c76e751fd1e2f221dec09cdc24040ee486886e943d5d7ffc256e8cf15c75e51"}, + {file = "hiredis-3.1.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:7d3880f213b6f14e9c69ce52beffd1748eecc8669698c4782761887273b6e1bd"}, + {file = "hiredis-3.1.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:87c2b3fe7e7c96eba376506a76e11514e07e848f737b254e0973e4b5c3a491e9"}, + {file = "hiredis-3.1.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:d3cfb4089e96f8f8ee9554da93148a9261aa6612ad2cc202c1a494c7b712e31f"}, + {file = "hiredis-3.1.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:4f12018e5c5f866a1c3f7017cb2d88e5c6f9440df2281e48865a2b6c40f247f4"}, + {file = "hiredis-3.1.0-cp311-cp311-win32.whl", hash = "sha256:107b66ce977bb2dff8f2239e68344360a75d05fed3d9fa0570ac4d3020ce2396"}, + {file = "hiredis-3.1.0-cp311-cp311-win_amd64.whl", hash = "sha256:8f1240bde53d3d1676f0aba61b3661560dc9a681cae24d9de33e650864029aa4"}, + {file = "hiredis-3.1.0-cp312-cp312-macosx_10_15_universal2.whl", hash = "sha256:f7c7f89e0bc4246115754e2eda078a111282f6d6ecc6fb458557b724fe6f2aac"}, + {file = "hiredis-3.1.0-cp312-cp312-macosx_10_15_x86_64.whl", hash = "sha256:3dbf9163296fa45fbddcfc4c5900f10e9ddadda37117dbfb641e327e536b53e0"}, + {file = "hiredis-3.1.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:af46a4be0e82df470f68f35316fa16cd1e134d1c5092fc1082e1aad64cce716d"}, + {file = "hiredis-3.1.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:bc63d698c43aea500a84d8b083f830c03808b6cf3933ae4d35a27f0a3d881652"}, + {file = "hiredis-3.1.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:676b3d88674134bfaaf70dac181d1790b0f33b3187bfb9da9221e17e0e624f83"}, + {file = "hiredis-3.1.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:aed10d9df1e2fb0011db2713ac64497462e9c2c0208b648c97569da772b959ca"}, + {file = "hiredis-3.1.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b5bd8adfe8742e331a94cccd782bffea251fa70d9a709e71f4510f50794d700"}, + {file = "hiredis-3.1.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9fc4e35b4afb0af6da55495dd0742ad32ab88150428a6ecdbb3085cbd60714e8"}, + {file = "hiredis-3.1.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:89b83e76eb00ab0464e7b0752a3ffcb02626e742e9509bc141424a9c3202e8dc"}, + {file = "hiredis-3.1.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:98ebf08c907836b70a8f40e030df8ab6f174dc7f6fa765251d813e89f14069d8"}, + {file = "hiredis-3.1.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:6c840b9cec086328f2ee2cfee0038b5d6bbb514bac7b5e579da6e346eaac056c"}, + {file = "hiredis-3.1.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:c5c44e9fa6f4462d0330cb5f5d46fa652512fc86b41d4d1974d0356f263e9105"}, + {file = "hiredis-3.1.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:e665b14ab50aa175cfa306fcb00fffd4e3ff02ceb36ca6a4df00b1246d6a73c4"}, + {file = "hiredis-3.1.0-cp312-cp312-win32.whl", hash = "sha256:bd33db977ac7af97e8d035ffadb163b00546be22e5f1297b2123f5f9bf0f8a21"}, + {file = "hiredis-3.1.0-cp312-cp312-win_amd64.whl", hash = "sha256:37aed4aa9348600145e2d019c7be27855e503ecc4906c6976ff2f3b52e3d5d97"}, + {file = "hiredis-3.1.0-cp313-cp313-macosx_10_15_universal2.whl", hash = "sha256:b87cddd8107487863fed6994de51e5594a0be267b0b19e213694e99cdd614623"}, + {file = "hiredis-3.1.0-cp313-cp313-macosx_10_15_x86_64.whl", hash = "sha256:d302deff8cb63a7feffc1844e4dafc8076e566bbf10c5aaaf0f4fe791b8a6bd0"}, + {file = "hiredis-3.1.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:4a018340c073cf88cb635b2bedff96619df2f666018c655e7911f46fa2c1c178"}, + {file = "hiredis-3.1.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f1e8ba6414ac1ae536129e18c069f3eb497df5a74e136e3566471620a4fa5f95"}, + {file = "hiredis-3.1.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a86b9fef256c2beb162244791fdc025aa55f936d6358e86e2020e512fe2e4972"}, + {file = "hiredis-3.1.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7acdc68e29a446ad17aadaff19c981a36b3bd8c894c3520412c8a7ab1c3e0de7"}, + {file = "hiredis-3.1.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c7e06baea05de57e1e7548064f505a6964e992674fe61b8f274afe2ac93b6371"}, + {file = "hiredis-3.1.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:35b5fc061c8a0dbfdb440053280504d6aaa8d9726bd4d1d0e1cfcbbdf0d60b73"}, + {file = "hiredis-3.1.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:c89d2dcb271d24c44f02264233b75d5db8c58831190fa92456a90b87fa17b748"}, + {file = "hiredis-3.1.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:aa36688c10a08f626fddcf68c2b1b91b0e90b070c26e550a4151a877f5c2d431"}, + {file = "hiredis-3.1.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:f3982a9c16c1c4bc05a00b65d01ffb8d80ea1a7b6b533be2f1a769d3e989d2c0"}, + {file = "hiredis-3.1.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:d1a6f889514ee2452300c9a06862fceedef22a2891f1c421a27b1ba52ef130b2"}, + {file = "hiredis-3.1.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:8a45ff7915392a55d9386bb235ea1d1eb9960615f301979f02143fc20036b699"}, + {file = "hiredis-3.1.0-cp313-cp313-win32.whl", hash = "sha256:539e5bb725b62b76a5319a4e68fc7085f01349abc2316ef3df608ea0883c51d2"}, + {file = "hiredis-3.1.0-cp313-cp313-win_amd64.whl", hash = "sha256:9020fd7e58f489fda6a928c31355add0e665fd6b87b21954e675cf9943eafa32"}, + {file = "hiredis-3.1.0-cp38-cp38-macosx_10_15_universal2.whl", hash = "sha256:b621a89fc29b3f4b01be6640ec81a6a94b5382bc78fecb876408d57a071e45aa"}, + {file = "hiredis-3.1.0-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:363e21fba55e1a26349dc9ca7da6b14332123879b6359bcee4a9acecb40ca33b"}, + {file = "hiredis-3.1.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:c156156798729eadc9ab76ffee96c88b93cc1c3b493f4dd0a4341f53939194ee"}, + {file = "hiredis-3.1.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e38d8a325f9a6afac1b1c72d996d1add9e1b99696ce9410538ba5e9aa8fdba02"}, + {file = "hiredis-3.1.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3004ef7436feb7bfa61c0b36d422b8fb8c29aaa1a514c9405f0fdee5e9694dd3"}, + {file = "hiredis-3.1.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:13f5b16f97d0bbd1c04ce367c49097d1214d60e11f9fee7ef2a9b54e0a6645c8"}, + {file = "hiredis-3.1.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:230dd0e77cb0f525f58a1306a7b4aaf078037fc5229110922332ca46f90821bb"}, + {file = "hiredis-3.1.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d968116caddd19d63120d1298e62b1bbc694db3360ed0d5df8c3a97edbc12552"}, + {file = "hiredis-3.1.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:511e36a6fa41d3efab3cd5cd70ac388ed825993b9e66fa3b0e47cf27a2f5ffee"}, + {file = "hiredis-3.1.0-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:c5cd20804e3cb0d31e7d899d8dd091f569c33fe40d4bade670a067ab7d31c2ac"}, + {file = "hiredis-3.1.0-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:09e89e7d34cfe5ca8f7a869fca827d1af0afe8aaddb26b38c01058730edb79ad"}, + {file = "hiredis-3.1.0-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:570cbf31413c77fe5e7c157f2943ca4400493ddd9cf2184731cfcafc753becd7"}, + {file = "hiredis-3.1.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:b9b4da8162cf289781732d6a5ba01d820c42c05943fcdb7de307d03639961db3"}, + {file = "hiredis-3.1.0-cp38-cp38-win32.whl", hash = "sha256:bc117a04bcb461d3bb1b2c5b417aee3442e1e8aa33ebc800481431f4c09fe0c5"}, + {file = "hiredis-3.1.0-cp38-cp38-win_amd64.whl", hash = "sha256:34f3f5f0354db2d6797a6fb08d2c036a50af62a1d919d122c1c784304ef49347"}, + {file = "hiredis-3.1.0-cp39-cp39-macosx_10_15_universal2.whl", hash = "sha256:a26fa888025badb5563f283cc19594c215a413e905729e59a5f7cf3f46d66c32"}, + {file = "hiredis-3.1.0-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:f50763cd819d4a52a47b5966d4bb47dee34b637c5fa6402509800eee6ecb61e6"}, + {file = "hiredis-3.1.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:b6d1c9e1fce5e0a94072667ae2bf0142b89ebbb1917d3531184e060a43f3ee11"}, + {file = "hiredis-3.1.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e38d7a56b1a79ed0bbb9e6fe376d82e3f4dcc646ae47472f2c858e19a597c112"}, + {file = "hiredis-3.1.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:4ef5ad8b91530e4d10a68562b0a380ea22705a60e88cecee086d7c63a38564ce"}, + {file = "hiredis-3.1.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:cf3d2299b054e57a9f97ca08704c2843e44f29b57dc69b76a2592ecd212efe1a"}, + {file = "hiredis-3.1.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:93811d60b0f73d0f049c86f4373a3833b4a38fce374ab151074d929553eb4304"}, + {file = "hiredis-3.1.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:18e703ff860c1d83abbcf57012b309ead02b56b60e85150c6c3bfb37cbb16ebf"}, + {file = "hiredis-3.1.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:f9ea0678806c53d96758e74c6a898f9d506a2e3367a344757f768bef9e069366"}, + {file = "hiredis-3.1.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:cf6844035abf47d52a1c3f4257255af3bf3b0f14d559b08eaa45885418c6c55d"}, + {file = "hiredis-3.1.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:7acf35cfa7ec9e1e7559c04e7095628f7d06049b5f24dcb58c1a55ef6dc689f8"}, + {file = "hiredis-3.1.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:b885695dce7a39b1fd9a609ed9c4cf312e53df2ec028d5a78af7a891b5fbea4d"}, + {file = "hiredis-3.1.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:1c22fa74ddd063396b19fe8445a1ae8b4190eff755d5750dda48e860a45b2ee7"}, + {file = "hiredis-3.1.0-cp39-cp39-win32.whl", hash = "sha256:0614e16339f1784df3bbd2800322e20b4127d3f3a3509f00a5562efddb2521aa"}, + {file = "hiredis-3.1.0-cp39-cp39-win_amd64.whl", hash = "sha256:c2bc713ee73ab9de4a0d68b0ab0f29612342b63173714742437b977584adb2d8"}, + {file = "hiredis-3.1.0-pp310-pypy310_pp73-macosx_10_15_x86_64.whl", hash = "sha256:07ab990d0835f36bf358dbb84db4541ac0a8f533128ec09af8f80a576eef2e88"}, + {file = "hiredis-3.1.0-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:5c54a88eb9d8ebc4e5eefaadbe2102a4f7499f9e413654172f40aefd25350959"}, + {file = "hiredis-3.1.0-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8095ef159896e5999a795b0f80e4d64281301a109e442a8d29cd750ca6bd8303"}, + {file = "hiredis-3.1.0-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0f8ca13e2476ffd6d5be4763f5868133506ddcfa5ce54b4dac231ebdc19be6c6"}, + {file = "hiredis-3.1.0-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:34d25aa25c10f966d5415795ed271da84605044dbf436c054966cea5442451b3"}, + {file = "hiredis-3.1.0-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:4180dc5f646b426e5fa1212e1348c167ee2a864b3a70d56579163d64a847dd1e"}, + {file = "hiredis-3.1.0-pp38-pypy38_pp73-macosx_10_15_x86_64.whl", hash = "sha256:d92144e0cd6e6e841a6ad343e9d58631626eeb4ac96b0322649379b5d4527447"}, + {file = "hiredis-3.1.0-pp38-pypy38_pp73-macosx_11_0_arm64.whl", hash = "sha256:fcb91ba42903de637b94a1b64477f381f94ad82c0742c264f9245be76a7a3cbc"}, + {file = "hiredis-3.1.0-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5ce71a797b5bc02c51da082428c00251ed6a7a67a03acbda5fbf9e8d028725f6"}, + {file = "hiredis-3.1.0-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2e04c7feb9467e3170cd4d5bee381775783d81bbc45d6147c1c0ce3b50dc04f9"}, + {file = "hiredis-3.1.0-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a31806306a60f3565c04c964d6bee0e9d4a5120e1da589e41976b53972edf635"}, + {file = "hiredis-3.1.0-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:bc51f594c2c0863ded6501642dc96701ca8bbea9ced4fa3af0a1aeda8aa634cb"}, + {file = "hiredis-3.1.0-pp39-pypy39_pp73-macosx_10_15_x86_64.whl", hash = "sha256:4663a319ab7d22c597b9421e5ea384fd583e044f2f1ca9a1b98d4fef8a0fea2f"}, + {file = "hiredis-3.1.0-pp39-pypy39_pp73-macosx_11_0_arm64.whl", hash = "sha256:8060fa256862b0c3de64a73ab45bc1ccf381caca464f2647af9075b200828948"}, + {file = "hiredis-3.1.0-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3e9445b7f117a9c8c8ccad97cb44daa55ddccff3cbc9079984eac56d982ba01f"}, + {file = "hiredis-3.1.0-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:732cf1c5cf1324f7bf3b6086976fe62a2ca98f0bf6316f31063c2c67be8797bc"}, + {file = "hiredis-3.1.0-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2102a94063d878c40df92f55199637a74f535e3a0b79ceba4a00538853a21be3"}, + {file = "hiredis-3.1.0-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:d968dde69e3fe903bf9ef00667669dcf04a3e096e33aaf138775106ead138bc8"}, + {file = "hiredis-3.1.0.tar.gz", hash = "sha256:51d40ac3611091020d7dea6b05ed62cb152bff595fa4f931e7b6479d777acf7c"}, ] [[package]] From e46295033819032eb3b50eb6e5a8e9adbff59a84 Mon Sep 17 00:00:00 2001 From: V02460 Date: Mon, 17 Feb 2025 15:25:42 +0100 Subject: [PATCH 34/65] Document consequences of replacing secrets (#18138) Document consequences of replacing secrets. The covered config options are `registration_shared_secret`, `macaroon_secret_key`, `form_secret` and `worker_replication_secret`. Even though I looked at the source code to check the added documentation is right, I would appreciate additional verification of the statements made. In an hand-wavy attempt at classifying how bad the consequences of secret replacement are, I added some explanations as warnings and others as regular paragraphs. Closes #17971 ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --- changelog.d/18138.doc | 1 + .../configuration/config_documentation.md | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 changelog.d/18138.doc diff --git a/changelog.d/18138.doc b/changelog.d/18138.doc new file mode 100644 index 0000000000..5f07cd2763 --- /dev/null +++ b/changelog.d/18138.doc @@ -0,0 +1 @@ +Document consequences of replacing secrets. \ No newline at end of file diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index 92ab6438c1..e3c06d5371 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -2592,6 +2592,14 @@ This is primarily intended for use with the `register_new_matrix_user` script (see [Registering a user](../../setup/installation.md#registering-a-user)); however, the interface is [documented](../../admin_api/register_api.html). +Replacing an existing `registration_shared_secret` with a new one requires users +of the [Shared-Secret Registration API](../../admin_api/register_api.html) to +start using the new secret for requesting any further one-time nonces. + +> ⚠️ **Warning** – The additional consequences of replacing +> [`macaroon_secret_key`](#macaroon_secret_key) will apply in case it delegates +> to `registration_shared_secret`. + See also [`registration_shared_secret_path`](#registration_shared_secret_path). Example configuration: @@ -3168,6 +3176,11 @@ A secret which is used to sign If none is specified, the `registration_shared_secret` is used, if one is given; otherwise, a secret key is derived from the signing key. +> ⚠️ **Warning** – Replacing an existing `macaroon_secret_key` with a new one +> will lead to invalidation of access tokens for all guest users. It will also +> break unsubscribe links in emails sent before the change. An unlucky user +> might encounter a broken SSO login flow and would have to start again. + Example configuration: ```yaml macaroon_secret_key: @@ -3195,6 +3208,9 @@ A secret which is used to calculate HMACs for form values, to stop falsification of values. Must be specified for the User Consent forms to work. +Replacing an existing `form_secret` with a new one might break the user consent +page for an unlucky user and require them to reopen the page from a new link. + Example configuration: ```yaml form_secret: @@ -4443,6 +4459,9 @@ HTTP requests from workers. The default, this value is omitted (equivalently `null`), which means that traffic between the workers and the main process is not authenticated. +Replacing an existing `worker_replication_secret` with a new one will break +communication with all workers that have not yet updated their secret. + Example configuration: ```yaml worker_replication_secret: "secret_secret" From 0c31783b4fa49fdf3b0e5b70f937ea0718114648 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Mon, 17 Feb 2025 16:39:26 +0100 Subject: [PATCH 35/65] Limit size of user directory search queries (#18172) If a user search has many words we can end up creating really large queries that take a long time for the database to process. Generally, such searches don't return any results anyway (due to limits on user ID and display name length). We "fix" this by cheating and only searching for the first ten words. --- changelog.d/18172.misc | 1 + synapse/storage/databases/main/user_directory.py | 8 +++++++- 2 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 changelog.d/18172.misc diff --git a/changelog.d/18172.misc b/changelog.d/18172.misc new file mode 100644 index 0000000000..49b6be263b --- /dev/null +++ b/changelog.d/18172.misc @@ -0,0 +1 @@ +Reduce database load of user search when using large search terms. diff --git a/synapse/storage/databases/main/user_directory.py b/synapse/storage/databases/main/user_directory.py index a51182de55..d6cd0774a8 100644 --- a/synapse/storage/databases/main/user_directory.py +++ b/synapse/storage/databases/main/user_directory.py @@ -1237,7 +1237,13 @@ def _parse_query_postgres(search_term: str) -> Tuple[str, str, str]: search_term = _filter_text_for_index(search_term) escaped_words = [] - for word in _parse_words(search_term): + for index, word in enumerate(_parse_words(search_term)): + if index >= 10: + # We limit how many terms we include, as otherwise it can use + # excessive database time if people accidentally search for large + # strings. + break + # Postgres tsvector and tsquery quoting rules: # words potentially containing punctuation should be quoted # and then existing quotes and backslashes should be doubled From 12dc6b102f071eb2eb84f2cff4cf92903276ffbb Mon Sep 17 00:00:00 2001 From: Eric Eastwood Date: Mon, 17 Feb 2025 10:23:04 -0600 Subject: [PATCH 36/65] Add support to proxy outbound requests from Synapse in tests (#18158) Adds new environment variables that can be used with the Docker image (`SYNAPSE_HTTP_PROXY`/`SYNAPSE_HTTPS_PROXY`/`SYNAPSE_NO_PROXY`) Useful for things like the [Secure Border Gateway](https://element.io/server-suite/secure-border-gateways) ### Why is this necessary? You can already configure the `HTTP_PROXY`/`HTTPS_PROXY` environment variables to proxy outbound requests but setting this globally in the Docker image affects all processes which isn't always desirable or workable in the case where the proxy is running in the Docker image itself (because the Debian packages will fail to download because the proxy isn't up and running yet) . Adding Synapse specific environment variables (`SYNAPSE_HTTP_PROXY`/`SYNAPSE_HTTPS_PROXY`/`SYNAPSE_NO_PROXY`) makes things much more targetable. --- changelog.d/18158.docker | 1 + docker/README.md | 3 +++ docker/conf-workers/synapse.supervisord.conf.j2 | 3 +++ docker/configure_workers_and_start.py | 7 +++++++ synapse/http/proxyagent.py | 6 ++++++ 5 files changed, 20 insertions(+) create mode 100644 changelog.d/18158.docker diff --git a/changelog.d/18158.docker b/changelog.d/18158.docker new file mode 100644 index 0000000000..60469690ff --- /dev/null +++ b/changelog.d/18158.docker @@ -0,0 +1 @@ +Add `SYNAPSE_HTTP_PROXY`/`SYNAPSE_HTTPS_PROXY`/`SYNAPSE_NO_PROXY` environment variables to pass through specifically to the Synapse process (instead of needing to apply [`http_proxy`/`https_proxy`/`no_proxy`](https://element-hq.github.io/synapse/latest/setup/forward_proxy.html) globally). diff --git a/docker/README.md b/docker/README.md index 8dba6fdb05..3438e9c441 100644 --- a/docker/README.md +++ b/docker/README.md @@ -114,6 +114,9 @@ The following environment variables are supported in `run` mode: is set via `docker run --user`, defaults to `991`, `991`. Note that this user must have permission to read the config files, and write to the data directories. * `TZ`: the [timezone](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones) the container will run with. Defaults to `UTC`. +* `SYNAPSE_HTTP_PROXY`: Passed through to the Synapse process as the `http_proxy` environment variable. +* `SYNAPSE_HTTPS_PROXY`: Passed through to the Synapse process as the `https_proxy` environment variable. +* `SYNAPSE_NO_PROXY`: Passed through to the Synapse process as `no_proxy` environment variable. For more complex setups (e.g. for workers) you can also pass your args directly to synapse using `run` mode. For example like this: diff --git a/docker/conf-workers/synapse.supervisord.conf.j2 b/docker/conf-workers/synapse.supervisord.conf.j2 index 481eb4fc92..4fb11b259e 100644 --- a/docker/conf-workers/synapse.supervisord.conf.j2 +++ b/docker/conf-workers/synapse.supervisord.conf.j2 @@ -1,5 +1,6 @@ {% if use_forking_launcher %} [program:synapse_fork] +environment=http_proxy="%(ENV_SYNAPSE_HTTP_PROXY)s",https_proxy="%(ENV_SYNAPSE_HTTPS_PROXY)s",no_proxy="%(ENV_SYNAPSE_NO_PROXY)s" command=/usr/local/bin/python -m synapse.app.complement_fork_starter {{ main_config_path }} synapse.app.homeserver @@ -20,6 +21,7 @@ exitcodes=0 {% else %} [program:synapse_main] +environment=http_proxy="%(ENV_SYNAPSE_HTTP_PROXY)s",https_proxy="%(ENV_SYNAPSE_HTTPS_PROXY)s",no_proxy="%(ENV_SYNAPSE_NO_PROXY)s" command=/usr/local/bin/prefix-log /usr/local/bin/python -m synapse.app.homeserver --config-path="{{ main_config_path }}" --config-path=/conf/workers/shared.yaml @@ -36,6 +38,7 @@ exitcodes=0 {% for worker in workers %} [program:synapse_{{ worker.name }}] +environment=http_proxy="%(ENV_SYNAPSE_HTTP_PROXY)s",https_proxy="%(ENV_SYNAPSE_HTTPS_PROXY)s",no_proxy="%(ENV_SYNAPSE_NO_PROXY)s" command=/usr/local/bin/prefix-log /usr/local/bin/python -m {{ worker.app }} --config-path="{{ main_config_path }}" --config-path=/conf/workers/shared.yaml diff --git a/docker/configure_workers_and_start.py b/docker/configure_workers_and_start.py index 15d8d7b558..6d73e8feaa 100755 --- a/docker/configure_workers_and_start.py +++ b/docker/configure_workers_and_start.py @@ -1099,6 +1099,13 @@ def main(args: List[str], environ: MutableMapping[str, str]) -> None: else: log("Could not find %s, will not use" % (jemallocpath,)) + # Empty strings are falsy in Python so this default is fine. We just can't have these + # be undefined because supervisord will complain about our + # `%(ENV_SYNAPSE_HTTP_PROXY)s` usage. + environ.setdefault("SYNAPSE_HTTP_PROXY", "") + environ.setdefault("SYNAPSE_HTTPS_PROXY", "") + environ.setdefault("SYNAPSE_NO_PROXY", "") + # Start supervisord, which will start Synapse, all of the configured worker # processes, redis, nginx etc. according to the config we created above. log("Starting supervisord") diff --git a/synapse/http/proxyagent.py b/synapse/http/proxyagent.py index c91cf30fd1..fd16ee42dd 100644 --- a/synapse/http/proxyagent.py +++ b/synapse/http/proxyagent.py @@ -150,6 +150,12 @@ class ProxyAgent(_AgentBase): http_proxy = proxies["http"].encode() if "http" in proxies else None https_proxy = proxies["https"].encode() if "https" in proxies else None no_proxy = proxies["no"] if "no" in proxies else None + logger.debug( + "Using proxy settings: http_proxy=%s, https_proxy=%s, no_proxy=%s", + http_proxy, + https_proxy, + no_proxy, + ) self.http_proxy_endpoint, self.http_proxy_creds = http_proxy_endpoint( http_proxy, self.proxy_reactor, contextFactory, **self._endpoint_kwargs From 2d4f28915eebb7109e8ac444d8f2d0d303d65013 Mon Sep 17 00:00:00 2001 From: qashlan Date: Tue, 18 Feb 2025 12:20:30 +0200 Subject: [PATCH 37/65] Add method to get current server time in milliseconds in ModuleApi (#18144) - Add `get_current_time_msec()` method to the [module API](https://matrix-org.github.io/synapse/latest/modules/writing_a_module.html) for sound time comparisons with Synapse. - Fixes #18104 Signed-off-by: Ahmed Qashlan Co-authored-by: Eric Eastwood Co-authored-by: Erik Johnston --- changelog.d/18144.feature | 1 + synapse/module_api/__init__.py | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 changelog.d/18144.feature diff --git a/changelog.d/18144.feature b/changelog.d/18144.feature new file mode 100644 index 0000000000..74a28e5a4e --- /dev/null +++ b/changelog.d/18144.feature @@ -0,0 +1 @@ +Add `get_current_time_msec()` method to the [module API](https://matrix-org.github.io/synapse/latest/modules/writing_a_module.html) for sound time comparisons with Synapse. \ No newline at end of file diff --git a/synapse/module_api/__init__.py b/synapse/module_api/__init__.py index 2a2f821427..bf9532e891 100644 --- a/synapse/module_api/__init__.py +++ b/synapse/module_api/__init__.py @@ -1846,6 +1846,10 @@ class ModuleApi: deactivation=deactivation, ) + def get_current_time_msec(self) -> int: + """Returns the current server time in milliseconds.""" + return self._clock.time_msec() + class PublicRoomListManager: """Contains methods for adding to, removing from and querying whether a room From 4aa725a7302e64a45591a7361c96b967596d16cb Mon Sep 17 00:00:00 2001 From: Quentin Gliech Date: Tue, 18 Feb 2025 14:33:34 +0100 Subject: [PATCH 38/65] 1.125.0rc1 --- CHANGES.md | 50 +++++++++++++++++++++++++++++++++++++++ changelog.d/17436.doc | 1 - changelog.d/17616.misc | 1 - changelog.d/17949.feature | 1 - changelog.d/17967.misc | 1 - changelog.d/18004.feature | 1 - changelog.d/18122.doc | 1 - changelog.d/18124.misc | 1 - changelog.d/18125.bugfix | 1 - changelog.d/18134.misc | 1 - changelog.d/18135.bugfix | 1 - changelog.d/18136.misc | 1 - changelog.d/18138.doc | 1 - changelog.d/18139.misc | 1 - changelog.d/18144.feature | 1 - changelog.d/18145.bugfix | 1 - changelog.d/18158.docker | 1 - changelog.d/18159.doc | 1 - changelog.d/18172.misc | 1 - debian/changelog | 6 +++++ pyproject.toml | 2 +- 21 files changed, 57 insertions(+), 19 deletions(-) delete mode 100644 changelog.d/17436.doc delete mode 100644 changelog.d/17616.misc delete mode 100644 changelog.d/17949.feature delete mode 100644 changelog.d/17967.misc delete mode 100644 changelog.d/18004.feature delete mode 100644 changelog.d/18122.doc delete mode 100644 changelog.d/18124.misc delete mode 100644 changelog.d/18125.bugfix delete mode 100644 changelog.d/18134.misc delete mode 100644 changelog.d/18135.bugfix delete mode 100644 changelog.d/18136.misc delete mode 100644 changelog.d/18138.doc delete mode 100644 changelog.d/18139.misc delete mode 100644 changelog.d/18144.feature delete mode 100644 changelog.d/18145.bugfix delete mode 100644 changelog.d/18158.docker delete mode 100644 changelog.d/18159.doc delete mode 100644 changelog.d/18172.misc diff --git a/CHANGES.md b/CHANGES.md index 58b410decb..d83430088c 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,53 @@ +# Synapse 1.125.0rc1 (2025-02-18) + +### Features + +- Add functionality to be able to use multiple values in SSO feature `attribute_requirements`. ([\#17949](https://github.com/element-hq/synapse/issues/17949)) +- Add experimental config options `admin_token_path` and `client_secret_path` for MSC 3861. ([\#18004](https://github.com/element-hq/synapse/issues/18004)) +- Add `get_current_time_msec()` method to the [module API](https://matrix-org.github.io/synapse/latest/modules/writing_a_module.html) for sound time comparisons with Synapse. ([\#18144](https://github.com/element-hq/synapse/issues/18144)) + +### Bugfixes + +- Fix a bug when updating a user 3pid with invalid returns 500 server error change to 400 with a message. ([\#18125](https://github.com/element-hq/synapse/issues/18125)) +- Fix user directory search when using a legacy module with a `check_username_for_spam` callback. Broke in v1.122.0. ([\#18135](https://github.com/element-hq/synapse/issues/18135)) +- Add rate limit `rc_presence.per_user`. This prevents load from excessive presence updates sent by clients via sync api. Also rate limit `/_matrix/client/v3/presence` as per the spec. Contributed by @rda0. ([\#18145](https://github.com/element-hq/synapse/issues/18145)) + +### Updates to the Docker image + +- Add `SYNAPSE_HTTP_PROXY`/`SYNAPSE_HTTPS_PROXY`/`SYNAPSE_NO_PROXY` environment variables to pass through specifically to the Synapse process (instead of needing to apply [`http_proxy`/`https_proxy`/`no_proxy`](https://element-hq.github.io/synapse/latest/setup/forward_proxy.html) globally). ([\#18158](https://github.com/element-hq/synapse/issues/18158)) + +### Improved Documentation + +- Add Oracle Linux 8 and 9 installation instructions. ([\#17436](https://github.com/element-hq/synapse/issues/17436)) +- Document missing server config options (`daemonize`, `print_pidfile`, `user_agent_suffix`, `use_frozen_dicts`, `manhole`). ([\#18122](https://github.com/element-hq/synapse/issues/18122)) +- Document consequences of replacing secrets. ([\#18138](https://github.com/element-hq/synapse/issues/18138)) +- Make `burst_count` field an integer in `rc_presence` config documentation example. ([\#18159](https://github.com/element-hq/synapse/issues/18159)) + +### Internal Changes + +- Overload DatabasePool.simple_select_one_txn to return non-None when the allow_none parameter is False. ([\#17616](https://github.com/element-hq/synapse/issues/17616)) +- Python 3.8 EOL: compile native extensions with the 3.9 ABI and use typing hints from the standard library. ([\#17967](https://github.com/element-hq/synapse/issues/17967)) +- Add log message when worker lock timeouts get large. ([\#18124](https://github.com/element-hq/synapse/issues/18124)) +- Make it explicit that you can buy an AGPL-alternative commercial license from Element. ([\#18134](https://github.com/element-hq/synapse/issues/18134)) +- Fix the 'Fix linting' GitHub Actions workflow. ([\#18136](https://github.com/element-hq/synapse/issues/18136)) +- Do not log at the exception-level when clients provide empty `since` token to `/sync` API. ([\#18139](https://github.com/element-hq/synapse/issues/18139)) +- Reduce database load of user search when using large search terms. ([\#18172](https://github.com/element-hq/synapse/issues/18172)) + + + +### Updates to locked dependencies + +* Bump bcrypt from 4.2.0 to 4.2.1. ([\#18127](https://github.com/element-hq/synapse/issues/18127)) +* Bump bytes from 1.9.0 to 1.10.0. ([\#18149](https://github.com/element-hq/synapse/issues/18149)) +* Bump gitpython from 3.1.43 to 3.1.44. ([\#18128](https://github.com/element-hq/synapse/issues/18128)) +* Bump hiredis from 3.0.0 to 3.1.0. ([\#18169](https://github.com/element-hq/synapse/issues/18169)) +* Bump serde_json from 1.0.137 to 1.0.138. ([\#18129](https://github.com/element-hq/synapse/issues/18129)) +* Bump service-identity from 24.1.0 to 24.2.0. ([\#18171](https://github.com/element-hq/synapse/issues/18171)) +* Bump sigstore/cosign-installer from 3.7.0 to 3.8.0. ([\#18147](https://github.com/element-hq/synapse/issues/18147)) +* Bump twine from 6.0.1 to 6.1.0. ([\#18170](https://github.com/element-hq/synapse/issues/18170)) +* Bump types-pyyaml from 6.0.12.20240917 to 6.0.12.20241230. ([\#18097](https://github.com/element-hq/synapse/issues/18097)) +* Bump ulid from 1.1.4 to 1.2.0. ([\#18148](https://github.com/element-hq/synapse/issues/18148)) + # Synapse 1.124.0 (2025-02-11) No significant changes since 1.124.0rc3. diff --git a/changelog.d/17436.doc b/changelog.d/17436.doc deleted file mode 100644 index 7846d3a8bf..0000000000 --- a/changelog.d/17436.doc +++ /dev/null @@ -1 +0,0 @@ -Add Oracle Linux 8 and 9 installation instructions. diff --git a/changelog.d/17616.misc b/changelog.d/17616.misc deleted file mode 100644 index 8250832dcd..0000000000 --- a/changelog.d/17616.misc +++ /dev/null @@ -1 +0,0 @@ -Overload DatabasePool.simple_select_one_txn to return non-None when the allow_none parameter is False. diff --git a/changelog.d/17949.feature b/changelog.d/17949.feature deleted file mode 100644 index ee9bb51e03..0000000000 --- a/changelog.d/17949.feature +++ /dev/null @@ -1 +0,0 @@ -Add functionality to be able to use multiple values in SSO feature `attribute_requirements`. diff --git a/changelog.d/17967.misc b/changelog.d/17967.misc deleted file mode 100644 index 60e61a6f7d..0000000000 --- a/changelog.d/17967.misc +++ /dev/null @@ -1 +0,0 @@ -Python 3.8 EOL: compile native extensions with the 3.9 ABI and use typing hints from the standard library. \ No newline at end of file diff --git a/changelog.d/18004.feature b/changelog.d/18004.feature deleted file mode 100644 index 8cacd1a0ef..0000000000 --- a/changelog.d/18004.feature +++ /dev/null @@ -1 +0,0 @@ -Add experimental config options `admin_token_path` and `client_secret_path` for MSC 3861. \ No newline at end of file diff --git a/changelog.d/18122.doc b/changelog.d/18122.doc deleted file mode 100644 index b7bb7107e2..0000000000 --- a/changelog.d/18122.doc +++ /dev/null @@ -1 +0,0 @@ -Document missing server config options (`daemonize`, `print_pidfile`, `user_agent_suffix`, `use_frozen_dicts`, `manhole`). diff --git a/changelog.d/18124.misc b/changelog.d/18124.misc deleted file mode 100644 index 8ac6a73a20..0000000000 --- a/changelog.d/18124.misc +++ /dev/null @@ -1 +0,0 @@ -Add log message when worker lock timeouts get large. diff --git a/changelog.d/18125.bugfix b/changelog.d/18125.bugfix deleted file mode 100644 index 0e6aa704ff..0000000000 --- a/changelog.d/18125.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix a bug when updating a user 3pid with invalid returns 500 server error change to 400 with a message. \ No newline at end of file diff --git a/changelog.d/18134.misc b/changelog.d/18134.misc deleted file mode 100644 index be90691516..0000000000 --- a/changelog.d/18134.misc +++ /dev/null @@ -1 +0,0 @@ -Make it explicit that you can buy an AGPL-alternative commercial license from Element. diff --git a/changelog.d/18135.bugfix b/changelog.d/18135.bugfix deleted file mode 100644 index 998914a811..0000000000 --- a/changelog.d/18135.bugfix +++ /dev/null @@ -1 +0,0 @@ -Fix user directory search when using a legacy module with a `check_username_for_spam` callback. Broke in v1.122.0. diff --git a/changelog.d/18136.misc b/changelog.d/18136.misc deleted file mode 100644 index ed88c38acf..0000000000 --- a/changelog.d/18136.misc +++ /dev/null @@ -1 +0,0 @@ -Fix the 'Fix linting' GitHub Actions workflow. diff --git a/changelog.d/18138.doc b/changelog.d/18138.doc deleted file mode 100644 index 5f07cd2763..0000000000 --- a/changelog.d/18138.doc +++ /dev/null @@ -1 +0,0 @@ -Document consequences of replacing secrets. \ No newline at end of file diff --git a/changelog.d/18139.misc b/changelog.d/18139.misc deleted file mode 100644 index f753033b8e..0000000000 --- a/changelog.d/18139.misc +++ /dev/null @@ -1 +0,0 @@ -Do not log at the exception-level when clients provide empty `since` token to `/sync` API. diff --git a/changelog.d/18144.feature b/changelog.d/18144.feature deleted file mode 100644 index 74a28e5a4e..0000000000 --- a/changelog.d/18144.feature +++ /dev/null @@ -1 +0,0 @@ -Add `get_current_time_msec()` method to the [module API](https://matrix-org.github.io/synapse/latest/modules/writing_a_module.html) for sound time comparisons with Synapse. \ No newline at end of file diff --git a/changelog.d/18145.bugfix b/changelog.d/18145.bugfix deleted file mode 100644 index ad9d353834..0000000000 --- a/changelog.d/18145.bugfix +++ /dev/null @@ -1 +0,0 @@ -Add rate limit `rc_presence.per_user`. This prevents load from excessive presence updates sent by clients via sync api. Also rate limit `/_matrix/client/v3/presence` as per the spec. Contributed by @rda0. \ No newline at end of file diff --git a/changelog.d/18158.docker b/changelog.d/18158.docker deleted file mode 100644 index 60469690ff..0000000000 --- a/changelog.d/18158.docker +++ /dev/null @@ -1 +0,0 @@ -Add `SYNAPSE_HTTP_PROXY`/`SYNAPSE_HTTPS_PROXY`/`SYNAPSE_NO_PROXY` environment variables to pass through specifically to the Synapse process (instead of needing to apply [`http_proxy`/`https_proxy`/`no_proxy`](https://element-hq.github.io/synapse/latest/setup/forward_proxy.html) globally). diff --git a/changelog.d/18159.doc b/changelog.d/18159.doc deleted file mode 100644 index 031f8a149f..0000000000 --- a/changelog.d/18159.doc +++ /dev/null @@ -1 +0,0 @@ -Make `burst_count` field an integer in `rc_presence` config documentation example. \ No newline at end of file diff --git a/changelog.d/18172.misc b/changelog.d/18172.misc deleted file mode 100644 index 49b6be263b..0000000000 --- a/changelog.d/18172.misc +++ /dev/null @@ -1 +0,0 @@ -Reduce database load of user search when using large search terms. diff --git a/debian/changelog b/debian/changelog index 3e20885c86..a0d5a7614f 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +matrix-synapse-py3 (1.125.0~rc1) stable; urgency=medium + + * New synapse release 1.125.0rc1. + + -- Synapse Packaging team Tue, 18 Feb 2025 13:32:49 +0000 + matrix-synapse-py3 (1.124.0) stable; urgency=medium * New Synapse release 1.124.0. diff --git a/pyproject.toml b/pyproject.toml index 2d982eae70..a40be93d7b 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -97,7 +97,7 @@ module-name = "synapse.synapse_rust" [tool.poetry] name = "matrix-synapse" -version = "1.124.0" +version = "1.125.0rc1" description = "Homeserver for the Matrix decentralised comms protocol" authors = ["Matrix.org Team and Contributors "] license = "AGPL-3.0-or-later" From 30fcd586fe8200dbf3505a6ff84ec5122e91da3c Mon Sep 17 00:00:00 2001 From: Quentin Gliech Date: Tue, 18 Feb 2025 14:47:13 +0100 Subject: [PATCH 39/65] Tweak changelog --- CHANGES.md | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index d83430088c..7db366d1d5 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -3,14 +3,13 @@ ### Features - Add functionality to be able to use multiple values in SSO feature `attribute_requirements`. ([\#17949](https://github.com/element-hq/synapse/issues/17949)) -- Add experimental config options `admin_token_path` and `client_secret_path` for MSC 3861. ([\#18004](https://github.com/element-hq/synapse/issues/18004)) +- Add experimental config options `admin_token_path` and `client_secret_path` for [MSC3861](https://github.com/matrix-org/matrix-spec-proposals/pull/3861). ([\#18004](https://github.com/element-hq/synapse/issues/18004)) - Add `get_current_time_msec()` method to the [module API](https://matrix-org.github.io/synapse/latest/modules/writing_a_module.html) for sound time comparisons with Synapse. ([\#18144](https://github.com/element-hq/synapse/issues/18144)) ### Bugfixes -- Fix a bug when updating a user 3pid with invalid returns 500 server error change to 400 with a message. ([\#18125](https://github.com/element-hq/synapse/issues/18125)) +- Update the response when a client attempts to add an invalid email address to the user's account from a 500, to a 400 with error text. ([\#18125](https://github.com/element-hq/synapse/issues/18125)) - Fix user directory search when using a legacy module with a `check_username_for_spam` callback. Broke in v1.122.0. ([\#18135](https://github.com/element-hq/synapse/issues/18135)) -- Add rate limit `rc_presence.per_user`. This prevents load from excessive presence updates sent by clients via sync api. Also rate limit `/_matrix/client/v3/presence` as per the spec. Contributed by @rda0. ([\#18145](https://github.com/element-hq/synapse/issues/18145)) ### Updates to the Docker image @@ -25,7 +24,7 @@ ### Internal Changes -- Overload DatabasePool.simple_select_one_txn to return non-None when the allow_none parameter is False. ([\#17616](https://github.com/element-hq/synapse/issues/17616)) +- Overload `DatabasePool.simple_select_one_txn` to return non-`None` when the `allow_none` parameter is `False`. ([\#17616](https://github.com/element-hq/synapse/issues/17616)) - Python 3.8 EOL: compile native extensions with the 3.9 ABI and use typing hints from the standard library. ([\#17967](https://github.com/element-hq/synapse/issues/17967)) - Add log message when worker lock timeouts get large. ([\#18124](https://github.com/element-hq/synapse/issues/18124)) - Make it explicit that you can buy an AGPL-alternative commercial license from Element. ([\#18134](https://github.com/element-hq/synapse/issues/18134)) From ecad88f5c5787f6a8b7e72d3cb97b868fb685484 Mon Sep 17 00:00:00 2001 From: Devon Hudson Date: Tue, 18 Feb 2025 14:44:38 +0000 Subject: [PATCH 40/65] Cleanup deleted state group references (#18165) ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --- changelog.d/18165.bugfix | 1 + synapse/storage/databases/state/store.py | 8 ++++++ tests/storage/test_purge.py | 35 +++++++++++++++++++++++- 3 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 changelog.d/18165.bugfix diff --git a/changelog.d/18165.bugfix b/changelog.d/18165.bugfix new file mode 100644 index 0000000000..eba5face52 --- /dev/null +++ b/changelog.d/18165.bugfix @@ -0,0 +1 @@ +Cleanup deleted state group references. diff --git a/synapse/storage/databases/state/store.py b/synapse/storage/databases/state/store.py index 8c7980e719..90d7beb92f 100644 --- a/synapse/storage/databases/state/store.py +++ b/synapse/storage/databases/state/store.py @@ -828,10 +828,18 @@ class StateGroupDataStore(StateBackgroundUpdateStore, SQLBaseStore): "DELETE FROM state_groups_state WHERE state_group = ?", [(sg,) for sg in state_groups_to_delete], ) + txn.execute_batch( + "DELETE FROM state_group_edges WHERE state_group = ?", + [(sg,) for sg in state_groups_to_delete], + ) txn.execute_batch( "DELETE FROM state_groups WHERE id = ?", [(sg,) for sg in state_groups_to_delete], ) + txn.execute_batch( + "DELETE FROM state_groups_pending_deletion WHERE state_group = ?", + [(sg,) for sg in state_groups_to_delete], + ) return True diff --git a/tests/storage/test_purge.py b/tests/storage/test_purge.py index 5d6a8518c0..916e42e731 100644 --- a/tests/storage/test_purge.py +++ b/tests/storage/test_purge.py @@ -247,7 +247,7 @@ class PurgeTests(HomeserverTestCase): 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 ) - # We expect that the unreferenced state group has been deleted. + # We expect that the unreferenced state group has been deleted from all tables. row = self.get_success( self.state_store.db_pool.simple_select_one_onecol( table="state_groups", @@ -259,6 +259,39 @@ class PurgeTests(HomeserverTestCase): ) self.assertIsNone(row) + row = self.get_success( + self.state_store.db_pool.simple_select_one_onecol( + table="state_groups_state", + keyvalues={"state_group": unreferenced_state_group}, + retcol="state_group", + allow_none=True, + desc="test_purge_unreferenced_state_group", + ) + ) + self.assertIsNone(row) + + row = self.get_success( + self.state_store.db_pool.simple_select_one_onecol( + table="state_group_edges", + keyvalues={"state_group": unreferenced_state_group}, + retcol="state_group", + allow_none=True, + desc="test_purge_unreferenced_state_group", + ) + ) + self.assertIsNone(row) + + row = self.get_success( + self.state_store.db_pool.simple_select_one_onecol( + table="state_groups_pending_deletion", + keyvalues={"state_group": unreferenced_state_group}, + retcol="state_group", + allow_none=True, + desc="test_purge_unreferenced_state_group", + ) + ) + self.assertIsNone(row) + # We expect there to now only be one state group for the room, which is # the state group of the last event (as the only outlier). state_groups = self.get_success( From f7bc63ef5787f20b63586bc7d8665cd2ce76c268 Mon Sep 17 00:00:00 2001 From: Quentin Gliech Date: Tue, 18 Feb 2025 15:47:35 +0100 Subject: [PATCH 41/65] Make sure we advertise registration as disabled when MSC3861 is enabled (#17661) This has been a problem with Element Web, as it will proble /register with an empty body, which gave this error: ``` curl -d '{}' -HContent-Type:application/json /_matrix/client/v3/register {"errcode": "M_UNKNOWN", "error": "Invalid username"} ``` And Element Web would choke on it. This changes that so we reply instead: ``` {"errcode": "M_FORBIDDEN", "error": "Registration has been disabled. Only m.login.application_service registrations are allowed."} ``` Also adds a test for this. See https://github.com/element-hq/element-web/issues/27993 --------- Co-authored-by: Andrew Morgan --- changelog.d/17661.bugfix | 1 + synapse/rest/client/register.py | 12 +++++--- tests/handlers/test_oauth_delegation.py | 37 ++++++++++++++++++++++++- 3 files changed, 45 insertions(+), 5 deletions(-) create mode 100644 changelog.d/17661.bugfix diff --git a/changelog.d/17661.bugfix b/changelog.d/17661.bugfix new file mode 100644 index 0000000000..33881bbc6a --- /dev/null +++ b/changelog.d/17661.bugfix @@ -0,0 +1 @@ +Make sure we advertise registration as disabled when MSC3861 is enabled. diff --git a/synapse/rest/client/register.py b/synapse/rest/client/register.py index ad76f188ab..58231d2b04 100644 --- a/synapse/rest/client/register.py +++ b/synapse/rest/client/register.py @@ -908,6 +908,14 @@ class RegisterAppServiceOnlyRestServlet(RestServlet): await self.ratelimiter.ratelimit(None, client_addr, update=False) + # Allow only ASes to use this API. + if body.get("type") != APP_SERVICE_REGISTRATION_TYPE: + raise SynapseError( + 403, + "Registration has been disabled. Only m.login.application_service registrations are allowed.", + errcode=Codes.FORBIDDEN, + ) + kind = parse_string(request, "kind", default="user") if kind == "guest": @@ -923,10 +931,6 @@ class RegisterAppServiceOnlyRestServlet(RestServlet): if not isinstance(desired_username, str) or len(desired_username) > 512: raise SynapseError(400, "Invalid username") - # Allow only ASes to use this API. - if body.get("type") != APP_SERVICE_REGISTRATION_TYPE: - raise SynapseError(403, "Non-application service registration type") - if not self.auth.has_access_token(request): raise SynapseError( 400, diff --git a/tests/handlers/test_oauth_delegation.py b/tests/handlers/test_oauth_delegation.py index ba2f8ff510..b9d66a6c52 100644 --- a/tests/handlers/test_oauth_delegation.py +++ b/tests/handlers/test_oauth_delegation.py @@ -43,6 +43,7 @@ from synapse.api.errors import ( OAuthInsufficientScopeError, SynapseError, ) +from synapse.appservice import ApplicationService from synapse.http.site import SynapseRequest from synapse.rest import admin from synapse.rest.client import account, devices, keys, login, logout, register @@ -575,6 +576,16 @@ class MSC3861OAuthDelegation(HomeserverTestCase): channel.json_body["errcode"], Codes.UNRECOGNIZED, channel.json_body ) + def expect_forbidden( + self, method: str, path: str, content: Union[bytes, str, JsonDict] = "" + ) -> None: + channel = self.make_request(method, path, content) + + self.assertEqual(channel.code, 403, channel.json_body) + self.assertEqual( + channel.json_body["errcode"], Codes.FORBIDDEN, channel.json_body + ) + def test_uia_endpoints(self) -> None: """Test that endpoints that were removed in MSC2964 are no longer available.""" @@ -629,11 +640,35 @@ class MSC3861OAuthDelegation(HomeserverTestCase): def test_registration_endpoints_removed(self) -> None: """Test that registration endpoints that were removed in MSC2964 are no longer available.""" + appservice = ApplicationService( + token="i_am_an_app_service", + id="1234", + namespaces={"users": [{"regex": r"@alice:.+", "exclusive": True}]}, + sender="@as_main:test", + ) + + self.hs.get_datastores().main.services_cache = [appservice] self.expect_unrecognized( "GET", "/_matrix/client/v1/register/m.login.registration_token/validity" ) + + # Registration is disabled + self.expect_forbidden( + "POST", + "/_matrix/client/v3/register", + {"username": "alice", "password": "hunter2"}, + ) + # This is still available for AS registrations - # self.expect_unrecognized("POST", "/_matrix/client/v3/register") + channel = self.make_request( + "POST", + "/_matrix/client/v3/register", + {"username": "alice", "type": "m.login.application_service"}, + shorthand=False, + access_token="i_am_an_app_service", + ) + self.assertEqual(channel.code, 200, channel.json_body) + self.expect_unrecognized("GET", "/_matrix/client/v3/register/available") self.expect_unrecognized( "POST", "/_matrix/client/v3/register/email/requestToken" From 1525a3b4d48a0f5657d61423e1f205bff9a77948 Mon Sep 17 00:00:00 2001 From: Quentin Gliech Date: Wed, 19 Feb 2025 11:55:15 +0100 Subject: [PATCH 42/65] Speedup the building of Docker images (#18038) This is a split off #18033 This uses a few tricks to speed up the building of docker images: - This switches to use `uv pip install` instead of `pip install`. This saves a bunch of time, especially when cross-compiling - I then looked at what packages were not using binary wheels: I upgraded MarkupSafe to have binaries for py3.12, and got back to Python 3.12 because hiredis didn't have builds for py3.13 with the version we were using - The generation of the requirements.txt is arch-agnostic, so I've switched this one to run on the build architecture, so that both arch can share it - The download of runtime depdendencies can be done on the build architecture through manual `apt-get download` plus `dpkg --extract` - We were using -slim images, but still installed a bunch of -dev dependencies. Turns out, all the dev dependencies were already installed in the non-slim image, which saves a bunch of time as well --- changelog.d/18038.docker | 1 + docker/Dockerfile | 169 ++++++++++++++++++++------------------- poetry.lock | 115 ++++++++++++++------------ 3 files changed, 150 insertions(+), 135 deletions(-) create mode 100644 changelog.d/18038.docker diff --git a/changelog.d/18038.docker b/changelog.d/18038.docker new file mode 100644 index 0000000000..accb09be2a --- /dev/null +++ b/changelog.d/18038.docker @@ -0,0 +1 @@ +Speed up the building of the Docker image. diff --git a/docker/Dockerfile b/docker/Dockerfile index a4931011a7..1dd65f2413 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -20,45 +20,16 @@ # `poetry export | pip install -r /dev/stdin`, but beware: we have experienced bugs in # in `poetry export` in the past. +ARG DEBIAN_VERSION=bookworm ARG PYTHON_VERSION=3.12 +ARG POETRY_VERSION=1.8.3 ### ### Stage 0: generate requirements.txt ### -# We hardcode the use of Debian bookworm here because this could change upstream -# and other Dockerfiles used for testing are expecting bookworm. -FROM docker.io/library/python:${PYTHON_VERSION}-slim-bookworm AS requirements - -# RUN --mount is specific to buildkit and is documented at -# https://github.com/moby/buildkit/blob/master/frontend/dockerfile/docs/syntax.md#build-mounts-run---mount. -# Here we use it to set up a cache for apt (and below for pip), to improve -# rebuild speeds on slow connections. -RUN \ - --mount=type=cache,target=/var/cache/apt,sharing=locked \ - --mount=type=cache,target=/var/lib/apt,sharing=locked \ - apt-get update -qq && apt-get install -yqq \ - build-essential curl git libffi-dev libssl-dev pkg-config \ - && rm -rf /var/lib/apt/lists/* - -# Install rust and ensure its in the PATH. -# (Rust may be needed to compile `cryptography`---which is one of poetry's -# dependencies---on platforms that don't have a `cryptography` wheel. -ENV RUSTUP_HOME=/rust -ENV CARGO_HOME=/cargo -ENV PATH=/cargo/bin:/rust/bin:$PATH -RUN mkdir /rust /cargo - -RUN curl -sSf https://sh.rustup.rs | sh -s -- -y --no-modify-path --default-toolchain stable --profile minimal - -# arm64 builds consume a lot of memory if `CARGO_NET_GIT_FETCH_WITH_CLI` is not -# set to true, so we expose it as a build-arg. -ARG CARGO_NET_GIT_FETCH_WITH_CLI=false -ENV CARGO_NET_GIT_FETCH_WITH_CLI=$CARGO_NET_GIT_FETCH_WITH_CLI - -# We install poetry in its own build stage to avoid its dependencies conflicting with -# synapse's dependencies. -RUN --mount=type=cache,target=/root/.cache/pip \ - pip install --user "poetry==1.3.2" +### This stage is platform-agnostic, so we can use the build platform in case of cross-compilation. +### +FROM --platform=$BUILDPLATFORM ghcr.io/astral-sh/uv:python${PYTHON_VERSION}-${DEBIAN_VERSION} AS requirements WORKDIR /synapse @@ -75,41 +46,30 @@ ARG TEST_ONLY_SKIP_DEP_HASH_VERIFICATION # Instead, we'll just install what a regular `pip install` would from PyPI. ARG TEST_ONLY_IGNORE_POETRY_LOCKFILE +# This silences a warning as uv isn't able to do hardlinks between its cache +# (mounted as --mount=type=cache) and the target directory. +ENV UV_LINK_MODE=copy + # Export the dependencies, but only if we're actually going to use the Poetry lockfile. # Otherwise, just create an empty requirements file so that the Dockerfile can # proceed. -RUN if [ -z "$TEST_ONLY_IGNORE_POETRY_LOCKFILE" ]; then \ - /root/.local/bin/poetry export --extras all -o /synapse/requirements.txt ${TEST_ONLY_SKIP_DEP_HASH_VERIFICATION:+--without-hashes}; \ +ARG POETRY_VERSION +RUN --mount=type=cache,target=/root/.cache/uv \ + if [ -z "$TEST_ONLY_IGNORE_POETRY_LOCKFILE" ]; then \ + uvx --with poetry-plugin-export==1.8.0 \ + poetry@${POETRY_VERSION} export --extras all -o /synapse/requirements.txt ${TEST_ONLY_SKIP_DEP_HASH_VERIFICATION:+--without-hashes}; \ else \ - touch /synapse/requirements.txt; \ + touch /synapse/requirements.txt; \ fi ### ### Stage 1: builder ### -FROM docker.io/library/python:${PYTHON_VERSION}-slim-bookworm AS builder - -# install the OS build deps -RUN \ - --mount=type=cache,target=/var/cache/apt,sharing=locked \ - --mount=type=cache,target=/var/lib/apt,sharing=locked \ - apt-get update -qq && apt-get install -yqq \ - build-essential \ - libffi-dev \ - libjpeg-dev \ - libpq-dev \ - libssl-dev \ - libwebp-dev \ - libxml++2.6-dev \ - libxslt1-dev \ - openssl \ - zlib1g-dev \ - git \ - curl \ - libicu-dev \ - pkg-config \ - && rm -rf /var/lib/apt/lists/* +FROM ghcr.io/astral-sh/uv:python${PYTHON_VERSION}-${DEBIAN_VERSION} AS builder +# This silences a warning as uv isn't able to do hardlinks between its cache +# (mounted as --mount=type=cache) and the target directory. +ENV UV_LINK_MODE=copy # Install rust and ensure its in the PATH ENV RUSTUP_HOME=/rust @@ -119,7 +79,6 @@ RUN mkdir /rust /cargo RUN curl -sSf https://sh.rustup.rs | sh -s -- -y --no-modify-path --default-toolchain stable --profile minimal - # arm64 builds consume a lot of memory if `CARGO_NET_GIT_FETCH_WITH_CLI` is not # set to true, so we expose it as a build-arg. ARG CARGO_NET_GIT_FETCH_WITH_CLI=false @@ -131,8 +90,8 @@ ENV CARGO_NET_GIT_FETCH_WITH_CLI=$CARGO_NET_GIT_FETCH_WITH_CLI # # This is aiming at installing the `[tool.poetry.depdendencies]` from pyproject.toml. COPY --from=requirements /synapse/requirements.txt /synapse/ -RUN --mount=type=cache,target=/root/.cache/pip \ - pip install --prefix="/install" --no-deps --no-warn-script-location -r /synapse/requirements.txt +RUN --mount=type=cache,target=/root/.cache/uv \ + uv pip install --prefix="/install" --no-deps -r /synapse/requirements.txt # Copy over the rest of the synapse source code. COPY synapse /synapse/synapse/ @@ -146,41 +105,85 @@ ARG TEST_ONLY_IGNORE_POETRY_LOCKFILE # Install the synapse package itself. # If we have populated requirements.txt, we don't install any dependencies # as we should already have those from the previous `pip install` step. -RUN --mount=type=cache,target=/synapse/target,sharing=locked \ +RUN \ + --mount=type=cache,target=/root/.cache/uv \ + --mount=type=cache,target=/synapse/target,sharing=locked \ --mount=type=cache,target=${CARGO_HOME}/registry,sharing=locked \ if [ -z "$TEST_ONLY_IGNORE_POETRY_LOCKFILE" ]; then \ - pip install --prefix="/install" --no-deps --no-warn-script-location /synapse[all]; \ + uv pip install --prefix="/install" --no-deps /synapse[all]; \ else \ - pip install --prefix="/install" --no-warn-script-location /synapse[all]; \ + uv pip install --prefix="/install" /synapse[all]; \ fi ### -### Stage 2: runtime +### Stage 2: runtime dependencies download for ARM64 and AMD64 +### +FROM --platform=$BUILDPLATFORM docker.io/library/debian:${DEBIAN_VERSION} AS runtime-deps + +# Tell apt to keep downloaded package files, as we're using cache mounts. +RUN rm -f /etc/apt/apt.conf.d/docker-clean; echo 'Binary::apt::APT::Keep-Downloaded-Packages "true";' > /etc/apt/apt.conf.d/keep-cache + +# Add both target architectures +RUN dpkg --add-architecture arm64 +RUN dpkg --add-architecture amd64 + +# Fetch the runtime dependencies debs for both architectures +# We do that by building a recursive list of packages we need to download with `apt-cache depends` +# and then downloading them with `apt-get download`. +RUN \ + --mount=type=cache,target=/var/cache/apt,sharing=locked \ + --mount=type=cache,target=/var/lib/apt,sharing=locked \ + apt-get update -qq && \ + apt-get install -y --no-install-recommends rsync && \ + apt-cache depends --recurse --no-recommends --no-suggests --no-conflicts --no-breaks --no-replaces --no-enhances --no-pre-depends \ + curl \ + gosu \ + libjpeg62-turbo \ + libpq5 \ + libwebp7 \ + xmlsec1 \ + libjemalloc2 \ + libicu \ + | grep '^\w' > /tmp/pkg-list && \ + for arch in arm64 amd64; do \ + mkdir -p /tmp/debs-${arch} && \ + cd /tmp/debs-${arch} && \ + apt-get download $(sed "s/$/:${arch}/" /tmp/pkg-list); \ + done + +# Extract the debs for each architecture +# On the runtime image, /lib is a symlink to /usr/lib, so we need to copy the +# libraries to the right place, else the `COPY` won't work. +# On amd64, we'll also have a /lib64 folder with ld-linux-x86-64.so.2, which is +# already present in the runtime image. +RUN \ + for arch in arm64 amd64; do \ + mkdir -p /install-${arch}/var/lib/dpkg/status.d/ && \ + for deb in /tmp/debs-${arch}/*.deb; do \ + package_name=$(dpkg-deb -I ${deb} | awk '/^ Package: .*$/ {print $2}'); \ + echo "Extracting: ${package_name}"; \ + dpkg --ctrl-tarfile $deb | tar -Ox ./control > /install-${arch}/var/lib/dpkg/status.d/${package_name}; \ + dpkg --extract $deb /install-${arch}; \ + done; \ + rsync -avr /install-${arch}/lib/ /install-${arch}/usr/lib; \ + rm -rf /install-${arch}/lib /install-${arch}/lib64; \ + done + + +### +### Stage 3: runtime ### -FROM docker.io/library/python:${PYTHON_VERSION}-slim-bookworm +FROM docker.io/library/python:${PYTHON_VERSION}-slim-${DEBIAN_VERSION} + +ARG TARGETARCH LABEL org.opencontainers.image.url='https://matrix.org/docs/projects/server/synapse' LABEL org.opencontainers.image.documentation='https://github.com/element-hq/synapse/blob/master/docker/README.md' LABEL org.opencontainers.image.source='https://github.com/element-hq/synapse.git' LABEL org.opencontainers.image.licenses='AGPL-3.0-or-later' -RUN \ - --mount=type=cache,target=/var/cache/apt,sharing=locked \ - --mount=type=cache,target=/var/lib/apt,sharing=locked \ - apt-get update -qq && apt-get install -yqq \ - curl \ - gosu \ - libjpeg62-turbo \ - libpq5 \ - libwebp7 \ - xmlsec1 \ - libjemalloc2 \ - libicu72 \ - libssl-dev \ - openssl \ - && rm -rf /var/lib/apt/lists/* - +COPY --from=runtime-deps /install-${TARGETARCH} / COPY --from=builder /install /usr/local COPY ./docker/start.py /start.py COPY ./docker/conf /conf diff --git a/poetry.lock b/poetry.lock index 0d388eff6a..427a8de85e 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1155,61 +1155,72 @@ testing = ["coverage", "pytest", "pytest-cov", "pytest-regressions"] [[package]] name = "markupsafe" -version = "2.1.2" +version = "3.0.2" description = "Safely add untrusted strings to HTML/XML markup." optional = false -python-versions = ">=3.7" +python-versions = ">=3.9" files = [ - {file = "MarkupSafe-2.1.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:665a36ae6f8f20a4676b53224e33d456a6f5a72657d9c83c2aa00765072f31f7"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:340bea174e9761308703ae988e982005aedf427de816d1afe98147668cc03036"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:22152d00bf4a9c7c83960521fc558f55a1adbc0631fbb00a9471e097b19d72e1"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:28057e985dace2f478e042eaa15606c7efccb700797660629da387eb289b9323"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ca244fa73f50a800cf8c3ebf7fd93149ec37f5cb9596aa8873ae2c1d23498601"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:d9d971ec1e79906046aa3ca266de79eac42f1dbf3612a05dc9368125952bd1a1"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:7e007132af78ea9df29495dbf7b5824cb71648d7133cf7848a2a5dd00d36f9ff"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:7313ce6a199651c4ed9d7e4cfb4aa56fe923b1adf9af3b420ee14e6d9a73df65"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-win32.whl", hash = "sha256:c4a549890a45f57f1ebf99c067a4ad0cb423a05544accaf2b065246827ed9603"}, - {file = "MarkupSafe-2.1.2-cp310-cp310-win_amd64.whl", hash = "sha256:835fb5e38fd89328e9c81067fd642b3593c33e1e17e2fdbf77f5676abb14a156"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:2ec4f2d48ae59bbb9d1f9d7efb9236ab81429a764dedca114f5fdabbc3788013"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:608e7073dfa9e38a85d38474c082d4281f4ce276ac0010224eaba11e929dd53a"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:65608c35bfb8a76763f37036547f7adfd09270fbdbf96608be2bead319728fcd"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f2bfb563d0211ce16b63c7cb9395d2c682a23187f54c3d79bfec33e6705473c6"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:da25303d91526aac3672ee6d49a2f3db2d9502a4a60b55519feb1a4c7714e07d"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:9cad97ab29dfc3f0249b483412c85c8ef4766d96cdf9dcf5a1e3caa3f3661cf1"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:085fd3201e7b12809f9e6e9bc1e5c96a368c8523fad5afb02afe3c051ae4afcc"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:1bea30e9bf331f3fef67e0a3877b2288593c98a21ccb2cf29b74c581a4eb3af0"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-win32.whl", hash = "sha256:7df70907e00c970c60b9ef2938d894a9381f38e6b9db73c5be35e59d92e06625"}, - {file = "MarkupSafe-2.1.2-cp311-cp311-win_amd64.whl", hash = "sha256:e55e40ff0cc8cc5c07996915ad367fa47da6b3fc091fdadca7f5403239c5fec3"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:a6e40afa7f45939ca356f348c8e23048e02cb109ced1eb8420961b2f40fb373a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cf877ab4ed6e302ec1d04952ca358b381a882fbd9d1b07cccbfd61783561f98a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:63ba06c9941e46fa389d389644e2d8225e0e3e5ebcc4ff1ea8506dce646f8c8a"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f1cd098434e83e656abf198f103a8207a8187c0fc110306691a2e94a78d0abb2"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:55f44b440d491028addb3b88f72207d71eeebfb7b5dbf0643f7c023ae1fba619"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:a6f2fcca746e8d5910e18782f976489939d54a91f9411c32051b4aab2bd7c513"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:0b462104ba25f1ac006fdab8b6a01ebbfbce9ed37fd37fd4acd70c67c973e460"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-win32.whl", hash = "sha256:7668b52e102d0ed87cb082380a7e2e1e78737ddecdde129acadb0eccc5423859"}, - {file = "MarkupSafe-2.1.2-cp37-cp37m-win_amd64.whl", hash = "sha256:6d6607f98fcf17e534162f0709aaad3ab7a96032723d8ac8750ffe17ae5a0666"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:a806db027852538d2ad7555b203300173dd1b77ba116de92da9afbc3a3be3eed"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:a4abaec6ca3ad8660690236d11bfe28dfd707778e2442b45addd2f086d6ef094"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f03a532d7dee1bed20bc4884194a16160a2de9ffc6354b3878ec9682bb623c54"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4cf06cdc1dda95223e9d2d3c58d3b178aa5dacb35ee7e3bbac10e4e1faacb419"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:22731d79ed2eb25059ae3df1dfc9cb1546691cc41f4e3130fe6bfbc3ecbbecfa"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:f8ffb705ffcf5ddd0e80b65ddf7bed7ee4f5a441ea7d3419e861a12eaf41af58"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:8db032bf0ce9022a8e41a22598eefc802314e81b879ae093f36ce9ddf39ab1ba"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:2298c859cfc5463f1b64bd55cb3e602528db6fa0f3cfd568d3605c50678f8f03"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-win32.whl", hash = "sha256:50c42830a633fa0cf9e7d27664637532791bfc31c731a87b202d2d8ac40c3ea2"}, - {file = "MarkupSafe-2.1.2-cp38-cp38-win_amd64.whl", hash = "sha256:bb06feb762bade6bf3c8b844462274db0c76acc95c52abe8dbed28ae3d44a147"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:99625a92da8229df6d44335e6fcc558a5037dd0a760e11d84be2260e6f37002f"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8bca7e26c1dd751236cfb0c6c72d4ad61d986e9a41bbf76cb445f69488b2a2bd"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:40627dcf047dadb22cd25ea7ecfe9cbf3bbbad0482ee5920b582f3809c97654f"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:40dfd3fefbef579ee058f139733ac336312663c6706d1163b82b3003fb1925c4"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:090376d812fb6ac5f171e5938e82e7f2d7adc2b629101cec0db8b267815c85e2"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:2e7821bffe00aa6bd07a23913b7f4e01328c3d5cc0b40b36c0bd81d362faeb65"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:c0a33bc9f02c2b17c3ea382f91b4db0e6cde90b63b296422a939886a7a80de1c"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:b8526c6d437855442cdd3d87eede9c425c4445ea011ca38d937db299382e6fa3"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-win32.whl", hash = "sha256:137678c63c977754abe9086a3ec011e8fd985ab90631145dfb9294ad09c102a7"}, - {file = "MarkupSafe-2.1.2-cp39-cp39-win_amd64.whl", hash = "sha256:0576fe974b40a400449768941d5d0858cc624e3249dfd1e0c33674e5c7ca7aed"}, - {file = "MarkupSafe-2.1.2.tar.gz", hash = "sha256:abcabc8c2b26036d62d4c746381a6f7cf60aafcc653198ad678306986b09450d"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7e94c425039cde14257288fd61dcfb01963e658efbc0ff54f5306b06054700f8"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:9e2d922824181480953426608b81967de705c3cef4d1af983af849d7bd619158"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:38a9ef736c01fccdd6600705b09dc574584b89bea478200c5fbf112a6b0d5579"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bbcb445fa71794da8f178f0f6d66789a28d7319071af7a496d4d507ed566270d"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:57cb5a3cf367aeb1d316576250f65edec5bb3be939e9247ae594b4bcbc317dfb"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:3809ede931876f5b2ec92eef964286840ed3540dadf803dd570c3b7e13141a3b"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e07c3764494e3776c602c1e78e298937c3315ccc9043ead7e685b7f2b8d47b3c"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:b424c77b206d63d500bcb69fa55ed8d0e6a3774056bdc4839fc9298a7edca171"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-win32.whl", hash = "sha256:fcabf5ff6eea076f859677f5f0b6b5c1a51e70a376b0579e0eadef8db48c6b50"}, + {file = "MarkupSafe-3.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:6af100e168aa82a50e186c82875a5893c5597a0c1ccdb0d8b40240b1f28b969a"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:9025b4018f3a1314059769c7bf15441064b2207cb3f065e6ea1e7359cb46db9d"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:93335ca3812df2f366e80509ae119189886b0f3c2b81325d39efdb84a1e2ae93"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2cb8438c3cbb25e220c2ab33bb226559e7afb3baec11c4f218ffa7308603c832"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a123e330ef0853c6e822384873bef7507557d8e4a082961e1defa947aa59ba84"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1e084f686b92e5b83186b07e8a17fc09e38fff551f3602b249881fec658d3eca"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:d8213e09c917a951de9d09ecee036d5c7d36cb6cb7dbaece4c71a60d79fb9798"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:5b02fb34468b6aaa40dfc198d813a641e3a63b98c2b05a16b9f80b7ec314185e"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:0bff5e0ae4ef2e1ae4fdf2dfd5b76c75e5c2fa4132d05fc1b0dabcd20c7e28c4"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-win32.whl", hash = "sha256:6c89876f41da747c8d3677a2b540fb32ef5715f97b66eeb0c6b66f5e3ef6f59d"}, + {file = "MarkupSafe-3.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:70a87b411535ccad5ef2f1df5136506a10775d267e197e4cf531ced10537bd6b"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:9778bd8ab0a994ebf6f84c2b949e65736d5575320a17ae8984a77fab08db94cf"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:846ade7b71e3536c4e56b386c2a47adf5741d2d8b94ec9dc3e92e5e1ee1e2225"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1c99d261bd2d5f6b59325c92c73df481e05e57f19837bdca8413b9eac4bd8028"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e17c96c14e19278594aa4841ec148115f9c7615a47382ecb6b82bd8fea3ab0c8"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:88416bd1e65dcea10bc7569faacb2c20ce071dd1f87539ca2ab364bf6231393c"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:2181e67807fc2fa785d0592dc2d6206c019b9502410671cc905d132a92866557"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:52305740fe773d09cffb16f8ed0427942901f00adedac82ec8b67752f58a1b22"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ad10d3ded218f1039f11a75f8091880239651b52e9bb592ca27de44eed242a48"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-win32.whl", hash = "sha256:0f4ca02bea9a23221c0182836703cbf8930c5e9454bacce27e767509fa286a30"}, + {file = "MarkupSafe-3.0.2-cp312-cp312-win_amd64.whl", hash = "sha256:8e06879fc22a25ca47312fbe7c8264eb0b662f6db27cb2d3bbbc74b1df4b9b87"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ba9527cdd4c926ed0760bc301f6728ef34d841f405abf9d4f959c478421e4efd"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:f8b3d067f2e40fe93e1ccdd6b2e1d16c43140e76f02fb1319a05cf2b79d99430"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:569511d3b58c8791ab4c2e1285575265991e6d8f8700c7be0e88f86cb0672094"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:15ab75ef81add55874e7ab7055e9c397312385bd9ced94920f2802310c930396"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f3818cb119498c0678015754eba762e0d61e5b52d34c8b13d770f0719f7b1d79"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:cdb82a876c47801bb54a690c5ae105a46b392ac6099881cdfb9f6e95e4014c6a"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:cabc348d87e913db6ab4aa100f01b08f481097838bdddf7c7a84b7575b7309ca"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:444dcda765c8a838eaae23112db52f1efaf750daddb2d9ca300bcae1039adc5c"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-win32.whl", hash = "sha256:bcf3e58998965654fdaff38e58584d8937aa3096ab5354d493c77d1fdd66d7a1"}, + {file = "MarkupSafe-3.0.2-cp313-cp313-win_amd64.whl", hash = "sha256:e6a2a455bd412959b57a172ce6328d2dd1f01cb2135efda2e4576e8a23fa3b0f"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-macosx_10_13_universal2.whl", hash = "sha256:b5a6b3ada725cea8a5e634536b1b01c30bcdcd7f9c6fff4151548d5bf6b3a36c"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-macosx_11_0_arm64.whl", hash = "sha256:a904af0a6162c73e3edcb969eeeb53a63ceeb5d8cf642fade7d39e7963a22ddb"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4aa4e5faecf353ed117801a068ebab7b7e09ffb6e1d5e412dc852e0da018126c"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c0ef13eaeee5b615fb07c9a7dadb38eac06a0608b41570d8ade51c56539e509d"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d16a81a06776313e817c951135cf7340a3e91e8c1ff2fac444cfd75fffa04afe"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_aarch64.whl", hash = "sha256:6381026f158fdb7c72a168278597a5e3a5222e83ea18f543112b2662a9b699c5"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_i686.whl", hash = "sha256:3d79d162e7be8f996986c064d1c7c817f6df3a77fe3d6859f6f9e7be4b8c213a"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-musllinux_1_2_x86_64.whl", hash = "sha256:131a3c7689c85f5ad20f9f6fb1b866f402c445b220c19fe4308c0b147ccd2ad9"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-win32.whl", hash = "sha256:ba8062ed2cf21c07a9e295d5b8a2a5ce678b913b45fdf68c32d95d6c1291e0b6"}, + {file = "MarkupSafe-3.0.2-cp313-cp313t-win_amd64.whl", hash = "sha256:e444a31f8db13eb18ada366ab3cf45fd4b31e4db1236a4448f68778c1d1a5a2f"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:eaa0a10b7f72326f1372a713e73c3f739b524b3af41feb43e4921cb529f5929a"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:48032821bbdf20f5799ff537c7ac3d1fba0ba032cfc06194faffa8cda8b560ff"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1a9d3f5f0901fdec14d8d2f66ef7d035f2157240a433441719ac9a3fba440b13"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:88b49a3b9ff31e19998750c38e030fc7bb937398b1f78cfa599aaef92d693144"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cfad01eed2c2e0c01fd0ecd2ef42c492f7f93902e39a42fc9ee1692961443a29"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:1225beacc926f536dc82e45f8a4d68502949dc67eea90eab715dea3a21c1b5f0"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:3169b1eefae027567d1ce6ee7cae382c57fe26e82775f460f0b2778beaad66c0"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:eb7972a85c54febfb25b5c4b4f3af4dcc731994c7da0d8a0b4a6eb0640e1d178"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-win32.whl", hash = "sha256:8c4e8c3ce11e1f92f6536ff07154f9d49677ebaaafc32db9db4620bc11ed480f"}, + {file = "MarkupSafe-3.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:6e296a513ca3d94054c2c881cc913116e90fd030ad1c656b3869762b754f5f8a"}, + {file = "markupsafe-3.0.2.tar.gz", hash = "sha256:ee55d3edf80167e48ea11a923c7386f4669df67d7994554387f84e7d8b0a2bf0"}, ] [[package]] From 6b4cc9f3f60fc1ee7da25c80d1aaba6d38705f3f Mon Sep 17 00:00:00 2001 From: Shay Date: Thu, 20 Feb 2025 10:40:30 -0800 Subject: [PATCH 43/65] Document suspension Admin API (#18162) Missed in the transition from experimental to stable. Fixes #18160 --- changelog.d/18162.misc | 1 + docs/admin_api/user_admin_api.md | 26 ++++++++++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 changelog.d/18162.misc diff --git a/changelog.d/18162.misc b/changelog.d/18162.misc new file mode 100644 index 0000000000..4d46979641 --- /dev/null +++ b/changelog.d/18162.misc @@ -0,0 +1 @@ +Document suspension Admin API. \ No newline at end of file diff --git a/docs/admin_api/user_admin_api.md b/docs/admin_api/user_admin_api.md index 2742d2d2cd..77ce800507 100644 --- a/docs/admin_api/user_admin_api.md +++ b/docs/admin_api/user_admin_api.md @@ -414,6 +414,32 @@ The following actions are **NOT** performed. The list may be incomplete. - Remove from monthly active users - Remove user's consent information (consent version and timestamp) +## Suspend/Unsuspend Account + +This API allows an admin to suspend/unsuspend an account. While an account is suspended, the user is +prohibited from sending invites, joining or knocking on rooms, sending messages, changing profile data, and redacting messages other than their own. + +The api is: + +``` +PUT /_synapse/admin/v1/suspend/ +``` + +with a body of: + +```json +{ + "suspend": true +} +``` + +To unsuspend a user, use the same endpoint with a body of: +```json +{ + "suspend": false +} +``` + ## Reset password **Note:** This API is disabled when MSC3861 is enabled. [See #15582](https://github.com/matrix-org/synapse/pull/15582) From caa1f9d806945e056261ec6879da3b1a1cc23b17 Mon Sep 17 00:00:00 2001 From: Eric Eastwood Date: Thu, 20 Feb 2025 17:56:53 -0600 Subject: [PATCH 44/65] Add support for overriding `id_token_signing_alg_values_supported` for an OpenID identity provider (#18177) Normally, when `discovery` is enabled, `id_token_signing_alg_values_supported` comes from the OpenID Discovery Document (`/.well-known/openid-configuration`). If nothing was specified, we default to supporting `RS256` in the downstream usage. This PR just adds support for adding a default/overriding the the discovered value [just like we do for other things like the `token_endpoint`](https://github.com/element-hq/synapse/blob/1525a3b4d48a0f5657d61423e1f205bff9a77948/docs/usage/configuration/config_documentation.md#oidc_providers), etc. --- changelog.d/18177.feature | 1 + .../configuration/config_documentation.md | 18 ++++++ synapse/config/oidc.py | 35 ++++++++++ synapse/handlers/oidc.py | 5 ++ tests/handlers/test_oidc.py | 64 +++++++++++++++++-- 5 files changed, 119 insertions(+), 4 deletions(-) create mode 100644 changelog.d/18177.feature diff --git a/changelog.d/18177.feature b/changelog.d/18177.feature new file mode 100644 index 0000000000..71d568474b --- /dev/null +++ b/changelog.d/18177.feature @@ -0,0 +1 @@ +Add support for specifying/overriding `id_token_signing_alg_values_supported` for an OpenID identity provider. diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index e3c06d5371..facf60a043 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -3579,6 +3579,24 @@ Options for each entry include: to `auto`, which uses PKCE if supported during metadata discovery. Set to `always` to force enable PKCE or `never` to force disable PKCE. +* `id_token_signing_alg_values_supported`: List of the JWS signing algorithms (`alg` + values) that are supported for signing the `id_token`. + + This is *not* required if `discovery` is disabled. We default to supporting `RS256` in + the downstream usage if no algorithms are configured here or in the discovery + document. + + According to the spec, the algorithm `"RS256"` MUST be included. The absolute rigid + approach would be to reject this provider as non-compliant if it's not included but we + simply allow whatever and see what happens (you're the one that configured the value + and cooperating with the identity provider). + + The `alg` value `"none"` MAY be supported but can only be used if the Authorization + Endpoint does not include `id_token` in the `response_type` (ex. + `/authorize?response_type=code` where `none` can apply, + `/authorize?response_type=code%20id_token` where `none` can't apply) (such as when + using the Authorization Code Flow). + * `scopes`: list of scopes to request. This should normally include the "openid" scope. Defaults to `["openid"]`. diff --git a/synapse/config/oidc.py b/synapse/config/oidc.py index d0a03baf55..fc4bc35b30 100644 --- a/synapse/config/oidc.py +++ b/synapse/config/oidc.py @@ -125,6 +125,10 @@ OIDC_PROVIDER_CONFIG_SCHEMA = { "enum": ["client_secret_basic", "client_secret_post", "none"], }, "pkce_method": {"type": "string", "enum": ["auto", "always", "never"]}, + "id_token_signing_alg_values_supported": { + "type": "array", + "items": {"type": "string"}, + }, "scopes": {"type": "array", "items": {"type": "string"}}, "authorization_endpoint": {"type": "string"}, "token_endpoint": {"type": "string"}, @@ -326,6 +330,9 @@ def _parse_oidc_config_dict( client_secret_jwt_key=client_secret_jwt_key, client_auth_method=client_auth_method, pkce_method=oidc_config.get("pkce_method", "auto"), + id_token_signing_alg_values_supported=oidc_config.get( + "id_token_signing_alg_values_supported" + ), scopes=oidc_config.get("scopes", ["openid"]), authorization_endpoint=oidc_config.get("authorization_endpoint"), token_endpoint=oidc_config.get("token_endpoint"), @@ -402,6 +409,34 @@ class OidcProviderConfig: # Valid values are 'auto', 'always', and 'never'. pkce_method: str + id_token_signing_alg_values_supported: Optional[List[str]] + """ + List of the JWS signing algorithms (`alg` values) that are supported for signing the + `id_token`. + + This is *not* required if `discovery` is disabled. We default to supporting `RS256` + in the downstream usage if no algorithms are configured here or in the discovery + document. + + According to the spec, the algorithm `"RS256"` MUST be included. The absolute rigid + approach would be to reject this provider as non-compliant if it's not included but + we can just allow whatever and see what happens (they're the ones that configured + the value and cooperating with the identity provider). It wouldn't be wise to add it + ourselves because absence of `RS256` might indicate that the provider actually + doesn't support it, despite the spec requirement. Adding it silently could lead to + failed authentication attempts or strange mismatch attacks. + + The `alg` value `"none"` MAY be supported but can only be used if the Authorization + Endpoint does not include `id_token` in the `response_type` (ex. + `/authorize?response_type=code` where `none` can apply, + `/authorize?response_type=code%20id_token` where `none` can't apply) (such as when + using the Authorization Code Flow). + + Spec: + - https://openid.net/specs/openid-connect-discovery-1_0.html#ProviderMetadata + - https://openid.net/specs/openid-connect-core-1_0.html#AuthorizationExamples + """ + # list of scopes to request scopes: Collection[str] diff --git a/synapse/handlers/oidc.py b/synapse/handlers/oidc.py index c9109c9e79..76b692928d 100644 --- a/synapse/handlers/oidc.py +++ b/synapse/handlers/oidc.py @@ -640,6 +640,11 @@ class OidcProvider: elif self._config.pkce_method == "never": metadata.pop("code_challenge_methods_supported", None) + if self._config.id_token_signing_alg_values_supported: + metadata["id_token_signing_alg_values_supported"] = ( + self._config.id_token_signing_alg_values_supported + ) + self._validate_metadata(metadata) return metadata diff --git a/tests/handlers/test_oidc.py b/tests/handlers/test_oidc.py index 1b43ee43c6..5ffc5a90a8 100644 --- a/tests/handlers/test_oidc.py +++ b/tests/handlers/test_oidc.py @@ -70,12 +70,16 @@ DEFAULT_CONFIG = { } # extends the default config with explicit OAuth2 endpoints instead of using discovery +# +# We add "explicit" to things to make them different from the discovered values to make +# sure that the explicit values override the discovered ones. EXPLICIT_ENDPOINT_CONFIG = { **DEFAULT_CONFIG, "discover": False, - "authorization_endpoint": ISSUER + "authorize", - "token_endpoint": ISSUER + "token", - "jwks_uri": ISSUER + "jwks", + "authorization_endpoint": ISSUER + "authorize-explicit", + "token_endpoint": ISSUER + "token-explicit", + "jwks_uri": ISSUER + "jwks-explicit", + "id_token_signing_alg_values_supported": ["RS256", ""], } @@ -259,12 +263,64 @@ class OidcHandlerTestCase(HomeserverTestCase): self.get_success(self.provider.load_metadata()) self.fake_server.get_metadata_handler.assert_not_called() + @override_config({"oidc_config": {**EXPLICIT_ENDPOINT_CONFIG, "discover": True}}) + def test_discovery_with_explicit_config(self) -> None: + """ + The handler should discover the endpoints from OIDC discovery document but + values are overriden by the explicit config. + """ + # This would throw if some metadata were invalid + metadata = self.get_success(self.provider.load_metadata()) + self.fake_server.get_metadata_handler.assert_called_once() + + self.assertEqual(metadata.issuer, self.fake_server.issuer) + # It seems like authlib does not have that defined in its metadata models + self.assertEqual( + metadata.get("userinfo_endpoint"), + self.fake_server.userinfo_endpoint, + ) + + # Ensure the values are overridden correctly since these were configured + # explicitly + self.assertEqual( + metadata.authorization_endpoint, + EXPLICIT_ENDPOINT_CONFIG["authorization_endpoint"], + ) + self.assertEqual( + metadata.token_endpoint, EXPLICIT_ENDPOINT_CONFIG["token_endpoint"] + ) + self.assertEqual(metadata.jwks_uri, EXPLICIT_ENDPOINT_CONFIG["jwks_uri"]) + self.assertEqual( + metadata.id_token_signing_alg_values_supported, + EXPLICIT_ENDPOINT_CONFIG["id_token_signing_alg_values_supported"], + ) + + # subsequent calls should be cached + self.reset_mocks() + self.get_success(self.provider.load_metadata()) + self.fake_server.get_metadata_handler.assert_not_called() + @override_config({"oidc_config": EXPLICIT_ENDPOINT_CONFIG}) def test_no_discovery(self) -> None: """When discovery is disabled, it should not try to load from discovery document.""" - self.get_success(self.provider.load_metadata()) + metadata = self.get_success(self.provider.load_metadata()) self.fake_server.get_metadata_handler.assert_not_called() + # Ensure the values are overridden correctly since these were configured + # explicitly + self.assertEqual( + metadata.authorization_endpoint, + EXPLICIT_ENDPOINT_CONFIG["authorization_endpoint"], + ) + self.assertEqual( + metadata.token_endpoint, EXPLICIT_ENDPOINT_CONFIG["token_endpoint"] + ) + self.assertEqual(metadata.jwks_uri, EXPLICIT_ENDPOINT_CONFIG["jwks_uri"]) + self.assertEqual( + metadata.id_token_signing_alg_values_supported, + EXPLICIT_ENDPOINT_CONFIG["id_token_signing_alg_values_supported"], + ) + @override_config({"oidc_config": DEFAULT_CONFIG}) def test_load_jwks(self) -> None: """JWKS loading is done once (then cached) if used.""" From 8fd7148e6ab3c1346dbf6715bf60c50b0d3fa3b8 Mon Sep 17 00:00:00 2001 From: Shay Date: Fri, 21 Feb 2025 02:06:44 -0800 Subject: [PATCH 45/65] Prevent suspended users from sending encrypted messages (#18157) Missed in the first round. --- changelog.d/18157.bugfix | 1 + synapse/handlers/message.py | 32 ++++++++-- tests/rest/client/test_rooms.py | 102 +++++++++++++++++++++++++++++++- 3 files changed, 128 insertions(+), 7 deletions(-) create mode 100644 changelog.d/18157.bugfix diff --git a/changelog.d/18157.bugfix b/changelog.d/18157.bugfix new file mode 100644 index 0000000000..307e9c96ff --- /dev/null +++ b/changelog.d/18157.bugfix @@ -0,0 +1 @@ +Prevent suspended users from sending encrypted messages. diff --git a/synapse/handlers/message.py b/synapse/handlers/message.py index df3010ecf6..4642b8b578 100644 --- a/synapse/handlers/message.py +++ b/synapse/handlers/message.py @@ -644,11 +644,33 @@ class EventCreationHandler: """ await self.auth_blocking.check_auth_blocking(requester=requester) - if event_dict["type"] == EventTypes.Message: - requester_suspended = await self.store.get_user_suspended_status( - requester.user.to_string() - ) - if requester_suspended: + requester_suspended = await self.store.get_user_suspended_status( + requester.user.to_string() + ) + if requester_suspended: + # We want to allow suspended users to perform "corrective" actions + # asked of them by server admins, such as redact their messages and + # leave rooms. + if event_dict["type"] in ["m.room.redaction", "m.room.member"]: + if event_dict["type"] == "m.room.redaction": + event = await self.store.get_event( + event_dict["content"]["redacts"], allow_none=True + ) + if event: + if event.sender != requester.user.to_string(): + raise SynapseError( + 403, + "You can only redact your own events while account is suspended.", + Codes.USER_ACCOUNT_SUSPENDED, + ) + if event_dict["type"] == "m.room.member": + if event_dict["content"]["membership"] != "leave": + raise SynapseError( + 403, + "Changing membership while account is suspended is not allowed.", + Codes.USER_ACCOUNT_SUSPENDED, + ) + else: raise SynapseError( 403, "Sending messages while account is suspended is not allowed.", diff --git a/tests/rest/client/test_rooms.py b/tests/rest/client/test_rooms.py index 604b585150..f6ec8a8b7b 100644 --- a/tests/rest/client/test_rooms.py +++ b/tests/rest/client/test_rooms.py @@ -1371,6 +1371,23 @@ class RoomJoinTestCase(RoomBase): ) self.assertEqual(channel.json_body["errcode"], "M_USER_SUSPENDED") + def test_suspended_user_can_leave_room(self) -> None: + channel = self.make_request( + "POST", f"/join/{self.room1}", access_token=self.tok1 + ) + self.assertEqual(channel.code, 200) + + # set the user as suspended + self.get_success(self.store.set_user_suspended_status(self.user1, True)) + + # leave room + channel = self.make_request( + "POST", + f"/rooms/{self.room1}/leave", + access_token=self.tok1, + ) + self.assertEqual(channel.code, 200) + class RoomAppserviceTsParamTestCase(unittest.HomeserverTestCase): servlets = [ @@ -3989,10 +4006,25 @@ class UserSuspensionTests(unittest.HomeserverTestCase): self.user2 = self.register_user("teresa", "hackme") self.tok2 = self.login("teresa", "hackme") - self.room1 = self.helper.create_room_as(room_creator=self.user1, tok=self.tok1) + self.admin = self.register_user("admin", "pass", True) + self.admin_tok = self.login("admin", "pass") + + self.room1 = self.helper.create_room_as( + room_creator=self.user1, tok=self.tok1, room_version="11" + ) self.store = hs.get_datastores().main - def test_suspended_user_cannot_send_message_to_room(self) -> None: + self.room2 = self.helper.create_room_as( + room_creator=self.user1, is_public=False, tok=self.tok1 + ) + self.helper.send_state( + self.room2, + EventTypes.RoomEncryption, + {EventContentFields.ENCRYPTION_ALGORITHM: "m.megolm.v1.aes-sha2"}, + tok=self.tok1, + ) + + def test_suspended_user_cannot_send_message_to_public_room(self) -> None: # set the user as suspended self.get_success(self.store.set_user_suspended_status(self.user1, True)) @@ -4004,6 +4036,24 @@ class UserSuspensionTests(unittest.HomeserverTestCase): ) self.assertEqual(channel.json_body["errcode"], "M_USER_SUSPENDED") + def test_suspended_user_cannot_send_message_to_encrypted_room(self) -> None: + channel = self.make_request( + "PUT", + f"/_synapse/admin/v1/suspend/{self.user1}", + {"suspend": True}, + access_token=self.admin_tok, + ) + self.assertEqual(channel.code, 200) + self.assertEqual(channel.json_body, {f"user_{self.user1}_suspended": True}) + + channel = self.make_request( + "PUT", + f"/rooms/{self.room2}/send/m.room.encrypted/1", + access_token=self.tok1, + content={}, + ) + self.assertEqual(channel.json_body["errcode"], "M_USER_SUSPENDED") + def test_suspended_user_cannot_change_profile_data(self) -> None: # set the user as suspended self.get_success(self.store.set_user_suspended_status(self.user1, True)) @@ -4069,3 +4119,51 @@ class UserSuspensionTests(unittest.HomeserverTestCase): shorthand=False, ) self.assertEqual(channel.code, 200) + + channel = self.make_request( + "PUT", + f"/_matrix/client/v3/rooms/{self.room1}/send/m.room.redaction/3456346", + access_token=self.tok1, + content={"reason": "bogus", "redacts": event_id}, + shorthand=False, + ) + self.assertEqual(channel.json_body["errcode"], "M_USER_SUSPENDED") + + channel = self.make_request( + "PUT", + f"/_matrix/client/v3/rooms/{self.room1}/send/m.room.redaction/3456346", + access_token=self.tok1, + content={"reason": "bogus", "redacts": event_id2}, + shorthand=False, + ) + self.assertEqual(channel.code, 200) + + def test_suspended_user_cannot_ban_others(self) -> None: + # user to ban joins room user1 created + self.make_request("POST", f"/rooms/{self.room1}/join", access_token=self.tok2) + + # suspend user1 + self.get_success(self.store.set_user_suspended_status(self.user1, True)) + + # user1 tries to ban other user while suspended + channel = self.make_request( + "POST", + f"/_matrix/client/v3/rooms/{self.room1}/ban", + access_token=self.tok1, + content={"reason": "spite", "user_id": self.user2}, + shorthand=False, + ) + self.assertEqual(channel.json_body["errcode"], "M_USER_SUSPENDED") + + # un-suspend user1 + self.get_success(self.store.set_user_suspended_status(self.user1, False)) + + # ban now goes through + channel = self.make_request( + "POST", + f"/_matrix/client/v3/rooms/{self.room1}/ban", + access_token=self.tok1, + content={"reason": "spite", "user_id": self.user2}, + shorthand=False, + ) + self.assertEqual(channel.code, 200) From b2a187f49b390b9c5afa56cca03b629fe0f169c5 Mon Sep 17 00:00:00 2001 From: Erik Johnston Date: Mon, 24 Feb 2025 13:06:16 +0100 Subject: [PATCH 46/65] Disable room list publication by default (#18175) This is in line with our general policy of ensuring that the default config is reasonably locked down. SyTest PR to fix tests: https://github.com/matrix-org/sytest/pull/1396 --- changelog.d/18175.misc | 1 + .../conf/workers-shared-extra.yaml.j2 | 5 +++++ docs/upgrade.md | 20 +++++++++++++++++++ .../configuration/config_documentation.md | 8 ++++++-- synapse/config/room_directory.py | 4 +--- tests/handlers/test_directory.py | 1 + tests/handlers/test_room_list.py | 6 ++++++ tests/rest/admin/test_room.py | 3 +++ tests/rest/client/test_rooms.py | 6 ++++++ 9 files changed, 49 insertions(+), 5 deletions(-) create mode 100644 changelog.d/18175.misc diff --git a/changelog.d/18175.misc b/changelog.d/18175.misc new file mode 100644 index 0000000000..c12a30b7c1 --- /dev/null +++ b/changelog.d/18175.misc @@ -0,0 +1 @@ +Disable room list publication by default. diff --git a/docker/complement/conf/workers-shared-extra.yaml.j2 b/docker/complement/conf/workers-shared-extra.yaml.j2 index 797d58e9b3..ac0c4bb851 100644 --- a/docker/complement/conf/workers-shared-extra.yaml.j2 +++ b/docker/complement/conf/workers-shared-extra.yaml.j2 @@ -139,4 +139,9 @@ caches: sync_response_cache_duration: 0 +# Complement assumes that it can publish to the room list by default. +room_list_publication_rules: + - action: allow + + {% include "shared-orig.yaml.j2" %} diff --git a/docs/upgrade.md b/docs/upgrade.md index 6c96cb91a3..7e4cd52e1d 100644 --- a/docs/upgrade.md +++ b/docs/upgrade.md @@ -117,6 +117,26 @@ each upgrade are complete before moving on to the next upgrade, to avoid stacking them up. You can monitor the currently running background updates with [the Admin API](usage/administration/admin_api/background_updates.html#status). +# Upgrading to v1.126.0 + +## Room list publication rules change + +The default [`room_list_publication_rules`] setting was changed to disallow +anyone (except server admins) from publishing to the room list by default. + +This is in line with Synapse policy of locking down features by default that can +be abused without moderation. + +To keep the previous behavior of allowing publication by default, add the +following to the config: + +```yaml +room_list_publication_rules: + - "action": "allow" +``` + +[`room_list_publication_rules`]: usage/configuration/config_documentation.md#room_list_publication_rules + # Upgrading to v1.122.0 ## Dropping support for PostgreSQL 11 and 12 diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index facf60a043..8d9a71fb5f 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -4245,8 +4245,8 @@ unwanted entries from being published in the public room list. The format of this option is the same as that for [`alias_creation_rules`](#alias_creation_rules): an optional list of 0 or more -rules. By default, no list is provided, meaning that all rooms may be -published to the room list. +rules. By default, no list is provided, meaning that no one may publish to the +room list (except server admins). Otherwise, requests to publish a room are matched against each rule in order. The first rule that matches decides if the request is allowed or denied. If no @@ -4272,6 +4272,10 @@ Note that the patterns match against fully qualified IDs, e.g. against of `alice`, `room` and `abcedgghijk`. +_Changed in Synapse 1.126.0: The default was changed to deny publishing to the +room list by default_ + + Example configuration: ```yaml diff --git a/synapse/config/room_directory.py b/synapse/config/room_directory.py index 704895cf9a..f0349b68f2 100644 --- a/synapse/config/room_directory.py +++ b/synapse/config/room_directory.py @@ -54,9 +54,7 @@ class RoomDirectoryConfig(Config): for rule in room_list_publication_rules ] else: - self._room_list_publication_rules = [ - _RoomDirectoryRule("room_list_publication_rules", {"action": "allow"}) - ] + self._room_list_publication_rules = [] def is_alias_creation_allowed(self, user_id: str, room_id: str, alias: str) -> bool: """Checks if the given user is allowed to create the given alias diff --git a/tests/handlers/test_directory.py b/tests/handlers/test_directory.py index 4a3e36ffde..b7058d8002 100644 --- a/tests/handlers/test_directory.py +++ b/tests/handlers/test_directory.py @@ -587,6 +587,7 @@ class TestRoomListSearchDisabled(unittest.HomeserverTestCase): self.room_list_handler = hs.get_room_list_handler() self.directory_handler = hs.get_directory_handler() + @unittest.override_config({"room_list_publication_rules": [{"action": "allow"}]}) def test_disabling_room_list(self) -> None: self.room_list_handler.enable_room_list_search = True self.directory_handler.enable_room_list_search = True diff --git a/tests/handlers/test_room_list.py b/tests/handlers/test_room_list.py index 4d22ef98c2..45cef09b22 100644 --- a/tests/handlers/test_room_list.py +++ b/tests/handlers/test_room_list.py @@ -6,6 +6,7 @@ from synapse.rest.client import directory, login, room from synapse.types import JsonDict from tests import unittest +from tests.utils import default_config class RoomListHandlerTestCase(unittest.HomeserverTestCase): @@ -30,6 +31,11 @@ class RoomListHandlerTestCase(unittest.HomeserverTestCase): assert channel.code == HTTPStatus.OK, f"couldn't publish room: {channel.result}" return room_id + def default_config(self) -> JsonDict: + config = default_config("test") + config["room_list_publication_rules"] = [{"action": "allow"}] + return config + def test_acls_applied_to_room_directory_results(self) -> None: """ Creates 3 rooms. Room 2 has an ACL that only permits the homeservers diff --git a/tests/rest/admin/test_room.py b/tests/rest/admin/test_room.py index 1817d67a00..1d44106bd7 100644 --- a/tests/rest/admin/test_room.py +++ b/tests/rest/admin/test_room.py @@ -1282,6 +1282,7 @@ class RoomTestCase(unittest.HomeserverTestCase): self.admin_user = self.register_user("admin", "pass", admin=True) self.admin_user_tok = self.login("admin", "pass") + @unittest.override_config({"room_list_publication_rules": [{"action": "allow"}]}) def test_list_rooms(self) -> None: """Test that we can list rooms""" # Create 3 test rooms @@ -1795,6 +1796,7 @@ class RoomTestCase(unittest.HomeserverTestCase): self.assertEqual(room_id, channel.json_body["rooms"][0].get("room_id")) self.assertEqual("ж", channel.json_body["rooms"][0].get("name")) + @unittest.override_config({"room_list_publication_rules": [{"action": "allow"}]}) def test_filter_public_rooms(self) -> None: self.helper.create_room_as( self.admin_user, tok=self.admin_user_tok, is_public=True @@ -1872,6 +1874,7 @@ class RoomTestCase(unittest.HomeserverTestCase): self.assertEqual(1, response.json_body["total_rooms"]) self.assertEqual(1, len(response.json_body["rooms"])) + @unittest.override_config({"room_list_publication_rules": [{"action": "allow"}]}) def test_single_room(self) -> None: """Test that a single room can be requested correctly""" # Create two test rooms diff --git a/tests/rest/client/test_rooms.py b/tests/rest/client/test_rooms.py index f6ec8a8b7b..a7108b905a 100644 --- a/tests/rest/client/test_rooms.py +++ b/tests/rest/client/test_rooms.py @@ -67,6 +67,7 @@ from tests.http.server._base import make_request_with_cancellation_test from tests.storage.test_stream import PaginationTestCase from tests.test_utils.event_injection import create_event from tests.unittest import override_config +from tests.utils import default_config PATH_PREFIX = b"/_matrix/client/api/v1" @@ -2565,6 +2566,11 @@ class PublicRoomsRoomTypeFilterTestCase(unittest.HomeserverTestCase): tok=self.token, ) + def default_config(self) -> JsonDict: + config = default_config("test") + config["room_list_publication_rules"] = [{"action": "allow"}] + return config + def make_public_rooms_request( self, room_types: Optional[List[Union[str, None]]], From 805e8705c7e918377191a18ec442872937ddd172 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 24 Feb 2025 12:27:56 +0000 Subject: [PATCH 47/65] Bump sigstore/cosign-installer from 3.8.0 to 3.8.1 (#18185) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [sigstore/cosign-installer](https://github.com/sigstore/cosign-installer) from 3.8.0 to 3.8.1.
Release notes

Sourced from sigstore/cosign-installer's releases.

v3.8.1

What's Changed

Full Changelog: https://github.com/sigstore/cosign-installer/compare/v3...v3.8.1

Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=sigstore/cosign-installer&package-manager=github_actions&previous-version=3.8.0&new-version=3.8.1)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/docker.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/docker.yml b/.github/workflows/docker.yml index b439cb0915..e8319364dd 100644 --- a/.github/workflows/docker.yml +++ b/.github/workflows/docker.yml @@ -30,7 +30,7 @@ jobs: run: docker buildx inspect - name: Install Cosign - uses: sigstore/cosign-installer@v3.8.0 + uses: sigstore/cosign-installer@v3.8.1 - name: Checkout repository uses: actions/checkout@v4 From a5c3fe6c1e3b341cab9a1b2b9927a4b89bf1492b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 24 Feb 2025 12:56:04 +0000 Subject: [PATCH 48/65] Bump types-psycopg2 from 2.9.21.20241019 to 2.9.21.20250121 (#18188) Bumps [types-psycopg2](https://github.com/python/typeshed) from 2.9.21.20241019 to 2.9.21.20250121.
Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=types-psycopg2&package-manager=pip&previous-version=2.9.21.20241019&new-version=2.9.21.20250121)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- poetry.lock | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/poetry.lock b/poetry.lock index 427a8de85e..189fa9063d 100644 --- a/poetry.lock +++ b/poetry.lock @@ -2832,13 +2832,13 @@ files = [ [[package]] name = "types-psycopg2" -version = "2.9.21.20241019" +version = "2.9.21.20250121" description = "Typing stubs for psycopg2" optional = false -python-versions = ">=3.8" +python-versions = ">=3.9" files = [ - {file = "types-psycopg2-2.9.21.20241019.tar.gz", hash = "sha256:bca89b988d2ebd19bcd08b177d22a877ea8b841decb10ed130afcf39404612fa"}, - {file = "types_psycopg2-2.9.21.20241019-py3-none-any.whl", hash = "sha256:44d091e67732d16a941baae48cd7b53bf91911bc36888652447cf1ef0c1fb3f6"}, + {file = "types_psycopg2-2.9.21.20250121-py3-none-any.whl", hash = "sha256:b890dc6f5a08b6433f0ff73a4ec9a834deedad3e914f2a4a6fd43df021f745f1"}, + {file = "types_psycopg2-2.9.21.20250121.tar.gz", hash = "sha256:2b0e2cd0f3747af1ae25a7027898716d80209604770ef3cbf350fe055b9c349b"}, ] [[package]] From b9276e21eee569c244ef40cab0285fc067dbf702 Mon Sep 17 00:00:00 2001 From: Quentin Gliech Date: Tue, 25 Feb 2025 11:34:33 +0100 Subject: [PATCH 49/65] Fix MSC4108 'rendez-vous' responses with some reverse proxy in the front of Synapse (#18178) MSC4108 relies on ETag to determine if something has changed on the rendez-vous channel. Strong and correct ETag comparison works if the response body is bit-for-bit identical, which isn't the case if a proxy in the middle compresses the response on the fly. This adds a `no-transform` directive to the `Cache-Control` header, which tells proxies not to transform the response body. Additionally, some proxies (nginx) will switch to `Transfer-Encoding: chunked` if it doesn't know the Content-Length of the response, and 'weakening' the ETag if that's the case. I've added `Content-Length` headers to all responses, to hopefully solve that. This basically fixes QR-code login when nginx or cloudflare is involved, with gzip/zstd/deflate compression enabled. --- changelog.d/18178.bugfix | 1 + rust/src/rendezvous/mod.rs | 6 +++++- tests/rest/client/test_rendezvous.py | 6 ++++-- 3 files changed, 10 insertions(+), 3 deletions(-) create mode 100644 changelog.d/18178.bugfix diff --git a/changelog.d/18178.bugfix b/changelog.d/18178.bugfix new file mode 100644 index 0000000000..d91327803b --- /dev/null +++ b/changelog.d/18178.bugfix @@ -0,0 +1 @@ +Fix MSC4108 QR-code login not working with some reverse-proxy setups. diff --git a/rust/src/rendezvous/mod.rs b/rust/src/rendezvous/mod.rs index 23de668102..3148e0f67a 100644 --- a/rust/src/rendezvous/mod.rs +++ b/rust/src/rendezvous/mod.rs @@ -47,7 +47,7 @@ fn prepare_headers(headers: &mut HeaderMap, session: &Session) { headers.typed_insert(AccessControlAllowOrigin::ANY); headers.typed_insert(AccessControlExposeHeaders::from_iter([ETAG])); headers.typed_insert(Pragma::no_cache()); - headers.typed_insert(CacheControl::new().with_no_store()); + headers.typed_insert(CacheControl::new().with_no_store().with_no_transform()); headers.typed_insert(session.etag()); headers.typed_insert(session.expires()); headers.typed_insert(session.last_modified()); @@ -192,10 +192,12 @@ impl RendezvousHandler { "url": uri, }) .to_string(); + let length = response.len() as _; let mut response = Response::new(response.as_bytes()); *response.status_mut() = StatusCode::CREATED; response.headers_mut().typed_insert(ContentType::json()); + response.headers_mut().typed_insert(ContentLength(length)); prepare_headers(response.headers_mut(), &session); http_response_to_twisted(twisted_request, response)?; @@ -299,6 +301,7 @@ impl RendezvousHandler { // proxy/cache setup which strips the ETag header if there is no Content-Type set. // Specifically, we noticed this behaviour when placing Synapse behind Cloudflare. response.headers_mut().typed_insert(ContentType::text()); + response.headers_mut().typed_insert(ContentLength(0)); http_response_to_twisted(twisted_request, response)?; @@ -316,6 +319,7 @@ impl RendezvousHandler { response .headers_mut() .typed_insert(AccessControlAllowOrigin::ANY); + response.headers_mut().typed_insert(ContentLength(0)); http_response_to_twisted(twisted_request, response)?; Ok(()) diff --git a/tests/rest/client/test_rendezvous.py b/tests/rest/client/test_rendezvous.py index ab701680a6..83a5cbdc15 100644 --- a/tests/rest/client/test_rendezvous.py +++ b/tests/rest/client/test_rendezvous.py @@ -117,10 +117,11 @@ class RendezvousServletTestCase(unittest.HomeserverTestCase): headers = dict(channel.headers.getAllRawHeaders()) self.assertIn(b"ETag", headers) self.assertIn(b"Expires", headers) + self.assertIn(b"Content-Length", headers) self.assertEqual(headers[b"Content-Type"], [b"application/json"]) self.assertEqual(headers[b"Access-Control-Allow-Origin"], [b"*"]) self.assertEqual(headers[b"Access-Control-Expose-Headers"], [b"etag"]) - self.assertEqual(headers[b"Cache-Control"], [b"no-store"]) + self.assertEqual(headers[b"Cache-Control"], [b"no-store, no-transform"]) self.assertEqual(headers[b"Pragma"], [b"no-cache"]) self.assertIn("url", channel.json_body) self.assertTrue(channel.json_body["url"].startswith("https://")) @@ -141,9 +142,10 @@ class RendezvousServletTestCase(unittest.HomeserverTestCase): self.assertEqual(headers[b"ETag"], [etag]) self.assertIn(b"Expires", headers) self.assertEqual(headers[b"Content-Type"], [b"text/plain"]) + self.assertEqual(headers[b"Content-Length"], [b"7"]) self.assertEqual(headers[b"Access-Control-Allow-Origin"], [b"*"]) self.assertEqual(headers[b"Access-Control-Expose-Headers"], [b"etag"]) - self.assertEqual(headers[b"Cache-Control"], [b"no-store"]) + self.assertEqual(headers[b"Cache-Control"], [b"no-store, no-transform"]) self.assertEqual(headers[b"Pragma"], [b"no-cache"]) self.assertEqual(channel.text_body, "foo=bar") From 1246e54d7fbd12cb32bd6fc6f3531517d35bcb45 Mon Sep 17 00:00:00 2001 From: Devon Hudson Date: Tue, 25 Feb 2025 08:10:32 -0700 Subject: [PATCH 50/65] 1.125.0 --- CHANGES.md | 7 +++++++ debian/changelog | 6 ++++++ pyproject.toml | 2 +- 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 7db366d1d5..df32bc59be 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -1,3 +1,10 @@ +# Synapse 1.125.0 (2025-02-25) + +No significant changes since 1.125.0rc1. + + + + # Synapse 1.125.0rc1 (2025-02-18) ### Features diff --git a/debian/changelog b/debian/changelog index a0d5a7614f..1ac1fc6644 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +matrix-synapse-py3 (1.125.0) stable; urgency=medium + + * New Synapse release 1.125.0. + + -- Synapse Packaging team Tue, 25 Feb 2025 08:10:07 -0700 + matrix-synapse-py3 (1.125.0~rc1) stable; urgency=medium * New synapse release 1.125.0rc1. diff --git a/pyproject.toml b/pyproject.toml index a40be93d7b..abacfa328d 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -97,7 +97,7 @@ module-name = "synapse.synapse_rust" [tool.poetry] name = "matrix-synapse" -version = "1.125.0rc1" +version = "1.125.0" description = "Homeserver for the Matrix decentralised comms protocol" authors = ["Matrix.org Team and Contributors "] license = "AGPL-3.0-or-later" From 5121f9210c989fcc909e78195133876dff3bc9b9 Mon Sep 17 00:00:00 2001 From: Devon Hudson Date: Tue, 25 Feb 2025 16:25:39 +0000 Subject: [PATCH 51/65] Add background job to clear unreferenced state groups (#18154) Fixes #18150 ### Pull Request Checklist * [X] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [X] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --------- Co-authored-by: Erik Johnston --- changelog.d/18154.feature | 1 + docs/development/database_schema.md | 2 +- synapse/_scripts/synapse_port_db.py | 30 +++ synapse/storage/controllers/purge_events.py | 246 +++++++++++++----- synapse/storage/databases/state/bg_updates.py | 10 +- synapse/storage/databases/state/deletion.py | 44 +++- synapse/storage/schema/__init__.py | 1 + .../02_delete_unreferenced_state_groups.sql | 16 ++ synapse/types/storage/__init__.py | 4 + tests/storage/test_purge.py | 97 +++++++ 10 files changed, 376 insertions(+), 75 deletions(-) create mode 100644 changelog.d/18154.feature create mode 100644 synapse/storage/schema/state/delta/89/02_delete_unreferenced_state_groups.sql diff --git a/changelog.d/18154.feature b/changelog.d/18154.feature new file mode 100644 index 0000000000..62e1b79a15 --- /dev/null +++ b/changelog.d/18154.feature @@ -0,0 +1 @@ +Add background job to clear unreferenced state groups. diff --git a/docs/development/database_schema.md b/docs/development/database_schema.md index 37a06acc12..620d1c16b0 100644 --- a/docs/development/database_schema.md +++ b/docs/development/database_schema.md @@ -162,7 +162,7 @@ by a unique name, the current status (stored in JSON), and some dependency infor * Whether the update requires a previous update to be complete. * A rough ordering for which to complete updates. -A new background updates needs to be added to the `background_updates` table: +A new background update needs to be added to the `background_updates` table: ```sql INSERT INTO background_updates (ordering, update_name, depends_on, progress_json) VALUES diff --git a/synapse/_scripts/synapse_port_db.py b/synapse/_scripts/synapse_port_db.py index 3f67a739a0..59065a0504 100755 --- a/synapse/_scripts/synapse_port_db.py +++ b/synapse/_scripts/synapse_port_db.py @@ -191,6 +191,11 @@ APPEND_ONLY_TABLES = [ IGNORED_TABLES = { + # Porting the auto generated sequence in this table is non-trivial. + # None of the entries in this list are mandatory for Synapse to keep working. + # If state group disk space is an issue after the port, the + # `delete_unreferenced_state_groups_bg_update` background task can be run again. + "state_groups_pending_deletion", # We don't port these tables, as they're a faff and we can regenerate # them anyway. "user_directory", @@ -216,6 +221,15 @@ IGNORED_TABLES = { } +# These background updates will not be applied upon creation of the postgres database. +IGNORED_BACKGROUND_UPDATES = { + # Reapplying this background update to the postgres database is unnecessary after + # already having waited for the SQLite database to complete all running background + # updates. + "delete_unreferenced_state_groups_bg_update", +} + + # Error returned by the run function. Used at the top-level part of the script to # handle errors and return codes. end_error: Optional[str] = None @@ -687,6 +701,20 @@ class Porter: # 0 means off. 1 means full. 2 means incremental. return autovacuum_setting != 0 + async def remove_ignored_background_updates_from_database(self) -> None: + def _remove_delete_unreferenced_state_groups_bg_updates( + txn: LoggingTransaction, + ) -> None: + txn.execute( + "DELETE FROM background_updates WHERE update_name = ANY(?)", + (list(IGNORED_BACKGROUND_UPDATES),), + ) + + await self.postgres_store.db_pool.runInteraction( + "remove_delete_unreferenced_state_groups_bg_updates", + _remove_delete_unreferenced_state_groups_bg_updates, + ) + async def run(self) -> None: """Ports the SQLite database to a PostgreSQL database. @@ -732,6 +760,8 @@ class Porter: self.hs_config.database.get_single_database() ) + await self.remove_ignored_background_updates_from_database() + await self.run_background_updates_on_postgres() self.progress.set_state("Creating port tables") diff --git a/synapse/storage/controllers/purge_events.py b/synapse/storage/controllers/purge_events.py index 47cec8c469..ef30bf2895 100644 --- a/synapse/storage/controllers/purge_events.py +++ b/synapse/storage/controllers/purge_events.py @@ -21,11 +21,18 @@ import itertools import logging -from typing import TYPE_CHECKING, Collection, Mapping, Set +from typing import ( + TYPE_CHECKING, + Collection, + Mapping, + Set, +) from synapse.logging.context import nested_logging_context from synapse.metrics.background_process_metrics import wrap_as_background_process +from synapse.storage.database import LoggingTransaction from synapse.storage.databases import Databases +from synapse.types.storage import _BackgroundUpdates if TYPE_CHECKING: from synapse.server import HomeServer @@ -44,6 +51,11 @@ class PurgeEventsStorageController: self._delete_state_groups_loop, 60 * 1000 ) + self.stores.state.db_pool.updates.register_background_update_handler( + _BackgroundUpdates.DELETE_UNREFERENCED_STATE_GROUPS_BG_UPDATE, + self._background_delete_unrefereneced_state_groups, + ) + async def purge_room(self, room_id: str) -> None: """Deletes all record of a room""" @@ -80,68 +92,6 @@ class PurgeEventsStorageController: sg_to_delete ) - async def _find_unreferenced_groups( - self, state_groups: Collection[int] - ) -> Set[int]: - """Used when purging history to figure out which state groups can be - deleted. - - Args: - state_groups: Set of state groups referenced by events - that are going to be deleted. - - Returns: - The set of state groups that can be deleted. - """ - # Set of events that we have found to be referenced by events - referenced_groups = set() - - # Set of state groups we've already seen - state_groups_seen = set(state_groups) - - # Set of state groups to handle next. - next_to_search = set(state_groups) - while next_to_search: - # We bound size of groups we're looking up at once, to stop the - # SQL query getting too big - if len(next_to_search) < 100: - current_search = next_to_search - next_to_search = set() - else: - current_search = set(itertools.islice(next_to_search, 100)) - next_to_search -= current_search - - referenced = await self.stores.main.get_referenced_state_groups( - current_search - ) - referenced_groups |= referenced - - # We don't continue iterating up the state group graphs for state - # groups that are referenced. - current_search -= referenced - - edges = await self.stores.state.get_previous_state_groups(current_search) - - prevs = set(edges.values()) - # We don't bother re-handling groups we've already seen - prevs -= state_groups_seen - next_to_search |= prevs - state_groups_seen |= prevs - - # We also check to see if anything referencing the state groups are - # also unreferenced. This helps ensure that we delete unreferenced - # state groups, if we don't then we will de-delta them when we - # delete the other state groups leading to increased DB usage. - next_edges = await self.stores.state.get_next_state_groups(current_search) - nexts = set(next_edges.keys()) - nexts -= state_groups_seen - next_to_search |= nexts - state_groups_seen |= nexts - - to_delete = state_groups_seen - referenced_groups - - return to_delete - @wrap_as_background_process("_delete_state_groups_loop") async def _delete_state_groups_loop(self) -> None: """Background task that deletes any state groups that may be pending @@ -203,3 +153,173 @@ class PurgeEventsStorageController: room_id, groups_to_sequences, ) + + async def _background_delete_unrefereneced_state_groups( + self, progress: dict, batch_size: int + ) -> int: + """This background update will slowly delete any unreferenced state groups""" + + last_checked_state_group = progress.get("last_checked_state_group") + max_state_group = progress.get("max_state_group") + + if last_checked_state_group is None or max_state_group is None: + # This is the first run. + last_checked_state_group = 0 + + max_state_group = await self.stores.state.db_pool.simple_select_one_onecol( + table="state_groups", + keyvalues={}, + retcol="MAX(id)", + allow_none=True, + desc="get_max_state_group", + ) + if max_state_group is None: + # There are no state groups so the background process is finished. + await self.stores.state.db_pool.updates._end_background_update( + _BackgroundUpdates.DELETE_UNREFERENCED_STATE_GROUPS_BG_UPDATE + ) + return batch_size + + ( + last_checked_state_group, + final_batch, + ) = await self._delete_unreferenced_state_groups_batch( + last_checked_state_group, batch_size, max_state_group + ) + + if not final_batch: + # There are more state groups to check. + progress = { + "last_checked_state_group": last_checked_state_group, + "max_state_group": max_state_group, + } + await self.stores.state.db_pool.updates._background_update_progress( + _BackgroundUpdates.DELETE_UNREFERENCED_STATE_GROUPS_BG_UPDATE, + progress, + ) + else: + # This background process is finished. + await self.stores.state.db_pool.updates._end_background_update( + _BackgroundUpdates.DELETE_UNREFERENCED_STATE_GROUPS_BG_UPDATE + ) + + return batch_size + + async def _delete_unreferenced_state_groups_batch( + self, + last_checked_state_group: int, + batch_size: int, + max_state_group: int, + ) -> tuple[int, bool]: + """Looks for unreferenced state groups starting from the last state group + checked, and any state groups which would become unreferenced if a state group + was deleted, and marks them for deletion. + + Args: + last_checked_state_group: The last state group that was checked. + batch_size: How many state groups to process in this iteration. + + Returns: + (last_checked_state_group, final_batch) + """ + + # Look for state groups that can be cleaned up. + def get_next_state_groups_txn(txn: LoggingTransaction) -> Set[int]: + state_group_sql = "SELECT id FROM state_groups WHERE ? < id AND id <= ? ORDER BY id LIMIT ?" + txn.execute( + state_group_sql, (last_checked_state_group, max_state_group, batch_size) + ) + + next_set = {row[0] for row in txn} + + return next_set + + next_set = await self.stores.state.db_pool.runInteraction( + "get_next_state_groups", get_next_state_groups_txn + ) + + final_batch = False + if len(next_set) < batch_size: + final_batch = True + else: + last_checked_state_group = max(next_set) + + if len(next_set) == 0: + return last_checked_state_group, final_batch + + # Find all state groups that can be deleted if the original set is deleted. + # This set includes the original set, as well as any state groups that would + # become unreferenced upon deleting the original set. + to_delete = await self._find_unreferenced_groups(next_set) + + if len(to_delete) == 0: + return last_checked_state_group, final_batch + + await self.stores.state_deletion.mark_state_groups_as_pending_deletion( + to_delete + ) + + return last_checked_state_group, final_batch + + async def _find_unreferenced_groups( + self, + state_groups: Collection[int], + ) -> Set[int]: + """Used when purging history to figure out which state groups can be + deleted. + + Args: + state_groups: Set of state groups referenced by events + that are going to be deleted. + + Returns: + The set of state groups that can be deleted. + """ + # Set of events that we have found to be referenced by events + referenced_groups = set() + + # Set of state groups we've already seen + state_groups_seen = set(state_groups) + + # Set of state groups to handle next. + next_to_search = set(state_groups) + while next_to_search: + # We bound size of groups we're looking up at once, to stop the + # SQL query getting too big + if len(next_to_search) < 100: + current_search = next_to_search + next_to_search = set() + else: + current_search = set(itertools.islice(next_to_search, 100)) + next_to_search -= current_search + + referenced = await self.stores.main.get_referenced_state_groups( + current_search + ) + referenced_groups |= referenced + + # We don't continue iterating up the state group graphs for state + # groups that are referenced. + current_search -= referenced + + edges = await self.stores.state.get_previous_state_groups(current_search) + + prevs = set(edges.values()) + # We don't bother re-handling groups we've already seen + prevs -= state_groups_seen + next_to_search |= prevs + state_groups_seen |= prevs + + # We also check to see if anything referencing the state groups are + # also unreferenced. This helps ensure that we delete unreferenced + # state groups, if we don't then we will de-delta them when we + # delete the other state groups leading to increased DB usage. + next_edges = await self.stores.state.get_next_state_groups(current_search) + nexts = set(next_edges.keys()) + nexts -= state_groups_seen + next_to_search |= nexts + state_groups_seen |= nexts + + to_delete = state_groups_seen - referenced_groups + + return to_delete diff --git a/synapse/storage/databases/state/bg_updates.py b/synapse/storage/databases/state/bg_updates.py index f7824cba0f..95fd0ae73a 100644 --- a/synapse/storage/databases/state/bg_updates.py +++ b/synapse/storage/databases/state/bg_updates.py @@ -20,7 +20,15 @@ # import logging -from typing import TYPE_CHECKING, Dict, List, Mapping, Optional, Tuple, Union +from typing import ( + TYPE_CHECKING, + Dict, + List, + Mapping, + Optional, + Tuple, + Union, +) from synapse.logging.opentracing import tag_args, trace from synapse.storage._base import SQLBaseStore diff --git a/synapse/storage/databases/state/deletion.py b/synapse/storage/databases/state/deletion.py index d4b1c20a45..f77c46f6ae 100644 --- a/synapse/storage/databases/state/deletion.py +++ b/synapse/storage/databases/state/deletion.py @@ -321,18 +321,42 @@ class StateDeletionDataStore: async def mark_state_groups_as_pending_deletion( self, state_groups: Collection[int] ) -> None: - """Mark the given state groups as pending deletion""" + """Mark the given state groups as pending deletion. + + If any of the state groups are already pending deletion, then those records are + left as is. + """ + + await self.db_pool.runInteraction( + "mark_state_groups_as_pending_deletion", + self._mark_state_groups_as_pending_deletion_txn, + state_groups, + ) + + def _mark_state_groups_as_pending_deletion_txn( + self, + txn: LoggingTransaction, + state_groups: Collection[int], + ) -> None: + sql = """ + INSERT INTO state_groups_pending_deletion (state_group, insertion_ts) + VALUES %s + ON CONFLICT (state_group) + DO NOTHING + """ now = self._clock.time_msec() - - await self.db_pool.simple_upsert_many( - table="state_groups_pending_deletion", - key_names=("state_group",), - key_values=[(state_group,) for state_group in state_groups], - value_names=("insertion_ts",), - value_values=[(now,) for _ in state_groups], - desc="mark_state_groups_as_pending_deletion", - ) + rows = [ + ( + state_group, + now, + ) + for state_group in state_groups + ] + if isinstance(txn.database_engine, PostgresEngine): + txn.execute_values(sql % ("?",), rows, fetch=False) + else: + txn.execute_batch(sql % ("(?, ?)",), rows) async def mark_state_groups_as_used(self, state_groups: Collection[int]) -> None: """Mark the given state groups as now being referenced""" diff --git a/synapse/storage/schema/__init__.py b/synapse/storage/schema/__init__.py index 49e648a92f..c90c2c6051 100644 --- a/synapse/storage/schema/__init__.py +++ b/synapse/storage/schema/__init__.py @@ -158,6 +158,7 @@ Changes in SCHEMA_VERSION = 88 Changes in SCHEMA_VERSION = 89 - Add `state_groups_pending_deletion` and `state_groups_persisting` tables. + - Add background update to delete unreferenced state groups. """ diff --git a/synapse/storage/schema/state/delta/89/02_delete_unreferenced_state_groups.sql b/synapse/storage/schema/state/delta/89/02_delete_unreferenced_state_groups.sql new file mode 100644 index 0000000000..184dc8564c --- /dev/null +++ b/synapse/storage/schema/state/delta/89/02_delete_unreferenced_state_groups.sql @@ -0,0 +1,16 @@ +-- +-- This file is licensed under the Affero General Public License (AGPL) version 3. +-- +-- Copyright (C) 2025 New Vector, Ltd +-- +-- This program is free software: you can redistribute it and/or modify +-- it under the terms of the GNU Affero General Public License as +-- published by the Free Software Foundation, either version 3 of the +-- License, or (at your option) any later version. +-- +-- See the GNU Affero General Public License for more details: +-- . + +-- Add a background update to delete any unreferenced state groups +INSERT INTO background_updates (ordering, update_name, progress_json) VALUES + (8902, 'delete_unreferenced_state_groups_bg_update', '{}'); diff --git a/synapse/types/storage/__init__.py b/synapse/types/storage/__init__.py index b5fa20a41a..d0a85ef208 100644 --- a/synapse/types/storage/__init__.py +++ b/synapse/types/storage/__init__.py @@ -48,3 +48,7 @@ class _BackgroundUpdates: SLIDING_SYNC_MEMBERSHIP_SNAPSHOTS_FIX_FORGOTTEN_COLUMN_BG_UPDATE = ( "sliding_sync_membership_snapshots_fix_forgotten_column_bg_update" ) + + DELETE_UNREFERENCED_STATE_GROUPS_BG_UPDATE = ( + "delete_unreferenced_state_groups_bg_update" + ) diff --git a/tests/storage/test_purge.py b/tests/storage/test_purge.py index 916e42e731..ecdc893405 100644 --- a/tests/storage/test_purge.py +++ b/tests/storage/test_purge.py @@ -24,6 +24,7 @@ from synapse.api.errors import NotFoundError, SynapseError from synapse.rest.client import room from synapse.server import HomeServer from synapse.types.state import StateFilter +from synapse.types.storage import _BackgroundUpdates from synapse.util import Clock from tests.unittest import HomeserverTestCase @@ -303,3 +304,99 @@ class PurgeTests(HomeserverTestCase): ) ) self.assertEqual(len(state_groups), 1) + + def test_clear_unreferenced_state_groups(self) -> None: + """Test that any unreferenced state groups are automatically cleaned up.""" + + self.helper.send(self.room_id, body="test1") + state1 = self.helper.send_state( + self.room_id, "org.matrix.test", body={"number": 2} + ) + # Create enough state events to require multiple batches of + # delete_unreferenced_state_groups_bg_update to be run. + for i in range(200): + self.helper.send_state(self.room_id, "org.matrix.test", body={"number": i}) + state2 = self.helper.send_state( + self.room_id, "org.matrix.test", body={"number": 3} + ) + self.helper.send(self.room_id, body="test4") + last = self.helper.send(self.room_id, body="test5") + + # Create an unreferenced state group that has a prev group of one of the + # to-be-purged events. + prev_group = self.get_success( + self.store._get_state_group_for_event(state1["event_id"]) + ) + unreferenced_state_group = self.get_success( + self.state_store.store_state_group( + event_id=last["event_id"], + room_id=self.room_id, + prev_group=prev_group, + delta_ids={("org.matrix.test", ""): state2["event_id"]}, + current_state_ids=None, + ) + ) + + another_unreferenced_state_group = self.get_success( + self.state_store.store_state_group( + event_id=last["event_id"], + room_id=self.room_id, + prev_group=unreferenced_state_group, + delta_ids={("org.matrix.test", ""): state2["event_id"]}, + current_state_ids=None, + ) + ) + + # Insert and run the background update. + self.get_success( + self.store.db_pool.simple_insert( + "background_updates", + { + "update_name": _BackgroundUpdates.DELETE_UNREFERENCED_STATE_GROUPS_BG_UPDATE, + "progress_json": "{}", + }, + ) + ) + self.store.db_pool.updates._all_done = False + self.wait_for_background_updates() + + # Advance so that the background job to delete the state groups runs + self.reactor.advance( + 1 + self.state_deletion_store.DELAY_BEFORE_DELETION_MS / 1000 + ) + + # We expect that the unreferenced state group has been deleted. + row = self.get_success( + self.state_store.db_pool.simple_select_one_onecol( + table="state_groups", + keyvalues={"id": unreferenced_state_group}, + retcol="id", + allow_none=True, + desc="test_purge_unreferenced_state_group", + ) + ) + self.assertIsNone(row) + + # We expect that the other unreferenced state group has also been deleted. + row = self.get_success( + self.state_store.db_pool.simple_select_one_onecol( + table="state_groups", + keyvalues={"id": another_unreferenced_state_group}, + retcol="id", + allow_none=True, + desc="test_purge_unreferenced_state_group", + ) + ) + self.assertIsNone(row) + + # We expect there to now only be one state group for the room, which is + # the state group of the last event (as the only outlier). + state_groups = self.get_success( + self.state_store.db_pool.simple_select_onecol( + table="state_groups", + keyvalues={"room_id": self.room_id}, + retcol="id", + desc="test_purge_unreferenced_state_group", + ) + ) + self.assertEqual(len(state_groups), 207) From 2159b3852e79103c791eda159c8895c3b5cf2ff5 Mon Sep 17 00:00:00 2001 From: V02460 Date: Tue, 25 Feb 2025 17:26:01 +0100 Subject: [PATCH 52/65] Add --no-secrets-in-config command line option (#18092) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds the `--no-secrets-in-config` command line option that makes Synapse reject all configurations containing keys with in-line secret values. Currently this rejects - `turn_shared_secret` - `registration_shared_secret` - `macaroon_secret_key` - `recaptcha_private_key` - `recaptcha_public_key` - `experimental_features.msc3861.client_secret` - `experimental_features.msc3861.jwk` - `experimental_features.msc3861.admin_token` - `form_secret` - `redis.password` - `worker_replication_secret` > [!TIP] > Hey, you! Yes, you! 😊 If you think this list is missing an item, please leave a comment below. Thanks :) This PR complements my other PRs[^1] that add the corresponding `_path` variants for this class of config options. It enables admins to enforce a policy of no secrets in configuration files and guards against accident and malice. Because I consider the flag `--no-secrets-in-config` to be security-relevant, I did not add a corresponding `--secrets-in-config` flag; this way, if Synapse command line options are appended at various places, there is no way to weaken the once-set setting with a succeeding flag. [^1]: [#17690](https://github.com/element-hq/synapse/pull/17690), [#17717](https://github.com/element-hq/synapse/pull/17717), [#17983](https://github.com/element-hq/synapse/pull/17983), [#17984](https://github.com/element-hq/synapse/pull/17984), [#18004](https://github.com/element-hq/synapse/pull/18004), [#18090](https://github.com/element-hq/synapse/pull/18090) ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --- changelog.d/18092.feature | 1 + synapse/config/_base.py | 32 +++++++++- synapse/config/_base.pyi | 6 +- synapse/config/captcha.py | 14 ++++- synapse/config/experimental.py | 30 +++++++++- synapse/config/key.py | 16 ++++- synapse/config/redis.py | 9 ++- synapse/config/registration.py | 9 ++- synapse/config/voip.py | 9 ++- synapse/config/workers.py | 9 ++- tests/config/test_load.py | 104 +++++++++++++++++++++++++++++++++ tests/config/test_workers.py | 2 +- 12 files changed, 227 insertions(+), 14 deletions(-) create mode 100644 changelog.d/18092.feature diff --git a/changelog.d/18092.feature b/changelog.d/18092.feature new file mode 100644 index 0000000000..26371cc810 --- /dev/null +++ b/changelog.d/18092.feature @@ -0,0 +1 @@ +Add the `--no-secrets-in-config` command line option. \ No newline at end of file diff --git a/synapse/config/_base.py b/synapse/config/_base.py index 912346a423..132ba26af9 100644 --- a/synapse/config/_base.py +++ b/synapse/config/_base.py @@ -589,6 +589,14 @@ class RootConfig: " Defaults to the directory containing the last config file", ) + config_parser.add_argument( + "--no-secrets-in-config", + dest="secrets_in_config", + action="store_false", + default=True, + help="Reject config options that expect an in-line secret as value.", + ) + cls.invoke_all_static("add_arguments", config_parser) @classmethod @@ -626,7 +634,10 @@ class RootConfig: config_dict = read_config_files(config_files) obj.parse_config_dict( - config_dict, config_dir_path=config_dir_path, data_dir_path=data_dir_path + config_dict, + config_dir_path=config_dir_path, + data_dir_path=data_dir_path, + allow_secrets_in_config=config_args.secrets_in_config, ) obj.invoke_all("read_arguments", config_args) @@ -653,6 +664,13 @@ class RootConfig: help="Specify config file. Can be given multiple times and" " may specify directories containing *.yaml files.", ) + parser.add_argument( + "--no-secrets-in-config", + dest="secrets_in_config", + action="store_false", + default=True, + help="Reject config options that expect an in-line secret as value.", + ) # we nest the mutually-exclusive group inside another group so that the help # text shows them in their own group. @@ -821,14 +839,21 @@ class RootConfig: return None obj.parse_config_dict( - config_dict, config_dir_path=config_dir_path, data_dir_path=data_dir_path + config_dict, + config_dir_path=config_dir_path, + data_dir_path=data_dir_path, + allow_secrets_in_config=config_args.secrets_in_config, ) obj.invoke_all("read_arguments", config_args) return obj def parse_config_dict( - self, config_dict: Dict[str, Any], config_dir_path: str, data_dir_path: str + self, + config_dict: Dict[str, Any], + config_dir_path: str, + data_dir_path: str, + allow_secrets_in_config: bool = True, ) -> None: """Read the information from the config dict into this Config object. @@ -846,6 +871,7 @@ class RootConfig: config_dict, config_dir_path=config_dir_path, data_dir_path=data_dir_path, + allow_secrets_in_config=allow_secrets_in_config, ) def generate_missing_files( diff --git a/synapse/config/_base.pyi b/synapse/config/_base.pyi index d9cb0da38b..55b0e2cbf4 100644 --- a/synapse/config/_base.pyi +++ b/synapse/config/_base.pyi @@ -132,7 +132,11 @@ class RootConfig: @classmethod def invoke_all_static(cls, func_name: str, *args: Any, **kwargs: Any) -> None: ... def parse_config_dict( - self, config_dict: Dict[str, Any], config_dir_path: str, data_dir_path: str + self, + config_dict: Dict[str, Any], + config_dir_path: str, + data_dir_path: str, + allow_secrets_in_config: bool = ..., ) -> None: ... def generate_config( self, diff --git a/synapse/config/captcha.py b/synapse/config/captcha.py index 84897c09c5..57d67abbc3 100644 --- a/synapse/config/captcha.py +++ b/synapse/config/captcha.py @@ -29,8 +29,15 @@ from ._base import Config, ConfigError class CaptchaConfig(Config): section = "captcha" - def read_config(self, config: JsonDict, **kwargs: Any) -> None: + def read_config( + self, config: JsonDict, allow_secrets_in_config: bool, **kwargs: Any + ) -> None: recaptcha_private_key = config.get("recaptcha_private_key") + if recaptcha_private_key and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("recaptcha_private_key",), + ) if recaptcha_private_key is not None and not isinstance( recaptcha_private_key, str ): @@ -38,6 +45,11 @@ class CaptchaConfig(Config): self.recaptcha_private_key = recaptcha_private_key recaptcha_public_key = config.get("recaptcha_public_key") + if recaptcha_public_key and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("recaptcha_public_key",), + ) if recaptcha_public_key is not None and not isinstance( recaptcha_public_key, str ): diff --git a/synapse/config/experimental.py b/synapse/config/experimental.py index 3beaeb8869..0a963b121a 100644 --- a/synapse/config/experimental.py +++ b/synapse/config/experimental.py @@ -250,7 +250,9 @@ class MSC3861: ) return self._admin_token - def check_config_conflicts(self, root: RootConfig) -> None: + def check_config_conflicts( + self, root: RootConfig, allow_secrets_in_config: bool + ) -> None: """Checks for any configuration conflicts with other parts of Synapse. Raises: @@ -260,6 +262,24 @@ class MSC3861: if not self.enabled: return + if self._client_secret and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("experimental", "msc3861", "client_secret"), + ) + + if self.jwk and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("experimental", "msc3861", "jwk"), + ) + + if self._admin_token and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("experimental", "msc3861", "admin_token"), + ) + if ( root.auth.password_enabled_for_reauth or root.auth.password_enabled_for_login @@ -350,7 +370,9 @@ class ExperimentalConfig(Config): section = "experimental" - def read_config(self, config: JsonDict, **kwargs: Any) -> None: + def read_config( + self, config: JsonDict, allow_secrets_in_config: bool, **kwargs: Any + ) -> None: experimental = config.get("experimental_features") or {} # MSC3026 (busy presence state) @@ -494,7 +516,9 @@ class ExperimentalConfig(Config): ) from exc # Check that none of the other config options conflict with MSC3861 when enabled - self.msc3861.check_config_conflicts(self.root) + self.msc3861.check_config_conflicts( + self.root, allow_secrets_in_config=allow_secrets_in_config + ) self.msc4028_push_encrypted_events = experimental.get( "msc4028_push_encrypted_events", False diff --git a/synapse/config/key.py b/synapse/config/key.py index 01aae09c13..6f99f39e81 100644 --- a/synapse/config/key.py +++ b/synapse/config/key.py @@ -112,7 +112,11 @@ class KeyConfig(Config): section = "key" def read_config( - self, config: JsonDict, config_dir_path: str, **kwargs: Any + self, + config: JsonDict, + config_dir_path: str, + allow_secrets_in_config: bool, + **kwargs: Any, ) -> None: # the signing key can be specified inline or in a separate file if "signing_key" in config: @@ -172,6 +176,11 @@ class KeyConfig(Config): ) macaroon_secret_key = config.get("macaroon_secret_key") + if macaroon_secret_key and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("macaroon_secret_key",), + ) macaroon_secret_key_path = config.get("macaroon_secret_key_path") if macaroon_secret_key_path: if macaroon_secret_key: @@ -193,6 +202,11 @@ class KeyConfig(Config): # a secret which is used to calculate HMACs for form values, to stop # falsification of values self.form_secret = config.get("form_secret", None) + if self.form_secret and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("form_secret",), + ) def generate_config_section( self, diff --git a/synapse/config/redis.py b/synapse/config/redis.py index 3f38fa11b0..948c95eef7 100644 --- a/synapse/config/redis.py +++ b/synapse/config/redis.py @@ -34,7 +34,9 @@ These are mutually incompatible. class RedisConfig(Config): section = "redis" - def read_config(self, config: JsonDict, **kwargs: Any) -> None: + def read_config( + self, config: JsonDict, allow_secrets_in_config: bool, **kwargs: Any + ) -> None: redis_config = config.get("redis") or {} self.redis_enabled = redis_config.get("enabled", False) @@ -48,6 +50,11 @@ class RedisConfig(Config): self.redis_path = redis_config.get("path", None) self.redis_dbid = redis_config.get("dbid", None) self.redis_password = redis_config.get("password") + if self.redis_password and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("redis", "password"), + ) redis_password_path = redis_config.get("password_path") if redis_password_path: if self.redis_password: diff --git a/synapse/config/registration.py b/synapse/config/registration.py index c7f3e6d35e..3cf7031656 100644 --- a/synapse/config/registration.py +++ b/synapse/config/registration.py @@ -43,7 +43,9 @@ You have configured both `registration_shared_secret` and class RegistrationConfig(Config): section = "registration" - def read_config(self, config: JsonDict, **kwargs: Any) -> None: + def read_config( + self, config: JsonDict, allow_secrets_in_config: bool, **kwargs: Any + ) -> None: self.enable_registration = strtobool( str(config.get("enable_registration", False)) ) @@ -68,6 +70,11 @@ class RegistrationConfig(Config): # read the shared secret, either inline or from an external file self.registration_shared_secret = config.get("registration_shared_secret") + if self.registration_shared_secret and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("registration_shared_secret",), + ) registration_shared_secret_path = config.get("registration_shared_secret_path") if registration_shared_secret_path: if self.registration_shared_secret: diff --git a/synapse/config/voip.py b/synapse/config/voip.py index 8614a41dd4..f33602d975 100644 --- a/synapse/config/voip.py +++ b/synapse/config/voip.py @@ -34,9 +34,16 @@ These are mutually incompatible. class VoipConfig(Config): section = "voip" - def read_config(self, config: JsonDict, **kwargs: Any) -> None: + def read_config( + self, config: JsonDict, allow_secrets_in_config: bool, **kwargs: Any + ) -> None: self.turn_uris = config.get("turn_uris", []) self.turn_shared_secret = config.get("turn_shared_secret") + if self.turn_shared_secret and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("turn_shared_secret",), + ) turn_shared_secret_path = config.get("turn_shared_secret_path") if turn_shared_secret_path: if self.turn_shared_secret: diff --git a/synapse/config/workers.py b/synapse/config/workers.py index ab896be307..632cef46ce 100644 --- a/synapse/config/workers.py +++ b/synapse/config/workers.py @@ -218,7 +218,9 @@ class WorkerConfig(Config): section = "worker" - def read_config(self, config: JsonDict, **kwargs: Any) -> None: + def read_config( + self, config: JsonDict, allow_secrets_in_config: bool, **kwargs: Any + ) -> None: self.worker_app = config.get("worker_app") # Canonicalise worker_app so that master always has None @@ -243,6 +245,11 @@ class WorkerConfig(Config): # The shared secret used for authentication when connecting to the main synapse. self.worker_replication_secret = config.get("worker_replication_secret", None) + if self.worker_replication_secret and not allow_secrets_in_config: + raise ConfigError( + "Config options that expect an in-line secret as value are disabled", + ("worker_replication_secret",), + ) self.worker_name = config.get("worker_name", self.worker_app) self.instance_name = self.worker_name or MAIN_PROCESS_INSTANCE_NAME diff --git a/tests/config/test_load.py b/tests/config/test_load.py index 220ca23aa7..18fb2e0c2c 100644 --- a/tests/config/test_load.py +++ b/tests/config/test_load.py @@ -21,6 +21,7 @@ # import tempfile from typing import Callable +from unittest import mock import yaml from parameterized import parameterized @@ -31,6 +32,11 @@ from synapse.config.homeserver import HomeServerConfig from tests.config.utils import ConfigFileTestCase +try: + import authlib +except ImportError: + authlib = None + try: import hiredis except ImportError: @@ -189,3 +195,101 @@ class ConfigLoadingFileTestCase(ConfigFileTestCase): config = HomeServerConfig.load_config("", ["-c", self.config_file]) self.assertEqual(get_secret(config), b"53C237") + + @parameterized.expand( + [ + "turn_shared_secret: 53C237", + "registration_shared_secret: 53C237", + "macaroon_secret_key: 53C237", + "recaptcha_private_key: 53C237", + "recaptcha_public_key: ¬53C237", + "form_secret: 53C237", + "worker_replication_secret: 53C237", + *[ + "experimental_features:\n" + " msc3861:\n" + " enabled: true\n" + " client_secret: 53C237" + ] + * (authlib is not None), + *[ + "experimental_features:\n" + " msc3861:\n" + " enabled: true\n" + " client_auth_method: private_key_jwt\n" + ' jwk: {{"mock": "mock"}}' + ] + * (authlib is not None), + *[ + "experimental_features:\n" + " msc3861:\n" + " enabled: true\n" + " admin_token: 53C237\n" + " client_secret_path: {secret_file}" + ] + * (authlib is not None), + *["redis:\n enabled: true\n password: 53C237"] * (hiredis is not None), + ] + ) + def test_no_secrets_in_config(self, config_line: str) -> None: + if authlib is not None: + patcher = mock.patch("authlib.jose.rfc7517.JsonWebKey.import_key") + self.addCleanup(patcher.stop) + patcher.start() + + with tempfile.NamedTemporaryFile(buffering=0) as secret_file: + # Only used for less mocking with admin_token + secret_file.write(b"53C237") + + self.generate_config_and_remove_lines_containing( + ["form_secret", "macaroon_secret_key", "registration_shared_secret"] + ) + # Check strict mode with no offenders. + HomeServerConfig.load_config( + "", ["-c", self.config_file, "--no-secrets-in-config"] + ) + self.add_lines_to_config( + ["", config_line.format(secret_file=secret_file.name)] + ) + # Check strict mode with a single offender. + with self.assertRaises(ConfigError): + HomeServerConfig.load_config( + "", ["-c", self.config_file, "--no-secrets-in-config"] + ) + + # Check lenient mode with a single offender. + HomeServerConfig.load_config("", ["-c", self.config_file]) + + def test_no_secrets_in_config_but_in_files(self) -> None: + with tempfile.NamedTemporaryFile(buffering=0) as secret_file: + secret_file.write(b"53C237") + + self.generate_config_and_remove_lines_containing( + ["form_secret", "macaroon_secret_key", "registration_shared_secret"] + ) + self.add_lines_to_config( + [ + "", + f"turn_shared_secret_path: {secret_file.name}", + f"registration_shared_secret_path: {secret_file.name}", + f"macaroon_secret_key_path: {secret_file.name}", + f"recaptcha_private_key_path: {secret_file.name}", + f"recaptcha_public_key_path: {secret_file.name}", + f"form_secret_path: {secret_file.name}", + f"worker_replication_secret_path: {secret_file.name}", + *[ + "experimental_features:\n" + " msc3861:\n" + " enabled: true\n" + f" admin_token_path: {secret_file.name}\n" + f" client_secret_path: {secret_file.name}\n" + # f" jwk_path: {secret_file.name}" + ] + * (authlib is not None), + *[f"redis:\n enabled: true\n password_path: {secret_file.name}"] + * (hiredis is not None), + ] + ) + HomeServerConfig.load_config( + "", ["-c", self.config_file, "--no-secrets-in-config"] + ) diff --git a/tests/config/test_workers.py b/tests/config/test_workers.py index 64c0285d01..3a21975b89 100644 --- a/tests/config/test_workers.py +++ b/tests/config/test_workers.py @@ -47,7 +47,7 @@ class WorkerDutyConfigTestCase(TestCase): "worker_app": worker_app, **extras, } - worker_config.read_config(worker_config_dict) + worker_config.read_config(worker_config_dict, allow_secrets_in_config=True) return worker_config def test_old_configs_master(self) -> None: From d901dff9e0168e885650e89fd55f90cfc84e9a90 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Feb 2025 17:07:33 +0000 Subject: [PATCH 53/65] Bump serde_json from 1.0.138 to 1.0.139 (#18186) Bumps [serde_json](https://github.com/serde-rs/json) from 1.0.138 to 1.0.139.
Release notes

Sourced from serde_json's releases.

v1.0.139

  • Documentation improvements
Commits
  • 4d4f53c Release 1.0.139
  • 5d6b32f Merge pull request #1242 from dtolnay/writefloat
  • e5bb8bd Document behavior of write_f32/f64 on non-finite floats
  • 7a79781 Merge pull request #1241 from dtolnay/doclink
  • 13591f1 Convert html links to intra-doc links
  • 1d7378e Unset doc-scrape-examples for lib target
  • 1174c5f Resolve unnecessary_semicolon pedantic clippy lint
  • See full diff in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=serde_json&package-manager=cargo&previous-version=1.0.138&new-version=1.0.139)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8831f7e6fd..0ed9c7bee1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -457,9 +457,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.138" +version = "1.0.139" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d434192e7da787e94a6ea7e9670b26a036d0ca41e0b7efb2676dd32bae872949" +checksum = "44f86c3acccc9c65b153fe1b85a3be07fe5515274ec9f0653b4a0875731c72a6" dependencies = [ "itoa", "memchr", From 5cf9f762a8ff431136b3969bc70101efaa8b696a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Feb 2025 17:08:01 +0000 Subject: [PATCH 54/65] Bump log from 0.4.25 to 0.4.26 (#18184) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [log](https://github.com/rust-lang/log) from 0.4.25 to 0.4.26.
Release notes

Sourced from log's releases.

0.4.26

What's Changed

Full Changelog: https://github.com/rust-lang/log/compare/0.4.25...0.4.26

Changelog

Sourced from log's changelog.

[0.4.26] - 2025-02-18

What's Changed

Full Changelog: https://github.com/rust-lang/log/compare/0.4.25...0.4.26

Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=log&package-manager=cargo&previous-version=0.4.25&new-version=0.4.26)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0ed9c7bee1..3308e9e370 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -223,9 +223,9 @@ checksum = "ae743338b92ff9146ce83992f766a31066a91a8c84a45e0e9f21e7cf6de6d346" [[package]] name = "log" -version = "0.4.25" +version = "0.4.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04cbf5b083de1c7e0222a7a51dbfdba1cbe1c6ab0b15e29fff3f6c077fd9cd9f" +checksum = "30bde2b3dc3671ae49d8e2e9f044c7c005836e7a023ee57cffa25ab82764bb9e" [[package]] name = "memchr" From 2a6b9e9cbcc12945a21b839e157599a477e2748b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Feb 2025 17:12:27 +0000 Subject: [PATCH 55/65] Bump authlib from 1.4.0 to 1.4.1 (#18190) Bumps [authlib](https://github.com/lepture/authlib) from 1.4.0 to 1.4.1.
Release notes

Sourced from authlib's releases.

Version 1.4.1

  • Improve garbage collection on OAuth clients. #698
  • Fix client parameters for httpx. #694
Changelog

Sourced from authlib's changelog.

Version 1.4.1

Released on Jan 28, 2025

  • Improve garbage collection on OAuth clients. :issue:698
  • Fix client parameters for httpx. :issue:694
Commits
  • 0e8f480 chore: release 1.4.1
  • c46e939 fix(client): improve garbage collection for oauth clients
  • 9188e21 fix(httpx): remove compact code for httpx
  • c7e2d9f fix(httpx): update test cases for httpx
  • ce1405d fix: improve garbage collection via #698
  • 532cce6 fix: update httpx client kwargs #694
  • fe12a57 chore: update readme
  • See full diff in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=authlib&package-manager=pip&previous-version=1.4.0&new-version=1.4.1)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- poetry.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/poetry.lock b/poetry.lock index 189fa9063d..62da2e2f76 100644 --- a/poetry.lock +++ b/poetry.lock @@ -32,13 +32,13 @@ tests-mypy = ["mypy (>=1.11.1)", "pytest-mypy-plugins"] [[package]] name = "authlib" -version = "1.4.0" +version = "1.4.1" description = "The ultimate Python library in building OAuth and OpenID Connect servers and clients." optional = true python-versions = ">=3.9" files = [ - {file = "Authlib-1.4.0-py2.py3-none-any.whl", hash = "sha256:4bb20b978c8b636222b549317c1815e1fe62234fc1c5efe8855d84aebf3a74e3"}, - {file = "authlib-1.4.0.tar.gz", hash = "sha256:1c1e6608b5ed3624aeeee136ca7f8c120d6f51f731aa152b153d54741840e1f2"}, + {file = "Authlib-1.4.1-py2.py3-none-any.whl", hash = "sha256:edc29c3f6a3e72cd9e9f45fff67fc663a2c364022eb0371c003f22d5405915c1"}, + {file = "authlib-1.4.1.tar.gz", hash = "sha256:30ead9ea4993cdbab821dc6e01e818362f92da290c04c7f6a1940f86507a790d"}, ] [package.dependencies] From f155eaa05fcf19fab01231932b1641ebc4607425 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Feb 2025 17:14:56 +0000 Subject: [PATCH 56/65] Bump click from 8.1.7 to 8.1.8 (#18189) Bumps [click](https://github.com/pallets/click) from 8.1.7 to 8.1.8.
Release notes

Sourced from click's releases.

8.1.8

This is the Click 8.1.8 fix release, which fixes bugs but does not otherwise change behavior and should not result in breaking changes compared to the latest feature release.

PyPI: https://pypi.org/project/click/8.1.8/ Changes: https://click.palletsprojects.com/en/stable/changes/#version-8-1-8 Milestone https://github.com/pallets/click/milestones/23?closed=1

  • Fix an issue with type hints for click.open_file(). #2717
  • Fix issue where error message for invalid click.Path displays on multiple lines. #2697
  • Fixed issue that prevented a default value of "" from being displayed in the help for an option. #2500
  • The test runner handles stripping color consistently on Windows. #2705
  • Show correct value for flag default when using default_map. #2632
  • Fix click.echo(color=...) passing color to coloroma so it can be forced on Windows. #2606.
  • More robust bash version check, fixing problem on Windows with git-bash. #2638
  • Cache the help option generated by the help_option_names setting to respect its eagerness. #2811
  • Replace uses of os.system with subprocess.Popen. #1476
  • Exceptions generated during a command will use the context's color setting when being displayed. #2193
  • Error message when defining option with invalid name is more descriptive. #2452
  • Refactor code generating default --help option to deduplicate code. #2563
  • Test CLIRunner resets patched _compat.should_strip_ansi. #2732
Changelog

Sourced from click's changelog.

Version 8.1.8

Unreleased

  • Fix an issue with type hints for click.open_file(). :issue:2717
  • Fix issue where error message for invalid click.Path displays on multiple lines. :issue:2697
  • Fixed issue that prevented a default value of "" from being displayed in the help for an option. :issue:2500
  • The test runner handles stripping color consistently on Windows. :issue:2705
  • Show correct value for flag default when using default_map. :issue:2632
  • Fix click.echo(color=...) passing color to coloroma so it can be forced on Windows. :issue:2606.
Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=click&package-manager=pip&previous-version=8.1.7&new-version=8.1.8)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- poetry.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/poetry.lock b/poetry.lock index 62da2e2f76..1ad631199a 100644 --- a/poetry.lock +++ b/poetry.lock @@ -304,13 +304,13 @@ files = [ [[package]] name = "click" -version = "8.1.7" +version = "8.1.8" description = "Composable command line interface toolkit" optional = false python-versions = ">=3.7" files = [ - {file = "click-8.1.7-py3-none-any.whl", hash = "sha256:ae74fb96c20a0277a1d615f1e4d73c8414f5a98db8b799a7931d1582f3390c28"}, - {file = "click-8.1.7.tar.gz", hash = "sha256:ca9853ad459e787e2192211578cc907e7594e294c7ccc834310722b41b9ca6de"}, + {file = "click-8.1.8-py3-none-any.whl", hash = "sha256:63c132bbbed01578a06712a2d1f497bb62d9c1c0d329b7903a866228027263b2"}, + {file = "click-8.1.8.tar.gz", hash = "sha256:ed53c9d8990d83c2a27deae68e4ee337473f6330c040a31d4225c9574d16096a"}, ] [package.dependencies] From 5e1d8f657da8c13791a3e3e4648b375b6ed02558 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Feb 2025 17:15:41 +0000 Subject: [PATCH 57/65] Bump anyhow from 1.0.95 to 1.0.96 (#18187) Bumps [anyhow](https://github.com/dtolnay/anyhow) from 1.0.95 to 1.0.96.
Release notes

Sourced from anyhow's releases.

1.0.96

  • Documentation improvements
Commits
  • f0aa0d3 Release 1.0.96
  • bc33c24 Convert html links to intra-doc links
  • 1cff785 Unset doc-scrape-examples for lib target
  • d71c806 More precise gitignore patterns
  • 3e40975 Remove **/*.rs.bk from project-specific gitignore
  • b880dd0 Ignore Cargo-generated tests/crate/target directory
  • 8891ce3 Merge pull request #404 from dtolnay/missingabi
  • 51a173e Ignore missing_abi lint in nightly-2025-01-16
  • 4d71a84 Ignore double_ended_iterator_last clippy lint
  • af0937e Update ui test suite to nightly-2025-01-02
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=anyhow&package-manager=cargo&previous-version=1.0.95&new-version=1.0.96)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3308e9e370..56d8654479 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -13,9 +13,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.95" +version = "1.0.96" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" +checksum = "6b964d184e89d9b6b67dd2715bc8e74cf3107fb2b529990c90cf517326150bf4" [[package]] name = "arc-swap" From 0fa7ffd76f16498fc35b066e49933e9019ca9253 Mon Sep 17 00:00:00 2001 From: Andrew Morgan <1342360+anoadragon453@users.noreply.github.com> Date: Tue, 25 Feb 2025 18:18:15 +0100 Subject: [PATCH 58/65] Move User Event Redaction Admin API version indicator to the correct place (#18152) Previously it was in the middle of the parameter definitions. --- changelog.d/18152.doc | 1 + docs/admin_api/user_admin_api.md | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) create mode 100644 changelog.d/18152.doc diff --git a/changelog.d/18152.doc b/changelog.d/18152.doc new file mode 100644 index 0000000000..5a02f26751 --- /dev/null +++ b/changelog.d/18152.doc @@ -0,0 +1 @@ +Move incorrectly placed version indicator in User Event Redaction Admin API docs. \ No newline at end of file diff --git a/docs/admin_api/user_admin_api.md b/docs/admin_api/user_admin_api.md index 77ce800507..875876081f 100644 --- a/docs/admin_api/user_admin_api.md +++ b/docs/admin_api/user_admin_api.md @@ -1494,13 +1494,13 @@ The following JSON body parameter must be provided: - `rooms` - A list of rooms to redact the user's events in. If an empty list is provided all events in all rooms the user is a member of will be redacted -_Added in Synapse 1.116.0._ - The following JSON body parameters are optional: - `reason` - Reason the redaction is being requested, ie "spam", "abuse", etc. This will be included in each redaction event, and be visible to users. - `limit` - a limit on the number of the user's events to search for ones that can be redacted (events are redacted newest to oldest) in each room, defaults to 1000 if not provided +_Added in Synapse 1.116.0._ + ## Check the status of a redaction process From fbb21b29bb3e7942c4cd051d3db7316a668cdaef Mon Sep 17 00:00:00 2001 From: Andrew Ferrazzutti Date: Tue, 25 Feb 2025 12:22:01 -0500 Subject: [PATCH 59/65] Define delayed event ratelimit category (#18019) Apply ratelimiting on delayed event management separately from messages. ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [ ] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --------- Co-authored-by: Andrew Morgan <1342360+anoadragon453@users.noreply.github.com> --- changelog.d/18019.feature | 1 + demo/start.sh | 3 + .../conf/workers-shared-extra.yaml.j2 | 4 + .../configuration/config_documentation.md | 23 +++ synapse/config/ratelimiting.py | 6 + synapse/handlers/delayed_events.py | 32 +++- tests/rest/client/test_delayed_events.py | 143 ++++++++++++++++++ tests/rest/client/test_rooms.py | 35 +++++ 8 files changed, 243 insertions(+), 4 deletions(-) create mode 100644 changelog.d/18019.feature diff --git a/changelog.d/18019.feature b/changelog.d/18019.feature new file mode 100644 index 0000000000..74e22df74a --- /dev/null +++ b/changelog.d/18019.feature @@ -0,0 +1 @@ +Define ratelimit configuration for delayed event management. diff --git a/demo/start.sh b/demo/start.sh index 7636c41f1f..e010302bf4 100755 --- a/demo/start.sh +++ b/demo/start.sh @@ -142,6 +142,9 @@ for port in 8080 8081 8082; do per_user: per_second: 1000 burst_count: 1000 + rc_delayed_event_mgmt: + per_second: 1000 + burst_count: 1000 RC ) echo "${ratelimiting}" >> "$port.config" diff --git a/docker/complement/conf/workers-shared-extra.yaml.j2 b/docker/complement/conf/workers-shared-extra.yaml.j2 index ac0c4bb851..9ab8fedcae 100644 --- a/docker/complement/conf/workers-shared-extra.yaml.j2 +++ b/docker/complement/conf/workers-shared-extra.yaml.j2 @@ -94,6 +94,10 @@ rc_presence: per_second: 9999 burst_count: 9999 +rc_delayed_event_mgmt: + per_second: 9999 + burst_count: 9999 + federation_rr_transactions_per_room_per_second: 9999 allow_device_name_lookup_over_federation: true diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index 8d9a71fb5f..8523c5f65f 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -1947,6 +1947,29 @@ rc_presence: burst_count: 1 ``` --- +### `rc_delayed_event_mgmt` + +Ratelimiting settings for delayed event management. + +This is a ratelimiting option that ratelimits +attempts to restart, cancel, or view delayed events +based on the sending client's account and device ID. +It defaults to: `per_second: 1`, `burst_count: 5`. + +Attempts to create or send delayed events are ratelimited not by this setting, but by `rc_message`. + +Setting this to a high value allows clients to make delayed event management requests often +(such as repeatedly restarting a delayed event with a short timeout, +or restarting several different delayed events all at once) +without the risk of being ratelimited. + +Example configuration: +```yaml +rc_delayed_event_mgmt: + per_second: 2 + burst_count: 20 +``` +--- ### `federation_rr_transactions_per_room_per_second` Sets outgoing federation transaction frequency for sending read-receipts, diff --git a/synapse/config/ratelimiting.py b/synapse/config/ratelimiting.py index 06af4da3c5..eb1dc2dacb 100644 --- a/synapse/config/ratelimiting.py +++ b/synapse/config/ratelimiting.py @@ -234,3 +234,9 @@ class RatelimitConfig(Config): "rc_presence.per_user", defaults={"per_second": 0.1, "burst_count": 1}, ) + + self.rc_delayed_event_mgmt = RatelimitSettings.parse( + config, + "rc_delayed_event_mgmt", + defaults={"per_second": 1, "burst_count": 5}, + ) diff --git a/synapse/handlers/delayed_events.py b/synapse/handlers/delayed_events.py index 3c88a96fd3..b3f40809a1 100644 --- a/synapse/handlers/delayed_events.py +++ b/synapse/handlers/delayed_events.py @@ -19,6 +19,7 @@ from twisted.internet.interfaces import IDelayedCall from synapse.api.constants import EventTypes from synapse.api.errors import ShadowBanError +from synapse.api.ratelimiting import Ratelimiter from synapse.config.workers import MAIN_PROCESS_INSTANCE_NAME from synapse.logging.opentracing import set_tag from synapse.metrics import event_processing_positions @@ -57,10 +58,19 @@ class DelayedEventsHandler: self._storage_controllers = hs.get_storage_controllers() self._config = hs.config self._clock = hs.get_clock() - self._request_ratelimiter = hs.get_request_ratelimiter() self._event_creation_handler = hs.get_event_creation_handler() self._room_member_handler = hs.get_room_member_handler() + self._request_ratelimiter = hs.get_request_ratelimiter() + + # Ratelimiter for management of existing delayed events, + # keyed by the sending user ID & device ID. + self._delayed_event_mgmt_ratelimiter = Ratelimiter( + store=self._store, + clock=self._clock, + cfg=self._config.ratelimiting.rc_delayed_event_mgmt, + ) + self._next_delayed_event_call: Optional[IDelayedCall] = None # The current position in the current_state_delta stream @@ -227,6 +237,9 @@ class DelayedEventsHandler: Raises: SynapseError: if the delayed event fails validation checks. """ + # Use standard request limiter for scheduling new delayed events. + # TODO: Instead apply ratelimiting based on the scheduled send time. + # See https://github.com/element-hq/synapse/issues/18021 await self._request_ratelimiter.ratelimit(requester) self._event_creation_handler.validator.validate_builder( @@ -285,7 +298,10 @@ class DelayedEventsHandler: NotFoundError: if no matching delayed event could be found. """ assert self._is_master - await self._request_ratelimiter.ratelimit(requester) + await self._delayed_event_mgmt_ratelimiter.ratelimit( + requester, + (requester.user.to_string(), requester.device_id), + ) await self._initialized_from_db next_send_ts = await self._store.cancel_delayed_event( @@ -308,7 +324,10 @@ class DelayedEventsHandler: NotFoundError: if no matching delayed event could be found. """ assert self._is_master - await self._request_ratelimiter.ratelimit(requester) + await self._delayed_event_mgmt_ratelimiter.ratelimit( + requester, + (requester.user.to_string(), requester.device_id), + ) await self._initialized_from_db next_send_ts = await self._store.restart_delayed_event( @@ -332,6 +351,8 @@ class DelayedEventsHandler: NotFoundError: if no matching delayed event could be found. """ assert self._is_master + # Use standard request limiter for sending delayed events on-demand, + # as an on-demand send is similar to sending a regular event. await self._request_ratelimiter.ratelimit(requester) await self._initialized_from_db @@ -415,7 +436,10 @@ class DelayedEventsHandler: async def get_all_for_user(self, requester: Requester) -> List[JsonDict]: """Return all pending delayed events requested by the given user.""" - await self._request_ratelimiter.ratelimit(requester) + await self._delayed_event_mgmt_ratelimiter.ratelimit( + requester, + (requester.user.to_string(), requester.device_id), + ) return await self._store.get_all_delayed_events_for_user( requester.user.localpart ) diff --git a/tests/rest/client/test_delayed_events.py b/tests/rest/client/test_delayed_events.py index 1793b38c4a..2c938390c8 100644 --- a/tests/rest/client/test_delayed_events.py +++ b/tests/rest/client/test_delayed_events.py @@ -109,6 +109,27 @@ class DelayedEventsTestCase(HomeserverTestCase): ) self.assertEqual(setter_expected, content.get(setter_key), content) + @unittest.override_config( + {"rc_delayed_event_mgmt": {"per_second": 0.5, "burst_count": 1}} + ) + def test_get_delayed_events_ratelimit(self) -> None: + args = ("GET", PATH_PREFIX) + + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.TOO_MANY_REQUESTS, channel.code, channel.result) + + # Add the current user to the ratelimit overrides, allowing them no ratelimiting. + self.get_success( + self.hs.get_datastores().main.set_ratelimit_for_user(self.user_id, 0, 0) + ) + + # Test that the request isn't ratelimited anymore. + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + def test_update_delayed_event_without_id(self) -> None: channel = self.make_request( "POST", @@ -206,6 +227,46 @@ class DelayedEventsTestCase(HomeserverTestCase): expect_code=HTTPStatus.NOT_FOUND, ) + @unittest.override_config( + {"rc_delayed_event_mgmt": {"per_second": 0.5, "burst_count": 1}} + ) + def test_cancel_delayed_event_ratelimit(self) -> None: + delay_ids = [] + for _ in range(2): + channel = self.make_request( + "POST", + _get_path_for_delayed_send(self.room_id, _EVENT_TYPE, 100000), + {}, + ) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + delay_id = channel.json_body.get("delay_id") + self.assertIsNotNone(delay_id) + delay_ids.append(delay_id) + + channel = self.make_request( + "POST", + f"{PATH_PREFIX}/{delay_ids.pop(0)}", + {"action": "cancel"}, + ) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + + args = ( + "POST", + f"{PATH_PREFIX}/{delay_ids.pop(0)}", + {"action": "cancel"}, + ) + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.TOO_MANY_REQUESTS, channel.code, channel.result) + + # Add the current user to the ratelimit overrides, allowing them no ratelimiting. + self.get_success( + self.hs.get_datastores().main.set_ratelimit_for_user(self.user_id, 0, 0) + ) + + # Test that the request isn't ratelimited anymore. + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + def test_send_delayed_state_event(self) -> None: state_key = "to_send_on_request" @@ -250,6 +311,44 @@ class DelayedEventsTestCase(HomeserverTestCase): ) self.assertEqual(setter_expected, content.get(setter_key), content) + @unittest.override_config({"rc_message": {"per_second": 3.5, "burst_count": 4}}) + def test_send_delayed_event_ratelimit(self) -> None: + delay_ids = [] + for _ in range(2): + channel = self.make_request( + "POST", + _get_path_for_delayed_send(self.room_id, _EVENT_TYPE, 100000), + {}, + ) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + delay_id = channel.json_body.get("delay_id") + self.assertIsNotNone(delay_id) + delay_ids.append(delay_id) + + channel = self.make_request( + "POST", + f"{PATH_PREFIX}/{delay_ids.pop(0)}", + {"action": "send"}, + ) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + + args = ( + "POST", + f"{PATH_PREFIX}/{delay_ids.pop(0)}", + {"action": "send"}, + ) + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.TOO_MANY_REQUESTS, channel.code, channel.result) + + # Add the current user to the ratelimit overrides, allowing them no ratelimiting. + self.get_success( + self.hs.get_datastores().main.set_ratelimit_for_user(self.user_id, 0, 0) + ) + + # Test that the request isn't ratelimited anymore. + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + def test_restart_delayed_state_event(self) -> None: state_key = "to_send_on_restarted_timeout" @@ -309,6 +408,46 @@ class DelayedEventsTestCase(HomeserverTestCase): ) self.assertEqual(setter_expected, content.get(setter_key), content) + @unittest.override_config( + {"rc_delayed_event_mgmt": {"per_second": 0.5, "burst_count": 1}} + ) + def test_restart_delayed_event_ratelimit(self) -> None: + delay_ids = [] + for _ in range(2): + channel = self.make_request( + "POST", + _get_path_for_delayed_send(self.room_id, _EVENT_TYPE, 100000), + {}, + ) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + delay_id = channel.json_body.get("delay_id") + self.assertIsNotNone(delay_id) + delay_ids.append(delay_id) + + channel = self.make_request( + "POST", + f"{PATH_PREFIX}/{delay_ids.pop(0)}", + {"action": "restart"}, + ) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + + args = ( + "POST", + f"{PATH_PREFIX}/{delay_ids.pop(0)}", + {"action": "restart"}, + ) + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.TOO_MANY_REQUESTS, channel.code, channel.result) + + # Add the current user to the ratelimit overrides, allowing them no ratelimiting. + self.get_success( + self.hs.get_datastores().main.set_ratelimit_for_user(self.user_id, 0, 0) + ) + + # Test that the request isn't ratelimited anymore. + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + def test_delayed_state_events_are_cancelled_by_more_recent_state(self) -> None: state_key = "to_be_cancelled" @@ -374,3 +513,7 @@ def _get_path_for_delayed_state( room_id: str, event_type: str, state_key: str, delay_ms: int ) -> str: return f"rooms/{room_id}/state/{event_type}/{state_key}?org.matrix.msc4140.delay={delay_ms}" + + +def _get_path_for_delayed_send(room_id: str, event_type: str, delay_ms: int) -> str: + return f"rooms/{room_id}/send/{event_type}?org.matrix.msc4140.delay={delay_ms}" diff --git a/tests/rest/client/test_rooms.py b/tests/rest/client/test_rooms.py index a7108b905a..dd8350ddd1 100644 --- a/tests/rest/client/test_rooms.py +++ b/tests/rest/client/test_rooms.py @@ -2399,6 +2399,41 @@ class RoomDelayedEventTestCase(RoomBase): ) self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + @unittest.override_config( + { + "max_event_delay_duration": "24h", + "rc_message": {"per_second": 1, "burst_count": 2}, + } + ) + def test_add_delayed_event_ratelimit(self) -> None: + """Test that requests to schedule new delayed events are ratelimited by a RateLimiter, + which ratelimits them correctly, including by not limiting when the requester is + exempt from ratelimiting. + """ + + # Test that new delayed events are correctly ratelimited. + args = ( + "POST", + ( + "rooms/%s/send/m.room.message?org.matrix.msc4140.delay=2000" + % self.room_id + ).encode("ascii"), + {"body": "test", "msgtype": "m.text"}, + ) + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.TOO_MANY_REQUESTS, channel.code, channel.result) + + # Add the current user to the ratelimit overrides, allowing them no ratelimiting. + self.get_success( + self.hs.get_datastores().main.set_ratelimit_for_user(self.user_id, 0, 0) + ) + + # Test that the new delayed events aren't ratelimited anymore. + channel = self.make_request(*args) + self.assertEqual(HTTPStatus.OK, channel.code, channel.result) + class RoomSearchTestCase(unittest.HomeserverTestCase): servlets = [ From 5219a9a21490c6300870179519323b5e5f844d50 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Feb 2025 17:55:23 +0000 Subject: [PATCH 60/65] Bump serde from 1.0.217 to 1.0.218 (#18183) Bumps [serde](https://github.com/serde-rs/serde) from 1.0.217 to 1.0.218.
Release notes

Sourced from serde's releases.

v1.0.218

  • Documentation improvements
Commits
  • 7bfd518 Release 1.0.218
  • 723a949 Merge pull request #2895 from dtolnay/stabledoc
  • 2b44efb Point standard library links to stable
  • 03dc0fc Merge pull request #2894 from dtolnay/doclink
  • 85cb0c4 Convert html links to intra-doc links
  • abe7194 Update ui test suite to nightly-2025-02-12
  • aaccac7 Unset doc-scrape-examples for lib target
  • 7cd4d84 Update ui test suite to nightly-2025-02-07
  • 04ff3e8 More precise gitignore patterns
  • dc3031b Remove *.sw[po] from gitignore
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=serde&package-manager=cargo&previous-version=1.0.217&new-version=1.0.218)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 56d8654479..845ff0ae9b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -437,18 +437,18 @@ checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" [[package]] name = "serde" -version = "1.0.217" +version = "1.0.218" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02fc4265df13d6fa1d00ecff087228cc0a2b5f3c0e87e258d8b94a156e984c70" +checksum = "e8dfc9d19bdbf6d17e22319da49161d5d0108e4188e8b680aef6299eed22df60" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.217" +version = "1.0.218" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" +checksum = "f09503e191f4e797cb8aac08e9a4a4695c5edf6a2e70e376d961ddd5c969f82b" dependencies = [ "proc-macro2", "quote", From c4e5a582fbb06391d721ce625d7f9817064f26f2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Feb 2025 18:03:26 +0000 Subject: [PATCH 61/65] Bump pyo3-log from 0.12.0 to 0.12.1 (#18046) Bumps [pyo3-log](https://github.com/vorner/pyo3-log) from 0.12.0 to 0.12.1.
Changelog

Sourced from pyo3-log's changelog.

0.12.1

  • Pass-through exceptions (#57)
Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=pyo3-log&package-manager=cargo&previous-version=0.12.0&new-version=0.12.1)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) You can trigger a rebase of this PR by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
> **Note** > Automatic rebases have been disabled on this pull request as it has been open for over 30 days. --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Devon Hudson Co-authored-by: Devon Hudson --- Cargo.lock | 4 ++-- changelog.d/18046.misc | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) create mode 100644 changelog.d/18046.misc diff --git a/Cargo.lock b/Cargo.lock index 845ff0ae9b..b9aa1c8a6b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -316,9 +316,9 @@ dependencies = [ [[package]] name = "pyo3-log" -version = "0.12.0" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3eb421dc86d38d08e04b927b02424db480be71b777fa3a56f32e2f2a3a1a3b08" +checksum = "be5bb22b77965a7b5394e9aae9897a0607b51df5167561ffc3b02643b4200bc7" dependencies = [ "arc-swap", "log", diff --git a/changelog.d/18046.misc b/changelog.d/18046.misc new file mode 100644 index 0000000000..8acbe6a3db --- /dev/null +++ b/changelog.d/18046.misc @@ -0,0 +1 @@ +Bump pyo3-log from 0.12.0 to 0.12.1. From 131607ee51bd5e922fe4aacd167557bd20659a82 Mon Sep 17 00:00:00 2001 From: V02460 Date: Tue, 25 Feb 2025 22:35:06 +0100 Subject: [PATCH 62/65] Add form_secret_path config option (#18090) I [was told](https://github.com/element-hq/synapse/pull/17983#issuecomment-2593370897) about another config option with a secret, so I got `form_secret` a companion: `form_secret_path` This PR makes NixOS and Kubernetes users a little bit happy. Includes docs and tests. ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --- changelog.d/18090.feature | 1 + docs/usage/configuration/config_documentation.md | 16 ++++++++++++++++ synapse/config/key.py | 16 ++++++++++++++-- tests/config/test_load.py | 7 ++++++- 4 files changed, 37 insertions(+), 3 deletions(-) create mode 100644 changelog.d/18090.feature diff --git a/changelog.d/18090.feature b/changelog.d/18090.feature new file mode 100644 index 0000000000..343e2f45eb --- /dev/null +++ b/changelog.d/18090.feature @@ -0,0 +1 @@ +Add `form_secret_path` config option. \ No newline at end of file diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index 8523c5f65f..f5a5aa2eb4 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -3238,6 +3238,22 @@ Example configuration: ```yaml form_secret: ``` +--- +### `form_secret_path` + +An alternative to [`form_secret`](#form_secret): +allows the secret to be specified in an external file. + +The file should be a plain text file, containing only the secret. +Synapse reads the secret from the given file once at startup. + +Example configuration: +```yaml +form_secret_path: /path/to/secrets/file +``` + +_Added in Synapse 1.125.0._ + --- ## Signing Keys Config options relating to signing keys diff --git a/synapse/config/key.py b/synapse/config/key.py index 6f99f39e81..337f98dbc1 100644 --- a/synapse/config/key.py +++ b/synapse/config/key.py @@ -96,6 +96,11 @@ Conflicting options 'macaroon_secret_key' and 'macaroon_secret_key_path' are both defined in config file. """ +CONFLICTING_FORM_SECRET_OPTS_ERROR = """\ +Conflicting options 'form_secret' and 'form_secret_path' are both defined in +config file. +""" + logger = logging.getLogger(__name__) @@ -201,12 +206,19 @@ class KeyConfig(Config): # a secret which is used to calculate HMACs for form values, to stop # falsification of values - self.form_secret = config.get("form_secret", None) - if self.form_secret and not allow_secrets_in_config: + form_secret = config.get("form_secret", None) + if form_secret and not allow_secrets_in_config: raise ConfigError( "Config options that expect an in-line secret as value are disabled", ("form_secret",), ) + form_secret_path = config.get("form_secret_path", None) + if form_secret_path: + if form_secret: + raise ConfigError(CONFLICTING_FORM_SECRET_OPTS_ERROR) + self.form_secret = read_file(form_secret_path, "form_secret_path").strip() + else: + self.form_secret = form_secret def generate_config_section( self, diff --git a/tests/config/test_load.py b/tests/config/test_load.py index 18fb2e0c2c..14dfa6e59d 100644 --- a/tests/config/test_load.py +++ b/tests/config/test_load.py @@ -138,6 +138,7 @@ class ConfigLoadingFileTestCase(ConfigFileTestCase): "turn_shared_secret_path: /does/not/exist", "registration_shared_secret_path: /does/not/exist", "macaroon_secret_key_path: /does/not/exist", + "form_secret_path: /does/not/exist", "experimental_features:\n msc3861:\n client_secret_path: /does/not/exist", "experimental_features:\n msc3861:\n admin_token_path: /does/not/exist", *["redis:\n enabled: true\n password_path: /does/not/exist"] @@ -165,6 +166,10 @@ class ConfigLoadingFileTestCase(ConfigFileTestCase): "macaroon_secret_key_path: {}", lambda c: c.key.macaroon_secret_key, ), + ( + "form_secret_path: {}", + lambda c: c.key.form_secret.encode("utf-8"), + ), ( "experimental_features:\n msc3861:\n client_secret_path: {}", lambda c: c.experimental.msc3861.client_secret().encode("utf-8"), @@ -186,7 +191,7 @@ class ConfigLoadingFileTestCase(ConfigFileTestCase): self, config_line: str, get_secret: Callable[[RootConfig], str] ) -> None: self.generate_config_and_remove_lines_containing( - ["registration_shared_secret", "macaroon_secret_key"] + ["form_secret", "macaroon_secret_key", "registration_shared_secret"] ) with tempfile.NamedTemporaryFile(buffering=0) as secret_file: secret_file.write(b"53C237") From c360da0f8b642d28601a06f2b5c3e072a5dbfca7 Mon Sep 17 00:00:00 2001 From: V02460 Date: Wed, 26 Feb 2025 16:55:10 +0100 Subject: [PATCH 63/65] Add worker_replication_secret_path config option (#18191) Workers now get their secrets from files, too! There are not many config options left to pathify :) Includes documentation and unit tests. ### Pull Request Checklist * [x] Pull request is based on the develop branch * [x] Pull request includes a [changelog file](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#changelog). The entry should: - Be a short description of your change which makes sense to users. "Fixed a bug that prevented receiving messages from other servers." instead of "Moved X method from `EventStore` to `EventWorkerStore`.". - Use markdown where necessary, mostly for `code blocks`. - End with either a period (.) or an exclamation mark (!). - Start with a capital letter. - Feel free to credit yourself, by adding a sentence "Contributed by @github_username." or "Contributed by [Your Name]." to the end of the entry. * [x] [Code style](https://element-hq.github.io/synapse/latest/code_style.html) is correct (run the [linters](https://element-hq.github.io/synapse/latest/development/contributing_guide.html#run-the-linters)) --------- Co-authored-by: Devon Hudson --- changelog.d/18191.feature | 1 + .../configuration/config_documentation.md | 18 +++++++++++++++- synapse/config/workers.py | 21 +++++++++++++++++-- tests/config/test_load.py | 5 +++++ 4 files changed, 42 insertions(+), 3 deletions(-) create mode 100644 changelog.d/18191.feature diff --git a/changelog.d/18191.feature b/changelog.d/18191.feature new file mode 100644 index 0000000000..f47c9e2275 --- /dev/null +++ b/changelog.d/18191.feature @@ -0,0 +1 @@ +Add `worker_replication_secret_path` config option. \ No newline at end of file diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index f5a5aa2eb4..ffee089304 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -3252,7 +3252,7 @@ Example configuration: form_secret_path: /path/to/secrets/file ``` -_Added in Synapse 1.125.0._ +_Added in Synapse 1.126.0._ --- ## Signing Keys @@ -4527,6 +4527,22 @@ Example configuration: ```yaml worker_replication_secret: "secret_secret" ``` +--- +### `worker_replication_secret_path` + +An alternative to [`worker_replication_secret`](#worker_replication_secret): +allows the secret to be specified in an external file. + +The file should be a plain text file, containing only the secret. +Synapse reads the secret from the given file once at startup. + +Example configuration: +```yaml +worker_replication_secret_path: /path/to/secrets/file +``` + +_Added in Synapse 1.126.0._ + --- ### `start_pushers` diff --git a/synapse/config/workers.py b/synapse/config/workers.py index 632cef46ce..5af50ee952 100644 --- a/synapse/config/workers.py +++ b/synapse/config/workers.py @@ -38,6 +38,7 @@ from synapse.config._base import ( ConfigError, RoutableShardedWorkerHandlingConfig, ShardedWorkerHandlingConfig, + read_file, ) from synapse.config._util import parse_and_validate_mapping from synapse.config.server import ( @@ -65,6 +66,11 @@ configuration under `main` inside the `instance_map`. See workers documentation `https://element-hq.github.io/synapse/latest/workers.html#worker-configuration` """ +CONFLICTING_WORKER_REPLICATION_SECRET_OPTS_ERROR = """\ +Conflicting options 'worker_replication_secret' and +'worker_replication_secret_path' are both defined in config file. +""" + # This allows for a handy knob when it's time to change from 'master' to # something with less 'history' MAIN_PROCESS_INSTANCE_NAME = "master" @@ -244,12 +250,23 @@ class WorkerConfig(Config): raise ConfigError(DIRECT_TCP_ERROR, ("worker_replication_port",)) # The shared secret used for authentication when connecting to the main synapse. - self.worker_replication_secret = config.get("worker_replication_secret", None) - if self.worker_replication_secret and not allow_secrets_in_config: + worker_replication_secret = config.get("worker_replication_secret", None) + if worker_replication_secret and not allow_secrets_in_config: raise ConfigError( "Config options that expect an in-line secret as value are disabled", ("worker_replication_secret",), ) + worker_replication_secret_path = config.get( + "worker_replication_secret_path", None + ) + if worker_replication_secret_path: + if worker_replication_secret: + raise ConfigError(CONFLICTING_WORKER_REPLICATION_SECRET_OPTS_ERROR) + self.worker_replication_secret = read_file( + worker_replication_secret_path, "worker_replication_secret_path" + ).strip() + else: + self.worker_replication_secret = worker_replication_secret self.worker_name = config.get("worker_name", self.worker_app) self.instance_name = self.worker_name or MAIN_PROCESS_INSTANCE_NAME diff --git a/tests/config/test_load.py b/tests/config/test_load.py index 14dfa6e59d..a5456ac6f8 100644 --- a/tests/config/test_load.py +++ b/tests/config/test_load.py @@ -139,6 +139,7 @@ class ConfigLoadingFileTestCase(ConfigFileTestCase): "registration_shared_secret_path: /does/not/exist", "macaroon_secret_key_path: /does/not/exist", "form_secret_path: /does/not/exist", + "worker_replication_secret_path: /does/not/exist", "experimental_features:\n msc3861:\n client_secret_path: /does/not/exist", "experimental_features:\n msc3861:\n admin_token_path: /does/not/exist", *["redis:\n enabled: true\n password_path: /does/not/exist"] @@ -170,6 +171,10 @@ class ConfigLoadingFileTestCase(ConfigFileTestCase): "form_secret_path: {}", lambda c: c.key.form_secret.encode("utf-8"), ), + ( + "worker_replication_secret_path: {}", + lambda c: c.worker.worker_replication_secret.encode("utf-8"), + ), ( "experimental_features:\n msc3861:\n client_secret_path: {}", lambda c: c.experimental.msc3861.client_secret().encode("utf-8"), From 154e23f6d76277cc8012dc7a5dfa5f22d62b9133 Mon Sep 17 00:00:00 2001 From: Andrew Morgan <1342360+anoadragon453@users.noreply.github.com> Date: Mon, 3 Mar 2025 09:40:48 +0000 Subject: [PATCH 64/65] Add `redirect_uri` option to `oidc_providers` entries (#18197) Allows overriding the `redirect_uri` parameter sent to both the authorization and token endpoints of the IdP. Typically this parameter is hardcoded to `/_synapse/client/oidc/callback`. Yet it can be useful in certain contexts to allow a different callback URL. For instance, if you would like to intercept the authorization code returned from the IdP and do something with it, before eventually calling Synapse's OIDC callback URL yourself. This change enables enterprise use cases but does not change the default behaviour. --- Best reviewed commit-by-commit. --------- Co-authored-by: Eric Eastwood --- changelog.d/18197.feature | 1 + .../configuration/config_documentation.md | 7 +++ synapse/config/oidc.py | 16 ++++++ synapse/handlers/oidc.py | 7 ++- tests/handlers/test_oidc.py | 50 +++++++++++++++++++ 5 files changed, 80 insertions(+), 1 deletion(-) create mode 100644 changelog.d/18197.feature diff --git a/changelog.d/18197.feature b/changelog.d/18197.feature new file mode 100644 index 0000000000..4572ac3bdb --- /dev/null +++ b/changelog.d/18197.feature @@ -0,0 +1 @@ +Add support for specifying/overriding `redirect_uri` in the authorization and token requests against an OpenID identity provider. \ No newline at end of file diff --git a/docs/usage/configuration/config_documentation.md b/docs/usage/configuration/config_documentation.md index ffee089304..d2d282f203 100644 --- a/docs/usage/configuration/config_documentation.md +++ b/docs/usage/configuration/config_documentation.md @@ -3662,6 +3662,13 @@ Options for each entry include: not included in `scopes`. Set to `userinfo_endpoint` to always use the userinfo endpoint. +* `redirect_uri`: An optional string, that if set will override the `redirect_uri` + parameter sent in the requests to the authorization and token endpoints. + Useful if you want to redirect the client to another endpoint as part of the + OIDC login. Be aware that the client must then call Synapse's OIDC callback + URL (`/_synapse/client/oidc/callback`) manually afterwards. + Must be a valid URL including scheme and path. + * `additional_authorization_parameters`: String to string dictionary that will be passed as additional parameters to the authorization grant URL. diff --git a/synapse/config/oidc.py b/synapse/config/oidc.py index fc4bc35b30..8ba0ba2c36 100644 --- a/synapse/config/oidc.py +++ b/synapse/config/oidc.py @@ -141,6 +141,9 @@ OIDC_PROVIDER_CONFIG_SCHEMA = { "type": "string", "enum": ["auto", "userinfo_endpoint"], }, + "redirect_uri": { + "type": ["string", "null"], + }, "allow_existing_users": {"type": "boolean"}, "user_mapping_provider": {"type": ["object", "null"]}, "attribute_requirements": { @@ -344,6 +347,7 @@ def _parse_oidc_config_dict( ), skip_verification=oidc_config.get("skip_verification", False), user_profile_method=oidc_config.get("user_profile_method", "auto"), + redirect_uri=oidc_config.get("redirect_uri"), allow_existing_users=oidc_config.get("allow_existing_users", False), user_mapping_provider_class=user_mapping_provider_class, user_mapping_provider_config=user_mapping_provider_config, @@ -467,6 +471,18 @@ class OidcProviderConfig: # values are: "auto" or "userinfo_endpoint". user_profile_method: str + redirect_uri: Optional[str] + """ + An optional replacement for Synapse's hardcoded `redirect_uri` URL + (`/_synapse/client/oidc/callback`). This can be used to send + the client to a different URL after it receives a response from the + `authorization_endpoint`. + + If this is set, the client is expected to call Synapse's OIDC callback URL + reproduced above itself with the necessary parameters and session cookie, in + order to complete OIDC login. + """ + # whether to allow a user logging in via OIDC to match a pre-existing account # instead of failing allow_existing_users: bool diff --git a/synapse/handlers/oidc.py b/synapse/handlers/oidc.py index 76b692928d..18efdd9f6e 100644 --- a/synapse/handlers/oidc.py +++ b/synapse/handlers/oidc.py @@ -382,7 +382,12 @@ class OidcProvider: self._macaroon_generaton = macaroon_generator self._config = provider - self._callback_url: str = hs.config.oidc.oidc_callback_url + + self._callback_url: str + if provider.redirect_uri is not None: + self._callback_url = provider.redirect_uri + else: + self._callback_url = hs.config.oidc.oidc_callback_url # Calculate the prefix for OIDC callback paths based on the public_baseurl. # We'll insert this into the Path= parameter of any session cookies we set. diff --git a/tests/handlers/test_oidc.py b/tests/handlers/test_oidc.py index 5ffc5a90a8..cfd9969563 100644 --- a/tests/handlers/test_oidc.py +++ b/tests/handlers/test_oidc.py @@ -57,6 +57,7 @@ CLIENT_ID = "test-client-id" CLIENT_SECRET = "test-client-secret" BASE_URL = "https://synapse/" CALLBACK_URL = BASE_URL + "_synapse/client/oidc/callback" +TEST_REDIRECT_URI = "https://test/oidc/callback" SCOPES = ["openid"] # config for common cases @@ -586,6 +587,24 @@ class OidcHandlerTestCase(HomeserverTestCase): code_verifier = get_value_from_macaroon(macaroon, "code_verifier") self.assertEqual(code_verifier, "") + @override_config( + {"oidc_config": {**DEFAULT_CONFIG, "redirect_uri": TEST_REDIRECT_URI}} + ) + def test_redirect_request_with_overridden_redirect_uri(self) -> None: + """The authorization endpoint redirect has the overridden `redirect_uri` value.""" + req = Mock(spec=["cookies"]) + req.cookies = [] + + url = urlparse( + self.get_success( + self.provider.handle_redirect_request(req, b"http://client/redirect") + ) + ) + + # Ensure that the redirect_uri in the returned url has been overridden. + params = parse_qs(url.query) + self.assertEqual(params["redirect_uri"], [TEST_REDIRECT_URI]) + @override_config({"oidc_config": DEFAULT_CONFIG}) def test_callback_error(self) -> None: """Errors from the provider returned in the callback are displayed.""" @@ -953,6 +972,37 @@ class OidcHandlerTestCase(HomeserverTestCase): self.assertEqual(args["client_id"], [CLIENT_ID]) self.assertEqual(args["redirect_uri"], [CALLBACK_URL]) + @override_config( + { + "oidc_config": { + **DEFAULT_CONFIG, + "redirect_uri": TEST_REDIRECT_URI, + } + } + ) + def test_code_exchange_with_overridden_redirect_uri(self) -> None: + """Code exchange behaves correctly and handles various error scenarios.""" + # Set up a fake IdP with a token endpoint handler. + token = { + "type": "Bearer", + "access_token": "aabbcc", + } + + self.fake_server.post_token_handler.side_effect = None + self.fake_server.post_token_handler.return_value = FakeResponse.json( + payload=token + ) + code = "code" + + # Exchange the code against the fake IdP. + self.get_success(self.provider._exchange_code(code, code_verifier="")) + + # Check that the `redirect_uri` parameter provided matches our + # overridden config value. + kwargs = self.fake_server.request.call_args[1] + args = parse_qs(kwargs["data"].decode("utf-8")) + self.assertEqual(args["redirect_uri"], [TEST_REDIRECT_URI]) + @override_config( { "oidc_config": { From 08c56c3acca4b3a4207c4a28b2b76b98a07403de Mon Sep 17 00:00:00 2001 From: Quentin Gliech Date: Tue, 4 Mar 2025 14:08:44 +0100 Subject: [PATCH 65/65] Support getting the device ID explicitly from MAS (#18174) The context for this is that the Matrix spec allows basically anything in the device ID. With MSC3861, we're restricting this to strings that can be represented as scopes. Whilst this works well for next-gen auth sessions, compatibility/legacy sessions still can have characters that can't be encoded (mainly spaces) in them. To work around that, we added in MAS a behaviour where the device_id is given as an explicit property of the token introspection response, and remove it from the scope. Because we don't expect users to rollout new Synapse and MAS versions in sync, we needed a way to 'advertise' support for this behaviour: the easiest way to do that was through an extra header in the introspection response. On the longer term, I expect MAS and Synapse to move away from the introspection endpoint, and instead define a specific API for Synapse -> MAS communication. PR on the MAS side: https://github.com/element-hq/matrix-authentication-service/pull/4067 --- changelog.d/18174.misc | 1 + synapse/api/auth/msc3861_delegated.py | 47 ++++++++++++++++--------- tests/handlers/test_oauth_delegation.py | 38 ++++++++++++++++++++ 3 files changed, 70 insertions(+), 16 deletions(-) create mode 100644 changelog.d/18174.misc diff --git a/changelog.d/18174.misc b/changelog.d/18174.misc new file mode 100644 index 0000000000..b7c7c4db53 --- /dev/null +++ b/changelog.d/18174.misc @@ -0,0 +1 @@ +Support device IDs that can't be represented in a scope when delegating auth to Matrix Authentication Service 0.15.0+. diff --git a/synapse/api/auth/msc3861_delegated.py b/synapse/api/auth/msc3861_delegated.py index f825b5c95e..e6bf271a1f 100644 --- a/synapse/api/auth/msc3861_delegated.py +++ b/synapse/api/auth/msc3861_delegated.py @@ -214,6 +214,9 @@ class MSC3861DelegatedAuth(BaseAuth): "Content-Type": "application/x-www-form-urlencoded", "User-Agent": str(self._http_client.user_agent, "utf-8"), "Accept": "application/json", + # Tell MAS that we support reading the device ID as an explicit + # value, not encoded in the scope. This is supported by MAS 0.15+ + "X-MAS-Supports-Device-Id": "1", } args = {"token": token, "token_type_hint": "access_token"} @@ -409,29 +412,41 @@ class MSC3861DelegatedAuth(BaseAuth): else: user_id = UserID.from_string(user_id_str) - # Find device_ids in scope - # We only allow a single device_id in the scope, so we find them all in the - # scope list, and raise if there are more than one. The OIDC server should be - # the one enforcing valid scopes, so we raise a 500 if we find an invalid scope. - device_ids = [ - tok[len(SCOPE_MATRIX_DEVICE_PREFIX) :] - for tok in scope - if tok.startswith(SCOPE_MATRIX_DEVICE_PREFIX) - ] + # MAS 0.15+ will give us the device ID as an explicit value for compatibility sessions + # If present, we get it from here, if not we get it in thee scope + device_id = introspection_result.get("device_id") + if device_id is not None: + # We got the device ID explicitly, just sanity check that it's a string + if not isinstance(device_id, str): + raise AuthError( + 500, + "Invalid device ID in introspection result", + ) + else: + # Find device_ids in scope + # We only allow a single device_id in the scope, so we find them all in the + # scope list, and raise if there are more than one. The OIDC server should be + # the one enforcing valid scopes, so we raise a 500 if we find an invalid scope. + device_ids = [ + tok[len(SCOPE_MATRIX_DEVICE_PREFIX) :] + for tok in scope + if tok.startswith(SCOPE_MATRIX_DEVICE_PREFIX) + ] - if len(device_ids) > 1: - raise AuthError( - 500, - "Multiple device IDs in scope", - ) + if len(device_ids) > 1: + raise AuthError( + 500, + "Multiple device IDs in scope", + ) + + device_id = device_ids[0] if device_ids else None - device_id = device_ids[0] if device_ids else None if device_id is not None: # Sanity check the device_id if len(device_id) > 255 or len(device_id) < 1: raise AuthError( 500, - "Invalid device ID in scope", + "Invalid device ID in introspection result", ) # Create the device on the fly if it does not exist diff --git a/tests/handlers/test_oauth_delegation.py b/tests/handlers/test_oauth_delegation.py index b9d66a6c52..5f8c25557a 100644 --- a/tests/handlers/test_oauth_delegation.py +++ b/tests/handlers/test_oauth_delegation.py @@ -380,6 +380,44 @@ class MSC3861OAuthDelegation(HomeserverTestCase): ) self.assertEqual(requester.device_id, DEVICE) + def test_active_user_with_device_explicit_device_id(self) -> None: + """The handler should return a requester with normal user rights and a device ID, given explicitly, as supported by MAS 0.15+""" + + self.http_client.request = AsyncMock( + return_value=FakeResponse.json( + code=200, + payload={ + "active": True, + "sub": SUBJECT, + "scope": " ".join([MATRIX_USER_SCOPE]), + "device_id": DEVICE, + "username": USERNAME, + }, + ) + ) + request = Mock(args={}) + request.args[b"access_token"] = [b"mockAccessToken"] + request.requestHeaders.getRawHeaders = mock_getRawHeaders() + requester = self.get_success(self.auth.get_user_by_req(request)) + self.http_client.get_json.assert_called_once_with(WELL_KNOWN) + self.http_client.request.assert_called_once_with( + method="POST", uri=INTROSPECTION_ENDPOINT, data=ANY, headers=ANY + ) + # It should have called with the 'X-MAS-Supports-Device-Id: 1' header + self.assertEqual( + self.http_client.request.call_args[1]["headers"].getRawHeaders( + b"X-MAS-Supports-Device-Id", + ), + [b"1"], + ) + self._assertParams() + self.assertEqual(requester.user.to_string(), "@%s:%s" % (USERNAME, SERVER_NAME)) + self.assertEqual(requester.is_guest, False) + self.assertEqual( + get_awaitable_result(self.auth.is_server_admin(requester)), False + ) + self.assertEqual(requester.device_id, DEVICE) + def test_multiple_devices(self) -> None: """The handler should raise an error if multiple devices are found in the scope."""