Simplify event persistence code (#16584)

The event persistence code used to handle multiple rooms
at a time, but was simplified to only ever be called with a
single room at a time (different rooms are now handled in
parallel). The code is still generic to multiple rooms causing
a lot of work that is unnecessary (e.g. unnecessary loops, and
partitioning data by room).

This strips out the ability to handle multiple rooms at once, greatly
simplifying the code.
This commit is contained in:
Patrick Cloke 2023-11-03 07:30:31 -04:00 committed by GitHub
parent 0afbef30cf
commit 92828a7f95
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
4 changed files with 324 additions and 310 deletions

1
changelog.d/16584.misc Normal file
View file

@ -0,0 +1 @@
Simplify persistance code to be per-room.

1
changelog.d/16586.misc Normal file
View file

@ -0,0 +1 @@
Avoid updating the stream cache unnecessarily.

View file

@ -542,13 +542,15 @@ class EventsPersistenceStorageController:
return await res.get_state(self._state_controller, StateFilter.all()) return await res.get_state(self._state_controller, StateFilter.all())
async def _persist_event_batch( async def _persist_event_batch(
self, _room_id: str, task: _PersistEventsTask self, room_id: str, task: _PersistEventsTask
) -> Dict[str, str]: ) -> Dict[str, str]:
"""Callback for the _event_persist_queue """Callback for the _event_persist_queue
Calculates the change to current state and forward extremities, and Calculates the change to current state and forward extremities, and
persists the given events and with those updates. persists the given events and with those updates.
Assumes that we are only persisting events for one room at a time.
Returns: Returns:
A dictionary of event ID to event ID we didn't persist as we already A dictionary of event ID to event ID we didn't persist as we already
had another event persisted with the same TXN ID. had another event persisted with the same TXN ID.
@ -594,41 +596,55 @@ class EventsPersistenceStorageController:
# We can't easily parallelize these since different chunks # We can't easily parallelize these since different chunks
# might contain the same event. :( # might contain the same event. :(
# NB: Assumes that we are only persisting events for one room new_forward_extremities = None
# at a time. state_delta_for_room = None
# map room_id->set[event_ids] giving the new forward
# extremities in each room
new_forward_extremities: Dict[str, Set[str]] = {}
# map room_id->(to_delete, to_insert) where to_delete is a list
# of type/state keys to remove from current state, and to_insert
# is a map (type,key)->event_id giving the state delta in each
# room
state_delta_for_room: Dict[str, DeltaState] = {}
if not backfilled: if not backfilled:
with Measure(self._clock, "_calculate_state_and_extrem"): with Measure(self._clock, "_calculate_state_and_extrem"):
# Work out the new "current state" for each room. # Work out the new "current state" for the room.
# We do this by working out what the new extremities are and then # We do this by working out what the new extremities are and then
# calculating the state from that. # calculating the state from that.
events_by_room: Dict[str, List[Tuple[EventBase, EventContext]]] = {} (
for event, context in chunk: new_forward_extremities,
events_by_room.setdefault(event.room_id, []).append( state_delta_for_room,
(event, context) ) = await self._calculate_new_forward_extremities_and_state_delta(
room_id, chunk
) )
for room_id, ev_ctx_rm in events_by_room.items(): await self.persist_events_store._persist_events_and_state_updates(
latest_event_ids = ( room_id,
await self.main_store.get_latest_event_ids_in_room(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,
) )
return replaced_events
async def _calculate_new_forward_extremities_and_state_delta(
self, room_id: str, ev_ctx_rm: List[Tuple[EventBase, EventContext]]
) -> Tuple[Optional[Set[str]], Optional[DeltaState]]:
"""Calculates the new forward extremities and state delta for a room
given events to persist.
Assumes that we are only persisting events for one room at a time.
Returns:
A tuple of:
A set of str giving the new forward extremities the room
The state delta for the room.
"""
latest_event_ids = await self.main_store.get_latest_event_ids_in_room(room_id)
new_latest_event_ids = await self._calculate_new_extremities( new_latest_event_ids = await self._calculate_new_extremities(
room_id, ev_ctx_rm, latest_event_ids room_id, ev_ctx_rm, latest_event_ids
) )
if new_latest_event_ids == latest_event_ids: if new_latest_event_ids == latest_event_ids:
# No change in extremities, so no change in state # No change in extremities, so no change in state
continue return (None, None)
# there should always be at least one forward extremity. # there should always be at least one forward extremity.
# (except during the initial persistence of the send_join # (except during the initial persistence of the send_join
@ -636,22 +652,18 @@ class EventsPersistenceStorageController:
# extremities, so we'll `continue` above and skip this bit.) # extremities, so we'll `continue` above and skip this bit.)
assert new_latest_event_ids, "No forward extremities left!" assert new_latest_event_ids, "No forward extremities left!"
new_forward_extremities[room_id] = new_latest_event_ids new_forward_extremities = new_latest_event_ids
len_1 = ( len_1 = len(latest_event_ids) == 1 and len(new_latest_event_ids) == 1
len(latest_event_ids) == 1
and len(new_latest_event_ids) == 1
)
if len_1: if len_1:
all_single_prev_not_state = all( all_single_prev_not_state = all(
len(event.prev_event_ids()) == 1 len(event.prev_event_ids()) == 1 and not event.is_state()
and not event.is_state()
for event, ctx in ev_ctx_rm for event, ctx in ev_ctx_rm
) )
# Don't bother calculating state if they're just # Don't bother calculating state if they're just
# a long chain of single ancestor non-state events. # a long chain of single ancestor non-state events.
if all_single_prev_not_state: if all_single_prev_not_state:
continue return (new_forward_extremities, None)
state_delta_counter.inc() state_delta_counter.inc()
if len(new_latest_event_ids) == 1: if len(new_latest_event_ids) == 1:
@ -674,9 +686,7 @@ class EventsPersistenceStorageController:
break break
logger.debug("Calculating state delta for room %s", room_id) logger.debug("Calculating state delta for room %s", room_id)
with Measure( with Measure(self._clock, "persist_events.get_new_state_after_events"):
self._clock, "persist_events.get_new_state_after_events"
):
res = await self._get_new_state_after_events( res = await self._get_new_state_after_events(
room_id, room_id,
ev_ctx_rm, ev_ctx_rm,
@ -691,7 +701,7 @@ class EventsPersistenceStorageController:
# extremities, so we'll `continue` above and skip this bit.) # extremities, so we'll `continue` above and skip this bit.)
assert new_latest_event_ids, "No forward extremities left!" assert new_latest_event_ids, "No forward extremities left!"
new_forward_extremities[room_id] = new_latest_event_ids new_forward_extremities = new_latest_event_ids
# If either are not None then there has been a change, # If either are not None then there has been a change,
# and we need to work out the delta (or use that # and we need to work out the delta (or use that
@ -703,12 +713,8 @@ class EventsPersistenceStorageController:
# removed keys entirely. # removed keys entirely.
delta = DeltaState([], delta_ids) delta = DeltaState([], delta_ids)
elif current_state is not None: elif current_state is not None:
with Measure( with Measure(self._clock, "persist_events.calculate_state_delta"):
self._clock, "persist_events.calculate_state_delta" delta = await self._calculate_state_delta(room_id, current_state)
):
delta = await self._calculate_state_delta(
room_id, current_state
)
if delta: if delta:
# If we have a change of state then lets check # If we have a change of state then lets check
@ -725,17 +731,7 @@ class EventsPersistenceStorageController:
logger.info("Server no longer in room %s", room_id) logger.info("Server no longer in room %s", room_id)
delta.no_longer_in_room = True delta.no_longer_in_room = True
state_delta_for_room[room_id] = delta return (new_forward_extremities, delta)
await self.persist_events_store._persist_events_and_state_updates(
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,
)
return replaced_events
async def _calculate_new_extremities( async def _calculate_new_extremities(
self, self,

View file

@ -79,7 +79,7 @@ class DeltaState:
Attributes: Attributes:
to_delete: List of type/state_keys to delete from current state to_delete: List of type/state_keys to delete from current state
to_insert: Map of state to upsert into current state to_insert: Map of state to upsert into current state
no_longer_in_room: The server is not longer in the room, so the room no_longer_in_room: The server is no longer in the room, so the room
should e.g. be removed from `current_state_events` table. should e.g. be removed from `current_state_events` table.
""" """
@ -131,22 +131,25 @@ class PersistEventsStore:
@trace @trace
async def _persist_events_and_state_updates( async def _persist_events_and_state_updates(
self, self,
room_id: str,
events_and_contexts: List[Tuple[EventBase, EventContext]], events_and_contexts: List[Tuple[EventBase, EventContext]],
*, *,
state_delta_for_room: Dict[str, DeltaState], state_delta_for_room: Optional[DeltaState],
new_forward_extremities: Dict[str, Set[str]], new_forward_extremities: Optional[Set[str]],
use_negative_stream_ordering: bool = False, use_negative_stream_ordering: bool = False,
inhibit_local_membership_updates: bool = False, inhibit_local_membership_updates: bool = False,
) -> None: ) -> None:
"""Persist a set of events alongside updates to the current state and """Persist a set of events alongside updates to the current state and
forward extremities tables. forward extremities tables.
Assumes that we are only persisting events for one room at a time.
Args: Args:
room_id:
events_and_contexts: events_and_contexts:
state_delta_for_room: Map from room_id to the delta to apply to state_delta_for_room: The delta to apply to the room state
room state new_forward_extremities: A set of event IDs that are the new forward
new_forward_extremities: Map from room_id to set of event IDs extremities of the room.
that are the new forward extremities of the room.
use_negative_stream_ordering: Whether to start stream_ordering on use_negative_stream_ordering: Whether to start stream_ordering on
the negative side and decrement. This should be set as True the negative side and decrement. This should be set as True
for backfilled events because backfilled events get a negative for backfilled events because backfilled events get a negative
@ -196,6 +199,7 @@ class PersistEventsStore:
await self.db_pool.runInteraction( await self.db_pool.runInteraction(
"persist_events", "persist_events",
self._persist_events_txn, self._persist_events_txn,
room_id=room_id,
events_and_contexts=events_and_contexts, events_and_contexts=events_and_contexts,
inhibit_local_membership_updates=inhibit_local_membership_updates, inhibit_local_membership_updates=inhibit_local_membership_updates,
state_delta_for_room=state_delta_for_room, state_delta_for_room=state_delta_for_room,
@ -221,9 +225,9 @@ class PersistEventsStore:
event_counter.labels(event.type, origin_type, origin_entity).inc() event_counter.labels(event.type, origin_type, origin_entity).inc()
for room_id, latest_event_ids in new_forward_extremities.items(): if new_forward_extremities:
self.store.get_latest_event_ids_in_room.prefill( self.store.get_latest_event_ids_in_room.prefill(
(room_id,), frozenset(latest_event_ids) (room_id,), frozenset(new_forward_extremities)
) )
async def _get_events_which_are_prevs(self, event_ids: Iterable[str]) -> List[str]: async def _get_events_which_are_prevs(self, event_ids: Iterable[str]) -> List[str]:
@ -336,10 +340,11 @@ class PersistEventsStore:
self, self,
txn: LoggingTransaction, txn: LoggingTransaction,
*, *,
room_id: str,
events_and_contexts: List[Tuple[EventBase, EventContext]], events_and_contexts: List[Tuple[EventBase, EventContext]],
inhibit_local_membership_updates: bool, inhibit_local_membership_updates: bool,
state_delta_for_room: Dict[str, DeltaState], state_delta_for_room: Optional[DeltaState],
new_forward_extremities: Dict[str, Set[str]], new_forward_extremities: Optional[Set[str]],
) -> None: ) -> None:
"""Insert some number of room events into the necessary database tables. """Insert some number of room events into the necessary database tables.
@ -347,8 +352,11 @@ class PersistEventsStore:
and the rejections table. Things reading from those table will need to check and the rejections table. Things reading from those table will need to check
whether the event was rejected. whether the event was rejected.
Assumes that we are only persisting events for one room at a time.
Args: Args:
txn txn
room_id: The room the events are from
events_and_contexts: events to persist events_and_contexts: events to persist
inhibit_local_membership_updates: Stop the local_current_membership inhibit_local_membership_updates: Stop the local_current_membership
from being updated by these events. This should be set to True from being updated by these events. This should be set to True
@ -357,10 +365,9 @@ class PersistEventsStore:
delete_existing True to purge existing table rows for the events delete_existing True to purge existing table rows for the events
from the database. This is useful when retrying due to from the database. This is useful when retrying due to
IntegrityError. IntegrityError.
state_delta_for_room: The current-state delta for each room. state_delta_for_room: The current-state delta for the room.
new_forward_extremities: The new forward extremities for each room. new_forward_extremities: The new forward extremities for the room:
For each room, a list of the event ids which are the forward a set of the event ids which are the forward extremities.
extremities.
Raises: Raises:
PartialStateConflictError: if attempting to persist a partial state event in PartialStateConflictError: if attempting to persist a partial state event in
@ -376,7 +383,6 @@ class PersistEventsStore:
# #
# Annoyingly SQLite doesn't support row level locking. # Annoyingly SQLite doesn't support row level locking.
if isinstance(self.database_engine, PostgresEngine): if isinstance(self.database_engine, PostgresEngine):
for room_id in {e.room_id for e, _ in events_and_contexts}:
txn.execute( txn.execute(
"SELECT room_version FROM rooms WHERE room_id = ? FOR SHARE", "SELECT room_version FROM rooms WHERE room_id = ? FOR SHARE",
(room_id,), (room_id,),
@ -419,7 +425,9 @@ class PersistEventsStore:
events_and_contexts events_and_contexts
) )
self._update_room_depths_txn(txn, events_and_contexts=events_and_contexts) self._update_room_depths_txn(
txn, room_id, events_and_contexts=events_and_contexts
)
# _update_outliers_txn filters out any events which have already been # _update_outliers_txn filters out any events which have already been
# persisted, and returns the filtered list. # persisted, and returns the filtered list.
@ -432,8 +440,10 @@ class PersistEventsStore:
self._store_event_txn(txn, events_and_contexts=events_and_contexts) self._store_event_txn(txn, events_and_contexts=events_and_contexts)
if new_forward_extremities:
self._update_forward_extremities_txn( self._update_forward_extremities_txn(
txn, txn,
room_id,
new_forward_extremities=new_forward_extremities, new_forward_extremities=new_forward_extremities,
max_stream_order=max_stream_order, max_stream_order=max_stream_order,
) )
@ -464,7 +474,10 @@ class PersistEventsStore:
# We call this last as it assumes we've inserted the events into # We call this last as it assumes we've inserted the events into
# room_memberships, where applicable. # room_memberships, where applicable.
# NB: This function invalidates all state related caches # NB: This function invalidates all state related caches
self._update_current_state_txn(txn, state_delta_for_room, min_stream_order) if state_delta_for_room:
self._update_current_state_txn(
txn, room_id, state_delta_for_room, min_stream_order
)
def _persist_event_auth_chain_txn( def _persist_event_auth_chain_txn(
self, self,
@ -1026,17 +1039,18 @@ class PersistEventsStore:
await self.db_pool.runInteraction( await self.db_pool.runInteraction(
"update_current_state", "update_current_state",
self._update_current_state_txn, self._update_current_state_txn,
state_delta_by_room={room_id: state_delta}, room_id,
delta_state=state_delta,
stream_id=stream_ordering, stream_id=stream_ordering,
) )
def _update_current_state_txn( def _update_current_state_txn(
self, self,
txn: LoggingTransaction, txn: LoggingTransaction,
state_delta_by_room: Dict[str, DeltaState], room_id: str,
delta_state: DeltaState,
stream_id: int, stream_id: int,
) -> None: ) -> None:
for room_id, delta_state in state_delta_by_room.items():
to_delete = delta_state.to_delete to_delete = delta_state.to_delete
to_insert = delta_state.to_insert to_insert = delta_state.to_insert
@ -1190,9 +1204,7 @@ class PersistEventsStore:
) )
# Invalidate the various caches # Invalidate the various caches
self.store._invalidate_state_caches_and_stream( self.store._invalidate_state_caches_and_stream(txn, room_id, members_changed)
txn, room_id, members_changed
)
# Check if any of the remote membership changes requires us to # Check if any of the remote membership changes requires us to
# unsubscribe from their device lists. # unsubscribe from their device lists.
@ -1232,10 +1244,10 @@ class PersistEventsStore:
def _update_forward_extremities_txn( def _update_forward_extremities_txn(
self, self,
txn: LoggingTransaction, txn: LoggingTransaction,
new_forward_extremities: Dict[str, Set[str]], room_id: str,
new_forward_extremities: Set[str],
max_stream_order: int, max_stream_order: int,
) -> None: ) -> None:
for room_id in new_forward_extremities.keys():
self.db_pool.simple_delete_txn( self.db_pool.simple_delete_txn(
txn, table="event_forward_extremities", keyvalues={"room_id": room_id} txn, table="event_forward_extremities", keyvalues={"room_id": room_id}
) )
@ -1244,11 +1256,7 @@ class PersistEventsStore:
txn, txn,
table="event_forward_extremities", table="event_forward_extremities",
keys=("event_id", "room_id"), keys=("event_id", "room_id"),
values=[ values=[(ev_id, room_id) for ev_id in new_forward_extremities],
(ev_id, room_id)
for room_id, new_extrem in new_forward_extremities.items()
for ev_id in new_extrem
],
) )
# We now insert into stream_ordering_to_exterm a mapping from room_id, # We now insert into stream_ordering_to_exterm a mapping from room_id,
# new stream_ordering to new forward extremeties in the room. # new stream_ordering to new forward extremeties in the room.
@ -1260,8 +1268,7 @@ class PersistEventsStore:
keys=("room_id", "event_id", "stream_ordering"), keys=("room_id", "event_id", "stream_ordering"),
values=[ values=[
(room_id, event_id, max_stream_order) (room_id, event_id, max_stream_order)
for room_id, new_extrem in new_forward_extremities.items() for event_id in new_forward_extremities
for event_id in new_extrem
], ],
) )
@ -1298,36 +1305,45 @@ class PersistEventsStore:
def _update_room_depths_txn( def _update_room_depths_txn(
self, self,
txn: LoggingTransaction, txn: LoggingTransaction,
room_id: str,
events_and_contexts: List[Tuple[EventBase, EventContext]], events_and_contexts: List[Tuple[EventBase, EventContext]],
) -> None: ) -> None:
"""Update min_depth for each room """Update min_depth for each room
Args: Args:
txn: db connection txn: db connection
room_id: The room ID
events_and_contexts: events we are persisting events_and_contexts: events we are persisting
""" """
depth_updates: Dict[str, int] = {} stream_ordering: Optional[int] = None
depth_update = 0
for event, context in events_and_contexts: for event, context in events_and_contexts:
# Then update the `stream_ordering` position to mark the latest # Don't update the stream ordering for backfilled events because
# event as the front of the room. This should not be done for # backfilled events have negative stream_ordering and happened in the
# backfilled events because backfilled events have negative # past, so we know that we don't need to update the stream_ordering
# stream_ordering and happened in the past so we know that we don't # tip/front for the room.
# need to update the stream_ordering tip/front for the room.
assert event.internal_metadata.stream_ordering is not None assert event.internal_metadata.stream_ordering is not None
if event.internal_metadata.stream_ordering >= 0: if event.internal_metadata.stream_ordering >= 0:
txn.call_after( if stream_ordering is None:
self.store._events_stream_cache.entity_has_changed, stream_ordering = event.internal_metadata.stream_ordering
event.room_id, else:
event.internal_metadata.stream_ordering, stream_ordering = max(
stream_ordering, event.internal_metadata.stream_ordering
) )
if not event.internal_metadata.is_outlier() and not context.rejected: if not event.internal_metadata.is_outlier() and not context.rejected:
depth_updates[event.room_id] = max( depth_update = max(event.depth, depth_update)
event.depth, depth_updates.get(event.room_id, event.depth)
# Then update the `stream_ordering` position to mark the latest event as
# the front of the room.
if stream_ordering is not None:
txn.call_after(
self.store._events_stream_cache.entity_has_changed,
room_id,
stream_ordering,
) )
for room_id, depth in depth_updates.items(): self._update_min_depth_for_room_txn(txn, room_id, depth_update)
self._update_min_depth_for_room_txn(txn, room_id, depth)
def _update_outliers_txn( def _update_outliers_txn(
self, self,