Merge "Implements notification listener and dispatcher"
This commit is contained in:
commit
15bfa35e0c
@ -15,6 +15,7 @@ Contents
|
||||
server
|
||||
rpcclient
|
||||
notifier
|
||||
notification_listener
|
||||
serializer
|
||||
exceptions
|
||||
opts
|
||||
|
14
doc/source/notification_listener.rst
Normal file
14
doc/source/notification_listener.rst
Normal file
@ -0,0 +1,14 @@
|
||||
---------------------
|
||||
Notification Listener
|
||||
---------------------
|
||||
|
||||
.. automodule:: oslo.messaging.notify.listener
|
||||
|
||||
.. currentmodule:: oslo.messaging
|
||||
|
||||
.. autofunction:: get_notification_listener
|
||||
|
||||
.. autoclass:: MessageHandlingServer
|
||||
:members:
|
||||
|
||||
.. autofunction:: get_local_context
|
@ -404,6 +404,16 @@ class AMQPDriverBase(base.BaseDriver):
|
||||
|
||||
return listener
|
||||
|
||||
def listen_for_notifications(self, targets_and_priorities):
|
||||
conn = self._get_connection(pooled=False)
|
||||
|
||||
listener = AMQPListener(self, conn)
|
||||
for target, priority in targets_and_priorities:
|
||||
conn.declare_topic_consumer('%s.%s' % (target.topic, priority),
|
||||
callback=listener,
|
||||
exchange_name=target.exchange)
|
||||
return listener
|
||||
|
||||
def cleanup(self):
|
||||
if self._connection_pool:
|
||||
self._connection_pool.empty()
|
||||
|
@ -73,6 +73,12 @@ class BaseDriver(object):
|
||||
def listen(self, target):
|
||||
"""Construct a Listener for the given target."""
|
||||
|
||||
@abc.abstractmethod
|
||||
def listen_for_notifications(self, targets_and_priorities):
|
||||
"""Construct a notification Listener for the given list of
|
||||
tuple of (target, priority).
|
||||
"""
|
||||
|
||||
@abc.abstractmethod
|
||||
def cleanup(self):
|
||||
"""Release all resources."""
|
||||
|
@ -161,5 +161,15 @@ class FakeDriver(base.BaseDriver):
|
||||
messaging.Target(topic=target.topic)])
|
||||
return listener
|
||||
|
||||
def listen_for_notifications(self, targets_and_priorities):
|
||||
# TODO(sileht): Handle the target.exchange
|
||||
exchange = self._get_exchange(self._default_exchange)
|
||||
|
||||
targets = [messaging.Target(topic='%s.%s' % (target.topic, priority))
|
||||
for target, priority in targets_and_priorities]
|
||||
listener = FakeListener(self, exchange, targets)
|
||||
|
||||
return listener
|
||||
|
||||
def cleanup(self):
|
||||
pass
|
||||
|
@ -959,5 +959,19 @@ class ZmqDriver(base.BaseDriver):
|
||||
|
||||
return listener
|
||||
|
||||
def listen_for_notifications(self, targets_and_priorities):
|
||||
conn = create_connection(self.conf)
|
||||
|
||||
listener = ZmqListener(self, None)
|
||||
for target, priority in targets_and_priorities:
|
||||
# NOTE(ewindisch): dot-priority in rpc notifier does not
|
||||
# work with our assumptions.
|
||||
# NOTE(sileht): create_consumer doesn't support target.exchange
|
||||
conn.create_consumer('%s-%s' % (target.topic, priority),
|
||||
listener)
|
||||
conn.consume_in_thread()
|
||||
|
||||
return listener
|
||||
|
||||
def cleanup(self):
|
||||
cleanup()
|
||||
|
@ -14,7 +14,9 @@
|
||||
# under the License.
|
||||
|
||||
__all__ = ['Notifier',
|
||||
'LoggingNotificationHandler']
|
||||
'LoggingNotificationHandler',
|
||||
'get_notification_listener']
|
||||
|
||||
from .notifier import *
|
||||
from .listener import *
|
||||
from .logger import *
|
||||
|
83
oslo/messaging/notify/dispatcher.py
Normal file
83
oslo/messaging/notify/dispatcher.py
Normal file
@ -0,0 +1,83 @@
|
||||
# Copyright 2011 OpenStack Foundation.
|
||||
# All Rights Reserved.
|
||||
# Copyright 2013 eNovance
|
||||
#
|
||||
# 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.
|
||||
|
||||
import itertools
|
||||
import logging
|
||||
|
||||
from oslo.messaging import localcontext
|
||||
from oslo.messaging import serializer as msg_serializer
|
||||
|
||||
|
||||
LOG = logging.getLogger(__name__)
|
||||
|
||||
PRIORITIES = ['audit', 'debug', 'info', 'warn', 'error', 'critical', 'sample']
|
||||
|
||||
|
||||
class NotificationDispatcher(object):
|
||||
"""A message dispatcher which understands Notification messages.
|
||||
|
||||
A MessageHandlingServer is constructed by passing a callable dispatcher
|
||||
which is invoked with context and message dictionaries each time a message
|
||||
is received.
|
||||
|
||||
NotifcationDispatcher is one such dispatcher which pass a raw notification
|
||||
message to the endpoints
|
||||
"""
|
||||
|
||||
def __init__(self, targets, endpoints, serializer):
|
||||
self.targets = targets
|
||||
self.endpoints = endpoints
|
||||
self.serializer = serializer or msg_serializer.NoOpSerializer()
|
||||
|
||||
self._callbacks_by_priority = {}
|
||||
for endpoint, prio in itertools.product(endpoints, PRIORITIES):
|
||||
if hasattr(endpoint, prio):
|
||||
method = getattr(endpoint, prio)
|
||||
self._callbacks_by_priority.setdefault(prio, []).append(method)
|
||||
|
||||
priorities = self._callbacks_by_priority.keys()
|
||||
self._targets_priorities = set(itertools.product(self.targets,
|
||||
priorities))
|
||||
|
||||
def _listen(self, transport):
|
||||
return transport._listen_for_notifications(self._targets_priorities)
|
||||
|
||||
def __call__(self, ctxt, message):
|
||||
"""Dispatch an RPC message to the appropriate endpoint method.
|
||||
|
||||
:param ctxt: the request context
|
||||
:type ctxt: dict
|
||||
:param message: the message payload
|
||||
:type message: dict
|
||||
"""
|
||||
ctxt = self.serializer.deserialize_context(ctxt)
|
||||
|
||||
publisher_id = message.get('publisher_id')
|
||||
event_type = message.get('event_type')
|
||||
priority = message.get('priority', '').lower()
|
||||
if priority not in PRIORITIES:
|
||||
LOG.warning('Unknown priority "%s"' % priority)
|
||||
return
|
||||
|
||||
payload = self.serializer.deserialize_entity(ctxt,
|
||||
message.get('payload'))
|
||||
|
||||
for callback in self._callbacks_by_priority.get(priority, []):
|
||||
localcontext.set_local_context(ctxt)
|
||||
try:
|
||||
callback(ctxt, publisher_id, event_type, payload)
|
||||
finally:
|
||||
localcontext.clear_local_context()
|
105
oslo/messaging/notify/listener.py
Normal file
105
oslo/messaging/notify/listener.py
Normal file
@ -0,0 +1,105 @@
|
||||
# Copyright 2011 OpenStack Foundation.
|
||||
# All Rights Reserved.
|
||||
# Copyright 2013 eNovance
|
||||
#
|
||||
# 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.
|
||||
"""
|
||||
A notification listener exposes a number of endpoints, each of which
|
||||
contain a set of methods. Each method corresponds to a notification priority.
|
||||
|
||||
To create a notification listener, you supply a transport, list of targets and
|
||||
a list of endpoints.
|
||||
|
||||
A transport can be obtained simply by calling the get_transport() method::
|
||||
|
||||
transport = messaging.get_transport(conf)
|
||||
|
||||
which will load the appropriate transport driver according to the user's
|
||||
messaging configuration configuration. See get_transport() for more details.
|
||||
|
||||
The target supplied when creating a notification listener expresses the topic
|
||||
and - optionally - the exchange to listen on. See Target for more details
|
||||
on these attributes.
|
||||
|
||||
Notification listener have start(), stop() and wait() messages to begin
|
||||
handling requests, stop handling requests and wait for all in-process
|
||||
requests to complete.
|
||||
|
||||
Each notification listener is associated with an executor which integrates the
|
||||
listener with a specific I/O handling framework. Currently, there are blocking
|
||||
and eventlet executors available.
|
||||
|
||||
A simple example of a notification listener with multiple endpoints might be::
|
||||
|
||||
from oslo.config import cfg
|
||||
from oslo import messaging
|
||||
|
||||
class NotificationEndpoint(object):
|
||||
def warn(self, ctxt, publisher_id, event_type, payload):
|
||||
do_something(payload)
|
||||
|
||||
class ErrorEndpoint(object):
|
||||
def error(self, ctxt, publisher_id, event_type, payload):
|
||||
do_something(payload)
|
||||
|
||||
transport = messaging.get_transport(cfg.CONF)
|
||||
targets = [
|
||||
messaging.Target(topic='notifications')
|
||||
messaging.Target(topic='notifications_bis')
|
||||
]
|
||||
endpoints = [
|
||||
NotificationEndpoint(),
|
||||
ErrorEndpoint(),
|
||||
]
|
||||
server = messaging.get_notification_listener(transport, targets, endpoints)
|
||||
server.start()
|
||||
server.wait()
|
||||
|
||||
A notifier sends a notification on a topic with a priority, the notification
|
||||
listener will receive this notification if the topic of this one have been set
|
||||
in one of the targets and if an endpoint implements the method named like the
|
||||
priority
|
||||
|
||||
Parameters to endpoint methods are the request context supplied by the client,
|
||||
the publisher_id of the notification message, the event_type, the payload.
|
||||
|
||||
By supplying a serializer object, a listener can deserialize a request context
|
||||
and arguments from - and serialize return values to - primitive types.
|
||||
"""
|
||||
|
||||
from oslo.messaging.notify import dispatcher as notify_dispatcher
|
||||
from oslo.messaging import server as msg_server
|
||||
|
||||
|
||||
def get_notification_listener(transport, targets, endpoints,
|
||||
executor='blocking', serializer=None):
|
||||
"""Construct a notification listener
|
||||
|
||||
The executor parameter controls how incoming messages will be received and
|
||||
dispatched. By default, the most simple executor is used - the blocking
|
||||
executor.
|
||||
|
||||
:param transport: the messaging transport
|
||||
:type transport: Transport
|
||||
:param targets: the exchanges and topics to listen on
|
||||
:type targets: list of Target
|
||||
:param endpoints: a list of endpoint objects
|
||||
:type endpoints: list
|
||||
:param executor: name of a message executor - e.g. 'eventlet', 'blocking'
|
||||
:type executor: str
|
||||
:param serializer: an optional entity serializer
|
||||
:type serializer: Serializer
|
||||
"""
|
||||
dispatcher = notify_dispatcher.NotificationDispatcher(targets, endpoints,
|
||||
serializer)
|
||||
return msg_server.MessageHandlingServer(transport, dispatcher, executor)
|
@ -99,6 +99,14 @@ class Transport(object):
|
||||
target)
|
||||
return self._driver.listen(target)
|
||||
|
||||
def _listen_for_notifications(self, targets_and_priorities):
|
||||
for target, priority in targets_and_priorities:
|
||||
if not target.topic:
|
||||
raise exceptions.InvalidTarget('A target must have '
|
||||
'topic specified',
|
||||
target)
|
||||
return self._driver.listen_for_notifications(targets_and_priorities)
|
||||
|
||||
def cleanup(self):
|
||||
"""Release all resources associated with this transport."""
|
||||
self._driver.cleanup()
|
||||
|
98
tests/test_notify_dispatcher.py
Normal file
98
tests/test_notify_dispatcher.py
Normal file
@ -0,0 +1,98 @@
|
||||
|
||||
# Copyright 2013 eNovance
|
||||
#
|
||||
# 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.
|
||||
|
||||
import itertools
|
||||
|
||||
import mock
|
||||
import testscenarios
|
||||
|
||||
from oslo import messaging
|
||||
from oslo.messaging.notify import dispatcher as notify_dispatcher
|
||||
from oslo.messaging.openstack.common import timeutils
|
||||
from tests import utils as test_utils
|
||||
|
||||
load_tests = testscenarios.load_tests_apply_scenarios
|
||||
|
||||
|
||||
notification_msg = dict(
|
||||
publisher_id="publisher_id",
|
||||
event_type="compute.start",
|
||||
payload={"info": "fuu"},
|
||||
message_id="uuid",
|
||||
timestamp=str(timeutils.utcnow())
|
||||
)
|
||||
|
||||
|
||||
class TestDispatcher(test_utils.BaseTestCase):
|
||||
|
||||
scenarios = [
|
||||
('no_endpoints',
|
||||
dict(endpoints=[],
|
||||
endpoints_expect_calls=[],
|
||||
priority='info')),
|
||||
('one_endpoints',
|
||||
dict(endpoints=[['warn']],
|
||||
endpoints_expect_calls=['warn'],
|
||||
priority='warn')),
|
||||
('two_endpoints_only_one_match',
|
||||
dict(endpoints=[['warn'], ['info']],
|
||||
endpoints_expect_calls=[None, 'info'],
|
||||
priority='info')),
|
||||
('two_endpoints_both_match',
|
||||
dict(endpoints=[['debug', 'info'], ['info', 'debug']],
|
||||
endpoints_expect_calls=['debug', 'debug'],
|
||||
priority='debug')),
|
||||
]
|
||||
|
||||
def test_dispatcher(self):
|
||||
endpoints = [mock.Mock(spec=endpoint_methods)
|
||||
for endpoint_methods in self.endpoints]
|
||||
msg = notification_msg.copy()
|
||||
msg['priority'] = self.priority
|
||||
|
||||
targets = [messaging.Target(topic='notifications')]
|
||||
dispatcher = notify_dispatcher.NotificationDispatcher(targets,
|
||||
endpoints,
|
||||
None)
|
||||
|
||||
# check it listen on wanted topics
|
||||
self.assertEqual(sorted(dispatcher._targets_priorities),
|
||||
sorted(set((targets[0], prio)
|
||||
for prio in itertools.chain.from_iterable(
|
||||
self.endpoints))))
|
||||
|
||||
dispatcher({}, msg)
|
||||
|
||||
# check endpoint callbacks are called or not
|
||||
for i, endpoint_methods in enumerate(self.endpoints):
|
||||
for m in endpoint_methods:
|
||||
if m == self.endpoints_expect_calls[i]:
|
||||
method = getattr(endpoints[i], m)
|
||||
expected = [mock.call({}, msg['publisher_id'],
|
||||
msg['event_type'],
|
||||
msg['payload'])]
|
||||
self.assertEqual(method.call_args_list, expected)
|
||||
else:
|
||||
self.assertEqual(endpoints[i].call_count, 0)
|
||||
|
||||
@mock.patch('oslo.messaging.notify.dispatcher.LOG')
|
||||
def test_dispatcher_unknown_prio(self, mylog):
|
||||
msg = notification_msg.copy()
|
||||
msg['priority'] = 'what???'
|
||||
dispatcher = notify_dispatcher.NotificationDispatcher([mock.Mock()],
|
||||
[mock.Mock()],
|
||||
None)
|
||||
dispatcher({}, msg)
|
||||
mylog.warning.assert_called_once_with('Unknown priority "what???"')
|
173
tests/test_notify_listener.py
Normal file
173
tests/test_notify_listener.py
Normal file
@ -0,0 +1,173 @@
|
||||
|
||||
# Copyright 2013 eNovance
|
||||
#
|
||||
# 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.
|
||||
|
||||
import threading
|
||||
|
||||
import mock
|
||||
from oslo.config import cfg
|
||||
import testscenarios
|
||||
|
||||
from oslo import messaging
|
||||
from oslo.messaging.notify import dispatcher
|
||||
from tests import utils as test_utils
|
||||
|
||||
load_tests = testscenarios.load_tests_apply_scenarios
|
||||
|
||||
|
||||
class ListenerSetupMixin(object):
|
||||
|
||||
class Listener(object):
|
||||
def __init__(self, transport, topics, endpoints, expect_messages):
|
||||
targets = [messaging.Target(topic=topic)
|
||||
for topic in topics]
|
||||
self._expect_messages = expect_messages
|
||||
self._received_msgs = 0
|
||||
self._listener = messaging.get_notification_listener(
|
||||
transport, targets, endpoints + [self])
|
||||
|
||||
def info(self, ctxt, publisher_id, event_type, payload):
|
||||
self._received_msgs += 1
|
||||
if self._expect_messages == self._received_msgs:
|
||||
# Check start() does nothing with a running listener
|
||||
self._listener.start()
|
||||
self._listener.stop()
|
||||
self._listener.wait()
|
||||
|
||||
def start(self):
|
||||
self._listener.start()
|
||||
|
||||
def _setup_listener(self, transport, endpoints, expect_messages,
|
||||
topics=None):
|
||||
listener = self.Listener(transport,
|
||||
topics=topics or ['testtopic'],
|
||||
expect_messages=expect_messages,
|
||||
endpoints=endpoints)
|
||||
|
||||
thread = threading.Thread(target=listener.start)
|
||||
thread.daemon = True
|
||||
thread.start()
|
||||
return thread
|
||||
|
||||
def _stop_listener(self, thread):
|
||||
thread.join(timeout=5)
|
||||
|
||||
def _setup_notifier(self, transport, topic='testtopic',
|
||||
publisher_id='testpublisher'):
|
||||
return messaging.Notifier(transport, topic=topic,
|
||||
driver='messaging',
|
||||
publisher_id=publisher_id)
|
||||
|
||||
|
||||
class TestNotifyListener(test_utils.BaseTestCase, ListenerSetupMixin):
|
||||
|
||||
def __init__(self, *args):
|
||||
super(TestNotifyListener, self).__init__(*args)
|
||||
ListenerSetupMixin.__init__(self)
|
||||
|
||||
def setUp(self):
|
||||
super(TestNotifyListener, self).setUp(conf=cfg.ConfigOpts())
|
||||
|
||||
def test_constructor(self):
|
||||
transport = messaging.get_transport(self.conf, url='fake:')
|
||||
target = messaging.Target(topic='foo')
|
||||
endpoints = [object()]
|
||||
|
||||
listener = messaging.get_notification_listener(transport, [target],
|
||||
endpoints)
|
||||
|
||||
self.assertIs(listener.conf, self.conf)
|
||||
self.assertIs(listener.transport, transport)
|
||||
self.assertIsInstance(listener.dispatcher,
|
||||
dispatcher.NotificationDispatcher)
|
||||
self.assertIs(listener.dispatcher.endpoints, endpoints)
|
||||
self.assertIs(listener.executor, 'blocking')
|
||||
|
||||
def test_no_target_topic(self):
|
||||
transport = messaging.get_transport(self.conf, url='fake:')
|
||||
|
||||
listener = messaging.get_notification_listener(transport,
|
||||
[messaging.Target()],
|
||||
[mock.Mock()])
|
||||
try:
|
||||
listener.start()
|
||||
except Exception as ex:
|
||||
self.assertIsInstance(ex, messaging.InvalidTarget, ex)
|
||||
else:
|
||||
self.assertTrue(False)
|
||||
|
||||
def test_unknown_executor(self):
|
||||
transport = messaging.get_transport(self.conf, url='fake:')
|
||||
|
||||
try:
|
||||
messaging.get_notification_listener(transport, [], [],
|
||||
executor='foo')
|
||||
except Exception as ex:
|
||||
self.assertIsInstance(ex, messaging.ExecutorLoadFailure)
|
||||
self.assertEqual(ex.executor, 'foo')
|
||||
else:
|
||||
self.assertTrue(False)
|
||||
|
||||
def test_one_topic(self):
|
||||
transport = messaging.get_transport(self.conf, url='fake:')
|
||||
|
||||
endpoint = mock.Mock()
|
||||
endpoint.info = mock.Mock()
|
||||
listener_thread = self._setup_listener(transport, [endpoint], 1)
|
||||
|
||||
notifier = self._setup_notifier(transport)
|
||||
notifier.info({}, 'an_event.start', 'test message')
|
||||
|
||||
self._stop_listener(listener_thread)
|
||||
|
||||
endpoint.info.assert_called_once_with(
|
||||
{}, 'testpublisher', 'an_event.start', 'test message')
|
||||
|
||||
def test_two_topics(self):
|
||||
transport = messaging.get_transport(self.conf, url='fake:')
|
||||
|
||||
endpoint = mock.Mock()
|
||||
endpoint.info = mock.Mock()
|
||||
topics = ["topic1", "topic2"]
|
||||
listener_thread = self._setup_listener(transport, [endpoint], 2,
|
||||
topics=topics)
|
||||
notifier = self._setup_notifier(transport, topic='topic1')
|
||||
notifier.info({}, 'an_event.start1', 'test')
|
||||
notifier = self._setup_notifier(transport, topic='topic2')
|
||||
notifier.info({}, 'an_event.start2', 'test')
|
||||
|
||||
self._stop_listener(listener_thread)
|
||||
|
||||
expected = [mock.call({}, 'testpublisher', 'an_event.start1', 'test'),
|
||||
mock.call({}, 'testpublisher', 'an_event.start2', 'test')]
|
||||
self.assertEqual(sorted(endpoint.info.call_args_list), expected)
|
||||
|
||||
def test_two_endpoints(self):
|
||||
transport = messaging.get_transport(self.conf, url='fake:')
|
||||
|
||||
endpoint1 = mock.Mock()
|
||||
endpoint1.info = mock.Mock()
|
||||
endpoint2 = mock.Mock()
|
||||
endpoint2.info = mock.Mock()
|
||||
listener_thread = self._setup_listener(transport,
|
||||
[endpoint1, endpoint2], 1)
|
||||
notifier = self._setup_notifier(transport)
|
||||
notifier.info({}, 'an_event.start', 'test')
|
||||
|
||||
self._stop_listener(listener_thread)
|
||||
|
||||
endpoint1.info.assert_called_once_with(
|
||||
{}, 'testpublisher', 'an_event.start', 'test')
|
||||
endpoint2.info.assert_called_once_with(
|
||||
{}, 'testpublisher', 'an_event.start', 'test')
|
@ -108,6 +108,11 @@ class TestRabbitTransportURL(test_utils.BaseTestCase):
|
||||
self._driver.listen(self._target)
|
||||
self.assertEqual(self._server_params[0], self.expected)
|
||||
|
||||
def test_transport_url_listen_for_notification(self):
|
||||
self._driver.listen_for_notifications(
|
||||
[(messaging.Target(topic='topic'), 'info')])
|
||||
self.assertEqual(self._server_params[0], self.expected)
|
||||
|
||||
def test_transport_url_send(self):
|
||||
self._driver.send(self._target, {}, {})
|
||||
self.assertEqual(self._server_params[0], self.expected)
|
||||
|
Loading…
Reference in New Issue
Block a user