mirror of
https://git.anonymousland.org/anonymousland/synapse.git
synced 2024-12-26 07:19:23 -05:00
Add some replication tests (#7278)
Specifically some tests for the typing stream, which means we test streams that fetch missing updates via HTTP (rather than via the DB). We also shuffle things around a bit so that we create two separate `HomeServer` objects, rather than trying to insert a slaved store into places. Note: `test_typing.py` is heavily inspired by `test_receipts.py`
This commit is contained in:
parent
ce207aa0ff
commit
fce663889b
1
changelog.d/7278.misc
Normal file
1
changelog.d/7278.misc
Normal file
@ -0,0 +1 @@
|
|||||||
|
Add some unit tests for replication.
|
@ -12,34 +12,67 @@
|
|||||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
# See the License for the specific language governing permissions and
|
# See the License for the specific language governing permissions and
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
|
import logging
|
||||||
|
from typing import Optional
|
||||||
|
|
||||||
from mock import Mock
|
from mock import Mock
|
||||||
|
|
||||||
|
import attr
|
||||||
|
|
||||||
|
from twisted.internet.interfaces import IConsumer, IPullProducer, IReactorTime
|
||||||
|
from twisted.internet.task import LoopingCall
|
||||||
|
from twisted.web.http import HTTPChannel
|
||||||
|
|
||||||
|
from synapse.app.generic_worker import GenericWorkerServer
|
||||||
|
from synapse.http.site import SynapseRequest
|
||||||
|
from synapse.replication.tcp.client import ReplicationDataHandler
|
||||||
from synapse.replication.tcp.handler import ReplicationCommandHandler
|
from synapse.replication.tcp.handler import ReplicationCommandHandler
|
||||||
from synapse.replication.tcp.protocol import ClientReplicationStreamProtocol
|
from synapse.replication.tcp.protocol import ClientReplicationStreamProtocol
|
||||||
from synapse.replication.tcp.resource import ReplicationStreamProtocolFactory
|
from synapse.replication.tcp.resource import ReplicationStreamProtocolFactory
|
||||||
|
from synapse.util import Clock
|
||||||
|
|
||||||
from tests import unittest
|
from tests import unittest
|
||||||
from tests.server import FakeTransport
|
from tests.server import FakeTransport
|
||||||
|
|
||||||
|
logger = logging.getLogger(__name__)
|
||||||
|
|
||||||
|
|
||||||
class BaseStreamTestCase(unittest.HomeserverTestCase):
|
class BaseStreamTestCase(unittest.HomeserverTestCase):
|
||||||
"""Base class for tests of the replication streams"""
|
"""Base class for tests of the replication streams"""
|
||||||
|
|
||||||
def make_homeserver(self, reactor, clock):
|
|
||||||
self.test_handler = Mock(wraps=TestReplicationDataHandler())
|
|
||||||
return self.setup_test_homeserver(replication_data_handler=self.test_handler)
|
|
||||||
|
|
||||||
def prepare(self, reactor, clock, hs):
|
def prepare(self, reactor, clock, hs):
|
||||||
# build a replication server
|
# build a replication server
|
||||||
server_factory = ReplicationStreamProtocolFactory(hs)
|
server_factory = ReplicationStreamProtocolFactory(hs)
|
||||||
self.streamer = hs.get_replication_streamer()
|
self.streamer = hs.get_replication_streamer()
|
||||||
self.server = server_factory.buildProtocol(None)
|
self.server = server_factory.buildProtocol(None)
|
||||||
|
|
||||||
repl_handler = ReplicationCommandHandler(hs)
|
# Make a new HomeServer object for the worker
|
||||||
repl_handler.handler = self.test_handler
|
config = self.default_config()
|
||||||
|
config["worker_app"] = "synapse.app.generic_worker"
|
||||||
|
config["worker_replication_host"] = "testserv"
|
||||||
|
config["worker_replication_http_port"] = "8765"
|
||||||
|
|
||||||
|
self.reactor.lookups["testserv"] = "1.2.3.4"
|
||||||
|
|
||||||
|
self.worker_hs = self.setup_test_homeserver(
|
||||||
|
http_client=None,
|
||||||
|
homeserverToUse=GenericWorkerServer,
|
||||||
|
config=config,
|
||||||
|
reactor=self.reactor,
|
||||||
|
)
|
||||||
|
|
||||||
|
# Since we use sqlite in memory databases we need to make sure the
|
||||||
|
# databases objects are the same.
|
||||||
|
self.worker_hs.get_datastore().db = hs.get_datastore().db
|
||||||
|
|
||||||
|
self.test_handler = Mock(
|
||||||
|
wraps=TestReplicationDataHandler(self.worker_hs.get_datastore())
|
||||||
|
)
|
||||||
|
self.worker_hs.replication_data_handler = self.test_handler
|
||||||
|
|
||||||
|
repl_handler = ReplicationCommandHandler(self.worker_hs)
|
||||||
self.client = ClientReplicationStreamProtocol(
|
self.client = ClientReplicationStreamProtocol(
|
||||||
hs, "client", "test", clock, repl_handler,
|
self.worker_hs, "client", "test", clock, repl_handler,
|
||||||
)
|
)
|
||||||
|
|
||||||
self._client_transport = None
|
self._client_transport = None
|
||||||
@ -74,11 +107,75 @@ class BaseStreamTestCase(unittest.HomeserverTestCase):
|
|||||||
self.streamer.on_notifier_poke()
|
self.streamer.on_notifier_poke()
|
||||||
self.pump(0.1)
|
self.pump(0.1)
|
||||||
|
|
||||||
|
def handle_http_replication_attempt(self) -> SynapseRequest:
|
||||||
|
"""Asserts that a connection attempt was made to the master HS on the
|
||||||
|
HTTP replication port, then proxies it to the master HS object to be
|
||||||
|
handled.
|
||||||
|
|
||||||
class TestReplicationDataHandler:
|
Returns:
|
||||||
|
The request object received by master HS.
|
||||||
|
"""
|
||||||
|
|
||||||
|
# We should have an outbound connection attempt.
|
||||||
|
clients = self.reactor.tcpClients
|
||||||
|
self.assertEqual(len(clients), 1)
|
||||||
|
(host, port, client_factory, _timeout, _bindAddress) = clients.pop(0)
|
||||||
|
self.assertEqual(host, "1.2.3.4")
|
||||||
|
self.assertEqual(port, 8765)
|
||||||
|
|
||||||
|
# Set up client side protocol
|
||||||
|
client_protocol = client_factory.buildProtocol(None)
|
||||||
|
|
||||||
|
request_factory = OneShotRequestFactory()
|
||||||
|
|
||||||
|
# Set up the server side protocol
|
||||||
|
channel = _PushHTTPChannel(self.reactor)
|
||||||
|
channel.requestFactory = request_factory
|
||||||
|
channel.site = self.site
|
||||||
|
|
||||||
|
# Connect client to server and vice versa.
|
||||||
|
client_to_server_transport = FakeTransport(
|
||||||
|
channel, self.reactor, client_protocol
|
||||||
|
)
|
||||||
|
client_protocol.makeConnection(client_to_server_transport)
|
||||||
|
|
||||||
|
server_to_client_transport = FakeTransport(
|
||||||
|
client_protocol, self.reactor, channel
|
||||||
|
)
|
||||||
|
channel.makeConnection(server_to_client_transport)
|
||||||
|
|
||||||
|
# The request will now be processed by `self.site` and the response
|
||||||
|
# streamed back.
|
||||||
|
self.reactor.advance(0)
|
||||||
|
|
||||||
|
# We tear down the connection so it doesn't get reused without our
|
||||||
|
# knowledge.
|
||||||
|
server_to_client_transport.loseConnection()
|
||||||
|
client_to_server_transport.loseConnection()
|
||||||
|
|
||||||
|
return request_factory.request
|
||||||
|
|
||||||
|
def assert_request_is_get_repl_stream_updates(
|
||||||
|
self, request: SynapseRequest, stream_name: str
|
||||||
|
):
|
||||||
|
"""Asserts that the given request is a HTTP replication request for
|
||||||
|
fetching updates for given stream.
|
||||||
|
"""
|
||||||
|
|
||||||
|
self.assertRegex(
|
||||||
|
request.path,
|
||||||
|
br"^/_synapse/replication/get_repl_stream_updates/%s/[^/]+$"
|
||||||
|
% (stream_name.encode("ascii"),),
|
||||||
|
)
|
||||||
|
|
||||||
|
self.assertEqual(request.method, b"GET")
|
||||||
|
|
||||||
|
|
||||||
|
class TestReplicationDataHandler(ReplicationDataHandler):
|
||||||
"""Drop-in for ReplicationDataHandler which just collects RDATA rows"""
|
"""Drop-in for ReplicationDataHandler which just collects RDATA rows"""
|
||||||
|
|
||||||
def __init__(self):
|
def __init__(self, hs):
|
||||||
|
super().__init__(hs)
|
||||||
self.streams = set()
|
self.streams = set()
|
||||||
self._received_rdata_rows = []
|
self._received_rdata_rows = []
|
||||||
|
|
||||||
@ -90,8 +187,118 @@ class TestReplicationDataHandler:
|
|||||||
return positions
|
return positions
|
||||||
|
|
||||||
async def on_rdata(self, stream_name, token, rows):
|
async def on_rdata(self, stream_name, token, rows):
|
||||||
|
await super().on_rdata(stream_name, token, rows)
|
||||||
for r in rows:
|
for r in rows:
|
||||||
self._received_rdata_rows.append((stream_name, token, r))
|
self._received_rdata_rows.append((stream_name, token, r))
|
||||||
|
|
||||||
async def on_position(self, stream_name, token):
|
|
||||||
|
@attr.s()
|
||||||
|
class OneShotRequestFactory:
|
||||||
|
"""A simple request factory that generates a single `SynapseRequest` and
|
||||||
|
stores it for future use. Can only be used once.
|
||||||
|
"""
|
||||||
|
|
||||||
|
request = attr.ib(default=None)
|
||||||
|
|
||||||
|
def __call__(self, *args, **kwargs):
|
||||||
|
assert self.request is None
|
||||||
|
|
||||||
|
self.request = SynapseRequest(*args, **kwargs)
|
||||||
|
return self.request
|
||||||
|
|
||||||
|
|
||||||
|
class _PushHTTPChannel(HTTPChannel):
|
||||||
|
"""A HTTPChannel that wraps pull producers to push producers.
|
||||||
|
|
||||||
|
This is a hack to get around the fact that HTTPChannel transparently wraps a
|
||||||
|
pull producer (which is what Synapse uses to reply to requests) with
|
||||||
|
`_PullToPush` to convert it to a push producer. Unfortunately `_PullToPush`
|
||||||
|
uses the standard reactor rather than letting us use our test reactor, which
|
||||||
|
makes it very hard to test.
|
||||||
|
"""
|
||||||
|
|
||||||
|
def __init__(self, reactor: IReactorTime):
|
||||||
|
super().__init__()
|
||||||
|
self.reactor = reactor
|
||||||
|
|
||||||
|
self._pull_to_push_producer = None
|
||||||
|
|
||||||
|
def registerProducer(self, producer, streaming):
|
||||||
|
# Convert pull producers to push producer.
|
||||||
|
if not streaming:
|
||||||
|
self._pull_to_push_producer = _PullToPushProducer(
|
||||||
|
self.reactor, producer, self
|
||||||
|
)
|
||||||
|
producer = self._pull_to_push_producer
|
||||||
|
|
||||||
|
super().registerProducer(producer, True)
|
||||||
|
|
||||||
|
def unregisterProducer(self):
|
||||||
|
if self._pull_to_push_producer:
|
||||||
|
# We need to manually stop the _PullToPushProducer.
|
||||||
|
self._pull_to_push_producer.stop()
|
||||||
|
|
||||||
|
|
||||||
|
class _PullToPushProducer:
|
||||||
|
"""A push producer that wraps a pull producer.
|
||||||
|
"""
|
||||||
|
|
||||||
|
def __init__(
|
||||||
|
self, reactor: IReactorTime, producer: IPullProducer, consumer: IConsumer
|
||||||
|
):
|
||||||
|
self._clock = Clock(reactor)
|
||||||
|
self._producer = producer
|
||||||
|
self._consumer = consumer
|
||||||
|
|
||||||
|
# While running we use a looping call with a zero delay to call
|
||||||
|
# resumeProducing on given producer.
|
||||||
|
self._looping_call = None # type: Optional[LoopingCall]
|
||||||
|
|
||||||
|
# We start writing next reactor tick.
|
||||||
|
self._start_loop()
|
||||||
|
|
||||||
|
def _start_loop(self):
|
||||||
|
"""Start the looping call to
|
||||||
|
"""
|
||||||
|
|
||||||
|
if not self._looping_call:
|
||||||
|
# Start a looping call which runs every tick.
|
||||||
|
self._looping_call = self._clock.looping_call(self._run_once, 0)
|
||||||
|
|
||||||
|
def stop(self):
|
||||||
|
"""Stops calling resumeProducing.
|
||||||
|
"""
|
||||||
|
if self._looping_call:
|
||||||
|
self._looping_call.stop()
|
||||||
|
self._looping_call = None
|
||||||
|
|
||||||
|
def pauseProducing(self):
|
||||||
|
"""Implements IPushProducer
|
||||||
|
"""
|
||||||
|
self.stop()
|
||||||
|
|
||||||
|
def resumeProducing(self):
|
||||||
|
"""Implements IPushProducer
|
||||||
|
"""
|
||||||
|
self._start_loop()
|
||||||
|
|
||||||
|
def stopProducing(self):
|
||||||
|
"""Implements IPushProducer
|
||||||
|
"""
|
||||||
|
self.stop()
|
||||||
|
self._producer.stopProducing()
|
||||||
|
|
||||||
|
def _run_once(self):
|
||||||
|
"""Calls resumeProducing on producer once.
|
||||||
|
"""
|
||||||
|
|
||||||
|
try:
|
||||||
|
self._producer.resumeProducing()
|
||||||
|
except Exception:
|
||||||
|
logger.exception("Failed to call resumeProducing")
|
||||||
|
try:
|
||||||
|
self._consumer.unregisterProducer()
|
||||||
|
except Exception:
|
||||||
pass
|
pass
|
||||||
|
|
||||||
|
self.stopProducing()
|
||||||
|
80
tests/replication/tcp/streams/test_typing.py
Normal file
80
tests/replication/tcp/streams/test_typing.py
Normal file
@ -0,0 +1,80 @@
|
|||||||
|
# -*- coding: utf-8 -*-
|
||||||
|
# Copyright 2020 The Matrix.org Foundation C.I.C.
|
||||||
|
#
|
||||||
|
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
# you may not use this file except in compliance with the License.
|
||||||
|
# You may obtain a copy of the License at
|
||||||
|
#
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
#
|
||||||
|
# Unless required by applicable law or agreed to in writing, software
|
||||||
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
# See the License for the specific language governing permissions and
|
||||||
|
# limitations under the License.
|
||||||
|
from synapse.handlers.typing import RoomMember
|
||||||
|
from synapse.replication.http import streams
|
||||||
|
from synapse.replication.tcp.streams import TypingStream
|
||||||
|
|
||||||
|
from tests.replication.tcp.streams._base import BaseStreamTestCase
|
||||||
|
|
||||||
|
USER_ID = "@feeling:blue"
|
||||||
|
|
||||||
|
|
||||||
|
class TypingStreamTestCase(BaseStreamTestCase):
|
||||||
|
servlets = [
|
||||||
|
streams.register_servlets,
|
||||||
|
]
|
||||||
|
|
||||||
|
def test_typing(self):
|
||||||
|
typing = self.hs.get_typing_handler()
|
||||||
|
|
||||||
|
room_id = "!bar:blue"
|
||||||
|
|
||||||
|
self.reconnect()
|
||||||
|
|
||||||
|
# make the client subscribe to the receipts stream
|
||||||
|
self.test_handler.streams.add("typing")
|
||||||
|
|
||||||
|
typing._push_update(member=RoomMember(room_id, USER_ID), typing=True)
|
||||||
|
|
||||||
|
self.reactor.advance(0)
|
||||||
|
|
||||||
|
# We should now see an attempt to connect to the master
|
||||||
|
request = self.handle_http_replication_attempt()
|
||||||
|
self.assert_request_is_get_repl_stream_updates(request, "typing")
|
||||||
|
|
||||||
|
self.test_handler.on_rdata.assert_called_once()
|
||||||
|
stream_name, token, rdata_rows = self.test_handler.on_rdata.call_args[0]
|
||||||
|
self.assertEqual(stream_name, "typing")
|
||||||
|
self.assertEqual(1, len(rdata_rows))
|
||||||
|
row = rdata_rows[0] # type: TypingStream.TypingStreamRow
|
||||||
|
self.assertEqual(room_id, row.room_id)
|
||||||
|
self.assertEqual([USER_ID], row.user_ids)
|
||||||
|
|
||||||
|
# Now let's disconnect and insert some data.
|
||||||
|
self.disconnect()
|
||||||
|
|
||||||
|
self.test_handler.on_rdata.reset_mock()
|
||||||
|
|
||||||
|
typing._push_update(member=RoomMember(room_id, USER_ID), typing=False)
|
||||||
|
|
||||||
|
self.test_handler.on_rdata.assert_not_called()
|
||||||
|
|
||||||
|
self.reconnect()
|
||||||
|
self.pump(0.1)
|
||||||
|
|
||||||
|
# We should now see an attempt to connect to the master
|
||||||
|
request = self.handle_http_replication_attempt()
|
||||||
|
self.assert_request_is_get_repl_stream_updates(request, "typing")
|
||||||
|
|
||||||
|
# The from token should be the token from the last RDATA we got.
|
||||||
|
self.assertEqual(int(request.args[b"from_token"][0]), token)
|
||||||
|
|
||||||
|
self.test_handler.on_rdata.assert_called_once()
|
||||||
|
stream_name, token, rdata_rows = self.test_handler.on_rdata.call_args[0]
|
||||||
|
self.assertEqual(stream_name, "typing")
|
||||||
|
self.assertEqual(1, len(rdata_rows))
|
||||||
|
row = rdata_rows[0] # type: TypingStream.TypingStreamRow
|
||||||
|
self.assertEqual(room_id, row.room_id)
|
||||||
|
self.assertEqual([], row.user_ids)
|
Loading…
Reference in New Issue
Block a user